From 283d7c326598e3778a389c7f87f46055a00cc07c Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 2 Apr 2024 11:57:40 -0400 Subject: [PATCH 01/86] add spark 3.3 reader --- .../apache/spark/sql/hudi/SparkAdapter.scala | 33 +++ .../spark/sql/adapter/Spark3_3Adapter.scala | 41 ++- .../parquet/Spark33HoodieParquetReader.scala | 261 ++++++++++++++++++ 3 files changed, 334 insertions(+), 1 deletion(-) create mode 100644 hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReader.scala diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala index 1c6111afe47f..0ed33c0e5ef7 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hudi import org.apache.avro.Schema +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hudi.client.utils.SparkRowSerDe import org.apache.hudi.common.table.HoodieTableMetaClient @@ -34,6 +35,7 @@ import org.apache.spark.sql.catalyst.util.DateFormatter import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.parser.HoodieExtendedParserInterface import org.apache.spark.sql.sources.{BaseRelation, Filter} import org.apache.spark.sql.types.{DataType, Metadata, StructType} @@ -214,4 +216,35 @@ trait SparkAdapter extends Serializable { * Tries to translate a Catalyst Expression into data source Filter */ def translateFilter(predicate: Expression, supportNestedPredicatePushdown: Boolean = false): Option[Filter] + + /** + * 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] + + /** + * Read an individual parquet file + * @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, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[Filter], + sharedConf: Configuration, + extraProps: Map[String, String]): Iterator[InternalRow] } diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala index e3d2cc9cd185..bf79666121a8 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.adapter import org.apache.avro.Schema +import org.apache.hadoop.conf.Configuration import org.apache.hudi.Spark33HoodieFileScanRDD import org.apache.spark.sql._ import org.apache.spark.sql.avro._ @@ -30,10 +31,11 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.METADATA_COL_ATTR_KEY import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark33LegacyHoodieParquetFileFormat} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark33HoodieParquetReader, Spark33LegacyHoodieParquetFileFormat} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hudi.analysis.TableValuedFunctions +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.parser.{HoodieExtendedParserInterface, HoodieSpark3_3ExtendedSqlParser} import org.apache.spark.sql.types.{DataType, Metadata, MetadataBuilder, StructType} import org.apache.spark.sql.vectorized.ColumnarBatchRow @@ -124,4 +126,41 @@ class Spark3_3Adapter extends BaseSpark3Adapter { case OFF_HEAP => "OFF_HEAP" case _ => throw new IllegalArgumentException(s"Invalid StorageLevel: $level") } + + /** + * 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 + */ + override def getPropsForReadingParquet(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): Map[String, String] = { + Spark33HoodieParquetReader.getPropsForReadingParquet(vectorized, sqlConf, options, hadoopConf) + } + + /** + * Read an individual parquet file + * + * @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]] + */ + override def readParquetFile(file: PartitionedFile, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[sources.Filter], + sharedConf: Configuration, + extraProps: Map[String, String]): Iterator[InternalRow] = { + Spark33HoodieParquetReader.readParquetFile(file, requiredSchema, partitionSchema, filters, + new Configuration(sharedConf), extraProps) + } } diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReader.scala new file mode 100644 index 000000000000..ef5a89578c8e --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReader.scala @@ -0,0 +1,261 @@ +/* + * 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.mapred.FileSplit +import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl +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._ +import org.apache.spark.TaskContext +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.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._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types._ + +import java.net.URI + +object Spark33HoodieParquetReader { + + /** + * 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.NESTED_SCHEMA_PRUNING_ENABLED.key, sqlConf.nestedSchemaPruningEnabled) + 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) + // Using string value of this conf to preserve compatibility across spark versions. See [HUDI-5868] + hadoopConf.setBoolean( + "spark.sql.legacy.parquet.nanosAsLong", + sqlConf.getConfString("spark.sql.legacy.parquet.nanosAsLong", "false").toBoolean + ) + + val parquetOptions = new ParquetOptions(options, sqlConf) + Map( + "enableVectorizedReader" -> vectorized.toString, + "datetimeRebaseModeInRead" -> parquetOptions.datetimeRebaseModeInRead, + "int96RebaseModeInRead" -> parquetOptions.int96RebaseModeInRead, + "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 v3.3.4 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, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[Filter], + sharedConf: Configuration, + extraProps: Map[String, String]): Iterator[InternalRow] = { + sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) + sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) + ParquetWriteSupport.setSchema(requiredSchema, sharedConf) + val enableVectorizedReader = extraProps("enableVectorizedReader").toBoolean + val datetimeRebaseModeInRead = extraProps("datetimeRebaseModeInRead") + val int96RebaseModeInRead = extraProps("int96RebaseModeInRead") + val enableParquetFilterPushDown = extraProps("enableParquetFilterPushDown").toBoolean + val pushDownDate = extraProps("pushDownDate").toBoolean + val pushDownTimestamp = extraProps("pushDownTimestamp").toBoolean + val pushDownDecimal = extraProps("pushDownDecimal").toBoolean + val pushDownInFilterThreshold = extraProps("pushDownInFilterThreshold").toInt + val pushDownStringStartWith = extraProps("pushDownStringStartWith").toBoolean + val isCaseSensitive = extraProps("isCaseSensitive").toBoolean + val timestampConversion = extraProps("timestampConversion").toBoolean + val enableOffHeapColumnVector = extraProps("enableOffHeapColumnVector").toBoolean + val capacity = extraProps("capacity").toInt + val returningBatch = extraProps("returningBatch").toBoolean + val enableRecordFilter = extraProps("enableRecordFilter").toBoolean + val timeZoneId = Option(extraProps("timeZoneId")) + assert(file.partitionValues.numFields == partitionSchema.size) + + val filePath = new Path(new URI(file.filePath)) + val split = new FileSplit(filePath, file.start, file.length, Array.empty[String]) + + + lazy val footerFileMetaData = + ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData + val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( + footerFileMetaData.getKeyValueMetaData.get, + datetimeRebaseModeInRead) + // Try to push down filters when filter push-down is enabled. + val pushed = if (enableParquetFilterPushDown) { + val parquetSchema = footerFileMetaData.getSchema + val parquetFilters = new ParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringStartWith, + pushDownInFilterThreshold, + isCaseSensitive, + datetimeRebaseSpec) + 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(_)) + .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.getZoneId(sharedConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key))) + } else { + None + } + + val int96RebaseSpec = DataSourceUtils.int96RebaseSpec( + footerFileMetaData.getKeyValueMetaData.get, + int96RebaseModeInRead) + + val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) + val hadoopAttemptContext = + new TaskAttemptContextImpl(sharedConf, 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()) + if (enableVectorizedReader) { + val vectorizedReader = new VectorizedParquetRecordReader( + convertTz.orNull, + datetimeRebaseSpec.mode.toString, + datetimeRebaseSpec.timeZone, + int96RebaseSpec.mode.toString, + int96RebaseSpec.timeZone, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) + // SPARK-37089: We cannot register a task completion listener to close this iterator here + // because downstream exec nodes have already registered their listeners. Since listeners + // are executed in reverse order of registration, a listener registered here would close the + // iterator while downstream exec nodes are still running. When off-heap column vectors are + // enabled, this can cause a use-after-free bug leading to a segfault. + // + // Instead, we use FileScanRDD's task completion listener to close this iterator. + val iter = new RecordReaderIterator(vectorizedReader) + try { + vectorizedReader.initialize(split, hadoopAttemptContext) + vectorizedReader.initBatch(partitionSchema, file.partitionValues) + if (returningBatch) { + vectorizedReader.enableReturningBatches() + } + + // UnsafeRowParquetRecordReader appends the columns internally to avoid another copy. + iter.asInstanceOf[Iterator[InternalRow]] + } catch { + case e: Throwable => + // SPARK-23457: In case there is an exception in initialization, close the iterator to + // avoid leaking resources. + iter.close() + throw e + } + } else { + // ParquetRecordReader returns InternalRow + val readSupport = new ParquetReadSupport( + convertTz, + enableVectorizedReader = false, + datetimeRebaseSpec, + int96RebaseSpec) + val reader = if (pushed.isDefined && enableRecordFilter) { + val parquetFilter = FilterCompat.get(pushed.get, null) + new ParquetRecordReader[InternalRow](readSupport, parquetFilter) + } else { + new ParquetRecordReader[InternalRow](readSupport) + } + val iter = new RecordReaderIterator[InternalRow](reader) + try { + reader.initialize(split, hadoopAttemptContext) + + val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes + val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + + if (partitionSchema.length == 0) { + // There is no partition columns + iter.map(unsafeProjection) + } else { + val joinedRow = new JoinedRow() + iter.map(d => unsafeProjection(joinedRow(d, file.partitionValues))) + } + } catch { + case e: Throwable => + // SPARK-23457: In case there is an exception in initialization, close the iterator to + // avoid leaking resources. + iter.close() + throw e + } + } + } +} From ef654282c45e6d20a21d51f383166b4a2f450819 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 2 Apr 2024 12:11:22 -0400 Subject: [PATCH 02/86] add spark3.4 --- .../spark/sql/adapter/Spark3_4Adapter.scala | 41 ++- .../parquet/Spark34HoodieParquetReader.scala | 271 ++++++++++++++++++ 2 files changed, 311 insertions(+), 1 deletion(-) create mode 100644 hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReader.scala diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala index 0ae5ef3dbf34..b17efe063710 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.adapter import org.apache.avro.Schema +import org.apache.hadoop.conf.Configuration import org.apache.hudi.Spark34HoodieFileScanRDD import org.apache.spark.sql.avro._ import org.apache.spark.sql.catalyst.InternalRow @@ -29,7 +30,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.METADATA_COL_ATTR_KEY import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark34LegacyHoodieParquetFileFormat} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark34HoodieParquetReader, Spark34LegacyHoodieParquetFileFormat} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hudi.analysis.TableValuedFunctions @@ -37,6 +38,7 @@ import org.apache.spark.sql.parser.{HoodieExtendedParserInterface, HoodieSpark3_ import org.apache.spark.sql.types.{DataType, Metadata, MetadataBuilder, StructType} import org.apache.spark.sql.vectorized.ColumnarBatchRow import org.apache.spark.sql._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel._ @@ -124,4 +126,41 @@ class Spark3_4Adapter extends BaseSpark3Adapter { case OFF_HEAP => "OFF_HEAP" case _ => throw new IllegalArgumentException(s"Invalid StorageLevel: $level") } + + /** + * 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 + */ + override def getPropsForReadingParquet(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): Map[String, String] = { + Spark34HoodieParquetReader.getPropsForReadingParquet(vectorized, sqlConf, options, hadoopConf) + } + + /** + * Read an individual parquet file + * + * @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]] + */ + override def readParquetFile(file: PartitionedFile, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[sources.Filter], + sharedConf: Configuration, + extraProps: Map[String, String]): Iterator[InternalRow] = { + Spark34HoodieParquetReader.readParquetFile(file, requiredSchema, partitionSchema, filters, + new Configuration(sharedConf), extraProps) + } } diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReader.scala new file mode 100644 index 000000000000..4549507f7e08 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReader.scala @@ -0,0 +1,271 @@ +/* + * 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.mapred.FileSplit +import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl +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._ +import org.apache.spark.TaskContext +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.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._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types._ + +object Spark34HoodieParquetReader { + + /** + * 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.NESTED_SCHEMA_PRUNING_ENABLED.key, sqlConf.nestedSchemaPruningEnabled) + 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) + // Using string value of this conf to preserve compatibility across spark versions. See [HUDI-5868] + hadoopConf.setBoolean( + SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, + sqlConf.getConfString( + SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, + SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.defaultValueString).toBoolean + ) + hadoopConf.setBoolean(SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key, sqlConf.parquetInferTimestampNTZEnabled) + + val returningBatch = sqlConf.parquetVectorizedReaderEnabled && + options.getOrElse(FileFormat.OPTION_RETURNING_BATCH, + throw new IllegalArgumentException( + "OPTION_RETURNING_BATCH should always be set for ParquetFileFormat. " + + "To workaround this issue, set spark.sql.parquet.enableVectorizedReader=false.")) + .equals("true") + + val parquetOptions = new ParquetOptions(options, sqlConf) + Map( + "enableVectorizedReader" -> vectorized.toString, + "datetimeRebaseModeInRead" -> parquetOptions.datetimeRebaseModeInRead, + "int96RebaseModeInRead" -> parquetOptions.int96RebaseModeInRead, + "enableParquetFilterPushDown" -> sqlConf.parquetFilterPushDown.toString, + "pushDownDate" -> sqlConf.parquetFilterPushDownDate.toString, + "pushDownTimestamp" -> sqlConf.parquetFilterPushDownTimestamp.toString, + "pushDownDecimal" -> sqlConf.parquetFilterPushDownDecimal.toString, + "pushDownStringPredicate" -> sqlConf.parquetFilterPushDownStringPredicate.toString, + "pushDownInFilterThreshold" -> sqlConf.parquetFilterPushDownInFilterThreshold.toString, + "isCaseSensitive" -> sqlConf.caseSensitiveAnalysis.toString, + "timestampConversion" -> sqlConf.isParquetINT96TimestampConversion.toString, + "enableOffHeapColumnVector" -> sqlConf.offHeapColumnVectorEnabled.toString, + "capacity" -> sqlConf.parquetVectorizedReaderBatchSize.toString, + "returningBatch" -> returningBatch.toString, + "enableRecordFilter" -> sqlConf.parquetRecordFilterEnabled.toString, + "timeZoneId" -> sqlConf.sessionLocalTimeZone + ) + } + + /** + * Read an individual parquet file + * Code from ParquetFileFormat#buildReaderWithPartitionValues from Spark v3.4.2 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, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[Filter], + sharedConf: Configuration, + extraProps: Map[String, String]): Iterator[InternalRow] = { + sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) + sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) + ParquetWriteSupport.setSchema(requiredSchema, sharedConf) + val enableVectorizedReader = extraProps("enableVectorizedReader").toBoolean + val datetimeRebaseModeInRead = extraProps("datetimeRebaseModeInRead") + val int96RebaseModeInRead = extraProps("int96RebaseModeInRead") + val enableParquetFilterPushDown = extraProps("enableParquetFilterPushDown").toBoolean + val pushDownDate = extraProps("pushDownDate").toBoolean + val pushDownTimestamp = extraProps("pushDownTimestamp").toBoolean + val pushDownDecimal = extraProps("pushDownDecimal").toBoolean + val pushDownStringPredicate = extraProps("pushDownStringPredicate").toBoolean + val pushDownInFilterThreshold = extraProps("pushDownInFilterThreshold").toInt + val isCaseSensitive = extraProps("isCaseSensitive").toBoolean + val timestampConversion = extraProps("timestampConversion").toBoolean + val enableOffHeapColumnVector = extraProps("enableOffHeapColumnVector").toBoolean + val capacity = extraProps("capacity").toInt + val returningBatch = extraProps("returningBatch").toBoolean + val enableRecordFilter = extraProps("enableRecordFilter").toBoolean + val timeZoneId = Option(extraProps("timeZoneId")) + + assert(file.partitionValues.numFields == partitionSchema.size) + + val filePath = file.toPath + val split = new FileSplit(filePath, file.start, file.length, Array.empty[String]) + + + lazy val footerFileMetaData = + ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData + val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( + footerFileMetaData.getKeyValueMetaData.get, + datetimeRebaseModeInRead) + // Try to push down filters when filter push-down is enabled. + val pushed = if (enableParquetFilterPushDown) { + val parquetSchema = footerFileMetaData.getSchema + val parquetFilters = new ParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringPredicate, + pushDownInFilterThreshold, + isCaseSensitive, + datetimeRebaseSpec) + 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(_)) + .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.getZoneId(sharedConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key))) + } else { + None + } + + val int96RebaseSpec = DataSourceUtils.int96RebaseSpec( + footerFileMetaData.getKeyValueMetaData.get, + int96RebaseModeInRead) + + val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) + val hadoopAttemptContext = + new TaskAttemptContextImpl(sharedConf, 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()) + if (enableVectorizedReader) { + val vectorizedReader = new VectorizedParquetRecordReader( + convertTz.orNull, + datetimeRebaseSpec.mode.toString, + datetimeRebaseSpec.timeZone, + int96RebaseSpec.mode.toString, + int96RebaseSpec.timeZone, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) + // SPARK-37089: We cannot register a task completion listener to close this iterator here + // because downstream exec nodes have already registered their listeners. Since listeners + // are executed in reverse order of registration, a listener registered here would close the + // iterator while downstream exec nodes are still running. When off-heap column vectors are + // enabled, this can cause a use-after-free bug leading to a segfault. + // + // Instead, we use FileScanRDD's task completion listener to close this iterator. + val iter = new RecordReaderIterator(vectorizedReader) + try { + vectorizedReader.initialize(split, hadoopAttemptContext) + vectorizedReader.initBatch(partitionSchema, file.partitionValues) + if (returningBatch) { + vectorizedReader.enableReturningBatches() + } + + // UnsafeRowParquetRecordReader appends the columns internally to avoid another copy. + iter.asInstanceOf[Iterator[InternalRow]] + } catch { + case e: Throwable => + // SPARK-23457: In case there is an exception in initialization, close the iterator to + // avoid leaking resources. + iter.close() + throw e + } + } else { + // ParquetRecordReader returns InternalRow + val readSupport = new ParquetReadSupport( + convertTz, + enableVectorizedReader = false, + datetimeRebaseSpec, + int96RebaseSpec) + val reader = if (pushed.isDefined && enableRecordFilter) { + val parquetFilter = FilterCompat.get(pushed.get, null) + new ParquetRecordReader[InternalRow](readSupport, parquetFilter) + } else { + new ParquetRecordReader[InternalRow](readSupport) + } + val readerWithRowIndexes = ParquetRowIndexUtil.addRowIndexToRecordReaderIfNeeded(reader, + requiredSchema) + val iter = new RecordReaderIterator[InternalRow](readerWithRowIndexes) + try { + readerWithRowIndexes.initialize(split, hadoopAttemptContext) + + val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes + val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + + if (partitionSchema.length == 0) { + // There is no partition columns + iter.map(unsafeProjection) + } else { + val joinedRow = new JoinedRow() + iter.map(d => unsafeProjection(joinedRow(d, file.partitionValues))) + } + } catch { + case e: Throwable => + // SPARK-23457: In case there is an exception in initialization, close the iterator to + // avoid leaking resources. + iter.close() + throw e + } + } + } + +} From 816814708ab7ef6c6be0332501ee3d0c75216074 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 2 Apr 2024 12:21:41 -0400 Subject: [PATCH 03/86] add spark 3.5 --- .../spark/sql/adapter/Spark3_5Adapter.scala | 41 ++- .../parquet/Spark35HoodieParquetReader.scala | 277 ++++++++++++++++++ 2 files changed, 317 insertions(+), 1 deletion(-) create mode 100644 hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReader.scala diff --git a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_5Adapter.scala b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_5Adapter.scala index d18291a1809d..c6bc5cbc8cb5 100644 --- a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_5Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_5Adapter.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.adapter import org.apache.avro.Schema +import org.apache.hadoop.conf.Configuration import org.apache.hudi.Spark35HoodieFileScanRDD import org.apache.spark.sql._ import org.apache.spark.sql.avro._ @@ -31,9 +32,10 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.METADATA_COL_ATTR_KEY import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark35LegacyHoodieParquetFileFormat} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark35HoodieParquetReader, Spark35LegacyHoodieParquetFileFormat} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.hudi.analysis.TableValuedFunctions +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.parser.{HoodieExtendedParserInterface, HoodieSpark3_5ExtendedSqlParser} import org.apache.spark.sql.types.{DataType, Metadata, MetadataBuilder, StructType} import org.apache.spark.sql.vectorized.ColumnarBatchRow @@ -124,4 +126,41 @@ class Spark3_5Adapter extends BaseSpark3Adapter { case OFF_HEAP => "OFF_HEAP" case _ => throw new IllegalArgumentException(s"Invalid StorageLevel: $level") } + + /** + * 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 + */ + override def getPropsForReadingParquet(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): Map[String, String] = { + Spark35HoodieParquetReader.getPropsForReadingParquet(vectorized, sqlConf, options, hadoopConf) + } + + /** + * Read an individual parquet file + * + * @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]] + */ + override def readParquetFile(file: PartitionedFile, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[sources.Filter], + sharedConf: Configuration, + extraProps: Map[String, String]): Iterator[InternalRow] = { + Spark35HoodieParquetReader.readParquetFile(file, requiredSchema, partitionSchema, filters, + new Configuration(sharedConf), extraProps) + } } diff --git a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReader.scala new file mode 100644 index 000000000000..890a66fed1a6 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReader.scala @@ -0,0 +1,277 @@ +/* + * 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.mapred.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.hadoop.{ParquetInputFormat, ParquetRecordReader} +import org.apache.spark.TaskContext +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.JoinedRow +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.execution.datasources.{DataSourceUtils, FileFormat, PartitionedFile, RecordReaderIterator} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types.StructType + +object Spark35HoodieParquetReader { + + /** + * 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.NESTED_SCHEMA_PRUNING_ENABLED.key, sqlConf.nestedSchemaPruningEnabled) + 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) + // Using string value of this conf to preserve compatibility across spark versions. See [HUDI-5868] + hadoopConf.setBoolean( + SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, + sqlConf.getConfString( + SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, + SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.defaultValueString).toBoolean + ) + hadoopConf.setBoolean(SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key, sqlConf.parquetInferTimestampNTZEnabled) + + val returningBatch = sqlConf.parquetVectorizedReaderEnabled && + options.getOrElse(FileFormat.OPTION_RETURNING_BATCH, + throw new IllegalArgumentException( + "OPTION_RETURNING_BATCH should always be set for ParquetFileFormat. " + + "To workaround this issue, set spark.sql.parquet.enableVectorizedReader=false.")) + .equals("true") + + val parquetOptions = new ParquetOptions(options, sqlConf) + Map( + "enableVectorizedReader" -> vectorized.toString, + "datetimeRebaseModeInRead" -> parquetOptions.datetimeRebaseModeInRead, + "int96RebaseModeInRead" -> parquetOptions.int96RebaseModeInRead, + "enableParquetFilterPushDown" -> sqlConf.parquetFilterPushDown.toString, + "pushDownDate" -> sqlConf.parquetFilterPushDownDate.toString, + "pushDownTimestamp" -> sqlConf.parquetFilterPushDownTimestamp.toString, + "pushDownDecimal" -> sqlConf.parquetFilterPushDownDecimal.toString, + "pushDownStringPredicate" -> sqlConf.parquetFilterPushDownStringPredicate.toString, + "pushDownInFilterThreshold" -> sqlConf.parquetFilterPushDownInFilterThreshold.toString, + "isCaseSensitive" -> sqlConf.caseSensitiveAnalysis.toString, + "timestampConversion" -> sqlConf.isParquetINT96TimestampConversion.toString, + "enableOffHeapColumnVector" -> sqlConf.offHeapColumnVectorEnabled.toString, + "capacity" -> sqlConf.parquetVectorizedReaderBatchSize.toString, + "returningBatch" -> returningBatch.toString, + "enableRecordFilter" -> sqlConf.parquetRecordFilterEnabled.toString, + "timeZoneId" -> sqlConf.sessionLocalTimeZone + ) + } + + /** + * Read an individual parquet file + * Code from ParquetFileFormat#buildReaderWithPartitionValues from Spark v3.5.1 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, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[Filter], + sharedConf: Configuration, + extraProps: Map[String, String]): Iterator[InternalRow] = { + sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) + sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) + ParquetWriteSupport.setSchema(requiredSchema, sharedConf) + val enableVectorizedReader = extraProps("enableVectorizedReader").toBoolean + val datetimeRebaseModeInRead = extraProps("datetimeRebaseModeInRead") + val int96RebaseModeInRead = extraProps("int96RebaseModeInRead") + val enableParquetFilterPushDown = extraProps("enableParquetFilterPushDown").toBoolean + val pushDownDate = extraProps("pushDownDate").toBoolean + val pushDownTimestamp = extraProps("pushDownTimestamp").toBoolean + val pushDownDecimal = extraProps("pushDownDecimal").toBoolean + val pushDownStringPredicate = extraProps("pushDownStringPredicate").toBoolean + val pushDownInFilterThreshold = extraProps("pushDownInFilterThreshold").toInt + val isCaseSensitive = extraProps("isCaseSensitive").toBoolean + val timestampConversion = extraProps("timestampConversion").toBoolean + val enableOffHeapColumnVector = extraProps("enableOffHeapColumnVector").toBoolean + val capacity = extraProps("capacity").toInt + val returningBatch = extraProps("returningBatch").toBoolean + val enableRecordFilter = extraProps("enableRecordFilter").toBoolean + val timeZoneId = Option(extraProps("timeZoneId")) + + assert(file.partitionValues.numFields == partitionSchema.size) + + val filePath = file.toPath + val split = new FileSplit(filePath, file.start, file.length, Array.empty[String]) + + val fileFooter = if (enableVectorizedReader) { + // When there are vectorized reads, we can avoid reading the footer twice by reading + // all row groups in advance and filter row groups according to filters that require + // push down (no need to read the footer metadata again). + ParquetFooterReader.readFooter(sharedConf, file, ParquetFooterReader.WITH_ROW_GROUPS) + } else { + ParquetFooterReader.readFooter(sharedConf, file, ParquetFooterReader.SKIP_ROW_GROUPS) + } + + val footerFileMetaData = fileFooter.getFileMetaData + val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( + footerFileMetaData.getKeyValueMetaData.get, + datetimeRebaseModeInRead) + val int96RebaseSpec = DataSourceUtils.int96RebaseSpec( + footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) + + // Try to push down filters when filter push-down is enabled. + val pushed = if (enableParquetFilterPushDown) { + val parquetSchema = footerFileMetaData.getSchema + val parquetFilters = new ParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringPredicate, + pushDownInFilterThreshold, + isCaseSensitive, + datetimeRebaseSpec) + 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(_)) + .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.getZoneId(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(sharedConf, 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()) + if (enableVectorizedReader) { + val vectorizedReader = new VectorizedParquetRecordReader( + convertTz.orNull, + datetimeRebaseSpec.mode.toString, + datetimeRebaseSpec.timeZone, + int96RebaseSpec.mode.toString, + int96RebaseSpec.timeZone, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) + // SPARK-37089: We cannot register a task completion listener to close this iterator here + // because downstream exec nodes have already registered their listeners. Since listeners + // are executed in reverse order of registration, a listener registered here would close the + // iterator while downstream exec nodes are still running. When off-heap column vectors are + // enabled, this can cause a use-after-free bug leading to a segfault. + // + // Instead, we use FileScanRDD's task completion listener to close this iterator. + val iter = new RecordReaderIterator(vectorizedReader) + try { + vectorizedReader.initialize(split, hadoopAttemptContext, Option.apply(fileFooter)) + vectorizedReader.initBatch(partitionSchema, file.partitionValues) + if (returningBatch) { + vectorizedReader.enableReturningBatches() + } + + // UnsafeRowParquetRecordReader appends the columns internally to avoid another copy. + iter.asInstanceOf[Iterator[InternalRow]] + } catch { + case e: Throwable => + // SPARK-23457: In case there is an exception in initialization, close the iterator to + // avoid leaking resources. + iter.close() + throw e + } + } else { + // ParquetRecordReader returns InternalRow + val readSupport = new ParquetReadSupport( + convertTz, + enableVectorizedReader = false, + datetimeRebaseSpec, + int96RebaseSpec) + val reader = if (pushed.isDefined && enableRecordFilter) { + val parquetFilter = FilterCompat.get(pushed.get, null) + new ParquetRecordReader[InternalRow](readSupport, parquetFilter) + } else { + new ParquetRecordReader[InternalRow](readSupport) + } + val readerWithRowIndexes = ParquetRowIndexUtil.addRowIndexToRecordReaderIfNeeded(reader, + requiredSchema) + val iter = new RecordReaderIterator[InternalRow](readerWithRowIndexes) + try { + readerWithRowIndexes.initialize(split, hadoopAttemptContext) + + val fullSchema = toAttributes(requiredSchema) ++ toAttributes(partitionSchema) + val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + + if (partitionSchema.length == 0) { + // There is no partition columns + iter.map(unsafeProjection) + } else { + val joinedRow = new JoinedRow() + iter.map(d => unsafeProjection(joinedRow(d, file.partitionValues))) + } + } catch { + case e: Throwable => + // SPARK-23457: In case there is an exception in initialization, close the iterator to + // avoid leaking resources. + iter.close() + throw e + } + } + } +} From 1a53f1e4b20538dd159af314e4eab7c1c247ddaa Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 2 Apr 2024 12:33:55 -0400 Subject: [PATCH 04/86] add spark 3.2 --- .../spark/sql/adapter/Spark3_2Adapter.scala | 41 ++- .../parquet/Spark32HoodieParquetReader.scala | 260 ++++++++++++++++++ 2 files changed, 300 insertions(+), 1 deletion(-) create mode 100644 hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReader.scala diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala index 3a5812a5faa4..d9e2869fdd7e 100644 --- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.adapter import org.apache.avro.Schema +import org.apache.hadoop.conf.Configuration import org.apache.hudi.Spark32HoodieFileScanRDD import org.apache.spark.sql._ import org.apache.spark.sql.avro._ @@ -30,10 +31,11 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{Command, DeleteFromTable, LogicalPlan} import org.apache.spark.sql.catalyst.util.METADATA_COL_ATTR_KEY import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark32LegacyHoodieParquetFileFormat} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark32HoodieParquetReader, Spark32LegacyHoodieParquetFileFormat} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hudi.analysis.TableValuedFunctions +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.parser.{HoodieExtendedParserInterface, HoodieSpark3_2ExtendedSqlParser} import org.apache.spark.sql.types.{DataType, Metadata, MetadataBuilder, StructType} import org.apache.spark.sql.vectorized.ColumnarUtils @@ -123,4 +125,41 @@ class Spark3_2Adapter extends BaseSpark3Adapter { case OFF_HEAP => "OFF_HEAP" case _ => throw new IllegalArgumentException(s"Invalid StorageLevel: $level") } + + /** + * 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 + */ + override def getPropsForReadingParquet(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): Map[String, String] = { + Spark32HoodieParquetReader.getPropsForReadingParquet(vectorized, sqlConf, options, hadoopConf) + } + + /** + * Read an individual parquet file + * + * @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]] + */ + override def readParquetFile(file: PartitionedFile, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[sources.Filter], + sharedConf: Configuration, + extraProps: Map[String, String]): Iterator[InternalRow] = { + Spark32HoodieParquetReader.readParquetFile(file, requiredSchema, partitionSchema, filters, + new Configuration(sharedConf), extraProps) + } } diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReader.scala new file mode 100644 index 000000000000..7d3da0a668c8 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReader.scala @@ -0,0 +1,260 @@ +/* + * 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.mapred.FileSplit +import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl +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._ +import org.apache.spark.TaskContext +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.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._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types._ + +import java.net.URI + +object Spark32HoodieParquetReader { + + /** + * 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.NESTED_SCHEMA_PRUNING_ENABLED.key, sqlConf.nestedSchemaPruningEnabled) + 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) + // Using string value of this conf to preserve compatibility across spark versions. See [HUDI-5868] + hadoopConf.setBoolean( + "spark.sql.legacy.parquet.nanosAsLong", + sqlConf.getConfString("spark.sql.legacy.parquet.nanosAsLong", "false").toBoolean + ) + + val parquetOptions = new ParquetOptions(options, sqlConf) + Map( + "enableVectorizedReader" -> vectorized.toString, + "datetimeRebaseModeInRead" -> parquetOptions.datetimeRebaseModeInRead, + "int96RebaseModeInRead" -> parquetOptions.int96RebaseModeInRead, + "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 v3.2.4 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, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[Filter], + sharedConf: Configuration, + extraProps: Map[String, String]): Iterator[InternalRow] = { + sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) + sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) + ParquetWriteSupport.setSchema(requiredSchema, sharedConf) + val enableVectorizedReader = extraProps("enableVectorizedReader").toBoolean + val datetimeRebaseModeInRead = extraProps("datetimeRebaseModeInRead") + val int96RebaseModeInRead = extraProps("int96RebaseModeInRead") + val enableParquetFilterPushDown = extraProps("enableParquetFilterPushDown").toBoolean + val pushDownDate = extraProps("pushDownDate").toBoolean + val pushDownTimestamp = extraProps("pushDownTimestamp").toBoolean + val pushDownDecimal = extraProps("pushDownDecimal").toBoolean + val pushDownInFilterThreshold = extraProps("pushDownInFilterThreshold").toInt + val pushDownStringStartWith = extraProps("pushDownStringStartWith").toBoolean + val isCaseSensitive = extraProps("isCaseSensitive").toBoolean + val timestampConversion = extraProps("timestampConversion").toBoolean + val enableOffHeapColumnVector = extraProps("enableOffHeapColumnVector").toBoolean + val capacity = extraProps("capacity").toInt + val returningBatch = extraProps("returningBatch").toBoolean + val enableRecordFilter = extraProps("enableRecordFilter").toBoolean + val timeZoneId = Option(extraProps("timeZoneId")) + + assert(file.partitionValues.numFields == partitionSchema.size) + + val filePath = new Path(new URI(file.filePath)) + val split = new FileSplit(filePath, file.start, file.length, Array.empty[String]) + + lazy val footerFileMetaData = + ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData + val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( + footerFileMetaData.getKeyValueMetaData.get, + datetimeRebaseModeInRead) + // Try to push down filters when filter push-down is enabled. + val pushed = if (enableParquetFilterPushDown) { + val parquetSchema = footerFileMetaData.getSchema + val parquetFilters = new ParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringStartWith, + pushDownInFilterThreshold, + isCaseSensitive, + datetimeRebaseSpec) + 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(_)) + .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.getZoneId(sharedConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key))) + } else { + None + } + + val int96RebaseSpec = DataSourceUtils.int96RebaseSpec( + footerFileMetaData.getKeyValueMetaData.get, + int96RebaseModeInRead) + + val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) + val hadoopAttemptContext = + new TaskAttemptContextImpl(sharedConf, 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()) + if (enableVectorizedReader) { + val vectorizedReader = new VectorizedParquetRecordReader( + convertTz.orNull, + datetimeRebaseSpec.mode.toString, + datetimeRebaseSpec.timeZone, + int96RebaseSpec.mode.toString, + int96RebaseSpec.timeZone, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) + // SPARK-37089: We cannot register a task completion listener to close this iterator here + // because downstream exec nodes have already registered their listeners. Since listeners + // are executed in reverse order of registration, a listener registered here would close the + // iterator while downstream exec nodes are still running. When off-heap column vectors are + // enabled, this can cause a use-after-free bug leading to a segfault. + // + // Instead, we use FileScanRDD's task completion listener to close this iterator. + val iter = new RecordReaderIterator(vectorizedReader) + try { + vectorizedReader.initialize(split, hadoopAttemptContext) + vectorizedReader.initBatch(partitionSchema, file.partitionValues) + if (returningBatch) { + vectorizedReader.enableReturningBatches() + } + + // UnsafeRowParquetRecordReader appends the columns internally to avoid another copy. + iter.asInstanceOf[Iterator[InternalRow]] + } catch { + case e: Throwable => + // SPARK-23457: In case there is an exception in initialization, close the iterator to + // avoid leaking resources. + iter.close() + throw e + } + } else { + // ParquetRecordReader returns InternalRow + val readSupport = new ParquetReadSupport( + convertTz, + enableVectorizedReader = false, + datetimeRebaseSpec, + int96RebaseSpec) + val reader = if (pushed.isDefined && enableRecordFilter) { + val parquetFilter = FilterCompat.get(pushed.get, null) + new ParquetRecordReader[InternalRow](readSupport, parquetFilter) + } else { + new ParquetRecordReader[InternalRow](readSupport) + } + val iter = new RecordReaderIterator[InternalRow](reader) + try { + reader.initialize(split, hadoopAttemptContext) + + val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes + val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + + if (partitionSchema.length == 0) { + // There is no partition columns + iter.map(unsafeProjection) + } else { + val joinedRow = new JoinedRow() + iter.map(d => unsafeProjection(joinedRow(d, file.partitionValues))) + } + } catch { + case e: Throwable => + // SPARK-23457: In case there is an exception in initialization, close the iterator to + // avoid leaking resources. + iter.close() + throw e + } + } + } +} From 97d99205e13248741ada5e804f976765913e093c Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 2 Apr 2024 12:42:53 -0400 Subject: [PATCH 05/86] add spark 3.1 --- .../spark/sql/adapter/Spark3_1Adapter.scala | 40 ++- .../parquet/Spark31HoodieParquetReader.scala | 235 ++++++++++++++++++ 2 files changed, 274 insertions(+), 1 deletion(-) create mode 100644 hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReader.scala diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala index 8ca072333d0e..827d6755c45c 100644 --- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.adapter import org.apache.avro.Schema +import org.apache.hadoop.conf.Configuration import org.apache.hudi.Spark31HoodieFileScanRDD import org.apache.spark.sql._ import org.apache.spark.sql.avro.{HoodieAvroDeserializer, HoodieAvroSerializer, HoodieSpark3_1AvroDeserializer, HoodieSpark3_1AvroSerializer} @@ -30,10 +31,11 @@ import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark31LegacyHoodieParquetFileFormat} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark31HoodieParquetReader, Spark31LegacyHoodieParquetFileFormat} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, HoodieSpark31PartitionedFileUtils, HoodieSparkPartitionedFileUtils, PartitionedFile} import org.apache.spark.sql.hudi.SparkAdapter +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.parser.{HoodieExtendedParserInterface, HoodieSpark3_1ExtendedSqlParser} import org.apache.spark.sql.types.{DataType, Metadata, MetadataBuilder, StructType} import org.apache.spark.sql.vectorized.ColumnarUtils @@ -121,4 +123,40 @@ class Spark3_1Adapter extends BaseSpark3Adapter { case _ => throw new IllegalArgumentException(s"Invalid StorageLevel: $level") } + /** + * 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 + */ + override def getPropsForReadingParquet(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): Map[String, String] = { + Spark31HoodieParquetReader.getPropsForReadingParquet(vectorized, sqlConf, options, hadoopConf) + } + + /** + * Read an individual parquet file + * + * @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]] + */ + override def readParquetFile(file: PartitionedFile, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[sources.Filter], + sharedConf: Configuration, + extraProps: Map[String, String]): Iterator[InternalRow] = { + Spark31HoodieParquetReader.readParquetFile(file, requiredSchema, partitionSchema, filters, + new Configuration(sharedConf), extraProps) + } } diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReader.scala new file mode 100644 index 000000000000..241d8d6b7946 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReader.scala @@ -0,0 +1,235 @@ +/* + * 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._ +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl +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._ +import org.apache.spark.TaskContext +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.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._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types._ + +import java.net.URI + +object Spark31HoodieParquetReader { + + /** + * 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.NESTED_SCHEMA_PRUNING_ENABLED.key, sqlConf.nestedSchemaPruningEnabled) + 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 v3.1.3 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, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[Filter], + sharedConf: Configuration, + extraProps: Map[String, String]): Iterator[InternalRow] = { + sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) + sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) + ParquetWriteSupport.setSchema(requiredSchema, sharedConf) + val enableVectorizedReader = extraProps("enableVectorizedReader").toBoolean + val enableParquetFilterPushDown = extraProps("enableParquetFilterPushDown").toBoolean + val pushDownDate = extraProps("pushDownDate").toBoolean + val pushDownTimestamp = extraProps("pushDownTimestamp").toBoolean + val pushDownDecimal = extraProps("pushDownDecimal").toBoolean + val pushDownInFilterThreshold = extraProps("pushDownInFilterThreshold").toInt + val pushDownStringStartWith = extraProps("pushDownStringStartWith").toBoolean + val isCaseSensitive = extraProps("isCaseSensitive").toBoolean + val timestampConversion = extraProps("timestampConversion").toBoolean + val enableOffHeapColumnVector = extraProps("enableOffHeapColumnVector").toBoolean + val capacity = extraProps("capacity").toInt + val returningBatch = extraProps("returningBatch").toBoolean + val enableRecordFilter = extraProps("enableRecordFilter").toBoolean + val timeZoneId = Option(extraProps("timeZoneId")) + + 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) + + 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 = new ParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringStartWith, + pushDownInFilterThreshold, + isCaseSensitive, + datetimeRebaseMode) + 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(_)) + .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.getZoneId(sharedConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key))) + } else { + None + } + + val int96RebaseMode = DataSourceUtils.int96RebaseMode( + footerFileMetaData.getKeyValueMetaData.get, + SQLConf.get.getConf(SQLConf.LEGACY_PARQUET_INT96_REBASE_MODE_IN_READ)) + + val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) + val hadoopAttemptContext = + new TaskAttemptContextImpl(sharedConf, 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()) + if (enableVectorizedReader) { + val vectorizedReader = new VectorizedParquetRecordReader( + convertTz.orNull, + datetimeRebaseMode.toString, + int96RebaseMode.toString, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) + val iter = new RecordReaderIterator(vectorizedReader) + // SPARK-23457 Register a task completion listener before `initialization`. + taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) + vectorizedReader.initialize(split, hadoopAttemptContext) + vectorizedReader.initBatch(partitionSchema, file.partitionValues) + if (returningBatch) { + vectorizedReader.enableReturningBatches() + } + + // UnsafeRowParquetRecordReader appends the columns internally to avoid another copy. + iter.asInstanceOf[Iterator[InternalRow]] + } else { + // ParquetRecordReader returns InternalRow + val readSupport = new ParquetReadSupport( + convertTz, + enableVectorizedReader = false, + datetimeRebaseMode, + int96RebaseMode) + val reader = if (pushed.isDefined && enableRecordFilter) { + val parquetFilter = FilterCompat.get(pushed.get, null) + new ParquetRecordReader[InternalRow](readSupport, parquetFilter) + } else { + new ParquetRecordReader[InternalRow](readSupport) + } + val iter = new RecordReaderIterator[InternalRow](reader) + // SPARK-23457 Register a task completion listener before `initialization`. + taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) + reader.initialize(split, hadoopAttemptContext) + + val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes + val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + + if (partitionSchema.length == 0) { + // There is no partition columns + iter.map(unsafeProjection) + } else { + val joinedRow = new JoinedRow() + iter.map(d => unsafeProjection(joinedRow(d, file.partitionValues))) + } + } + } +} From b9d7ce4f458e0ca6a5bdba6042d3bafb72ad801b Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 2 Apr 2024 12:53:45 -0400 Subject: [PATCH 06/86] add spark 3.0 --- .../spark/sql/adapter/Spark3_0Adapter.scala | 41 +++- .../parquet/Spark30HoodieParquetReader.scala | 222 ++++++++++++++++++ 2 files changed, 262 insertions(+), 1 deletion(-) create mode 100644 hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala index 22a9f090fb33..22b94faade94 100644 --- a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.adapter import org.apache.avro.Schema +import org.apache.hadoop.conf.Configuration import org.apache.hudi.Spark30HoodieFileScanRDD import org.apache.spark.sql._ import org.apache.spark.sql.avro.{HoodieAvroDeserializer, HoodieAvroSerializer, HoodieSpark3_0AvroDeserializer, HoodieSpark3_0AvroSerializer} @@ -29,10 +30,11 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark30LegacyHoodieParquetFileFormat} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark30HoodieParquetReader, Spark30LegacyHoodieParquetFileFormat} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, HoodieSpark30PartitionedFileUtils, HoodieSparkPartitionedFileUtils, PartitionedFile} import org.apache.spark.sql.hudi.SparkAdapter +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.parser.{HoodieExtendedParserInterface, HoodieSpark3_0ExtendedSqlParser} import org.apache.spark.sql.types.{DataType, Metadata, MetadataBuilder, StructType} import org.apache.spark.sql.vectorized.ColumnarUtils @@ -118,4 +120,41 @@ class Spark3_0Adapter extends BaseSpark3Adapter { case OFF_HEAP => "OFF_HEAP" case _ => throw new IllegalArgumentException(s"Invalid StorageLevel: $level") } + + /** + * 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 + */ + override def getPropsForReadingParquet(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): Map[String, String] = { + Spark30HoodieParquetReader.getPropsForReadingParquet(vectorized, sqlConf, options, hadoopConf) + } + + /** + * Read an individual parquet file + * + * @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]] + */ + override def readParquetFile(file: PartitionedFile, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[sources.Filter], + sharedConf: Configuration, + extraProps: Map[String, String]): Iterator[InternalRow] = { + Spark30HoodieParquetReader.readParquetFile(file, requiredSchema, partitionSchema, filters, + new Configuration(sharedConf), extraProps) + } } diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala new file mode 100644 index 000000000000..a29fa123e130 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.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._ +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl +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._ +import org.apache.spark.TaskContext +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.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._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types._ + +import java.net.URI + +object Spark30HoodieParquetReader { + + /** + * 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.NESTED_SCHEMA_PRUNING_ENABLED.key, sqlConf.nestedSchemaPruningEnabled) + 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 v3.0.3 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, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[Filter], + sharedConf: Configuration, + extraProps: Map[String, String]): Iterator[InternalRow] = { + sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) + sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) + ParquetWriteSupport.setSchema(requiredSchema, sharedConf) + val enableVectorizedReader = extraProps("enableVectorizedReader").toBoolean + val enableParquetFilterPushDown = extraProps("enableParquetFilterPushDown").toBoolean + val pushDownDate = extraProps("pushDownDate").toBoolean + val pushDownTimestamp = extraProps("pushDownTimestamp").toBoolean + val pushDownDecimal = extraProps("pushDownDecimal").toBoolean + val pushDownInFilterThreshold = extraProps("pushDownInFilterThreshold").toInt + val pushDownStringStartWith = extraProps("pushDownStringStartWith").toBoolean + val isCaseSensitive = extraProps("isCaseSensitive").toBoolean + val timestampConversion = extraProps("timestampConversion").toBoolean + val enableOffHeapColumnVector = extraProps("enableOffHeapColumnVector").toBoolean + val capacity = extraProps("capacity").toInt + val returningBatch = extraProps("returningBatch").toBoolean + val enableRecordFilter = extraProps("enableRecordFilter").toBoolean + val timeZoneId = Option(extraProps("timeZoneId")) + + 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) + + 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(parquetSchema, 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(_)) + .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.getZoneId(sharedConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key))) + } else { + None + } + + val datetimeRebaseMode = DataSourceUtils.datetimeRebaseMode( + footerFileMetaData.getKeyValueMetaData.get, + SQLConf.get.getConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ)) + + val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) + val hadoopAttemptContext = + new TaskAttemptContextImpl(sharedConf, 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()) + if (enableVectorizedReader) { + val vectorizedReader = new VectorizedParquetRecordReader( + convertTz.orNull, + datetimeRebaseMode.toString, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) + val iter = new RecordReaderIterator(vectorizedReader) + // SPARK-23457 Register a task completion listener before `initialization`. + taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) + vectorizedReader.initialize(split, hadoopAttemptContext) + vectorizedReader.initBatch(partitionSchema, file.partitionValues) + if (returningBatch) { + vectorizedReader.enableReturningBatches() + } + + // UnsafeRowParquetRecordReader appends the columns internally to avoid another copy. + iter.asInstanceOf[Iterator[InternalRow]] + } else { + // ParquetRecordReader returns InternalRow + val readSupport = new ParquetReadSupport( + convertTz, enableVectorizedReader = false, datetimeRebaseMode) + val reader = if (pushed.isDefined && enableRecordFilter) { + val parquetFilter = FilterCompat.get(pushed.get, null) + new ParquetRecordReader[InternalRow](readSupport, parquetFilter) + } else { + new ParquetRecordReader[InternalRow](readSupport) + } + val iter = new RecordReaderIterator[InternalRow](reader) + // SPARK-23457 Register a task completion listener before `initialization`. + taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) + reader.initialize(split, hadoopAttemptContext) + + val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes + val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + + if (partitionSchema.length == 0) { + // There is no partition columns + iter.map(unsafeProjection) + } else { + val joinedRow = new JoinedRow() + iter.map(d => unsafeProjection(joinedRow(d, file.partitionValues))) + } + } + } + +} From a20e9d4c236a04becc36724f22972c8eb925c15d Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 2 Apr 2024 13:05:55 -0400 Subject: [PATCH 07/86] add spark 2.4 --- .../spark/sql/adapter/Spark2Adapter.scala | 42 +++- .../parquet/Spark24HoodieParquetReader.scala | 222 ++++++++++++++++++ 2 files changed, 263 insertions(+), 1 deletion(-) create mode 100644 hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala index 932e3dd05f0d..0f86389150dc 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.adapter import org.apache.avro.Schema +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileStatus import org.apache.hadoop.fs.Path import org.apache.hudi.client.utils.SparkRowSerDe import org.apache.hudi.common.table.HoodieTableMetaClient @@ -31,9 +33,10 @@ import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.{Command, DeleteFromTable} import org.apache.spark.sql.catalyst.util.DateFormatter import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark24LegacyHoodieParquetFileFormat} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark24HoodieParquetReader, Spark24LegacyHoodieParquetFileFormat} import org.apache.spark.sql.execution.vectorized.MutableColumnarRow import org.apache.spark.sql.hudi.SparkAdapter +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.hudi.parser.HoodieSpark2ExtendedSqlParser import org.apache.spark.sql.parser.HoodieExtendedParserInterface import org.apache.spark.sql.sources.{BaseRelation, Filter} @@ -205,4 +208,41 @@ class Spark2Adapter extends SparkAdapter { batch.setNumRows(numRows) batch } + + /** + * 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 + */ + override def getPropsForReadingParquet(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): Map[String, String] = { + Spark24HoodieParquetReader.getPropsForReadingParquet(vectorized, sqlConf, options, hadoopConf) + } + + /** + * Read an individual parquet file + * + * @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]] + */ + override def readParquetFile(file: PartitionedFile, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[sources.Filter], + sharedConf: Configuration, + extraProps: Map[String, String]): Iterator[InternalRow] = { + Spark24HoodieParquetReader.readParquetFile(file, requiredSchema, partitionSchema, filters, + new Configuration(sharedConf), extraProps) + } } diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala new file mode 100644 index 000000000000..45a752562f24 --- /dev/null +++ b/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, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[Filter], + sharedConf: Configuration, + extraProps: Map[String, String]): Iterator[InternalRow] = { + sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) + sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) + ParquetWriteSupport.setSchema(requiredSchema, sharedConf) + val enableVectorizedReader = extraProps("enableVectorizedReader").toBoolean + val enableParquetFilterPushDown = extraProps("enableParquetFilterPushDown").toBoolean + val pushDownDate = extraProps("pushDownDate").toBoolean + val pushDownTimestamp = extraProps("pushDownTimestamp").toBoolean + val pushDownDecimal = extraProps("pushDownDecimal").toBoolean + val pushDownInFilterThreshold = extraProps("pushDownInFilterThreshold").toInt + val pushDownStringStartWith = extraProps("pushDownStringStartWith").toBoolean + val isCaseSensitive = extraProps("isCaseSensitive").toBoolean + val timestampConversion = extraProps("timestampConversion").toBoolean + val enableOffHeapColumnVector = extraProps("enableOffHeapColumnVector").toBoolean + val capacity = extraProps("capacity").toInt + val returningBatch = extraProps("returningBatch").toBoolean + val enableRecordFilter = extraProps("enableRecordFilter").toBoolean + val timeZoneId = Option(extraProps("timeZoneId")) + + assert(file.partitionValues.numFields == partitionSchema.size) + + val fileSplit = + new FileSplit(new Path(new URI(file.filePath)), file.start, file.length, Array.empty) + val filePath = fileSplit.getPath + + val split = + new org.apache.parquet.hadoop.ParquetInputSplit( + filePath, + fileSplit.getStart, + fileSplit.getStart + fileSplit.getLength, + fileSplit.getLength, + fileSplit.getLocations, + null) + + 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(sharedConf, 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()) + if (enableVectorizedReader) { + val vectorizedReader = new VectorizedParquetRecordReader( + convertTz.orNull, enableOffHeapColumnVector && taskContext.isDefined, capacity) + val iter = new RecordReaderIterator(vectorizedReader) + // SPARK-23457 Register a task completion lister before `initialization`. + taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) + vectorizedReader.initialize(split, hadoopAttemptContext) + vectorizedReader.initBatch(partitionSchema, file.partitionValues) + if (returningBatch) { + vectorizedReader.enableReturningBatches() + } + + // UnsafeRowParquetRecordReader appends the columns internally to avoid another copy. + iter.asInstanceOf[Iterator[InternalRow]] + } else { + // ParquetRecordReader returns UnsafeRow + val reader = if (pushed.isDefined && enableRecordFilter) { + val parquetFilter = FilterCompat.get(pushed.get, null) + new ParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz), parquetFilter) + } else { + new ParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz)) + } + val iter = new RecordReaderIterator(reader) + // SPARK-23457 Register a task completion lister before `initialization`. + taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) + reader.initialize(split, hadoopAttemptContext) + + 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 + // the type in next() and we get a class cast exception. If we make that function return + // Object, then we can defer the cast until later! + if (partitionSchema.length == 0) { + // There is no partition columns + iter.asInstanceOf[Iterator[InternalRow]] + } else { + iter.asInstanceOf[Iterator[InternalRow]] + .map(d => appendPartitionColumns(joinedRow(d, file.partitionValues))) + } + } + } + +} From abe7839661e43acc9eb7f49b4bea3885dcaf8d5f Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Wed, 3 Apr 2024 17:24:52 -0400 Subject: [PATCH 08/86] spark 3.3 use properties class --- .../apache/spark/sql/hudi/SparkAdapter.scala | 5 +- .../Spark3HoodieParquetReaderProperties.scala | 57 +++++++++++++++ .../spark/sql/adapter/Spark3_3Adapter.scala | 7 +- .../parquet/Spark33HoodieParquetReader.scala | 72 +++++++++---------- ...Spark33HoodieParquetReaderProperties.scala | 54 ++++++++++++++ 5 files changed, 154 insertions(+), 41 deletions(-) create mode 100644 hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3HoodieParquetReaderProperties.scala create mode 100644 hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReaderProperties.scala diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala index 0ed33c0e5ef7..4474b8ac5762 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.util.DateFormatter import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.hudi.execution.datasources.parquet.SparkHoodieParquetReaderProperties import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.parser.HoodieExtendedParserInterface import org.apache.spark.sql.sources.{BaseRelation, Filter} @@ -229,7 +230,7 @@ trait SparkAdapter extends Serializable { def getPropsForReadingParquet(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): Map[String, String] + hadoopConf: Configuration): SparkHoodieParquetReaderProperties /** * Read an individual parquet file @@ -246,5 +247,5 @@ trait SparkAdapter extends Serializable { partitionSchema: StructType, filters: Seq[Filter], sharedConf: Configuration, - extraProps: Map[String, String]): Iterator[InternalRow] + props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] } diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3HoodieParquetReaderProperties.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3HoodieParquetReaderProperties.scala new file mode 100644 index 000000000000..ce47be345e8c --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3HoodieParquetReaderProperties.scala @@ -0,0 +1,57 @@ +/* + * 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.spark.sql.hudi.execution.datasources.parquet.SparkHoodieParquetReaderProperties + + +abstract class Spark3HoodieParquetReaderProperties(enableVectorizedReader: Boolean, + datetimeRebaseModeInRead: String, + int96RebaseModeInRead: String, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringStartWith: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) + extends SparkHoodieParquetReaderProperties( + enableVectorizedReader = enableVectorizedReader, + datetimeRebaseModeInRead = datetimeRebaseModeInRead, + int96RebaseModeInRead = int96RebaseModeInRead, + enableParquetFilterPushDown = enableParquetFilterPushDown, + pushDownDate = pushDownDate, + pushDownTimestamp = pushDownTimestamp, + pushDownDecimal = pushDownDecimal, + pushDownInFilterThreshold = pushDownInFilterThreshold, + pushDownStringStartWith = pushDownStringStartWith, + isCaseSensitive = isCaseSensitive, + timestampConversion = timestampConversion, + enableOffHeapColumnVector = enableOffHeapColumnVector, + capacity = capacity, + returningBatch = returningBatch, + enableRecordFilter = enableRecordFilter, + timeZoneId = timeZoneId) diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala index bf79666121a8..6edd158dfbc2 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Sp import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hudi.analysis.TableValuedFunctions +import org.apache.spark.sql.hudi.execution.datasources.parquet.SparkHoodieParquetReaderProperties import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.parser.{HoodieExtendedParserInterface, HoodieSpark3_3ExtendedSqlParser} import org.apache.spark.sql.types.{DataType, Metadata, MetadataBuilder, StructType} @@ -139,7 +140,7 @@ class Spark3_3Adapter extends BaseSpark3Adapter { override def getPropsForReadingParquet(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): Map[String, String] = { + hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { Spark33HoodieParquetReader.getPropsForReadingParquet(vectorized, sqlConf, options, hadoopConf) } @@ -159,8 +160,8 @@ class Spark3_3Adapter extends BaseSpark3Adapter { partitionSchema: StructType, filters: Seq[sources.Filter], sharedConf: Configuration, - extraProps: Map[String, String]): Iterator[InternalRow] = { + props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { Spark33HoodieParquetReader.readParquetFile(file, requiredSchema, partitionSchema, filters, - new Configuration(sharedConf), extraProps) + new Configuration(sharedConf), props) } } diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReader.scala index ef5a89578c8e..6a6036158640 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReader.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.expressions.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._ +import org.apache.spark.sql.hudi.execution.datasources.parquet.SparkHoodieParquetReaderProperties import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ @@ -55,7 +56,7 @@ object Spark33HoodieParquetReader { def getPropsForReadingParquet(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): Map[String, String] = { + hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { //set hadoopconf hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) @@ -70,24 +71,22 @@ object Spark33HoodieParquetReader { ) val parquetOptions = new ParquetOptions(options, sqlConf) - Map( - "enableVectorizedReader" -> vectorized.toString, - "datetimeRebaseModeInRead" -> parquetOptions.datetimeRebaseModeInRead, - "int96RebaseModeInRead" -> parquetOptions.int96RebaseModeInRead, - "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 - ) + new Spark33HoodieParquetReaderProperties(enableVectorizedReader = vectorized, + datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead, + int96RebaseModeInRead = parquetOptions.int96RebaseModeInRead, + enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, + pushDownDate = sqlConf.parquetFilterPushDownDate, + pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp, + pushDownDecimal = sqlConf.parquetFilterPushDownDecimal, + pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold, + pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith, + isCaseSensitive = sqlConf.caseSensitiveAnalysis, + timestampConversion = sqlConf.isParquetINT96TimestampConversion, + enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, + capacity = sqlConf.parquetVectorizedReaderBatchSize, + returningBatch = sqlConf.parquetVectorizedReaderEnabled, + enableRecordFilter = sqlConf.parquetRecordFilterEnabled, + timeZoneId = Some(sqlConf.sessionLocalTimeZone)) } /** @@ -107,26 +106,27 @@ object Spark33HoodieParquetReader { partitionSchema: StructType, filters: Seq[Filter], sharedConf: Configuration, - extraProps: Map[String, String]): Iterator[InternalRow] = { + props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) ParquetWriteSupport.setSchema(requiredSchema, sharedConf) - val enableVectorizedReader = extraProps("enableVectorizedReader").toBoolean - val datetimeRebaseModeInRead = extraProps("datetimeRebaseModeInRead") - val int96RebaseModeInRead = extraProps("int96RebaseModeInRead") - val enableParquetFilterPushDown = extraProps("enableParquetFilterPushDown").toBoolean - val pushDownDate = extraProps("pushDownDate").toBoolean - val pushDownTimestamp = extraProps("pushDownTimestamp").toBoolean - val pushDownDecimal = extraProps("pushDownDecimal").toBoolean - val pushDownInFilterThreshold = extraProps("pushDownInFilterThreshold").toInt - val pushDownStringStartWith = extraProps("pushDownStringStartWith").toBoolean - val isCaseSensitive = extraProps("isCaseSensitive").toBoolean - val timestampConversion = extraProps("timestampConversion").toBoolean - val enableOffHeapColumnVector = extraProps("enableOffHeapColumnVector").toBoolean - val capacity = extraProps("capacity").toInt - val returningBatch = extraProps("returningBatch").toBoolean - val enableRecordFilter = extraProps("enableRecordFilter").toBoolean - val timeZoneId = Option(extraProps("timeZoneId")) + val properties = props.asInstanceOf[Spark33HoodieParquetReaderProperties] + val enableVectorizedReader = properties.enableVectorizedReader + val datetimeRebaseModeInRead = properties.datetimeRebaseModeInRead + val int96RebaseModeInRead = properties.int96RebaseModeInRead + val enableParquetFilterPushDown = properties.enableParquetFilterPushDown + val pushDownDate = properties.pushDownDate + val pushDownTimestamp = properties.pushDownTimestamp + val pushDownDecimal = properties.pushDownDecimal + val pushDownInFilterThreshold = properties.pushDownInFilterThreshold + val pushDownStringStartWith = properties.pushDownStringStartWith + val isCaseSensitive = properties.isCaseSensitive + val timestampConversion = properties.timestampConversion + val enableOffHeapColumnVector = properties.enableOffHeapColumnVector + val capacity = properties.capacity + val returningBatch = properties.returningBatch + val enableRecordFilter = properties.enableRecordFilter + val timeZoneId = properties.timeZoneId assert(file.partitionValues.numFields == partitionSchema.size) val filePath = new Path(new URI(file.filePath)) diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReaderProperties.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReaderProperties.scala new file mode 100644 index 000000000000..79649b07f0a3 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReaderProperties.scala @@ -0,0 +1,54 @@ +/* + * 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 + +case class Spark33HoodieParquetReaderProperties(enableVectorizedReader: Boolean, + datetimeRebaseModeInRead: String, + int96RebaseModeInRead: String, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringStartWith: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) + extends Spark3HoodieParquetReaderProperties( + enableVectorizedReader = enableVectorizedReader, + datetimeRebaseModeInRead = datetimeRebaseModeInRead, + int96RebaseModeInRead = int96RebaseModeInRead, + enableParquetFilterPushDown = enableParquetFilterPushDown, + pushDownDate = pushDownDate, + pushDownTimestamp = pushDownTimestamp, + pushDownDecimal = pushDownDecimal, + pushDownInFilterThreshold = pushDownInFilterThreshold, + pushDownStringStartWith = pushDownStringStartWith, + isCaseSensitive = isCaseSensitive, + timestampConversion = timestampConversion, + enableOffHeapColumnVector = enableOffHeapColumnVector, + capacity = capacity, + returningBatch = returningBatch, + enableRecordFilter = enableRecordFilter, + timeZoneId = timeZoneId) From 865526e2bb6d40e51fe7b72bb5313701efb6df19 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Wed, 3 Apr 2024 17:58:31 -0400 Subject: [PATCH 09/86] spark 3.2 add props class --- .../SparkHoodieParquetReaderProperties.scala | 37 ++++++++++ .../apache/spark/sql/hudi/SparkAdapter.scala | 3 +- .../Spark3HoodieParquetReaderProperties.scala | 2 - .../spark/sql/adapter/Spark3_2Adapter.scala | 8 +-- .../parquet/Spark32HoodieParquetReader.scala | 71 +++++++++---------- ...Spark32HoodieParquetReaderProperties.scala | 54 ++++++++++++++ 6 files changed, 131 insertions(+), 44 deletions(-) create mode 100644 hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderProperties.scala create mode 100644 hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReaderProperties.scala diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderProperties.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderProperties.scala new file mode 100644 index 000000000000..ad4750117086 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderProperties.scala @@ -0,0 +1,37 @@ +/* + * 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 + +abstract class SparkHoodieParquetReaderProperties(enableVectorizedReader: Boolean, + datetimeRebaseModeInRead: String, + int96RebaseModeInRead: String, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringStartWith: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala index 4474b8ac5762..4e65b22fd370 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala @@ -34,8 +34,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} import org.apache.spark.sql.catalyst.util.DateFormatter import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.hudi.execution.datasources.parquet.SparkHoodieParquetReaderProperties +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, SparkHoodieParquetReaderProperties} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.parser.HoodieExtendedParserInterface import org.apache.spark.sql.sources.{BaseRelation, Filter} diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3HoodieParquetReaderProperties.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3HoodieParquetReaderProperties.scala index ce47be345e8c..5e65094c518d 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3HoodieParquetReaderProperties.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3HoodieParquetReaderProperties.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.execution.datasources.parquet -import org.apache.spark.sql.hudi.execution.datasources.parquet.SparkHoodieParquetReaderProperties - abstract class Spark3HoodieParquetReaderProperties(enableVectorizedReader: Boolean, datetimeRebaseModeInRead: String, diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala index d9e2869fdd7e..6215594a8493 100644 --- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{Command, DeleteFromTable, LogicalPlan} import org.apache.spark.sql.catalyst.util.METADATA_COL_ATTR_KEY import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark32HoodieParquetReader, Spark32LegacyHoodieParquetFileFormat} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark32HoodieParquetReader, Spark32LegacyHoodieParquetFileFormat, SparkHoodieParquetReaderProperties} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hudi.analysis.TableValuedFunctions @@ -138,7 +138,7 @@ class Spark3_2Adapter extends BaseSpark3Adapter { override def getPropsForReadingParquet(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): Map[String, String] = { + hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { Spark32HoodieParquetReader.getPropsForReadingParquet(vectorized, sqlConf, options, hadoopConf) } @@ -158,8 +158,8 @@ class Spark3_2Adapter extends BaseSpark3Adapter { partitionSchema: StructType, filters: Seq[sources.Filter], sharedConf: Configuration, - extraProps: Map[String, String]): Iterator[InternalRow] = { + props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { Spark32HoodieParquetReader.readParquetFile(file, requiredSchema, partitionSchema, filters, - new Configuration(sharedConf), extraProps) + new Configuration(sharedConf), props) } } diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReader.scala index 7d3da0a668c8..bb32480cc622 100644 --- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReader.scala @@ -54,7 +54,7 @@ object Spark32HoodieParquetReader { def getPropsForReadingParquet(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): Map[String, String] = { + hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { //set hadoopconf hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) @@ -69,24 +69,22 @@ object Spark32HoodieParquetReader { ) val parquetOptions = new ParquetOptions(options, sqlConf) - Map( - "enableVectorizedReader" -> vectorized.toString, - "datetimeRebaseModeInRead" -> parquetOptions.datetimeRebaseModeInRead, - "int96RebaseModeInRead" -> parquetOptions.int96RebaseModeInRead, - "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 - ) + Spark32HoodieParquetReaderProperties(enableVectorizedReader = vectorized, + datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead, + int96RebaseModeInRead = parquetOptions.int96RebaseModeInRead, + enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, + pushDownDate = sqlConf.parquetFilterPushDownDate, + pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp, + pushDownDecimal = sqlConf.parquetFilterPushDownDecimal, + pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold, + pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith, + isCaseSensitive = sqlConf.caseSensitiveAnalysis, + timestampConversion = sqlConf.isParquetINT96TimestampConversion, + enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, + capacity = sqlConf.parquetVectorizedReaderBatchSize, + returningBatch = sqlConf.parquetVectorizedReaderEnabled, + enableRecordFilter = sqlConf.parquetRecordFilterEnabled, + timeZoneId = Some(sqlConf.sessionLocalTimeZone)) } /** @@ -106,26 +104,27 @@ object Spark32HoodieParquetReader { partitionSchema: StructType, filters: Seq[Filter], sharedConf: Configuration, - extraProps: Map[String, String]): Iterator[InternalRow] = { + props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) ParquetWriteSupport.setSchema(requiredSchema, sharedConf) - val enableVectorizedReader = extraProps("enableVectorizedReader").toBoolean - val datetimeRebaseModeInRead = extraProps("datetimeRebaseModeInRead") - val int96RebaseModeInRead = extraProps("int96RebaseModeInRead") - val enableParquetFilterPushDown = extraProps("enableParquetFilterPushDown").toBoolean - val pushDownDate = extraProps("pushDownDate").toBoolean - val pushDownTimestamp = extraProps("pushDownTimestamp").toBoolean - val pushDownDecimal = extraProps("pushDownDecimal").toBoolean - val pushDownInFilterThreshold = extraProps("pushDownInFilterThreshold").toInt - val pushDownStringStartWith = extraProps("pushDownStringStartWith").toBoolean - val isCaseSensitive = extraProps("isCaseSensitive").toBoolean - val timestampConversion = extraProps("timestampConversion").toBoolean - val enableOffHeapColumnVector = extraProps("enableOffHeapColumnVector").toBoolean - val capacity = extraProps("capacity").toInt - val returningBatch = extraProps("returningBatch").toBoolean - val enableRecordFilter = extraProps("enableRecordFilter").toBoolean - val timeZoneId = Option(extraProps("timeZoneId")) + val properties = props.asInstanceOf[Spark32HoodieParquetReaderProperties] + val enableVectorizedReader = properties.enableVectorizedReader + val datetimeRebaseModeInRead = properties.datetimeRebaseModeInRead + val int96RebaseModeInRead = properties.int96RebaseModeInRead + val enableParquetFilterPushDown = properties.enableParquetFilterPushDown + val pushDownDate = properties.pushDownDate + val pushDownTimestamp = properties.pushDownTimestamp + val pushDownDecimal = properties.pushDownDecimal + val pushDownInFilterThreshold = properties.pushDownInFilterThreshold + val pushDownStringStartWith = properties.pushDownStringStartWith + val isCaseSensitive = properties.isCaseSensitive + val timestampConversion = properties.timestampConversion + val enableOffHeapColumnVector = properties.enableOffHeapColumnVector + val capacity = properties.capacity + val returningBatch = properties.returningBatch + val enableRecordFilter = properties.enableRecordFilter + val timeZoneId = properties.timeZoneId assert(file.partitionValues.numFields == partitionSchema.size) diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReaderProperties.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReaderProperties.scala new file mode 100644 index 000000000000..b2cbcfa06a5e --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReaderProperties.scala @@ -0,0 +1,54 @@ +/* + * 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 + +case class Spark32HoodieParquetReaderProperties(enableVectorizedReader: Boolean, + datetimeRebaseModeInRead: String, + int96RebaseModeInRead: String, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringStartWith: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) + extends Spark3HoodieParquetReaderProperties( + enableVectorizedReader = enableVectorizedReader, + datetimeRebaseModeInRead = datetimeRebaseModeInRead, + int96RebaseModeInRead = int96RebaseModeInRead, + enableParquetFilterPushDown = enableParquetFilterPushDown, + pushDownDate = pushDownDate, + pushDownTimestamp = pushDownTimestamp, + pushDownDecimal = pushDownDecimal, + pushDownInFilterThreshold = pushDownInFilterThreshold, + pushDownStringStartWith = pushDownStringStartWith, + isCaseSensitive = isCaseSensitive, + timestampConversion = timestampConversion, + enableOffHeapColumnVector = enableOffHeapColumnVector, + capacity = capacity, + returningBatch = returningBatch, + enableRecordFilter = enableRecordFilter, + timeZoneId = timeZoneId) From bab974a26018ef5a42cb7e08fc95b0e0bb94f75e Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 4 Apr 2024 10:12:58 -0400 Subject: [PATCH 10/86] spark 3.4 add properties --- .../SparkHoodieParquetReaderProperties.scala | 1 - .../Spark3HoodieParquetReaderProperties.scala | 2 - .../spark/sql/adapter/Spark3_4Adapter.scala | 10 +-- .../parquet/Spark34HoodieParquetReader.scala | 72 +++++++++---------- ...Spark34HoodieParquetReaderProperties.scala | 53 ++++++++++++++ 5 files changed, 94 insertions(+), 44 deletions(-) create mode 100644 hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReaderProperties.scala diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderProperties.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderProperties.scala index ad4750117086..1aa237b00166 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderProperties.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderProperties.scala @@ -27,7 +27,6 @@ abstract class SparkHoodieParquetReaderProperties(enableVectorizedReader: Boolea pushDownTimestamp: Boolean, pushDownDecimal: Boolean, pushDownInFilterThreshold: Int, - pushDownStringStartWith: Boolean, isCaseSensitive: Boolean, timestampConversion: Boolean, enableOffHeapColumnVector: Boolean, diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3HoodieParquetReaderProperties.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3HoodieParquetReaderProperties.scala index 5e65094c518d..f14590d1d028 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3HoodieParquetReaderProperties.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3HoodieParquetReaderProperties.scala @@ -28,7 +28,6 @@ abstract class Spark3HoodieParquetReaderProperties(enableVectorizedReader: Boole pushDownTimestamp: Boolean, pushDownDecimal: Boolean, pushDownInFilterThreshold: Int, - pushDownStringStartWith: Boolean, isCaseSensitive: Boolean, timestampConversion: Boolean, enableOffHeapColumnVector: Boolean, @@ -45,7 +44,6 @@ abstract class Spark3HoodieParquetReaderProperties(enableVectorizedReader: Boole pushDownTimestamp = pushDownTimestamp, pushDownDecimal = pushDownDecimal, pushDownInFilterThreshold = pushDownInFilterThreshold, - pushDownStringStartWith = pushDownStringStartWith, isCaseSensitive = isCaseSensitive, timestampConversion = timestampConversion, enableOffHeapColumnVector = enableOffHeapColumnVector, diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala index b17efe063710..ce98f1980f05 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.METADATA_COL_ATTR_KEY import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark34HoodieParquetReader, Spark34LegacyHoodieParquetFileFormat} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark34HoodieParquetReader, Spark34LegacyHoodieParquetFileFormat, SparkHoodieParquetReaderProperties} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hudi.analysis.TableValuedFunctions @@ -139,7 +139,7 @@ class Spark3_4Adapter extends BaseSpark3Adapter { override def getPropsForReadingParquet(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): Map[String, String] = { + hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { Spark34HoodieParquetReader.getPropsForReadingParquet(vectorized, sqlConf, options, hadoopConf) } @@ -151,7 +151,7 @@ class Spark3_4Adapter extends BaseSpark3Adapter { * @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 + * @param props 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]] */ override def readParquetFile(file: PartitionedFile, @@ -159,8 +159,8 @@ class Spark3_4Adapter extends BaseSpark3Adapter { partitionSchema: StructType, filters: Seq[sources.Filter], sharedConf: Configuration, - extraProps: Map[String, String]): Iterator[InternalRow] = { + props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { Spark34HoodieParquetReader.readParquetFile(file, requiredSchema, partitionSchema, filters, - new Configuration(sharedConf), extraProps) + new Configuration(sharedConf), props) } } diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReader.scala index 4549507f7e08..a019b36ff233 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReader.scala @@ -51,7 +51,7 @@ object Spark34HoodieParquetReader { def getPropsForReadingParquet(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): Map[String, String] = { + hadoopConf: Configuration): SparkHoodieParquetReaderProperties= { //set hadoopconf hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) @@ -76,24 +76,23 @@ object Spark34HoodieParquetReader { .equals("true") val parquetOptions = new ParquetOptions(options, sqlConf) - Map( - "enableVectorizedReader" -> vectorized.toString, - "datetimeRebaseModeInRead" -> parquetOptions.datetimeRebaseModeInRead, - "int96RebaseModeInRead" -> parquetOptions.int96RebaseModeInRead, - "enableParquetFilterPushDown" -> sqlConf.parquetFilterPushDown.toString, - "pushDownDate" -> sqlConf.parquetFilterPushDownDate.toString, - "pushDownTimestamp" -> sqlConf.parquetFilterPushDownTimestamp.toString, - "pushDownDecimal" -> sqlConf.parquetFilterPushDownDecimal.toString, - "pushDownStringPredicate" -> sqlConf.parquetFilterPushDownStringPredicate.toString, - "pushDownInFilterThreshold" -> sqlConf.parquetFilterPushDownInFilterThreshold.toString, - "isCaseSensitive" -> sqlConf.caseSensitiveAnalysis.toString, - "timestampConversion" -> sqlConf.isParquetINT96TimestampConversion.toString, - "enableOffHeapColumnVector" -> sqlConf.offHeapColumnVectorEnabled.toString, - "capacity" -> sqlConf.parquetVectorizedReaderBatchSize.toString, - "returningBatch" -> returningBatch.toString, - "enableRecordFilter" -> sqlConf.parquetRecordFilterEnabled.toString, - "timeZoneId" -> sqlConf.sessionLocalTimeZone - ) + Spark34HoodieParquetReaderProperties( + enableVectorizedReader = vectorized, + datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead, + int96RebaseModeInRead = parquetOptions.int96RebaseModeInRead, + enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, + pushDownDate = sqlConf.parquetFilterPushDownDate, + pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp, + pushDownDecimal = sqlConf.parquetFilterPushDownDecimal, + pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold, + pushDownStringPredicate = sqlConf.parquetFilterPushDownStringPredicate, + isCaseSensitive = sqlConf.caseSensitiveAnalysis, + timestampConversion = sqlConf.isParquetINT96TimestampConversion, + enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, + capacity = sqlConf.parquetVectorizedReaderBatchSize, + returningBatch = returningBatch, + enableRecordFilter = sqlConf.parquetRecordFilterEnabled, + timeZoneId = Some(sqlConf.sessionLocalTimeZone)) } /** @@ -113,26 +112,27 @@ object Spark34HoodieParquetReader { partitionSchema: StructType, filters: Seq[Filter], sharedConf: Configuration, - extraProps: Map[String, String]): Iterator[InternalRow] = { + props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) ParquetWriteSupport.setSchema(requiredSchema, sharedConf) - val enableVectorizedReader = extraProps("enableVectorizedReader").toBoolean - val datetimeRebaseModeInRead = extraProps("datetimeRebaseModeInRead") - val int96RebaseModeInRead = extraProps("int96RebaseModeInRead") - val enableParquetFilterPushDown = extraProps("enableParquetFilterPushDown").toBoolean - val pushDownDate = extraProps("pushDownDate").toBoolean - val pushDownTimestamp = extraProps("pushDownTimestamp").toBoolean - val pushDownDecimal = extraProps("pushDownDecimal").toBoolean - val pushDownStringPredicate = extraProps("pushDownStringPredicate").toBoolean - val pushDownInFilterThreshold = extraProps("pushDownInFilterThreshold").toInt - val isCaseSensitive = extraProps("isCaseSensitive").toBoolean - val timestampConversion = extraProps("timestampConversion").toBoolean - val enableOffHeapColumnVector = extraProps("enableOffHeapColumnVector").toBoolean - val capacity = extraProps("capacity").toInt - val returningBatch = extraProps("returningBatch").toBoolean - val enableRecordFilter = extraProps("enableRecordFilter").toBoolean - val timeZoneId = Option(extraProps("timeZoneId")) + val properties = props.asInstanceOf[Spark34HoodieParquetReaderProperties] + val enableVectorizedReader = properties.enableVectorizedReader + val datetimeRebaseModeInRead = properties.datetimeRebaseModeInRead + val int96RebaseModeInRead = properties.int96RebaseModeInRead + val enableParquetFilterPushDown = properties.enableParquetFilterPushDown + val pushDownDate = properties.pushDownDate + val pushDownTimestamp = properties.pushDownTimestamp + val pushDownDecimal = properties.pushDownDecimal + val pushDownInFilterThreshold = properties.pushDownInFilterThreshold + val pushDownStringPredicate = properties.pushDownStringPredicate + val isCaseSensitive = properties.isCaseSensitive + val timestampConversion = properties.timestampConversion + val enableOffHeapColumnVector = properties.enableOffHeapColumnVector + val capacity = properties.capacity + val returningBatch = properties.returningBatch + val enableRecordFilter = properties.enableRecordFilter + val timeZoneId = properties.timeZoneId assert(file.partitionValues.numFields == partitionSchema.size) diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReaderProperties.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReaderProperties.scala new file mode 100644 index 000000000000..d868320d8ad3 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReaderProperties.scala @@ -0,0 +1,53 @@ +/* + * 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 + +case class Spark34HoodieParquetReaderProperties(enableVectorizedReader: Boolean, + datetimeRebaseModeInRead: String, + int96RebaseModeInRead: String, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringPredicate: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) + extends Spark3HoodieParquetReaderProperties( + enableVectorizedReader = enableVectorizedReader, + datetimeRebaseModeInRead = datetimeRebaseModeInRead, + int96RebaseModeInRead = int96RebaseModeInRead, + enableParquetFilterPushDown = enableParquetFilterPushDown, + pushDownDate = pushDownDate, + pushDownTimestamp = pushDownTimestamp, + pushDownDecimal = pushDownDecimal, + pushDownInFilterThreshold = pushDownInFilterThreshold, + isCaseSensitive = isCaseSensitive, + timestampConversion = timestampConversion, + enableOffHeapColumnVector = enableOffHeapColumnVector, + capacity = capacity, + returningBatch = returningBatch, + enableRecordFilter = enableRecordFilter, + timeZoneId = timeZoneId) From 0eb218586f309d5dd2eeb5d2c8cd0ca3b7f54565 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 4 Apr 2024 10:21:36 -0400 Subject: [PATCH 11/86] add spark 3.5 properties --- .../apache/spark/sql/hudi/SparkAdapter.scala | 13 ++-- .../spark/sql/adapter/Spark3_5Adapter.scala | 12 +-- .../parquet/Spark35HoodieParquetReader.scala | 76 +++++++++---------- ...Spark35HoodieParquetReaderProperties.scala | 53 +++++++++++++ 4 files changed, 104 insertions(+), 50 deletions(-) create mode 100644 hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReaderProperties.scala diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala index 4e65b22fd370..b2c6ebe97fe4 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala @@ -224,7 +224,7 @@ trait SparkAdapter extends Serializable { * @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 + * @return properties needed for reading a parquet file */ def getPropsForReadingParquet(vectorized: Boolean, sqlConf: SQLConf, @@ -233,12 +233,13 @@ trait SparkAdapter extends Serializable { /** * Read an individual parquet file - * @param file parquet file to read - * @param requiredSchema desired output schema of the data + * + * @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 + * @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 props 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, diff --git a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_5Adapter.scala b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_5Adapter.scala index c6bc5cbc8cb5..a4bd14649032 100644 --- a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_5Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_5Adapter.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.METADATA_COL_ATTR_KEY import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark35HoodieParquetReader, Spark35LegacyHoodieParquetFileFormat} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark35HoodieParquetReader, Spark35LegacyHoodieParquetFileFormat, SparkHoodieParquetReaderProperties} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.hudi.analysis.TableValuedFunctions import org.apache.spark.sql.internal.SQLConf @@ -134,12 +134,12 @@ class Spark3_5Adapter extends BaseSpark3Adapter { * @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 + * @return properties needed for reading a parquet file */ override def getPropsForReadingParquet(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): Map[String, String] = { + hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { Spark35HoodieParquetReader.getPropsForReadingParquet(vectorized, sqlConf, options, hadoopConf) } @@ -151,7 +151,7 @@ class Spark3_5Adapter extends BaseSpark3Adapter { * @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 + * @param props 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]] */ override def readParquetFile(file: PartitionedFile, @@ -159,8 +159,8 @@ class Spark3_5Adapter extends BaseSpark3Adapter { partitionSchema: StructType, filters: Seq[sources.Filter], sharedConf: Configuration, - extraProps: Map[String, String]): Iterator[InternalRow] = { + props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { Spark35HoodieParquetReader.readParquetFile(file, requiredSchema, partitionSchema, filters, - new Configuration(sharedConf), extraProps) + new Configuration(sharedConf), props) } } diff --git a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReader.scala index 890a66fed1a6..1dc5a00279b7 100644 --- a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReader.scala @@ -46,12 +46,12 @@ object Spark35HoodieParquetReader { * @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 + * @return properties needed for reading a parquet file */ def getPropsForReadingParquet(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): Map[String, String] = { + hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { //set hadoopconf hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) @@ -76,24 +76,23 @@ object Spark35HoodieParquetReader { .equals("true") val parquetOptions = new ParquetOptions(options, sqlConf) - Map( - "enableVectorizedReader" -> vectorized.toString, - "datetimeRebaseModeInRead" -> parquetOptions.datetimeRebaseModeInRead, - "int96RebaseModeInRead" -> parquetOptions.int96RebaseModeInRead, - "enableParquetFilterPushDown" -> sqlConf.parquetFilterPushDown.toString, - "pushDownDate" -> sqlConf.parquetFilterPushDownDate.toString, - "pushDownTimestamp" -> sqlConf.parquetFilterPushDownTimestamp.toString, - "pushDownDecimal" -> sqlConf.parquetFilterPushDownDecimal.toString, - "pushDownStringPredicate" -> sqlConf.parquetFilterPushDownStringPredicate.toString, - "pushDownInFilterThreshold" -> sqlConf.parquetFilterPushDownInFilterThreshold.toString, - "isCaseSensitive" -> sqlConf.caseSensitiveAnalysis.toString, - "timestampConversion" -> sqlConf.isParquetINT96TimestampConversion.toString, - "enableOffHeapColumnVector" -> sqlConf.offHeapColumnVectorEnabled.toString, - "capacity" -> sqlConf.parquetVectorizedReaderBatchSize.toString, - "returningBatch" -> returningBatch.toString, - "enableRecordFilter" -> sqlConf.parquetRecordFilterEnabled.toString, - "timeZoneId" -> sqlConf.sessionLocalTimeZone - ) + Spark35HoodieParquetReaderProperties( + enableVectorizedReader = vectorized, + datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead, + int96RebaseModeInRead = parquetOptions.int96RebaseModeInRead, + enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, + pushDownDate = sqlConf.parquetFilterPushDownDate, + pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp, + pushDownDecimal = sqlConf.parquetFilterPushDownDecimal, + pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold, + pushDownStringPredicate = sqlConf.parquetFilterPushDownStringPredicate, + isCaseSensitive = sqlConf.caseSensitiveAnalysis, + timestampConversion = sqlConf.isParquetINT96TimestampConversion, + enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, + capacity = sqlConf.parquetVectorizedReaderBatchSize, + returningBatch = returningBatch, + enableRecordFilter = sqlConf.parquetRecordFilterEnabled, + timeZoneId = Some(sqlConf.sessionLocalTimeZone)) } /** @@ -105,7 +104,7 @@ object Spark35HoodieParquetReader { * @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 + * @param props 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, @@ -113,26 +112,27 @@ object Spark35HoodieParquetReader { partitionSchema: StructType, filters: Seq[Filter], sharedConf: Configuration, - extraProps: Map[String, String]): Iterator[InternalRow] = { + props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) ParquetWriteSupport.setSchema(requiredSchema, sharedConf) - val enableVectorizedReader = extraProps("enableVectorizedReader").toBoolean - val datetimeRebaseModeInRead = extraProps("datetimeRebaseModeInRead") - val int96RebaseModeInRead = extraProps("int96RebaseModeInRead") - val enableParquetFilterPushDown = extraProps("enableParquetFilterPushDown").toBoolean - val pushDownDate = extraProps("pushDownDate").toBoolean - val pushDownTimestamp = extraProps("pushDownTimestamp").toBoolean - val pushDownDecimal = extraProps("pushDownDecimal").toBoolean - val pushDownStringPredicate = extraProps("pushDownStringPredicate").toBoolean - val pushDownInFilterThreshold = extraProps("pushDownInFilterThreshold").toInt - val isCaseSensitive = extraProps("isCaseSensitive").toBoolean - val timestampConversion = extraProps("timestampConversion").toBoolean - val enableOffHeapColumnVector = extraProps("enableOffHeapColumnVector").toBoolean - val capacity = extraProps("capacity").toInt - val returningBatch = extraProps("returningBatch").toBoolean - val enableRecordFilter = extraProps("enableRecordFilter").toBoolean - val timeZoneId = Option(extraProps("timeZoneId")) + val properties = props.asInstanceOf[Spark35HoodieParquetReaderProperties] + val enableVectorizedReader = properties.enableVectorizedReader + val datetimeRebaseModeInRead = properties.datetimeRebaseModeInRead + val int96RebaseModeInRead = properties.int96RebaseModeInRead + val enableParquetFilterPushDown = properties.enableParquetFilterPushDown + val pushDownDate = properties.pushDownDate + val pushDownTimestamp = properties.pushDownTimestamp + val pushDownDecimal = properties.pushDownDecimal + val pushDownInFilterThreshold = properties.pushDownInFilterThreshold + val pushDownStringPredicate = properties.pushDownStringPredicate + val isCaseSensitive = properties.isCaseSensitive + val timestampConversion = properties.timestampConversion + val enableOffHeapColumnVector = properties.enableOffHeapColumnVector + val capacity = properties.capacity + val returningBatch = properties.returningBatch + val enableRecordFilter = properties.enableRecordFilter + val timeZoneId = properties.timeZoneId assert(file.partitionValues.numFields == partitionSchema.size) diff --git a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReaderProperties.scala b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReaderProperties.scala new file mode 100644 index 000000000000..edb50211434e --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReaderProperties.scala @@ -0,0 +1,53 @@ +/* + * 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 + +case class Spark35HoodieParquetReaderProperties(enableVectorizedReader: Boolean, + datetimeRebaseModeInRead: String, + int96RebaseModeInRead: String, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringPredicate: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) + extends Spark3HoodieParquetReaderProperties( + enableVectorizedReader = enableVectorizedReader, + datetimeRebaseModeInRead = datetimeRebaseModeInRead, + int96RebaseModeInRead = int96RebaseModeInRead, + enableParquetFilterPushDown = enableParquetFilterPushDown, + pushDownDate = pushDownDate, + pushDownTimestamp = pushDownTimestamp, + pushDownDecimal = pushDownDecimal, + pushDownInFilterThreshold = pushDownInFilterThreshold, + isCaseSensitive = isCaseSensitive, + timestampConversion = timestampConversion, + enableOffHeapColumnVector = enableOffHeapColumnVector, + capacity = capacity, + returningBatch = returningBatch, + enableRecordFilter = enableRecordFilter, + timeZoneId = timeZoneId) From 10a577fe1147d7f312a341cd2a6093eba3aa5224 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 4 Apr 2024 10:32:10 -0400 Subject: [PATCH 12/86] add properties spark 3.1 --- .../SparkHoodieParquetReaderProperties.scala | 2 - .../Spark3HoodieParquetReaderProperties.scala | 28 ++++---- .../spark/sql/adapter/Spark3_1Adapter.scala | 12 ++-- .../parquet/Spark31HoodieParquetReader.scala | 69 +++++++++---------- ...Spark31HoodieParquetReaderProperties.scala | 49 +++++++++++++ 5 files changed, 101 insertions(+), 59 deletions(-) create mode 100644 hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReaderProperties.scala diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderProperties.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderProperties.scala index 1aa237b00166..e05e1692e4c3 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderProperties.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderProperties.scala @@ -20,8 +20,6 @@ package org.apache.spark.sql.execution.datasources.parquet abstract class SparkHoodieParquetReaderProperties(enableVectorizedReader: Boolean, - datetimeRebaseModeInRead: String, - int96RebaseModeInRead: String, enableParquetFilterPushDown: Boolean, pushDownDate: Boolean, pushDownTimestamp: Boolean, diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3HoodieParquetReaderProperties.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3HoodieParquetReaderProperties.scala index f14590d1d028..661c5161e7b3 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3HoodieParquetReaderProperties.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3HoodieParquetReaderProperties.scala @@ -21,24 +21,20 @@ package org.apache.spark.sql.execution.datasources.parquet abstract class Spark3HoodieParquetReaderProperties(enableVectorizedReader: Boolean, - datetimeRebaseModeInRead: String, - int96RebaseModeInRead: String, - enableParquetFilterPushDown: Boolean, - pushDownDate: Boolean, - pushDownTimestamp: Boolean, - pushDownDecimal: Boolean, - pushDownInFilterThreshold: Int, - isCaseSensitive: Boolean, - timestampConversion: Boolean, - enableOffHeapColumnVector: Boolean, - capacity: Int, - returningBatch: Boolean, - enableRecordFilter: Boolean, - timeZoneId: Option[String]) + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) extends SparkHoodieParquetReaderProperties( enableVectorizedReader = enableVectorizedReader, - datetimeRebaseModeInRead = datetimeRebaseModeInRead, - int96RebaseModeInRead = int96RebaseModeInRead, enableParquetFilterPushDown = enableParquetFilterPushDown, pushDownDate = pushDownDate, pushDownTimestamp = pushDownTimestamp, diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala index 827d6755c45c..afba27d06f45 100644 --- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark31HoodieParquetReader, Spark31LegacyHoodieParquetFileFormat} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark31HoodieParquetReader, Spark31LegacyHoodieParquetFileFormat, SparkHoodieParquetReaderProperties} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, HoodieSpark31PartitionedFileUtils, HoodieSparkPartitionedFileUtils, PartitionedFile} import org.apache.spark.sql.hudi.SparkAdapter @@ -130,12 +130,12 @@ class Spark3_1Adapter extends BaseSpark3Adapter { * @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 + * @return properties needed for reading a parquet file */ override def getPropsForReadingParquet(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): Map[String, String] = { + hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { Spark31HoodieParquetReader.getPropsForReadingParquet(vectorized, sqlConf, options, hadoopConf) } @@ -147,7 +147,7 @@ class Spark3_1Adapter extends BaseSpark3Adapter { * @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 + * @param props 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]] */ override def readParquetFile(file: PartitionedFile, @@ -155,8 +155,8 @@ class Spark3_1Adapter extends BaseSpark3Adapter { partitionSchema: StructType, filters: Seq[sources.Filter], sharedConf: Configuration, - extraProps: Map[String, String]): Iterator[InternalRow] = { + props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { Spark31HoodieParquetReader.readParquetFile(file, requiredSchema, partitionSchema, filters, - new Configuration(sharedConf), extraProps) + new Configuration(sharedConf), props) } } diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReader.scala index 241d8d6b7946..dae28da3e7d8 100644 --- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReader.scala @@ -48,12 +48,12 @@ object Spark31HoodieParquetReader { * @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 + * @return properties needed for reading a parquet file */ def getPropsForReadingParquet(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): Map[String, String] = { + hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { //set hadoopconf hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) @@ -61,23 +61,21 @@ object Spark31HoodieParquetReader { 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 - ) + Spark31HoodieParquetReaderProperties( + enableVectorizedReader = vectorized, + enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, + pushDownDate = sqlConf.parquetFilterPushDownDate, + pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp, + pushDownDecimal = sqlConf.parquetFilterPushDownDecimal, + pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold, + pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith, + isCaseSensitive = sqlConf.caseSensitiveAnalysis, + timestampConversion = sqlConf.isParquetINT96TimestampConversion, + enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, + capacity = sqlConf.parquetVectorizedReaderBatchSize, + returningBatch = sqlConf.parquetVectorizedReaderEnabled, + enableRecordFilter = sqlConf.parquetRecordFilterEnabled, + timeZoneId = Some(sqlConf.sessionLocalTimeZone)) } /** @@ -89,7 +87,7 @@ object Spark31HoodieParquetReader { * @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 + * @param props 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, @@ -97,24 +95,25 @@ object Spark31HoodieParquetReader { partitionSchema: StructType, filters: Seq[Filter], sharedConf: Configuration, - extraProps: Map[String, String]): Iterator[InternalRow] = { + props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) ParquetWriteSupport.setSchema(requiredSchema, sharedConf) - val enableVectorizedReader = extraProps("enableVectorizedReader").toBoolean - val enableParquetFilterPushDown = extraProps("enableParquetFilterPushDown").toBoolean - val pushDownDate = extraProps("pushDownDate").toBoolean - val pushDownTimestamp = extraProps("pushDownTimestamp").toBoolean - val pushDownDecimal = extraProps("pushDownDecimal").toBoolean - val pushDownInFilterThreshold = extraProps("pushDownInFilterThreshold").toInt - val pushDownStringStartWith = extraProps("pushDownStringStartWith").toBoolean - val isCaseSensitive = extraProps("isCaseSensitive").toBoolean - val timestampConversion = extraProps("timestampConversion").toBoolean - val enableOffHeapColumnVector = extraProps("enableOffHeapColumnVector").toBoolean - val capacity = extraProps("capacity").toInt - val returningBatch = extraProps("returningBatch").toBoolean - val enableRecordFilter = extraProps("enableRecordFilter").toBoolean - val timeZoneId = Option(extraProps("timeZoneId")) + val properties = props.asInstanceOf[Spark31HoodieParquetReaderProperties] + val enableVectorizedReader = properties.enableVectorizedReader + val enableParquetFilterPushDown = properties.enableParquetFilterPushDown + val pushDownDate = properties.pushDownDate + val pushDownTimestamp = properties.pushDownTimestamp + val pushDownDecimal = properties.pushDownDecimal + val pushDownInFilterThreshold = properties.pushDownInFilterThreshold + val pushDownStringStartWith = properties.pushDownStringStartWith + val isCaseSensitive = properties.isCaseSensitive + val timestampConversion = properties.timestampConversion + val enableOffHeapColumnVector = properties.enableOffHeapColumnVector + val capacity = properties.capacity + val returningBatch = properties.returningBatch + val enableRecordFilter = properties.enableRecordFilter + val timeZoneId = properties.timeZoneId assert(file.partitionValues.numFields == partitionSchema.size) diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReaderProperties.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReaderProperties.scala new file mode 100644 index 000000000000..194800e83a35 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReaderProperties.scala @@ -0,0 +1,49 @@ +/* + * 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 + +case class Spark31HoodieParquetReaderProperties(enableVectorizedReader: Boolean, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringStartWith: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) + extends Spark3HoodieParquetReaderProperties( + enableVectorizedReader = enableVectorizedReader, + enableParquetFilterPushDown = enableParquetFilterPushDown, + pushDownDate = pushDownDate, + pushDownTimestamp = pushDownTimestamp, + pushDownDecimal = pushDownDecimal, + pushDownInFilterThreshold = pushDownInFilterThreshold, + isCaseSensitive = isCaseSensitive, + timestampConversion = timestampConversion, + enableOffHeapColumnVector = enableOffHeapColumnVector, + capacity = capacity, + returningBatch = returningBatch, + enableRecordFilter = enableRecordFilter, + timeZoneId = timeZoneId) From 3c7ecf1e4377ac0a031d73e4becbfb67a848090a Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 4 Apr 2024 10:38:38 -0400 Subject: [PATCH 13/86] add props spark 3.0 --- .../spark/sql/adapter/Spark3_0Adapter.scala | 12 ++-- .../parquet/Spark30HoodieParquetReader.scala | 68 +++++++++---------- ...Spark30HoodieParquetReaderProperties.scala | 49 +++++++++++++ 3 files changed, 89 insertions(+), 40 deletions(-) create mode 100644 hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReaderProperties.scala diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala index 22b94faade94..d2582446f3de 100644 --- a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark30HoodieParquetReader, Spark30LegacyHoodieParquetFileFormat} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark30HoodieParquetReader, Spark30LegacyHoodieParquetFileFormat, SparkHoodieParquetReaderProperties} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, HoodieSpark30PartitionedFileUtils, HoodieSparkPartitionedFileUtils, PartitionedFile} import org.apache.spark.sql.hudi.SparkAdapter @@ -128,12 +128,12 @@ class Spark3_0Adapter extends BaseSpark3Adapter { * @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 + * @return properties needed for reading a parquet file */ override def getPropsForReadingParquet(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): Map[String, String] = { + hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { Spark30HoodieParquetReader.getPropsForReadingParquet(vectorized, sqlConf, options, hadoopConf) } @@ -145,7 +145,7 @@ class Spark3_0Adapter extends BaseSpark3Adapter { * @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 + * @param props 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]] */ override def readParquetFile(file: PartitionedFile, @@ -153,8 +153,8 @@ class Spark3_0Adapter extends BaseSpark3Adapter { partitionSchema: StructType, filters: Seq[sources.Filter], sharedConf: Configuration, - extraProps: Map[String, String]): Iterator[InternalRow] = { + props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { Spark30HoodieParquetReader.readParquetFile(file, requiredSchema, partitionSchema, filters, - new Configuration(sharedConf), extraProps) + new Configuration(sharedConf), props) } } diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala index a29fa123e130..e7a89bcb74b0 100644 --- a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala @@ -48,12 +48,12 @@ object Spark30HoodieParquetReader { * @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 + * @return properties needed for reading a parquet file */ def getPropsForReadingParquet(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): Map[String, String] = { + hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { //set hadoopconf hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) @@ -62,22 +62,21 @@ object Spark30HoodieParquetReader { 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 - ) + Spark30HoodieParquetReaderProperties( + enableVectorizedReader = vectorized, + enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, + pushDownDate = sqlConf.parquetFilterPushDownDate, + pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp, + pushDownDecimal = sqlConf.parquetFilterPushDownDecimal, + pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold, + pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith, + isCaseSensitive = sqlConf.caseSensitiveAnalysis, + timestampConversion = sqlConf.isParquetINT96TimestampConversion, + enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, + capacity = sqlConf.parquetVectorizedReaderBatchSize, + returningBatch = sqlConf.parquetVectorizedReaderEnabled, + enableRecordFilter = sqlConf.parquetRecordFilterEnabled, + timeZoneId = Some(sqlConf.sessionLocalTimeZone)) } /** @@ -89,7 +88,7 @@ object Spark30HoodieParquetReader { * @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 + * @param props 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, @@ -97,24 +96,25 @@ object Spark30HoodieParquetReader { partitionSchema: StructType, filters: Seq[Filter], sharedConf: Configuration, - extraProps: Map[String, String]): Iterator[InternalRow] = { + props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) ParquetWriteSupport.setSchema(requiredSchema, sharedConf) - val enableVectorizedReader = extraProps("enableVectorizedReader").toBoolean - val enableParquetFilterPushDown = extraProps("enableParquetFilterPushDown").toBoolean - val pushDownDate = extraProps("pushDownDate").toBoolean - val pushDownTimestamp = extraProps("pushDownTimestamp").toBoolean - val pushDownDecimal = extraProps("pushDownDecimal").toBoolean - val pushDownInFilterThreshold = extraProps("pushDownInFilterThreshold").toInt - val pushDownStringStartWith = extraProps("pushDownStringStartWith").toBoolean - val isCaseSensitive = extraProps("isCaseSensitive").toBoolean - val timestampConversion = extraProps("timestampConversion").toBoolean - val enableOffHeapColumnVector = extraProps("enableOffHeapColumnVector").toBoolean - val capacity = extraProps("capacity").toInt - val returningBatch = extraProps("returningBatch").toBoolean - val enableRecordFilter = extraProps("enableRecordFilter").toBoolean - val timeZoneId = Option(extraProps("timeZoneId")) + val properties = props.asInstanceOf[Spark30HoodieParquetReaderProperties] + val enableVectorizedReader = properties.enableVectorizedReader + val enableParquetFilterPushDown = properties.enableParquetFilterPushDown + val pushDownDate = properties.pushDownDate + val pushDownTimestamp = properties.pushDownTimestamp + val pushDownDecimal = properties.pushDownDecimal + val pushDownInFilterThreshold = properties.pushDownInFilterThreshold + val pushDownStringStartWith = properties.pushDownStringStartWith + val isCaseSensitive = properties.isCaseSensitive + val timestampConversion = properties.timestampConversion + val enableOffHeapColumnVector = properties.enableOffHeapColumnVector + val capacity = properties.capacity + val returningBatch = properties.returningBatch + val enableRecordFilter = properties.enableRecordFilter + val timeZoneId = properties.timeZoneId assert(file.partitionValues.numFields == partitionSchema.size) diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReaderProperties.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReaderProperties.scala new file mode 100644 index 000000000000..17a4316acf11 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReaderProperties.scala @@ -0,0 +1,49 @@ +/* + * 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 + +case class Spark30HoodieParquetReaderProperties(enableVectorizedReader: Boolean, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringStartWith: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) + extends Spark3HoodieParquetReaderProperties( + enableVectorizedReader = enableVectorizedReader, + enableParquetFilterPushDown = enableParquetFilterPushDown, + pushDownDate = pushDownDate, + pushDownTimestamp = pushDownTimestamp, + pushDownDecimal = pushDownDecimal, + pushDownInFilterThreshold = pushDownInFilterThreshold, + isCaseSensitive = isCaseSensitive, + timestampConversion = timestampConversion, + enableOffHeapColumnVector = enableOffHeapColumnVector, + capacity = capacity, + returningBatch = returningBatch, + enableRecordFilter = enableRecordFilter, + timeZoneId = timeZoneId) From 700013b3ea9d708dfcc2afdbc83611f3d664e292 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 4 Apr 2024 11:23:17 -0400 Subject: [PATCH 14/86] add properties spark 2.4 --- .../SparkHoodieParquetReaderProperties.scala | 29 ++++++++ ...ileGroupReaderBasedParquetFileFormat.scala | 36 ++++++++++ .../spark/sql/adapter/Spark2Adapter.scala | 15 ++-- .../parquet/Spark24HoodieParquetReader.scala | 70 +++++++++---------- ...Spark24HoodieParquetReaderProperties.scala | 50 +++++++++++++ 5 files changed, 157 insertions(+), 43 deletions(-) create mode 100644 hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReaderProperties.scala diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderProperties.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderProperties.scala index e05e1692e4c3..359b230d26f7 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderProperties.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderProperties.scala @@ -32,3 +32,32 @@ abstract class SparkHoodieParquetReaderProperties(enableVectorizedReader: Boolea returningBatch: Boolean, enableRecordFilter: Boolean, timeZoneId: Option[String]) + +abstract class Spark24To31HoodieParquetReaderProperties(enableVectorizedReader: Boolean, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringStartWith: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) + extends SparkHoodieParquetReaderProperties( + enableVectorizedReader = enableVectorizedReader, + enableParquetFilterPushDown = enableParquetFilterPushDown, + pushDownDate = pushDownDate, + pushDownTimestamp = pushDownTimestamp, + pushDownDecimal = pushDownDecimal, + pushDownInFilterThreshold = pushDownInFilterThreshold, + isCaseSensitive = isCaseSensitive, + timestampConversion = timestampConversion, + enableOffHeapColumnVector = enableOffHeapColumnVector, + capacity = capacity, + returningBatch = returningBatch, + enableRecordFilter = enableRecordFilter, + timeZoneId = timeZoneId) \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala index 3de30c770c7a..61448167a5f4 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala @@ -40,6 +40,7 @@ import org.apache.spark.sql.catalyst.expressions.JoinedRow import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.parquet.HoodieFileGroupReaderBasedParquetFileFormat.{ROW_INDEX, ROW_INDEX_TEMPORARY_COLUMN_NAME, getAppliedFilters, getAppliedRequiredSchema, getLogFilesFromSlice, getRecordKeyRelatedFilters, makeCloseableFileGroupMappingRecordIterator} import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isMetaField +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.{LongType, Metadata, MetadataBuilder, StringType, StructField, StructType} import org.apache.spark.util.SerializableConfiguration @@ -57,6 +58,41 @@ trait HoodieFormatTrait { def getRequiredFilters: Seq[Filter] } +trait SparkHoodieParquetReader { + + /** + * 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 properties needed for reading a parquet file + */ + def getPropsForReadingParquet(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkHoodieParquetReaderProperties + + /** + * Read an individual parquet file + * + * @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 props 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, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[Filter], + sharedConf: Configuration, + props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] +} + /** * This class utilizes {@link HoodieFileGroupReader} and its related classes to support reading * from Parquet formatted base files and their log files. diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala index 0f86389150dc..bb771ae03aa3 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.adapter import org.apache.avro.Schema import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileStatus import org.apache.hadoop.fs.Path import org.apache.hudi.client.utils.SparkRowSerDe import org.apache.hudi.common.table.HoodieTableMetaClient @@ -33,11 +32,11 @@ import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.{Command, DeleteFromTable} import org.apache.spark.sql.catalyst.util.DateFormatter import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark24HoodieParquetReader, Spark24LegacyHoodieParquetFileFormat} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark24HoodieParquetReader, Spark24LegacyHoodieParquetFileFormat, SparkHoodieParquetReaderProperties} import org.apache.spark.sql.execution.vectorized.MutableColumnarRow import org.apache.spark.sql.hudi.SparkAdapter -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.hudi.parser.HoodieSpark2ExtendedSqlParser +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.parser.HoodieExtendedParserInterface import org.apache.spark.sql.sources.{BaseRelation, Filter} import org.apache.spark.sql.types.{DataType, Metadata, MetadataBuilder, StructType} @@ -216,12 +215,12 @@ class Spark2Adapter extends SparkAdapter { * @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 + * @return properties needed for reading a parquet file */ override def getPropsForReadingParquet(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): Map[String, String] = { + hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { Spark24HoodieParquetReader.getPropsForReadingParquet(vectorized, sqlConf, options, hadoopConf) } @@ -233,7 +232,7 @@ class Spark2Adapter extends SparkAdapter { * @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 + * @param props 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]] */ override def readParquetFile(file: PartitionedFile, @@ -241,8 +240,8 @@ class Spark2Adapter extends SparkAdapter { partitionSchema: StructType, filters: Seq[sources.Filter], sharedConf: Configuration, - extraProps: Map[String, String]): Iterator[InternalRow] = { + props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { Spark24HoodieParquetReader.readParquetFile(file, requiredSchema, partitionSchema, filters, - new Configuration(sharedConf), extraProps) + new Configuration(sharedConf), props) } } diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala index 45a752562f24..ae5998f20257 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala @@ -42,7 +42,7 @@ import org.apache.spark.util.SerializableConfiguration import java.net.URI -object Spark24HoodieParquetReader { +object Spark24HoodieParquetReader extends SparkHoodieParquetReader { /** * Get properties needed to read a parquet file @@ -51,12 +51,12 @@ object Spark24HoodieParquetReader { * @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 + * @return properties needed for reading a parquet file */ def getPropsForReadingParquet(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): Map[String, String] = { + hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { //set hadoopconf hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) @@ -64,22 +64,21 @@ object Spark24HoodieParquetReader { 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 - ) + Spark24HoodieParquetReaderProperties( + enableVectorizedReader = vectorized, + enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, + pushDownDate = sqlConf.parquetFilterPushDownDate, + pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp, + pushDownDecimal = sqlConf.parquetFilterPushDownDecimal, + pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold, + pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith, + isCaseSensitive = sqlConf.caseSensitiveAnalysis, + timestampConversion = sqlConf.isParquetINT96TimestampConversion, + enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, + capacity = sqlConf.parquetVectorizedReaderBatchSize, + returningBatch = sqlConf.parquetVectorizedReaderEnabled, + enableRecordFilter = sqlConf.parquetRecordFilterEnabled, + timeZoneId = Some(sqlConf.sessionLocalTimeZone)) } /** @@ -91,7 +90,7 @@ object Spark24HoodieParquetReader { * @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 + * @param props 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, @@ -99,24 +98,25 @@ object Spark24HoodieParquetReader { partitionSchema: StructType, filters: Seq[Filter], sharedConf: Configuration, - extraProps: Map[String, String]): Iterator[InternalRow] = { + props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) ParquetWriteSupport.setSchema(requiredSchema, sharedConf) - val enableVectorizedReader = extraProps("enableVectorizedReader").toBoolean - val enableParquetFilterPushDown = extraProps("enableParquetFilterPushDown").toBoolean - val pushDownDate = extraProps("pushDownDate").toBoolean - val pushDownTimestamp = extraProps("pushDownTimestamp").toBoolean - val pushDownDecimal = extraProps("pushDownDecimal").toBoolean - val pushDownInFilterThreshold = extraProps("pushDownInFilterThreshold").toInt - val pushDownStringStartWith = extraProps("pushDownStringStartWith").toBoolean - val isCaseSensitive = extraProps("isCaseSensitive").toBoolean - val timestampConversion = extraProps("timestampConversion").toBoolean - val enableOffHeapColumnVector = extraProps("enableOffHeapColumnVector").toBoolean - val capacity = extraProps("capacity").toInt - val returningBatch = extraProps("returningBatch").toBoolean - val enableRecordFilter = extraProps("enableRecordFilter").toBoolean - val timeZoneId = Option(extraProps("timeZoneId")) + val properties = props.asInstanceOf[Spark24HoodieParquetReaderProperties] + val enableVectorizedReader = properties.enableVectorizedReader + val enableParquetFilterPushDown = properties.enableParquetFilterPushDown + val pushDownDate = properties.pushDownDate + val pushDownTimestamp = properties.pushDownTimestamp + val pushDownDecimal = properties.pushDownDecimal + val pushDownInFilterThreshold = properties.pushDownInFilterThreshold + val pushDownStringStartWith = properties.pushDownStringStartWith + val isCaseSensitive = properties.isCaseSensitive + val timestampConversion = properties.timestampConversion + val enableOffHeapColumnVector = properties.enableOffHeapColumnVector + val capacity = properties.capacity + val returningBatch = properties.returningBatch + val enableRecordFilter = properties.enableRecordFilter + val timeZoneId = properties.timeZoneId assert(file.partitionValues.numFields == partitionSchema.size) diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReaderProperties.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReaderProperties.scala new file mode 100644 index 000000000000..f83544a781a7 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReaderProperties.scala @@ -0,0 +1,50 @@ +/* + * 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 + +case class Spark24HoodieParquetReaderProperties(enableVectorizedReader: Boolean, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringStartWith: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) + extends Spark24To31HoodieParquetReaderProperties( + enableVectorizedReader = enableVectorizedReader, + enableParquetFilterPushDown = enableParquetFilterPushDown, + pushDownDate = pushDownDate, + pushDownTimestamp = pushDownTimestamp, + pushDownDecimal = pushDownDecimal, + pushDownInFilterThreshold = pushDownInFilterThreshold, + pushDownStringStartWith = pushDownStringStartWith, + isCaseSensitive = isCaseSensitive, + timestampConversion = timestampConversion, + enableOffHeapColumnVector = enableOffHeapColumnVector, + capacity = capacity, + returningBatch = returningBatch, + enableRecordFilter = enableRecordFilter, + timeZoneId = timeZoneId) From 37f52eb5ef004d036cfbf59ebeab0986610536fd Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 4 Apr 2024 11:25:55 -0400 Subject: [PATCH 15/86] fix 3.0 --- .../parquet/Spark30HoodieParquetReader.scala | 2 +- ...Spark30HoodieParquetReaderProperties.scala | 29 ++++++++++--------- 2 files changed, 16 insertions(+), 15 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala index e7a89bcb74b0..a8c4c9e2842b 100644 --- a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.types._ import java.net.URI -object Spark30HoodieParquetReader { +object Spark30HoodieParquetReader extends SparkHoodieParquetReader { /** * Get properties needed to read a parquet file diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReaderProperties.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReaderProperties.scala index 17a4316acf11..9870bfa193dd 100644 --- a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReaderProperties.scala +++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReaderProperties.scala @@ -33,17 +33,18 @@ case class Spark30HoodieParquetReaderProperties(enableVectorizedReader: Boolean, returningBatch: Boolean, enableRecordFilter: Boolean, timeZoneId: Option[String]) - extends Spark3HoodieParquetReaderProperties( - enableVectorizedReader = enableVectorizedReader, - enableParquetFilterPushDown = enableParquetFilterPushDown, - pushDownDate = pushDownDate, - pushDownTimestamp = pushDownTimestamp, - pushDownDecimal = pushDownDecimal, - pushDownInFilterThreshold = pushDownInFilterThreshold, - isCaseSensitive = isCaseSensitive, - timestampConversion = timestampConversion, - enableOffHeapColumnVector = enableOffHeapColumnVector, - capacity = capacity, - returningBatch = returningBatch, - enableRecordFilter = enableRecordFilter, - timeZoneId = timeZoneId) + extends Spark24To31HoodieParquetReaderProperties( + enableVectorizedReader = enableVectorizedReader, + enableParquetFilterPushDown = enableParquetFilterPushDown, + pushDownDate = pushDownDate, + pushDownTimestamp = pushDownTimestamp, + pushDownDecimal = pushDownDecimal, + pushDownInFilterThreshold = pushDownInFilterThreshold, + pushDownStringStartWith = pushDownStringStartWith, + isCaseSensitive = isCaseSensitive, + timestampConversion = timestampConversion, + enableOffHeapColumnVector = enableOffHeapColumnVector, + capacity = capacity, + returningBatch = returningBatch, + enableRecordFilter = enableRecordFilter, + timeZoneId = timeZoneId) From b9c159244cb7c6c8399ddcb95f89af9d8eab22f4 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 4 Apr 2024 12:40:58 -0400 Subject: [PATCH 16/86] refactor to get rid of properties, spark 3.1 --- .../parquet/SparkHoodieParquetReader.scala | 46 ++++++ .../apache/spark/sql/hudi/SparkAdapter.scala | 32 +--- ...ileGroupReaderBasedParquetFileFormat.scala | 35 ----- .../SparkHoodieParquetReaderBase.scala | 98 +++++++++++++ .../spark/sql/adapter/Spark3_1Adapter.scala | 37 +---- .../parquet/Spark31HoodieParquetReader.scala | 137 ++++++++++-------- ...Spark31HoodieParquetReaderProperties.scala | 49 ------- 7 files changed, 232 insertions(+), 202 deletions(-) create mode 100644 hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReader.scala create mode 100644 hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala delete mode 100644 hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReaderProperties.scala diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReader.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReader.scala new file mode 100644 index 000000000000..7eff545136fd --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReader.scala @@ -0,0 +1,46 @@ +/* + * 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.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.StructType + +trait SparkHoodieParquetReader extends Serializable { + + /** + * Read an individual parquet file + * + * @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 + * @return iterator of rows read from the file output type says [[InternalRow]] but could be [[ColumnarBatch]] + */ + def read(file: PartitionedFile, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[Filter], + sharedConf: Configuration): Iterator[InternalRow] + +} diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala index b2c6ebe97fe4..f87d229f4b45 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} import org.apache.spark.sql.catalyst.util.DateFormatter import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, SparkHoodieParquetReaderProperties} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, SparkHoodieParquetReader, SparkHoodieParquetReaderProperties} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.parser.HoodieExtendedParserInterface import org.apache.spark.sql.sources.{BaseRelation, Filter} @@ -218,34 +218,16 @@ trait SparkAdapter extends Serializable { def translateFilter(predicate: Expression, supportNestedPredicatePushdown: Boolean = false): Option[Filter] /** - * Get properties needed to read a parquet file + * Get parquet file reader * * @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 properties needed for reading a parquet file + * @return parquet file reader */ - def getPropsForReadingParquet(vectorized: Boolean, - sqlConf: SQLConf, - options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReaderProperties - - /** - * Read an individual parquet file - * - * @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 props 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, - requiredSchema: StructType, - partitionSchema: StructType, - filters: Seq[Filter], - sharedConf: Configuration, - props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] + def createHoodieParquetFileReader(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkHoodieParquetReader } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala index 61448167a5f4..ec7d75a5624a 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala @@ -58,41 +58,6 @@ trait HoodieFormatTrait { def getRequiredFilters: Seq[Filter] } -trait SparkHoodieParquetReader { - - /** - * 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 properties needed for reading a parquet file - */ - def getPropsForReadingParquet(vectorized: Boolean, - sqlConf: SQLConf, - options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReaderProperties - - /** - * Read an individual parquet file - * - * @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 props 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, - requiredSchema: StructType, - partitionSchema: StructType, - filters: Seq[Filter], - sharedConf: Configuration, - props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] -} - /** * This class utilizes {@link HoodieFileGroupReader} and its related classes to support reading * from Parquet formatted base files and their log files. diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala new file mode 100644 index 000000000000..382c700595e3 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala @@ -0,0 +1,98 @@ +/* + * 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.spark.sql.catalyst.InternalRow +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 + +abstract class SparkHoodieParquetReaderBase(enableVectorizedReader: Boolean, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) extends SparkHoodieParquetReader { + + /** + * Read an individual parquet file + * + * @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 + * @return iterator of rows read from the file output type says [[InternalRow]] but could be [[ColumnarBatch]] + */ + final def read(file: PartitionedFile, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[Filter], + sharedConf: Configuration): Iterator[InternalRow] = { + sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) + sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) + ParquetWriteSupport.setSchema(requiredSchema, sharedConf) + doRead(file, requiredSchema, partitionSchema, filters, sharedConf) + } + + + /** + * Implemented for each spark version + * + * @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 + * @return iterator of rows read from the file output type says [[InternalRow]] but could be [[ColumnarBatch]] + */ + protected def doRead(file: PartitionedFile, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[Filter], + sharedConf: Configuration): Iterator[InternalRow] + +} + +trait SparkHoodieParquetReaderBuilder { + /** + * Get parquet file reader + * + * @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 properties needed for reading a parquet file + */ + + def build(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkHoodieParquetReader +} diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala index afba27d06f45..d330886d3791 100644 --- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark31HoodieParquetReader, Spark31LegacyHoodieParquetFileFormat, SparkHoodieParquetReaderProperties} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark31HoodieParquetReader, Spark31LegacyHoodieParquetFileFormat, SparkHoodieParquetReader} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, HoodieSpark31PartitionedFileUtils, HoodieSparkPartitionedFileUtils, PartitionedFile} import org.apache.spark.sql.hudi.SparkAdapter @@ -124,39 +124,18 @@ class Spark3_1Adapter extends BaseSpark3Adapter { } /** - * Get properties needed to read a parquet file + * Get parquet file reader * * @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 properties needed for reading a parquet file + * @return parquet file reader */ - override def getPropsForReadingParquet(vectorized: Boolean, - sqlConf: SQLConf, - options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { - Spark31HoodieParquetReader.getPropsForReadingParquet(vectorized, sqlConf, options, hadoopConf) - } - - /** - * Read an individual parquet file - * - * @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 props 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]] - */ - override def readParquetFile(file: PartitionedFile, - requiredSchema: StructType, - partitionSchema: StructType, - filters: Seq[sources.Filter], - sharedConf: Configuration, - props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { - Spark31HoodieParquetReader.readParquetFile(file, requiredSchema, partitionSchema, filters, - new Configuration(sharedConf), props) + override def createHoodieParquetFileReader(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkHoodieParquetReader = { + Spark31HoodieParquetReader.build(vectorized, sqlConf, options, hadoopConf) } } diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReader.scala index dae28da3e7d8..5c1bb612f7db 100644 --- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReader.scala @@ -39,44 +39,33 @@ import org.apache.spark.sql.types._ import java.net.URI -object Spark31HoodieParquetReader { - - /** - * 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 properties needed for reading a parquet file - */ - def getPropsForReadingParquet(vectorized: Boolean, - sqlConf: SQLConf, - options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { - //set hadoopconf - hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) - hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) - hadoopConf.setBoolean(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key, sqlConf.nestedSchemaPruningEnabled) - 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) - Spark31HoodieParquetReaderProperties( - enableVectorizedReader = vectorized, - enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, - pushDownDate = sqlConf.parquetFilterPushDownDate, - pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp, - pushDownDecimal = sqlConf.parquetFilterPushDownDecimal, - pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold, - pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith, - isCaseSensitive = sqlConf.caseSensitiveAnalysis, - timestampConversion = sqlConf.isParquetINT96TimestampConversion, - enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, - capacity = sqlConf.parquetVectorizedReaderBatchSize, - returningBatch = sqlConf.parquetVectorizedReaderEnabled, - enableRecordFilter = sqlConf.parquetRecordFilterEnabled, - timeZoneId = Some(sqlConf.sessionLocalTimeZone)) - } +class Spark31HoodieParquetReader(enableVectorizedReader: Boolean, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringStartWith: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) extends SparkHoodieParquetReaderBase( + enableVectorizedReader = enableVectorizedReader, + enableParquetFilterPushDown = enableParquetFilterPushDown, + pushDownDate = pushDownDate, + pushDownTimestamp = pushDownTimestamp, + pushDownDecimal = pushDownDecimal, + pushDownInFilterThreshold = pushDownInFilterThreshold, + isCaseSensitive = isCaseSensitive, + timestampConversion = timestampConversion, + enableOffHeapColumnVector = enableOffHeapColumnVector, + capacity = capacity, + returningBatch = returningBatch, + enableRecordFilter = enableRecordFilter, + timeZoneId = timeZoneId) { /** * Read an individual parquet file @@ -87,34 +76,13 @@ object Spark31HoodieParquetReader { * @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 props 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, - requiredSchema: StructType, - partitionSchema: StructType, - filters: Seq[Filter], - sharedConf: Configuration, - props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { - sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) - sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) - ParquetWriteSupport.setSchema(requiredSchema, sharedConf) - val properties = props.asInstanceOf[Spark31HoodieParquetReaderProperties] - val enableVectorizedReader = properties.enableVectorizedReader - val enableParquetFilterPushDown = properties.enableParquetFilterPushDown - val pushDownDate = properties.pushDownDate - val pushDownTimestamp = properties.pushDownTimestamp - val pushDownDecimal = properties.pushDownDecimal - val pushDownInFilterThreshold = properties.pushDownInFilterThreshold - val pushDownStringStartWith = properties.pushDownStringStartWith - val isCaseSensitive = properties.isCaseSensitive - val timestampConversion = properties.timestampConversion - val enableOffHeapColumnVector = properties.enableOffHeapColumnVector - val capacity = properties.capacity - val returningBatch = properties.returningBatch - val enableRecordFilter = properties.enableRecordFilter - val timeZoneId = properties.timeZoneId - + protected def doRead(file: PartitionedFile, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[Filter], + sharedConf: Configuration): Iterator[InternalRow] = { assert(file.partitionValues.numFields == partitionSchema.size) val filePath = new Path(new URI(file.filePath)) @@ -232,3 +200,44 @@ object Spark31HoodieParquetReader { } } } + +object Spark31HoodieParquetReader extends SparkHoodieParquetReaderBuilder { + + /** + * Get parquet file reader + * + * @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 parquet file reader + */ + def build(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkHoodieParquetReader = { + //set hadoopconf + hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) + hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) + hadoopConf.setBoolean(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key, sqlConf.nestedSchemaPruningEnabled) + 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) + new Spark31HoodieParquetReader( + enableVectorizedReader = vectorized, + enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, + pushDownDate = sqlConf.parquetFilterPushDownDate, + pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp, + pushDownDecimal = sqlConf.parquetFilterPushDownDecimal, + pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold, + pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith, + isCaseSensitive = sqlConf.caseSensitiveAnalysis, + timestampConversion = sqlConf.isParquetINT96TimestampConversion, + enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, + capacity = sqlConf.parquetVectorizedReaderBatchSize, + returningBatch = sqlConf.parquetVectorizedReaderEnabled, + enableRecordFilter = sqlConf.parquetRecordFilterEnabled, + timeZoneId = Some(sqlConf.sessionLocalTimeZone)) + } + +} diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReaderProperties.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReaderProperties.scala deleted file mode 100644 index 194800e83a35..000000000000 --- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReaderProperties.scala +++ /dev/null @@ -1,49 +0,0 @@ -/* - * 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 - -case class Spark31HoodieParquetReaderProperties(enableVectorizedReader: Boolean, - enableParquetFilterPushDown: Boolean, - pushDownDate: Boolean, - pushDownTimestamp: Boolean, - pushDownDecimal: Boolean, - pushDownInFilterThreshold: Int, - pushDownStringStartWith: Boolean, - isCaseSensitive: Boolean, - timestampConversion: Boolean, - enableOffHeapColumnVector: Boolean, - capacity: Int, - returningBatch: Boolean, - enableRecordFilter: Boolean, - timeZoneId: Option[String]) - extends Spark3HoodieParquetReaderProperties( - enableVectorizedReader = enableVectorizedReader, - enableParquetFilterPushDown = enableParquetFilterPushDown, - pushDownDate = pushDownDate, - pushDownTimestamp = pushDownTimestamp, - pushDownDecimal = pushDownDecimal, - pushDownInFilterThreshold = pushDownInFilterThreshold, - isCaseSensitive = isCaseSensitive, - timestampConversion = timestampConversion, - enableOffHeapColumnVector = enableOffHeapColumnVector, - capacity = capacity, - returningBatch = returningBatch, - enableRecordFilter = enableRecordFilter, - timeZoneId = timeZoneId) From e3957c5f44051c1463d4d901f16fd9a7afc2276a Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 4 Apr 2024 12:48:56 -0400 Subject: [PATCH 17/86] remove props spark 3.0 --- .../spark/sql/adapter/Spark3_0Adapter.scala | 37 +---- .../parquet/Spark30HoodieParquetReader.scala | 137 +++++++++--------- ...Spark30HoodieParquetReaderProperties.scala | 50 ------- 3 files changed, 80 insertions(+), 144 deletions(-) delete mode 100644 hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReaderProperties.scala diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala index d2582446f3de..2a472c44f57d 100644 --- a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark30HoodieParquetReader, Spark30LegacyHoodieParquetFileFormat, SparkHoodieParquetReaderProperties} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark30HoodieParquetReader, Spark30LegacyHoodieParquetFileFormat, SparkHoodieParquetReader} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, HoodieSpark30PartitionedFileUtils, HoodieSparkPartitionedFileUtils, PartitionedFile} import org.apache.spark.sql.hudi.SparkAdapter @@ -122,39 +122,18 @@ class Spark3_0Adapter extends BaseSpark3Adapter { } /** - * Get properties needed to read a parquet file + * Get parquet file reader * * @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 properties needed for reading a parquet file + * @return parquet file reader */ - override def getPropsForReadingParquet(vectorized: Boolean, - sqlConf: SQLConf, - options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { - Spark30HoodieParquetReader.getPropsForReadingParquet(vectorized, sqlConf, options, hadoopConf) - } - - /** - * Read an individual parquet file - * - * @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 props 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]] - */ - override def readParquetFile(file: PartitionedFile, - requiredSchema: StructType, - partitionSchema: StructType, - filters: Seq[sources.Filter], - sharedConf: Configuration, - props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { - Spark30HoodieParquetReader.readParquetFile(file, requiredSchema, partitionSchema, filters, - new Configuration(sharedConf), props) + override def createHoodieParquetFileReader(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkHoodieParquetReader = { + Spark30HoodieParquetReader.build(vectorized, sqlConf, options, hadoopConf) } } diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala index a8c4c9e2842b..3d950fb5cbdd 100644 --- a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala @@ -39,45 +39,33 @@ import org.apache.spark.sql.types._ import java.net.URI -object Spark30HoodieParquetReader extends SparkHoodieParquetReader { - - /** - * 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 properties needed for reading a parquet file - */ - def getPropsForReadingParquet(vectorized: Boolean, - sqlConf: SQLConf, - options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { - //set hadoopconf - hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) - hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) - hadoopConf.setBoolean(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key, sqlConf.nestedSchemaPruningEnabled) - 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) - - Spark30HoodieParquetReaderProperties( - enableVectorizedReader = vectorized, - enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, - pushDownDate = sqlConf.parquetFilterPushDownDate, - pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp, - pushDownDecimal = sqlConf.parquetFilterPushDownDecimal, - pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold, - pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith, - isCaseSensitive = sqlConf.caseSensitiveAnalysis, - timestampConversion = sqlConf.isParquetINT96TimestampConversion, - enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, - capacity = sqlConf.parquetVectorizedReaderBatchSize, - returningBatch = sqlConf.parquetVectorizedReaderEnabled, - enableRecordFilter = sqlConf.parquetRecordFilterEnabled, - timeZoneId = Some(sqlConf.sessionLocalTimeZone)) - } +class Spark30HoodieParquetReader(enableVectorizedReader: Boolean, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringStartWith: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) extends SparkHoodieParquetReaderBase( + enableVectorizedReader = enableVectorizedReader, + enableParquetFilterPushDown = enableParquetFilterPushDown, + pushDownDate = pushDownDate, + pushDownTimestamp = pushDownTimestamp, + pushDownDecimal = pushDownDecimal, + pushDownInFilterThreshold = pushDownInFilterThreshold, + isCaseSensitive = isCaseSensitive, + timestampConversion = timestampConversion, + enableOffHeapColumnVector = enableOffHeapColumnVector, + capacity = capacity, + returningBatch = returningBatch, + enableRecordFilter = enableRecordFilter, + timeZoneId = timeZoneId) { /** * Read an individual parquet file @@ -88,34 +76,13 @@ object Spark30HoodieParquetReader extends SparkHoodieParquetReader { * @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 props 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, - requiredSchema: StructType, - partitionSchema: StructType, - filters: Seq[Filter], - sharedConf: Configuration, - props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { - sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) - sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) - ParquetWriteSupport.setSchema(requiredSchema, sharedConf) - val properties = props.asInstanceOf[Spark30HoodieParquetReaderProperties] - val enableVectorizedReader = properties.enableVectorizedReader - val enableParquetFilterPushDown = properties.enableParquetFilterPushDown - val pushDownDate = properties.pushDownDate - val pushDownTimestamp = properties.pushDownTimestamp - val pushDownDecimal = properties.pushDownDecimal - val pushDownInFilterThreshold = properties.pushDownInFilterThreshold - val pushDownStringStartWith = properties.pushDownStringStartWith - val isCaseSensitive = properties.isCaseSensitive - val timestampConversion = properties.timestampConversion - val enableOffHeapColumnVector = properties.enableOffHeapColumnVector - val capacity = properties.capacity - val returningBatch = properties.returningBatch - val enableRecordFilter = properties.enableRecordFilter - val timeZoneId = properties.timeZoneId - + protected def doRead(file: PartitionedFile, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[Filter], + sharedConf: Configuration): Iterator[InternalRow] = { assert(file.partitionValues.numFields == partitionSchema.size) val filePath = new Path(new URI(file.filePath)) @@ -220,3 +187,43 @@ object Spark30HoodieParquetReader extends SparkHoodieParquetReader { } } + +object Spark30HoodieParquetReader extends SparkHoodieParquetReaderBuilder { + /** + * Get parquet file reader + * + * @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 parquet file reader + */ + def build(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkHoodieParquetReader = { + //set hadoopconf + hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) + hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) + hadoopConf.setBoolean(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key, sqlConf.nestedSchemaPruningEnabled) + 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) + + new Spark30HoodieParquetReader( + enableVectorizedReader = vectorized, + enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, + pushDownDate = sqlConf.parquetFilterPushDownDate, + pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp, + pushDownDecimal = sqlConf.parquetFilterPushDownDecimal, + pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold, + pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith, + isCaseSensitive = sqlConf.caseSensitiveAnalysis, + timestampConversion = sqlConf.isParquetINT96TimestampConversion, + enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, + capacity = sqlConf.parquetVectorizedReaderBatchSize, + returningBatch = sqlConf.parquetVectorizedReaderEnabled, + enableRecordFilter = sqlConf.parquetRecordFilterEnabled, + timeZoneId = Some(sqlConf.sessionLocalTimeZone)) + } +} diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReaderProperties.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReaderProperties.scala deleted file mode 100644 index 9870bfa193dd..000000000000 --- a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReaderProperties.scala +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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 - -case class Spark30HoodieParquetReaderProperties(enableVectorizedReader: Boolean, - enableParquetFilterPushDown: Boolean, - pushDownDate: Boolean, - pushDownTimestamp: Boolean, - pushDownDecimal: Boolean, - pushDownInFilterThreshold: Int, - pushDownStringStartWith: Boolean, - isCaseSensitive: Boolean, - timestampConversion: Boolean, - enableOffHeapColumnVector: Boolean, - capacity: Int, - returningBatch: Boolean, - enableRecordFilter: Boolean, - timeZoneId: Option[String]) - extends Spark24To31HoodieParquetReaderProperties( - enableVectorizedReader = enableVectorizedReader, - enableParquetFilterPushDown = enableParquetFilterPushDown, - pushDownDate = pushDownDate, - pushDownTimestamp = pushDownTimestamp, - pushDownDecimal = pushDownDecimal, - pushDownInFilterThreshold = pushDownInFilterThreshold, - pushDownStringStartWith = pushDownStringStartWith, - isCaseSensitive = isCaseSensitive, - timestampConversion = timestampConversion, - enableOffHeapColumnVector = enableOffHeapColumnVector, - capacity = capacity, - returningBatch = returningBatch, - enableRecordFilter = enableRecordFilter, - timeZoneId = timeZoneId) From 7345f6bdbcc8a1e1524b98273a4be98aa452d5d5 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 4 Apr 2024 13:22:04 -0400 Subject: [PATCH 18/86] use class model for spark 3.3 --- .../spark/sql/adapter/Spark3_2Adapter.scala | 37 +---- .../parquet/Spark32HoodieParquetReader.scala | 150 +++++++++--------- ...Spark32HoodieParquetReaderProperties.scala | 54 ------- 3 files changed, 87 insertions(+), 154 deletions(-) delete mode 100644 hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReaderProperties.scala diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala index 6215594a8493..a584946e8a08 100644 --- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{Command, DeleteFromTable, LogicalPlan} import org.apache.spark.sql.catalyst.util.METADATA_COL_ATTR_KEY import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark32HoodieParquetReader, Spark32LegacyHoodieParquetFileFormat, SparkHoodieParquetReaderProperties} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark32HoodieParquetReader, Spark32LegacyHoodieParquetFileFormat, SparkHoodieParquetReader} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hudi.analysis.TableValuedFunctions @@ -127,39 +127,18 @@ class Spark3_2Adapter extends BaseSpark3Adapter { } /** - * Get properties needed to read a parquet file + * Get parquet file reader * * @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 + * @return parquet file reader */ - override def getPropsForReadingParquet(vectorized: Boolean, - sqlConf: SQLConf, - options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { - Spark32HoodieParquetReader.getPropsForReadingParquet(vectorized, sqlConf, options, hadoopConf) - } - - /** - * Read an individual parquet file - * - * @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]] - */ - override def readParquetFile(file: PartitionedFile, - requiredSchema: StructType, - partitionSchema: StructType, - filters: Seq[sources.Filter], - sharedConf: Configuration, - props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { - Spark32HoodieParquetReader.readParquetFile(file, requiredSchema, partitionSchema, filters, - new Configuration(sharedConf), props) + override def createHoodieParquetFileReader(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkHoodieParquetReader = { + Spark32HoodieParquetReader.build(vectorized, sqlConf, options, hadoopConf) } } diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReader.scala index bb32480cc622..8db3319273b1 100644 --- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReader.scala @@ -40,52 +40,35 @@ import org.apache.spark.sql.types._ import java.net.URI -object Spark32HoodieParquetReader { - - /** - * 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): SparkHoodieParquetReaderProperties = { - //set hadoopconf - hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) - hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) - hadoopConf.setBoolean(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key, sqlConf.nestedSchemaPruningEnabled) - 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) - // Using string value of this conf to preserve compatibility across spark versions. See [HUDI-5868] - hadoopConf.setBoolean( - "spark.sql.legacy.parquet.nanosAsLong", - sqlConf.getConfString("spark.sql.legacy.parquet.nanosAsLong", "false").toBoolean - ) - - val parquetOptions = new ParquetOptions(options, sqlConf) - Spark32HoodieParquetReaderProperties(enableVectorizedReader = vectorized, - datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead, - int96RebaseModeInRead = parquetOptions.int96RebaseModeInRead, - enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, - pushDownDate = sqlConf.parquetFilterPushDownDate, - pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp, - pushDownDecimal = sqlConf.parquetFilterPushDownDecimal, - pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold, - pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith, - isCaseSensitive = sqlConf.caseSensitiveAnalysis, - timestampConversion = sqlConf.isParquetINT96TimestampConversion, - enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, - capacity = sqlConf.parquetVectorizedReaderBatchSize, - returningBatch = sqlConf.parquetVectorizedReaderEnabled, - enableRecordFilter = sqlConf.parquetRecordFilterEnabled, - timeZoneId = Some(sqlConf.sessionLocalTimeZone)) - } +class Spark32HoodieParquetReader(enableVectorizedReader: Boolean, + datetimeRebaseModeInRead: String, + int96RebaseModeInRead: String, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringStartWith: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) extends SparkHoodieParquetReaderBase( + enableVectorizedReader = enableVectorizedReader, + enableParquetFilterPushDown = enableParquetFilterPushDown, + pushDownDate = pushDownDate, + pushDownTimestamp = pushDownTimestamp, + pushDownDecimal = pushDownDecimal, + pushDownInFilterThreshold = pushDownInFilterThreshold, + isCaseSensitive = isCaseSensitive, + timestampConversion = timestampConversion, + enableOffHeapColumnVector = enableOffHeapColumnVector, + capacity = capacity, + returningBatch = returningBatch, + enableRecordFilter = enableRecordFilter, + timeZoneId = timeZoneId) { /** * Read an individual parquet file @@ -96,36 +79,13 @@ object Spark32HoodieParquetReader { * @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, + protected def doRead(file: PartitionedFile, requiredSchema: StructType, partitionSchema: StructType, filters: Seq[Filter], - sharedConf: Configuration, - props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { - sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) - sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) - ParquetWriteSupport.setSchema(requiredSchema, sharedConf) - val properties = props.asInstanceOf[Spark32HoodieParquetReaderProperties] - val enableVectorizedReader = properties.enableVectorizedReader - val datetimeRebaseModeInRead = properties.datetimeRebaseModeInRead - val int96RebaseModeInRead = properties.int96RebaseModeInRead - val enableParquetFilterPushDown = properties.enableParquetFilterPushDown - val pushDownDate = properties.pushDownDate - val pushDownTimestamp = properties.pushDownTimestamp - val pushDownDecimal = properties.pushDownDecimal - val pushDownInFilterThreshold = properties.pushDownInFilterThreshold - val pushDownStringStartWith = properties.pushDownStringStartWith - val isCaseSensitive = properties.isCaseSensitive - val timestampConversion = properties.timestampConversion - val enableOffHeapColumnVector = properties.enableOffHeapColumnVector - val capacity = properties.capacity - val returningBatch = properties.returningBatch - val enableRecordFilter = properties.enableRecordFilter - val timeZoneId = properties.timeZoneId - + sharedConf: Configuration): Iterator[InternalRow] = { assert(file.partitionValues.numFields == partitionSchema.size) val filePath = new Path(new URI(file.filePath)) @@ -257,3 +217,51 @@ object Spark32HoodieParquetReader { } } } + +object Spark32HoodieParquetReader extends SparkHoodieParquetReaderBuilder { + /** + * Get parquet file reader + * + * @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 parquet file reader + */ + def build(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkHoodieParquetReader = { + //set hadoopconf + hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) + hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) + hadoopConf.setBoolean(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key, sqlConf.nestedSchemaPruningEnabled) + 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) + // Using string value of this conf to preserve compatibility across spark versions. See [HUDI-5868] + hadoopConf.setBoolean( + "spark.sql.legacy.parquet.nanosAsLong", + sqlConf.getConfString("spark.sql.legacy.parquet.nanosAsLong", "false").toBoolean + ) + + val parquetOptions = new ParquetOptions(options, sqlConf) + new Spark32HoodieParquetReader( + enableVectorizedReader = vectorized, + datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead, + int96RebaseModeInRead = parquetOptions.int96RebaseModeInRead, + enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, + pushDownDate = sqlConf.parquetFilterPushDownDate, + pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp, + pushDownDecimal = sqlConf.parquetFilterPushDownDecimal, + pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold, + pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith, + isCaseSensitive = sqlConf.caseSensitiveAnalysis, + timestampConversion = sqlConf.isParquetINT96TimestampConversion, + enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, + capacity = sqlConf.parquetVectorizedReaderBatchSize, + returningBatch = sqlConf.parquetVectorizedReaderEnabled, + enableRecordFilter = sqlConf.parquetRecordFilterEnabled, + timeZoneId = Some(sqlConf.sessionLocalTimeZone)) + } +} diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReaderProperties.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReaderProperties.scala deleted file mode 100644 index b2cbcfa06a5e..000000000000 --- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReaderProperties.scala +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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 - -case class Spark32HoodieParquetReaderProperties(enableVectorizedReader: Boolean, - datetimeRebaseModeInRead: String, - int96RebaseModeInRead: String, - enableParquetFilterPushDown: Boolean, - pushDownDate: Boolean, - pushDownTimestamp: Boolean, - pushDownDecimal: Boolean, - pushDownInFilterThreshold: Int, - pushDownStringStartWith: Boolean, - isCaseSensitive: Boolean, - timestampConversion: Boolean, - enableOffHeapColumnVector: Boolean, - capacity: Int, - returningBatch: Boolean, - enableRecordFilter: Boolean, - timeZoneId: Option[String]) - extends Spark3HoodieParquetReaderProperties( - enableVectorizedReader = enableVectorizedReader, - datetimeRebaseModeInRead = datetimeRebaseModeInRead, - int96RebaseModeInRead = int96RebaseModeInRead, - enableParquetFilterPushDown = enableParquetFilterPushDown, - pushDownDate = pushDownDate, - pushDownTimestamp = pushDownTimestamp, - pushDownDecimal = pushDownDecimal, - pushDownInFilterThreshold = pushDownInFilterThreshold, - pushDownStringStartWith = pushDownStringStartWith, - isCaseSensitive = isCaseSensitive, - timestampConversion = timestampConversion, - enableOffHeapColumnVector = enableOffHeapColumnVector, - capacity = capacity, - returningBatch = returningBatch, - enableRecordFilter = enableRecordFilter, - timeZoneId = timeZoneId) From 2942a6cdcae907506f2ab7599bb2581fd8ab0f04 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 4 Apr 2024 13:33:08 -0400 Subject: [PATCH 19/86] remove props spark 3.3 --- .../Spark3HoodieParquetReaderProperties.scala | 49 ------ .../spark/sql/adapter/Spark3_3Adapter.scala | 38 +---- .../parquet/Spark33HoodieParquetReader.scala | 149 +++++++++--------- ...Spark33HoodieParquetReaderProperties.scala | 54 ------- 4 files changed, 86 insertions(+), 204 deletions(-) delete mode 100644 hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3HoodieParquetReaderProperties.scala delete mode 100644 hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReaderProperties.scala diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3HoodieParquetReaderProperties.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3HoodieParquetReaderProperties.scala deleted file mode 100644 index 661c5161e7b3..000000000000 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3HoodieParquetReaderProperties.scala +++ /dev/null @@ -1,49 +0,0 @@ -/* - * 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 - - -abstract class Spark3HoodieParquetReaderProperties(enableVectorizedReader: Boolean, - enableParquetFilterPushDown: Boolean, - pushDownDate: Boolean, - pushDownTimestamp: Boolean, - pushDownDecimal: Boolean, - pushDownInFilterThreshold: Int, - isCaseSensitive: Boolean, - timestampConversion: Boolean, - enableOffHeapColumnVector: Boolean, - capacity: Int, - returningBatch: Boolean, - enableRecordFilter: Boolean, - timeZoneId: Option[String]) - extends SparkHoodieParquetReaderProperties( - enableVectorizedReader = enableVectorizedReader, - enableParquetFilterPushDown = enableParquetFilterPushDown, - pushDownDate = pushDownDate, - pushDownTimestamp = pushDownTimestamp, - pushDownDecimal = pushDownDecimal, - pushDownInFilterThreshold = pushDownInFilterThreshold, - isCaseSensitive = isCaseSensitive, - timestampConversion = timestampConversion, - enableOffHeapColumnVector = enableOffHeapColumnVector, - capacity = capacity, - returningBatch = returningBatch, - enableRecordFilter = enableRecordFilter, - timeZoneId = timeZoneId) diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala index 6edd158dfbc2..1f0ad5f32aed 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala @@ -31,11 +31,10 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.METADATA_COL_ATTR_KEY import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark33HoodieParquetReader, Spark33LegacyHoodieParquetFileFormat} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark33HoodieParquetReader, Spark33LegacyHoodieParquetFileFormat, SparkHoodieParquetReader} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hudi.analysis.TableValuedFunctions -import org.apache.spark.sql.hudi.execution.datasources.parquet.SparkHoodieParquetReaderProperties import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.parser.{HoodieExtendedParserInterface, HoodieSpark3_3ExtendedSqlParser} import org.apache.spark.sql.types.{DataType, Metadata, MetadataBuilder, StructType} @@ -129,39 +128,18 @@ class Spark3_3Adapter extends BaseSpark3Adapter { } /** - * Get properties needed to read a parquet file + * Get parquet file reader * * @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 + * @return parquet file reader */ - override def getPropsForReadingParquet(vectorized: Boolean, - sqlConf: SQLConf, - options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { - Spark33HoodieParquetReader.getPropsForReadingParquet(vectorized, sqlConf, options, hadoopConf) - } - - /** - * Read an individual parquet file - * - * @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]] - */ - override def readParquetFile(file: PartitionedFile, - requiredSchema: StructType, - partitionSchema: StructType, - filters: Seq[sources.Filter], - sharedConf: Configuration, - props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { - Spark33HoodieParquetReader.readParquetFile(file, requiredSchema, partitionSchema, filters, - new Configuration(sharedConf), props) + override def createHoodieParquetFileReader(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkHoodieParquetReader = { + Spark33HoodieParquetReader.build(vectorized, sqlConf, options, hadoopConf) } } diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReader.scala index 6a6036158640..557753e54841 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReader.scala @@ -35,59 +35,41 @@ import org.apache.spark.sql.catalyst.expressions.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._ -import org.apache.spark.sql.hudi.execution.datasources.parquet.SparkHoodieParquetReaderProperties import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ import java.net.URI -object Spark33HoodieParquetReader { - - /** - * 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): SparkHoodieParquetReaderProperties = { - //set hadoopconf - hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) - hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) - hadoopConf.setBoolean(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key, sqlConf.nestedSchemaPruningEnabled) - 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) - // Using string value of this conf to preserve compatibility across spark versions. See [HUDI-5868] - hadoopConf.setBoolean( - "spark.sql.legacy.parquet.nanosAsLong", - sqlConf.getConfString("spark.sql.legacy.parquet.nanosAsLong", "false").toBoolean - ) - - val parquetOptions = new ParquetOptions(options, sqlConf) - new Spark33HoodieParquetReaderProperties(enableVectorizedReader = vectorized, - datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead, - int96RebaseModeInRead = parquetOptions.int96RebaseModeInRead, - enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, - pushDownDate = sqlConf.parquetFilterPushDownDate, - pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp, - pushDownDecimal = sqlConf.parquetFilterPushDownDecimal, - pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold, - pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith, - isCaseSensitive = sqlConf.caseSensitiveAnalysis, - timestampConversion = sqlConf.isParquetINT96TimestampConversion, - enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, - capacity = sqlConf.parquetVectorizedReaderBatchSize, - returningBatch = sqlConf.parquetVectorizedReaderEnabled, - enableRecordFilter = sqlConf.parquetRecordFilterEnabled, - timeZoneId = Some(sqlConf.sessionLocalTimeZone)) - } +class Spark33HoodieParquetReader(enableVectorizedReader: Boolean, + datetimeRebaseModeInRead: String, + int96RebaseModeInRead: String, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringStartWith: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) extends SparkHoodieParquetReaderBase( + enableVectorizedReader = enableVectorizedReader, + enableParquetFilterPushDown = enableParquetFilterPushDown, + pushDownDate = pushDownDate, + pushDownTimestamp = pushDownTimestamp, + pushDownDecimal = pushDownDecimal, + pushDownInFilterThreshold = pushDownInFilterThreshold, + isCaseSensitive = isCaseSensitive, + timestampConversion = timestampConversion, + enableOffHeapColumnVector = enableOffHeapColumnVector, + capacity = capacity, + returningBatch = returningBatch, + enableRecordFilter = enableRecordFilter, + timeZoneId = timeZoneId) { /** * Read an individual parquet file @@ -98,35 +80,13 @@ object Spark33HoodieParquetReader { * @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, + protected def doRead(file: PartitionedFile, requiredSchema: StructType, partitionSchema: StructType, filters: Seq[Filter], - sharedConf: Configuration, - props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { - sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) - sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) - ParquetWriteSupport.setSchema(requiredSchema, sharedConf) - val properties = props.asInstanceOf[Spark33HoodieParquetReaderProperties] - val enableVectorizedReader = properties.enableVectorizedReader - val datetimeRebaseModeInRead = properties.datetimeRebaseModeInRead - val int96RebaseModeInRead = properties.int96RebaseModeInRead - val enableParquetFilterPushDown = properties.enableParquetFilterPushDown - val pushDownDate = properties.pushDownDate - val pushDownTimestamp = properties.pushDownTimestamp - val pushDownDecimal = properties.pushDownDecimal - val pushDownInFilterThreshold = properties.pushDownInFilterThreshold - val pushDownStringStartWith = properties.pushDownStringStartWith - val isCaseSensitive = properties.isCaseSensitive - val timestampConversion = properties.timestampConversion - val enableOffHeapColumnVector = properties.enableOffHeapColumnVector - val capacity = properties.capacity - val returningBatch = properties.returningBatch - val enableRecordFilter = properties.enableRecordFilter - val timeZoneId = properties.timeZoneId + sharedConf: Configuration): Iterator[InternalRow] = { assert(file.partitionValues.numFields == partitionSchema.size) val filePath = new Path(new URI(file.filePath)) @@ -259,3 +219,50 @@ object Spark33HoodieParquetReader { } } } + +object Spark33HoodieParquetReader extends SparkHoodieParquetReaderBuilder { + /** + * Get parquet file reader + * + * @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 parquet file reader + */ + def build(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkHoodieParquetReader = { + //set hadoopconf + hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) + hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) + hadoopConf.setBoolean(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key, sqlConf.nestedSchemaPruningEnabled) + 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) + // Using string value of this conf to preserve compatibility across spark versions. See [HUDI-5868] + hadoopConf.setBoolean( + "spark.sql.legacy.parquet.nanosAsLong", + sqlConf.getConfString("spark.sql.legacy.parquet.nanosAsLong", "false").toBoolean + ) + + val parquetOptions = new ParquetOptions(options, sqlConf) + new Spark33HoodieParquetReader(enableVectorizedReader = vectorized, + datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead, + int96RebaseModeInRead = parquetOptions.int96RebaseModeInRead, + enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, + pushDownDate = sqlConf.parquetFilterPushDownDate, + pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp, + pushDownDecimal = sqlConf.parquetFilterPushDownDecimal, + pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold, + pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith, + isCaseSensitive = sqlConf.caseSensitiveAnalysis, + timestampConversion = sqlConf.isParquetINT96TimestampConversion, + enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, + capacity = sqlConf.parquetVectorizedReaderBatchSize, + returningBatch = sqlConf.parquetVectorizedReaderEnabled, + enableRecordFilter = sqlConf.parquetRecordFilterEnabled, + timeZoneId = Some(sqlConf.sessionLocalTimeZone)) + } +} diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReaderProperties.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReaderProperties.scala deleted file mode 100644 index 79649b07f0a3..000000000000 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReaderProperties.scala +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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 - -case class Spark33HoodieParquetReaderProperties(enableVectorizedReader: Boolean, - datetimeRebaseModeInRead: String, - int96RebaseModeInRead: String, - enableParquetFilterPushDown: Boolean, - pushDownDate: Boolean, - pushDownTimestamp: Boolean, - pushDownDecimal: Boolean, - pushDownInFilterThreshold: Int, - pushDownStringStartWith: Boolean, - isCaseSensitive: Boolean, - timestampConversion: Boolean, - enableOffHeapColumnVector: Boolean, - capacity: Int, - returningBatch: Boolean, - enableRecordFilter: Boolean, - timeZoneId: Option[String]) - extends Spark3HoodieParquetReaderProperties( - enableVectorizedReader = enableVectorizedReader, - datetimeRebaseModeInRead = datetimeRebaseModeInRead, - int96RebaseModeInRead = int96RebaseModeInRead, - enableParquetFilterPushDown = enableParquetFilterPushDown, - pushDownDate = pushDownDate, - pushDownTimestamp = pushDownTimestamp, - pushDownDecimal = pushDownDecimal, - pushDownInFilterThreshold = pushDownInFilterThreshold, - pushDownStringStartWith = pushDownStringStartWith, - isCaseSensitive = isCaseSensitive, - timestampConversion = timestampConversion, - enableOffHeapColumnVector = enableOffHeapColumnVector, - capacity = capacity, - returningBatch = returningBatch, - enableRecordFilter = enableRecordFilter, - timeZoneId = timeZoneId) From 2012131dfec3ad4c837f43a24d4e8bb5177c420c Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 4 Apr 2024 13:42:52 -0400 Subject: [PATCH 20/86] remove props spark 3.4 --- .../spark/sql/adapter/Spark3_4Adapter.scala | 37 +--- .../parquet/Spark34HoodieParquetReader.scala | 170 +++++++++--------- ...Spark34HoodieParquetReaderProperties.scala | 53 ------ 3 files changed, 96 insertions(+), 164 deletions(-) delete mode 100644 hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReaderProperties.scala diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala index ce98f1980f05..db498a508884 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.METADATA_COL_ATTR_KEY import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark34HoodieParquetReader, Spark34LegacyHoodieParquetFileFormat, SparkHoodieParquetReaderProperties} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark34HoodieParquetReader, Spark34LegacyHoodieParquetFileFormat, SparkHoodieParquetReader} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hudi.analysis.TableValuedFunctions @@ -128,39 +128,18 @@ class Spark3_4Adapter extends BaseSpark3Adapter { } /** - * Get properties needed to read a parquet file + * Get parquet file reader * * @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 + * @return parquet file reader */ - override def getPropsForReadingParquet(vectorized: Boolean, - sqlConf: SQLConf, - options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { - Spark34HoodieParquetReader.getPropsForReadingParquet(vectorized, sqlConf, options, hadoopConf) - } - - /** - * Read an individual parquet file - * - * @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 props 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]] - */ - override def readParquetFile(file: PartitionedFile, - requiredSchema: StructType, - partitionSchema: StructType, - filters: Seq[sources.Filter], - sharedConf: Configuration, - props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { - Spark34HoodieParquetReader.readParquetFile(file, requiredSchema, partitionSchema, filters, - new Configuration(sharedConf), props) + override def createHoodieParquetFileReader(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkHoodieParquetReader = { + Spark34HoodieParquetReader.build(vectorized, sqlConf, options, hadoopConf) } } diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReader.scala index a019b36ff233..67b89bb579bb 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReader.scala @@ -37,63 +37,35 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ -object Spark34HoodieParquetReader { - - /** - * 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): SparkHoodieParquetReaderProperties= { - //set hadoopconf - hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) - hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) - hadoopConf.setBoolean(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key, sqlConf.nestedSchemaPruningEnabled) - 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) - // Using string value of this conf to preserve compatibility across spark versions. See [HUDI-5868] - hadoopConf.setBoolean( - SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, - sqlConf.getConfString( - SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, - SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.defaultValueString).toBoolean - ) - hadoopConf.setBoolean(SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key, sqlConf.parquetInferTimestampNTZEnabled) - - val returningBatch = sqlConf.parquetVectorizedReaderEnabled && - options.getOrElse(FileFormat.OPTION_RETURNING_BATCH, - throw new IllegalArgumentException( - "OPTION_RETURNING_BATCH should always be set for ParquetFileFormat. " + - "To workaround this issue, set spark.sql.parquet.enableVectorizedReader=false.")) - .equals("true") - - val parquetOptions = new ParquetOptions(options, sqlConf) - Spark34HoodieParquetReaderProperties( - enableVectorizedReader = vectorized, - datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead, - int96RebaseModeInRead = parquetOptions.int96RebaseModeInRead, - enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, - pushDownDate = sqlConf.parquetFilterPushDownDate, - pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp, - pushDownDecimal = sqlConf.parquetFilterPushDownDecimal, - pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold, - pushDownStringPredicate = sqlConf.parquetFilterPushDownStringPredicate, - isCaseSensitive = sqlConf.caseSensitiveAnalysis, - timestampConversion = sqlConf.isParquetINT96TimestampConversion, - enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, - capacity = sqlConf.parquetVectorizedReaderBatchSize, - returningBatch = returningBatch, - enableRecordFilter = sqlConf.parquetRecordFilterEnabled, - timeZoneId = Some(sqlConf.sessionLocalTimeZone)) - } +class Spark34HoodieParquetReader(enableVectorizedReader: Boolean, + datetimeRebaseModeInRead: String, + int96RebaseModeInRead: String, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringPredicate: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) extends SparkHoodieParquetReaderBase( + enableVectorizedReader = enableVectorizedReader, + enableParquetFilterPushDown = enableParquetFilterPushDown, + pushDownDate = pushDownDate, + pushDownTimestamp = pushDownTimestamp, + pushDownDecimal = pushDownDecimal, + pushDownInFilterThreshold = pushDownInFilterThreshold, + isCaseSensitive = isCaseSensitive, + timestampConversion = timestampConversion, + enableOffHeapColumnVector = enableOffHeapColumnVector, + capacity = capacity, + returningBatch = returningBatch, + enableRecordFilter = enableRecordFilter, + timeZoneId = timeZoneId) { /** * Read an individual parquet file @@ -104,36 +76,13 @@ object Spark34HoodieParquetReader { * @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, + protected def doRead(file: PartitionedFile, requiredSchema: StructType, partitionSchema: StructType, filters: Seq[Filter], - sharedConf: Configuration, - props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { - sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) - sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) - ParquetWriteSupport.setSchema(requiredSchema, sharedConf) - val properties = props.asInstanceOf[Spark34HoodieParquetReaderProperties] - val enableVectorizedReader = properties.enableVectorizedReader - val datetimeRebaseModeInRead = properties.datetimeRebaseModeInRead - val int96RebaseModeInRead = properties.int96RebaseModeInRead - val enableParquetFilterPushDown = properties.enableParquetFilterPushDown - val pushDownDate = properties.pushDownDate - val pushDownTimestamp = properties.pushDownTimestamp - val pushDownDecimal = properties.pushDownDecimal - val pushDownInFilterThreshold = properties.pushDownInFilterThreshold - val pushDownStringPredicate = properties.pushDownStringPredicate - val isCaseSensitive = properties.isCaseSensitive - val timestampConversion = properties.timestampConversion - val enableOffHeapColumnVector = properties.enableOffHeapColumnVector - val capacity = properties.capacity - val returningBatch = properties.returningBatch - val enableRecordFilter = properties.enableRecordFilter - val timeZoneId = properties.timeZoneId - + sharedConf: Configuration): Iterator[InternalRow] = { assert(file.partitionValues.numFields == partitionSchema.size) val filePath = file.toPath @@ -267,5 +216,62 @@ object Spark34HoodieParquetReader { } } } +} +object Spark34HoodieParquetReader extends SparkHoodieParquetReaderBuilder { + /** + * Get parquet file reader + * + * @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 parquet file reader + */ + def build(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkHoodieParquetReader = { + //set hadoopconf + hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) + hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) + hadoopConf.setBoolean(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key, sqlConf.nestedSchemaPruningEnabled) + 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) + // Using string value of this conf to preserve compatibility across spark versions. See [HUDI-5868] + hadoopConf.setBoolean( + SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, + sqlConf.getConfString( + SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, + SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.defaultValueString).toBoolean + ) + hadoopConf.setBoolean(SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key, sqlConf.parquetInferTimestampNTZEnabled) + + val returningBatch = sqlConf.parquetVectorizedReaderEnabled && + options.getOrElse(FileFormat.OPTION_RETURNING_BATCH, + throw new IllegalArgumentException( + "OPTION_RETURNING_BATCH should always be set for ParquetFileFormat. " + + "To workaround this issue, set spark.sql.parquet.enableVectorizedReader=false.")) + .equals("true") + + val parquetOptions = new ParquetOptions(options, sqlConf) + new Spark34HoodieParquetReader( + enableVectorizedReader = vectorized, + datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead, + int96RebaseModeInRead = parquetOptions.int96RebaseModeInRead, + enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, + pushDownDate = sqlConf.parquetFilterPushDownDate, + pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp, + pushDownDecimal = sqlConf.parquetFilterPushDownDecimal, + pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold, + pushDownStringPredicate = sqlConf.parquetFilterPushDownStringPredicate, + isCaseSensitive = sqlConf.caseSensitiveAnalysis, + timestampConversion = sqlConf.isParquetINT96TimestampConversion, + enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, + capacity = sqlConf.parquetVectorizedReaderBatchSize, + returningBatch = returningBatch, + enableRecordFilter = sqlConf.parquetRecordFilterEnabled, + timeZoneId = Some(sqlConf.sessionLocalTimeZone)) + } } diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReaderProperties.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReaderProperties.scala deleted file mode 100644 index d868320d8ad3..000000000000 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReaderProperties.scala +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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 - -case class Spark34HoodieParquetReaderProperties(enableVectorizedReader: Boolean, - datetimeRebaseModeInRead: String, - int96RebaseModeInRead: String, - enableParquetFilterPushDown: Boolean, - pushDownDate: Boolean, - pushDownTimestamp: Boolean, - pushDownDecimal: Boolean, - pushDownInFilterThreshold: Int, - pushDownStringPredicate: Boolean, - isCaseSensitive: Boolean, - timestampConversion: Boolean, - enableOffHeapColumnVector: Boolean, - capacity: Int, - returningBatch: Boolean, - enableRecordFilter: Boolean, - timeZoneId: Option[String]) - extends Spark3HoodieParquetReaderProperties( - enableVectorizedReader = enableVectorizedReader, - datetimeRebaseModeInRead = datetimeRebaseModeInRead, - int96RebaseModeInRead = int96RebaseModeInRead, - enableParquetFilterPushDown = enableParquetFilterPushDown, - pushDownDate = pushDownDate, - pushDownTimestamp = pushDownTimestamp, - pushDownDecimal = pushDownDecimal, - pushDownInFilterThreshold = pushDownInFilterThreshold, - isCaseSensitive = isCaseSensitive, - timestampConversion = timestampConversion, - enableOffHeapColumnVector = enableOffHeapColumnVector, - capacity = capacity, - returningBatch = returningBatch, - enableRecordFilter = enableRecordFilter, - timeZoneId = timeZoneId) From e40072e6af2a04a43fa301982645cf0245121f24 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 4 Apr 2024 13:51:51 -0400 Subject: [PATCH 21/86] remove props spark 3.5 --- .../spark/sql/adapter/Spark3_5Adapter.scala | 37 +--- .../parquet/Spark35HoodieParquetReader.scala | 171 +++++++++--------- ...Spark35HoodieParquetReaderProperties.scala | 53 ------ 3 files changed, 97 insertions(+), 164 deletions(-) delete mode 100644 hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReaderProperties.scala diff --git a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_5Adapter.scala b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_5Adapter.scala index a4bd14649032..3f5e53b5b687 100644 --- a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_5Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_5Adapter.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.METADATA_COL_ATTR_KEY import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark35HoodieParquetReader, Spark35LegacyHoodieParquetFileFormat, SparkHoodieParquetReaderProperties} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark35HoodieParquetReader, Spark35LegacyHoodieParquetFileFormat, SparkHoodieParquetReader} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.hudi.analysis.TableValuedFunctions import org.apache.spark.sql.internal.SQLConf @@ -128,39 +128,18 @@ class Spark3_5Adapter extends BaseSpark3Adapter { } /** - * Get properties needed to read a parquet file + * Get parquet file reader * * @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 properties needed for reading a parquet file + * @return parquet file reader */ - override def getPropsForReadingParquet(vectorized: Boolean, - sqlConf: SQLConf, - options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { - Spark35HoodieParquetReader.getPropsForReadingParquet(vectorized, sqlConf, options, hadoopConf) - } - - /** - * Read an individual parquet file - * - * @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 props 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]] - */ - override def readParquetFile(file: PartitionedFile, - requiredSchema: StructType, - partitionSchema: StructType, - filters: Seq[sources.Filter], - sharedConf: Configuration, - props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { - Spark35HoodieParquetReader.readParquetFile(file, requiredSchema, partitionSchema, filters, - new Configuration(sharedConf), props) + override def createHoodieParquetFileReader(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkHoodieParquetReader = { + Spark35HoodieParquetReader.build(vectorized, sqlConf, options, hadoopConf) } } diff --git a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReader.scala index 1dc5a00279b7..48271ba5f005 100644 --- a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReader.scala @@ -37,63 +37,35 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType -object Spark35HoodieParquetReader { - - /** - * 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 properties needed for reading a parquet file - */ - def getPropsForReadingParquet(vectorized: Boolean, - sqlConf: SQLConf, - options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { - //set hadoopconf - hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) - hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) - hadoopConf.setBoolean(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key, sqlConf.nestedSchemaPruningEnabled) - 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) - // Using string value of this conf to preserve compatibility across spark versions. See [HUDI-5868] - hadoopConf.setBoolean( - SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, - sqlConf.getConfString( - SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, - SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.defaultValueString).toBoolean - ) - hadoopConf.setBoolean(SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key, sqlConf.parquetInferTimestampNTZEnabled) - - val returningBatch = sqlConf.parquetVectorizedReaderEnabled && - options.getOrElse(FileFormat.OPTION_RETURNING_BATCH, - throw new IllegalArgumentException( - "OPTION_RETURNING_BATCH should always be set for ParquetFileFormat. " + - "To workaround this issue, set spark.sql.parquet.enableVectorizedReader=false.")) - .equals("true") - - val parquetOptions = new ParquetOptions(options, sqlConf) - Spark35HoodieParquetReaderProperties( - enableVectorizedReader = vectorized, - datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead, - int96RebaseModeInRead = parquetOptions.int96RebaseModeInRead, - enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, - pushDownDate = sqlConf.parquetFilterPushDownDate, - pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp, - pushDownDecimal = sqlConf.parquetFilterPushDownDecimal, - pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold, - pushDownStringPredicate = sqlConf.parquetFilterPushDownStringPredicate, - isCaseSensitive = sqlConf.caseSensitiveAnalysis, - timestampConversion = sqlConf.isParquetINT96TimestampConversion, - enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, - capacity = sqlConf.parquetVectorizedReaderBatchSize, - returningBatch = returningBatch, - enableRecordFilter = sqlConf.parquetRecordFilterEnabled, - timeZoneId = Some(sqlConf.sessionLocalTimeZone)) - } +class Spark35HoodieParquetReader(enableVectorizedReader: Boolean, + datetimeRebaseModeInRead: String, + int96RebaseModeInRead: String, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringPredicate: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) extends SparkHoodieParquetReaderBase( + enableVectorizedReader = enableVectorizedReader, + enableParquetFilterPushDown = enableParquetFilterPushDown, + pushDownDate = pushDownDate, + pushDownTimestamp = pushDownTimestamp, + pushDownDecimal = pushDownDecimal, + pushDownInFilterThreshold = pushDownInFilterThreshold, + isCaseSensitive = isCaseSensitive, + timestampConversion = timestampConversion, + enableOffHeapColumnVector = enableOffHeapColumnVector, + capacity = capacity, + returningBatch = returningBatch, + enableRecordFilter = enableRecordFilter, + timeZoneId = timeZoneId) { /** * Read an individual parquet file @@ -104,36 +76,13 @@ object Spark35HoodieParquetReader { * @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 props 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, + protected def doRead(file: PartitionedFile, requiredSchema: StructType, partitionSchema: StructType, filters: Seq[Filter], - sharedConf: Configuration, - props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { - sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) - sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) - ParquetWriteSupport.setSchema(requiredSchema, sharedConf) - val properties = props.asInstanceOf[Spark35HoodieParquetReaderProperties] - val enableVectorizedReader = properties.enableVectorizedReader - val datetimeRebaseModeInRead = properties.datetimeRebaseModeInRead - val int96RebaseModeInRead = properties.int96RebaseModeInRead - val enableParquetFilterPushDown = properties.enableParquetFilterPushDown - val pushDownDate = properties.pushDownDate - val pushDownTimestamp = properties.pushDownTimestamp - val pushDownDecimal = properties.pushDownDecimal - val pushDownInFilterThreshold = properties.pushDownInFilterThreshold - val pushDownStringPredicate = properties.pushDownStringPredicate - val isCaseSensitive = properties.isCaseSensitive - val timestampConversion = properties.timestampConversion - val enableOffHeapColumnVector = properties.enableOffHeapColumnVector - val capacity = properties.capacity - val returningBatch = properties.returningBatch - val enableRecordFilter = properties.enableRecordFilter - val timeZoneId = properties.timeZoneId - + sharedConf: Configuration): Iterator[InternalRow] = { assert(file.partitionValues.numFields == partitionSchema.size) val filePath = file.toPath @@ -275,3 +224,61 @@ object Spark35HoodieParquetReader { } } } + +object Spark35HoodieParquetReader extends SparkHoodieParquetReaderBuilder { + /** + * Get parquet file reader + * + * @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 parquet file reader + */ + def build(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkHoodieParquetReader = { + //set hadoopconf + hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) + hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) + hadoopConf.setBoolean(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key, sqlConf.nestedSchemaPruningEnabled) + 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) + // Using string value of this conf to preserve compatibility across spark versions. See [HUDI-5868] + hadoopConf.setBoolean( + SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, + sqlConf.getConfString( + SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, + SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.defaultValueString).toBoolean + ) + hadoopConf.setBoolean(SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key, sqlConf.parquetInferTimestampNTZEnabled) + + val returningBatch = sqlConf.parquetVectorizedReaderEnabled && + options.getOrElse(FileFormat.OPTION_RETURNING_BATCH, + throw new IllegalArgumentException( + "OPTION_RETURNING_BATCH should always be set for ParquetFileFormat. " + + "To workaround this issue, set spark.sql.parquet.enableVectorizedReader=false.")) + .equals("true") + + val parquetOptions = new ParquetOptions(options, sqlConf) + new Spark35HoodieParquetReader( + enableVectorizedReader = vectorized, + datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead, + int96RebaseModeInRead = parquetOptions.int96RebaseModeInRead, + enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, + pushDownDate = sqlConf.parquetFilterPushDownDate, + pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp, + pushDownDecimal = sqlConf.parquetFilterPushDownDecimal, + pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold, + pushDownStringPredicate = sqlConf.parquetFilterPushDownStringPredicate, + isCaseSensitive = sqlConf.caseSensitiveAnalysis, + timestampConversion = sqlConf.isParquetINT96TimestampConversion, + enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, + capacity = sqlConf.parquetVectorizedReaderBatchSize, + returningBatch = returningBatch, + enableRecordFilter = sqlConf.parquetRecordFilterEnabled, + timeZoneId = Some(sqlConf.sessionLocalTimeZone)) + } +} diff --git a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReaderProperties.scala b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReaderProperties.scala deleted file mode 100644 index edb50211434e..000000000000 --- a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReaderProperties.scala +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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 - -case class Spark35HoodieParquetReaderProperties(enableVectorizedReader: Boolean, - datetimeRebaseModeInRead: String, - int96RebaseModeInRead: String, - enableParquetFilterPushDown: Boolean, - pushDownDate: Boolean, - pushDownTimestamp: Boolean, - pushDownDecimal: Boolean, - pushDownInFilterThreshold: Int, - pushDownStringPredicate: Boolean, - isCaseSensitive: Boolean, - timestampConversion: Boolean, - enableOffHeapColumnVector: Boolean, - capacity: Int, - returningBatch: Boolean, - enableRecordFilter: Boolean, - timeZoneId: Option[String]) - extends Spark3HoodieParquetReaderProperties( - enableVectorizedReader = enableVectorizedReader, - datetimeRebaseModeInRead = datetimeRebaseModeInRead, - int96RebaseModeInRead = int96RebaseModeInRead, - enableParquetFilterPushDown = enableParquetFilterPushDown, - pushDownDate = pushDownDate, - pushDownTimestamp = pushDownTimestamp, - pushDownDecimal = pushDownDecimal, - pushDownInFilterThreshold = pushDownInFilterThreshold, - isCaseSensitive = isCaseSensitive, - timestampConversion = timestampConversion, - enableOffHeapColumnVector = enableOffHeapColumnVector, - capacity = capacity, - returningBatch = returningBatch, - enableRecordFilter = enableRecordFilter, - timeZoneId = timeZoneId) From 5813cbf24bfac914f265ade1935eeceb369b9a73 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 4 Apr 2024 14:01:34 -0400 Subject: [PATCH 22/86] remove props spark 2.4 --- .../SparkHoodieParquetReaderProperties.scala | 63 -------- .../spark/sql/adapter/Spark2Adapter.scala | 37 +---- .../parquet/Spark24HoodieParquetReader.scala | 141 +++++++++--------- ...Spark24HoodieParquetReaderProperties.scala | 50 ------- 4 files changed, 80 insertions(+), 211 deletions(-) delete mode 100644 hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderProperties.scala delete mode 100644 hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReaderProperties.scala diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderProperties.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderProperties.scala deleted file mode 100644 index 359b230d26f7..000000000000 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderProperties.scala +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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 - -abstract class SparkHoodieParquetReaderProperties(enableVectorizedReader: Boolean, - enableParquetFilterPushDown: Boolean, - pushDownDate: Boolean, - pushDownTimestamp: Boolean, - pushDownDecimal: Boolean, - pushDownInFilterThreshold: Int, - isCaseSensitive: Boolean, - timestampConversion: Boolean, - enableOffHeapColumnVector: Boolean, - capacity: Int, - returningBatch: Boolean, - enableRecordFilter: Boolean, - timeZoneId: Option[String]) - -abstract class Spark24To31HoodieParquetReaderProperties(enableVectorizedReader: Boolean, - enableParquetFilterPushDown: Boolean, - pushDownDate: Boolean, - pushDownTimestamp: Boolean, - pushDownDecimal: Boolean, - pushDownInFilterThreshold: Int, - pushDownStringStartWith: Boolean, - isCaseSensitive: Boolean, - timestampConversion: Boolean, - enableOffHeapColumnVector: Boolean, - capacity: Int, - returningBatch: Boolean, - enableRecordFilter: Boolean, - timeZoneId: Option[String]) - extends SparkHoodieParquetReaderProperties( - enableVectorizedReader = enableVectorizedReader, - enableParquetFilterPushDown = enableParquetFilterPushDown, - pushDownDate = pushDownDate, - pushDownTimestamp = pushDownTimestamp, - pushDownDecimal = pushDownDecimal, - pushDownInFilterThreshold = pushDownInFilterThreshold, - isCaseSensitive = isCaseSensitive, - timestampConversion = timestampConversion, - enableOffHeapColumnVector = enableOffHeapColumnVector, - capacity = capacity, - returningBatch = returningBatch, - enableRecordFilter = enableRecordFilter, - timeZoneId = timeZoneId) \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala index bb771ae03aa3..f1c38ee85aaf 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.{Command, DeleteFromTable} import org.apache.spark.sql.catalyst.util.DateFormatter import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark24HoodieParquetReader, Spark24LegacyHoodieParquetFileFormat, SparkHoodieParquetReaderProperties} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark24HoodieParquetReader, Spark24LegacyHoodieParquetFileFormat, SparkHoodieParquetReader} import org.apache.spark.sql.execution.vectorized.MutableColumnarRow import org.apache.spark.sql.hudi.SparkAdapter import org.apache.spark.sql.hudi.parser.HoodieSpark2ExtendedSqlParser @@ -209,39 +209,18 @@ class Spark2Adapter extends SparkAdapter { } /** - * Get properties needed to read a parquet file + * Get parquet file reader * * @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 properties needed for reading a parquet file + * @return parquet file reader */ - override def getPropsForReadingParquet(vectorized: Boolean, - sqlConf: SQLConf, - options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { - Spark24HoodieParquetReader.getPropsForReadingParquet(vectorized, sqlConf, options, hadoopConf) - } - - /** - * Read an individual parquet file - * - * @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 props 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]] - */ - override def readParquetFile(file: PartitionedFile, - requiredSchema: StructType, - partitionSchema: StructType, - filters: Seq[sources.Filter], - sharedConf: Configuration, - props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { - Spark24HoodieParquetReader.readParquetFile(file, requiredSchema, partitionSchema, filters, - new Configuration(sharedConf), props) + override def createHoodieParquetFileReader(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkHoodieParquetReader = { + Spark24HoodieParquetReader.build(vectorized, sqlConf, options, hadoopConf) } } diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala index ae5998f20257..86b5c0e99ffe 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala @@ -28,58 +28,44 @@ 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.expressions.{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 org.apache.spark.sql.types.StructType import java.net.URI -object Spark24HoodieParquetReader extends SparkHoodieParquetReader { - - /** - * 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 properties needed for reading a parquet file - */ - def getPropsForReadingParquet(vectorized: Boolean, - sqlConf: SQLConf, - options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReaderProperties = { - //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) - - Spark24HoodieParquetReaderProperties( - enableVectorizedReader = vectorized, - enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, - pushDownDate = sqlConf.parquetFilterPushDownDate, - pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp, - pushDownDecimal = sqlConf.parquetFilterPushDownDecimal, - pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold, - pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith, - isCaseSensitive = sqlConf.caseSensitiveAnalysis, - timestampConversion = sqlConf.isParquetINT96TimestampConversion, - enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, - capacity = sqlConf.parquetVectorizedReaderBatchSize, - returningBatch = sqlConf.parquetVectorizedReaderEnabled, - enableRecordFilter = sqlConf.parquetRecordFilterEnabled, - timeZoneId = Some(sqlConf.sessionLocalTimeZone)) - } +class Spark24HoodieParquetReader(enableVectorizedReader: Boolean, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringStartWith: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) extends SparkHoodieParquetReaderBase( + enableVectorizedReader = enableVectorizedReader, + enableParquetFilterPushDown = enableParquetFilterPushDown, + pushDownDate = pushDownDate, + pushDownTimestamp = pushDownTimestamp, + pushDownDecimal = pushDownDecimal, + pushDownInFilterThreshold = pushDownInFilterThreshold, + isCaseSensitive = isCaseSensitive, + timestampConversion = timestampConversion, + enableOffHeapColumnVector = enableOffHeapColumnVector, + capacity = capacity, + returningBatch = returningBatch, + enableRecordFilter = enableRecordFilter, + timeZoneId = timeZoneId) { /** * Read an individual parquet file @@ -90,34 +76,13 @@ object Spark24HoodieParquetReader extends SparkHoodieParquetReader { * @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 props 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, - requiredSchema: StructType, - partitionSchema: StructType, - filters: Seq[Filter], - sharedConf: Configuration, - props: SparkHoodieParquetReaderProperties): Iterator[InternalRow] = { - sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) - sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) - ParquetWriteSupport.setSchema(requiredSchema, sharedConf) - val properties = props.asInstanceOf[Spark24HoodieParquetReaderProperties] - val enableVectorizedReader = properties.enableVectorizedReader - val enableParquetFilterPushDown = properties.enableParquetFilterPushDown - val pushDownDate = properties.pushDownDate - val pushDownTimestamp = properties.pushDownTimestamp - val pushDownDecimal = properties.pushDownDecimal - val pushDownInFilterThreshold = properties.pushDownInFilterThreshold - val pushDownStringStartWith = properties.pushDownStringStartWith - val isCaseSensitive = properties.isCaseSensitive - val timestampConversion = properties.timestampConversion - val enableOffHeapColumnVector = properties.enableOffHeapColumnVector - val capacity = properties.capacity - val returningBatch = properties.returningBatch - val enableRecordFilter = properties.enableRecordFilter - val timeZoneId = properties.timeZoneId - + protected def doRead(file: PartitionedFile, + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[Filter], + sharedConf: Configuration): Iterator[InternalRow] = { assert(file.partitionValues.numFields == partitionSchema.size) val fileSplit = @@ -218,5 +183,43 @@ object Spark24HoodieParquetReader extends SparkHoodieParquetReader { } } } +} + +object Spark24HoodieParquetReader extends SparkHoodieParquetReaderBuilder { + /** + * Get parquet file reader + * + * @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 parquet file reader + */ + def build(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkHoodieParquetReader = { + //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) + new Spark24HoodieParquetReader( + enableVectorizedReader = vectorized, + enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, + pushDownDate = sqlConf.parquetFilterPushDownDate, + pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp, + pushDownDecimal = sqlConf.parquetFilterPushDownDecimal, + pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold, + pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith, + isCaseSensitive = sqlConf.caseSensitiveAnalysis, + timestampConversion = sqlConf.isParquetINT96TimestampConversion, + enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled, + capacity = sqlConf.parquetVectorizedReaderBatchSize, + returningBatch = sqlConf.parquetVectorizedReaderEnabled, + enableRecordFilter = sqlConf.parquetRecordFilterEnabled, + timeZoneId = Some(sqlConf.sessionLocalTimeZone)) + } } diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReaderProperties.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReaderProperties.scala deleted file mode 100644 index f83544a781a7..000000000000 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReaderProperties.scala +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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 - -case class Spark24HoodieParquetReaderProperties(enableVectorizedReader: Boolean, - enableParquetFilterPushDown: Boolean, - pushDownDate: Boolean, - pushDownTimestamp: Boolean, - pushDownDecimal: Boolean, - pushDownInFilterThreshold: Int, - pushDownStringStartWith: Boolean, - isCaseSensitive: Boolean, - timestampConversion: Boolean, - enableOffHeapColumnVector: Boolean, - capacity: Int, - returningBatch: Boolean, - enableRecordFilter: Boolean, - timeZoneId: Option[String]) - extends Spark24To31HoodieParquetReaderProperties( - enableVectorizedReader = enableVectorizedReader, - enableParquetFilterPushDown = enableParquetFilterPushDown, - pushDownDate = pushDownDate, - pushDownTimestamp = pushDownTimestamp, - pushDownDecimal = pushDownDecimal, - pushDownInFilterThreshold = pushDownInFilterThreshold, - pushDownStringStartWith = pushDownStringStartWith, - isCaseSensitive = isCaseSensitive, - timestampConversion = timestampConversion, - enableOffHeapColumnVector = enableOffHeapColumnVector, - capacity = capacity, - returningBatch = returningBatch, - enableRecordFilter = enableRecordFilter, - timeZoneId = timeZoneId) From 0f008226941809a6940c54425bd9fec2b28e5d91 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 4 Apr 2024 14:14:14 -0400 Subject: [PATCH 23/86] remove change --- .../parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala index ec7d75a5624a..3de30c770c7a 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala @@ -40,7 +40,6 @@ import org.apache.spark.sql.catalyst.expressions.JoinedRow import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.parquet.HoodieFileGroupReaderBasedParquetFileFormat.{ROW_INDEX, ROW_INDEX_TEMPORARY_COLUMN_NAME, getAppliedFilters, getAppliedRequiredSchema, getLogFilesFromSlice, getRecordKeyRelatedFilters, makeCloseableFileGroupMappingRecordIterator} import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isMetaField -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.{LongType, Metadata, MetadataBuilder, StringType, StructField, StructType} import org.apache.spark.util.SerializableConfiguration From 867593d87c13cc1963488e0b66e9292ffb88a192 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 4 Apr 2024 14:18:49 -0400 Subject: [PATCH 24/86] remove bad import --- .../src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala index f87d229f4b45..ee00fb9d39ee 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} import org.apache.spark.sql.catalyst.util.DateFormatter import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, SparkHoodieParquetReader, SparkHoodieParquetReaderProperties} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, SparkHoodieParquetReader} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.parser.HoodieExtendedParserInterface import org.apache.spark.sql.sources.{BaseRelation, Filter} From 64965e6f4156428c7075ad1e948520bd292e41dd Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 2 Apr 2024 13:36:48 -0400 Subject: [PATCH 25/86] add spark 3.3 --- .../Spark3ParquetSchemaEvolutionUtils.scala | 194 ++++++++++++++++++ ...ark32PlusParquetSchemaEvolutionUtils.scala | 53 +++++ .../parquet/Spark33HoodieParquetReader.scala | 34 +-- .../Spark33ParquetSchemaEvolutionUtils.scala | 35 ++++ 4 files changed, 304 insertions(+), 12 deletions(-) create mode 100644 hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/Spark3ParquetSchemaEvolutionUtils.scala create mode 100644 hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusParquetSchemaEvolutionUtils.scala create mode 100644 hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33ParquetSchemaEvolutionUtils.scala diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/Spark3ParquetSchemaEvolutionUtils.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/Spark3ParquetSchemaEvolutionUtils.scala new file mode 100644 index 000000000000..ba72ed158705 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/Spark3ParquetSchemaEvolutionUtils.scala @@ -0,0 +1,194 @@ +/* + * 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 + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hudi.client.utils.SparkInternalSchemaConverter +import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.util +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.hadoop.metadata.FileMetaData +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Cast, UnsafeProjection} +import org.apache.spark.sql.execution.datasources.Spark3ParquetSchemaEvolutionUtils.pruneInternalSchema +import org.apache.spark.sql.execution.datasources.parquet.{HoodieParquetFileFormatHelper, ParquetReadSupport} +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types.{AtomicType, DataType, StructField, StructType} + +import scala.collection.convert.ImplicitConversions.`collection AsScalaIterable` + +abstract class Spark3ParquetSchemaEvolutionUtils(sharedConf: Configuration, + filePath: Path, + requiredSchema: StructType, + partitionSchema: StructType) { + // Fetch internal schema + private lazy val internalSchemaStr: String = sharedConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA) + + private lazy val querySchemaOption: util.Option[InternalSchema] = pruneInternalSchema(internalSchemaStr, requiredSchema) + + var shouldUseInternalSchema: Boolean = !isNullOrEmpty(internalSchemaStr) && querySchemaOption.isPresent + + private lazy val tablePath: String = sharedConf.get(SparkInternalSchemaConverter.HOODIE_TABLE_PATH) + private lazy val fileSchema: InternalSchema = if (shouldUseInternalSchema) { + val commitInstantTime = FSUtils.getCommitTime(filePath.getName).toLong; + val validCommits = sharedConf.get(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST) + InternalSchemaCache.getInternalSchemaByVersionId(commitInstantTime, tablePath, sharedConf, if (validCommits == null) "" else validCommits) + } else { + null + } + + def rebuildFilterFromParquet(filter: Filter): Filter = { + rebuildFilterFromParquetHelper(filter, fileSchema, querySchemaOption.orElse(null)) + } + + private def rebuildFilterFromParquetHelper(oldFilter: Filter, fileSchema: InternalSchema, querySchema: InternalSchema): Filter = { + if (fileSchema == null || querySchema == null) { + oldFilter + } else { + oldFilter match { + case eq: EqualTo => + val newAttribute = InternalSchemaUtils.reBuildFilterName(eq.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else eq.copy(attribute = newAttribute) + case eqs: EqualNullSafe => + val newAttribute = InternalSchemaUtils.reBuildFilterName(eqs.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else eqs.copy(attribute = newAttribute) + case gt: GreaterThan => + val newAttribute = InternalSchemaUtils.reBuildFilterName(gt.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else gt.copy(attribute = newAttribute) + case gtr: GreaterThanOrEqual => + val newAttribute = InternalSchemaUtils.reBuildFilterName(gtr.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else gtr.copy(attribute = newAttribute) + case lt: LessThan => + val newAttribute = InternalSchemaUtils.reBuildFilterName(lt.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else lt.copy(attribute = newAttribute) + case lte: LessThanOrEqual => + val newAttribute = InternalSchemaUtils.reBuildFilterName(lte.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else lte.copy(attribute = newAttribute) + case i: In => + val newAttribute = InternalSchemaUtils.reBuildFilterName(i.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else i.copy(attribute = newAttribute) + case isn: IsNull => + val newAttribute = InternalSchemaUtils.reBuildFilterName(isn.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else isn.copy(attribute = newAttribute) + case isnn: IsNotNull => + val newAttribute = InternalSchemaUtils.reBuildFilterName(isnn.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else isnn.copy(attribute = newAttribute) + case And(left, right) => + And(rebuildFilterFromParquetHelper(left, fileSchema, querySchema), rebuildFilterFromParquetHelper(right, fileSchema, querySchema)) + case Or(left, right) => + Or(rebuildFilterFromParquetHelper(left, fileSchema, querySchema), rebuildFilterFromParquetHelper(right, fileSchema, querySchema)) + case Not(child) => + Not(rebuildFilterFromParquetHelper(child, fileSchema, querySchema)) + case ssw: StringStartsWith => + val newAttribute = InternalSchemaUtils.reBuildFilterName(ssw.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else ssw.copy(attribute = newAttribute) + case ses: StringEndsWith => + val newAttribute = InternalSchemaUtils.reBuildFilterName(ses.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else ses.copy(attribute = newAttribute) + case sc: StringContains => + val newAttribute = InternalSchemaUtils.reBuildFilterName(sc.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else sc.copy(attribute = newAttribute) + case AlwaysTrue => + AlwaysTrue + case AlwaysFalse => + AlwaysFalse + case _ => + AlwaysTrue + } + } + } + + protected var typeChangeInfos: java.util.Map[Integer, Pair[DataType, DataType]] = null + + def getHadoopConfClone(footerFileMetaData: FileMetaData, enableVectorizedReader: Boolean): Configuration = { + // Clone new conf + val hadoopAttemptConf = new Configuration(sharedConf) + typeChangeInfos = if (shouldUseInternalSchema) { + val mergedInternalSchema = new InternalSchemaMerger(fileSchema, querySchemaOption.get(), true, true).mergeSchema() + val mergedSchema = SparkInternalSchemaConverter.constructSparkSchemaFromInternalSchema(mergedInternalSchema) + + hadoopAttemptConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, mergedSchema.json) + + SparkInternalSchemaConverter.collectTypeChangedCols(querySchemaOption.get(), mergedInternalSchema) + } else { + val (implicitTypeChangeInfo, sparkRequestSchema) = HoodieParquetFileFormatHelper.buildImplicitSchemaChangeInfo(hadoopAttemptConf, footerFileMetaData, requiredSchema) + if (!implicitTypeChangeInfo.isEmpty) { + shouldUseInternalSchema = true + hadoopAttemptConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, sparkRequestSchema.json) + } + implicitTypeChangeInfo + } + + if (enableVectorizedReader && shouldUseInternalSchema && + !typeChangeInfos.values().forall(_.getLeft.isInstanceOf[AtomicType])) { + throw new IllegalArgumentException( + "Nested types with type changes(implicit or explicit) cannot be read in vectorized mode. " + + "To workaround this issue, set spark.sql.parquet.enableVectorizedReader=false.") + } + + hadoopAttemptConf + } + + def generateUnsafeProjection(fullSchema: Seq[AttributeReference], timeZoneId: Option[String]): UnsafeProjection = { + + if (typeChangeInfos.isEmpty) { + GenerateUnsafeProjection.generate(fullSchema, fullSchema) + } else { + // find type changed. + val newSchema = new StructType(requiredSchema.fields.zipWithIndex.map { case (f, i) => + if (typeChangeInfos.containsKey(i)) { + StructField(f.name, typeChangeInfos.get(i).getRight, f.nullable, f.metadata) + } else f + }) + val newFullSchema = toAttributes(newSchema) ++ toAttributes(partitionSchema) + val castSchema = newFullSchema.zipWithIndex.map { case (attr, i) => + if (typeChangeInfos.containsKey(i)) { + val srcType = typeChangeInfos.get(i).getRight + val dstType = typeChangeInfos.get(i).getLeft + val needTimeZone = Cast.needsTimeZone(srcType, dstType) + Cast(attr, dstType, if (needTimeZone) timeZoneId else None) + } else attr + } + GenerateUnsafeProjection.generate(castSchema, newFullSchema) + } + } + + protected def toAttributes(schema: StructType): Seq[AttributeReference] +} + +object Spark3ParquetSchemaEvolutionUtils { + def pruneInternalSchema(internalSchemaStr: String, requiredSchema: StructType): util.Option[InternalSchema] = { + if (!isNullOrEmpty(internalSchemaStr) ) { + val querySchemaOption = SerDeHelper.fromJson(internalSchemaStr) + if (querySchemaOption.isPresent && requiredSchema.nonEmpty) { + util.Option.of(SparkInternalSchemaConverter.convertAndPruneStructTypeToInternalSchema(requiredSchema, querySchemaOption.get())) + } else { + querySchemaOption + } + } else { + util.Option.empty() + } + } +} diff --git a/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusParquetSchemaEvolutionUtils.scala b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusParquetSchemaEvolutionUtils.scala new file mode 100644 index 000000000000..2be3159742bd --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusParquetSchemaEvolutionUtils.scala @@ -0,0 +1,53 @@ +/* + * 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.spark.TaskContext +import org.apache.spark.sql.catalyst.util.RebaseDateTime +import org.apache.spark.sql.execution.datasources.Spark3ParquetSchemaEvolutionUtils +import org.apache.spark.sql.types.StructType + +import java.time.ZoneId + +abstract class Spark32PlusParquetSchemaEvolutionUtils(sharedConf: Configuration, + filePath: Path, + requiredSchema: StructType, + partitionSchema: StructType) extends + Spark3ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) { + + def buildVectorizedReader(convertTz: Option[ZoneId], + datetimeRebaseSpec: RebaseDateTime.RebaseSpec, + int96RebaseSpec: RebaseDateTime.RebaseSpec, + enableOffHeapColumnVector: Boolean, + taskContext: Option[TaskContext], + capacity: Int): VectorizedParquetRecordReader = { + new Spark32PlusHoodieVectorizedParquetRecordReader( + convertTz.orNull, + datetimeRebaseSpec.mode.toString, + datetimeRebaseSpec.timeZone, + int96RebaseSpec.mode.toString, + int96RebaseSpec.timeZone, + enableOffHeapColumnVector && taskContext.isDefined, + capacity, + typeChangeInfos + ) + } +} diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReader.scala index 557753e54841..0a2913ac1a14 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReader.scala @@ -32,7 +32,6 @@ import org.apache.parquet.hadoop._ import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.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._ import org.apache.spark.sql.internal.SQLConf @@ -92,6 +91,7 @@ class Spark33HoodieParquetReader(enableVectorizedReader: Boolean, val filePath = new Path(new URI(file.filePath)) val split = new FileSplit(filePath, file.start, file.length, Array.empty[String]) + val schemaEvolutionUtils = new Spark33ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) lazy val footerFileMetaData = ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData @@ -110,7 +110,7 @@ class Spark33HoodieParquetReader(enableVectorizedReader: Boolean, pushDownInFilterThreshold, isCaseSensitive, datetimeRebaseSpec) - filters + filters.map(schemaEvolutionUtils.rebuildFilterFromParquet) // 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. @@ -141,7 +141,7 @@ class Spark33HoodieParquetReader(enableVectorizedReader: Boolean, val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val hadoopAttemptContext = - new TaskAttemptContextImpl(sharedConf, attemptId) + new TaskAttemptContextImpl(schemaEvolutionUtils.getHadoopConfClone(footerFileMetaData, enableVectorizedReader), attemptId) // Try to push down filters when filter push-down is enabled. // Notice: This push-down is RowGroups level, not individual records. @@ -150,14 +150,24 @@ class Spark33HoodieParquetReader(enableVectorizedReader: Boolean, } val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { - val vectorizedReader = new VectorizedParquetRecordReader( - convertTz.orNull, - datetimeRebaseSpec.mode.toString, - datetimeRebaseSpec.timeZone, - int96RebaseSpec.mode.toString, - int96RebaseSpec.timeZone, - enableOffHeapColumnVector && taskContext.isDefined, - capacity) + val vectorizedReader = if (schemaEvolutionUtils.shouldUseInternalSchema) { + schemaEvolutionUtils.buildVectorizedReader( + convertTz, + datetimeRebaseSpec, + int96RebaseSpec, + enableOffHeapColumnVector, + taskContext, + capacity) + } else { + new VectorizedParquetRecordReader( + convertTz.orNull, + datetimeRebaseSpec.mode.toString, + datetimeRebaseSpec.timeZone, + int96RebaseSpec.mode.toString, + int96RebaseSpec.timeZone, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) + } // SPARK-37089: We cannot register a task completion listener to close this iterator here // because downstream exec nodes have already registered their listeners. Since listeners // are executed in reverse order of registration, a listener registered here would close the @@ -200,7 +210,7 @@ class Spark33HoodieParquetReader(enableVectorizedReader: Boolean, reader.initialize(split, hadoopAttemptContext) val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes - val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + val unsafeProjection = schemaEvolutionUtils.generateUnsafeProjection(fullSchema, timeZoneId) if (partitionSchema.length == 0) { // There is no partition columns diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33ParquetSchemaEvolutionUtils.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33ParquetSchemaEvolutionUtils.scala new file mode 100644 index 000000000000..b14a3bf99847 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33ParquetSchemaEvolutionUtils.scala @@ -0,0 +1,35 @@ +/* + * 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.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.types.StructType + +class Spark33ParquetSchemaEvolutionUtils(sharedConf: Configuration, + filePath: Path, + requiredSchema: StructType, + partitionSchema: StructType) extends + Spark32PlusParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema){ + override protected def toAttributes(schema: StructType): Seq[AttributeReference] = { + schema.toAttributes + } +} From 6b0ca8875aba8fbbdac663a9bad7521eddfac477 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 2 Apr 2024 13:42:01 -0400 Subject: [PATCH 26/86] add spark 3.4 --- .../parquet/Spark34HoodieParquetReader.scala | 34 +++++++++++------- .../Spark34ParquetSchemaEvolutionUtils.scala | 35 +++++++++++++++++++ 2 files changed, 57 insertions(+), 12 deletions(-) create mode 100644 hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34ParquetSchemaEvolutionUtils.scala diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReader.scala index 67b89bb579bb..aed437b847c0 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReader.scala @@ -30,7 +30,6 @@ import org.apache.parquet.hadoop._ import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.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._ import org.apache.spark.sql.internal.SQLConf @@ -88,6 +87,7 @@ class Spark34HoodieParquetReader(enableVectorizedReader: Boolean, val filePath = file.toPath val split = new FileSplit(filePath, file.start, file.length, Array.empty[String]) + val schemaEvolutionUtils = new Spark34ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) lazy val footerFileMetaData = ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData @@ -106,7 +106,7 @@ class Spark34HoodieParquetReader(enableVectorizedReader: Boolean, pushDownInFilterThreshold, isCaseSensitive, datetimeRebaseSpec) - filters + filters.map(schemaEvolutionUtils.rebuildFilterFromParquet) // 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. @@ -137,7 +137,7 @@ class Spark34HoodieParquetReader(enableVectorizedReader: Boolean, val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val hadoopAttemptContext = - new TaskAttemptContextImpl(sharedConf, attemptId) + new TaskAttemptContextImpl(schemaEvolutionUtils.getHadoopConfClone(footerFileMetaData, enableVectorizedReader), attemptId) // Try to push down filters when filter push-down is enabled. // Notice: This push-down is RowGroups level, not individual records. @@ -146,14 +146,24 @@ class Spark34HoodieParquetReader(enableVectorizedReader: Boolean, } val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { - val vectorizedReader = new VectorizedParquetRecordReader( - convertTz.orNull, - datetimeRebaseSpec.mode.toString, - datetimeRebaseSpec.timeZone, - int96RebaseSpec.mode.toString, - int96RebaseSpec.timeZone, - enableOffHeapColumnVector && taskContext.isDefined, - capacity) + val vectorizedReader = if (schemaEvolutionUtils.shouldUseInternalSchema) { + schemaEvolutionUtils.buildVectorizedReader( + convertTz, + datetimeRebaseSpec, + int96RebaseSpec, + enableOffHeapColumnVector, + taskContext, + capacity) + } else { + new VectorizedParquetRecordReader( + convertTz.orNull, + datetimeRebaseSpec.mode.toString, + datetimeRebaseSpec.timeZone, + int96RebaseSpec.mode.toString, + int96RebaseSpec.timeZone, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) + } // SPARK-37089: We cannot register a task completion listener to close this iterator here // because downstream exec nodes have already registered their listeners. Since listeners // are executed in reverse order of registration, a listener registered here would close the @@ -198,7 +208,7 @@ class Spark34HoodieParquetReader(enableVectorizedReader: Boolean, readerWithRowIndexes.initialize(split, hadoopAttemptContext) val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes - val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + val unsafeProjection = schemaEvolutionUtils.generateUnsafeProjection(fullSchema, timeZoneId) if (partitionSchema.length == 0) { // There is no partition columns diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34ParquetSchemaEvolutionUtils.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34ParquetSchemaEvolutionUtils.scala new file mode 100644 index 000000000000..8854e5d54272 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34ParquetSchemaEvolutionUtils.scala @@ -0,0 +1,35 @@ +/* + * 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.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.types.StructType + +class Spark34ParquetSchemaEvolutionUtils(sharedConf: Configuration, + filePath: Path, + requiredSchema: StructType, + partitionSchema: StructType) extends + Spark32PlusParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema){ + override protected def toAttributes(schema: StructType): Seq[AttributeReference] = { + schema.toAttributes + } +} From 5a557e1e3ffbc1ceced8e36ce66c0e805c256a85 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 2 Apr 2024 13:47:06 -0400 Subject: [PATCH 27/86] add spark 3.5 --- .../parquet/Spark35HoodieParquetReader.scala | 34 +++++++++++------- .../Spark35ParquetSchemaEvolutionUtils.scala | 36 +++++++++++++++++++ 2 files changed, 58 insertions(+), 12 deletions(-) create mode 100644 hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35ParquetSchemaEvolutionUtils.scala diff --git a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReader.scala index 48271ba5f005..a466548f9e6c 100644 --- a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReader.scala @@ -29,7 +29,6 @@ import org.apache.parquet.hadoop.{ParquetInputFormat, ParquetRecordReader} import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.JoinedRow -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.{DataSourceUtils, FileFormat, PartitionedFile, RecordReaderIterator} @@ -88,6 +87,8 @@ class Spark35HoodieParquetReader(enableVectorizedReader: Boolean, val filePath = file.toPath val split = new FileSplit(filePath, file.start, file.length, Array.empty[String]) + val schemaEvolutionUtils = new Spark35ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) + val fileFooter = if (enableVectorizedReader) { // When there are vectorized reads, we can avoid reading the footer twice by reading // all row groups in advance and filter row groups according to filters that require @@ -116,7 +117,7 @@ class Spark35HoodieParquetReader(enableVectorizedReader: Boolean, pushDownInFilterThreshold, isCaseSensitive, datetimeRebaseSpec) - filters + filters.map(schemaEvolutionUtils.rebuildFilterFromParquet) // 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. @@ -144,7 +145,7 @@ class Spark35HoodieParquetReader(enableVectorizedReader: Boolean, val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val hadoopAttemptContext = - new TaskAttemptContextImpl(sharedConf, attemptId) + new TaskAttemptContextImpl(schemaEvolutionUtils.getHadoopConfClone(footerFileMetaData, enableVectorizedReader), attemptId) // Try to push down filters when filter push-down is enabled. // Notice: This push-down is RowGroups level, not individual records. @@ -153,14 +154,23 @@ class Spark35HoodieParquetReader(enableVectorizedReader: Boolean, } val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { - val vectorizedReader = new VectorizedParquetRecordReader( - convertTz.orNull, - datetimeRebaseSpec.mode.toString, - datetimeRebaseSpec.timeZone, - int96RebaseSpec.mode.toString, - int96RebaseSpec.timeZone, - enableOffHeapColumnVector && taskContext.isDefined, - capacity) + val vectorizedReader = if (schemaEvolutionUtils.shouldUseInternalSchema) { + schemaEvolutionUtils.buildVectorizedReader(convertTz, + datetimeRebaseSpec, + int96RebaseSpec, + enableOffHeapColumnVector, + taskContext, + capacity) + } else { + new VectorizedParquetRecordReader( + convertTz.orNull, + datetimeRebaseSpec.mode.toString, + datetimeRebaseSpec.timeZone, + int96RebaseSpec.mode.toString, + int96RebaseSpec.timeZone, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) + } // SPARK-37089: We cannot register a task completion listener to close this iterator here // because downstream exec nodes have already registered their listeners. Since listeners // are executed in reverse order of registration, a listener registered here would close the @@ -205,7 +215,7 @@ class Spark35HoodieParquetReader(enableVectorizedReader: Boolean, readerWithRowIndexes.initialize(split, hadoopAttemptContext) val fullSchema = toAttributes(requiredSchema) ++ toAttributes(partitionSchema) - val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + val unsafeProjection = schemaEvolutionUtils.generateUnsafeProjection(fullSchema, timeZoneId) if (partitionSchema.length == 0) { // There is no partition columns diff --git a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35ParquetSchemaEvolutionUtils.scala b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35ParquetSchemaEvolutionUtils.scala new file mode 100644 index 000000000000..d08f3792a5cc --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35ParquetSchemaEvolutionUtils.scala @@ -0,0 +1,36 @@ +/* + * 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.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.types.DataTypeUtils +import org.apache.spark.sql.types.StructType + +class Spark35ParquetSchemaEvolutionUtils(sharedConf: Configuration, + filePath: Path, + requiredSchema: StructType, + partitionSchema: StructType) extends + Spark32PlusParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema){ + override protected def toAttributes(schema: StructType): Seq[AttributeReference] = { + DataTypeUtils.toAttributes(schema) + } +} From 2f48bdde27c8803e12d03901bc1de8011df1e54c Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 2 Apr 2024 13:53:51 -0400 Subject: [PATCH 28/86] add spark 3.2 --- .../parquet/Spark32HoodieParquetReader.scala | 113 +++++++++++++++--- .../Spark32ParquetSchemaEvolutionUtils.scala | 35 ++++++ 2 files changed, 129 insertions(+), 19 deletions(-) create mode 100644 hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32ParquetSchemaEvolutionUtils.scala diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReader.scala index 8db3319273b1..d000df59a0af 100644 --- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReader.scala @@ -24,6 +24,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.mapred.FileSplit import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl +import org.apache.hudi.HoodieSparkUtils import org.apache.parquet.filter2.compat.FilterCompat import org.apache.parquet.filter2.predicate.FilterApi import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS @@ -31,7 +32,6 @@ import org.apache.parquet.hadoop._ import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.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._ import org.apache.spark.sql.internal.SQLConf @@ -91,6 +91,8 @@ class Spark32HoodieParquetReader(enableVectorizedReader: Boolean, val filePath = new Path(new URI(file.filePath)) val split = new FileSplit(filePath, file.start, file.length, Array.empty[String]) + val schemaEvolutionUtils = new Spark32ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) + lazy val footerFileMetaData = ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( @@ -99,16 +101,36 @@ class Spark32HoodieParquetReader(enableVectorizedReader: Boolean, // Try to push down filters when filter push-down is enabled. val pushed = if (enableParquetFilterPushDown) { val parquetSchema = footerFileMetaData.getSchema - val parquetFilters = new ParquetFilters( - parquetSchema, - pushDownDate, - pushDownTimestamp, - pushDownDecimal, - pushDownStringStartWith, - pushDownInFilterThreshold, - isCaseSensitive, - datetimeRebaseSpec) - filters + val parquetFilters = if (HoodieSparkUtils.gteqSpark3_2_1) { + // NOTE: Below code could only be compiled against >= Spark 3.2.1, + // and unfortunately won't compile against Spark 3.2.0 + // However this code is runtime-compatible w/ both Spark 3.2.0 and >= Spark 3.2.1 + val datetimeRebaseSpec = + DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) + new ParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringStartWith, + pushDownInFilterThreshold, + isCaseSensitive, + datetimeRebaseSpec) + } else { + // Spark 3.2.0 + val datetimeRebaseMode = + Spark32DataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) + createParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringStartWith, + pushDownInFilterThreshold, + isCaseSensitive, + datetimeRebaseMode) + } + filters.map(schemaEvolutionUtils.rebuildFilterFromParquet) // 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. @@ -139,7 +161,7 @@ class Spark32HoodieParquetReader(enableVectorizedReader: Boolean, val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val hadoopAttemptContext = - new TaskAttemptContextImpl(sharedConf, attemptId) + new TaskAttemptContextImpl(schemaEvolutionUtils.getHadoopConfClone(footerFileMetaData, enableVectorizedReader), attemptId) // Try to push down filters when filter push-down is enabled. // Notice: This push-down is RowGroups level, not individual records. @@ -148,7 +170,15 @@ class Spark32HoodieParquetReader(enableVectorizedReader: Boolean, } val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { - val vectorizedReader = new VectorizedParquetRecordReader( + val vectorizedReader = if (schemaEvolutionUtils.shouldUseInternalSchema) { + schemaEvolutionUtils.buildVectorizedReader(convertTz, + datetimeRebaseSpec, + int96RebaseSpec, + enableOffHeapColumnVector, + taskContext, + capacity) + } else { + new VectorizedParquetRecordReader( convertTz.orNull, datetimeRebaseSpec.mode.toString, datetimeRebaseSpec.timeZone, @@ -156,6 +186,7 @@ class Spark32HoodieParquetReader(enableVectorizedReader: Boolean, int96RebaseSpec.timeZone, enableOffHeapColumnVector && taskContext.isDefined, capacity) + } // SPARK-37089: We cannot register a task completion listener to close this iterator here // because downstream exec nodes have already registered their listeners. Since listeners // are executed in reverse order of registration, a listener registered here would close the @@ -182,11 +213,31 @@ class Spark32HoodieParquetReader(enableVectorizedReader: Boolean, } } else { // ParquetRecordReader returns InternalRow - val readSupport = new ParquetReadSupport( - convertTz, - enableVectorizedReader = false, - datetimeRebaseSpec, - int96RebaseSpec) + val readSupport = if (HoodieSparkUtils.gteqSpark3_2_1) { + // ParquetRecordReader returns InternalRow + // NOTE: Below code could only be compiled against >= Spark 3.2.1, + // and unfortunately won't compile against Spark 3.2.0 + // However this code is runtime-compatible w/ both Spark 3.2.0 and >= Spark 3.2.1 + val int96RebaseSpec = + DataSourceUtils.int96RebaseSpec(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) + val datetimeRebaseSpec = + DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) + new ParquetReadSupport( + convertTz, + enableVectorizedReader = false, + datetimeRebaseSpec, + int96RebaseSpec) + } else { + val datetimeRebaseMode = + Spark32DataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) + val int96RebaseMode = + Spark32DataSourceUtils.int96RebaseMode(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) + createParquetReadSupport( + convertTz, + /* enableVectorizedReader = */ false, + datetimeRebaseMode, + int96RebaseMode) + } val reader = if (pushed.isDefined && enableRecordFilter) { val parquetFilter = FilterCompat.get(pushed.get, null) new ParquetRecordReader[InternalRow](readSupport, parquetFilter) @@ -198,7 +249,7 @@ class Spark32HoodieParquetReader(enableVectorizedReader: Boolean, reader.initialize(split, hadoopAttemptContext) val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes - val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + val unsafeProjection = schemaEvolutionUtils.generateUnsafeProjection(fullSchema, timeZoneId) if (partitionSchema.length == 0) { // There is no partition columns @@ -216,6 +267,30 @@ class Spark32HoodieParquetReader(enableVectorizedReader: Boolean, } } } + + /** + * NOTE: This method is specific to Spark 3.2.0 + */ + private def createParquetFilters(args: Any*): ParquetFilters = { + // NOTE: ParquetFilters ctor args contain Scala enum, therefore we can't look it + // up by arg types, and have to instead rely on the number of args based on individual class; + // the ctor order is not guaranteed + val ctor = classOf[ParquetFilters].getConstructors.maxBy(_.getParameterCount) + ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*) + .asInstanceOf[ParquetFilters] + } + + /** + * NOTE: This method is specific to Spark 3.2.0 + */ + private def createParquetReadSupport(args: Any*): ParquetReadSupport = { + // NOTE: ParquetReadSupport ctor args contain Scala enum, therefore we can't look it + // up by arg types, and have to instead rely on the number of args based on individual class; + // the ctor order is not guaranteed + val ctor = classOf[ParquetReadSupport].getConstructors.maxBy(_.getParameterCount) + ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*) + .asInstanceOf[ParquetReadSupport] + } } object Spark32HoodieParquetReader extends SparkHoodieParquetReaderBuilder { diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32ParquetSchemaEvolutionUtils.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32ParquetSchemaEvolutionUtils.scala new file mode 100644 index 000000000000..43741e8193de --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32ParquetSchemaEvolutionUtils.scala @@ -0,0 +1,35 @@ +/* + * 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.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.types.StructType + +class Spark32ParquetSchemaEvolutionUtils(sharedConf: Configuration, + filePath: Path, + requiredSchema: StructType, + partitionSchema: StructType) extends + Spark32PlusParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema){ + override protected def toAttributes(schema: StructType): Seq[AttributeReference] = { + schema.toAttributes + } +} From 9845f373d88f01c6b162010fe6cbe82ace91d42f Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 2 Apr 2024 13:59:58 -0400 Subject: [PATCH 29/86] add spark 3.1 --- .../parquet/Spark31HoodieParquetReader.scala | 70 +++++++++++++------ .../Spark31ParquetSchemaEvolutionUtils.scala | 57 +++++++++++++++ 2 files changed, 107 insertions(+), 20 deletions(-) create mode 100644 hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31ParquetSchemaEvolutionUtils.scala diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReader.scala index 5c1bb612f7db..fe3e729a2956 100644 --- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReader.scala @@ -23,6 +23,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl +import org.apache.hudi.HoodieSparkUtils import org.apache.parquet.filter2.compat.FilterCompat import org.apache.parquet.filter2.predicate.FilterApi import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS @@ -30,7 +31,6 @@ import org.apache.parquet.hadoop._ import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.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._ import org.apache.spark.sql.internal.SQLConf @@ -95,6 +95,8 @@ class Spark31HoodieParquetReader(enableVectorizedReader: Boolean, Array.empty, null) + val schemaEvolutionUtils = new Spark31ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) + lazy val footerFileMetaData = ParquetFileReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData val datetimeRebaseMode = DataSourceUtils.datetimeRebaseMode( @@ -103,16 +105,27 @@ class Spark31HoodieParquetReader(enableVectorizedReader: Boolean, // Try to push down filters when filter push-down is enabled. val pushed = if (enableParquetFilterPushDown) { val parquetSchema = footerFileMetaData.getSchema - val parquetFilters = new ParquetFilters( - parquetSchema, - pushDownDate, - pushDownTimestamp, - pushDownDecimal, - pushDownStringStartWith, - pushDownInFilterThreshold, - isCaseSensitive, - datetimeRebaseMode) - filters + val parquetFilters = if (HoodieSparkUtils.gteqSpark3_1_3) { + createParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringStartWith, + pushDownInFilterThreshold, + isCaseSensitive, + datetimeRebaseMode) + } else { + createParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringStartWith, + pushDownInFilterThreshold, + isCaseSensitive) + } + filters.map(schemaEvolutionUtils.rebuildFilterFromParquet) // 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. @@ -143,7 +156,7 @@ class Spark31HoodieParquetReader(enableVectorizedReader: Boolean, val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val hadoopAttemptContext = - new TaskAttemptContextImpl(sharedConf, attemptId) + new TaskAttemptContextImpl(schemaEvolutionUtils.getHadoopConfClone(footerFileMetaData, enableVectorizedReader), attemptId) // Try to push down filters when filter push-down is enabled. // Notice: This push-down is RowGroups level, not individual records. @@ -152,12 +165,23 @@ class Spark31HoodieParquetReader(enableVectorizedReader: Boolean, } val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { - val vectorizedReader = new VectorizedParquetRecordReader( - convertTz.orNull, - datetimeRebaseMode.toString, - int96RebaseMode.toString, - enableOffHeapColumnVector && taskContext.isDefined, - capacity) + val vectorizedReader = if (schemaEvolutionUtils.shouldUseInternalSchema) { + schemaEvolutionUtils.buildVectorizedReader( + convertTz, + datetimeRebaseMode, + int96RebaseMode, + enableOffHeapColumnVector, + taskContext, + capacity) + } else { + new VectorizedParquetRecordReader( + convertTz.orNull, + datetimeRebaseMode.toString, + int96RebaseMode.toString, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) + } + val iter = new RecordReaderIterator(vectorizedReader) // SPARK-23457 Register a task completion listener before `initialization`. taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) @@ -188,8 +212,7 @@ class Spark31HoodieParquetReader(enableVectorizedReader: Boolean, reader.initialize(split, hadoopAttemptContext) val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes - val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) - + val unsafeProjection = schemaEvolutionUtils.generateUnsafeProjection(fullSchema, timeZoneId) if (partitionSchema.length == 0) { // There is no partition columns iter.map(unsafeProjection) @@ -199,6 +222,13 @@ class Spark31HoodieParquetReader(enableVectorizedReader: Boolean, } } } + + private def createParquetFilters(args: Any*): ParquetFilters = { + // ParquetFilters bears a single ctor (in Spark 3.1) + val ctor = classOf[ParquetFilters].getConstructors.head + ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*) + .asInstanceOf[ParquetFilters] + } } object Spark31HoodieParquetReader extends SparkHoodieParquetReaderBuilder { diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31ParquetSchemaEvolutionUtils.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31ParquetSchemaEvolutionUtils.scala new file mode 100644 index 000000000000..4ad8265ba524 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31ParquetSchemaEvolutionUtils.scala @@ -0,0 +1,57 @@ +/* + * 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.spark.TaskContext +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.execution.datasources.Spark3ParquetSchemaEvolutionUtils +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StructType + +import java.time.ZoneId + +class Spark31ParquetSchemaEvolutionUtils(sharedConf: Configuration, + filePath: Path, + requiredSchema: StructType, + partitionSchema: StructType) extends + Spark3ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) { + + + override protected def toAttributes(schema: StructType): Seq[AttributeReference] = { + schema.toAttributes + } + + def buildVectorizedReader(convertTz: Option[ZoneId], + datetimeRebaseMode: SQLConf.LegacyBehaviorPolicy.Value, + int96RebaseMode: SQLConf.LegacyBehaviorPolicy.Value, + enableOffHeapColumnVector: Boolean, + taskContext: Option[TaskContext], + capacity: Int): VectorizedParquetRecordReader = { + new Spark31HoodieVectorizedParquetRecordReader( + convertTz.orNull, + datetimeRebaseMode.toString, + int96RebaseMode.toString, + enableOffHeapColumnVector && taskContext.isDefined, + capacity, + typeChangeInfos) + } +} From 5e536fef44fc016c8f0ef8ebd539909bb783fe7f Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 2 Apr 2024 14:06:54 -0400 Subject: [PATCH 30/86] add spark 3.0 --- .../parquet/Spark30HoodieParquetReader.scala | 43 +++++++++++---- .../Spark30ParquetSchemaEvolutionUtils.scala | 55 +++++++++++++++++++ 2 files changed, 87 insertions(+), 11 deletions(-) create mode 100644 hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30ParquetSchemaEvolutionUtils.scala diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala index 3d950fb5cbdd..fc379cac7087 100644 --- a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala @@ -30,7 +30,6 @@ import org.apache.parquet.hadoop._ import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.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._ import org.apache.spark.sql.internal.SQLConf @@ -95,14 +94,22 @@ class Spark30HoodieParquetReader(enableVectorizedReader: Boolean, Array.empty, null) + val schemaEvolutionUtils = new Spark30ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) + 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(parquetSchema, pushDownDate, pushDownTimestamp, - pushDownDecimal, pushDownStringStartWith, pushDownInFilterThreshold, isCaseSensitive) - filters + val parquetFilters = createParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringStartWith, + pushDownInFilterThreshold, + isCaseSensitive) + filters.map(schemaEvolutionUtils.rebuildFilterFromParquet) // 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. @@ -133,7 +140,7 @@ class Spark30HoodieParquetReader(enableVectorizedReader: Boolean, val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val hadoopAttemptContext = - new TaskAttemptContextImpl(sharedConf, attemptId) + new TaskAttemptContextImpl(schemaEvolutionUtils.getHadoopConfClone(footerFileMetaData, enableVectorizedReader), attemptId) // Try to push down filters when filter push-down is enabled. // Notice: This push-down is RowGroups level, not individual records. @@ -142,11 +149,20 @@ class Spark30HoodieParquetReader(enableVectorizedReader: Boolean, } val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { - val vectorizedReader = new VectorizedParquetRecordReader( - convertTz.orNull, - datetimeRebaseMode.toString, - enableOffHeapColumnVector && taskContext.isDefined, - capacity) + val vectorizedReader = if (schemaEvolutionUtils.shouldUseInternalSchema) { + schemaEvolutionUtils.buildVectorizedReader( + convertTz, + datetimeRebaseMode, + enableOffHeapColumnVector, + taskContext, + capacity) + } else { + new VectorizedParquetRecordReader( + convertTz.orNull, + datetimeRebaseMode.toString, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) + } val iter = new RecordReaderIterator(vectorizedReader) // SPARK-23457 Register a task completion listener before `initialization`. taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) @@ -174,7 +190,7 @@ class Spark30HoodieParquetReader(enableVectorizedReader: Boolean, reader.initialize(split, hadoopAttemptContext) val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes - val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + val unsafeProjection = schemaEvolutionUtils.generateUnsafeProjection(fullSchema, timeZoneId) if (partitionSchema.length == 0) { // There is no partition columns @@ -186,6 +202,11 @@ class Spark30HoodieParquetReader(enableVectorizedReader: Boolean, } } + private def createParquetFilters(args: Any*): ParquetFilters = { + val ctor = classOf[ParquetFilters].getConstructors.head + ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*) + .asInstanceOf[ParquetFilters] + } } object Spark30HoodieParquetReader extends SparkHoodieParquetReaderBuilder { diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30ParquetSchemaEvolutionUtils.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30ParquetSchemaEvolutionUtils.scala new file mode 100644 index 000000000000..0ff2ede9ba7d --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30ParquetSchemaEvolutionUtils.scala @@ -0,0 +1,55 @@ +/* + * 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.spark.TaskContext +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.execution.datasources.Spark3ParquetSchemaEvolutionUtils +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StructType + +import java.time.ZoneId + +class Spark30ParquetSchemaEvolutionUtils(sharedConf: Configuration, + filePath: Path, + requiredSchema: StructType, + partitionSchema: StructType) extends + Spark3ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) { + + + override protected def toAttributes(schema: StructType): Seq[AttributeReference] = { + schema.toAttributes + } + + def buildVectorizedReader(convertTz: Option[ZoneId], + datetimeRebaseMode: SQLConf.LegacyBehaviorPolicy.Value, + enableOffHeapColumnVector: Boolean, + taskContext: Option[TaskContext], + capacity: Int): VectorizedParquetRecordReader = { + new Spark30HoodieVectorizedParquetRecordReader( + convertTz.orNull, + datetimeRebaseMode.toString, + enableOffHeapColumnVector && taskContext.isDefined, + capacity, + typeChangeInfos) + } +} From 9dc9adea466ffb92a549d893b0a55b277ac81a26 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 2 Apr 2024 15:22:38 -0400 Subject: [PATCH 31/86] add spark 2.4 --- .../parquet/Spark24HoodieParquetReader.scala | 60 ++++++++++++++++--- 1 file changed, 51 insertions(+), 9 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala index 86b5c0e99ffe..faccd136a8fa 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala @@ -131,8 +131,17 @@ class Spark24HoodieParquetReader(enableVectorizedReader: Boolean, } val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) + + // Clone new conf + val hadoopAttemptConf = new Configuration(sharedConf) + val (implicitTypeChangeInfos, sparkRequestSchema) = HoodieParquetFileFormatHelper.buildImplicitSchemaChangeInfo(hadoopAttemptConf, footerFileMetaData, requiredSchema) + + if (!implicitTypeChangeInfos.isEmpty) { + hadoopAttemptConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, sparkRequestSchema.json) + } + val hadoopAttemptContext = - new TaskAttemptContextImpl(sharedConf, attemptId) + new TaskAttemptContextImpl(hadoopAttemptConf, attemptId) // Try to push down filters when filter push-down is enabled. // Notice: This push-down is RowGroups level, not individual records. @@ -141,8 +150,20 @@ class Spark24HoodieParquetReader(enableVectorizedReader: Boolean, } val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { - val vectorizedReader = new VectorizedParquetRecordReader( - convertTz.orNull, enableOffHeapColumnVector && taskContext.isDefined, capacity) + val vectorizedReader = if (!implicitTypeChangeInfos.isEmpty) { + new Spark24HoodieVectorizedParquetRecordReader( + convertTz.orNull, + enableOffHeapColumnVector && taskContext.isDefined, + capacity, + implicitTypeChangeInfos + ) + } else { + new VectorizedParquetRecordReader( + convertTz.orNull, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) + } + val iter = new RecordReaderIterator(vectorizedReader) // SPARK-23457 Register a task completion lister before `initialization`. taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) @@ -156,11 +177,12 @@ class Spark24HoodieParquetReader(enableVectorizedReader: Boolean, iter.asInstanceOf[Iterator[InternalRow]] } else { // ParquetRecordReader returns UnsafeRow + val readSupport = new ParquetReadSupport(convertTz) val reader = if (pushed.isDefined && enableRecordFilter) { val parquetFilter = FilterCompat.get(pushed.get, null) - new ParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz), parquetFilter) + new ParquetRecordReader[UnsafeRow](readSupport, parquetFilter) } else { - new ParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz)) + new ParquetRecordReader[UnsafeRow](readSupport) } val iter = new RecordReaderIterator(reader) // SPARK-23457 Register a task completion lister before `initialization`. @@ -168,18 +190,38 @@ class Spark24HoodieParquetReader(enableVectorizedReader: Boolean, reader.initialize(split, hadoopAttemptContext) val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes - val joinedRow = new JoinedRow() - val appendPartitionColumns = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + val unsafeProjection = if (implicitTypeChangeInfos.isEmpty) { + GenerateUnsafeProjection.generate(fullSchema, fullSchema) + } else { + val newFullSchema = new StructType(requiredSchema.fields.zipWithIndex.map { case (f, i) => + if (implicitTypeChangeInfos.containsKey(i)) { + StructField(f.name, implicitTypeChangeInfos.get(i).getRight, f.nullable, f.metadata) + } else f + }).toAttributes ++ partitionSchema.toAttributes + val castSchema = newFullSchema.zipWithIndex.map { case (attr, i) => + if (implicitTypeChangeInfos.containsKey(i)) { + val srcType = implicitTypeChangeInfos.get(i).getRight + val dstType = implicitTypeChangeInfos.get(i).getLeft + val needTimeZone = Cast.needsTimeZone(srcType, dstType) + Cast(attr, dstType, if (needTimeZone) timeZoneId else None) + } else attr + } + GenerateUnsafeProjection.generate(castSchema, newFullSchema) + } // This is a horrible erasure hack... if we type the iterator above, then it actually check // the type in next() and we get a class cast exception. If we make that function return // Object, then we can defer the cast until later! + // + // NOTE: We're making appending of the partitioned values to the rows read from the + // data file configurable if (partitionSchema.length == 0) { // There is no partition columns - iter.asInstanceOf[Iterator[InternalRow]] + iter.asInstanceOf[Iterator[InternalRow]].map(unsafeProjection) } else { + val joinedRow = new JoinedRow() iter.asInstanceOf[Iterator[InternalRow]] - .map(d => appendPartitionColumns(joinedRow(d, file.partitionValues))) + .map(d => unsafeProjection(joinedRow(d, file.partitionValues))) } } } From b12c0183f2f517c9ae6065936a67a48c131bb482 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 4 Apr 2024 14:58:05 -0400 Subject: [PATCH 32/86] fix 2.4 --- .../datasources/parquet/Spark24HoodieParquetReader.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala index faccd136a8fa..fff16dbb8e84 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala @@ -30,12 +30,12 @@ import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, Parquet import org.apache.spark.TaskContext 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.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.StructType +import org.apache.spark.sql.types.{StructField, StructType} import java.net.URI From 749c651e56221dc26c6998ff9e878f59716290b9 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 2 Apr 2024 16:56:32 -0400 Subject: [PATCH 33/86] integrate schema evolution into the fg reader --- .../BaseSparkInternalRowReaderContext.java | 15 +- ...rkFileFormatInternalRowReaderContext.scala | 212 +++++++++--- .../org/apache/hudi/avro/AvroSchemaUtils.java | 18 +- .../common/engine/HoodieReaderContext.java | 18 +- .../log/block/HoodieParquetDataBlock.java | 2 +- .../read/HoodieBaseFileGroupRecordBuffer.java | 64 +++- .../table/read/HoodieFileGroupReader.java | 103 ++++-- .../HoodieKeyBasedFileGroupRecordBuffer.java | 8 +- ...diePositionBasedFileGroupRecordBuffer.java | 32 +- .../HoodieUnmergedFileGroupRecordBuffer.java | 8 +- .../schema/action/InternalSchemaMerger.java | 20 +- .../convert/AvroInternalSchemaConverter.java | 90 +++++- .../apache/hudi/avro/TestAvroSchemaUtils.java | 14 +- .../read/TestHoodieFileGroupReaderBase.java | 9 +- .../HoodieFileGroupReaderTestUtils.java | 4 +- .../reader/HoodieTestReaderContext.java | 17 +- .../scala/org/apache/hudi/DefaultSource.scala | 9 +- .../hudi/HoodieHadoopFsRelationFactory.scala | 39 ++- ...ileGroupReaderBasedParquetFileFormat.scala | 302 ++++-------------- ...ileGroupReaderBasedParquetFileFormat.scala | 54 +--- ...diePositionBasedFileGroupRecordBuffer.java | 6 +- .../TestAvroSchemaResolutionSupport.scala | 30 +- .../apache/hudi/TestHoodieParquetBloom.scala | 15 +- .../TestHoodieFileGroupReaderOnSpark.scala | 30 +- ...tSpark35RecordPositionMetadataColumn.scala | 104 ++---- .../spark/sql/hudi/ddl/TestSpark3DDL.scala | 6 + ...oodieDeltaStreamerSchemaEvolutionBase.java | 2 +- 27 files changed, 665 insertions(+), 566 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java index d0b27e507fe5..a07602b3885d 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java @@ -38,11 +38,15 @@ import org.apache.spark.sql.HoodieUnsafeRowUtils; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.UnsafeProjection; +import org.apache.spark.sql.catalyst.expressions.UnsafeRow; import org.apache.spark.sql.types.StructType; +import java.util.Collections; import java.util.Map; import java.util.function.UnaryOperator; +import scala.Function1; + import static org.apache.hudi.common.model.HoodieRecord.RECORD_KEY_METADATA_FIELD; import static org.apache.hudi.common.model.HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID; import static org.apache.spark.sql.HoodieInternalRowUtils.getCachedSchema; @@ -141,4 +145,13 @@ public UnaryOperator projectRecord(Schema from, Schema to) { UnsafeProjection projection = HoodieInternalRowUtils.generateUnsafeProjectionAlias(getCachedSchema(from), getCachedSchema(to)); return projection::apply; } -} + + @Override + public UnaryOperator projectRecordUnsafe(Schema from, Schema to, Map renamedColumns) { + StructType structType = HoodieInternalRowUtils.getCachedSchema(from); + StructType newStructType = HoodieInternalRowUtils.getCachedSchema(to); + Function1 unsafeRowWriter = + HoodieInternalRowUtils.getCachedUnsafeRowWriter(structType, newStructType, renamedColumns); + return row -> (InternalRow) unsafeRowWriter.apply(row); + } +} \ No newline at end of file diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala index 963035caf217..96a5634de5a7 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala @@ -23,20 +23,23 @@ import org.apache.avro.Schema import org.apache.avro.generic.IndexedRecord import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.hudi.SparkFileFormatInternalRowReaderContext.{ROW_INDEX_TEMPORARY_COLUMN_NAME, getAppliedRequiredSchema} +import org.apache.hudi.avro.AvroSchemaUtils import org.apache.hudi.common.engine.HoodieReaderContext import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.util.ValidationUtils.checkState -import org.apache.hudi.common.util.collection.{ClosableIterator, CloseableMappingIterator} +import org.apache.hudi.common.util.collection.{CachingIterator, ClosableIterator, CloseableMappingIterator} import org.apache.hudi.io.storage.{HoodieSparkFileReaderFactory, HoodieSparkParquetReader} import org.apache.hudi.util.CloseableInternalRowIterator +import org.apache.spark.sql.HoodieInternalRowUtils import org.apache.spark.sql.avro.HoodieAvroDeserializer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.{LongType, MetadataBuilder, StructField, StructType} import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} -import org.apache.spark.sql.HoodieInternalRowUtils import scala.collection.mutable @@ -46,27 +49,29 @@ import scala.collection.mutable * * This uses Spark parquet reader to read parquet data files or parquet log blocks. * - * @param readermaps our intention is to build the reader inside of getFileRecordIterator, but since it is called from - * the executor, we will need to port a bunch of the code from ParquetFileFormat for each spark version - * for now, we pass in a map of the different readers we expect to create + * @param baseFileReader A reader that transforms a {@link PartitionedFile} to an iterator of + * {@link InternalRow}. This is required for reading the base file and + * not required for reading a file group with only log files. + * @param partitionValues The values for a partition in which the file group lives. */ -class SparkFileFormatInternalRowReaderContext(readerMaps: mutable.Map[Long, PartitionedFile => Iterator[InternalRow]]) extends BaseSparkInternalRowReaderContext { +class SparkFileFormatInternalRowReaderContext(extraProps: Map[String, String], + recordKeyColumn: String, + filters: Seq[Filter], + shouldUseRecordPosition: Boolean) extends BaseSparkInternalRowReaderContext { lazy val sparkAdapter = SparkAdapterSupport.sparkAdapter lazy val sparkFileReaderFactory = new HoodieSparkFileReaderFactory val deserializerMap: mutable.Map[Schema, HoodieAvroDeserializer] = mutable.Map() + lazy val recordKeyFilters: Seq[Filter] = filters.filter(f => f.references.exists(c => c.equalsIgnoreCase(recordKeyColumn))) override def getFileRecordIterator(filePath: Path, start: Long, length: Long, dataSchema: Schema, requiredSchema: Schema, - conf: Configuration): ClosableIterator[InternalRow] = { - // partition value is empty because the spark parquet reader will append the partition columns to - // each row if they are given. That is the only usage of the partition values in the reader. - val fileInfo = sparkAdapter.getSparkPartitionedFileUtils - .createPartitionedFile(InternalRow.empty, filePath, start, length) + conf: Configuration, + isMerge: Boolean): ClosableIterator[InternalRow] = { + val structType: StructType = HoodieInternalRowUtils.getCachedSchema(requiredSchema) if (FSUtils.isLogFile(filePath)) { - val structType: StructType = HoodieInternalRowUtils.getCachedSchema(requiredSchema) val projection: UnsafeProjection = HoodieInternalRowUtils.getCachedUnsafeProjection(structType, structType) new CloseableMappingIterator[InternalRow, UnsafeRow]( sparkFileReaderFactory.newParquetFileReader(conf, filePath).asInstanceOf[HoodieSparkParquetReader] @@ -79,16 +84,24 @@ class SparkFileFormatInternalRowReaderContext(readerMaps: mutable.Map[Long, Part } }).asInstanceOf[ClosableIterator[InternalRow]] } else { - val schemaPairHashKey = generateSchemaPairHashKey(dataSchema, requiredSchema) - if (!readerMaps.contains(schemaPairHashKey)) { - throw new IllegalStateException("schemas don't hash to a known reader") - } - new CloseableInternalRowIterator(readerMaps(schemaPairHashKey).apply(fileInfo)) + // partition value is empty because the spark parquet reader will append the partition columns to + // each row if they are given. That is the only usage of the partition values in the reader. + val fileInfo = sparkAdapter.getSparkPartitionedFileUtils + .createPartitionedFile(InternalRow.empty, filePath, start, length) + new CloseableInternalRowIterator(sparkAdapter.readParquetFile(fileInfo, + getAppliedRequiredSchema(structType, shouldUseRecordPosition && isMerge), StructType(Seq.empty), + getFiltersForRead(isMerge), conf, extraProps)) } } - private def generateSchemaPairHashKey(dataSchema: Schema, requestedSchema: Schema): Long = { - dataSchema.hashCode() + requestedSchema.hashCode() + private def getFiltersForRead(isMerge: Boolean): Seq[Filter] = { + if (!isMerge) { + filters + } else if (!shouldUseRecordPosition) { + recordKeyFilters + } else { + Seq.empty + } } /** @@ -107,46 +120,143 @@ class SparkFileFormatInternalRowReaderContext(readerMaps: mutable.Map[Long, Part } override def mergeBootstrapReaders(skeletonFileIterator: ClosableIterator[InternalRow], - dataFileIterator: ClosableIterator[InternalRow]): ClosableIterator[InternalRow] = { - doBootstrapMerge(skeletonFileIterator.asInstanceOf[ClosableIterator[Any]], - dataFileIterator.asInstanceOf[ClosableIterator[Any]]) + skeletonRequiredSchema: Schema, + dataFileIterator: ClosableIterator[InternalRow], + dataRequiredSchema: Schema): ClosableIterator[InternalRow] = { + doBootstrapMerge(skeletonFileIterator.asInstanceOf[ClosableIterator[Any]], skeletonRequiredSchema, + dataFileIterator.asInstanceOf[ClosableIterator[Any]], dataRequiredSchema) } - protected def doBootstrapMerge(skeletonFileIterator: ClosableIterator[Any], dataFileIterator: ClosableIterator[Any]): ClosableIterator[InternalRow] = { - new ClosableIterator[Any] { - val combinedRow = new JoinedRow() + protected def doBootstrapMerge(skeletonFileIterator: ClosableIterator[Any], + skeletonRequiredSchema: Schema, + dataFileIterator: ClosableIterator[Any], + dataRequiredSchema: Schema): ClosableIterator[InternalRow] = { + if (shouldUseRecordPosition) { + assert(AvroSchemaUtils.containsFieldInSchema(skeletonRequiredSchema, ROW_INDEX_TEMPORARY_COLUMN_NAME)) + val javaSet = new java.util.HashSet[String]() + javaSet.add(ROW_INDEX_TEMPORARY_COLUMN_NAME) + val skeletonProjection = projectRecord(skeletonRequiredSchema, + AvroSchemaUtils.removeFieldsFromSchema(skeletonRequiredSchema, javaSet)) + new CachingIterator[InternalRow] { + val combinedRow = new JoinedRow() - override def hasNext: Boolean = { - //If the iterators are out of sync it is probably due to filter pushdown - checkState(dataFileIterator.hasNext == skeletonFileIterator.hasNext, - "Bootstrap data-file iterator and skeleton-file iterator have to be in-sync!") - dataFileIterator.hasNext && skeletonFileIterator.hasNext - } + private def getPos(row: InternalRow): Long = { + row.getLong(row.numFields-1) + } + + private def getNextSkeleton: (InternalRow, Long) = { + val nextSkeletonRow = skeletonFileIterator.next().asInstanceOf[InternalRow] + (nextSkeletonRow, getPos(nextSkeletonRow)) + } - override def next(): Any = { - (skeletonFileIterator.next(), dataFileIterator.next()) match { - case (s: ColumnarBatch, d: ColumnarBatch) => - val numCols = s.numCols() + d.numCols() - val vecs: Array[ColumnVector] = new Array[ColumnVector](numCols) - for (i <- 0 until numCols) { - if (i < s.numCols()) { - vecs(i) = s.column(i) + private def getNextData: (InternalRow, Long) = { + val nextSkeletonRow = skeletonFileIterator.next().asInstanceOf[InternalRow] + (nextSkeletonRow, getPos(nextSkeletonRow)) + } + + override def close(): Unit = { + skeletonFileIterator.close() + dataFileIterator.close() + } + + override protected def doHasNext(): Boolean = { + if (!dataFileIterator.hasNext || !skeletonFileIterator.hasNext) { + false + } else { + var nextSkeleton = getNextSkeleton + var nextData = getNextData + while (nextSkeleton._2 != nextData._2) { + if (nextSkeleton._2 > nextData._2) { + if (!dataFileIterator.hasNext) { + return false + } else { + nextData = getNextData + } } else { - vecs(i) = d.column(i - s.numCols()) + if (!skeletonFileIterator.hasNext) { + return false + } else { + nextSkeleton = getNextSkeleton + } } } - assert(s.numRows() == d.numRows()) - sparkAdapter.makeColumnarBatch(vecs, s.numRows()) - case (_: ColumnarBatch, _: InternalRow) => throw new IllegalStateException("InternalRow ColumnVector mismatch") - case (_: InternalRow, _: ColumnarBatch) => throw new IllegalStateException("InternalRow ColumnVector mismatch") - case (s: InternalRow, d: InternalRow) => combinedRow(s, d) + nextRecord = combinedRow(skeletonProjection.apply(nextSkeleton._1), nextData._1) + true + } } } + } else { + new ClosableIterator[Any] { + val combinedRow = new JoinedRow() - override def close(): Unit = { - skeletonFileIterator.close() - dataFileIterator.close() - } - }.asInstanceOf[ClosableIterator[InternalRow]] + override def hasNext: Boolean = { + //If the iterators are out of sync it is probably due to filter pushdown + checkState(dataFileIterator.hasNext == skeletonFileIterator.hasNext, + "Bootstrap data-file iterator and skeleton-file iterator have to be in-sync!") + dataFileIterator.hasNext && skeletonFileIterator.hasNext + } + + override def next(): Any = { + (skeletonFileIterator.next(), dataFileIterator.next()) match { + case (s: ColumnarBatch, d: ColumnarBatch) => + val numCols = s.numCols() + d.numCols() + val vecs: Array[ColumnVector] = new Array[ColumnVector](numCols) + for (i <- 0 until numCols) { + if (i < s.numCols()) { + vecs(i) = s.column(i) + } else { + vecs(i) = d.column(i - s.numCols()) + } + } + assert(s.numRows() == d.numRows()) + sparkAdapter.makeColumnarBatch(vecs, s.numRows()) + case (_: ColumnarBatch, _: InternalRow) => throw new IllegalStateException("InternalRow ColumnVector mismatch") + case (_: InternalRow, _: ColumnarBatch) => throw new IllegalStateException("InternalRow ColumnVector mismatch") + case (s: InternalRow, d: InternalRow) => combinedRow(s, d) + } + } + + override def close(): Unit = { + skeletonFileIterator.close() + dataFileIterator.close() + } + }.asInstanceOf[ClosableIterator[InternalRow]] + } + + } + + override def shouldUseRecordPositionMerging(): Boolean = { + shouldUseRecordPosition } + } + +object SparkFileFormatInternalRowReaderContext { + // From "ParquetFileFormat.scala": The names of the field for record position. + private val ROW_INDEX = "row_index" + private val ROW_INDEX_TEMPORARY_COLUMN_NAME = s"_tmp_metadata_$ROW_INDEX" + + // From "namedExpressions.scala": Used to construct to record position field metadata. + private val FILE_SOURCE_GENERATED_METADATA_COL_ATTR_KEY = "__file_source_generated_metadata_col" + private val FILE_SOURCE_METADATA_COL_ATTR_KEY = "__file_source_metadata_col" + private val METADATA_COL_ATTR_KEY = "__metadata_col" + + def getRecordKeyRelatedFilters(filters: Seq[Filter], recordKeyColumn: String): Seq[Filter] = { + filters.filter(f => f.references.exists(c => c.equalsIgnoreCase(recordKeyColumn))) + } + + def getAppliedRequiredSchema(requiredSchema: StructType, + shouldUseRecordPosition: Boolean): StructType = { + if (shouldUseRecordPosition) { + val metadata = new MetadataBuilder() + .putString(METADATA_COL_ATTR_KEY, ROW_INDEX_TEMPORARY_COLUMN_NAME) + .putBoolean(FILE_SOURCE_METADATA_COL_ATTR_KEY, value = true) + .putString(FILE_SOURCE_GENERATED_METADATA_COL_ATTR_KEY, ROW_INDEX_TEMPORARY_COLUMN_NAME) + .build() + val rowIndexField = StructField(ROW_INDEX_TEMPORARY_COLUMN_NAME, LongType, nullable = false, metadata) + StructType(requiredSchema.fields :+ rowIndexField) + } else { + requiredSchema + } + } +} \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java index 027f6ccb37d2..03a07c84a511 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java @@ -231,7 +231,13 @@ private static Option findNestedField(Schema schema, String[] fiel if (!nestedPart.isPresent()) { return Option.empty(); } - return nestedPart; + boolean isUnion = false; + if (foundSchema.getType().equals(Schema.Type.UNION)) { + isUnion = true; + foundSchema = resolveNullableSchema(foundSchema); + } + Schema newSchema = Schema.createRecord(foundSchema.getName(), foundSchema.getDoc(), foundSchema.getNamespace(), false, Collections.singletonList(nestedPart.get())); + return Option.of(new Schema.Field(foundField.name(), isUnion ? createNullableSchema(newSchema) : newSchema, foundField.doc(), foundField.defaultVal())); } public static Schema appendFieldsToSchemaDedupNested(Schema schema, List newFields) { @@ -291,6 +297,16 @@ private static Schema appendFieldsToSchemaBase(Schema schema, List return newSchema; } + public static Schema removeFieldsFromSchema(Schema schema, Set fieldNames) { + List fields = schema.getFields().stream() + .filter(field -> !fieldNames.contains(field.name())) + .map(field -> new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultVal())) + .collect(Collectors.toList()); + Schema newSchema = Schema.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(), schema.isError()); + newSchema.setFields(fields); + return newSchema; + } + /** * Passed in {@code Union} schema and will try to resolve the field with the {@code fieldSchemaFullName} * w/in the union returning its corresponding schema diff --git a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java index 86a875c9df33..c0c0e40206f7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java @@ -79,7 +79,7 @@ public abstract class HoodieReaderContext { * @return {@link ClosableIterator} that can return all records through iteration. */ public abstract ClosableIterator getFileRecordIterator( - Path filePath, long start, long length, Schema dataSchema, Schema requiredSchema, Configuration conf) throws IOException; + Path filePath, long start, long length, Schema dataSchema, Schema requiredSchema, Configuration conf, boolean isMerge) throws IOException; /** * Converts an Avro record, e.g., serialized in the log files, to an engine-specific record. @@ -199,7 +199,10 @@ public Map updateSchemaAndResetOrderingValInMetadata(Map mergeBootstrapReaders(ClosableIterator skeletonFileIterator, ClosableIterator dataFileIterator); + public abstract ClosableIterator mergeBootstrapReaders(ClosableIterator skeletonFileIterator, + Schema skeletonRequiredSchema, + ClosableIterator dataFileIterator, + Schema dataRequiredSchema); /** * Creates a function that will reorder records of schema "from" to schema of "to" @@ -211,6 +214,10 @@ public Map updateSchemaAndResetOrderingValInMetadata(Map projectRecord(Schema from, Schema to); + public UnaryOperator projectRecordUnsafe(Schema from, Schema to, Map renamedColumns) { + throw new UnsupportedOperationException("Schema on read is not supported for this reader."); + } + /** * Extracts the record position value from the record itself. * @@ -220,6 +227,13 @@ public long extractRecordPosition(T record, Schema schema, String fieldName, lon return providedPositionIfNeeded; } + /** + * returns true if record position should be used for merging + */ + public boolean shouldUseRecordPositionMerging() { + return false; + } + /** * Constructs engine specific delete record. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java index 77f784dfc388..03828f951077 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java @@ -184,7 +184,7 @@ protected ClosableIterator readRecordsFromBlockPayload(HoodieReaderContex Schema writerSchema = new Schema.Parser().parse(this.getLogBlockHeader().get(HeaderMetadataType.SCHEMA)); return readerContext.getFileRecordIterator( - inlineLogFilePath, 0, blockContentLoc.getBlockSize(), writerSchema, readerSchema, inlineConf); + inlineLogFilePath, 0, blockContentLoc.getBlockSize(), writerSchema, readerSchema, inlineConf, false); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java index 70ddb5abff2f..1ee4b2e36d9b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java @@ -24,20 +24,26 @@ import org.apache.hudi.common.model.DeleteRecord; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.KeySpec; import org.apache.hudi.common.table.log.block.HoodieDataBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.common.util.DefaultSizeEstimator; import org.apache.hudi.common.util.HoodieRecordSizeEstimator; +import org.apache.hudi.common.util.InternalSchemaCache; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.collection.ClosableIterator; +import org.apache.hudi.common.util.collection.CloseableMappingIterator; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieCorruptedDataException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieKeyException; import org.apache.hudi.exception.HoodieValidationException; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.action.InternalSchemaMerger; +import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter; import org.apache.avro.Schema; import org.roaringbitmap.longlong.Roaring64NavigableMap; @@ -49,8 +55,10 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Function; import static org.apache.hudi.common.engine.HoodieReaderContext.INTERNAL_META_SCHEMA; +import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME; public abstract class HoodieBaseFileGroupRecordBuffer implements HoodieFileGroupRecordBuffer { protected final HoodieReaderContext readerContext; @@ -65,10 +73,14 @@ public abstract class HoodieBaseFileGroupRecordBuffer implements HoodieFileGr protected Iterator, Map>> logRecordIterator; protected T nextRecord; protected boolean enablePartialMerging = false; + protected InternalSchema internalSchema; + protected HoodieTableMetaClient hoodieTableMetaClient; public HoodieBaseFileGroupRecordBuffer(HoodieReaderContext readerContext, Schema readerSchema, Schema baseFileSchema, + InternalSchema internalSchema, + HoodieTableMetaClient hoodieTableMetaClient, Option partitionNameOverrideOpt, Option partitionPathFieldOpt, HoodieRecordMerger recordMerger, @@ -84,6 +96,9 @@ public HoodieBaseFileGroupRecordBuffer(HoodieReaderContext readerContext, this.partitionPathFieldOpt = partitionPathFieldOpt; this.recordMerger = recordMerger; this.payloadProps = payloadProps; + this.internalSchema = internalSchema == null || internalSchema.isEmptySchema() + ? InternalSchema.getEmptyInternalSchema() : AvroInternalSchemaConverter.pruneAvroSchemaToInternalSchema(readerSchema, internalSchema); + this.hoodieTableMetaClient = hoodieTableMetaClient; try { // Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, new DefaultSizeEstimator<>(), @@ -173,7 +188,7 @@ protected Option>> doProcessNextDataRecord(T record, if (!combinedRecordAndSchemaOpt.isPresent()) { return Option.empty(); } - + Pair combinedRecordAndSchema = combinedRecordAndSchemaOpt.get(); HoodieRecord combinedRecord = combinedRecordAndSchema.getLeft(); @@ -232,7 +247,6 @@ protected Option doProcessNextDeletedRecord(DeleteRecord deleteRec * @param dataBlock * @param keySpecOpt * @return - * @throws IOException */ protected Pair, Schema> getRecordsIterator(HoodieDataBlock dataBlock, Option keySpecOpt) { ClosableIterator blockRecordsIterator; @@ -242,7 +256,44 @@ protected Pair, Schema> getRecordsIterator(HoodieDataBlock d } else { blockRecordsIterator = dataBlock.getEngineRecordIterator(readerContext); } - return Pair.of(blockRecordsIterator, dataBlock.getSchema()); + Option, Schema>> schemaEvolutionTransformerOpt = + composeEvolvedSchemaTransformer(dataBlock); + + // In case when schema has been evolved original persisted records will have to be + // transformed to adhere to the new schema + Function transformer = + schemaEvolutionTransformerOpt.map(Pair::getLeft) + .orElse(Function.identity()); + + Schema schema = schemaEvolutionTransformerOpt.map(Pair::getRight) + .orElseGet(dataBlock::getSchema); + + return Pair.of(new CloseableMappingIterator<>(blockRecordsIterator, transformer), schema); + } + + /** + * Get final Read Schema for support evolution. + * step1: find the fileSchema for current dataBlock. + * step2: determine whether fileSchema is compatible with the final read internalSchema. + * step3: merge fileSchema and read internalSchema to produce final read schema. + * + * @param dataBlock current processed block + * @return final read schema. + */ + protected Option, Schema>> composeEvolvedSchemaTransformer( + HoodieDataBlock dataBlock) { + if (internalSchema.isEmptySchema()) { + return Option.empty(); + } + + long currentInstantTime = Long.parseLong(dataBlock.getLogBlockHeader().get(INSTANT_TIME)); + InternalSchema fileSchema = InternalSchemaCache.searchSchemaAndCache(currentInstantTime, + hoodieTableMetaClient, false); + Pair> mergedInternalSchema = new InternalSchemaMerger(fileSchema, internalSchema, + true, false, false).mergeSchemaGetRenamed(); + Schema mergedAvroSchema = AvroInternalSchemaConverter.convert(mergedInternalSchema.getLeft(), readerSchema.getFullName()); + assert mergedAvroSchema.equals(readerSchema); + return Option.of(Pair.of(readerContext.projectRecordUnsafe(dataBlock.getSchema(), mergedAvroSchema, mergedInternalSchema.getRight()), mergedAvroSchema)); } /** @@ -256,7 +307,7 @@ protected Pair, Schema> getRecordsIterator(HoodieDataBlock d * @throws IOException */ protected Option merge(Option older, Map olderInfoMap, - Option newer, Map newerInfoMap) throws IOException { + Option newer, Map newerInfoMap) throws IOException { if (!older.isPresent()) { return newer; } @@ -275,6 +326,9 @@ protected Option merge(Option older, Map olderInfoMap, if (mergedRecord.isPresent() && !mergedRecord.get().getLeft().isDelete(mergedRecord.get().getRight(), payloadProps)) { + if (!mergedRecord.get().getRight().equals(readerSchema)) { + return Option.ofNullable((T) mergedRecord.get().getLeft().rewriteRecordWithNewSchema(mergedRecord.get().getRight(), null, readerSchema).getData()); + } return Option.ofNullable((T) mergedRecord.get().getLeft().getData()); } return Option.empty(); @@ -333,4 +387,4 @@ protected static List extractRecordPositions(HoodieLogBlock logBlock) thro return blockPositions; } -} +} \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java index 3984177bdbfa..010f6ad4bb1e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java @@ -30,6 +30,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieMergedLogRecordReader; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.CachingIterator; @@ -38,6 +39,7 @@ import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.internal.schema.InternalSchema; import org.apache.avro.Schema; import org.apache.hadoop.conf.Configuration; @@ -50,11 +52,12 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.function.UnaryOperator; import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchema; +import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchemaDedupNested; import static org.apache.hudi.avro.AvroSchemaUtils.findNestedField; import static org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath; import static org.apache.hudi.common.util.ConfigUtils.getBooleanWithAltKeys; @@ -97,6 +100,13 @@ public final class HoodieFileGroupReader implements Closeable { private final Option> outputConverter; + private final InternalSchema internalSchema; + + private boolean needMORMerge; + private boolean needMerge; + + private final boolean shouldMergeWithPosition; + public HoodieFileGroupReader(HoodieReaderContext readerContext, Configuration hadoopConf, String tablePath, @@ -104,6 +114,8 @@ public HoodieFileGroupReader(HoodieReaderContext readerContext, FileSlice fileSlice, Schema dataSchema, Schema requestedSchema, + Option internalSchemaOpt, + HoodieTableMetaClient hoodieTableMetaClient, TypedProperties props, HoodieTableConfig tableConfig, long start, @@ -113,6 +125,7 @@ public HoodieFileGroupReader(HoodieReaderContext readerContext, String spillableMapBasePath, ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled) { + this.internalSchema = internalSchemaOpt.orElse(InternalSchema.getEmptyInternalSchema()); this.readerContext = readerContext; this.hadoopConf = hadoopConf; this.hoodieBaseFileOption = fileSlice.getBaseFile(); @@ -126,7 +139,8 @@ public HoodieFileGroupReader(HoodieReaderContext readerContext, this.dataSchema = dataSchema; this.requestedSchema = requestedSchema; this.hoodieTableConfig = tableConfig; - this.requiredSchema = generateRequiredSchema(); + this.shouldMergeWithPosition = shouldUseRecordPosition && readerContext.shouldUseRecordPositionMerging(); + this.requiredSchema = prepareSchema(); if (!requestedSchema.equals(requiredSchema)) { this.outputConverter = Option.of(readerContext.projectRecord(requiredSchema, requestedSchema)); } else { @@ -137,12 +151,12 @@ public HoodieFileGroupReader(HoodieReaderContext readerContext, this.readerState.mergeProps.putAll(props); this.recordBuffer = this.logFiles.isEmpty() ? null - : shouldUseRecordPosition + : this.shouldMergeWithPosition ? new HoodiePositionBasedFileGroupRecordBuffer<>( - readerContext, requiredSchema, requiredSchema, Option.empty(), Option.empty(), + readerContext, requiredSchema, requiredSchema, internalSchema, hoodieTableMetaClient, Option.empty(), Option.empty(), recordMerger, props, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled) : new HoodieKeyBasedFileGroupRecordBuffer<>( - readerContext, requiredSchema, requiredSchema, Option.empty(), Option.empty(), + readerContext, requiredSchema, requiredSchema, internalSchema, hoodieTableMetaClient, Option.empty(), Option.empty(), recordMerger, props, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled); } @@ -171,18 +185,18 @@ private ClosableIterator makeBaseFileIterator() throws IOException { } return readerContext.getFileRecordIterator(baseFile.getHadoopPath(), start, length, - dataSchema, requiredSchema, hadoopConf); + dataSchema, requiredSchema, hadoopConf, this.needMerge); } private Schema generateRequiredSchema() { //might need to change this if other queries than mor have mandatory fields - if (logFiles.isEmpty()) { + if (!needMORMerge) { return requestedSchema; } List addedFields = new ArrayList<>(); for (String field : recordMerger.getMandatoryFieldsForMerging(hoodieTableConfig)) { - if (requestedSchema.getField(field) == null) { + if (!findNestedField(requestedSchema, field).isPresent()) { Option foundFieldOpt = findNestedField(dataSchema, field); if (!foundFieldOpt.isPresent()) { throw new IllegalArgumentException("Field: " + field + " does not exist in the table schema"); @@ -193,25 +207,40 @@ private Schema generateRequiredSchema() { } if (addedFields.isEmpty()) { - return maybeReorderForBootstrap(requestedSchema); + return requestedSchema; } - return maybeReorderForBootstrap(appendFieldsToSchema(requestedSchema, addedFields)); + return appendFieldsToSchemaDedupNested(requestedSchema, addedFields); } - private Schema maybeReorderForBootstrap(Schema input) { - if (this.hoodieBaseFileOption.isPresent() && this.hoodieBaseFileOption.get().getBootstrapBaseFile().isPresent()) { - Pair, List> requiredFields = getDataAndMetaCols(input); - if (!(requiredFields.getLeft().isEmpty() || requiredFields.getRight().isEmpty())) { - return createSchemaFromFields(Stream.concat(requiredFields.getLeft().stream(), requiredFields.getRight().stream()) - .collect(Collectors.toList())); - } - } - return input; + private Schema prepareSchema() { + this.needMORMerge = !logFiles.isEmpty(); + Schema preReorderRequiredSchema = generateRequiredSchema(); + Pair, List> requiredFields = getDataAndMetaCols(preReorderRequiredSchema); + boolean needBootstrapMerge = hoodieBaseFileOption.isPresent() && hoodieBaseFileOption.get().getBootstrapBaseFile().isPresent() + && !requiredFields.getLeft().isEmpty() && !requiredFields.getRight().isEmpty(); + this.needMerge = needBootstrapMerge || this.needMORMerge; + Schema preMergeSchema = needBootstrapMerge + ? createSchemaFromFields(Stream.concat(requiredFields.getLeft().stream(), requiredFields.getRight().stream()).collect(Collectors.toList())) + : preReorderRequiredSchema; + return this.shouldMergeWithPosition && this.needMerge + ? addPositionalMergeCol(preMergeSchema) + : preMergeSchema; + } + + private Schema addPositionalMergeCol(Schema input) { + return appendFieldsToSchemaDedupNested(input, Collections.singletonList(getPositionalMergeField())); + } + + private Schema.Field getPositionalMergeField() { + return new Schema.Field(HoodiePositionBasedFileGroupRecordBuffer.ROW_INDEX_TEMPORARY_COLUMN_NAME, + Schema.create(Schema.Type.LONG), "", 0L); } private static Pair, List> getDataAndMetaCols(Schema schema) { Map> fieldsByMeta = schema.getFields().stream() + //if there are no data fields, then we don't want to think the temp col is a data col + .filter(f -> !Objects.equals(f.name(), HoodiePositionBasedFileGroupRecordBuffer.ROW_INDEX_TEMPORARY_COLUMN_NAME)) .collect(Collectors.partitioningBy(f -> HoodieRecord.HOODIE_META_COLUMNS_WITH_OPERATION.contains(f.name()))); return Pair.of(fieldsByMeta.getOrDefault(true, Collections.emptyList()), fieldsByMeta.getOrDefault(false, Collections.emptyList())); @@ -232,23 +261,34 @@ private ClosableIterator makeBootstrapBaseFileIterator(HoodieBaseFile baseFil BaseFile dataFile = baseFile.getBootstrapBaseFile().get(); Pair,List> requiredFields = getDataAndMetaCols(requiredSchema); Pair,List> allFields = getDataAndMetaCols(dataSchema); - - Option> dataFileIterator = requiredFields.getRight().isEmpty() ? Option.empty() : - Option.of(readerContext.getFileRecordIterator(dataFile.getHadoopPath(), 0, dataFile.getFileLen(), - createSchemaFromFields(allFields.getRight()), createSchemaFromFields(requiredFields.getRight()), hadoopConf)); - - Option> skeletonFileIterator = requiredFields.getLeft().isEmpty() ? Option.empty() : - Option.of(readerContext.getFileRecordIterator(baseFile.getHadoopPath(), 0, baseFile.getFileLen(), - createSchemaFromFields(allFields.getLeft()), createSchemaFromFields(requiredFields.getLeft()), hadoopConf)); + Option,Schema>> dataFileIterator = + makeBootstrapBaseFileIteratorHelper(requiredFields.getRight(), allFields.getRight(), dataFile); + Option,Schema>> skeletonFileIterator = + makeBootstrapBaseFileIteratorHelper(requiredFields.getLeft(), allFields.getLeft(), baseFile); if (!dataFileIterator.isPresent() && !skeletonFileIterator.isPresent()) { throw new IllegalStateException("should not be here if only partition cols are required"); } else if (!dataFileIterator.isPresent()) { - return skeletonFileIterator.get(); + return skeletonFileIterator.get().getLeft(); } else if (!skeletonFileIterator.isPresent()) { - return dataFileIterator.get(); + return dataFileIterator.get().getLeft(); } else { - return readerContext.mergeBootstrapReaders(skeletonFileIterator.get(), dataFileIterator.get()); + return readerContext.mergeBootstrapReaders(skeletonFileIterator.get().getLeft(), skeletonFileIterator.get().getRight(), + dataFileIterator.get().getLeft(), dataFileIterator.get().getRight()); + } + } + + private Option,Schema>> makeBootstrapBaseFileIteratorHelper(List requiredFields, + List allFields, + BaseFile file) throws IOException { + if (requiredFields.isEmpty()) { + return Option.empty(); + } + if (needMerge && shouldMergeWithPosition) { + requiredFields.add(getPositionalMergeField()); } + Schema requiredSchema = createSchemaFromFields(requiredFields); + return Option.of(Pair.of(readerContext.getFileRecordIterator(file.getHadoopPath(), 0, file.getFileLen(), + createSchemaFromFields(allFields), requiredSchema, hadoopConf, this.needMerge), requiredSchema)); } /** @@ -285,6 +325,7 @@ private void scanLogFiles() { .withLogFiles(logFiles) .withLatestInstantTime(readerState.latestCommitTime) .withReaderSchema(readerState.logRecordAvroSchema) + .withInternalSchema(internalSchema) .withReadBlocksLazily(getBooleanWithAltKeys(props, HoodieReaderConfig.COMPACTION_LAZY_BLOCK_READ_ENABLE)) .withReverseReader(false) .withBufferSize(getIntWithAltKeys(props, HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE)) @@ -342,4 +383,4 @@ public void close() { } } } -} +} \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieKeyBasedFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieKeyBasedFileGroupRecordBuffer.java index 0430a42e8639..f651578613cf 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieKeyBasedFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieKeyBasedFileGroupRecordBuffer.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.engine.HoodieReaderContext; import org.apache.hudi.common.model.DeleteRecord; import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.KeySpec; import org.apache.hudi.common.table.log.block.HoodieDataBlock; import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; @@ -31,6 +32,7 @@ import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.internal.schema.InternalSchema; import org.apache.avro.Schema; @@ -51,6 +53,8 @@ public class HoodieKeyBasedFileGroupRecordBuffer extends HoodieBaseFileGroupR public HoodieKeyBasedFileGroupRecordBuffer(HoodieReaderContext readerContext, Schema readerSchema, Schema baseFileSchema, + InternalSchema internalSchema, + HoodieTableMetaClient hoodieTableMetaClient, Option partitionNameOverrideOpt, Option partitionPathFieldOpt, HoodieRecordMerger recordMerger, @@ -59,7 +63,7 @@ public HoodieKeyBasedFileGroupRecordBuffer(HoodieReaderContext readerContext, String spillableMapBasePath, ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled) { - super(readerContext, readerSchema, baseFileSchema, partitionNameOverrideOpt, partitionPathFieldOpt, + super(readerContext, readerSchema, baseFileSchema, internalSchema, hoodieTableMetaClient, partitionNameOverrideOpt, partitionPathFieldOpt, recordMerger, payloadProps, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled); } @@ -165,4 +169,4 @@ protected boolean doHasNext() throws IOException { } return false; } -} +} \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java index 50e969343e15..a9b77a9130a0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.engine.HoodieReaderContext; import org.apache.hudi.common.model.DeleteRecord; import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.KeySpec; import org.apache.hudi.common.table.log.block.HoodieDataBlock; import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; @@ -31,6 +32,7 @@ import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.internal.schema.InternalSchema; import org.apache.avro.Schema; @@ -43,6 +45,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Function; import static org.apache.hudi.common.model.HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID; /** @@ -52,12 +55,15 @@ * {@link #hasNext} method is called. */ public class HoodiePositionBasedFileGroupRecordBuffer extends HoodieBaseFileGroupRecordBuffer { - private static final String ROW_INDEX_COLUMN_NAME = "row_index"; + public static final String ROW_INDEX_COLUMN_NAME = "row_index"; + public static final String ROW_INDEX_TEMPORARY_COLUMN_NAME = "_tmp_metadata_" + ROW_INDEX_COLUMN_NAME; private long nextRecordPosition = 0L; public HoodiePositionBasedFileGroupRecordBuffer(HoodieReaderContext readerContext, Schema readerSchema, Schema baseFileSchema, + InternalSchema internalSchema, + HoodieTableMetaClient hoodieTableMetaClient, Option partitionNameOverrideOpt, Option partitionPathFieldOpt, HoodieRecordMerger recordMerger, @@ -66,7 +72,7 @@ public HoodiePositionBasedFileGroupRecordBuffer(HoodieReaderContext readerCon String spillableMapBasePath, ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled) { - super(readerContext, readerSchema, baseFileSchema, partitionNameOverrideOpt, partitionPathFieldOpt, + super(readerContext, readerSchema, baseFileSchema, internalSchema, hoodieTableMetaClient, partitionNameOverrideOpt, partitionPathFieldOpt, recordMerger, payloadProps, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled); } @@ -92,10 +98,22 @@ public void processDataBlock(HoodieDataBlock dataBlock, Option keySpecO // partial merging. enablePartialMerging = true; } - + // Extract positions from data block. List recordPositions = extractRecordPositions(dataBlock); + Option, Schema>> schemaEvolutionTransformerOpt = + composeEvolvedSchemaTransformer(dataBlock); + + // In case when schema has been evolved original persisted records will have to be + // transformed to adhere to the new schema + Function transformer = + schemaEvolutionTransformerOpt.map(Pair::getLeft) + .orElse(Function.identity()); + + Schema evolvedSchema = schemaEvolutionTransformerOpt.map(Pair::getRight) + .orElseGet(dataBlock::getSchema); + // TODO: return an iterator that can generate sequence number with the record. // Then we can hide this logic into data block. try (ClosableIterator recordIterator = dataBlock.getEngineRecordIterator(readerContext)) { @@ -110,9 +128,11 @@ public void processDataBlock(HoodieDataBlock dataBlock, Option keySpecO } long recordPosition = recordPositions.get(recordIndex++); + + T evolvedNextRecord = transformer.apply(nextRecord); processNextDataRecord( - nextRecord, - readerContext.generateMetadataForRecord(nextRecord, readerSchema), + evolvedNextRecord, + readerContext.generateMetadataForRecord(evolvedNextRecord, evolvedSchema), recordPosition ); } @@ -209,4 +229,4 @@ protected boolean doHasNext() throws IOException { return false; } -} +} \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieUnmergedFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieUnmergedFileGroupRecordBuffer.java index 76aa28308c44..e7e47ba0c98c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieUnmergedFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieUnmergedFileGroupRecordBuffer.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.engine.HoodieReaderContext; import org.apache.hudi.common.model.DeleteRecord; import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.KeySpec; import org.apache.hudi.common.table.log.block.HoodieDataBlock; import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; @@ -32,6 +33,7 @@ import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.internal.schema.InternalSchema; import org.apache.avro.Schema; @@ -50,6 +52,8 @@ public HoodieUnmergedFileGroupRecordBuffer( HoodieReaderContext readerContext, Schema readerSchema, Schema baseFileSchema, + InternalSchema internalSchema, + HoodieTableMetaClient hoodieTableMetaClient, Option partitionNameOverrideOpt, Option partitionPathFieldOpt, HoodieRecordMerger recordMerger, @@ -58,7 +62,7 @@ public HoodieUnmergedFileGroupRecordBuffer( String spillableMapBasePath, ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled) { - super(readerContext, readerSchema, baseFileSchema, partitionNameOverrideOpt, partitionPathFieldOpt, + super(readerContext, readerSchema, baseFileSchema, internalSchema, hoodieTableMetaClient, partitionNameOverrideOpt, partitionPathFieldOpt, recordMerger, payloadProps, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled); } @@ -143,4 +147,4 @@ public void processNextDeletedRecord(DeleteRecord deleteRecord, Serializable ind public boolean containsLogRecord(String recordKey) { return records.containsKey(recordKey); } -} +} \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java index 9ed55a7e5739..fc3ba94ea594 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java @@ -19,12 +19,15 @@ package org.apache.hudi.internal.schema.action; import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.internal.schema.Type; import org.apache.hudi.internal.schema.Types; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; /** * Auxiliary class. @@ -59,6 +62,8 @@ public class InternalSchemaMerger { // we can pass colNewName to reWriteRecordWithNewSchema directly, everything is ok. private boolean useColNameFromFileSchema = true; + private final Map renamedFields = new HashMap<>(); + public InternalSchemaMerger(InternalSchema fileSchema, InternalSchema querySchema, boolean ignoreRequiredAttribute, boolean useColumnTypeFromFileSchema, boolean useColNameFromFileSchema) { this.fileSchema = fileSchema; this.querySchema = querySchema; @@ -81,6 +86,17 @@ public InternalSchema mergeSchema() { return new InternalSchema(record); } + /** + * Create final read schema to read avro/parquet file. + * + * @return read schema to read avro/parquet file. + */ + public Pair> mergeSchemaGetRenamed() { + Types.RecordType record = (Types.RecordType) mergeType(querySchema.getRecord(), 0); + InternalSchema internalSchema = new InternalSchema(record); + return Pair.of(internalSchema, renamedFields); + } + /** * Create final read schema to read avro/parquet file. * this is auxiliary function used by mergeSchema. @@ -150,6 +166,9 @@ private Types.Field dealWithRename(int fieldId, Type newType, Types.Field oldFie String nameFromQuerySchema = querySchema.findField(fieldId).name(); String finalFieldName = useColNameFromFileSchema ? nameFromFileSchema : nameFromQuerySchema; Type typeFromFileSchema = fieldFromFileSchema.type(); + if (!useColNameFromFileSchema) { + renamedFields.put(nameFromQuerySchema, nameFromFileSchema); + } // Current design mechanism guarantees nestedType change is not allowed, so no need to consider. if (newType.isNestedType()) { return Types.Field.get(oldField.fieldId(), oldField.isOptional(), @@ -214,4 +233,3 @@ private Type buildPrimitiveType(Type.PrimitiveType typeFromQuerySchema, int curr } } } - diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java index 0f8d1606f2ad..fc28554ff9bc 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java @@ -18,12 +18,14 @@ package org.apache.hudi.internal.schema.convert; +import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieNullSchemaTypeException; import org.apache.hudi.internal.schema.HoodieSchemaException; import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.internal.schema.Type; import org.apache.hudi.internal.schema.Types; +import org.apache.hudi.internal.schema.utils.InternalSchemaUtils; import org.apache.avro.JsonProperties; import org.apache.avro.LogicalType; @@ -39,6 +41,7 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; +import static org.apache.avro.Schema.Type.STRING; import static org.apache.avro.Schema.Type.UNION; /** @@ -70,6 +73,75 @@ public static Schema convert(InternalSchema internalSchema, String name) { return buildAvroSchemaFromInternalSchema(internalSchema, name); } + public static InternalSchema pruneAvroSchemaToInternalSchema(Schema schema, InternalSchema originSchema) { + List pruneNames = collectColNamesFromSchema(schema); + return InternalSchemaUtils.pruneInternalSchema(originSchema, pruneNames); + } + + /** + * Collect all the leaf nodes names. + * + * @param schema a avro schema. + * @return leaf nodes full names. + */ + private static List collectColNamesFromSchema(Schema schema) { + List result = new ArrayList<>(); + Deque visited = new LinkedList<>(); + collectColNamesFromAvroSchema(schema, visited, result); + return result; + } + + private static void collectColNamesFromAvroSchema(Schema schema, Deque visited, List resultSet) { + switch (schema.getType()) { + case RECORD: + List fields = schema.getFields(); + for (Schema.Field f : fields) { + visited.push(f.name()); + collectColNamesFromAvroSchema(f.schema(), visited, resultSet); + visited.pop(); + addFullName(f.schema(), f.name(), visited, resultSet); + } + return; + + case UNION: + collectColNamesFromAvroSchema(AvroSchemaUtils.resolveNullableSchema(schema), visited, resultSet); + return; + + case ARRAY: + visited.push("element"); + collectColNamesFromAvroSchema(schema.getElementType(), visited, resultSet); + visited.pop(); + addFullName(schema.getElementType(), "element", visited, resultSet); + return; + + case MAP: + addFullName(STRING, "key", visited, resultSet); + visited.push("value"); + collectColNamesFromAvroSchema(schema.getValueType(), visited, resultSet); + visited.pop(); + addFullName(schema.getValueType(), "value", visited, resultSet); + return; + + default: + return; + } + } + + private static void addFullName(Schema schema, String name, Deque visited, List resultSet) { + addFullName(AvroSchemaUtils.resolveNullableSchema(schema).getType(), name, visited, resultSet); + } + + private static void addFullName(Schema.Type type, String name, Deque visited, List resultSet) { + switch (type) { + case RECORD: + case ARRAY: + case MAP: + return; + default: + resultSet.add(InternalSchemaUtils.createFullName(name, visited)); + } + } + /** * Converting from avro -> internal schema -> avro * causes null to always be first in unions. @@ -254,20 +326,20 @@ private static Type visitAvroPrimitiveToBuildInternalType(Schema primitive) { String name = logical.getName(); if (logical instanceof LogicalTypes.Decimal) { return Types.DecimalType.get( - ((LogicalTypes.Decimal) logical).getPrecision(), - ((LogicalTypes.Decimal) logical).getScale()); + ((LogicalTypes.Decimal) logical).getPrecision(), + ((LogicalTypes.Decimal) logical).getScale()); } else if (logical instanceof LogicalTypes.Date) { return Types.DateType.get(); } else if ( - logical instanceof LogicalTypes.TimeMillis - || logical instanceof LogicalTypes.TimeMicros) { + logical instanceof LogicalTypes.TimeMillis + || logical instanceof LogicalTypes.TimeMicros) { return Types.TimeType.get(); } else if ( - logical instanceof LogicalTypes.TimestampMillis - || logical instanceof LogicalTypes.TimestampMicros) { + logical instanceof LogicalTypes.TimestampMillis + || logical instanceof LogicalTypes.TimestampMicros) { return Types.TimestampType.get(); } else if (LogicalTypes.uuid().getName().equals(name)) { return Types.UUIDType.get(); @@ -492,10 +564,10 @@ private static Schema visitInternalPrimitiveToBuildAvroPrimitiveType(Type.Primit return LogicalTypes.decimal(decimal.precision(), decimal.scale()) .addToSchema(fixedSchema); } - + default: throw new UnsupportedOperationException( - "Unsupported type ID: " + primitive.typeId()); + "Unsupported type ID: " + primitive.typeId()); } } @@ -510,4 +582,4 @@ private static int computeMinBytesForPrecision(int precision) { } return numBytes; } -} +} \ No newline at end of file diff --git a/hudi-common/src/test/java/org/apache/hudi/avro/TestAvroSchemaUtils.java b/hudi-common/src/test/java/org/apache/hudi/avro/TestAvroSchemaUtils.java index 1417f2f67dcc..37cea0d93a91 100644 --- a/hudi-common/src/test/java/org/apache/hudi/avro/TestAvroSchemaUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/avro/TestAvroSchemaUtils.java @@ -18,6 +18,7 @@ package org.apache.hudi.avro; +import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.SchemaBackwardsCompatibilityException; import org.apache.hudi.exception.SchemaCompatibilityException; @@ -28,6 +29,7 @@ import java.util.Collections; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -256,10 +258,9 @@ public void testBrokenSchema() { () -> AvroSchemaUtils.checkSchemaCompatible(FULL_SCHEMA, BROKEN_SCHEMA, true, false, Collections.emptySet())); } - /* [HUDI-7045] should uncomment this test @Test public void testAppendFieldsToSchemaDedupNested() { - Schema full_schema = new Schema.Parser().parse("{\n" + Schema fullSchema = new Schema.Parser().parse("{\n" + " \"type\": \"record\",\n" + " \"namespace\": \"example.schema\",\n" + " \"name\": \"source\",\n" @@ -292,7 +293,7 @@ public void testAppendFieldsToSchemaDedupNested() { + " ]\n" + "}\n"); - Schema missing_field_schema = new Schema.Parser().parse("{\n" + Schema missingFieldSchema = new Schema.Parser().parse("{\n" + " \"type\": \"record\",\n" + " \"namespace\": \"example.schema\",\n" + " \"name\": \"source\",\n" @@ -321,9 +322,8 @@ public void testAppendFieldsToSchemaDedupNested() { + " ]\n" + "}\n"); - Option missingField = AvroSchemaUtils.findNestedField(full_schema, "nested_record.long"); - assertTrue(missingField.isPresent()); - assertEquals(full_schema, AvroSchemaUtils.appendFieldsToSchemaDedupNested(missing_field_schema, Collections.singletonList(missingField.get()))); + Option missingField = AvroSchemaUtils.findNestedField(fullSchema, "nested_record.long"); + assertTrue(missingField.isPresent()); + assertEquals(fullSchema, AvroSchemaUtils.appendFieldsToSchemaDedupNested(missingFieldSchema, Collections.singletonList(missingField.get()))); } - */ } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java index 8001cbe45d37..8ca2517eade0 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java @@ -34,6 +34,7 @@ import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.SyncableFileSystemView; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.metadata.HoodieTableMetadata; @@ -68,7 +69,7 @@ public abstract class TestHoodieFileGroupReaderBase { public abstract String getBasePath(); - public abstract HoodieReaderContext getHoodieReaderContext(String tablePath, Schema avroSchema); + public abstract HoodieReaderContext getHoodieReaderContext(String tablePath, Schema avroSchema, Configuration hadoopConf); public abstract void commitToTable(List recordList, String operation, Map writeConfigs); @@ -169,13 +170,15 @@ private void validateOutputFromFileGroupReader(Configuration hadoopConf, } assertEquals(containsBaseFile, fileSlice.getBaseFile().isPresent()); HoodieFileGroupReader fileGroupReader = new HoodieFileGroupReader<>( - getHoodieReaderContext(tablePath, avroSchema), + getHoodieReaderContext(tablePath, avroSchema, hadoopConf), hadoopConf, tablePath, metaClient.getActiveTimeline().lastInstant().get().getTimestamp(), fileSlice, avroSchema, avroSchema, + Option.empty(), + metaClient, props, metaClient.getTableConfig(), 0, @@ -193,4 +196,4 @@ private void validateOutputFromFileGroupReader(Configuration hadoopConf, validateRecordsInFileGroup(tablePath, actualRecordList, avroSchema, fileSlice.getFileId()); } -} +} \ No newline at end of file diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestUtils.java index add4032b6c2d..e72a71a4c5ab 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestUtils.java @@ -119,6 +119,8 @@ public HoodieFileGroupReader build( fileSlice, schema, schema, + Option.empty(), + null, props, tableConfig, start, @@ -130,4 +132,4 @@ public HoodieFileGroupReader build( false); } } -} +} \ No newline at end of file diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieTestReaderContext.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieTestReaderContext.java index 3aad5e9a0aa3..e2f9bd841489 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieTestReaderContext.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieTestReaderContext.java @@ -76,7 +76,8 @@ public ClosableIterator getFileRecordIterator( long length, Schema dataSchema, Schema requiredSchema, - Configuration conf + Configuration conf, + boolean isMerge ) throws IOException { HoodieAvroParquetReader reader = new HoodieAvroParquetReader(conf, filePath); return reader.getIndexedRecordIterator(dataSchema, requiredSchema); @@ -164,9 +165,10 @@ public IndexedRecord seal(IndexedRecord record) { } @Override - public ClosableIterator mergeBootstrapReaders( - ClosableIterator skeletonFileIterator, - ClosableIterator dataFileIterator) { + public ClosableIterator mergeBootstrapReaders(ClosableIterator skeletonFileIterator, + Schema skeletonRequiredSchema, + ClosableIterator dataFileIterator, + Schema dataRequiredSchema) { return null; } @@ -226,4 +228,9 @@ private Object getFieldValueFromIndexedRecord( int pos = field.pos(); return record.get(pos); } -} + + @Override + public boolean shouldUseRecordPositionMerging() { + return true; + } +} \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala index be3d2f4ed4bf..0b1ff9b5e7f1 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala @@ -25,7 +25,7 @@ import org.apache.hudi.common.HoodieSchemaNotFoundException import org.apache.hudi.common.config.{HoodieCommonConfig, HoodieReaderConfig} import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_READ} -import org.apache.hudi.common.model.{HoodieTableType, WriteConcurrencyMode} +import org.apache.hudi.common.model.WriteConcurrencyMode import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.common.util.ConfigUtils import org.apache.hudi.common.util.ValidationUtils.checkState @@ -47,9 +47,9 @@ import scala.collection.JavaConversions.mapAsJavaMap import scala.collection.JavaConverters._ /** - * Hoodie Spark Datasource, for reading and writing hoodie tables - * - */ + * Hoodie Spark Datasource, for reading and writing hoodie tables + * + */ class DefaultSource extends RelationProvider with SchemaRelationProvider with CreatableRelationProvider @@ -263,7 +263,6 @@ object DefaultSource { val useNewParquetFileFormat = parameters.getOrDefault(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), HoodieReaderConfig.FILE_GROUP_READER_ENABLED.defaultValue().toString).toBoolean && !metaClient.isMetadataTable && (globPaths == null || globPaths.isEmpty) && - !parameters.getOrDefault(SCHEMA_EVOLUTION_ENABLED.key(), SCHEMA_EVOLUTION_ENABLED.defaultValue().toString).toBoolean && parameters.getOrElse(REALTIME_MERGE.key(), REALTIME_MERGE.defaultValue()).equalsIgnoreCase(REALTIME_PAYLOAD_COMBINE_OPT_VAL) if (metaClient.getCommitsTimeline.filterCompletedInstants.countInstants() == 0) { new EmptyRelation(sqlContext, resolveSchema(metaClient, parameters, Some(schema))) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala index e8ca19e2421e..02b4efa37d00 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.mapred.JobConf import org.apache.hudi.HoodieBaseRelation.{convertToAvroSchema, isSchemaEvolutionEnabledOnRead} import org.apache.hudi.HoodieConversionUtils.toScalaOption import org.apache.hudi.HoodieFileIndex.getConfigProperties +import org.apache.hudi.client.utils.SparkInternalSchemaConverter import org.apache.hudi.common.config.HoodieMetadataConfig.{DEFAULT_METADATA_ENABLE_FOR_READERS, ENABLE} import org.apache.hudi.common.config.{ConfigProperty, HoodieMetadataConfig, HoodieReaderConfig, TypedProperties} import org.apache.hudi.common.model.HoodieRecord @@ -35,6 +36,7 @@ import org.apache.hudi.common.util.ValidationUtils.checkState import org.apache.hudi.config.HoodieBootstrapConfig.DATA_QUERIES_ONLY import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter +import org.apache.hudi.internal.schema.utils.SerDeHelper import org.apache.hudi.internal.schema.{HoodieSchemaException, InternalSchema} import org.apache.hudi.metadata.HoodieTableMetadataUtil import org.apache.spark.sql.catalyst.analysis.Resolver @@ -64,7 +66,7 @@ abstract class HoodieBaseHadoopFsRelationFactory(val sqlContext: SQLContext, val schemaSpec: Option[StructType] ) extends SparkAdapterSupport with HoodieHadoopFsRelationFactory { protected lazy val sparkSession: SparkSession = sqlContext.sparkSession - protected lazy val optParams: Map[String, String] = options + protected var optParams: Map[String, String] = options protected lazy val hadoopConfig: Configuration = new Configuration(sqlContext.sparkContext.hadoopConfiguration) protected lazy val jobConf = new JobConf(hadoopConfig) @@ -85,7 +87,13 @@ abstract class HoodieBaseHadoopFsRelationFactory(val sqlContext: SQLContext, specifiedQueryTimestamp.map(schemaResolver.getTableInternalSchemaFromCommitMetadata) .getOrElse(schemaResolver.getTableInternalSchemaFromCommitMetadata) } match { - case Success(internalSchemaOpt) => toScalaOption(internalSchemaOpt) + case Success(internalSchemaOpt) => + if (internalSchemaOpt.isPresent) { + optParams = optParams + (SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA -> SerDeHelper.toJson(internalSchemaOpt.get())) + optParams = optParams + (SparkInternalSchemaConverter.HOODIE_TABLE_PATH -> basePath.toString) + optParams = optParams + (SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST -> timeline.getInstants.iterator.asScala.map(_.getFileName).mkString(",")) + } + toScalaOption(internalSchemaOpt) case Failure(e) => None } @@ -160,15 +168,13 @@ abstract class HoodieBaseHadoopFsRelationFactory(val sqlContext: SQLContext, val shouldExtractPartitionValueFromPath = optParams.getOrElse(DataSourceReadOptions.EXTRACT_PARTITION_VALUES_FROM_PARTITION_PATH.key, DataSourceReadOptions.EXTRACT_PARTITION_VALUES_FROM_PARTITION_PATH.defaultValue.toString).toBoolean - val shouldUseBootstrapFastRead = optParams.getOrElse(DATA_QUERIES_ONLY.key(), "false").toBoolean - - shouldOmitPartitionColumns || shouldExtractPartitionValueFromPath || shouldUseBootstrapFastRead + shouldOmitPartitionColumns || shouldExtractPartitionValueFromPath } protected lazy val mandatoryFieldsForMerging: Seq[String] = Seq(recordKeyField) ++ preCombineFieldOpt.map(Seq(_)).getOrElse(Seq()) - protected lazy val shouldUseRecordPosition: Boolean = checkIfAConfigurationEnabled(HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS) + protected lazy val shouldUseRecordPosition: Boolean = checkIfAConfigurationEnabled(HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS) && HoodieSparkUtils.gteqSpark3_5 protected def queryTimestamp: Option[String] = specifiedQueryTimestamp.orElse(toScalaOption(timeline.lastInstant()).map(_.getTimestamp)) @@ -180,13 +186,13 @@ abstract class HoodieBaseHadoopFsRelationFactory(val sqlContext: SQLContext, metaClient.getCommitsAndCompactionTimeline.filterCompletedInstants protected def getConfigValue(config: ConfigProperty[String], - defaultValueOption: Option[String] = Option.empty): String = { + defaultValueOption: Option[String] = Option.empty): String = { optParams.getOrElse(config.key(), sqlContext.getConf(config.key(), defaultValueOption.getOrElse(config.defaultValue()))) } protected def checkIfAConfigurationEnabled(config: ConfigProperty[java.lang.Boolean], - defaultValueOption: Option[String] = Option.empty): Boolean = { + defaultValueOption: Option[String] = Option.empty): Boolean = { optParams.getOrElse(config.key(), sqlContext.getConf(config.key(), defaultValueOption.getOrElse(String.valueOf(config.defaultValue())))).toBoolean } @@ -289,10 +295,10 @@ class HoodieMergeOnReadIncrementalHadoopFsRelationFactory(override val sqlContex } class HoodieCopyOnWriteSnapshotHadoopFsRelationFactory(override val sqlContext: SQLContext, - override val metaClient: HoodieTableMetaClient, - override val options: Map[String, String], - override val schemaSpec: Option[StructType], - isBootstrap: Boolean) + override val metaClient: HoodieTableMetaClient, + override val options: Map[String, String], + override val schemaSpec: Option[StructType], + isBootstrap: Boolean) extends HoodieMergeOnReadSnapshotHadoopFsRelationFactory(sqlContext, metaClient, options, schemaSpec, isBootstrap) { override val mandatoryFields: Seq[String] = Seq.empty @@ -348,10 +354,10 @@ class HoodieCopyOnWriteIncrementalHadoopFsRelationFactory(override val sqlContex } class HoodieMergeOnReadCDCHadoopFsRelationFactory(override val sqlContext: SQLContext, - override val metaClient: HoodieTableMetaClient, - override val options: Map[String, String], - override val schemaSpec: Option[StructType], - isBootstrap: Boolean) + override val metaClient: HoodieTableMetaClient, + override val options: Map[String, String], + override val schemaSpec: Option[StructType], + isBootstrap: Boolean) extends HoodieMergeOnReadIncrementalHadoopFsRelationFactory(sqlContext, metaClient, options, schemaSpec, isBootstrap) { override val fileIndex = new HoodieCDCFileIndex( sparkSession, metaClient, schemaSpec, options, FileStatusCache.getOrCreate(sparkSession), true, true) @@ -375,4 +381,3 @@ class HoodieCopyOnWriteCDCHadoopFsRelationFactory(override val sqlContext: SQLCo override def buildPartitionSchema(): StructType = StructType(Nil) } - diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala index 3de30c770c7a..b9285faab719 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala @@ -22,33 +22,26 @@ import org.apache.hadoop.fs.Path import org.apache.hudi.MergeOnReadSnapshotRelation.createPartitionedFile import org.apache.hudi.avro.AvroSchemaUtils import org.apache.hudi.cdc.{CDCFileGroupIterator, CDCRelation, HoodieCDCFileGroupSplit} -import org.apache.hudi.common.config.{HoodieCommonConfig, HoodieMemoryConfig, HoodieStorageConfig, TypedProperties} -import org.apache.hudi.common.engine.HoodieReaderContext -import org.apache.hudi.common.model.{FileSlice, HoodieLogFile, HoodieRecord} -import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} -import org.apache.hudi.common.table.read.HoodieFileGroupReader -import org.apache.hudi.{AvroConversionUtils, HoodieFileIndex, HoodiePartitionCDCFileGroupMapping, HoodiePartitionFileSliceMapping, HoodieSparkUtils, HoodieTableSchema, HoodieTableState, SparkAdapterSupport, SparkFileFormatInternalRowReaderContext} +import org.apache.hudi.client.utils.SparkInternalSchemaConverter +import org.apache.hudi.common.config.{HoodieCommonConfig, HoodieMemoryConfig, TypedProperties} import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.table.read.HoodieFileGroupReader +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} import org.apache.hudi.common.util.FileIOUtils -import org.apache.hudi.common.util.collection.ExternalSpillableMap import org.apache.hudi.common.util.collection.ExternalSpillableMap.DiskMapType -import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.internal.schema.InternalSchema +import org.apache.hudi.{AvroConversionUtils, HoodieFileIndex, HoodiePartitionCDCFileGroupMapping, HoodiePartitionFileSliceMapping, HoodieTableSchema, HoodieTableState, SparkAdapterSupport, SparkFileFormatInternalRowReaderContext} import org.apache.spark.sql.HoodieCatalystExpressionUtils.generateUnsafeProjection import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.JoinedRow import org.apache.spark.sql.execution.datasources.PartitionedFile -import org.apache.spark.sql.execution.datasources.parquet.HoodieFileGroupReaderBasedParquetFileFormat.{ROW_INDEX, ROW_INDEX_TEMPORARY_COLUMN_NAME, getAppliedFilters, getAppliedRequiredSchema, getLogFilesFromSlice, getRecordKeyRelatedFilters, makeCloseableFileGroupMappingRecordIterator} -import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isMetaField import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types.{LongType, Metadata, MetadataBuilder, StringType, StructField, StructType} +import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration import java.io.Closeable import java.util.Locale -import scala.annotation.tailrec -import scala.collection.mutable -import scala.jdk.CollectionConverters.asScalaIteratorConverter trait HoodieFormatTrait { @@ -71,24 +64,36 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, isIncremental: Boolean, shouldUseRecordPosition: Boolean, requiredFilters: Seq[Filter] - ) extends ParquetFileFormat with SparkAdapterSupport with HoodieFormatTrait { + ) extends ParquetFileFormat with SparkAdapterSupport with HoodieFormatTrait { def getRequiredFilters: Seq[Filter] = requiredFilters + private val sanitizedTableName = AvroSchemaUtils.getAvroRecordQualifiedName(tableName) + /** * Support batch needs to remain consistent, even if one side of a bootstrap merge can support * while the other side can't */ - private var supportBatchCalled = false - private var supportBatchResult = false + /* +private var supportBatchCalled = false +private var supportBatchResult = false + +override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { + if (!supportBatchCalled || supportBatchResult) { + supportBatchCalled = true + supportBatchResult = tableSchema.internalSchema.isEmpty && !isMOR && !isIncremental && !isBootstrap && super.supportBatch(sparkSession, schema) + } + supportBatchResult +} + */ + override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = false - private val sanitizedTableName = AvroSchemaUtils.getAvroRecordQualifiedName(tableName) - override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { - if (!supportBatchCalled || supportBatchResult) { - supportBatchCalled = true - supportBatchResult = !isMOR && !isIncremental && !isBootstrap && super.supportBatch(sparkSession, schema) - } - supportBatchResult + private val supportBatchResult = false + + private lazy val internalSchemaOpt: org.apache.hudi.common.util.Option[InternalSchema] = if (tableSchema.internalSchema.isEmpty) { + org.apache.hudi.common.util.Option.empty() + } else { + org.apache.hudi.common.util.Option.of(tableSchema.internalSchema.get) } override def isSplitable(sparkSession: SparkSession, @@ -107,19 +112,23 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, val dataSchema = StructType(tableSchema.structTypeSchema.fields.filterNot(f => partitionColumns.contains(f.name))) val outputSchema = StructType(requiredSchema.fields ++ partitionSchema.fields) spark.conf.set("spark.sql.parquet.enableVectorizedReader", supportBatchResult) - val requiredSchemaWithMandatory = generateRequiredSchemaWithMandatory(requiredSchema, dataSchema, partitionSchema) - val isCount = requiredSchemaWithMandatory.isEmpty - val requiredSchemaSplits = requiredSchemaWithMandatory.fields.partition(f => HoodieRecord.HOODIE_META_COLUMNS_WITH_OPERATION.contains(f.name)) - val requiredMeta = StructType(requiredSchemaSplits._1) - val requiredWithoutMeta = StructType(requiredSchemaSplits._2) + val isCount = requiredSchema.isEmpty && !isMOR && !isIncremental val augmentedHadoopConf = FSUtils.buildInlineConf(hadoopConf) - val (baseFileReader, preMergeBaseFileReader, readerMaps, cdcFileReader) = buildFileReaders( - spark, dataSchema, partitionSchema, requiredSchema, filters, options, augmentedHadoopConf, - requiredSchemaWithMandatory, requiredWithoutMeta, requiredMeta) + setSchemaEvolutionConfigs(augmentedHadoopConf, options) + val baseFileReader = super.buildReaderWithPartitionValues(spark, dataSchema, partitionSchema, requiredSchema, + filters ++ requiredFilters, options, new Configuration(hadoopConf)) + val cdcFileReader = super.buildReaderWithPartitionValues( + spark, + tableSchema.structTypeSchema, + StructType(Nil), + tableSchema.structTypeSchema, + Nil, + options, + new Configuration(hadoopConf)) val requestedAvroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(requiredSchema, sanitizedTableName) val dataAvroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(dataSchema, sanitizedTableName) - + val extraProps = spark.sparkContext.broadcast(sparkAdapter.getPropsForReadingParquet(supportBatchResult, spark.sessionState.conf, options, augmentedHadoopConf)) val broadcastedHadoopConf = spark.sparkContext.broadcast(new SerializableConfiguration(augmentedHadoopConf)) val broadcastedDataSchema = spark.sparkContext.broadcast(dataAvroSchema) val broadcastedRequestedSchema = spark.sparkContext.broadcast(requestedAvroSchema) @@ -129,20 +138,15 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, file.partitionValues match { // Snapshot or incremental queries. case fileSliceMapping: HoodiePartitionFileSliceMapping => - val filePath = sparkAdapter.getSparkPartitionedFileUtils.getPathFromPartitionedFile(file) - val filegroupName = if (FSUtils.isLogFile(filePath)) { - FSUtils.getFileId(filePath.getName).substring(1) - } else { - FSUtils.getFileId(filePath.getName) - } + val filegroupName = FSUtils.getFileIdFromFilePath(sparkAdapter + .getSparkPartitionedFileUtils.getPathFromPartitionedFile(file)) fileSliceMapping.getSlice(filegroupName) match { case Some(fileSlice) if !isCount => if (requiredSchema.isEmpty && !fileSlice.getLogFiles.findAny().isPresent) { val hoodieBaseFile = fileSlice.getBaseFile.get() baseFileReader(createPartitionedFile(fileSliceMapping.getPartitionValues, hoodieBaseFile.getHadoopPath, 0, hoodieBaseFile.getFileLen)) } else { - val readerContext: HoodieReaderContext[InternalRow] = new SparkFileFormatInternalRowReaderContext( - readerMaps) + val readerContext = new SparkFileFormatInternalRowReaderContext(extraProps.value, tableState.recordKeyField, filters, shouldUseRecordPosition) val serializedHadoopConf = broadcastedHadoopConf.value.value val metaClient: HoodieTableMetaClient = HoodieTableMetaClient .builder().setConf(serializedHadoopConf).setBasePath(tableState.tablePath).build @@ -154,6 +158,8 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, fileSlice, broadcastedDataSchema.value, broadcastedRequestedSchema.value, + internalSchemaOpt, + metaClient, metaClient.getTableConfig.getProps, metaClient.getTableConfig, file.start, @@ -173,8 +179,8 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, fileSliceMapping.getPartitionValues) } - // TODO: Use FileGroupReader here: HUDI-6942. - case _ => baseFileReader(file) + case _ => sparkAdapter.readParquetFile(file, requiredSchema, partitionSchema, filters, + broadcastedHadoopConf.value.value, extraProps.value) } // CDC queries. case hoodiePartitionCDCFileGroupSliceMapping: HoodiePartitionCDCFileGroupMapping => @@ -182,12 +188,21 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, val fileGroupSplit: HoodieCDCFileGroupSplit = HoodieCDCFileGroupSplit(fileSplits) buildCDCRecordIterator( fileGroupSplit, cdcFileReader, broadcastedHadoopConf.value.value, fileIndexProps, requiredSchema) - // TODO: Use FileGroupReader here: HUDI-6942. - case _ => baseFileReader(file) + + case _ => sparkAdapter.readParquetFile(file, requiredSchema, partitionSchema, filters, + broadcastedHadoopConf.value.value, extraProps.value) } } } + protected def setSchemaEvolutionConfigs(conf: Configuration, options: Map[String, String]): Unit = { + if (internalSchemaOpt.isPresent) { + options.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA).foreach(s => conf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, s)) + options.get(SparkInternalSchemaConverter.HOODIE_TABLE_PATH).foreach(s => conf.set(SparkInternalSchemaConverter.HOODIE_TABLE_PATH, s)) + options.get(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST).foreach(s => conf.set(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST, s)) + } + } + protected def buildCDCRecordIterator(cdcFileGroupSplit: HoodieCDCFileGroupSplit, cdcFileReader: PartitionedFile => Iterator[InternalRow], hadoopConf: Configuration, @@ -228,203 +243,6 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, makeCloseableFileGroupMappingRecordIterator(iter, d => unsafeProjection(d)) } - private def generateRequiredSchemaWithMandatory(requiredSchema: StructType, - dataSchema: StructType, - partitionSchema: StructType): StructType = { - val metaFields = Seq( - StructField(HoodieRecord.COMMIT_TIME_METADATA_FIELD, StringType), - StructField(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, StringType), - StructField(HoodieRecord.RECORD_KEY_METADATA_FIELD, StringType), - StructField(HoodieRecord.PARTITION_PATH_METADATA_FIELD, StringType), - StructField(HoodieRecord.FILENAME_METADATA_FIELD, StringType)) - - // Helper method to get the StructField for nested fields - @tailrec - def findNestedField(schema: StructType, fieldParts: Array[String]): Option[StructField] = { - fieldParts.toList match { - case head :: Nil => schema.fields.find(_.name == head) // If it's the last part, find and return the field - case head :: tail => // If there are more parts, find the field and its nested fields - schema.fields.find(_.name == head) match { - case Some(StructField(_, nested: StructType, _, _)) => findNestedField(nested, tail.toArray) - case _ => None // The path is not valid - } - case _ => None // Empty path, should not happen if the input is correct - } - } - - def findMetaField(name: String): Option[StructField] = { - metaFields.find(f => f.name == name) - } - - val added: mutable.Buffer[StructField] = mutable.Buffer[StructField]() - for (field <- mandatoryFields) { - if (requiredSchema.getFieldIndex(field).isEmpty) { - // Support for nested fields - val fieldParts = field.split("\\.") - val fieldToAdd = findNestedField(dataSchema, fieldParts) - .orElse(findNestedField(partitionSchema, fieldParts)) - .orElse(findMetaField(field)) - .getOrElse(throw new IllegalArgumentException(s"Field $field does not exist in the table schema")) - added.append(fieldToAdd) - } - } - val addedFields = StructType(added.toArray) - StructType(requiredSchema.toArray ++ addedFields.fields) - } - - protected def buildFileReaders(sparkSession: SparkSession, dataSchema: StructType, partitionSchema: StructType, - requiredSchema: StructType, filters: Seq[Filter], options: Map[String, String], - hadoopConf: Configuration, requiredSchemaWithMandatory: StructType, - requiredWithoutMeta: StructType, requiredMeta: StructType): - (PartitionedFile => Iterator[InternalRow], - PartitionedFile => Iterator[InternalRow], - mutable.Map[Long, PartitionedFile => Iterator[InternalRow]], - PartitionedFile => Iterator[InternalRow]) = { - - val m = scala.collection.mutable.Map[Long, PartitionedFile => Iterator[InternalRow]]() - - val recordKeyRelatedFilters = getRecordKeyRelatedFilters(filters, tableState.recordKeyField) - val baseFileReader = super.buildReaderWithPartitionValues(sparkSession, dataSchema, partitionSchema, requiredSchema, - filters ++ requiredFilters, options, new Configuration(hadoopConf)) - m.put(generateKey(dataSchema, requiredSchema), baseFileReader) - - // File reader for reading a Hoodie base file that needs to be merged with log files - // Add support for reading files using inline file system. - val appliedRequiredSchema: StructType = getAppliedRequiredSchema( - requiredSchemaWithMandatory, shouldUseRecordPosition, ROW_INDEX_TEMPORARY_COLUMN_NAME) - val appliedFilters = getAppliedFilters( - requiredFilters, recordKeyRelatedFilters, shouldUseRecordPosition) - val preMergeBaseFileReader = super.buildReaderWithPartitionValues( - sparkSession, - dataSchema, - StructType(Nil), - appliedRequiredSchema, - appliedFilters, - options, - new Configuration(hadoopConf)) - m.put(generateKey(dataSchema, appliedRequiredSchema), preMergeBaseFileReader) - - val cdcFileReader = super.buildReaderWithPartitionValues( - sparkSession, - tableSchema.structTypeSchema, - StructType(Nil), - tableSchema.structTypeSchema, - Nil, - options, - new Configuration(hadoopConf)) - - //Rules for appending partitions and filtering in the bootstrap readers: - // 1. if it is mor, we don't want to filter data or append partitions - // 2. if we need to merge the bootstrap base and skeleton files then we cannot filter - // 3. if we need to merge the bootstrap base and skeleton files then we should never append partitions to the - // skeleton reader - val needMetaCols = requiredMeta.nonEmpty - val needDataCols = requiredWithoutMeta.nonEmpty - - //file reader for bootstrap skeleton files - if (needMetaCols && isBootstrap) { - val key = generateKey(HoodieSparkUtils.getMetaSchema, requiredMeta) - if (needDataCols || isMOR) { - // no filter and no append - m.put(key, super.buildReaderWithPartitionValues(sparkSession, HoodieSparkUtils.getMetaSchema, StructType(Seq.empty), - requiredMeta, Seq.empty, options, new Configuration(hadoopConf))) - } else { - // filter - m.put(key, super.buildReaderWithPartitionValues(sparkSession, HoodieSparkUtils.getMetaSchema, StructType(Seq.empty), - requiredMeta, filters ++ requiredFilters, options, new Configuration(hadoopConf))) - } - - val requestedMeta = StructType(requiredSchema.fields.filter(sf => isMetaField(sf.name))) - m.put(generateKey(HoodieSparkUtils.getMetaSchema, requestedMeta), - super.buildReaderWithPartitionValues(sparkSession, HoodieSparkUtils.getMetaSchema, StructType(Seq.empty), requestedMeta, - Seq.empty, options, new Configuration(hadoopConf))) - } - - //file reader for bootstrap base files - if (needDataCols && isBootstrap) { - val dataSchemaWithoutMeta = StructType(dataSchema.fields.filterNot(sf => isMetaField(sf.name))) - val key = generateKey(dataSchemaWithoutMeta, requiredWithoutMeta) - if (isMOR || needMetaCols) { - m.put(key, super.buildReaderWithPartitionValues(sparkSession, dataSchemaWithoutMeta, StructType(Seq.empty), requiredWithoutMeta, - Seq.empty, options, new Configuration(hadoopConf))) - // no filter and no append - - } else { - // filter - m.put(key, super.buildReaderWithPartitionValues(sparkSession, dataSchemaWithoutMeta, StructType(Seq.empty), requiredWithoutMeta, - filters ++ requiredFilters, options, new Configuration(hadoopConf))) - } - - val requestedWithoutMeta = StructType(requiredSchema.fields.filterNot(sf => isMetaField(sf.name))) - m.put(generateKey(dataSchemaWithoutMeta, requestedWithoutMeta), - super.buildReaderWithPartitionValues(sparkSession, dataSchemaWithoutMeta, StructType(Seq.empty), requestedWithoutMeta, - Seq.empty, options, new Configuration(hadoopConf))) - } - - (baseFileReader, preMergeBaseFileReader, m, cdcFileReader) - } - - protected def generateKey(dataSchema: StructType, requestedSchema: StructType): Long = { - AvroConversionUtils.convertStructTypeToAvroSchema(dataSchema, sanitizedTableName).hashCode() + AvroConversionUtils.convertStructTypeToAvroSchema(requestedSchema, sanitizedTableName).hashCode() - } -} - -object HoodieFileGroupReaderBasedParquetFileFormat { - // From "ParquetFileFormat.scala": The names of the field for record position. - private val ROW_INDEX = "row_index" - private val ROW_INDEX_TEMPORARY_COLUMN_NAME = s"_tmp_metadata_$ROW_INDEX" - - // From "namedExpressions.scala": Used to construct to record position field metadata. - private val FILE_SOURCE_GENERATED_METADATA_COL_ATTR_KEY = "__file_source_generated_metadata_col" - private val FILE_SOURCE_METADATA_COL_ATTR_KEY = "__file_source_metadata_col" - private val METADATA_COL_ATTR_KEY = "__metadata_col" - - def getRecordKeyRelatedFilters(filters: Seq[Filter], recordKeyColumn: String): Seq[Filter] = { - filters.filter(f => f.references.exists(c => c.equalsIgnoreCase(recordKeyColumn))) - } - - def getLogFilesFromSlice(fileSlice: FileSlice): List[HoodieLogFile] = { - fileSlice.getLogFiles.sorted(HoodieLogFile.getLogFileComparator).iterator().asScala.toList - } - - def getFieldMetadata(name: String, internalName: String): Metadata = { - new MetadataBuilder() - .putString(METADATA_COL_ATTR_KEY, name) - .putBoolean(FILE_SOURCE_METADATA_COL_ATTR_KEY, value = true) - .putString(FILE_SOURCE_GENERATED_METADATA_COL_ATTR_KEY, internalName) - .build() - } - - def getAppliedRequiredSchema(requiredSchema: StructType, - shouldUseRecordPosition: Boolean, - recordPositionColumn: String): StructType = { - if (shouldAddRecordPositionColumn(shouldUseRecordPosition)) { - val metadata = getFieldMetadata(recordPositionColumn, ROW_INDEX_TEMPORARY_COLUMN_NAME) - val rowIndexField = StructField(recordPositionColumn, LongType, nullable = false, metadata) - StructType(requiredSchema.fields :+ rowIndexField) - } else { - requiredSchema - } - } - - def getAppliedFilters(requiredFilters: Seq[Filter], - recordKeyRelatedFilters: Seq[Filter], - shouldUseRecordPosition: Boolean): Seq[Filter] = { - if (shouldAddRecordKeyFilters(shouldUseRecordPosition)) { - requiredFilters ++ recordKeyRelatedFilters - } else { - requiredFilters - } - } - - def shouldAddRecordPositionColumn(shouldUseRecordPosition: Boolean): Boolean = { - HoodieSparkUtils.gteqSpark3_5 && shouldUseRecordPosition - } - - def shouldAddRecordKeyFilters(shouldUseRecordPosition: Boolean): Boolean = { - (!shouldUseRecordPosition) || HoodieSparkUtils.gteqSpark3_5 - } - def makeCloseableFileGroupMappingRecordIterator(closeableFileGroupRecordIterator: HoodieFileGroupReader.HoodieFileGroupReaderIterator[InternalRow], mappingFunction: Function[InternalRow, InternalRow]): Iterator[InternalRow] = { new Iterator[InternalRow] with Closeable { diff --git a/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/execution/datasources/parquet/TestHoodieFileGroupReaderBasedParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/execution/datasources/parquet/TestHoodieFileGroupReaderBasedParquetFileFormat.scala index 5a5456cb1734..7c8b2d6108d7 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/execution/datasources/parquet/TestHoodieFileGroupReaderBasedParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/execution/datasources/parquet/TestHoodieFileGroupReaderBasedParquetFileFormat.scala @@ -19,11 +19,9 @@ package org.apache.spark.execution.datasources.parquet -import org.apache.hudi.HoodieSparkUtils +import org.apache.hudi.SparkFileFormatInternalRowReaderContext import org.apache.hudi.testutils.SparkClientFunctionalTestHarness -import org.apache.spark.sql.execution.datasources.parquet.HoodieFileGroupReaderBasedParquetFileFormat import org.apache.spark.sql.sources.{EqualTo, GreaterThan, IsNotNull} -import org.apache.spark.sql.types.{LongType, StringType, StructField, StructType} import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Test @@ -34,7 +32,7 @@ class TestHoodieFileGroupReaderBasedParquetFileFormat extends SparkClientFunctio IsNotNull("non_key_column"), EqualTo("non_key_column", 1) ) - val filtersWithoutKeyColumn = HoodieFileGroupReaderBasedParquetFileFormat.getRecordKeyRelatedFilters( + val filtersWithoutKeyColumn = SparkFileFormatInternalRowReaderContext.getRecordKeyRelatedFilters( filters, "key_column"); assertEquals(0, filtersWithoutKeyColumn.size) @@ -42,55 +40,9 @@ class TestHoodieFileGroupReaderBasedParquetFileFormat extends SparkClientFunctio EqualTo("key_column", 1), GreaterThan("non_key_column", 2) ) - val filtersWithKeyColumn = HoodieFileGroupReaderBasedParquetFileFormat.getRecordKeyRelatedFilters( + val filtersWithKeyColumn = SparkFileFormatInternalRowReaderContext.getRecordKeyRelatedFilters( filtersWithKeys, "key_column") assertEquals(1, filtersWithKeyColumn.size) assertEquals("key_column", filtersWithKeyColumn.head.references.head) } - - @Test - def testGetAppliedRequiredSchema(): Unit = { - val fields = Array( - StructField("column_a", LongType, nullable = false), - StructField("column_b", StringType, nullable = false)) - val requiredSchema = StructType(fields) - - val appliedSchema: StructType = HoodieFileGroupReaderBasedParquetFileFormat.getAppliedRequiredSchema( - requiredSchema, shouldUseRecordPosition = true, "row_index") - if (HoodieSparkUtils.gteqSpark3_5) { - assertEquals(3, appliedSchema.fields.length) - } else { - assertEquals(2, appliedSchema.fields.length) - } - - val schemaWithoutRowIndexColumn = HoodieFileGroupReaderBasedParquetFileFormat.getAppliedRequiredSchema( - requiredSchema, shouldUseRecordPosition = false, "row_index") - assertEquals(2, schemaWithoutRowIndexColumn.fields.length) - } - - @Test - def testGetAppliedFilters(): Unit = { - val filters = Seq( - IsNotNull("non_key_column"), - EqualTo("non_key_column", 1) - ) - val keyRelatedFilters = Seq( - EqualTo("key_column", 2) - ) - - val appliedFilters = HoodieFileGroupReaderBasedParquetFileFormat.getAppliedFilters( - filters, keyRelatedFilters, shouldUseRecordPosition = true - ) - if (!HoodieSparkUtils.gteqSpark3_5) { - assertEquals(2, appliedFilters.size) - } else { - assertEquals(3, appliedFilters.size) - } - - val appliedFiltersWithoutUsingRecordPosition = HoodieFileGroupReaderBasedParquetFileFormat.getAppliedFilters( - filters, keyRelatedFilters, shouldUseRecordPosition = false - ) - assertEquals(3, appliedFiltersWithoutUsingRecordPosition.size) - } } - diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodiePositionBasedFileGroupRecordBuffer.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodiePositionBasedFileGroupRecordBuffer.java index da410f1f4a79..68a874d07eda 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodiePositionBasedFileGroupRecordBuffer.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodiePositionBasedFileGroupRecordBuffer.java @@ -38,6 +38,7 @@ import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieValidationException; +import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.avro.Schema; @@ -98,9 +99,11 @@ public void prepareBuffer(boolean useCustomMerger) throws Exception { ? Option.empty() : Option.of(partitionPaths[0]); buffer = new HoodiePositionBasedFileGroupRecordBuffer<>( - getHoodieReaderContext(getBasePath(), avroSchema), + getHoodieReaderContext(getBasePath(), avroSchema, getHadoopConf()), avroSchema, avroSchema, + InternalSchema.getEmptyInternalSchema(), + metaClient, partitionNameOpt, partitionFields, useCustomMerger ? new CustomMerger() : new HoodieSparkRecordMerger(), @@ -197,4 +200,3 @@ public HoodieRecord.HoodieRecordType getRecordType() { } } } - diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala index 619647555194..e81aca2069fa 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala @@ -146,11 +146,7 @@ class TestAvroSchemaResolutionSupport extends HoodieClientTestBase with ScalaAss upsertData(upsertDf, tempRecordPath, isCow) // read out the table - val readDf = spark.read.format("hudi") - // NOTE: type promotion is not supported for the custom file format and the filegroup reader - // HUDI-7045 and PR#10007 in progress to fix the issue - .option(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), "false") - .load(tempRecordPath) + val readDf = spark.read.format("hudi").load(tempRecordPath) readDf.printSchema() readDf.show(false) readDf.foreach(_ => {}) @@ -389,11 +385,7 @@ class TestAvroSchemaResolutionSupport extends HoodieClientTestBase with ScalaAss withSQLConf("spark.sql.parquet.enableNestedColumnVectorizedReader" -> "false") { // read out the table - val readDf = spark.read.format("hudi") - // NOTE: long to int type change is not supported for the custom file format and the filegroup reader - // HUDI-7045 and PR#10007 in progress to fix the issue - .option(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), "false") - .load(tempRecordPath) + val readDf = spark.read.format("hudi").load(tempRecordPath) readDf.printSchema() readDf.show(false) readDf.foreach(_ => {}) @@ -487,11 +479,7 @@ class TestAvroSchemaResolutionSupport extends HoodieClientTestBase with ScalaAss withSQLConf("spark.sql.parquet.enableNestedColumnVectorizedReader" -> "false") { // read out the table - val readDf = spark.read.format("hudi") - // NOTE: type promotion is not supported for the custom file format and the filegroup reader - // HUDI-7045 and PR#10007 in progress to fix the issue - .option(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), "false") - .load(tempRecordPath) + val readDf = spark.read.format("hudi").load(tempRecordPath) readDf.printSchema() readDf.show(false) readDf.foreach(_ => {}) @@ -555,11 +543,7 @@ class TestAvroSchemaResolutionSupport extends HoodieClientTestBase with ScalaAss withSQLConf("spark.sql.parquet.enableNestedColumnVectorizedReader" -> "false") { // read out the table - val readDf = spark.read.format("hudi") - // NOTE: type promotion is not supported for the custom file format and the filegroup reader - // HUDI-7045 and PR#10007 in progress to fix the issue - .option(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), "false") - .load(tempRecordPath) + val readDf = spark.read.format("hudi").load(tempRecordPath) readDf.printSchema() readDf.show(false) readDf.foreach(_ => {}) @@ -826,11 +810,7 @@ class TestAvroSchemaResolutionSupport extends HoodieClientTestBase with ScalaAss upsertData(df7, tempRecordPath) // read out the table - val readDf = spark.read.format("hudi") - // NOTE: type promotion is not supported for the custom file format and the filegroup reader - // HUDI-7045 and PR#10007 in progress to fix the issue - .option(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), "false") - .load(tempRecordPath) + val readDf = spark.read.format("hudi").load(tempRecordPath) readDf.printSchema() readDf.show(false) readDf.foreach(_ => {}) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieParquetBloom.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieParquetBloom.scala index a6f3a0e7368b..32b3ea0b66f1 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieParquetBloom.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieParquetBloom.scala @@ -17,17 +17,12 @@ package org.apache.hudi -import org.apache.spark.sql._ -import org.apache.spark.sql.hudi.HoodieSparkSessionExtension -import org.apache.spark.util.AccumulatorV2 -import org.apache.spark.SparkContext -import org.apache.hudi.testutils.HoodieClientTestUtils.getSparkConfForTest -import org.apache.hudi.DataSourceWriteOptions -import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.common.model.{HoodieTableType, WriteOperationType} +import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.testutils.HoodieSparkClientTestBase +import org.apache.spark.sql._ +import org.apache.spark.util.AccumulatorV2 import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.EnumSource @@ -65,11 +60,11 @@ class TestHoodieParquetBloomFilter extends HoodieSparkClientTestBase with ScalaA sparkSession.sparkContext.register(accu) // this one shall skip partition scanning thanks to bloom when spark >=3 - sparkSession.read.format("hudi").load(basePath).filter("bloom_col = '3'").foreachPartition((it: Iterator[Row]) => it.foreach(_ => accu.add(0))) + sparkSession.read.format("hudi").load(basePath).filter("bloom_col = '3'").foreachPartition((it: Iterator[Row]) => it.foreach(_ => accu.add(1))) assertEquals(if (currentSparkSupportParquetBloom()) 0 else 1, accu.value) // this one will trigger one partition scan - sparkSession.read.format("hudi").load(basePath).filter("bloom_col = '2'").foreachPartition((it: Iterator[Row]) => it.foreach(_ => accu.add(0))) + sparkSession.read.format("hudi").load(basePath).filter("bloom_col = '2'").foreachPartition((it: Iterator[Row]) => it.foreach(_ => accu.add(1))) assertEquals(1, accu.value) } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala index 377e2dd9d7cf..7ff30d5a66ee 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala @@ -25,12 +25,11 @@ import org.apache.hudi.common.config.HoodieReaderConfig.FILE_GROUP_READER_ENABLE import org.apache.hudi.common.engine.HoodieReaderContext import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.{HoodieRecord, WriteOperationType} -import org.apache.hudi.{AvroConversionUtils, SparkFileFormatInternalRowReaderContext} +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.util.ValidationUtils.checkState +import org.apache.hudi.{SparkAdapterSupport, SparkFileFormatInternalRowReaderContext} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.datasources.PartitionedFile -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.functions.col -import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{Dataset, HoodieInternalRowUtils, HoodieUnsafeUtils, Row, SaveMode, SparkSession} import org.apache.spark.{HoodieSparkKryoRegistrar, SparkConf} import org.junit.jupiter.api.Assertions.assertEquals @@ -43,7 +42,7 @@ import scala.collection.JavaConversions._ * Tests {@link HoodieFileGroupReader} with {@link SparkFileFormatInternalRowReaderContext} * on Spark */ -class TestHoodieFileGroupReaderOnSpark extends TestHoodieFileGroupReaderBase[InternalRow] { +class TestHoodieFileGroupReaderOnSpark extends TestHoodieFileGroupReaderBase[InternalRow] with SparkAdapterSupport { var spark: SparkSession = _ @BeforeEach @@ -80,16 +79,17 @@ class TestHoodieFileGroupReaderOnSpark extends TestHoodieFileGroupReaderBase[Int tempDir.toAbsolutePath.toUri.toString } - override def getHoodieReaderContext(tablePath: String, avroSchema: Schema): HoodieReaderContext[InternalRow] = { - val parquetFileFormat = new ParquetFileFormat - val structTypeSchema = AvroConversionUtils.convertAvroSchemaToStructType(avroSchema) - - val recordReaderIterator = parquetFileFormat.buildReaderWithPartitionValues( - spark, structTypeSchema, StructType(Seq.empty), structTypeSchema, Seq.empty, Map.empty, getHadoopConf) - - val m = scala.collection.mutable.Map[Long, PartitionedFile => Iterator[InternalRow]]() - m.put(2*avroSchema.hashCode(), recordReaderIterator) - new SparkFileFormatInternalRowReaderContext(m) + override def getHoodieReaderContext(tablePath: String, avroSchema: Schema, hadoopConf: Configuration): HoodieReaderContext[InternalRow] = { + val extraProps = sparkAdapter.getPropsForReadingParquet(vectorized = false, spark.sessionState.conf, Map.empty, hadoopConf) + val metaClient = HoodieTableMetaClient.builder().setConf(getHadoopConf).setBasePath(tablePath).build + val recordKeyField = if (metaClient.getTableConfig.populateMetaFields()) { + HoodieRecord.RECORD_KEY_METADATA_FIELD + } else { + val keyFields = metaClient.getTableConfig.getRecordKeyFields.get() + checkState(keyFields.length == 1) + keyFields.head + } + new SparkFileFormatInternalRowReaderContext(extraProps, recordKeyField, Seq.empty, false) } override def commitToTable(recordList: util.List[String], operation: String, options: util.Map[String, String]): Unit = { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala index 6065eb683a7b..959648fdbd8f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala @@ -21,22 +21,18 @@ package org.apache.hudi.common.table.read import org.apache.hadoop.conf.Configuration import org.apache.hudi.SparkAdapterSupport.sparkAdapter -import org.apache.hudi.common.config.{HoodieReaderConfig, HoodieStorageConfig} -import org.apache.hudi.common.model.HoodieTableType +import org.apache.hudi.common.config.HoodieStorageConfig +import org.apache.hudi.common.model.{HoodieRecord, HoodieTableType} import org.apache.hudi.common.testutils.HoodieTestTable import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.testutils.SparkClientFunctionalTestHarness -import org.apache.hudi.util.CloseableInternalRowIterator -import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkUtils} -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.datasources.parquet.{HoodieFileGroupReaderBasedParquetFileFormat, ParquetFileFormat} +import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkUtils, SparkFileFormatInternalRowReaderContext} import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} -import org.junit.jupiter.api.Assertions.{assertArrayEquals, assertEquals, assertTrue} +import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.{BeforeEach, Test} class TestSpark35RecordPositionMetadataColumn extends SparkClientFunctionalTestHarness { private val PARQUET_FORMAT = "parquet" - private val ROW_INDEX_COLUMN = "_tmp_metadata_row_index" private val SPARK_MERGER = "org.apache.hudi.HoodieSparkRecordMerger" @BeforeEach @@ -67,81 +63,49 @@ class TestSpark35RecordPositionMetadataColumn extends SparkClientFunctionalTestH @Test def testRecordPositionColumn(): Unit = { - val _spark = spark - // Prepare the schema - val dataSchema = new StructType( - Array( - StructField("userid", IntegerType, nullable = false), - StructField("country", StringType, nullable = false), - StructField("ts", StringType, nullable = false) + if (HoodieSparkUtils.gteqSpark3_5) { + val _spark = spark + // Prepare the schema + val dataSchema = new StructType( + Array( + StructField("userid", IntegerType, nullable = false), + StructField("country", StringType, nullable = false), + StructField("ts", StringType, nullable = false) + ) ) - ) - val requiredSchema = HoodieFileGroupReaderBasedParquetFileFormat - .getAppliedRequiredSchema( - dataSchema, - shouldUseRecordPosition = true, - ROW_INDEX_COLUMN) + val hadoopConf = new Configuration(spark().sparkContext.hadoopConfiguration) + val props = Map("spark.sql.parquet.enableVectorizedReader" -> "false") + _spark.conf.set("spark.sql.parquet.enableVectorizedReader", "false") + val extraProps = sparkAdapter.getPropsForReadingParquet(vectorized = false, _spark.sessionState.conf, props, hadoopConf) + val requiredSchema = SparkFileFormatInternalRowReaderContext.getAppliedRequiredSchema(dataSchema, shouldUseRecordPosition = true) - // Confirm if the schema is as expected. - if (HoodieSparkUtils.gteqSpark3_5) { + // Confirm if the schema is as expected. assertEquals(4, requiredSchema.fields.length) assertEquals( "StructField(_tmp_metadata_row_index,LongType,false)", requiredSchema.fields(3).toString) - } - // Prepare the file and Parquet file reader. - _spark.conf.set("spark.sql.parquet.enableVectorizedReader", "false") - val metaClient = getHoodieMetaClient( - _spark.sparkContext.hadoopConfiguration, basePath) - val fileReader = new ParquetFileFormat().buildReaderWithPartitionValues( - _spark, - dataSchema, - StructType(Nil), - requiredSchema, - Nil, - Map.empty, - new Configuration(spark().sparkContext.hadoopConfiguration)) - val allBaseFiles = HoodieTestTable.of(metaClient).listAllBaseFiles - assertTrue(allBaseFiles.nonEmpty) + // Prepare the file and Parquet file reader. + val metaClient = getHoodieMetaClient( + _spark.sparkContext.hadoopConfiguration, basePath) + val allBaseFiles = HoodieTestTable.of(metaClient).listAllBaseFiles + assertTrue(allBaseFiles.nonEmpty) + val readerContext = new SparkFileFormatInternalRowReaderContext(extraProps, HoodieRecord.RECORD_KEY_METADATA_FIELD, Seq.empty, true) + //dataschema param is set to null because it is not used + val fileRecordIterator = readerContext.getFileRecordIterator(allBaseFiles.head.getPath, 0, allBaseFiles.head.getLen, null, + sparkAdapter.getAvroSchemaConverters.toAvroType(dataSchema, nullable = true, "record"), hadoopConf, true) - // Make sure we can read all the positions out from base file. - // Here we don't add filters since enabling filter push-down - // for parquet file is tricky. - if (HoodieSparkUtils.gteqSpark3_5) { - val fileInfo = sparkAdapter.getSparkPartitionedFileUtils - .createPartitionedFile( - InternalRow.empty, - allBaseFiles.head.getPath, - 0, - allBaseFiles.head.getLen) - val iterator = new CloseableInternalRowIterator(fileReader.apply(fileInfo)) + // Make sure we can read all the positions out from base file. + // Here we don't add filters since enabling filter push-down + // for parquet file is tricky. var rowIndices: Set[Long] = Set() - while (iterator.hasNext) { - val row = iterator.next() + while (fileRecordIterator.hasNext) { + val row = fileRecordIterator.next() rowIndices += row.getLong(3) } - iterator.close() + fileRecordIterator.close() val expectedRowIndices: Set[Long] = Set(0L, 1L, 2L, 3L) assertEquals(expectedRowIndices, rowIndices) } } - - @Test - def testUseFileGroupReaderDirectly(): Unit = { - val _spark = spark - import _spark.implicits._ - - // Read the records out with record positions. - val allRecords = _spark.read.format("hudi") - .option(HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS.key, "true") - .load(basePath) - - // Ensure the number of outcomes are correct for all Spark versions - // including Spark3.5. - val usRecords = allRecords - .select("userid") - .filter("country = 'US'").map(_.getInt(0)).collect() - assertArrayEquals(Array[Int](1, 3), usRecords) - } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestSpark3DDL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestSpark3DDL.scala index 9f23494ae799..d3685458db62 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestSpark3DDL.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestSpark3DDL.scala @@ -138,6 +138,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { spark.sessionState.catalog.dropTable(TableIdentifier(tableName), true, true) spark.sessionState.catalog.refreshTable(TableIdentifier(tableName)) spark.sessionState.conf.unsetConf(DataSourceWriteOptions.SPARK_SQL_INSERT_INTO_OPERATION.key) + spark.sessionState.conf.unsetConf("spark.sql.storeAssignmentPolicy") } } }) @@ -175,6 +176,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { checkAnswer(spark.sql(s"select col0 from $tableName where id = 1").collect())( Seq("11") ) + spark.sessionState.conf.unsetConf("spark.sql.storeAssignmentPolicy") } } } @@ -336,6 +338,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { spark.sql(s"select id, col1_new, col2 from $tableName where id = 1 or id = 6 or id = 2 or id = 11 order by id").show(false) } } + spark.sessionState.conf.unsetConf("spark.sql.storeAssignmentPolicy") spark.sessionState.conf.unsetConf(DataSourceWriteOptions.SPARK_SQL_INSERT_INTO_OPERATION.key) } } @@ -706,6 +709,8 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { } test("Test schema auto evolution") { + //This test will be flakey for mor until [HUDI-6798] is landed and we can set the merge mode + spark.sql("set hoodie.file.group.reader.enabled=false") withTempDir { tmp => Seq("COPY_ON_WRITE", "MERGE_ON_READ").foreach { tableType => // for complex schema. @@ -799,6 +804,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { } } } + spark.sqlContext.conf.unsetConf("hoodie.file.group.reader.enabled"); } test("Test DATE to STRING conversions when vectorized reading is not enabled") { diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionBase.java index 3670c5b6007a..120154fc9db6 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionBase.java @@ -156,7 +156,7 @@ protected HoodieDeltaStreamer.Config getDeltaStreamerConfig(String[] transformer protected HoodieDeltaStreamer.Config getDeltaStreamerConfig(String[] transformerClasses, boolean nullForDeletedCols, TypedProperties extraProps) throws IOException { - extraProps.setProperty(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), "false"); + extraProps.setProperty(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), "true"); extraProps.setProperty("hoodie.datasource.write.table.type", tableType); extraProps.setProperty("hoodie.datasource.write.row.writer.enable", rowWriterEnable.toString()); extraProps.setProperty(DataSourceWriteOptions.SET_NULL_FOR_MISSING_COLUMNS().key(), Boolean.toString(nullForDeletedCols)); From d59fd1bb95adf43998b20501c4df7f9fce405742 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 4 Apr 2024 14:49:19 -0400 Subject: [PATCH 34/86] finish rebase --- .../SparkFileFormatInternalRowReaderContext.scala | 8 ++++---- ...oodieFileGroupReaderBasedParquetFileFormat.scala | 13 +++++++------ .../read/TestHoodieFileGroupReaderOnSpark.scala | 4 ++-- .../TestSpark35RecordPositionMetadataColumn.scala | 4 ++-- 4 files changed, 15 insertions(+), 14 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala index 96a5634de5a7..0cf15c1a990b 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.avro.HoodieAvroDeserializer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.execution.datasources.PartitionedFile -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, SparkHoodieParquetReader} import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.{LongType, MetadataBuilder, StructField, StructType} import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} @@ -54,7 +54,7 @@ import scala.collection.mutable * not required for reading a file group with only log files. * @param partitionValues The values for a partition in which the file group lives. */ -class SparkFileFormatInternalRowReaderContext(extraProps: Map[String, String], +class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkHoodieParquetReader, recordKeyColumn: String, filters: Seq[Filter], shouldUseRecordPosition: Boolean) extends BaseSparkInternalRowReaderContext { @@ -88,9 +88,9 @@ class SparkFileFormatInternalRowReaderContext(extraProps: Map[String, String], // each row if they are given. That is the only usage of the partition values in the reader. val fileInfo = sparkAdapter.getSparkPartitionedFileUtils .createPartitionedFile(InternalRow.empty, filePath, start, length) - new CloseableInternalRowIterator(sparkAdapter.readParquetFile(fileInfo, + new CloseableInternalRowIterator(parquetFileReader.read(fileInfo, getAppliedRequiredSchema(structType, shouldUseRecordPosition && isMerge), StructType(Seq.empty), - getFiltersForRead(isMerge), conf, extraProps)) + getFiltersForRead(isMerge), conf)) } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala index b9285faab719..938bb792bc6f 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala @@ -128,7 +128,7 @@ override def supportBatch(sparkSession: SparkSession, schema: StructType): Boole val requestedAvroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(requiredSchema, sanitizedTableName) val dataAvroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(dataSchema, sanitizedTableName) - val extraProps = spark.sparkContext.broadcast(sparkAdapter.getPropsForReadingParquet(supportBatchResult, spark.sessionState.conf, options, augmentedHadoopConf)) + val parquetFileReader = spark.sparkContext.broadcast(sparkAdapter.createHoodieParquetFileReader(supportBatchResult, spark.sessionState.conf, options, augmentedHadoopConf)) val broadcastedHadoopConf = spark.sparkContext.broadcast(new SerializableConfiguration(augmentedHadoopConf)) val broadcastedDataSchema = spark.sparkContext.broadcast(dataAvroSchema) val broadcastedRequestedSchema = spark.sparkContext.broadcast(requestedAvroSchema) @@ -146,7 +146,8 @@ override def supportBatch(sparkSession: SparkSession, schema: StructType): Boole val hoodieBaseFile = fileSlice.getBaseFile.get() baseFileReader(createPartitionedFile(fileSliceMapping.getPartitionValues, hoodieBaseFile.getHadoopPath, 0, hoodieBaseFile.getFileLen)) } else { - val readerContext = new SparkFileFormatInternalRowReaderContext(extraProps.value, tableState.recordKeyField, filters, shouldUseRecordPosition) + val readerContext = new SparkFileFormatInternalRowReaderContext(parquetFileReader.value, + tableState.recordKeyField, filters, shouldUseRecordPosition) val serializedHadoopConf = broadcastedHadoopConf.value.value val metaClient: HoodieTableMetaClient = HoodieTableMetaClient .builder().setConf(serializedHadoopConf).setBasePath(tableState.tablePath).build @@ -179,8 +180,8 @@ override def supportBatch(sparkSession: SparkSession, schema: StructType): Boole fileSliceMapping.getPartitionValues) } - case _ => sparkAdapter.readParquetFile(file, requiredSchema, partitionSchema, filters, - broadcastedHadoopConf.value.value, extraProps.value) + case _ => parquetFileReader.value.read(file, requiredSchema, partitionSchema, filters, + broadcastedHadoopConf.value.value) } // CDC queries. case hoodiePartitionCDCFileGroupSliceMapping: HoodiePartitionCDCFileGroupMapping => @@ -189,8 +190,8 @@ override def supportBatch(sparkSession: SparkSession, schema: StructType): Boole buildCDCRecordIterator( fileGroupSplit, cdcFileReader, broadcastedHadoopConf.value.value, fileIndexProps, requiredSchema) - case _ => sparkAdapter.readParquetFile(file, requiredSchema, partitionSchema, filters, - broadcastedHadoopConf.value.value, extraProps.value) + case _ => parquetFileReader.value.read(file, requiredSchema, partitionSchema, filters, + broadcastedHadoopConf.value.value) } } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala index 7ff30d5a66ee..139a38edbf42 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala @@ -80,7 +80,7 @@ class TestHoodieFileGroupReaderOnSpark extends TestHoodieFileGroupReaderBase[Int } override def getHoodieReaderContext(tablePath: String, avroSchema: Schema, hadoopConf: Configuration): HoodieReaderContext[InternalRow] = { - val extraProps = sparkAdapter.getPropsForReadingParquet(vectorized = false, spark.sessionState.conf, Map.empty, hadoopConf) + val reader = sparkAdapter.createHoodieParquetFileReader(vectorized = false, spark.sessionState.conf, Map.empty, hadoopConf) val metaClient = HoodieTableMetaClient.builder().setConf(getHadoopConf).setBasePath(tablePath).build val recordKeyField = if (metaClient.getTableConfig.populateMetaFields()) { HoodieRecord.RECORD_KEY_METADATA_FIELD @@ -89,7 +89,7 @@ class TestHoodieFileGroupReaderOnSpark extends TestHoodieFileGroupReaderBase[Int checkState(keyFields.length == 1) keyFields.head } - new SparkFileFormatInternalRowReaderContext(extraProps, recordKeyField, Seq.empty, false) + new SparkFileFormatInternalRowReaderContext(reader, recordKeyField, Seq.empty, false) } override def commitToTable(recordList: util.List[String], operation: String, options: util.Map[String, String]): Unit = { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala index 959648fdbd8f..617cfbee7567 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala @@ -76,7 +76,7 @@ class TestSpark35RecordPositionMetadataColumn extends SparkClientFunctionalTestH val hadoopConf = new Configuration(spark().sparkContext.hadoopConfiguration) val props = Map("spark.sql.parquet.enableVectorizedReader" -> "false") _spark.conf.set("spark.sql.parquet.enableVectorizedReader", "false") - val extraProps = sparkAdapter.getPropsForReadingParquet(vectorized = false, _spark.sessionState.conf, props, hadoopConf) + val reader = sparkAdapter.createHoodieParquetFileReader(vectorized = false, _spark.sessionState.conf, props, hadoopConf) val requiredSchema = SparkFileFormatInternalRowReaderContext.getAppliedRequiredSchema(dataSchema, shouldUseRecordPosition = true) // Confirm if the schema is as expected. @@ -90,7 +90,7 @@ class TestSpark35RecordPositionMetadataColumn extends SparkClientFunctionalTestH _spark.sparkContext.hadoopConfiguration, basePath) val allBaseFiles = HoodieTestTable.of(metaClient).listAllBaseFiles assertTrue(allBaseFiles.nonEmpty) - val readerContext = new SparkFileFormatInternalRowReaderContext(extraProps, HoodieRecord.RECORD_KEY_METADATA_FIELD, Seq.empty, true) + val readerContext = new SparkFileFormatInternalRowReaderContext(reader, HoodieRecord.RECORD_KEY_METADATA_FIELD, Seq.empty, true) //dataschema param is set to null because it is not used val fileRecordIterator = readerContext.getFileRecordIterator(allBaseFiles.head.getPath, 0, allBaseFiles.head.getLen, null, sparkAdapter.getAvroSchemaConverters.toAvroType(dataSchema, nullable = true, "record"), hadoopConf, true) From b7a7c272eb2c884e42016628b10e3030ebf43869 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Fri, 5 Apr 2024 12:33:25 -0400 Subject: [PATCH 35/86] use augment hadoop conf --- .../parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala index 938bb792bc6f..7d2aaa47adf5 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala @@ -116,7 +116,7 @@ override def supportBatch(sparkSession: SparkSession, schema: StructType): Boole val augmentedHadoopConf = FSUtils.buildInlineConf(hadoopConf) setSchemaEvolutionConfigs(augmentedHadoopConf, options) val baseFileReader = super.buildReaderWithPartitionValues(spark, dataSchema, partitionSchema, requiredSchema, - filters ++ requiredFilters, options, new Configuration(hadoopConf)) + filters ++ requiredFilters, options, new Configuration(augmentedHadoopConf)) val cdcFileReader = super.buildReaderWithPartitionValues( spark, tableSchema.structTypeSchema, From 8319069bcc7de044f8ee0e289792613ca57215ac Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Fri, 5 Apr 2024 15:09:56 -0400 Subject: [PATCH 36/86] clone hadoop conf for log file read --- .../io/storage/HoodieSparkFileReaderFactory.java | 13 +++++++------ .../parquet/SparkHoodieParquetReaderBase.scala | 8 ++++---- 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReaderFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReaderFactory.java index d06b69139059..60ae86d9e122 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReaderFactory.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReaderFactory.java @@ -33,15 +33,16 @@ public class HoodieSparkFileReaderFactory extends HoodieFileReaderFactory { @Override public HoodieFileReader newParquetFileReader(Configuration conf, Path path) { - conf.setIfUnset(SQLConf.PARQUET_BINARY_AS_STRING().key(), SQLConf.PARQUET_BINARY_AS_STRING().defaultValueString()); - conf.setIfUnset(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), SQLConf.PARQUET_INT96_AS_TIMESTAMP().defaultValueString()); - conf.setIfUnset(SQLConf.CASE_SENSITIVE().key(), SQLConf.CASE_SENSITIVE().defaultValueString()); + Configuration configClone = new Configuration(conf); + configClone.setIfUnset(SQLConf.PARQUET_BINARY_AS_STRING().key(), SQLConf.PARQUET_BINARY_AS_STRING().defaultValueString()); + configClone.setIfUnset(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), SQLConf.PARQUET_INT96_AS_TIMESTAMP().defaultValueString()); + configClone.setIfUnset(SQLConf.CASE_SENSITIVE().key(), SQLConf.CASE_SENSITIVE().defaultValueString()); // Using string value of this conf to preserve compatibility across spark versions. - conf.setIfUnset("spark.sql.legacy.parquet.nanosAsLong", "false"); + configClone.setIfUnset("spark.sql.legacy.parquet.nanosAsLong", "false"); // This is a required config since Spark 3.4.0: SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED // Using string value of this conf to preserve compatibility across spark versions. - conf.setIfUnset("spark.sql.parquet.inferTimestampNTZ.enabled", "true"); - return new HoodieSparkParquetReader(conf, path); + configClone.setIfUnset("spark.sql.parquet.inferTimestampNTZ.enabled", "true"); + return new HoodieSparkParquetReader(configClone, path); } @Override diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala index 382c700595e3..c9e6c8d33a20 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala @@ -51,10 +51,10 @@ abstract class SparkHoodieParquetReaderBase(enableVectorizedReader: Boolean, * @return iterator of rows read from the file output type says [[InternalRow]] but could be [[ColumnarBatch]] */ final def read(file: PartitionedFile, - requiredSchema: StructType, - partitionSchema: StructType, - filters: Seq[Filter], - sharedConf: Configuration): Iterator[InternalRow] = { + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[Filter], + sharedConf: Configuration): Iterator[InternalRow] = { sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) ParquetWriteSupport.setSchema(requiredSchema, sharedConf) From 28200fd3db4c98241062c682178291635e972c98 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Fri, 5 Apr 2024 15:43:10 -0400 Subject: [PATCH 37/86] try copying conf in reader --- .../parquet/SparkHoodieParquetReaderBase.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala index c9e6c8d33a20..b89d96433942 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala @@ -55,10 +55,11 @@ abstract class SparkHoodieParquetReaderBase(enableVectorizedReader: Boolean, partitionSchema: StructType, filters: Seq[Filter], sharedConf: Configuration): Iterator[InternalRow] = { - sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) - sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) - ParquetWriteSupport.setSchema(requiredSchema, sharedConf) - doRead(file, requiredSchema, partitionSchema, filters, sharedConf) + val conf = new Configuration(sharedConf) + conf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) + conf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) + ParquetWriteSupport.setSchema(requiredSchema, conf) + doRead(file, requiredSchema, partitionSchema, filters, conf) } From 8ca12f25bce0d5cfb2dc56785db076e0fa90df33 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Fri, 5 Apr 2024 16:47:37 -0400 Subject: [PATCH 38/86] create a copy of the conf when reading --- .../parquet/SparkHoodieParquetReaderBase.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala index 382c700595e3..c74810db2b6b 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala @@ -55,10 +55,11 @@ abstract class SparkHoodieParquetReaderBase(enableVectorizedReader: Boolean, partitionSchema: StructType, filters: Seq[Filter], sharedConf: Configuration): Iterator[InternalRow] = { - sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) - sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) - ParquetWriteSupport.setSchema(requiredSchema, sharedConf) - doRead(file, requiredSchema, partitionSchema, filters, sharedConf) + val conf = new Configuration(sharedConf) + conf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) + conf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) + ParquetWriteSupport.setSchema(requiredSchema, conf) + doRead(file, requiredSchema, partitionSchema, filters, conf) } From 088f69ed54db32d1686caa4f4571111f6fc9aed0 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Fri, 5 Apr 2024 16:48:41 -0400 Subject: [PATCH 39/86] make conf copy during read --- .../parquet/SparkHoodieParquetReaderBase.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala index 382c700595e3..c74810db2b6b 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala @@ -55,10 +55,11 @@ abstract class SparkHoodieParquetReaderBase(enableVectorizedReader: Boolean, partitionSchema: StructType, filters: Seq[Filter], sharedConf: Configuration): Iterator[InternalRow] = { - sharedConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) - sharedConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) - ParquetWriteSupport.setSchema(requiredSchema, sharedConf) - doRead(file, requiredSchema, partitionSchema, filters, sharedConf) + val conf = new Configuration(sharedConf) + conf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) + conf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) + ParquetWriteSupport.setSchema(requiredSchema, conf) + doRead(file, requiredSchema, partitionSchema, filters, conf) } From c5d77d501854a7e4af2a5d1460a4173dc13100ec Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 9 Apr 2024 13:27:30 -0400 Subject: [PATCH 40/86] recordkey filters is reverse --- .../apache/hudi/SparkFileFormatInternalRowReaderContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala index 0cf15c1a990b..58cd7e2a68cd 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala @@ -97,7 +97,7 @@ class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkHoodieParq private def getFiltersForRead(isMerge: Boolean): Seq[Filter] = { if (!isMerge) { filters - } else if (!shouldUseRecordPosition) { + } else if (shouldUseRecordPosition) { recordKeyFilters } else { Seq.empty From 3b8b4f1416e0d24dfd11e0eafe19d33eb7520077 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 9 Apr 2024 21:01:44 -0400 Subject: [PATCH 41/86] fix position based merging --- .../BaseSparkInternalRowReaderContext.java | 5 + ...rkFileFormatInternalRowReaderContext.scala | 57 ++++-- .../common/engine/HoodieReaderContext.java | 9 +- .../table/log/BaseHoodieLogRecordReader.java | 25 +-- .../log/HoodieMergedLogRecordReader.java | 53 ++--- .../log/block/HoodieParquetDataBlock.java | 2 +- .../read/HoodieBaseFileGroupRecordBuffer.java | 12 +- .../table/read/HoodieFileGroupReader.java | 144 ++------------ .../HoodieFileGroupReaderSchemaHandler.java | 183 ++++++++++++++++++ .../read/HoodieFileGroupReaderState.java | 17 +- .../HoodieKeyBasedFileGroupRecordBuffer.java | 11 +- ...diePositionBasedFileGroupRecordBuffer.java | 8 +- .../HoodiePositionBasedSchemaHandler.java | 68 +++++++ .../HoodieUnmergedFileGroupRecordBuffer.java | 8 +- .../reader/HoodieTestReaderContext.java | 3 +- ...diePositionBasedFileGroupRecordBuffer.java | 4 - .../TestHoodieFileGroupReaderOnSpark.scala | 6 +- ...tSpark35RecordPositionMetadataColumn.scala | 7 +- 18 files changed, 374 insertions(+), 248 deletions(-) create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedSchemaHandler.java diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java index a07602b3885d..552d18ccd9b5 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java @@ -43,6 +43,7 @@ import java.util.Collections; import java.util.Map; +import java.util.function.Function; import java.util.function.UnaryOperator; import scala.Function1; @@ -154,4 +155,8 @@ public UnaryOperator projectRecordUnsafe(Schema from, Schema to, Ma HoodieInternalRowUtils.getCachedUnsafeRowWriter(structType, newStructType, renamedColumns); return row -> (InternalRow) unsafeRowWriter.apply(row); } + + protected UnaryOperator getIdentityProjection() { + return row -> row; + } } \ No newline at end of file diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala index 58cd7e2a68cd..b8a1dd29ffdf 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala @@ -23,7 +23,7 @@ import org.apache.avro.Schema import org.apache.avro.generic.IndexedRecord import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hudi.SparkFileFormatInternalRowReaderContext.{ROW_INDEX_TEMPORARY_COLUMN_NAME, getAppliedRequiredSchema} +import org.apache.hudi.SparkFileFormatInternalRowReaderContext.{ROW_INDEX_TEMPORARY_COLUMN_NAME, getAppliedRequiredSchema, hasIndexTempColumn} import org.apache.hudi.avro.AvroSchemaUtils import org.apache.hudi.common.engine.HoodieReaderContext import org.apache.hudi.common.fs.FSUtils @@ -68,8 +68,7 @@ class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkHoodieParq length: Long, dataSchema: Schema, requiredSchema: Schema, - conf: Configuration, - isMerge: Boolean): ClosableIterator[InternalRow] = { + conf: Configuration): ClosableIterator[InternalRow] = { val structType: StructType = HoodieInternalRowUtils.getCachedSchema(requiredSchema) if (FSUtils.isLogFile(filePath)) { val projection: UnsafeProjection = HoodieInternalRowUtils.getCachedUnsafeProjection(structType, structType) @@ -88,20 +87,23 @@ class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkHoodieParq // each row if they are given. That is the only usage of the partition values in the reader. val fileInfo = sparkAdapter.getSparkPartitionedFileUtils .createPartitionedFile(InternalRow.empty, filePath, start, length) + val (readSchema, readFilters) = getSchemaAndFiltersForRead(structType) new CloseableInternalRowIterator(parquetFileReader.read(fileInfo, - getAppliedRequiredSchema(structType, shouldUseRecordPosition && isMerge), StructType(Seq.empty), - getFiltersForRead(isMerge), conf)) + readSchema, StructType(Seq.empty), readFilters, conf)) } } - private def getFiltersForRead(isMerge: Boolean): Seq[Filter] = { - if (!isMerge) { - filters - } else if (shouldUseRecordPosition) { - recordKeyFilters - } else { - Seq.empty - } + private def getSchemaAndFiltersForRead(structType: StructType): (StructType, Seq[Filter]) = { + (readerState.hasLogFiles, readerState.needsBootstrapMerge, shouldUseRecordPosition) match { + case (false, false, _) => + (structType, filters) + case (false, true, true) if shouldUseRecordPosition => + (getAppliedRequiredSchema(structType), filters) + case (true, _, true) if shouldUseRecordPosition => + (getAppliedRequiredSchema(structType), recordKeyFilters) + case (_, _, _) => + (structType, Seq.empty) + } } /** @@ -133,10 +135,20 @@ class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkHoodieParq dataRequiredSchema: Schema): ClosableIterator[InternalRow] = { if (shouldUseRecordPosition) { assert(AvroSchemaUtils.containsFieldInSchema(skeletonRequiredSchema, ROW_INDEX_TEMPORARY_COLUMN_NAME)) + assert(AvroSchemaUtils.containsFieldInSchema(dataRequiredSchema, ROW_INDEX_TEMPORARY_COLUMN_NAME)) val javaSet = new java.util.HashSet[String]() javaSet.add(ROW_INDEX_TEMPORARY_COLUMN_NAME) val skeletonProjection = projectRecord(skeletonRequiredSchema, AvroSchemaUtils.removeFieldsFromSchema(skeletonRequiredSchema, javaSet)) + //If we have log files, we will want to do position based merging with those as well, + //so leave a temporary column at the end + val dataProjection = if (readerState.hasLogFiles) { + getIdentityProjection + } else { + projectRecord(dataRequiredSchema, + AvroSchemaUtils.removeFieldsFromSchema(dataRequiredSchema, javaSet)) + } + new CachingIterator[InternalRow] { val combinedRow = new JoinedRow() @@ -180,7 +192,7 @@ class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkHoodieParq } } } - nextRecord = combinedRow(skeletonProjection.apply(nextSkeleton._1), nextData._1) + nextRecord = combinedRow(skeletonProjection.apply(nextSkeleton._1), dataProjection.apply(nextData._1)) true } } @@ -245,18 +257,21 @@ object SparkFileFormatInternalRowReaderContext { filters.filter(f => f.references.exists(c => c.equalsIgnoreCase(recordKeyColumn))) } - def getAppliedRequiredSchema(requiredSchema: StructType, - shouldUseRecordPosition: Boolean): StructType = { - if (shouldUseRecordPosition) { + def hasIndexTempColumn(structType: StructType): Boolean = { + structType.fields.exists(isIndexTempColumn) + } + + def isIndexTempColumn(field: StructField): Boolean = { + field.name.equals(ROW_INDEX_TEMPORARY_COLUMN_NAME) + } + + def getAppliedRequiredSchema(requiredSchema: StructType): StructType = { val metadata = new MetadataBuilder() .putString(METADATA_COL_ATTR_KEY, ROW_INDEX_TEMPORARY_COLUMN_NAME) .putBoolean(FILE_SOURCE_METADATA_COL_ATTR_KEY, value = true) .putString(FILE_SOURCE_GENERATED_METADATA_COL_ATTR_KEY, ROW_INDEX_TEMPORARY_COLUMN_NAME) .build() val rowIndexField = StructField(ROW_INDEX_TEMPORARY_COLUMN_NAME, LongType, nullable = false, metadata) - StructType(requiredSchema.fields :+ rowIndexField) - } else { - requiredSchema - } + StructType(requiredSchema.fields.filterNot(isIndexTempColumn) :+ rowIndexField) } } \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java index c0c0e40206f7..d7b9f0cf02cb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.table.read.HoodieFileGroupReaderState; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ClosableIterator; @@ -48,6 +49,12 @@ * and {@code RowData} in Flink. */ public abstract class HoodieReaderContext { + protected HoodieFileGroupReaderState readerState = new HoodieFileGroupReaderState<>(); + + public HoodieFileGroupReaderState getReaderState() { + return readerState; + } + // These internal key names are only used in memory for record metadata and merging, // and should not be persisted to storage. public static final String INTERNAL_META_RECORD_KEY = "_0"; @@ -79,7 +86,7 @@ public abstract class HoodieReaderContext { * @return {@link ClosableIterator} that can return all records through iteration. */ public abstract ClosableIterator getFileRecordIterator( - Path filePath, long start, long length, Schema dataSchema, Schema requiredSchema, Configuration conf, boolean isMerge) throws IOException; + Path filePath, long start, long length, Schema dataSchema, Schema requiredSchema, Configuration conf) throws IOException; /** * Converts an Avro record, e.g., serialized in the log files, to an engine-specific record. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java index e5ee5ccd7e87..da5399f77ccf 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java @@ -31,6 +31,7 @@ import org.apache.hudi.common.table.log.block.HoodieDataBlock; import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock; +import org.apache.hudi.common.table.read.HoodieFileGroupReaderState; import org.apache.hudi.common.table.read.HoodieFileGroupRecordBuffer; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; @@ -86,6 +87,7 @@ public abstract class BaseHoodieLogRecordReader { // Log-Blocks belonging to inflight delta-instants are filtered-out using this high-watermark. private final String latestInstantTime; protected final HoodieReaderContext readerContext; + protected final HoodieFileGroupReaderState readerState; protected final HoodieTableMetaClient hoodieTableMetaClient; // Merge strategy to use when combining records from log private final String payloadClassFQN; @@ -141,20 +143,21 @@ public abstract class BaseHoodieLogRecordReader { protected HoodieFileGroupRecordBuffer recordBuffer; protected BaseHoodieLogRecordReader(HoodieReaderContext readerContext, - FileSystem fs, String basePath, List logFilePaths, - Schema readerSchema, String latestInstantTime, boolean readBlocksLazily, + FileSystem fs, + List logFilePaths, + boolean readBlocksLazily, boolean reverseReader, int bufferSize, Option instantRange, boolean withOperationField, boolean forceFullScan, Option partitionNameOverride, - InternalSchema internalSchema, Option keyFieldOverride, boolean enableOptimizedLogBlocksScan, HoodieRecordMerger recordMerger, HoodieFileGroupRecordBuffer recordBuffer) { this.readerContext = readerContext; - this.readerSchema = readerSchema; - this.latestInstantTime = latestInstantTime; - this.hoodieTableMetaClient = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(basePath).build(); + this.readerState = readerContext.getReaderState(); + this.readerSchema = readerState.schemaHandler.getRequiredSchema(); + this.latestInstantTime = readerState.latestCommitTime; + this.hoodieTableMetaClient = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(readerState.tablePath).build(); // load class from the payload fully qualified class name HoodieTableConfig tableConfig = this.hoodieTableMetaClient.getTableConfig(); this.payloadClassFQN = tableConfig.getPayloadClass(); @@ -174,7 +177,7 @@ protected BaseHoodieLogRecordReader(HoodieReaderContext readerContext, this.instantRange = instantRange; this.withOperationField = withOperationField; this.forceFullScan = forceFullScan; - this.internalSchema = internalSchema == null ? InternalSchema.getEmptyInternalSchema() : internalSchema; + this.internalSchema = readerState.schemaHandler.getInternalSchema(); this.enableOptimizedLogBlocksScan = enableOptimizedLogBlocksScan; if (keyFieldOverride.isPresent()) { @@ -841,16 +844,8 @@ public abstract static class Builder { public abstract Builder withFileSystem(FileSystem fs); - public abstract Builder withBasePath(String basePath); - public abstract Builder withLogFiles(List hoodieLogFiles); - public abstract Builder withReaderSchema(Schema schema); - - public abstract Builder withInternalSchema(InternalSchema internalSchema); - - public abstract Builder withLatestInstantTime(String latestInstantTime); - public abstract Builder withReadBlocksLazily(boolean readBlocksLazily); public abstract Builder withReverseReader(boolean reverseReader); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordReader.java index 6b31c2009072..b100b3773175 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordReader.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodiePreCombineAvroRecordMerger; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.table.read.HoodieFileGroupReaderState; import org.apache.hudi.common.table.read.HoodieFileGroupRecordBuffer; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.HoodieRecordUtils; @@ -31,9 +32,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.internal.schema.InternalSchema; -import org.apache.avro.Schema; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.slf4j.Logger; @@ -70,18 +69,16 @@ public class HoodieMergedLogRecordReader extends BaseHoodieLogRecordReader @SuppressWarnings("unchecked") private HoodieMergedLogRecordReader(HoodieReaderContext readerContext, - FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, - String latestInstantTime, boolean readBlocksLazily, + FileSystem fs, List logFilePaths, boolean readBlocksLazily, boolean reverseReader, int bufferSize, Option instantRange, boolean withOperationField, boolean forceFullScan, Option partitionName, - InternalSchema internalSchema, Option keyFieldOverride, boolean enableOptimizedLogBlocksScan, HoodieRecordMerger recordMerger, HoodieFileGroupRecordBuffer recordBuffer) { - super(readerContext, fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, - instantRange, withOperationField, forceFullScan, partitionName, internalSchema, keyFieldOverride, enableOptimizedLogBlocksScan, + super(readerContext, fs, logFilePaths, readBlocksLazily, reverseReader, bufferSize, + instantRange, withOperationField, forceFullScan, partitionName, keyFieldOverride, enableOptimizedLogBlocksScan, recordMerger, recordBuffer); this.scannedPrefixes = new HashSet<>(); @@ -217,12 +214,9 @@ public void close() { */ public static class Builder extends BaseHoodieLogRecordReader.Builder { private HoodieReaderContext readerContext; + private HoodieFileGroupReaderState readerState; private FileSystem fs; - private String basePath; private List logFilePaths; - private Schema readerSchema; - private InternalSchema internalSchema = InternalSchema.getEmptyInternalSchema(); - private String latestInstantTime; private boolean readBlocksLazily; private boolean reverseReader; private int bufferSize; @@ -253,12 +247,6 @@ public Builder withFileSystem(FileSystem fs) { return this; } - @Override - public Builder withBasePath(String basePath) { - this.basePath = basePath; - return this; - } - @Override public Builder withLogFiles(List hoodieLogFiles) { this.logFilePaths = hoodieLogFiles.stream() @@ -268,18 +256,6 @@ public Builder withLogFiles(List hoodieLogFiles) { return this; } - @Override - public Builder withReaderSchema(Schema schema) { - this.readerSchema = schema; - return this; - } - - @Override - public Builder withLatestInstantTime(String latestInstantTime) { - this.latestInstantTime = latestInstantTime; - return this; - } - @Override public Builder withReadBlocksLazily(boolean readBlocksLazily) { this.readBlocksLazily = readBlocksLazily; @@ -304,12 +280,6 @@ public Builder withInstantRange(Option instantRange) { return this; } - @Override - public Builder withInternalSchema(InternalSchema internalSchema) { - this.internalSchema = internalSchema; - return this; - } - public Builder withOperationField(boolean withOperationField) { this.withOperationField = withOperationField; return this; @@ -350,18 +320,19 @@ public Builder withRecordBuffer(HoodieFileGroupRecordBuffer recordBuffer) @Override public HoodieMergedLogRecordReader build() { - if (this.partitionName == null && CollectionUtils.nonEmpty(this.logFilePaths)) { - this.partitionName = getRelativePartitionPath(new Path(basePath), new Path(this.logFilePaths.get(0)).getParent()); - } ValidationUtils.checkArgument(recordMerger != null); ValidationUtils.checkArgument(recordBuffer != null); + ValidationUtils.checkArgument(readerContext != null); + if (this.partitionName == null && CollectionUtils.nonEmpty(this.logFilePaths)) { + this.partitionName = getRelativePartitionPath(new Path(readerState.tablePath), new Path(this.logFilePaths.get(0)).getParent()); + } return new HoodieMergedLogRecordReader<>( - readerContext, fs, basePath, logFilePaths, readerSchema, - latestInstantTime, readBlocksLazily, reverseReader, + readerContext, fs, logFilePaths, + readBlocksLazily, reverseReader, bufferSize, instantRange, withOperationField, forceFullScan, - Option.ofNullable(partitionName), internalSchema, + Option.ofNullable(partitionName), Option.ofNullable(keyFieldOverride), enableOptimizedLogBlocksScan, recordMerger, recordBuffer); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java index 03828f951077..77f784dfc388 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java @@ -184,7 +184,7 @@ protected ClosableIterator readRecordsFromBlockPayload(HoodieReaderContex Schema writerSchema = new Schema.Parser().parse(this.getLogBlockHeader().get(HeaderMetadataType.SCHEMA)); return readerContext.getFileRecordIterator( - inlineLogFilePath, 0, blockContentLoc.getBlockSize(), writerSchema, readerSchema, inlineConf, false); + inlineLogFilePath, 0, blockContentLoc.getBlockSize(), writerSchema, readerSchema, inlineConf); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java index 1ee4b2e36d9b..3029e1942d48 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java @@ -62,8 +62,8 @@ public abstract class HoodieBaseFileGroupRecordBuffer implements HoodieFileGroupRecordBuffer { protected final HoodieReaderContext readerContext; + protected final HoodieFileGroupReaderState readerState; protected final Schema readerSchema; - protected final Schema baseFileSchema; protected final Option partitionNameOverrideOpt; protected final Option partitionPathFieldOpt; protected final HoodieRecordMerger recordMerger; @@ -77,9 +77,6 @@ public abstract class HoodieBaseFileGroupRecordBuffer implements HoodieFileGr protected HoodieTableMetaClient hoodieTableMetaClient; public HoodieBaseFileGroupRecordBuffer(HoodieReaderContext readerContext, - Schema readerSchema, - Schema baseFileSchema, - InternalSchema internalSchema, HoodieTableMetaClient hoodieTableMetaClient, Option partitionNameOverrideOpt, Option partitionPathFieldOpt, @@ -90,14 +87,13 @@ public HoodieBaseFileGroupRecordBuffer(HoodieReaderContext readerContext, ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled) { this.readerContext = readerContext; - this.readerSchema = readerSchema; - this.baseFileSchema = baseFileSchema; + this.readerState = readerContext.getReaderState(); + this.readerSchema = readerState.schemaHandler.requiredSchema; this.partitionNameOverrideOpt = partitionNameOverrideOpt; this.partitionPathFieldOpt = partitionPathFieldOpt; this.recordMerger = recordMerger; this.payloadProps = payloadProps; - this.internalSchema = internalSchema == null || internalSchema.isEmptySchema() - ? InternalSchema.getEmptyInternalSchema() : AvroInternalSchemaConverter.pruneAvroSchemaToInternalSchema(readerSchema, internalSchema); + this.internalSchema = readerState.schemaHandler.getInternalSchema(); this.hoodieTableMetaClient = hoodieTableMetaClient; try { // Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java index 010f6ad4bb1e..7e059bbb79e1 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java @@ -27,7 +27,6 @@ import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieLogFile; -import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -48,17 +47,10 @@ import java.io.Closeable; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; import java.util.List; -import java.util.Map; -import java.util.Objects; import java.util.function.UnaryOperator; import java.util.stream.Collectors; -import java.util.stream.Stream; -import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchemaDedupNested; -import static org.apache.hudi.avro.AvroSchemaUtils.findNestedField; import static org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath; import static org.apache.hudi.common.util.ConfigUtils.getBooleanWithAltKeys; import static org.apache.hudi.common.util.ConfigUtils.getIntWithAltKeys; @@ -84,27 +76,11 @@ public final class HoodieFileGroupReader implements Closeable { private final long length; // Core structure to store and process records. private final HoodieFileGroupRecordBuffer recordBuffer; - private final HoodieFileGroupReaderState readerState = new HoodieFileGroupReaderState(); + private final HoodieFileGroupReaderState readerState; private ClosableIterator baseFileIterator; private HoodieRecordMerger recordMerger; - - private final Schema dataSchema; - - // requestedSchema: the schema that the caller requests - private final Schema requestedSchema; - - // requiredSchema: the requestedSchema with any additional columns required for merging etc - private final Schema requiredSchema; - - private final HoodieTableConfig hoodieTableConfig; - private final Option> outputConverter; - - private final InternalSchema internalSchema; - - private boolean needMORMerge; - private boolean needMerge; - + private final HoodieFileGroupReaderSchemaHandler schemaHandler; private final boolean shouldMergeWithPosition; public HoodieFileGroupReader(HoodieReaderContext readerContext, @@ -125,8 +101,8 @@ public HoodieFileGroupReader(HoodieReaderContext readerContext, String spillableMapBasePath, ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled) { - this.internalSchema = internalSchemaOpt.orElse(InternalSchema.getEmptyInternalSchema()); this.readerContext = readerContext; + this.readerState = readerContext.getReaderState(); this.hadoopConf = hadoopConf; this.hoodieBaseFileOption = fileSlice.getBaseFile(); this.logFiles = fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); @@ -136,28 +112,20 @@ public HoodieFileGroupReader(HoodieReaderContext readerContext, this.recordMerger = readerContext.getRecordMerger(tableConfig.getRecordMergerStrategy()); this.readerState.tablePath = tablePath; this.readerState.latestCommitTime = latestCommitTime; - this.dataSchema = dataSchema; - this.requestedSchema = requestedSchema; - this.hoodieTableConfig = tableConfig; this.shouldMergeWithPosition = shouldUseRecordPosition && readerContext.shouldUseRecordPositionMerging(); - this.requiredSchema = prepareSchema(); - if (!requestedSchema.equals(requiredSchema)) { - this.outputConverter = Option.of(readerContext.projectRecord(requiredSchema, requestedSchema)); - } else { - this.outputConverter = Option.empty(); - } - this.readerState.baseFileAvroSchema = requiredSchema; - this.readerState.logRecordAvroSchema = requiredSchema; - this.readerState.mergeProps.putAll(props); + readerState.hasBootstrapBaseFile = hoodieBaseFileOption.isPresent() && hoodieBaseFileOption.get().getBootstrapBaseFile().isPresent(); + readerState.schemaHandler = shouldMergeWithPosition + ? new HoodiePositionBasedSchemaHandler<>(readerContext, dataSchema, requestedSchema, internalSchemaOpt, tableConfig) + : new HoodieFileGroupReaderSchemaHandler<>(readerContext, dataSchema, requestedSchema, internalSchemaOpt, tableConfig); + this.schemaHandler = readerState.schemaHandler; + this.outputConverter = schemaHandler.getOutputConverter(); this.recordBuffer = this.logFiles.isEmpty() ? null : this.shouldMergeWithPosition - ? new HoodiePositionBasedFileGroupRecordBuffer<>( - readerContext, requiredSchema, requiredSchema, internalSchema, hoodieTableMetaClient, Option.empty(), Option.empty(), - recordMerger, props, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled) - : new HoodieKeyBasedFileGroupRecordBuffer<>( - readerContext, requiredSchema, requiredSchema, internalSchema, hoodieTableMetaClient, Option.empty(), Option.empty(), - recordMerger, props, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled); + ? new HoodiePositionBasedFileGroupRecordBuffer<>(readerContext, hoodieTableMetaClient, Option.empty(), + Option.empty(), recordMerger, props, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled) + : new HoodieKeyBasedFileGroupRecordBuffer<>(readerContext, hoodieTableMetaClient, Option.empty(), + Option.empty(), recordMerger, props, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled); } /** @@ -185,82 +153,13 @@ private ClosableIterator makeBaseFileIterator() throws IOException { } return readerContext.getFileRecordIterator(baseFile.getHadoopPath(), start, length, - dataSchema, requiredSchema, hadoopConf, this.needMerge); - } - - private Schema generateRequiredSchema() { - //might need to change this if other queries than mor have mandatory fields - if (!needMORMerge) { - return requestedSchema; - } - - List addedFields = new ArrayList<>(); - for (String field : recordMerger.getMandatoryFieldsForMerging(hoodieTableConfig)) { - if (!findNestedField(requestedSchema, field).isPresent()) { - Option foundFieldOpt = findNestedField(dataSchema, field); - if (!foundFieldOpt.isPresent()) { - throw new IllegalArgumentException("Field: " + field + " does not exist in the table schema"); - } - Schema.Field foundField = foundFieldOpt.get(); - addedFields.add(foundField); - } - } - - if (addedFields.isEmpty()) { - return requestedSchema; - } - - return appendFieldsToSchemaDedupNested(requestedSchema, addedFields); - } - - private Schema prepareSchema() { - this.needMORMerge = !logFiles.isEmpty(); - Schema preReorderRequiredSchema = generateRequiredSchema(); - Pair, List> requiredFields = getDataAndMetaCols(preReorderRequiredSchema); - boolean needBootstrapMerge = hoodieBaseFileOption.isPresent() && hoodieBaseFileOption.get().getBootstrapBaseFile().isPresent() - && !requiredFields.getLeft().isEmpty() && !requiredFields.getRight().isEmpty(); - this.needMerge = needBootstrapMerge || this.needMORMerge; - Schema preMergeSchema = needBootstrapMerge - ? createSchemaFromFields(Stream.concat(requiredFields.getLeft().stream(), requiredFields.getRight().stream()).collect(Collectors.toList())) - : preReorderRequiredSchema; - return this.shouldMergeWithPosition && this.needMerge - ? addPositionalMergeCol(preMergeSchema) - : preMergeSchema; - } - - private Schema addPositionalMergeCol(Schema input) { - return appendFieldsToSchemaDedupNested(input, Collections.singletonList(getPositionalMergeField())); - } - - private Schema.Field getPositionalMergeField() { - return new Schema.Field(HoodiePositionBasedFileGroupRecordBuffer.ROW_INDEX_TEMPORARY_COLUMN_NAME, - Schema.create(Schema.Type.LONG), "", 0L); - } - - private static Pair, List> getDataAndMetaCols(Schema schema) { - Map> fieldsByMeta = schema.getFields().stream() - //if there are no data fields, then we don't want to think the temp col is a data col - .filter(f -> !Objects.equals(f.name(), HoodiePositionBasedFileGroupRecordBuffer.ROW_INDEX_TEMPORARY_COLUMN_NAME)) - .collect(Collectors.partitioningBy(f -> HoodieRecord.HOODIE_META_COLUMNS_WITH_OPERATION.contains(f.name()))); - return Pair.of(fieldsByMeta.getOrDefault(true, Collections.emptyList()), - fieldsByMeta.getOrDefault(false, Collections.emptyList())); - } - - private Schema createSchemaFromFields(List fields) { - //fields have positions set, so we need to remove them due to avro setFields implementation - for (int i = 0; i < fields.size(); i++) { - Schema.Field curr = fields.get(i); - fields.set(i, new Schema.Field(curr.name(), curr.schema(), curr.doc(), curr.defaultVal())); - } - Schema newSchema = Schema.createRecord(dataSchema.getName(), dataSchema.getDoc(), dataSchema.getNamespace(), dataSchema.isError()); - newSchema.setFields(fields); - return newSchema; + schemaHandler.getDataSchema(), schemaHandler.getRequiredSchema(), hadoopConf); } private ClosableIterator makeBootstrapBaseFileIterator(HoodieBaseFile baseFile) throws IOException { BaseFile dataFile = baseFile.getBootstrapBaseFile().get(); - Pair,List> requiredFields = getDataAndMetaCols(requiredSchema); - Pair,List> allFields = getDataAndMetaCols(dataSchema); + Pair,List> requiredFields = schemaHandler.getBootstrapRequiredFields(); + Pair,List> allFields = schemaHandler.getBootstrapDataFields(); Option,Schema>> dataFileIterator = makeBootstrapBaseFileIteratorHelper(requiredFields.getRight(), allFields.getRight(), dataFile); Option,Schema>> skeletonFileIterator = @@ -283,12 +182,9 @@ private Option,Schema>> makeBootstrapBaseFileIteratorHe if (requiredFields.isEmpty()) { return Option.empty(); } - if (needMerge && shouldMergeWithPosition) { - requiredFields.add(getPositionalMergeField()); - } - Schema requiredSchema = createSchemaFromFields(requiredFields); + Schema requiredSchema = schemaHandler.createSchemaFromFields(requiredFields); return Option.of(Pair.of(readerContext.getFileRecordIterator(file.getHadoopPath(), 0, file.getFileLen(), - createSchemaFromFields(allFields), requiredSchema, hadoopConf, this.needMerge), requiredSchema)); + schemaHandler.createSchemaFromFields(allFields), requiredSchema, hadoopConf), requiredSchema)); } /** @@ -321,11 +217,7 @@ private void scanLogFiles() { HoodieMergedLogRecordReader logRecordReader = HoodieMergedLogRecordReader.newBuilder() .withHoodieReaderContext(readerContext) .withFileSystem(fs) - .withBasePath(readerState.tablePath) .withLogFiles(logFiles) - .withLatestInstantTime(readerState.latestCommitTime) - .withReaderSchema(readerState.logRecordAvroSchema) - .withInternalSchema(internalSchema) .withReadBlocksLazily(getBooleanWithAltKeys(props, HoodieReaderConfig.COMPACTION_LAZY_BLOCK_READ_ENABLE)) .withReverseReader(false) .withBufferSize(getIntWithAltKeys(props, HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE)) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java new file mode 100644 index 000000000000..6a7bf8404b86 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java @@ -0,0 +1,183 @@ +/* + * 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.hudi.common.table.read; + +import org.apache.hudi.common.engine.HoodieReaderContext; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter; + +import org.apache.avro.Schema; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.function.UnaryOperator; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchemaDedupNested; +import static org.apache.hudi.avro.AvroSchemaUtils.findNestedField; + +public class HoodieFileGroupReaderSchemaHandler { + + protected HoodieFileGroupReaderState readerState; + + + + protected final Schema dataSchema; + + // requestedSchema: the schema that the caller requests + protected final Schema requestedSchema; + + // requiredSchema: the requestedSchema with any additional columns required for merging etc + protected final Schema requiredSchema; + + protected final InternalSchema internalSchema; + + + protected final HoodieTableConfig hoodieTableConfig; + + protected final HoodieReaderContext readerContext; + + protected final boolean hasBootstrapBaseFile; + protected final boolean needsBootstrapMerge; + + protected final boolean needsMORMerge; + + public HoodieFileGroupReaderSchemaHandler(HoodieReaderContext readerContext, + Schema dataSchema, + Schema requestedSchema, + Option internalSchemaOpt, + HoodieTableConfig hoodieTableConfig) { + this.readerContext = readerContext; + this.readerState = readerContext.getReaderState(); + this.dataSchema = dataSchema; + this.requestedSchema = requestedSchema; + this.hoodieTableConfig = hoodieTableConfig; + this.hasBootstrapBaseFile = readerState.hasBootstrapBaseFile; + this.needsBootstrapMerge = readerState.needsBootstrapMerge; + this.needsMORMerge = readerState.hasLogFiles; + this.requiredSchema = prepareSchema(); + this.internalSchema = pruneInternalSchema(requiredSchema, internalSchemaOpt); + } + + public Schema getDataSchema() { + return this.dataSchema; + } + + public Schema getRequestedSchema() { + return this.requestedSchema; + } + + public Schema getRequiredSchema() { + return this.requiredSchema; + } + + public InternalSchema getInternalSchema() { + return this.internalSchema; + } + + public Option> getOutputConverter() { + if (!requestedSchema.equals(requiredSchema)) { + return Option.of(readerContext.projectRecord(requiredSchema, requestedSchema)); + } + return Option.empty(); + } + + private static InternalSchema pruneInternalSchema(Schema requiredSchema, Option internalSchemaOption) { + if (!internalSchemaOption.isPresent()) { + return InternalSchema.getEmptyInternalSchema(); + } + InternalSchema notPruned = internalSchemaOption.get(); + if (notPruned == null || notPruned.isEmptySchema()) { + return InternalSchema.getEmptyInternalSchema(); + } + + return AvroInternalSchemaConverter.pruneAvroSchemaToInternalSchema(requiredSchema, notPruned); + } + + private Schema generateRequiredSchema() { + //might need to change this if other queries than mor have mandatory fields + if (!needsMORMerge) { + return requestedSchema; + } + + List addedFields = new ArrayList<>(); + for (String field : readerState.recordMerger.getMandatoryFieldsForMerging(hoodieTableConfig)) { + if (!findNestedField(requestedSchema, field).isPresent()) { + Option foundFieldOpt = findNestedField(dataSchema, field); + if (!foundFieldOpt.isPresent()) { + throw new IllegalArgumentException("Field: " + field + " does not exist in the table schema"); + } + Schema.Field foundField = foundFieldOpt.get(); + addedFields.add(foundField); + } + } + + if (addedFields.isEmpty()) { + return requestedSchema; + } + + return appendFieldsToSchemaDedupNested(requestedSchema, addedFields); + } + + protected Schema prepareSchema() { + Schema preReorderRequiredSchema = generateRequiredSchema(); + Pair, List> requiredFields = getDataAndMetaCols(preReorderRequiredSchema); + readerState.needsBootstrapMerge = readerState.hasBootstrapBaseFile && !requiredFields.getLeft().isEmpty() && !requiredFields.getRight().isEmpty(); + return needsBootstrapMerge + ? createSchemaFromFields(Stream.concat(requiredFields.getLeft().stream(), requiredFields.getRight().stream()).collect(Collectors.toList())) + : preReorderRequiredSchema; + } + + public Pair,List> getBootstrapRequiredFields() { + return getDataAndMetaCols(requiredSchema); + } + + public Pair,List> getBootstrapDataFields() { + return getDataAndMetaCols(dataSchema); + } + + private static Pair, List> getDataAndMetaCols(Schema schema) { + Map> fieldsByMeta = schema.getFields().stream() + //if there are no data fields, then we don't want to think the temp col is a data col + .filter(f -> !Objects.equals(f.name(), HoodiePositionBasedFileGroupRecordBuffer.ROW_INDEX_TEMPORARY_COLUMN_NAME)) + .collect(Collectors.partitioningBy(f -> HoodieRecord.HOODIE_META_COLUMNS_WITH_OPERATION.contains(f.name()))); + return Pair.of(fieldsByMeta.getOrDefault(true, Collections.emptyList()), + fieldsByMeta.getOrDefault(false, Collections.emptyList())); + } + + public Schema createSchemaFromFields(List fields) { + //fields have positions set, so we need to remove them due to avro setFields implementation + for (int i = 0; i < fields.size(); i++) { + Schema.Field curr = fields.get(i); + fields.set(i, new Schema.Field(curr.name(), curr.schema(), curr.doc(), curr.defaultVal())); + } + Schema newSchema = Schema.createRecord(dataSchema.getName(), dataSchema.getDoc(), dataSchema.getNamespace(), dataSchema.isError()); + newSchema.setFields(fields); + return newSchema; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderState.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderState.java index e50713bb40a0..9af06c1578e0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderState.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderState.java @@ -19,18 +19,21 @@ package org.apache.hudi.common.table.read; -import org.apache.hudi.common.config.TypedProperties; - -import org.apache.avro.Schema; +import org.apache.hudi.common.model.HoodieRecordMerger; /** * A class holding the state that is needed by {@code HoodieFileGroupReader}, * e.g., schema, merging strategy, etc. */ -public class HoodieFileGroupReaderState { +public class HoodieFileGroupReaderState { + public HoodieFileGroupReaderSchemaHandler schemaHandler; public String tablePath; public String latestCommitTime; - public Schema baseFileAvroSchema; - public Schema logRecordAvroSchema; - public TypedProperties mergeProps = new TypedProperties(); + public HoodieRecordMerger recordMerger; + public boolean hasLogFiles; + public boolean hasBootstrapBaseFile; + public boolean needsBootstrapMerge; + + + } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieKeyBasedFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieKeyBasedFileGroupRecordBuffer.java index f651578613cf..689e43ce1d83 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieKeyBasedFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieKeyBasedFileGroupRecordBuffer.java @@ -32,7 +32,6 @@ import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.internal.schema.InternalSchema; import org.apache.avro.Schema; @@ -50,10 +49,8 @@ * {@link #hasNext} method is called. */ public class HoodieKeyBasedFileGroupRecordBuffer extends HoodieBaseFileGroupRecordBuffer { + public HoodieKeyBasedFileGroupRecordBuffer(HoodieReaderContext readerContext, - Schema readerSchema, - Schema baseFileSchema, - InternalSchema internalSchema, HoodieTableMetaClient hoodieTableMetaClient, Option partitionNameOverrideOpt, Option partitionPathFieldOpt, @@ -63,7 +60,7 @@ public HoodieKeyBasedFileGroupRecordBuffer(HoodieReaderContext readerContext, String spillableMapBasePath, ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled) { - super(readerContext, readerSchema, baseFileSchema, internalSchema, hoodieTableMetaClient, partitionNameOverrideOpt, partitionPathFieldOpt, + super(readerContext, hoodieTableMetaClient, partitionNameOverrideOpt, partitionPathFieldOpt, recordMerger, payloadProps, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled); } @@ -138,10 +135,10 @@ protected boolean doHasNext() throws IOException { while (baseFileIterator.hasNext()) { T baseRecord = baseFileIterator.next(); - String recordKey = readerContext.getRecordKey(baseRecord, baseFileSchema); + String recordKey = readerContext.getRecordKey(baseRecord, readerSchema); Pair, Map> logRecordInfo = records.remove(recordKey); Map metadata = readerContext.generateMetadataForRecord( - baseRecord, baseFileSchema); + baseRecord, readerSchema); Option resultRecord = logRecordInfo != null ? merge(Option.of(baseRecord), metadata, logRecordInfo.getLeft(), logRecordInfo.getRight()) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java index a9b77a9130a0..8727bfd45e0b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java @@ -32,7 +32,6 @@ import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.internal.schema.InternalSchema; import org.apache.avro.Schema; @@ -60,9 +59,6 @@ public class HoodiePositionBasedFileGroupRecordBuffer extends HoodieBaseFileG private long nextRecordPosition = 0L; public HoodiePositionBasedFileGroupRecordBuffer(HoodieReaderContext readerContext, - Schema readerSchema, - Schema baseFileSchema, - InternalSchema internalSchema, HoodieTableMetaClient hoodieTableMetaClient, Option partitionNameOverrideOpt, Option partitionPathFieldOpt, @@ -72,7 +68,7 @@ public HoodiePositionBasedFileGroupRecordBuffer(HoodieReaderContext readerCon String spillableMapBasePath, ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled) { - super(readerContext, readerSchema, baseFileSchema, internalSchema, hoodieTableMetaClient, partitionNameOverrideOpt, partitionPathFieldOpt, + super(readerContext, hoodieTableMetaClient, partitionNameOverrideOpt, partitionPathFieldOpt, recordMerger, payloadProps, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled); } @@ -200,7 +196,7 @@ protected boolean doHasNext() throws IOException { Pair, Map> logRecordInfo = records.remove(nextRecordPosition++); Map metadata = readerContext.generateMetadataForRecord( - baseRecord, baseFileSchema); + baseRecord, readerSchema); Option resultRecord = logRecordInfo != null ? merge(Option.of(baseRecord), metadata, logRecordInfo.getLeft(), logRecordInfo.getRight()) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedSchemaHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedSchemaHandler.java new file mode 100644 index 000000000000..a4940ca45653 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedSchemaHandler.java @@ -0,0 +1,68 @@ +/* + * 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.hudi.common.table.read; + +import org.apache.hudi.common.engine.HoodieReaderContext; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.internal.schema.InternalSchema; + +import org.apache.avro.Schema; + +import java.util.Collections; +import java.util.List; + +import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchemaDedupNested; + +public class HoodiePositionBasedSchemaHandler extends HoodieFileGroupReaderSchemaHandler { + public HoodiePositionBasedSchemaHandler(HoodieReaderContext readerContext, + Schema dataSchema, + Schema requestedSchema, + Option internalSchemaOpt, + HoodieTableConfig hoodieTableConfig) { + super(readerContext, dataSchema, requestedSchema, internalSchemaOpt, hoodieTableConfig); + + } + + @Override + protected Schema prepareSchema() { + Schema preMergeSchema = super.prepareSchema(); + return readerState.hasLogFiles + ? addPositionalMergeCol(preMergeSchema) + : preMergeSchema; + } + + private Schema addPositionalMergeCol(Schema input) { + return appendFieldsToSchemaDedupNested(input, Collections.singletonList(getPositionalMergeField())); + } + + private Schema.Field getPositionalMergeField() { + return new Schema.Field(HoodiePositionBasedFileGroupRecordBuffer.ROW_INDEX_TEMPORARY_COLUMN_NAME, + Schema.create(Schema.Type.LONG), "", 0L); + } + + @Override + public Schema createSchemaFromFields(List fields) { + if (readerState.hasLogFiles) { + fields.add(getPositionalMergeField()); + } + return super.createSchemaFromFields(fields); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieUnmergedFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieUnmergedFileGroupRecordBuffer.java index e7e47ba0c98c..325f9ca4f93e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieUnmergedFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieUnmergedFileGroupRecordBuffer.java @@ -33,7 +33,6 @@ import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.internal.schema.InternalSchema; import org.apache.avro.Schema; @@ -50,9 +49,6 @@ public class HoodieUnmergedFileGroupRecordBuffer extends HoodieBaseFileGroupR public HoodieUnmergedFileGroupRecordBuffer( HoodieReaderContext readerContext, - Schema readerSchema, - Schema baseFileSchema, - InternalSchema internalSchema, HoodieTableMetaClient hoodieTableMetaClient, Option partitionNameOverrideOpt, Option partitionPathFieldOpt, @@ -62,8 +58,8 @@ public HoodieUnmergedFileGroupRecordBuffer( String spillableMapBasePath, ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled) { - super(readerContext, readerSchema, baseFileSchema, internalSchema, hoodieTableMetaClient, partitionNameOverrideOpt, partitionPathFieldOpt, - recordMerger, payloadProps, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled); + super(readerContext, hoodieTableMetaClient, partitionNameOverrideOpt, partitionPathFieldOpt, recordMerger, + payloadProps, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled); } @Override diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieTestReaderContext.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieTestReaderContext.java index e2f9bd841489..91138e6ee237 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieTestReaderContext.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieTestReaderContext.java @@ -76,8 +76,7 @@ public ClosableIterator getFileRecordIterator( long length, Schema dataSchema, Schema requiredSchema, - Configuration conf, - boolean isMerge + Configuration conf ) throws IOException { HoodieAvroParquetReader reader = new HoodieAvroParquetReader(conf, filePath); return reader.getIndexedRecordIterator(dataSchema, requiredSchema); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodiePositionBasedFileGroupRecordBuffer.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodiePositionBasedFileGroupRecordBuffer.java index 68a874d07eda..03afd0ceaa85 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodiePositionBasedFileGroupRecordBuffer.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodiePositionBasedFileGroupRecordBuffer.java @@ -38,7 +38,6 @@ import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieValidationException; -import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.avro.Schema; @@ -100,9 +99,6 @@ public void prepareBuffer(boolean useCustomMerger) throws Exception { buffer = new HoodiePositionBasedFileGroupRecordBuffer<>( getHoodieReaderContext(getBasePath(), avroSchema, getHadoopConf()), - avroSchema, - avroSchema, - InternalSchema.getEmptyInternalSchema(), metaClient, partitionNameOpt, partitionFields, diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala index 139a38edbf42..6a4d0b01b358 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala @@ -21,6 +21,7 @@ package org.apache.hudi.common.table.read import org.apache.avro.Schema import org.apache.hadoop.conf.Configuration +import org.apache.hudi.common.util.Option import org.apache.hudi.common.config.HoodieReaderConfig.FILE_GROUP_READER_ENABLED import org.apache.hudi.common.engine.HoodieReaderContext import org.apache.hudi.common.fs.FSUtils @@ -89,7 +90,10 @@ class TestHoodieFileGroupReaderOnSpark extends TestHoodieFileGroupReaderBase[Int checkState(keyFields.length == 1) keyFields.head } - new SparkFileFormatInternalRowReaderContext(reader, recordKeyField, Seq.empty, false) + val ctx = new SparkFileFormatInternalRowReaderContext(reader, recordKeyField, Seq.empty, false) + val readerState = ctx.getReaderState + readerState.schemaHandler = new HoodiePositionBasedSchemaHandler[InternalRow](ctx, avroSchema, avroSchema, Option.empty(), metaClient.getTableConfig) + ctx } override def commitToTable(recordList: util.List[String], operation: String, options: util.Map[String, String]): Unit = { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala index 617cfbee7567..50561d3731d5 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala @@ -77,7 +77,7 @@ class TestSpark35RecordPositionMetadataColumn extends SparkClientFunctionalTestH val props = Map("spark.sql.parquet.enableVectorizedReader" -> "false") _spark.conf.set("spark.sql.parquet.enableVectorizedReader", "false") val reader = sparkAdapter.createHoodieParquetFileReader(vectorized = false, _spark.sessionState.conf, props, hadoopConf) - val requiredSchema = SparkFileFormatInternalRowReaderContext.getAppliedRequiredSchema(dataSchema, shouldUseRecordPosition = true) + val requiredSchema = SparkFileFormatInternalRowReaderContext.getAppliedRequiredSchema(dataSchema) // Confirm if the schema is as expected. assertEquals(4, requiredSchema.fields.length) @@ -91,9 +91,12 @@ class TestSpark35RecordPositionMetadataColumn extends SparkClientFunctionalTestH val allBaseFiles = HoodieTestTable.of(metaClient).listAllBaseFiles assertTrue(allBaseFiles.nonEmpty) val readerContext = new SparkFileFormatInternalRowReaderContext(reader, HoodieRecord.RECORD_KEY_METADATA_FIELD, Seq.empty, true) + val readerState = readerContext.getReaderState + readerState.hasLogFiles = true + readerState.needsBootstrapMerge //dataschema param is set to null because it is not used val fileRecordIterator = readerContext.getFileRecordIterator(allBaseFiles.head.getPath, 0, allBaseFiles.head.getLen, null, - sparkAdapter.getAvroSchemaConverters.toAvroType(dataSchema, nullable = true, "record"), hadoopConf, true) + sparkAdapter.getAvroSchemaConverters.toAvroType(dataSchema, nullable = true, "record"), hadoopConf) // Make sure we can read all the positions out from base file. // Here we don't add filters since enabling filter push-down From 70cef40f06ad614ecb7e00530201e06190d8f268 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 9 Apr 2024 22:03:01 -0400 Subject: [PATCH 42/86] fix mor --- ...parkFileFormatInternalRowReaderContext.scala | 2 +- .../table/read/HoodieFileGroupReader.java | 13 +++++++------ .../HoodieFileGroupReaderSchemaHandler.java | 4 ++-- .../table/read/HoodieFileGroupReaderState.java | 17 +++++++---------- 4 files changed, 17 insertions(+), 19 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala index b8a1dd29ffdf..8b7e3d1684f8 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala @@ -94,7 +94,7 @@ class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkHoodieParq } private def getSchemaAndFiltersForRead(structType: StructType): (StructType, Seq[Filter]) = { - (readerState.hasLogFiles, readerState.needsBootstrapMerge, shouldUseRecordPosition) match { + (readerState.hasLogFiles.booleanValue(), readerState.needsBootstrapMerge.booleanValue(), shouldUseRecordPosition) match { case (false, false, _) => (structType, filters) case (false, true, true) if shouldUseRecordPosition => diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java index 7e059bbb79e1..aa806e1cf0ef 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java @@ -76,12 +76,11 @@ public final class HoodieFileGroupReader implements Closeable { private final long length; // Core structure to store and process records. private final HoodieFileGroupRecordBuffer recordBuffer; - private final HoodieFileGroupReaderState readerState; + private final HoodieFileGroupReaderState readerState; private ClosableIterator baseFileIterator; - private HoodieRecordMerger recordMerger; + private final HoodieRecordMerger recordMerger; private final Option> outputConverter; - private final HoodieFileGroupReaderSchemaHandler schemaHandler; - private final boolean shouldMergeWithPosition; + private final HoodieFileGroupReaderSchemaHandler schemaHandler; public HoodieFileGroupReader(HoodieReaderContext readerContext, Configuration hadoopConf, @@ -110,9 +109,11 @@ public HoodieFileGroupReader(HoodieReaderContext readerContext, this.start = start; this.length = length; this.recordMerger = readerContext.getRecordMerger(tableConfig.getRecordMergerStrategy()); + this.readerState.recordMerger = this.recordMerger; this.readerState.tablePath = tablePath; this.readerState.latestCommitTime = latestCommitTime; - this.shouldMergeWithPosition = shouldUseRecordPosition && readerContext.shouldUseRecordPositionMerging(); + boolean shouldMergeWithPosition = shouldUseRecordPosition && readerContext.shouldUseRecordPositionMerging(); + readerState.hasLogFiles = !this.logFiles.isEmpty(); readerState.hasBootstrapBaseFile = hoodieBaseFileOption.isPresent() && hoodieBaseFileOption.get().getBootstrapBaseFile().isPresent(); readerState.schemaHandler = shouldMergeWithPosition ? new HoodiePositionBasedSchemaHandler<>(readerContext, dataSchema, requestedSchema, internalSchemaOpt, tableConfig) @@ -121,7 +122,7 @@ public HoodieFileGroupReader(HoodieReaderContext readerContext, this.outputConverter = schemaHandler.getOutputConverter(); this.recordBuffer = this.logFiles.isEmpty() ? null - : this.shouldMergeWithPosition + : shouldMergeWithPosition ? new HoodiePositionBasedFileGroupRecordBuffer<>(readerContext, hoodieTableMetaClient, Option.empty(), Option.empty(), recordMerger, props, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled) : new HoodieKeyBasedFileGroupRecordBuffer<>(readerContext, hoodieTableMetaClient, Option.empty(), diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java index 6a7bf8404b86..92f65bc6d207 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java @@ -43,7 +43,7 @@ public class HoodieFileGroupReaderSchemaHandler { - protected HoodieFileGroupReaderState readerState; + protected HoodieFileGroupReaderState readerState; @@ -78,9 +78,9 @@ public HoodieFileGroupReaderSchemaHandler(HoodieReaderContext readerContext, this.requestedSchema = requestedSchema; this.hoodieTableConfig = hoodieTableConfig; this.hasBootstrapBaseFile = readerState.hasBootstrapBaseFile; - this.needsBootstrapMerge = readerState.needsBootstrapMerge; this.needsMORMerge = readerState.hasLogFiles; this.requiredSchema = prepareSchema(); + this.needsBootstrapMerge = readerState.needsBootstrapMerge; this.internalSchema = pruneInternalSchema(requiredSchema, internalSchemaOpt); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderState.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderState.java index 9af06c1578e0..5fbd9d5e6d7f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderState.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderState.java @@ -26,14 +26,11 @@ * e.g., schema, merging strategy, etc. */ public class HoodieFileGroupReaderState { - public HoodieFileGroupReaderSchemaHandler schemaHandler; - public String tablePath; - public String latestCommitTime; - public HoodieRecordMerger recordMerger; - public boolean hasLogFiles; - public boolean hasBootstrapBaseFile; - public boolean needsBootstrapMerge; - - - + public HoodieFileGroupReaderSchemaHandler schemaHandler = null; + public String tablePath = null; + public String latestCommitTime = null; + public HoodieRecordMerger recordMerger = null; + public Boolean hasLogFiles = null; + public Boolean hasBootstrapBaseFile = null; + public Boolean needsBootstrapMerge = null; } From 56125afb6253d87d639e82ca686a579bb9a39a63 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 9 Apr 2024 22:51:47 -0400 Subject: [PATCH 43/86] fix some tests --- .../read/HoodieBaseFileGroupRecordBuffer.java | 2 +- ...stHoodiePositionBasedFileGroupRecordBuffer.java | 14 ++++++++++++-- .../read/TestHoodieFileGroupReaderOnSpark.scala | 5 +---- 3 files changed, 14 insertions(+), 7 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java index 3029e1942d48..3efa0ccdf6d4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java @@ -88,7 +88,7 @@ public HoodieBaseFileGroupRecordBuffer(HoodieReaderContext readerContext, boolean isBitCaskDiskMapCompressionEnabled) { this.readerContext = readerContext; this.readerState = readerContext.getReaderState(); - this.readerSchema = readerState.schemaHandler.requiredSchema; + this.readerSchema = readerState.schemaHandler.getRequiredSchema(); this.partitionNameOverrideOpt = partitionNameOverrideOpt; this.partitionPathFieldOpt = partitionPathFieldOpt; this.recordMerger = recordMerger; diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodiePositionBasedFileGroupRecordBuffer.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodiePositionBasedFileGroupRecordBuffer.java index 03afd0ceaa85..53ddb4fc3a0f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodiePositionBasedFileGroupRecordBuffer.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodiePositionBasedFileGroupRecordBuffer.java @@ -21,6 +21,7 @@ import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.engine.HoodieReaderContext; import org.apache.hudi.common.model.DeleteRecord; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordMerger; @@ -29,7 +30,9 @@ import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock; +import org.apache.hudi.common.table.read.HoodieFileGroupReaderState; import org.apache.hudi.common.table.read.HoodiePositionBasedFileGroupRecordBuffer; +import org.apache.hudi.common.table.read.HoodiePositionBasedSchemaHandler; import org.apache.hudi.common.table.read.TestHoodieFileGroupReaderOnSpark; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.SchemaTestUtil; @@ -97,12 +100,19 @@ public void prepareBuffer(boolean useCustomMerger) throws Exception { Option partitionNameOpt = StringUtils.isNullOrEmpty(partitionPaths[0]) ? Option.empty() : Option.of(partitionPaths[0]); + HoodieReaderContext ctx = getHoodieReaderContext(getBasePath(), avroSchema, getHadoopConf()); + HoodieFileGroupReaderState state = ctx.getReaderState(); + state.hasBootstrapBaseFile = false; + state.hasLogFiles = true; + state.needsBootstrapMerge = false; + state.recordMerger = useCustomMerger ? new CustomMerger() : new HoodieSparkRecordMerger(); + state.schemaHandler = new HoodiePositionBasedSchemaHandler(ctx, avroSchema, avroSchema, Option.empty(), metaClient.getTableConfig()); buffer = new HoodiePositionBasedFileGroupRecordBuffer<>( - getHoodieReaderContext(getBasePath(), avroSchema, getHadoopConf()), + ctx, metaClient, partitionNameOpt, partitionFields, - useCustomMerger ? new CustomMerger() : new HoodieSparkRecordMerger(), + state.recordMerger, new TypedProperties(), 1024 * 1024 * 1000, metaClient.getTempFolderPath(), diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala index 6a4d0b01b358..2bba33c5192f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala @@ -90,10 +90,7 @@ class TestHoodieFileGroupReaderOnSpark extends TestHoodieFileGroupReaderBase[Int checkState(keyFields.length == 1) keyFields.head } - val ctx = new SparkFileFormatInternalRowReaderContext(reader, recordKeyField, Seq.empty, false) - val readerState = ctx.getReaderState - readerState.schemaHandler = new HoodiePositionBasedSchemaHandler[InternalRow](ctx, avroSchema, avroSchema, Option.empty(), metaClient.getTableConfig) - ctx + new SparkFileFormatInternalRowReaderContext(reader, recordKeyField, Seq.empty, false) } override def commitToTable(recordList: util.List[String], operation: String, options: util.Map[String, String]): Unit = { From 31eb84b8fc7e0d8066633ff8f6bc92b14b8660e3 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 9 Apr 2024 23:09:30 -0400 Subject: [PATCH 44/86] add validations --- .../hudi/common/table/read/HoodieFileGroupReader.java | 4 +++- .../table/read/HoodieFileGroupReaderSchemaHandler.java | 5 +++++ .../java/org/apache/hudi/common/util/ValidationUtils.java | 6 ++++++ 3 files changed, 14 insertions(+), 1 deletion(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java index aa806e1cf0ef..6625f7859cbe 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java @@ -32,6 +32,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieMergedLogRecordReader; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.CachingIterator; import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.common.util.collection.EmptyIterator; @@ -212,6 +213,7 @@ public T next() { } private void scanLogFiles() { + ValidationUtils.checkNotNull(readerState.tablePath); String path = readerState.tablePath; FileSystem fs = readerContext.getFs(path, hadoopConf); @@ -223,7 +225,7 @@ private void scanLogFiles() { .withReverseReader(false) .withBufferSize(getIntWithAltKeys(props, HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE)) .withPartition(getRelativePartitionPath( - new Path(readerState.tablePath), logFiles.get(0).getPath().getParent())) + new Path(path), logFiles.get(0).getPath().getParent())) .withRecordMerger(recordMerger) .withRecordBuffer(recordBuffer) .build(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java index 92f65bc6d207..a715bc8f0c7d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter; @@ -74,12 +75,16 @@ public HoodieFileGroupReaderSchemaHandler(HoodieReaderContext readerContext, HoodieTableConfig hoodieTableConfig) { this.readerContext = readerContext; this.readerState = readerContext.getReaderState(); + ValidationUtils.checkNotNull(readerState.hasBootstrapBaseFile); + ValidationUtils.checkNotNull(readerState.hasLogFiles); + ValidationUtils.checkNotNull(readerState.recordMerger); this.dataSchema = dataSchema; this.requestedSchema = requestedSchema; this.hoodieTableConfig = hoodieTableConfig; this.hasBootstrapBaseFile = readerState.hasBootstrapBaseFile; this.needsMORMerge = readerState.hasLogFiles; this.requiredSchema = prepareSchema(); + ValidationUtils.checkNotNull(readerState.needsBootstrapMerge); this.needsBootstrapMerge = readerState.needsBootstrapMerge; this.internalSchema = pruneInternalSchema(requiredSchema, internalSchemaOpt); } diff --git a/hudi-io/src/main/java/org/apache/hudi/common/util/ValidationUtils.java b/hudi-io/src/main/java/org/apache/hudi/common/util/ValidationUtils.java index 3350c9a86081..b12ec9c46757 100644 --- a/hudi-io/src/main/java/org/apache/hudi/common/util/ValidationUtils.java +++ b/hudi-io/src/main/java/org/apache/hudi/common/util/ValidationUtils.java @@ -76,4 +76,10 @@ public static void checkState(final boolean expression, String errorMessage) { throw new IllegalStateException(errorMessage); } } + + public static void checkNotNull(Object o) { + if (o == null) { + throw new IllegalStateException(); + } + } } From 9c723d060870d975efca67f769816a98bb662c49 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Wed, 10 Apr 2024 09:57:10 -0400 Subject: [PATCH 45/86] fixed broken test --- .../table/read/TestSpark35RecordPositionMetadataColumn.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala index 50561d3731d5..416525d19e10 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala @@ -93,7 +93,8 @@ class TestSpark35RecordPositionMetadataColumn extends SparkClientFunctionalTestH val readerContext = new SparkFileFormatInternalRowReaderContext(reader, HoodieRecord.RECORD_KEY_METADATA_FIELD, Seq.empty, true) val readerState = readerContext.getReaderState readerState.hasLogFiles = true - readerState.needsBootstrapMerge + readerState.needsBootstrapMerge = false + readerState.hasBootstrapBaseFile = false //dataschema param is set to null because it is not used val fileRecordIterator = readerContext.getFileRecordIterator(allBaseFiles.head.getPath, 0, allBaseFiles.head.getLen, null, sparkAdapter.getAvroSchemaConverters.toAvroType(dataSchema, nullable = true, "record"), hadoopConf) From f475aa96f3228ca16cb15bff654fe7e01086fe92 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Wed, 10 Apr 2024 12:17:20 -0400 Subject: [PATCH 46/86] fix set and use wrong var --- .../common/table/read/HoodieFileGroupReaderSchemaHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java index a715bc8f0c7d..aad23f500f2d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java @@ -153,7 +153,7 @@ protected Schema prepareSchema() { Schema preReorderRequiredSchema = generateRequiredSchema(); Pair, List> requiredFields = getDataAndMetaCols(preReorderRequiredSchema); readerState.needsBootstrapMerge = readerState.hasBootstrapBaseFile && !requiredFields.getLeft().isEmpty() && !requiredFields.getRight().isEmpty(); - return needsBootstrapMerge + return readerState.needsBootstrapMerge ? createSchemaFromFields(Stream.concat(requiredFields.getLeft().stream(), requiredFields.getRight().stream()).collect(Collectors.toList())) : preReorderRequiredSchema; } From 1e4657afd949bc1610adb947abd26a35bd89d884 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Wed, 10 Apr 2024 12:29:31 -0400 Subject: [PATCH 47/86] only update and use reader state in the constructor --- .../HoodieFileGroupReaderSchemaHandler.java | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java index aad23f500f2d..e0273b26aacc 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java @@ -21,6 +21,7 @@ import org.apache.hudi.common.engine.HoodieReaderContext; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; @@ -63,8 +64,10 @@ public class HoodieFileGroupReaderSchemaHandler { protected final HoodieReaderContext readerContext; + protected final HoodieRecordMerger recordMerger; + protected final boolean hasBootstrapBaseFile; - protected final boolean needsBootstrapMerge; + protected boolean needsBootstrapMerge; protected final boolean needsMORMerge; @@ -76,17 +79,17 @@ public HoodieFileGroupReaderSchemaHandler(HoodieReaderContext readerContext, this.readerContext = readerContext; this.readerState = readerContext.getReaderState(); ValidationUtils.checkNotNull(readerState.hasBootstrapBaseFile); + this.hasBootstrapBaseFile = readerState.hasBootstrapBaseFile; ValidationUtils.checkNotNull(readerState.hasLogFiles); + this.needsMORMerge = readerState.hasLogFiles; ValidationUtils.checkNotNull(readerState.recordMerger); + this.recordMerger = readerState.recordMerger; this.dataSchema = dataSchema; this.requestedSchema = requestedSchema; this.hoodieTableConfig = hoodieTableConfig; - this.hasBootstrapBaseFile = readerState.hasBootstrapBaseFile; - this.needsMORMerge = readerState.hasLogFiles; this.requiredSchema = prepareSchema(); - ValidationUtils.checkNotNull(readerState.needsBootstrapMerge); - this.needsBootstrapMerge = readerState.needsBootstrapMerge; this.internalSchema = pruneInternalSchema(requiredSchema, internalSchemaOpt); + readerState.needsBootstrapMerge = this.needsBootstrapMerge; } public Schema getDataSchema() { @@ -131,7 +134,7 @@ private Schema generateRequiredSchema() { } List addedFields = new ArrayList<>(); - for (String field : readerState.recordMerger.getMandatoryFieldsForMerging(hoodieTableConfig)) { + for (String field : recordMerger.getMandatoryFieldsForMerging(hoodieTableConfig)) { if (!findNestedField(requestedSchema, field).isPresent()) { Option foundFieldOpt = findNestedField(dataSchema, field); if (!foundFieldOpt.isPresent()) { @@ -152,8 +155,8 @@ private Schema generateRequiredSchema() { protected Schema prepareSchema() { Schema preReorderRequiredSchema = generateRequiredSchema(); Pair, List> requiredFields = getDataAndMetaCols(preReorderRequiredSchema); - readerState.needsBootstrapMerge = readerState.hasBootstrapBaseFile && !requiredFields.getLeft().isEmpty() && !requiredFields.getRight().isEmpty(); - return readerState.needsBootstrapMerge + this.needsBootstrapMerge = hasBootstrapBaseFile && !requiredFields.getLeft().isEmpty() && !requiredFields.getRight().isEmpty(); + return needsBootstrapMerge ? createSchemaFromFields(Stream.concat(requiredFields.getLeft().stream(), requiredFields.getRight().stream()).collect(Collectors.toList())) : preReorderRequiredSchema; } From 15acc2e870fb880a56de561be9abb72f28fa588d Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Wed, 10 Apr 2024 18:00:01 -0400 Subject: [PATCH 48/86] use data block schema to read untransformed record --- .../common/table/read/HoodieBaseFileGroupRecordBuffer.java | 4 ++-- .../table/read/HoodiePositionBasedFileGroupRecordBuffer.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java index 3efa0ccdf6d4..a3aa8559fc56 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java @@ -335,8 +335,8 @@ protected Option merge(Option older, Map olderInfoMap, * 1. A set of pre-specified keys exists. * 2. The key of the record is not contained in the set. */ - protected boolean shouldSkip(T record, String keyFieldName, boolean isFullKey, Set keys) { - String recordKey = readerContext.getValue(record, readerSchema, keyFieldName).toString(); + protected boolean shouldSkip(T record, String keyFieldName, boolean isFullKey, Set keys, Schema dataBlockSchema) { + String recordKey = readerContext.getValue(record, dataBlockSchema, keyFieldName).toString(); // Can not extract the record key, throw. if (recordKey == null || recordKey.isEmpty()) { throw new HoodieKeyException("Can not extract the key for a record"); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java index 8727bfd45e0b..623982b78d8b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java @@ -118,7 +118,7 @@ public void processDataBlock(HoodieDataBlock dataBlock, Option keySpecO T nextRecord = recordIterator.next(); // Skip a record if it is not contained in the specified keys. - if (shouldSkip(nextRecord, dataBlock.getKeyFieldName(), isFullKey, keys)) { + if (shouldSkip(nextRecord, dataBlock.getKeyFieldName(), isFullKey, keys, dataBlock.getSchema())) { recordIndex++; continue; } From 8205971561917ad2ec5a41a03edcb0c8e8da93c4 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Wed, 10 Apr 2024 21:33:47 -0400 Subject: [PATCH 49/86] add test --- .../TestSparkParquetReaderFormat.scala | 67 +++++++++++++++++++ .../TestSparkHoodieParquetReader.java | 47 +++++++++++++ .../apache/hudi/util/JavaConversions.scala | 17 ++++- 3 files changed, 130 insertions(+), 1 deletion(-) create mode 100644 hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TestSparkParquetReaderFormat.scala create mode 100644 hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkHoodieParquetReader.java diff --git a/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TestSparkParquetReaderFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TestSparkParquetReaderFormat.scala new file mode 100644 index 000000000000..00556c7f0de1 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TestSparkParquetReaderFormat.scala @@ -0,0 +1,67 @@ +/* + * 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. + */ + +/* + * 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.hudi.SparkAdapterSupport +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableConfiguration + +class TestSparkParquetReaderFormat extends ParquetFileFormat with SparkAdapterSupport { + + override def buildReaderWithPartitionValues(sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { + val reader = sparkAdapter.createHoodieParquetFileReader(false, sparkSession.sqlContext.conf, options, hadoopConf) + val broadcastedHadoopConf = + sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) + (file: PartitionedFile) => { + reader.read(file, requiredSchema, partitionSchema, filters, broadcastedHadoopConf.value.value) + } + + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkHoodieParquetReader.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkHoodieParquetReader.java new file mode 100644 index 000000000000..a6625096d641 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkHoodieParquetReader.java @@ -0,0 +1,47 @@ +/* + * 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.hudi.functional; + +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.util.JavaConversions; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class TestSparkHoodieParquetReader extends TestBootstrapReadBase { + + @Test + public void testReader() { + sqlContext.setConf("spark.sql.parquet.enableVectorizedReader", "false"); + dataGen = new HoodieTestDataGenerator(dashPartitionPaths); + int n = 10; + Dataset inserts = makeInsertDf("000", n); + inserts.write().format("parquet").save(bootstrapBasePath); + Dataset individualReader = JavaConversions.createTestDataFrame(sparkSession, bootstrapBasePath); + Dataset sparkParquetReader = sparkSession.read().format("parquet").load(bootstrapBasePath); + assertEquals(sparkParquetReader.count(), n); + assertEquals(individualReader.count(), n); + assertEquals(sparkParquetReader.except(individualReader).count(), 0); + assertEquals(individualReader.except(sparkParquetReader).count(), 0); + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/util/JavaConversions.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/util/JavaConversions.scala index c9abe0909758..f58f0f2bbf07 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/util/JavaConversions.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/util/JavaConversions.scala @@ -18,9 +18,16 @@ package org.apache.hudi.util +import org.apache.hadoop.conf.Configuration +import org.apache.hudi.SparkAdapterSupport +import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.execution.datasources.parquet.{SparkHoodieParquetReader, TestSparkParquetReaderFormat} +import org.apache.spark.sql.internal.SQLConf + import java.util.function.Predicate -object JavaConversions { +object JavaConversions extends SparkAdapterSupport { def getPredicate[T](function1: (T) => Boolean): Predicate[T] = { new Predicate[T] { override def test(t: T): Boolean = function1.apply(t) @@ -34,4 +41,12 @@ object JavaConversions { } } } + + def createTestDataFrame(sparkSession: SparkSession, paths: String): DataFrame = { + sparkSession.sqlContext.baseRelationToDataFrame(DataSource.apply( + sparkSession = sparkSession, + className = "org.apache.spark.sql.execution.datasources.parquet.TestSparkParquetReaderFormat", + paths = paths.split(",").toSeq + ).resolveRelation()) + } } From 815b6fd6af5676590079cf6f9e23b7a2fdb4ccd8 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 11 Apr 2024 11:05:58 -0400 Subject: [PATCH 50/86] allow vectorized read and comment better --- .../TestSparkParquetReaderFormat.scala | 24 ++++++++++++------- .../TestSparkHoodieParquetReader.java | 3 ++- .../apache/hudi/util/JavaConversions.scala | 11 ++++++--- 3 files changed, 26 insertions(+), 12 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TestSparkParquetReaderFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TestSparkParquetReaderFormat.scala index 00556c7f0de1..e6e1b8c15a0d 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TestSparkParquetReaderFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TestSparkParquetReaderFormat.scala @@ -47,21 +47,29 @@ import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration +/** + * Class used to test [[SparkHoodieParquetReader]] + * This class should have the same functionality as [[ParquetFileFormat]] + */ class TestSparkParquetReaderFormat extends ParquetFileFormat with SparkAdapterSupport { override def buildReaderWithPartitionValues(sparkSession: SparkSession, - dataSchema: StructType, - partitionSchema: StructType, - requiredSchema: StructType, - filters: Seq[Filter], - options: Map[String, String], - hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { - val reader = sparkAdapter.createHoodieParquetFileReader(false, sparkSession.sqlContext.conf, options, hadoopConf) + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = { + //reader must be created outsize of the lambda. This happens on the driver + val reader = sparkAdapter.createHoodieParquetFileReader(supportBatch(sparkSession, + StructType(partitionSchema.fields ++ requiredSchema.fields)), + sparkSession.sqlContext.conf, options, hadoopConf) val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) + (file: PartitionedFile) => { + //code inside the lambda will run on the executor reader.read(file, requiredSchema, partitionSchema, filters, broadcastedHadoopConf.value.value) } - } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkHoodieParquetReader.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkHoodieParquetReader.java index a6625096d641..5130b504ffb0 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkHoodieParquetReader.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkHoodieParquetReader.java @@ -24,15 +24,16 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; +import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertEquals; +@Tag("functional") public class TestSparkHoodieParquetReader extends TestBootstrapReadBase { @Test public void testReader() { - sqlContext.setConf("spark.sql.parquet.enableVectorizedReader", "false"); dataGen = new HoodieTestDataGenerator(dashPartitionPaths); int n = 10; Dataset inserts = makeInsertDf("000", n); diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/util/JavaConversions.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/util/JavaConversions.scala index f58f0f2bbf07..52333e726282 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/util/JavaConversions.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/util/JavaConversions.scala @@ -18,12 +18,10 @@ package org.apache.hudi.util -import org.apache.hadoop.conf.Configuration +import org.apache.spark.sql.execution.datasources.parquet.TestSparkParquetReaderFormat import org.apache.hudi.SparkAdapterSupport import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.{DataFrame, SparkSession} -import org.apache.spark.sql.execution.datasources.parquet.{SparkHoodieParquetReader, TestSparkParquetReaderFormat} -import org.apache.spark.sql.internal.SQLConf import java.util.function.Predicate @@ -42,6 +40,13 @@ object JavaConversions extends SparkAdapterSupport { } } + /** + * Read parquet files using [[TestSparkParquetReaderFormat]] + * + * @param sparkSession the spark session + * @param paths comma seperated list of parquet files or directories containing parquet files + * @return dataframe containing the data from the input paths + */ def createTestDataFrame(sparkSession: SparkSession, paths: String): DataFrame = { sparkSession.sqlContext.baseRelationToDataFrame(DataSource.apply( sparkSession = sparkSession, From 120226ac7bc6eeb735307745dfa47782a311470b Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 11 Apr 2024 17:43:03 -0400 Subject: [PATCH 51/86] address review comments 3.5 --- ...tReader.scala => SparkParquetReader.scala} | 4 +- .../apache/spark/sql/hudi/SparkAdapter.scala | 10 ++--- ...ase.scala => SparkParquetReaderBase.scala} | 38 +++++++++---------- .../TestSparkParquetReaderFormat.scala | 4 +- ...eader.java => TestSparkParquetReader.java} | 14 +++---- .../spark/sql/adapter/Spark3_5Adapter.scala | 12 +++--- ...eader.scala => Spark35ParquetReader.scala} | 38 +++++++++---------- 7 files changed, 59 insertions(+), 61 deletions(-) rename hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/{SparkHoodieParquetReader.scala => SparkParquetReader.scala} (97%) rename hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/{SparkHoodieParquetReaderBase.scala => SparkParquetReaderBase.scala} (74%) rename hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/{TestSparkHoodieParquetReader.java => TestSparkParquetReader.java} (74%) rename hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/{Spark35HoodieParquetReader.scala => Spark35ParquetReader.scala} (91%) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReader.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReader.scala similarity index 97% rename from hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReader.scala rename to hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReader.scala index 7eff545136fd..920e4cb0e0b1 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReader.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReader.scala @@ -25,8 +25,7 @@ import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType -trait SparkHoodieParquetReader extends Serializable { - +trait SparkParquetReader extends Serializable { /** * Read an individual parquet file * @@ -42,5 +41,4 @@ trait SparkHoodieParquetReader extends Serializable { partitionSchema: StructType, filters: Seq[Filter], sharedConf: Configuration): Iterator[InternalRow] - } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala index ee00fb9d39ee..91fe6dabc2e3 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} import org.apache.spark.sql.catalyst.util.DateFormatter import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, SparkHoodieParquetReader} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, SparkParquetReader} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.parser.HoodieExtendedParserInterface import org.apache.spark.sql.sources.{BaseRelation, Filter} @@ -226,8 +226,8 @@ trait SparkAdapter extends Serializable { * @param hadoopConf some configs will be set for the hadoopConf * @return parquet file reader */ - def createHoodieParquetFileReader(vectorized: Boolean, - sqlConf: SQLConf, - options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReader + def createParquetFileReader(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkParquetReader } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReaderBase.scala similarity index 74% rename from hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReaderBase.scala index c74810db2b6b..f7bfcd9e900d 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkHoodieParquetReaderBase.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReaderBase.scala @@ -26,19 +26,19 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType -abstract class SparkHoodieParquetReaderBase(enableVectorizedReader: Boolean, - enableParquetFilterPushDown: Boolean, - pushDownDate: Boolean, - pushDownTimestamp: Boolean, - pushDownDecimal: Boolean, - pushDownInFilterThreshold: Int, - isCaseSensitive: Boolean, - timestampConversion: Boolean, - enableOffHeapColumnVector: Boolean, - capacity: Int, - returningBatch: Boolean, - enableRecordFilter: Boolean, - timeZoneId: Option[String]) extends SparkHoodieParquetReader { +abstract class SparkParquetReaderBase(enableVectorizedReader: Boolean, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) extends SparkParquetReader { /** * Read an individual parquet file @@ -51,10 +51,10 @@ abstract class SparkHoodieParquetReaderBase(enableVectorizedReader: Boolean, * @return iterator of rows read from the file output type says [[InternalRow]] but could be [[ColumnarBatch]] */ final def read(file: PartitionedFile, - requiredSchema: StructType, - partitionSchema: StructType, - filters: Seq[Filter], - sharedConf: Configuration): Iterator[InternalRow] = { + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[Filter], + sharedConf: Configuration): Iterator[InternalRow] = { val conf = new Configuration(sharedConf) conf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) conf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) @@ -81,7 +81,7 @@ abstract class SparkHoodieParquetReaderBase(enableVectorizedReader: Boolean, } -trait SparkHoodieParquetReaderBuilder { +trait SparkParquetReaderBuilder { /** * Get parquet file reader * @@ -95,5 +95,5 @@ trait SparkHoodieParquetReaderBuilder { def build(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReader + hadoopConf: Configuration): SparkParquetReader } diff --git a/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TestSparkParquetReaderFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TestSparkParquetReaderFormat.scala index e6e1b8c15a0d..3a16e2d46b62 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TestSparkParquetReaderFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TestSparkParquetReaderFormat.scala @@ -48,7 +48,7 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration /** - * Class used to test [[SparkHoodieParquetReader]] + * Class used to test [[SparkParquetReader]] * This class should have the same functionality as [[ParquetFileFormat]] */ class TestSparkParquetReaderFormat extends ParquetFileFormat with SparkAdapterSupport { @@ -61,7 +61,7 @@ class TestSparkParquetReaderFormat extends ParquetFileFormat with SparkAdapterSu options: Map[String, String], hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = { //reader must be created outsize of the lambda. This happens on the driver - val reader = sparkAdapter.createHoodieParquetFileReader(supportBatch(sparkSession, + val reader = sparkAdapter.createParquetFileReader(supportBatch(sparkSession, StructType(partitionSchema.fields ++ requiredSchema.fields)), sparkSession.sqlContext.conf, options, hadoopConf) val broadcastedHadoopConf = diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkHoodieParquetReader.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkParquetReader.java similarity index 74% rename from hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkHoodieParquetReader.java rename to hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkParquetReader.java index 5130b504ffb0..1c755ce3c8fd 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkHoodieParquetReader.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkParquetReader.java @@ -30,7 +30,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; @Tag("functional") -public class TestSparkHoodieParquetReader extends TestBootstrapReadBase { +public class TestSparkParquetReader extends TestBootstrapReadBase { @Test public void testReader() { @@ -38,11 +38,11 @@ public void testReader() { int n = 10; Dataset inserts = makeInsertDf("000", n); inserts.write().format("parquet").save(bootstrapBasePath); - Dataset individualReader = JavaConversions.createTestDataFrame(sparkSession, bootstrapBasePath); - Dataset sparkParquetReader = sparkSession.read().format("parquet").load(bootstrapBasePath); - assertEquals(sparkParquetReader.count(), n); - assertEquals(individualReader.count(), n); - assertEquals(sparkParquetReader.except(individualReader).count(), 0); - assertEquals(individualReader.except(sparkParquetReader).count(), 0); + Dataset parquetReadRows = JavaConversions.createTestDataFrame(sparkSession, bootstrapBasePath); + Dataset datasourceReadRows = sparkSession.read().format("parquet").load(bootstrapBasePath); + assertEquals(datasourceReadRows.count(), n); + assertEquals(parquetReadRows.count(), n); + assertEquals(datasourceReadRows.except(parquetReadRows).count(), 0); + assertEquals(parquetReadRows.except(datasourceReadRows).count(), 0); } } diff --git a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_5Adapter.scala b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_5Adapter.scala index 3f5e53b5b687..a3e0c19621b9 100644 --- a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_5Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_5Adapter.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.METADATA_COL_ATTR_KEY import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark35HoodieParquetReader, Spark35LegacyHoodieParquetFileFormat, SparkHoodieParquetReader} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark35ParquetReader, Spark35LegacyHoodieParquetFileFormat, SparkParquetReader} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.hudi.analysis.TableValuedFunctions import org.apache.spark.sql.internal.SQLConf @@ -136,10 +136,10 @@ class Spark3_5Adapter extends BaseSpark3Adapter { * @param hadoopConf some configs will be set for the hadoopConf * @return parquet file reader */ - override def createHoodieParquetFileReader(vectorized: Boolean, - sqlConf: SQLConf, - options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReader = { - Spark35HoodieParquetReader.build(vectorized, sqlConf, options, hadoopConf) + override def createParquetFileReader(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkParquetReader = { + Spark35ParquetReader.build(vectorized, sqlConf, options, hadoopConf) } } diff --git a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35ParquetReader.scala similarity index 91% rename from hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReader.scala rename to hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35ParquetReader.scala index 48271ba5f005..f088efd07e1e 100644 --- a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35ParquetReader.scala @@ -37,22 +37,22 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType -class Spark35HoodieParquetReader(enableVectorizedReader: Boolean, - datetimeRebaseModeInRead: String, - int96RebaseModeInRead: String, - enableParquetFilterPushDown: Boolean, - pushDownDate: Boolean, - pushDownTimestamp: Boolean, - pushDownDecimal: Boolean, - pushDownInFilterThreshold: Int, - pushDownStringPredicate: Boolean, - isCaseSensitive: Boolean, - timestampConversion: Boolean, - enableOffHeapColumnVector: Boolean, - capacity: Int, - returningBatch: Boolean, - enableRecordFilter: Boolean, - timeZoneId: Option[String]) extends SparkHoodieParquetReaderBase( +class Spark35ParquetReader(enableVectorizedReader: Boolean, + datetimeRebaseModeInRead: String, + int96RebaseModeInRead: String, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringPredicate: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) extends SparkParquetReaderBase( enableVectorizedReader = enableVectorizedReader, enableParquetFilterPushDown = enableParquetFilterPushDown, pushDownDate = pushDownDate, @@ -225,7 +225,7 @@ class Spark35HoodieParquetReader(enableVectorizedReader: Boolean, } } -object Spark35HoodieParquetReader extends SparkHoodieParquetReaderBuilder { +object Spark35ParquetReader extends SparkParquetReaderBuilder { /** * Get parquet file reader * @@ -238,7 +238,7 @@ object Spark35HoodieParquetReader extends SparkHoodieParquetReaderBuilder { def build(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReader = { + hadoopConf: Configuration): SparkParquetReader = { //set hadoopconf hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) @@ -263,7 +263,7 @@ object Spark35HoodieParquetReader extends SparkHoodieParquetReaderBuilder { .equals("true") val parquetOptions = new ParquetOptions(options, sqlConf) - new Spark35HoodieParquetReader( + new Spark35ParquetReader( enableVectorizedReader = vectorized, datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead, int96RebaseModeInRead = parquetOptions.int96RebaseModeInRead, From dbdefad652d5c51b19175ca70374b7737a004952 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 11 Apr 2024 17:51:59 -0400 Subject: [PATCH 52/86] rename spark 3.4 --- .../spark/sql/adapter/Spark3_4Adapter.scala | 12 +++--- ...eader.scala => Spark34ParquetReader.scala} | 38 +++++++++---------- 2 files changed, 25 insertions(+), 25 deletions(-) rename hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/{Spark34HoodieParquetReader.scala => Spark34ParquetReader.scala} (90%) diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala index db498a508884..1e2807df55b3 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_4Adapter.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.METADATA_COL_ATTR_KEY import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark34HoodieParquetReader, Spark34LegacyHoodieParquetFileFormat, SparkHoodieParquetReader} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark34LegacyHoodieParquetFileFormat, Spark34ParquetReader, SparkParquetReader} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hudi.analysis.TableValuedFunctions @@ -136,10 +136,10 @@ class Spark3_4Adapter extends BaseSpark3Adapter { * @param hadoopConf some configs will be set for the hadoopConf * @return parquet file reader */ - override def createHoodieParquetFileReader(vectorized: Boolean, - sqlConf: SQLConf, - options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReader = { - Spark34HoodieParquetReader.build(vectorized, sqlConf, options, hadoopConf) + override def createParquetFileReader(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkParquetReader = { + Spark34ParquetReader.build(vectorized, sqlConf, options, hadoopConf) } } diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34ParquetReader.scala similarity index 90% rename from hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReader.scala rename to hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34ParquetReader.scala index 67b89bb579bb..73db889a0448 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34ParquetReader.scala @@ -37,22 +37,22 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ -class Spark34HoodieParquetReader(enableVectorizedReader: Boolean, - datetimeRebaseModeInRead: String, - int96RebaseModeInRead: String, - enableParquetFilterPushDown: Boolean, - pushDownDate: Boolean, - pushDownTimestamp: Boolean, - pushDownDecimal: Boolean, - pushDownInFilterThreshold: Int, - pushDownStringPredicate: Boolean, - isCaseSensitive: Boolean, - timestampConversion: Boolean, - enableOffHeapColumnVector: Boolean, - capacity: Int, - returningBatch: Boolean, - enableRecordFilter: Boolean, - timeZoneId: Option[String]) extends SparkHoodieParquetReaderBase( +class Spark34ParquetReader(enableVectorizedReader: Boolean, + datetimeRebaseModeInRead: String, + int96RebaseModeInRead: String, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringPredicate: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) extends SparkParquetReaderBase( enableVectorizedReader = enableVectorizedReader, enableParquetFilterPushDown = enableParquetFilterPushDown, pushDownDate = pushDownDate, @@ -218,7 +218,7 @@ class Spark34HoodieParquetReader(enableVectorizedReader: Boolean, } } -object Spark34HoodieParquetReader extends SparkHoodieParquetReaderBuilder { +object Spark34ParquetReader extends SparkParquetReaderBuilder { /** * Get parquet file reader * @@ -231,7 +231,7 @@ object Spark34HoodieParquetReader extends SparkHoodieParquetReaderBuilder { def build(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReader = { + hadoopConf: Configuration): SparkParquetReader = { //set hadoopconf hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) @@ -256,7 +256,7 @@ object Spark34HoodieParquetReader extends SparkHoodieParquetReaderBuilder { .equals("true") val parquetOptions = new ParquetOptions(options, sqlConf) - new Spark34HoodieParquetReader( + new Spark34ParquetReader( enableVectorizedReader = vectorized, datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead, int96RebaseModeInRead = parquetOptions.int96RebaseModeInRead, From f9508355a6383753d2fbc5bd83e749d794bb5a80 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 11 Apr 2024 17:59:21 -0400 Subject: [PATCH 53/86] rename for spark3.3 --- .../spark/sql/adapter/Spark3_3Adapter.scala | 12 +++--- ...eader.scala => Spark33ParquetReader.scala} | 38 +++++++++---------- 2 files changed, 25 insertions(+), 25 deletions(-) rename hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/{Spark33HoodieParquetReader.scala => Spark33ParquetReader.scala} (89%) diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala index 1f0ad5f32aed..c11c404c33a0 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.METADATA_COL_ATTR_KEY import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark33HoodieParquetReader, Spark33LegacyHoodieParquetFileFormat, SparkHoodieParquetReader} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark33LegacyHoodieParquetFileFormat, Spark33ParquetReader, SparkParquetReader} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hudi.analysis.TableValuedFunctions @@ -136,10 +136,10 @@ class Spark3_3Adapter extends BaseSpark3Adapter { * @param hadoopConf some configs will be set for the hadoopConf * @return parquet file reader */ - override def createHoodieParquetFileReader(vectorized: Boolean, - sqlConf: SQLConf, - options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReader = { - Spark33HoodieParquetReader.build(vectorized, sqlConf, options, hadoopConf) + override def createParquetFileReader(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkParquetReader = { + Spark33ParquetReader.build(vectorized, sqlConf, options, hadoopConf) } } diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33ParquetReader.scala similarity index 89% rename from hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReader.scala rename to hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33ParquetReader.scala index 557753e54841..0bd8cca3599b 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33ParquetReader.scala @@ -41,22 +41,22 @@ import org.apache.spark.sql.types._ import java.net.URI -class Spark33HoodieParquetReader(enableVectorizedReader: Boolean, - datetimeRebaseModeInRead: String, - int96RebaseModeInRead: String, - enableParquetFilterPushDown: Boolean, - pushDownDate: Boolean, - pushDownTimestamp: Boolean, - pushDownDecimal: Boolean, - pushDownInFilterThreshold: Int, - pushDownStringStartWith: Boolean, - isCaseSensitive: Boolean, - timestampConversion: Boolean, - enableOffHeapColumnVector: Boolean, - capacity: Int, - returningBatch: Boolean, - enableRecordFilter: Boolean, - timeZoneId: Option[String]) extends SparkHoodieParquetReaderBase( +class Spark33ParquetReader(enableVectorizedReader: Boolean, + datetimeRebaseModeInRead: String, + int96RebaseModeInRead: String, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringStartWith: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) extends SparkParquetReaderBase( enableVectorizedReader = enableVectorizedReader, enableParquetFilterPushDown = enableParquetFilterPushDown, pushDownDate = pushDownDate, @@ -220,7 +220,7 @@ class Spark33HoodieParquetReader(enableVectorizedReader: Boolean, } } -object Spark33HoodieParquetReader extends SparkHoodieParquetReaderBuilder { +object Spark33ParquetReader extends SparkParquetReaderBuilder { /** * Get parquet file reader * @@ -233,7 +233,7 @@ object Spark33HoodieParquetReader extends SparkHoodieParquetReaderBuilder { def build(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReader = { + hadoopConf: Configuration): SparkParquetReader = { //set hadoopconf hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) @@ -248,7 +248,7 @@ object Spark33HoodieParquetReader extends SparkHoodieParquetReaderBuilder { ) val parquetOptions = new ParquetOptions(options, sqlConf) - new Spark33HoodieParquetReader(enableVectorizedReader = vectorized, + new Spark33ParquetReader(enableVectorizedReader = vectorized, datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead, int96RebaseModeInRead = parquetOptions.int96RebaseModeInRead, enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, From 75da5dd8c326bd9685cb76c79bdc2b3b503f142f Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 11 Apr 2024 18:04:17 -0400 Subject: [PATCH 54/86] rename for spark 3.2 --- .../spark/sql/adapter/Spark3_2Adapter.scala | 12 +++--- ...eader.scala => Spark32ParquetReader.scala} | 38 +++++++++---------- 2 files changed, 25 insertions(+), 25 deletions(-) rename hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/{Spark32HoodieParquetReader.scala => Spark32ParquetReader.scala} (90%) diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala index a584946e8a08..ea486c7383b5 100644 --- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{Command, DeleteFromTable, LogicalPlan} import org.apache.spark.sql.catalyst.util.METADATA_COL_ATTR_KEY import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark32HoodieParquetReader, Spark32LegacyHoodieParquetFileFormat, SparkHoodieParquetReader} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark32LegacyHoodieParquetFileFormat, Spark32ParquetReader, SparkParquetReader} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hudi.analysis.TableValuedFunctions @@ -135,10 +135,10 @@ class Spark3_2Adapter extends BaseSpark3Adapter { * @param hadoopConf some configs will be set for the hadoopConf * @return parquet file reader */ - override def createHoodieParquetFileReader(vectorized: Boolean, - sqlConf: SQLConf, - options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReader = { - Spark32HoodieParquetReader.build(vectorized, sqlConf, options, hadoopConf) + override def createParquetFileReader(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkParquetReader = { + Spark32ParquetReader.build(vectorized, sqlConf, options, hadoopConf) } } diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32ParquetReader.scala similarity index 90% rename from hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReader.scala rename to hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32ParquetReader.scala index 8db3319273b1..5437a18cd4bd 100644 --- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32ParquetReader.scala @@ -40,22 +40,22 @@ import org.apache.spark.sql.types._ import java.net.URI -class Spark32HoodieParquetReader(enableVectorizedReader: Boolean, - datetimeRebaseModeInRead: String, - int96RebaseModeInRead: String, - enableParquetFilterPushDown: Boolean, - pushDownDate: Boolean, - pushDownTimestamp: Boolean, - pushDownDecimal: Boolean, - pushDownInFilterThreshold: Int, - pushDownStringStartWith: Boolean, - isCaseSensitive: Boolean, - timestampConversion: Boolean, - enableOffHeapColumnVector: Boolean, - capacity: Int, - returningBatch: Boolean, - enableRecordFilter: Boolean, - timeZoneId: Option[String]) extends SparkHoodieParquetReaderBase( +class Spark32ParquetReader(enableVectorizedReader: Boolean, + datetimeRebaseModeInRead: String, + int96RebaseModeInRead: String, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringStartWith: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) extends SparkParquetReaderBase( enableVectorizedReader = enableVectorizedReader, enableParquetFilterPushDown = enableParquetFilterPushDown, pushDownDate = pushDownDate, @@ -218,7 +218,7 @@ class Spark32HoodieParquetReader(enableVectorizedReader: Boolean, } } -object Spark32HoodieParquetReader extends SparkHoodieParquetReaderBuilder { +object Spark32ParquetReader extends SparkParquetReaderBuilder { /** * Get parquet file reader * @@ -231,7 +231,7 @@ object Spark32HoodieParquetReader extends SparkHoodieParquetReaderBuilder { def build(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReader = { + hadoopConf: Configuration): SparkParquetReader = { //set hadoopconf hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) @@ -246,7 +246,7 @@ object Spark32HoodieParquetReader extends SparkHoodieParquetReaderBuilder { ) val parquetOptions = new ParquetOptions(options, sqlConf) - new Spark32HoodieParquetReader( + new Spark32ParquetReader( enableVectorizedReader = vectorized, datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead, int96RebaseModeInRead = parquetOptions.int96RebaseModeInRead, From e7e4b5174c2a38398ec5c4b23f460cb941631230 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 11 Apr 2024 18:09:50 -0400 Subject: [PATCH 55/86] rename spark 3.1 --- .../spark/sql/adapter/Spark3_1Adapter.scala | 12 +++---- ...eader.scala => Spark31ParquetReader.scala} | 34 +++++++++---------- 2 files changed, 23 insertions(+), 23 deletions(-) rename hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/{Spark31HoodieParquetReader.scala => Spark31ParquetReader.scala} (90%) diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala index d330886d3791..21f897afe1c0 100644 --- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_1Adapter.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.connector.catalog.V2TableWithV1Fallback -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark31HoodieParquetReader, Spark31LegacyHoodieParquetFileFormat, SparkHoodieParquetReader} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark31LegacyHoodieParquetFileFormat, Spark31ParquetReader, SparkParquetReader} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, HoodieSpark31PartitionedFileUtils, HoodieSparkPartitionedFileUtils, PartitionedFile} import org.apache.spark.sql.hudi.SparkAdapter @@ -132,10 +132,10 @@ class Spark3_1Adapter extends BaseSpark3Adapter { * @param hadoopConf some configs will be set for the hadoopConf * @return parquet file reader */ - override def createHoodieParquetFileReader(vectorized: Boolean, - sqlConf: SQLConf, - options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReader = { - Spark31HoodieParquetReader.build(vectorized, sqlConf, options, hadoopConf) + override def createParquetFileReader(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkParquetReader = { + Spark31ParquetReader.build(vectorized, sqlConf, options, hadoopConf) } } diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31ParquetReader.scala similarity index 90% rename from hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReader.scala rename to hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31ParquetReader.scala index 5c1bb612f7db..0862da766913 100644 --- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31ParquetReader.scala @@ -39,20 +39,20 @@ import org.apache.spark.sql.types._ import java.net.URI -class Spark31HoodieParquetReader(enableVectorizedReader: Boolean, - enableParquetFilterPushDown: Boolean, - pushDownDate: Boolean, - pushDownTimestamp: Boolean, - pushDownDecimal: Boolean, - pushDownInFilterThreshold: Int, - pushDownStringStartWith: Boolean, - isCaseSensitive: Boolean, - timestampConversion: Boolean, - enableOffHeapColumnVector: Boolean, - capacity: Int, - returningBatch: Boolean, - enableRecordFilter: Boolean, - timeZoneId: Option[String]) extends SparkHoodieParquetReaderBase( +class Spark31ParquetReader(enableVectorizedReader: Boolean, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringStartWith: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) extends SparkParquetReaderBase( enableVectorizedReader = enableVectorizedReader, enableParquetFilterPushDown = enableParquetFilterPushDown, pushDownDate = pushDownDate, @@ -201,7 +201,7 @@ class Spark31HoodieParquetReader(enableVectorizedReader: Boolean, } } -object Spark31HoodieParquetReader extends SparkHoodieParquetReaderBuilder { +object Spark31ParquetReader extends SparkParquetReaderBuilder { /** * Get parquet file reader @@ -215,7 +215,7 @@ object Spark31HoodieParquetReader extends SparkHoodieParquetReaderBuilder { def build(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReader = { + hadoopConf: Configuration): SparkParquetReader = { //set hadoopconf hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) @@ -223,7 +223,7 @@ object Spark31HoodieParquetReader extends SparkHoodieParquetReaderBuilder { 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) - new Spark31HoodieParquetReader( + new Spark31ParquetReader( enableVectorizedReader = vectorized, enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, pushDownDate = sqlConf.parquetFilterPushDownDate, From 81da1a72b77e43339723ed0019807e3375828efb Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 11 Apr 2024 18:15:55 -0400 Subject: [PATCH 56/86] rename spark 30 --- .../spark/sql/adapter/Spark3_0Adapter.scala | 8 ++--- ...eader.scala => Spark30ParquetReader.scala} | 34 +++++++++---------- 2 files changed, 21 insertions(+), 21 deletions(-) rename hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/{Spark30HoodieParquetReader.scala => Spark30ParquetReader.scala} (89%) diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala index 2a472c44f57d..8fbcf5a060b4 100644 --- a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala +++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_0Adapter.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark30HoodieParquetReader, Spark30LegacyHoodieParquetFileFormat, SparkHoodieParquetReader} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark30LegacyHoodieParquetFileFormat, Spark30ParquetReader, SparkParquetReader} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, HoodieSpark30PartitionedFileUtils, HoodieSparkPartitionedFileUtils, PartitionedFile} import org.apache.spark.sql.hudi.SparkAdapter @@ -130,10 +130,10 @@ class Spark3_0Adapter extends BaseSpark3Adapter { * @param hadoopConf some configs will be set for the hadoopConf * @return parquet file reader */ - override def createHoodieParquetFileReader(vectorized: Boolean, + override def createParquetFileReader(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReader = { - Spark30HoodieParquetReader.build(vectorized, sqlConf, options, hadoopConf) + hadoopConf: Configuration): SparkParquetReader = { + Spark30ParquetReader.build(vectorized, sqlConf, options, hadoopConf) } } diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30ParquetReader.scala similarity index 89% rename from hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala rename to hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30ParquetReader.scala index 3d950fb5cbdd..9088676c3354 100644 --- a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30ParquetReader.scala @@ -39,20 +39,20 @@ import org.apache.spark.sql.types._ import java.net.URI -class Spark30HoodieParquetReader(enableVectorizedReader: Boolean, - enableParquetFilterPushDown: Boolean, - pushDownDate: Boolean, - pushDownTimestamp: Boolean, - pushDownDecimal: Boolean, - pushDownInFilterThreshold: Int, - pushDownStringStartWith: Boolean, - isCaseSensitive: Boolean, - timestampConversion: Boolean, - enableOffHeapColumnVector: Boolean, - capacity: Int, - returningBatch: Boolean, - enableRecordFilter: Boolean, - timeZoneId: Option[String]) extends SparkHoodieParquetReaderBase( +class Spark30ParquetReader(enableVectorizedReader: Boolean, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringStartWith: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) extends SparkParquetReaderBase( enableVectorizedReader = enableVectorizedReader, enableParquetFilterPushDown = enableParquetFilterPushDown, pushDownDate = pushDownDate, @@ -188,7 +188,7 @@ class Spark30HoodieParquetReader(enableVectorizedReader: Boolean, } -object Spark30HoodieParquetReader extends SparkHoodieParquetReaderBuilder { +object Spark30ParquetReader extends SparkParquetReaderBuilder { /** * Get parquet file reader * @@ -201,7 +201,7 @@ object Spark30HoodieParquetReader extends SparkHoodieParquetReaderBuilder { def build(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReader = { + hadoopConf: Configuration): SparkParquetReader = { //set hadoopconf hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) @@ -210,7 +210,7 @@ object Spark30HoodieParquetReader extends SparkHoodieParquetReaderBuilder { hadoopConf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING.key, sqlConf.isParquetBinaryAsString) hadoopConf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, sqlConf.isParquetINT96AsTimestamp) - new Spark30HoodieParquetReader( + new Spark30ParquetReader( enableVectorizedReader = vectorized, enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, pushDownDate = sqlConf.parquetFilterPushDownDate, From 1c68439b5487382b354a0cc2b39b2e95577a52dd Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 11 Apr 2024 18:21:48 -0400 Subject: [PATCH 57/86] rename for spark 2 --- .../spark/sql/adapter/Spark2Adapter.scala | 12 +++---- ...eader.scala => Spark24ParquetReader.scala} | 34 +++++++++---------- 2 files changed, 23 insertions(+), 23 deletions(-) rename hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/{Spark24HoodieParquetReader.scala => Spark24ParquetReader.scala} (90%) diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala index f1c38ee85aaf..7b5f5847562b 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.{Command, DeleteFromTable} import org.apache.spark.sql.catalyst.util.DateFormatter import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark24HoodieParquetReader, Spark24LegacyHoodieParquetFileFormat, SparkHoodieParquetReader} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark24LegacyHoodieParquetFileFormat, Spark24ParquetReader, SparkParquetReader} import org.apache.spark.sql.execution.vectorized.MutableColumnarRow import org.apache.spark.sql.hudi.SparkAdapter import org.apache.spark.sql.hudi.parser.HoodieSpark2ExtendedSqlParser @@ -217,10 +217,10 @@ class Spark2Adapter extends SparkAdapter { * @param hadoopConf some configs will be set for the hadoopConf * @return parquet file reader */ - override def createHoodieParquetFileReader(vectorized: Boolean, - sqlConf: SQLConf, - options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReader = { - Spark24HoodieParquetReader.build(vectorized, sqlConf, options, hadoopConf) + override def createParquetFileReader(vectorized: Boolean, + sqlConf: SQLConf, + options: Map[String, String], + hadoopConf: Configuration): SparkParquetReader = { + Spark24ParquetReader.build(vectorized, sqlConf, options, hadoopConf) } } diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24ParquetReader.scala similarity index 90% rename from hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala rename to hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24ParquetReader.scala index 86b5c0e99ffe..7fa30a362224 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24ParquetReader.scala @@ -39,20 +39,20 @@ import org.apache.spark.sql.types.StructType import java.net.URI -class Spark24HoodieParquetReader(enableVectorizedReader: Boolean, - enableParquetFilterPushDown: Boolean, - pushDownDate: Boolean, - pushDownTimestamp: Boolean, - pushDownDecimal: Boolean, - pushDownInFilterThreshold: Int, - pushDownStringStartWith: Boolean, - isCaseSensitive: Boolean, - timestampConversion: Boolean, - enableOffHeapColumnVector: Boolean, - capacity: Int, - returningBatch: Boolean, - enableRecordFilter: Boolean, - timeZoneId: Option[String]) extends SparkHoodieParquetReaderBase( +class Spark24ParquetReader(enableVectorizedReader: Boolean, + enableParquetFilterPushDown: Boolean, + pushDownDate: Boolean, + pushDownTimestamp: Boolean, + pushDownDecimal: Boolean, + pushDownInFilterThreshold: Int, + pushDownStringStartWith: Boolean, + isCaseSensitive: Boolean, + timestampConversion: Boolean, + enableOffHeapColumnVector: Boolean, + capacity: Int, + returningBatch: Boolean, + enableRecordFilter: Boolean, + timeZoneId: Option[String]) extends SparkParquetReaderBase( enableVectorizedReader = enableVectorizedReader, enableParquetFilterPushDown = enableParquetFilterPushDown, pushDownDate = pushDownDate, @@ -185,7 +185,7 @@ class Spark24HoodieParquetReader(enableVectorizedReader: Boolean, } } -object Spark24HoodieParquetReader extends SparkHoodieParquetReaderBuilder { +object Spark24ParquetReader extends SparkParquetReaderBuilder { /** * Get parquet file reader * @@ -198,7 +198,7 @@ object Spark24HoodieParquetReader extends SparkHoodieParquetReaderBuilder { def build(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], - hadoopConf: Configuration): SparkHoodieParquetReader = { + hadoopConf: Configuration): SparkParquetReader = { //set hadoopconf hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) @@ -206,7 +206,7 @@ object Spark24HoodieParquetReader extends SparkHoodieParquetReaderBuilder { hadoopConf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING.key, sqlConf.isParquetBinaryAsString) hadoopConf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, sqlConf.isParquetINT96AsTimestamp) - new Spark24HoodieParquetReader( + new Spark24ParquetReader( enableVectorizedReader = vectorized, enableParquetFilterPushDown = sqlConf.parquetFilterPushDown, pushDownDate = sqlConf.parquetFilterPushDownDate, From f6c5bebf97872d05f27137febbc727d5ad9f8e78 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 11 Apr 2024 18:22:55 -0400 Subject: [PATCH 58/86] remove empty line --- .../sql/execution/datasources/parquet/Spark31ParquetReader.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31ParquetReader.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31ParquetReader.scala index 0862da766913..94a5efaee615 100644 --- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31ParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31ParquetReader.scala @@ -239,5 +239,4 @@ object Spark31ParquetReader extends SparkParquetReaderBuilder { enableRecordFilter = sqlConf.parquetRecordFilterEnabled, timeZoneId = Some(sqlConf.sessionLocalTimeZone)) } - } From 8f1ba6d46d8777f39c522d8bcac545ba3d4fd544 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 11 Apr 2024 22:03:31 -0400 Subject: [PATCH 59/86] address hidden review comments --- .../parquet/SparkParquetReaderBase.scala | 3 --- .../TestSparkParquetReaderFormat.scala | 19 ------------------- 2 files changed, 22 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReaderBase.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReaderBase.scala index f7bfcd9e900d..2b47da764565 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReaderBase.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkParquetReaderBase.scala @@ -39,7 +39,6 @@ abstract class SparkParquetReaderBase(enableVectorizedReader: Boolean, returningBatch: Boolean, enableRecordFilter: Boolean, timeZoneId: Option[String]) extends SparkParquetReader { - /** * Read an individual parquet file * @@ -62,7 +61,6 @@ abstract class SparkParquetReaderBase(enableVectorizedReader: Boolean, doRead(file, requiredSchema, partitionSchema, filters, conf) } - /** * Implemented for each spark version * @@ -91,7 +89,6 @@ trait SparkParquetReaderBuilder { * @param hadoopConf some configs will be set for the hadoopConf * @return properties needed for reading a parquet file */ - def build(vectorized: Boolean, sqlConf: SQLConf, options: Map[String, String], diff --git a/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TestSparkParquetReaderFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TestSparkParquetReaderFormat.scala index 3a16e2d46b62..bf513847cfcb 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TestSparkParquetReaderFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/TestSparkParquetReaderFormat.scala @@ -17,25 +17,6 @@ * under the License. */ -/* - * 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 From ee7a0e3a401dd0d2c0f2d1095256fb9f27c9802f Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Fri, 12 Apr 2024 10:55:33 -0400 Subject: [PATCH 60/86] finish merge --- .../hudi/SparkFileFormatInternalRowReaderContext.scala | 6 +++--- .../HoodieFileGroupReaderBasedParquetFileFormat.scala | 2 +- .../table/read/TestHoodieFileGroupReaderOnSpark.scala | 3 +-- .../read/TestSpark35RecordPositionMetadataColumn.scala | 2 +- 4 files changed, 6 insertions(+), 7 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala index 8b7e3d1684f8..59213297b138 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala @@ -23,7 +23,7 @@ import org.apache.avro.Schema import org.apache.avro.generic.IndexedRecord import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hudi.SparkFileFormatInternalRowReaderContext.{ROW_INDEX_TEMPORARY_COLUMN_NAME, getAppliedRequiredSchema, hasIndexTempColumn} +import org.apache.hudi.SparkFileFormatInternalRowReaderContext.{ROW_INDEX_TEMPORARY_COLUMN_NAME, getAppliedRequiredSchema} import org.apache.hudi.avro.AvroSchemaUtils import org.apache.hudi.common.engine.HoodieReaderContext import org.apache.hudi.common.fs.FSUtils @@ -36,7 +36,7 @@ import org.apache.spark.sql.avro.HoodieAvroDeserializer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.execution.datasources.PartitionedFile -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, SparkHoodieParquetReader} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, SparkParquetReader} import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.{LongType, MetadataBuilder, StructField, StructType} import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} @@ -54,7 +54,7 @@ import scala.collection.mutable * not required for reading a file group with only log files. * @param partitionValues The values for a partition in which the file group lives. */ -class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkHoodieParquetReader, +class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkParquetReader, recordKeyColumn: String, filters: Seq[Filter], shouldUseRecordPosition: Boolean) extends BaseSparkInternalRowReaderContext { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala index 7d2aaa47adf5..c1ad8b528aeb 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala @@ -128,7 +128,7 @@ override def supportBatch(sparkSession: SparkSession, schema: StructType): Boole val requestedAvroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(requiredSchema, sanitizedTableName) val dataAvroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(dataSchema, sanitizedTableName) - val parquetFileReader = spark.sparkContext.broadcast(sparkAdapter.createHoodieParquetFileReader(supportBatchResult, spark.sessionState.conf, options, augmentedHadoopConf)) + val parquetFileReader = spark.sparkContext.broadcast(sparkAdapter.createParquetFileReader(supportBatchResult, spark.sessionState.conf, options, augmentedHadoopConf)) val broadcastedHadoopConf = spark.sparkContext.broadcast(new SerializableConfiguration(augmentedHadoopConf)) val broadcastedDataSchema = spark.sparkContext.broadcast(dataAvroSchema) val broadcastedRequestedSchema = spark.sparkContext.broadcast(requestedAvroSchema) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala index 2bba33c5192f..11ccebc469a5 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala @@ -21,7 +21,6 @@ package org.apache.hudi.common.table.read import org.apache.avro.Schema import org.apache.hadoop.conf.Configuration -import org.apache.hudi.common.util.Option import org.apache.hudi.common.config.HoodieReaderConfig.FILE_GROUP_READER_ENABLED import org.apache.hudi.common.engine.HoodieReaderContext import org.apache.hudi.common.fs.FSUtils @@ -81,7 +80,7 @@ class TestHoodieFileGroupReaderOnSpark extends TestHoodieFileGroupReaderBase[Int } override def getHoodieReaderContext(tablePath: String, avroSchema: Schema, hadoopConf: Configuration): HoodieReaderContext[InternalRow] = { - val reader = sparkAdapter.createHoodieParquetFileReader(vectorized = false, spark.sessionState.conf, Map.empty, hadoopConf) + val reader = sparkAdapter.createParquetFileReader(vectorized = false, spark.sessionState.conf, Map.empty, hadoopConf) val metaClient = HoodieTableMetaClient.builder().setConf(getHadoopConf).setBasePath(tablePath).build val recordKeyField = if (metaClient.getTableConfig.populateMetaFields()) { HoodieRecord.RECORD_KEY_METADATA_FIELD diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala index 416525d19e10..6180cfbad820 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala @@ -76,7 +76,7 @@ class TestSpark35RecordPositionMetadataColumn extends SparkClientFunctionalTestH val hadoopConf = new Configuration(spark().sparkContext.hadoopConfiguration) val props = Map("spark.sql.parquet.enableVectorizedReader" -> "false") _spark.conf.set("spark.sql.parquet.enableVectorizedReader", "false") - val reader = sparkAdapter.createHoodieParquetFileReader(vectorized = false, _spark.sessionState.conf, props, hadoopConf) + val reader = sparkAdapter.createParquetFileReader(vectorized = false, _spark.sessionState.conf, props, hadoopConf) val requiredSchema = SparkFileFormatInternalRowReaderContext.getAppliedRequiredSchema(dataSchema) // Confirm if the schema is as expected. From c8f507bcac03c7183893400487a1885400c46853 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Fri, 12 Apr 2024 17:57:46 -0400 Subject: [PATCH 61/86] add missing import --- .../execution/datasources/parquet/Spark24ParquetReader.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24ParquetReader.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24ParquetReader.scala index efd316e93d1d..42808f337b70 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24ParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24ParquetReader.scala @@ -30,7 +30,7 @@ import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, Parquet import org.apache.spark.TaskContext 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.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 From be7795021e2cffe600a109448ed02e5860385b9f Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Mon, 15 Apr 2024 11:58:37 -0400 Subject: [PATCH 62/86] address comments and add changes to legacy 3.5 --- .../Spark3ParquetSchemaEvolutionUtils.scala | 10 +- ...ark32PlusParquetSchemaEvolutionUtils.scala | 49 ++- ...Spark35LegacyHoodieParquetFileFormat.scala | 405 ++++-------------- .../parquet/Spark35ParquetReader.scala | 35 +- .../Spark35ParquetSchemaEvolutionUtils.scala | 36 -- 5 files changed, 121 insertions(+), 414 deletions(-) delete mode 100644 hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35ParquetSchemaEvolutionUtils.scala diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/Spark3ParquetSchemaEvolutionUtils.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/Spark3ParquetSchemaEvolutionUtils.scala index ba72ed158705..4f83ef5a5699 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/Spark3ParquetSchemaEvolutionUtils.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/Spark3ParquetSchemaEvolutionUtils.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.hudi.SparkAdapterSupport import org.apache.hudi.client.utils.SparkInternalSchemaConverter import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.util @@ -30,6 +31,7 @@ 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.hadoop.metadata.FileMetaData +import org.apache.spark.sql.HoodieSchemaUtils import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Cast, UnsafeProjection} import org.apache.spark.sql.execution.datasources.Spark3ParquetSchemaEvolutionUtils.pruneInternalSchema @@ -42,7 +44,7 @@ import scala.collection.convert.ImplicitConversions.`collection AsScalaIterable` abstract class Spark3ParquetSchemaEvolutionUtils(sharedConf: Configuration, filePath: Path, requiredSchema: StructType, - partitionSchema: StructType) { + partitionSchema: StructType) extends SparkAdapterSupport{ // Fetch internal schema private lazy val internalSchemaStr: String = sharedConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA) @@ -50,6 +52,8 @@ abstract class Spark3ParquetSchemaEvolutionUtils(sharedConf: Configuration, var shouldUseInternalSchema: Boolean = !isNullOrEmpty(internalSchemaStr) && querySchemaOption.isPresent + private lazy val schemaUtils: HoodieSchemaUtils = sparkAdapter.getSchemaUtils + private lazy val tablePath: String = sharedConf.get(SparkInternalSchemaConverter.HOODIE_TABLE_PATH) private lazy val fileSchema: InternalSchema = if (shouldUseInternalSchema) { val commitInstantTime = FSUtils.getCommitTime(filePath.getName).toLong; @@ -162,7 +166,7 @@ abstract class Spark3ParquetSchemaEvolutionUtils(sharedConf: Configuration, StructField(f.name, typeChangeInfos.get(i).getRight, f.nullable, f.metadata) } else f }) - val newFullSchema = toAttributes(newSchema) ++ toAttributes(partitionSchema) + val newFullSchema = schemaUtils.toAttributes(newSchema) ++ schemaUtils.toAttributes(partitionSchema) val castSchema = newFullSchema.zipWithIndex.map { case (attr, i) => if (typeChangeInfos.containsKey(i)) { val srcType = typeChangeInfos.get(i).getRight @@ -174,8 +178,6 @@ abstract class Spark3ParquetSchemaEvolutionUtils(sharedConf: Configuration, GenerateUnsafeProjection.generate(castSchema, newFullSchema) } } - - protected def toAttributes(schema: StructType): Seq[AttributeReference] } object Spark3ParquetSchemaEvolutionUtils { diff --git a/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusParquetSchemaEvolutionUtils.scala b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusParquetSchemaEvolutionUtils.scala index 2be3159742bd..05523bcd9f04 100644 --- a/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusParquetSchemaEvolutionUtils.scala +++ b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusParquetSchemaEvolutionUtils.scala @@ -27,27 +27,38 @@ import org.apache.spark.sql.types.StructType import java.time.ZoneId -abstract class Spark32PlusParquetSchemaEvolutionUtils(sharedConf: Configuration, - filePath: Path, - requiredSchema: StructType, - partitionSchema: StructType) extends +class Spark32PlusParquetSchemaEvolutionUtils(sharedConf: Configuration, + filePath: Path, + requiredSchema: StructType, + partitionSchema: StructType) extends Spark3ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) { - def buildVectorizedReader(convertTz: Option[ZoneId], - datetimeRebaseSpec: RebaseDateTime.RebaseSpec, - int96RebaseSpec: RebaseDateTime.RebaseSpec, - enableOffHeapColumnVector: Boolean, - taskContext: Option[TaskContext], + def buildVectorizedReader(convertTz: ZoneId, + datetimeRebaseMode: String, + datetimeRebaseTz: String, + int96RebaseMode: String, + int96RebaseTz: String, + useOffHeap: Boolean, capacity: Int): VectorizedParquetRecordReader = { - new Spark32PlusHoodieVectorizedParquetRecordReader( - convertTz.orNull, - datetimeRebaseSpec.mode.toString, - datetimeRebaseSpec.timeZone, - int96RebaseSpec.mode.toString, - int96RebaseSpec.timeZone, - enableOffHeapColumnVector && taskContext.isDefined, - capacity, - typeChangeInfos - ) + if (shouldUseInternalSchema) { + new Spark32PlusHoodieVectorizedParquetRecordReader( + convertTz, + datetimeRebaseMode, + datetimeRebaseTz, + int96RebaseMode, + int96RebaseTz, + useOffHeap, + capacity, + typeChangeInfos) + } else { + new VectorizedParquetRecordReader( + convertTz, + datetimeRebaseMode, + datetimeRebaseTz, + int96RebaseMode, + int96RebaseTz, + useOffHeap, + capacity) + } } } diff --git a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35LegacyHoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35LegacyHoodieParquetFileFormat.scala index b6177b942fcf..e853e2788864 100644 --- a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35LegacyHoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35LegacyHoodieParquetFileFormat.scala @@ -21,36 +21,21 @@ import org.apache.hadoop.conf.Configuration 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 -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.{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.codegen.GenerateUnsafeProjection -import org.apache.spark.sql.catalyst.expressions.{Cast, JoinedRow} -import org.apache.spark.sql.catalyst.types.DataTypeUtils +import org.apache.spark.sql.catalyst.expressions.JoinedRow +import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.WholeStageCodegenExec -import org.apache.spark.sql.execution.datasources.parquet.Spark35LegacyHoodieParquetFileFormat._ -import org.apache.spark.sql.execution.datasources.{DataSourceUtils, PartitionedFile, RecordReaderIterator} +import org.apache.spark.sql.execution.datasources.{DataSourceUtils, FileFormat, 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.sql.types.StructType import org.apache.spark.util.SerializableConfiguration -import scala.collection.convert.ImplicitConversions.`collection AsScalaIterable` - /** * This class is an extension of [[ParquetFileFormat]] overriding Spark-specific behavior * that's not possible to customize in any other way @@ -63,15 +48,6 @@ import scala.collection.convert.ImplicitConversions.`collection AsScalaIterable` */ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValues: Boolean) extends ParquetFileFormat { - def supportsColumnar(sparkSession: SparkSession, schema: StructType): Boolean = { - val conf = sparkSession.sessionState.conf - // Only output columnar if there is WSCG to read it. - val requiredWholeStageCodegenSettings = - conf.wholeStageEnabled && !WholeStageCodegenExec.isTooManyFields(conf, schema) - requiredWholeStageCodegenSettings && - supportBatch(sparkSession, schema) - } - override def buildReaderWithPartitionValues(sparkSession: SparkSession, dataSchema: StructType, partitionSchema: StructType, @@ -96,8 +72,6 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu SQLConf.CASE_SENSITIVE.key, sparkSession.sessionState.conf.caseSensitiveAnalysis) - ParquetWriteSupport.setSchema(requiredSchema, hadoopConf) - // Sets flags for `ParquetToSparkSchemaConverter` hadoopConf.setBoolean( SQLConf.PARQUET_BINARY_AS_STRING.key, @@ -105,23 +79,12 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu hadoopConf.setBoolean( SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, sparkSession.sessionState.conf.isParquetINT96AsTimestamp) - // Using string value of this conf to preserve compatibility across spark versions. + hadoopConf.setBoolean( + SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key, + sparkSession.sessionState.conf.parquetInferTimestampNTZEnabled) hadoopConf.setBoolean( SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, - sparkSession.sessionState.conf.getConfString( - SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, - SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.defaultValueString).toBoolean - ) - hadoopConf.setBoolean(SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key, sparkSession.sessionState.conf.parquetInferTimestampNTZEnabled) - hadoopConf.setBoolean(SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, sparkSession.sessionState.conf.legacyParquetNanosAsLong) - val internalSchemaStr = hadoopConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA) - // For Spark DataSource v1, there's no Physical Plan projection/schema pruning w/in Spark itself, - // therefore it's safe to do schema projection here - if (!isNullOrEmpty(internalSchemaStr)) { - val prunedInternalSchemaStr = - pruneInternalSchema(internalSchemaStr, requiredSchema) - hadoopConf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, prunedInternalSchemaStr) - } + sparkSession.sessionState.conf.legacyParquetNanosAsLong) val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) @@ -132,7 +95,8 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val resultSchema = StructType(partitionSchema.fields ++ requiredSchema.fields) val sqlConf = sparkSession.sessionState.conf val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled - val enableVectorizedReader: Boolean = supportBatch(sparkSession, resultSchema) + val enableVectorizedReader: Boolean = + ParquetUtils.isBatchReadSupportedForSchema(sqlConf, resultSchema) val enableRecordFilter: Boolean = sqlConf.parquetRecordFilterEnabled val timestampConversion: Boolean = sqlConf.isParquetINT96TimestampConversion val capacity = sqlConf.parquetVectorizedReaderBatchSize @@ -140,7 +104,7 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val pushDownDate = sqlConf.parquetFilterPushDownDate val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal - val pushDownStringStartWith = sqlConf.parquetFilterPushDownStringPredicate + val pushDownStringPredicate = sqlConf.parquetFilterPushDownStringPredicate val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold val isCaseSensitive = sqlConf.caseSensitiveAnalysis val parquetOptions = new ParquetOptions(options, sparkSession.sessionState.conf) @@ -150,72 +114,61 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu // Should always be set by FileSourceScanExec creating this. // Check conf before checking option, to allow working around an issue by changing conf. val returningBatch = sparkSession.sessionState.conf.parquetVectorizedReaderEnabled && - supportsColumnar(sparkSession, resultSchema).toString.equals("true") + options.getOrElse(FileFormat.OPTION_RETURNING_BATCH, + throw new IllegalArgumentException( + "OPTION_RETURNING_BATCH should always be set for ParquetFileFormat. " + + "To workaround this issue, set spark.sql.parquet.enableVectorizedReader=false.")) + .equals("true") + if (returningBatch) { + // If the passed option said that we are to return batches, we need to also be able to + // do this based on config and resultSchema. + assert(supportBatch(sparkSession, resultSchema)) + } (file: PartitionedFile) => { assert(!shouldAppendPartitionValues || file.partitionValues.numFields == partitionSchema.size) - val filePath = file.filePath.toPath + + val filePath = file.toPath val split = new FileSplit(filePath, file.start, file.length, Array.empty[String]) val sharedConf = broadcastedHadoopConf.value.value + val schemaEvolutionUtils = new Spark32PlusParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) - // Fetch internal schema - val internalSchemaStr = sharedConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA) - // Internal schema has to be pruned at this point - val querySchemaOption = SerDeHelper.fromJson(internalSchemaStr) - - var shouldUseInternalSchema = !isNullOrEmpty(internalSchemaStr) && querySchemaOption.isPresent - - val tablePath = sharedConf.get(SparkInternalSchemaConverter.HOODIE_TABLE_PATH) - val fileSchema = if (shouldUseInternalSchema) { - val commitInstantTime = FSUtils.getCommitTime(filePath.getName).toLong; - val validCommits = sharedConf.get(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST) - InternalSchemaCache.getInternalSchemaByVersionId(commitInstantTime, tablePath, sharedConf, if (validCommits == null) "" else validCommits) + val fileFooter = if (enableVectorizedReader) { + // When there are vectorized reads, we can avoid reading the footer twice by reading + // all row groups in advance and filter row groups according to filters that require + // push down (no need to read the footer metadata again). + ParquetFooterReader.readFooter(sharedConf, file, ParquetFooterReader.WITH_ROW_GROUPS) } else { - null + ParquetFooterReader.readFooter(sharedConf, file, ParquetFooterReader.SKIP_ROW_GROUPS) } - lazy val footerFileMetaData = - ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData + val footerFileMetaData = fileFooter.getFileMetaData + val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( + footerFileMetaData.getKeyValueMetaData.get, + datetimeRebaseModeInRead) + val int96RebaseSpec = DataSourceUtils.int96RebaseSpec( + footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) + // Try to push down filters when filter push-down is enabled. val pushed = if (enableParquetFilterPushDown) { val parquetSchema = footerFileMetaData.getSchema - val parquetFilters = if (HoodieSparkUtils.gteqSpark3_2_1) { - // NOTE: Below code could only be compiled against >= Spark 3.2.1, - // and unfortunately won't compile against Spark 3.2.0 - // However this code is runtime-compatible w/ both Spark 3.2.0 and >= Spark 3.2.1 - val datetimeRebaseSpec = - DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) - new ParquetFilters( - parquetSchema, - pushDownDate, - pushDownTimestamp, - pushDownDecimal, - pushDownStringStartWith, - pushDownInFilterThreshold, - isCaseSensitive, - datetimeRebaseSpec) - } else { - // Spark 3.2.0 - val datetimeRebaseMode = - Spark35DataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) - createParquetFilters( - parquetSchema, - pushDownDate, - pushDownTimestamp, - pushDownDecimal, - pushDownStringStartWith, - pushDownInFilterThreshold, - isCaseSensitive, - datetimeRebaseMode) - } - filters.map(rebuildFilterFromParquet(_, fileSchema, querySchemaOption.orElse(null))) + val parquetFilters = new ParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringPredicate, + pushDownInFilterThreshold, + isCaseSensitive, + datetimeRebaseSpec) + filters.map(schemaEvolutionUtils.rebuildFilterFromParquet) // 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) + .flatMap(parquetFilters.createFilter(_)) .reduceOption(FilterApi.and) } else { None @@ -236,35 +189,10 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu None } - val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) - - // Clone new conf - val hadoopAttemptConf = new Configuration(broadcastedHadoopConf.value.value) - val typeChangeInfos: java.util.Map[Integer, Pair[DataType, DataType]] = if (shouldUseInternalSchema) { - val mergedInternalSchema = new InternalSchemaMerger(fileSchema, querySchemaOption.get(), true, true).mergeSchema() - val mergedSchema = SparkInternalSchemaConverter.constructSparkSchemaFromInternalSchema(mergedInternalSchema) - - hadoopAttemptConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, mergedSchema.json) - - SparkInternalSchemaConverter.collectTypeChangedCols(querySchemaOption.get(), mergedInternalSchema) - } else { - val (implicitTypeChangeInfo, sparkRequestSchema) = HoodieParquetFileFormatHelper.buildImplicitSchemaChangeInfo(hadoopAttemptConf, footerFileMetaData, requiredSchema) - if (!implicitTypeChangeInfo.isEmpty) { - shouldUseInternalSchema = true - hadoopAttemptConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, sparkRequestSchema.json) - } - implicitTypeChangeInfo - } - - if (enableVectorizedReader && shouldUseInternalSchema && - !typeChangeInfos.values().forall(_.getLeft.isInstanceOf[AtomicType])) { - throw new IllegalArgumentException( - "Nested types with type changes(implicit or explicit) cannot be read in vectorized mode. " + - "To workaround this issue, set spark.sql.parquet.enableVectorizedReader=false.") - } + val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val hadoopAttemptContext = - new TaskAttemptContextImpl(hadoopAttemptConf, attemptId) + new TaskAttemptContextImpl(schemaEvolutionUtils.getHadoopConfClone(footerFileMetaData, enableVectorizedReader), attemptId) // Try to push down filters when filter push-down is enabled. // Notice: This push-down is RowGroups level, not individual records. @@ -273,51 +201,14 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu } val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { - val vectorizedReader = - if (shouldUseInternalSchema) { - val int96RebaseSpec = - DataSourceUtils.int96RebaseSpec(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) - val datetimeRebaseSpec = - DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) - new Spark32PlusHoodieVectorizedParquetRecordReader( - convertTz.orNull, - datetimeRebaseSpec.mode.toString, - datetimeRebaseSpec.timeZone, - int96RebaseSpec.mode.toString, - int96RebaseSpec.timeZone, - enableOffHeapColumnVector && taskContext.isDefined, - capacity, - typeChangeInfos) - } else if (HoodieSparkUtils.gteqSpark3_2_1) { - // NOTE: Below code could only be compiled against >= Spark 3.2.1, - // and unfortunately won't compile against Spark 3.2.0 - // However this code is runtime-compatible w/ both Spark 3.2.0 and >= Spark 3.2.1 - val int96RebaseSpec = - DataSourceUtils.int96RebaseSpec(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) - val datetimeRebaseSpec = - DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) - new VectorizedParquetRecordReader( - convertTz.orNull, - datetimeRebaseSpec.mode.toString, - datetimeRebaseSpec.timeZone, - int96RebaseSpec.mode.toString, - int96RebaseSpec.timeZone, - enableOffHeapColumnVector && taskContext.isDefined, - capacity) - } else { - // Spark 3.2.0 - val datetimeRebaseMode = - Spark35DataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) - val int96RebaseMode = - Spark35DataSourceUtils.int96RebaseMode(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) - createVectorizedParquetRecordReader( - convertTz.orNull, - datetimeRebaseMode.toString, - int96RebaseMode.toString, - enableOffHeapColumnVector && taskContext.isDefined, - capacity) - } - + val vectorizedReader = schemaEvolutionUtils.buildVectorizedReader( + convertTz.orNull, + datetimeRebaseSpec.mode.toString, + datetimeRebaseSpec.timeZone, + int96RebaseSpec.mode.toString, + int96RebaseSpec.timeZone, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) // SPARK-37089: We cannot register a task completion listener to close this iterator here // because downstream exec nodes have already registered their listeners. Since listeners // are executed in reverse order of registration, a listener registered here would close the @@ -327,17 +218,8 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu // Instead, we use FileScanRDD's task completion listener to close this iterator. val iter = new RecordReaderIterator(vectorizedReader) try { - vectorizedReader.initialize(split, hadoopAttemptContext) - - // NOTE: We're making appending of the partitioned values to the rows read from the - // data file configurable - if (shouldAppendPartitionValues) { - logDebug(s"Appending $partitionSchema ${file.partitionValues}") - vectorizedReader.initBatch(partitionSchema, file.partitionValues) - } else { - vectorizedReader.initBatch(StructType(Nil), InternalRow.empty) - } - + vectorizedReader.initialize(split, hadoopAttemptContext, Option.apply(fileFooter)) + vectorizedReader.initBatch(partitionSchema, file.partitionValues) if (returningBatch) { vectorizedReader.enableReturningBatches() } @@ -352,64 +234,26 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu throw e } } else { - logDebug(s"Falling back to parquet-mr") - val readSupport = if (HoodieSparkUtils.gteqSpark3_2_1) { - // ParquetRecordReader returns InternalRow - // NOTE: Below code could only be compiled against >= Spark 3.2.1, - // and unfortunately won't compile against Spark 3.2.0 - // However this code is runtime-compatible w/ both Spark 3.2.0 and >= Spark 3.2.1 - val int96RebaseSpec = - DataSourceUtils.int96RebaseSpec(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) - val datetimeRebaseSpec = - DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) - new ParquetReadSupport( - convertTz, - enableVectorizedReader = false, - datetimeRebaseSpec, - int96RebaseSpec) - } else { - val datetimeRebaseMode = - Spark35DataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) - val int96RebaseMode = - Spark35DataSourceUtils.int96RebaseMode(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) - createParquetReadSupport( - convertTz, - /* enableVectorizedReader = */ false, - datetimeRebaseMode, - int96RebaseMode) - } - + // ParquetRecordReader returns InternalRow + val readSupport = new ParquetReadSupport( + convertTz, + enableVectorizedReader = false, + datetimeRebaseSpec, + int96RebaseSpec) val reader = if (pushed.isDefined && enableRecordFilter) { val parquetFilter = FilterCompat.get(pushed.get, null) new ParquetRecordReader[InternalRow](readSupport, parquetFilter) } else { new ParquetRecordReader[InternalRow](readSupport) } - val iter = new RecordReaderIterator[InternalRow](reader) + val readerWithRowIndexes = ParquetRowIndexUtil.addRowIndexToRecordReaderIfNeeded(reader, + requiredSchema) + val iter = new RecordReaderIterator[InternalRow](readerWithRowIndexes) try { - reader.initialize(split, hadoopAttemptContext) + readerWithRowIndexes.initialize(split, hadoopAttemptContext) - val fullSchema = DataTypeUtils.toAttributes(requiredSchema) ++ DataTypeUtils.toAttributes(partitionSchema) - val unsafeProjection = if (typeChangeInfos.isEmpty) { - GenerateUnsafeProjection.generate(fullSchema, fullSchema) - } else { - // find type changed. - val newSchema = new StructType(requiredSchema.fields.zipWithIndex.map { case (f, i) => - if (typeChangeInfos.containsKey(i)) { - StructField(f.name, typeChangeInfos.get(i).getRight, f.nullable, f.metadata) - } else f - }) - val newFullSchema = DataTypeUtils.toAttributes(newSchema) ++ DataTypeUtils.toAttributes(partitionSchema) - val castSchema = newFullSchema.zipWithIndex.map { case (attr, i) => - if (typeChangeInfos.containsKey(i)) { - val srcType = typeChangeInfos.get(i).getRight - val dstType = typeChangeInfos.get(i).getLeft - val needTimeZone = Cast.needsTimeZone(srcType, dstType) - Cast(attr, dstType, if (needTimeZone) timeZoneId else None) - } else attr - } - GenerateUnsafeProjection.generate(castSchema, newFullSchema) - } + val fullSchema = toAttributes(requiredSchema) ++ toAttributes(partitionSchema) + val unsafeProjection = schemaEvolutionUtils.generateUnsafeProjection(fullSchema, timeZoneId) // NOTE: We're making appending of the partitioned values to the rows read from the // data file configurable @@ -432,108 +276,3 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu } } -object Spark35LegacyHoodieParquetFileFormat { - - /** - * NOTE: This method is specific to Spark 3.2.0 - */ - private def createParquetFilters(args: Any*): ParquetFilters = { - // NOTE: ParquetFilters ctor args contain Scala enum, therefore we can't look it - // up by arg types, and have to instead rely on the number of args based on individual class; - // the ctor order is not guaranteed - val ctor = classOf[ParquetFilters].getConstructors.maxBy(_.getParameterCount) - ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*) - .asInstanceOf[ParquetFilters] - } - - /** - * NOTE: This method is specific to Spark 3.2.0 - */ - private def createParquetReadSupport(args: Any*): ParquetReadSupport = { - // NOTE: ParquetReadSupport ctor args contain Scala enum, therefore we can't look it - // up by arg types, and have to instead rely on the number of args based on individual class; - // the ctor order is not guaranteed - val ctor = classOf[ParquetReadSupport].getConstructors.maxBy(_.getParameterCount) - ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*) - .asInstanceOf[ParquetReadSupport] - } - - /** - * NOTE: This method is specific to Spark 3.2.0 - */ - private def createVectorizedParquetRecordReader(args: Any*): VectorizedParquetRecordReader = { - // NOTE: ParquetReadSupport ctor args contain Scala enum, therefore we can't look it - // up by arg types, and have to instead rely on the number of args based on individual class; - // the ctor order is not guaranteed - val ctor = classOf[VectorizedParquetRecordReader].getConstructors.maxBy(_.getParameterCount) - ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*) - .asInstanceOf[VectorizedParquetRecordReader] - } - - def pruneInternalSchema(internalSchemaStr: String, requiredSchema: StructType): String = { - val querySchemaOption = SerDeHelper.fromJson(internalSchemaStr) - if (querySchemaOption.isPresent && requiredSchema.nonEmpty) { - val prunedSchema = SparkInternalSchemaConverter.convertAndPruneStructTypeToInternalSchema(requiredSchema, querySchemaOption.get()) - SerDeHelper.toJson(prunedSchema) - } else { - internalSchemaStr - } - } - - private def rebuildFilterFromParquet(oldFilter: Filter, fileSchema: InternalSchema, querySchema: InternalSchema): Filter = { - if (fileSchema == null || querySchema == null) { - oldFilter - } else { - oldFilter match { - case eq: EqualTo => - val newAttribute = InternalSchemaUtils.reBuildFilterName(eq.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else eq.copy(attribute = newAttribute) - case eqs: EqualNullSafe => - val newAttribute = InternalSchemaUtils.reBuildFilterName(eqs.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else eqs.copy(attribute = newAttribute) - case gt: GreaterThan => - val newAttribute = InternalSchemaUtils.reBuildFilterName(gt.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else gt.copy(attribute = newAttribute) - case gtr: GreaterThanOrEqual => - val newAttribute = InternalSchemaUtils.reBuildFilterName(gtr.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else gtr.copy(attribute = newAttribute) - case lt: LessThan => - val newAttribute = InternalSchemaUtils.reBuildFilterName(lt.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else lt.copy(attribute = newAttribute) - case lte: LessThanOrEqual => - val newAttribute = InternalSchemaUtils.reBuildFilterName(lte.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else lte.copy(attribute = newAttribute) - case i: In => - val newAttribute = InternalSchemaUtils.reBuildFilterName(i.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else i.copy(attribute = newAttribute) - case isn: IsNull => - val newAttribute = InternalSchemaUtils.reBuildFilterName(isn.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else isn.copy(attribute = newAttribute) - case isnn: IsNotNull => - val newAttribute = InternalSchemaUtils.reBuildFilterName(isnn.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else isnn.copy(attribute = newAttribute) - case And(left, right) => - And(rebuildFilterFromParquet(left, fileSchema, querySchema), rebuildFilterFromParquet(right, fileSchema, querySchema)) - case Or(left, right) => - Or(rebuildFilterFromParquet(left, fileSchema, querySchema), rebuildFilterFromParquet(right, fileSchema, querySchema)) - case Not(child) => - Not(rebuildFilterFromParquet(child, fileSchema, querySchema)) - case ssw: StringStartsWith => - val newAttribute = InternalSchemaUtils.reBuildFilterName(ssw.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else ssw.copy(attribute = newAttribute) - case ses: StringEndsWith => - val newAttribute = InternalSchemaUtils.reBuildFilterName(ses.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else ses.copy(attribute = newAttribute) - case sc: StringContains => - val newAttribute = InternalSchemaUtils.reBuildFilterName(sc.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else sc.copy(attribute = newAttribute) - case AlwaysTrue => - AlwaysTrue - case AlwaysFalse => - AlwaysFalse - case _ => - AlwaysTrue - } - } - } -} diff --git a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35ParquetReader.scala b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35ParquetReader.scala index efefa97e908a..39c2906c0e5e 100644 --- a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35ParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35ParquetReader.scala @@ -78,16 +78,16 @@ class Spark35ParquetReader(enableVectorizedReader: Boolean, * @return iterator of rows read from the file output type says [[InternalRow]] but could be [[ColumnarBatch]] */ protected def doRead(file: PartitionedFile, - requiredSchema: StructType, - partitionSchema: StructType, - filters: Seq[Filter], - sharedConf: Configuration): Iterator[InternalRow] = { + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[Filter], + sharedConf: Configuration): Iterator[InternalRow] = { assert(file.partitionValues.numFields == partitionSchema.size) val filePath = file.toPath val split = new FileSplit(filePath, file.start, file.length, Array.empty[String]) - val schemaEvolutionUtils = new Spark35ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) + val schemaEvolutionUtils = new Spark32PlusParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) val fileFooter = if (enableVectorizedReader) { // When there are vectorized reads, we can avoid reading the footer twice by reading @@ -154,23 +154,14 @@ class Spark35ParquetReader(enableVectorizedReader: Boolean, } val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { - val vectorizedReader = if (schemaEvolutionUtils.shouldUseInternalSchema) { - schemaEvolutionUtils.buildVectorizedReader(convertTz, - datetimeRebaseSpec, - int96RebaseSpec, - enableOffHeapColumnVector, - taskContext, - capacity) - } else { - new VectorizedParquetRecordReader( - convertTz.orNull, - datetimeRebaseSpec.mode.toString, - datetimeRebaseSpec.timeZone, - int96RebaseSpec.mode.toString, - int96RebaseSpec.timeZone, - enableOffHeapColumnVector && taskContext.isDefined, - capacity) - } + val vectorizedReader = schemaEvolutionUtils.buildVectorizedReader( + convertTz.orNull, + datetimeRebaseSpec.mode.toString, + datetimeRebaseSpec.timeZone, + int96RebaseSpec.mode.toString, + int96RebaseSpec.timeZone, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) // SPARK-37089: We cannot register a task completion listener to close this iterator here // because downstream exec nodes have already registered their listeners. Since listeners // are executed in reverse order of registration, a listener registered here would close the diff --git a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35ParquetSchemaEvolutionUtils.scala b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35ParquetSchemaEvolutionUtils.scala deleted file mode 100644 index d08f3792a5cc..000000000000 --- a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35ParquetSchemaEvolutionUtils.scala +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.spark.sql.catalyst.expressions.AttributeReference -import org.apache.spark.sql.catalyst.types.DataTypeUtils -import org.apache.spark.sql.types.StructType - -class Spark35ParquetSchemaEvolutionUtils(sharedConf: Configuration, - filePath: Path, - requiredSchema: StructType, - partitionSchema: StructType) extends - Spark32PlusParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema){ - override protected def toAttributes(schema: StructType): Seq[AttributeReference] = { - DataTypeUtils.toAttributes(schema) - } -} From 8943bb4eaf741096203bed688905977d4bf59160 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Mon, 15 Apr 2024 13:27:52 -0400 Subject: [PATCH 63/86] spark 3.4 update legacy --- ...Spark34LegacyHoodieParquetFileFormat.scala | 388 +++--------------- .../parquet/Spark34ParquetReader.scala | 28 +- .../Spark34ParquetSchemaEvolutionUtils.scala | 35 -- 3 files changed, 71 insertions(+), 380 deletions(-) delete mode 100644 hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34ParquetSchemaEvolutionUtils.scala diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala index a1cfbb96212b..eb603f1f035f 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala @@ -21,15 +21,6 @@ import org.apache.hadoop.conf.Configuration 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 -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 @@ -37,19 +28,14 @@ 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.codegen.GenerateUnsafeProjection -import org.apache.spark.sql.catalyst.expressions.{Cast, JoinedRow} +import org.apache.spark.sql.catalyst.expressions.JoinedRow import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.WholeStageCodegenExec -import org.apache.spark.sql.execution.datasources.parquet.Spark34LegacyHoodieParquetFileFormat._ -import org.apache.spark.sql.execution.datasources.{DataSourceUtils, PartitionedFile, RecordReaderIterator} +import org.apache.spark.sql.execution.datasources.{DataSourceUtils, FileFormat, 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.sql.types.StructType import org.apache.spark.util.SerializableConfiguration -import scala.collection.convert.ImplicitConversions.`collection AsScalaIterable` - /** * This class is an extension of [[ParquetFileFormat]] overriding Spark-specific behavior * that's not possible to customize in any other way @@ -62,15 +48,6 @@ import scala.collection.convert.ImplicitConversions.`collection AsScalaIterable` */ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValues: Boolean) extends ParquetFileFormat { - def supportsColumnar(sparkSession: SparkSession, schema: StructType): Boolean = { - val conf = sparkSession.sessionState.conf - // Only output columnar if there is WSCG to read it. - val requiredWholeStageCodegenSettings = - conf.wholeStageEnabled && !WholeStageCodegenExec.isTooManyFields(conf, schema) - requiredWholeStageCodegenSettings && - supportBatch(sparkSession, schema) - } - override def buildReaderWithPartitionValues(sparkSession: SparkSession, dataSchema: StructType, partitionSchema: StructType, @@ -104,23 +81,12 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu hadoopConf.setBoolean( SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, sparkSession.sessionState.conf.isParquetINT96AsTimestamp) - // Using string value of this conf to preserve compatibility across spark versions. + hadoopConf.setBoolean( + SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key, + sparkSession.sessionState.conf.parquetInferTimestampNTZEnabled) hadoopConf.setBoolean( SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, - sparkSession.sessionState.conf.getConfString( - SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, - SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.defaultValueString).toBoolean - ) - hadoopConf.setBoolean(SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key, sparkSession.sessionState.conf.parquetInferTimestampNTZEnabled) - hadoopConf.setBoolean(SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, sparkSession.sessionState.conf.legacyParquetNanosAsLong) - val internalSchemaStr = hadoopConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA) - // For Spark DataSource v1, there's no Physical Plan projection/schema pruning w/in Spark itself, - // therefore it's safe to do schema projection here - if (!isNullOrEmpty(internalSchemaStr)) { - val prunedInternalSchemaStr = - pruneInternalSchema(internalSchemaStr, requiredSchema) - hadoopConf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, prunedInternalSchemaStr) - } + sparkSession.sessionState.conf.legacyParquetNanosAsLong) val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) @@ -131,7 +97,8 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val resultSchema = StructType(partitionSchema.fields ++ requiredSchema.fields) val sqlConf = sparkSession.sessionState.conf val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled - val enableVectorizedReader: Boolean = supportBatch(sparkSession, resultSchema) + val enableVectorizedReader: Boolean = + ParquetUtils.isBatchReadSupportedForSchema(sqlConf, resultSchema) val enableRecordFilter: Boolean = sqlConf.parquetRecordFilterEnabled val timestampConversion: Boolean = sqlConf.isParquetINT96TimestampConversion val capacity = sqlConf.parquetVectorizedReaderBatchSize @@ -139,7 +106,7 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val pushDownDate = sqlConf.parquetFilterPushDownDate val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal - val pushDownStringStartWith = sqlConf.parquetFilterPushDownStringPredicate + val pushDownStringPredicate = sqlConf.parquetFilterPushDownStringPredicate val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold val isCaseSensitive = sqlConf.caseSensitiveAnalysis val parquetOptions = new ParquetOptions(options, sparkSession.sessionState.conf) @@ -149,7 +116,18 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu // Should always be set by FileSourceScanExec creating this. // Check conf before checking option, to allow working around an issue by changing conf. val returningBatch = sparkSession.sessionState.conf.parquetVectorizedReaderEnabled && - supportsColumnar(sparkSession, resultSchema).toString.equals("true") + options.get(FileFormat.OPTION_RETURNING_BATCH) + .getOrElse { + throw new IllegalArgumentException( + "OPTION_RETURNING_BATCH should always be set for ParquetFileFormat. " + + "To workaround this issue, set spark.sql.parquet.enableVectorizedReader=false.") + } + .equals("true") + if (returningBatch) { + // If the passed option said that we are to return batches, we need to also be able to + // do this based on config and resultSchema. + assert(supportBatch(sparkSession, resultSchema)) + } (file: PartitionedFile) => { @@ -160,61 +138,30 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val sharedConf = broadcastedHadoopConf.value.value - // Fetch internal schema - val internalSchemaStr = sharedConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA) - // Internal schema has to be pruned at this point - val querySchemaOption = SerDeHelper.fromJson(internalSchemaStr) - - var shouldUseInternalSchema = !isNullOrEmpty(internalSchemaStr) && querySchemaOption.isPresent - - val tablePath = sharedConf.get(SparkInternalSchemaConverter.HOODIE_TABLE_PATH) - val fileSchema = if (shouldUseInternalSchema) { - val commitInstantTime = FSUtils.getCommitTime(filePath.getName).toLong; - val validCommits = sharedConf.get(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST) - InternalSchemaCache.getInternalSchemaByVersionId(commitInstantTime, tablePath, sharedConf, if (validCommits == null) "" else validCommits) - } else { - null - } + val schemaEvolutionUtils = new Spark32PlusParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) lazy val footerFileMetaData = ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData + val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( + footerFileMetaData.getKeyValueMetaData.get, + datetimeRebaseModeInRead) // Try to push down filters when filter push-down is enabled. val pushed = if (enableParquetFilterPushDown) { val parquetSchema = footerFileMetaData.getSchema - val parquetFilters = if (HoodieSparkUtils.gteqSpark3_2_1) { - // NOTE: Below code could only be compiled against >= Spark 3.2.1, - // and unfortunately won't compile against Spark 3.2.0 - // However this code is runtime-compatible w/ both Spark 3.2.0 and >= Spark 3.2.1 - val datetimeRebaseSpec = - DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) - new ParquetFilters( - parquetSchema, - pushDownDate, - pushDownTimestamp, - pushDownDecimal, - pushDownStringStartWith, - pushDownInFilterThreshold, - isCaseSensitive, - datetimeRebaseSpec) - } else { - // Spark 3.2.0 - val datetimeRebaseMode = - Spark34DataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) - createParquetFilters( - parquetSchema, - pushDownDate, - pushDownTimestamp, - pushDownDecimal, - pushDownStringStartWith, - pushDownInFilterThreshold, - isCaseSensitive, - datetimeRebaseMode) - } - filters.map(rebuildFilterFromParquet(_, fileSchema, querySchemaOption.orElse(null))) + val parquetFilters = new ParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringPredicate, + pushDownInFilterThreshold, + isCaseSensitive, + datetimeRebaseSpec) + filters.map(schemaEvolutionUtils.rebuildFilterFromParquet) // 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) + .flatMap(parquetFilters.createFilter(_)) .reduceOption(FilterApi.and) } else { None @@ -235,35 +182,13 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu None } - val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) - - // Clone new conf - val hadoopAttemptConf = new Configuration(broadcastedHadoopConf.value.value) - val typeChangeInfos: java.util.Map[Integer, Pair[DataType, DataType]] = if (shouldUseInternalSchema) { - val mergedInternalSchema = new InternalSchemaMerger(fileSchema, querySchemaOption.get(), true, true).mergeSchema() - val mergedSchema = SparkInternalSchemaConverter.constructSparkSchemaFromInternalSchema(mergedInternalSchema) - - hadoopAttemptConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, mergedSchema.json) - - SparkInternalSchemaConverter.collectTypeChangedCols(querySchemaOption.get(), mergedInternalSchema) - } else { - val (implicitTypeChangeInfo, sparkRequestSchema) = HoodieParquetFileFormatHelper.buildImplicitSchemaChangeInfo(hadoopAttemptConf, footerFileMetaData, requiredSchema) - if (!implicitTypeChangeInfo.isEmpty) { - shouldUseInternalSchema = true - hadoopAttemptConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, sparkRequestSchema.json) - } - implicitTypeChangeInfo - } - - if (enableVectorizedReader && shouldUseInternalSchema && - !typeChangeInfos.values().forall(_.getLeft.isInstanceOf[AtomicType])) { - throw new IllegalArgumentException( - "Nested types with type changes(implicit or explicit) cannot be read in vectorized mode. " + - "To workaround this issue, set spark.sql.parquet.enableVectorizedReader=false.") - } + val int96RebaseSpec = DataSourceUtils.int96RebaseSpec( + footerFileMetaData.getKeyValueMetaData.get, + int96RebaseModeInRead) + val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val hadoopAttemptContext = - new TaskAttemptContextImpl(hadoopAttemptConf, attemptId) + new TaskAttemptContextImpl(schemaEvolutionUtils.getHadoopConfClone(footerFileMetaData, enableVectorizedReader), attemptId) // Try to push down filters when filter push-down is enabled. // Notice: This push-down is RowGroups level, not individual records. @@ -272,51 +197,14 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu } val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { - val vectorizedReader = - if (shouldUseInternalSchema) { - val int96RebaseSpec = - DataSourceUtils.int96RebaseSpec(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) - val datetimeRebaseSpec = - DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) - new Spark32PlusHoodieVectorizedParquetRecordReader( - convertTz.orNull, - datetimeRebaseSpec.mode.toString, - datetimeRebaseSpec.timeZone, - int96RebaseSpec.mode.toString, - int96RebaseSpec.timeZone, - enableOffHeapColumnVector && taskContext.isDefined, - capacity, - typeChangeInfos) - } else if (HoodieSparkUtils.gteqSpark3_2_1) { - // NOTE: Below code could only be compiled against >= Spark 3.2.1, - // and unfortunately won't compile against Spark 3.2.0 - // However this code is runtime-compatible w/ both Spark 3.2.0 and >= Spark 3.2.1 - val int96RebaseSpec = - DataSourceUtils.int96RebaseSpec(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) - val datetimeRebaseSpec = - DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) - new VectorizedParquetRecordReader( - convertTz.orNull, - datetimeRebaseSpec.mode.toString, - datetimeRebaseSpec.timeZone, - int96RebaseSpec.mode.toString, - int96RebaseSpec.timeZone, - enableOffHeapColumnVector && taskContext.isDefined, - capacity) - } else { - // Spark 3.2.0 - val datetimeRebaseMode = - Spark34DataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) - val int96RebaseMode = - Spark34DataSourceUtils.int96RebaseMode(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) - createVectorizedParquetRecordReader( - convertTz.orNull, - datetimeRebaseMode.toString, - int96RebaseMode.toString, - enableOffHeapColumnVector && taskContext.isDefined, - capacity) - } - + val vectorizedReader = schemaEvolutionUtils.buildVectorizedReader( + convertTz.orNull, + datetimeRebaseSpec.mode.toString, + datetimeRebaseSpec.timeZone, + int96RebaseSpec.mode.toString, + int96RebaseSpec.timeZone, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) // SPARK-37089: We cannot register a task completion listener to close this iterator here // because downstream exec nodes have already registered their listeners. Since listeners // are executed in reverse order of registration, a listener registered here would close the @@ -327,16 +215,7 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val iter = new RecordReaderIterator(vectorizedReader) try { vectorizedReader.initialize(split, hadoopAttemptContext) - - // NOTE: We're making appending of the partitioned values to the rows read from the - // data file configurable - if (shouldAppendPartitionValues) { - logDebug(s"Appending $partitionSchema ${file.partitionValues}") - vectorizedReader.initBatch(partitionSchema, file.partitionValues) - } else { - vectorizedReader.initBatch(StructType(Nil), InternalRow.empty) - } - + vectorizedReader.initBatch(partitionSchema, file.partitionValues) if (returningBatch) { vectorizedReader.enableReturningBatches() } @@ -351,63 +230,26 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu throw e } } else { - logDebug(s"Falling back to parquet-mr") - val readSupport = if (HoodieSparkUtils.gteqSpark3_2_1) { - // ParquetRecordReader returns InternalRow - // NOTE: Below code could only be compiled against >= Spark 3.2.1, - // and unfortunately won't compile against Spark 3.2.0 - // However this code is runtime-compatible w/ both Spark 3.2.0 and >= Spark 3.2.1 - val int96RebaseSpec = - DataSourceUtils.int96RebaseSpec(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) - val datetimeRebaseSpec = - DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) - new ParquetReadSupport( - convertTz, - enableVectorizedReader = false, - datetimeRebaseSpec, - int96RebaseSpec) - } else { - val datetimeRebaseMode = - Spark34DataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) - val int96RebaseMode = - Spark34DataSourceUtils.int96RebaseMode(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) - createParquetReadSupport( - convertTz, - /* enableVectorizedReader = */ false, - datetimeRebaseMode, - int96RebaseMode) - } - + // ParquetRecordReader returns InternalRow + val readSupport = new ParquetReadSupport( + convertTz, + enableVectorizedReader = false, + datetimeRebaseSpec, + int96RebaseSpec) val reader = if (pushed.isDefined && enableRecordFilter) { val parquetFilter = FilterCompat.get(pushed.get, null) new ParquetRecordReader[InternalRow](readSupport, parquetFilter) } else { new ParquetRecordReader[InternalRow](readSupport) } - val iter = new RecordReaderIterator[InternalRow](reader) + val readerWithRowIndexes = ParquetRowIndexUtil.addRowIndexToRecordReaderIfNeeded(reader, + requiredSchema) + val iter = new RecordReaderIterator[InternalRow](readerWithRowIndexes) try { - reader.initialize(split, hadoopAttemptContext) + readerWithRowIndexes.initialize(split, hadoopAttemptContext) val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes - val unsafeProjection = if (typeChangeInfos.isEmpty) { - GenerateUnsafeProjection.generate(fullSchema, fullSchema) - } else { - // find type changed. - val newFullSchema = new StructType(requiredSchema.fields.zipWithIndex.map { case (f, i) => - if (typeChangeInfos.containsKey(i)) { - StructField(f.name, typeChangeInfos.get(i).getRight, f.nullable, f.metadata) - } else f - }).toAttributes ++ partitionSchema.toAttributes - val castSchema = newFullSchema.zipWithIndex.map { case (attr, i) => - if (typeChangeInfos.containsKey(i)) { - val srcType = typeChangeInfos.get(i).getRight - val dstType = typeChangeInfos.get(i).getLeft - val needTimeZone = Cast.needsTimeZone(srcType, dstType) - Cast(attr, dstType, if (needTimeZone) timeZoneId else None) - } else attr - } - GenerateUnsafeProjection.generate(castSchema, newFullSchema) - } + val unsafeProjection = schemaEvolutionUtils.generateUnsafeProjection(fullSchema, timeZoneId) // NOTE: We're making appending of the partitioned values to the rows read from the // data file configurable @@ -429,109 +271,3 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu } } } - -object Spark34LegacyHoodieParquetFileFormat { - - /** - * NOTE: This method is specific to Spark 3.2.0 - */ - private def createParquetFilters(args: Any*): ParquetFilters = { - // NOTE: ParquetFilters ctor args contain Scala enum, therefore we can't look it - // up by arg types, and have to instead rely on the number of args based on individual class; - // the ctor order is not guaranteed - val ctor = classOf[ParquetFilters].getConstructors.maxBy(_.getParameterCount) - ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*) - .asInstanceOf[ParquetFilters] - } - - /** - * NOTE: This method is specific to Spark 3.2.0 - */ - private def createParquetReadSupport(args: Any*): ParquetReadSupport = { - // NOTE: ParquetReadSupport ctor args contain Scala enum, therefore we can't look it - // up by arg types, and have to instead rely on the number of args based on individual class; - // the ctor order is not guaranteed - val ctor = classOf[ParquetReadSupport].getConstructors.maxBy(_.getParameterCount) - ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*) - .asInstanceOf[ParquetReadSupport] - } - - /** - * NOTE: This method is specific to Spark 3.2.0 - */ - private def createVectorizedParquetRecordReader(args: Any*): VectorizedParquetRecordReader = { - // NOTE: ParquetReadSupport ctor args contain Scala enum, therefore we can't look it - // up by arg types, and have to instead rely on the number of args based on individual class; - // the ctor order is not guaranteed - val ctor = classOf[VectorizedParquetRecordReader].getConstructors.maxBy(_.getParameterCount) - ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*) - .asInstanceOf[VectorizedParquetRecordReader] - } - - def pruneInternalSchema(internalSchemaStr: String, requiredSchema: StructType): String = { - val querySchemaOption = SerDeHelper.fromJson(internalSchemaStr) - if (querySchemaOption.isPresent && requiredSchema.nonEmpty) { - val prunedSchema = SparkInternalSchemaConverter.convertAndPruneStructTypeToInternalSchema(requiredSchema, querySchemaOption.get()) - SerDeHelper.toJson(prunedSchema) - } else { - internalSchemaStr - } - } - - private def rebuildFilterFromParquet(oldFilter: Filter, fileSchema: InternalSchema, querySchema: InternalSchema): Filter = { - if (fileSchema == null || querySchema == null) { - oldFilter - } else { - oldFilter match { - case eq: EqualTo => - val newAttribute = InternalSchemaUtils.reBuildFilterName(eq.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else eq.copy(attribute = newAttribute) - case eqs: EqualNullSafe => - val newAttribute = InternalSchemaUtils.reBuildFilterName(eqs.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else eqs.copy(attribute = newAttribute) - case gt: GreaterThan => - val newAttribute = InternalSchemaUtils.reBuildFilterName(gt.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else gt.copy(attribute = newAttribute) - case gtr: GreaterThanOrEqual => - val newAttribute = InternalSchemaUtils.reBuildFilterName(gtr.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else gtr.copy(attribute = newAttribute) - case lt: LessThan => - val newAttribute = InternalSchemaUtils.reBuildFilterName(lt.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else lt.copy(attribute = newAttribute) - case lte: LessThanOrEqual => - val newAttribute = InternalSchemaUtils.reBuildFilterName(lte.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else lte.copy(attribute = newAttribute) - case i: In => - val newAttribute = InternalSchemaUtils.reBuildFilterName(i.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else i.copy(attribute = newAttribute) - case isn: IsNull => - val newAttribute = InternalSchemaUtils.reBuildFilterName(isn.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else isn.copy(attribute = newAttribute) - case isnn: IsNotNull => - val newAttribute = InternalSchemaUtils.reBuildFilterName(isnn.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else isnn.copy(attribute = newAttribute) - case And(left, right) => - And(rebuildFilterFromParquet(left, fileSchema, querySchema), rebuildFilterFromParquet(right, fileSchema, querySchema)) - case Or(left, right) => - Or(rebuildFilterFromParquet(left, fileSchema, querySchema), rebuildFilterFromParquet(right, fileSchema, querySchema)) - case Not(child) => - Not(rebuildFilterFromParquet(child, fileSchema, querySchema)) - case ssw: StringStartsWith => - val newAttribute = InternalSchemaUtils.reBuildFilterName(ssw.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else ssw.copy(attribute = newAttribute) - case ses: StringEndsWith => - val newAttribute = InternalSchemaUtils.reBuildFilterName(ses.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else ses.copy(attribute = newAttribute) - case sc: StringContains => - val newAttribute = InternalSchemaUtils.reBuildFilterName(sc.attribute, fileSchema, querySchema) - if (newAttribute.isEmpty) AlwaysTrue else sc.copy(attribute = newAttribute) - case AlwaysTrue => - AlwaysTrue - case AlwaysFalse => - AlwaysFalse - case _ => - AlwaysTrue - } - } - } -} diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34ParquetReader.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34ParquetReader.scala index e02cb7280eae..77e90b82849b 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34ParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34ParquetReader.scala @@ -87,7 +87,7 @@ class Spark34ParquetReader(enableVectorizedReader: Boolean, val filePath = file.toPath val split = new FileSplit(filePath, file.start, file.length, Array.empty[String]) - val schemaEvolutionUtils = new Spark34ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) + val schemaEvolutionUtils = new Spark32PlusParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) lazy val footerFileMetaData = ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData @@ -146,24 +146,14 @@ class Spark34ParquetReader(enableVectorizedReader: Boolean, } val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { - val vectorizedReader = if (schemaEvolutionUtils.shouldUseInternalSchema) { - schemaEvolutionUtils.buildVectorizedReader( - convertTz, - datetimeRebaseSpec, - int96RebaseSpec, - enableOffHeapColumnVector, - taskContext, - capacity) - } else { - new VectorizedParquetRecordReader( - convertTz.orNull, - datetimeRebaseSpec.mode.toString, - datetimeRebaseSpec.timeZone, - int96RebaseSpec.mode.toString, - int96RebaseSpec.timeZone, - enableOffHeapColumnVector && taskContext.isDefined, - capacity) - } + val vectorizedReader = schemaEvolutionUtils.buildVectorizedReader( + convertTz.orNull, + datetimeRebaseSpec.mode.toString, + datetimeRebaseSpec.timeZone, + int96RebaseSpec.mode.toString, + int96RebaseSpec.timeZone, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) // SPARK-37089: We cannot register a task completion listener to close this iterator here // because downstream exec nodes have already registered their listeners. Since listeners // are executed in reverse order of registration, a listener registered here would close the diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34ParquetSchemaEvolutionUtils.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34ParquetSchemaEvolutionUtils.scala deleted file mode 100644 index 8854e5d54272..000000000000 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34ParquetSchemaEvolutionUtils.scala +++ /dev/null @@ -1,35 +0,0 @@ -/* - * 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.spark.sql.catalyst.expressions.AttributeReference -import org.apache.spark.sql.types.StructType - -class Spark34ParquetSchemaEvolutionUtils(sharedConf: Configuration, - filePath: Path, - requiredSchema: StructType, - partitionSchema: StructType) extends - Spark32PlusParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema){ - override protected def toAttributes(schema: StructType): Seq[AttributeReference] = { - schema.toAttributes - } -} From 4c3242159414786c927f13b83013b045c517ff65 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Mon, 15 Apr 2024 14:06:13 -0400 Subject: [PATCH 64/86] make changes to spark 3.3 and restore legacy for 3.4 and 3.5 --- .../parquet/Spark33ParquetReader.scala | 36 +- .../Spark33ParquetSchemaEvolutionUtils.scala | 35 -- ...Spark34LegacyHoodieParquetFileFormat.scala | 388 ++++++++++++++--- ...Spark35LegacyHoodieParquetFileFormat.scala | 405 ++++++++++++++---- 4 files changed, 672 insertions(+), 192 deletions(-) delete mode 100644 hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33ParquetSchemaEvolutionUtils.scala diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33ParquetReader.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33ParquetReader.scala index 7ebc23aee8cf..8d882d7945d3 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33ParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33ParquetReader.scala @@ -82,16 +82,16 @@ class Spark33ParquetReader(enableVectorizedReader: Boolean, * @return iterator of rows read from the file output type says [[InternalRow]] but could be [[ColumnarBatch]] */ protected def doRead(file: PartitionedFile, - requiredSchema: StructType, - partitionSchema: StructType, - filters: Seq[Filter], - sharedConf: Configuration): Iterator[InternalRow] = { + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[Filter], + sharedConf: Configuration): Iterator[InternalRow] = { assert(file.partitionValues.numFields == partitionSchema.size) val filePath = new Path(new URI(file.filePath)) val split = new FileSplit(filePath, file.start, file.length, Array.empty[String]) - val schemaEvolutionUtils = new Spark33ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) + val schemaEvolutionUtils = new Spark32PlusParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) lazy val footerFileMetaData = ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData @@ -150,24 +150,14 @@ class Spark33ParquetReader(enableVectorizedReader: Boolean, } val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { - val vectorizedReader = if (schemaEvolutionUtils.shouldUseInternalSchema) { - schemaEvolutionUtils.buildVectorizedReader( - convertTz, - datetimeRebaseSpec, - int96RebaseSpec, - enableOffHeapColumnVector, - taskContext, - capacity) - } else { - new VectorizedParquetRecordReader( - convertTz.orNull, - datetimeRebaseSpec.mode.toString, - datetimeRebaseSpec.timeZone, - int96RebaseSpec.mode.toString, - int96RebaseSpec.timeZone, - enableOffHeapColumnVector && taskContext.isDefined, - capacity) - } + val vectorizedReader = schemaEvolutionUtils.buildVectorizedReader( + convertTz.orNull, + datetimeRebaseSpec.mode.toString, + datetimeRebaseSpec.timeZone, + int96RebaseSpec.mode.toString, + int96RebaseSpec.timeZone, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) // SPARK-37089: We cannot register a task completion listener to close this iterator here // because downstream exec nodes have already registered their listeners. Since listeners // are executed in reverse order of registration, a listener registered here would close the diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33ParquetSchemaEvolutionUtils.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33ParquetSchemaEvolutionUtils.scala deleted file mode 100644 index b14a3bf99847..000000000000 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33ParquetSchemaEvolutionUtils.scala +++ /dev/null @@ -1,35 +0,0 @@ -/* - * 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.spark.sql.catalyst.expressions.AttributeReference -import org.apache.spark.sql.types.StructType - -class Spark33ParquetSchemaEvolutionUtils(sharedConf: Configuration, - filePath: Path, - requiredSchema: StructType, - partitionSchema: StructType) extends - Spark32PlusParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema){ - override protected def toAttributes(schema: StructType): Seq[AttributeReference] = { - schema.toAttributes - } -} diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala index eb603f1f035f..a1cfbb96212b 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala @@ -21,6 +21,15 @@ import org.apache.hadoop.conf.Configuration 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 +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 @@ -28,14 +37,19 @@ 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.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.{DataSourceUtils, FileFormat, PartitionedFile, RecordReaderIterator} +import org.apache.spark.sql.execution.WholeStageCodegenExec +import org.apache.spark.sql.execution.datasources.parquet.Spark34LegacyHoodieParquetFileFormat._ +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.StructType +import org.apache.spark.sql.types.{AtomicType, DataType, StructField, StructType} import org.apache.spark.util.SerializableConfiguration +import scala.collection.convert.ImplicitConversions.`collection AsScalaIterable` + /** * This class is an extension of [[ParquetFileFormat]] overriding Spark-specific behavior * that's not possible to customize in any other way @@ -48,6 +62,15 @@ import org.apache.spark.util.SerializableConfiguration */ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValues: Boolean) extends ParquetFileFormat { + def supportsColumnar(sparkSession: SparkSession, schema: StructType): Boolean = { + val conf = sparkSession.sessionState.conf + // Only output columnar if there is WSCG to read it. + val requiredWholeStageCodegenSettings = + conf.wholeStageEnabled && !WholeStageCodegenExec.isTooManyFields(conf, schema) + requiredWholeStageCodegenSettings && + supportBatch(sparkSession, schema) + } + override def buildReaderWithPartitionValues(sparkSession: SparkSession, dataSchema: StructType, partitionSchema: StructType, @@ -81,12 +104,23 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu hadoopConf.setBoolean( SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, sparkSession.sessionState.conf.isParquetINT96AsTimestamp) - hadoopConf.setBoolean( - SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key, - sparkSession.sessionState.conf.parquetInferTimestampNTZEnabled) + // Using string value of this conf to preserve compatibility across spark versions. hadoopConf.setBoolean( SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, - sparkSession.sessionState.conf.legacyParquetNanosAsLong) + sparkSession.sessionState.conf.getConfString( + SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, + SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.defaultValueString).toBoolean + ) + hadoopConf.setBoolean(SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key, sparkSession.sessionState.conf.parquetInferTimestampNTZEnabled) + hadoopConf.setBoolean(SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, sparkSession.sessionState.conf.legacyParquetNanosAsLong) + val internalSchemaStr = hadoopConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA) + // For Spark DataSource v1, there's no Physical Plan projection/schema pruning w/in Spark itself, + // therefore it's safe to do schema projection here + if (!isNullOrEmpty(internalSchemaStr)) { + val prunedInternalSchemaStr = + pruneInternalSchema(internalSchemaStr, requiredSchema) + hadoopConf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, prunedInternalSchemaStr) + } val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) @@ -97,8 +131,7 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val resultSchema = StructType(partitionSchema.fields ++ requiredSchema.fields) val sqlConf = sparkSession.sessionState.conf val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled - val enableVectorizedReader: Boolean = - ParquetUtils.isBatchReadSupportedForSchema(sqlConf, resultSchema) + val enableVectorizedReader: Boolean = supportBatch(sparkSession, resultSchema) val enableRecordFilter: Boolean = sqlConf.parquetRecordFilterEnabled val timestampConversion: Boolean = sqlConf.isParquetINT96TimestampConversion val capacity = sqlConf.parquetVectorizedReaderBatchSize @@ -106,7 +139,7 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val pushDownDate = sqlConf.parquetFilterPushDownDate val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal - val pushDownStringPredicate = sqlConf.parquetFilterPushDownStringPredicate + val pushDownStringStartWith = sqlConf.parquetFilterPushDownStringPredicate val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold val isCaseSensitive = sqlConf.caseSensitiveAnalysis val parquetOptions = new ParquetOptions(options, sparkSession.sessionState.conf) @@ -116,18 +149,7 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu // Should always be set by FileSourceScanExec creating this. // Check conf before checking option, to allow working around an issue by changing conf. val returningBatch = sparkSession.sessionState.conf.parquetVectorizedReaderEnabled && - options.get(FileFormat.OPTION_RETURNING_BATCH) - .getOrElse { - throw new IllegalArgumentException( - "OPTION_RETURNING_BATCH should always be set for ParquetFileFormat. " + - "To workaround this issue, set spark.sql.parquet.enableVectorizedReader=false.") - } - .equals("true") - if (returningBatch) { - // If the passed option said that we are to return batches, we need to also be able to - // do this based on config and resultSchema. - assert(supportBatch(sparkSession, resultSchema)) - } + supportsColumnar(sparkSession, resultSchema).toString.equals("true") (file: PartitionedFile) => { @@ -138,30 +160,61 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val sharedConf = broadcastedHadoopConf.value.value - val schemaEvolutionUtils = new Spark32PlusParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) + // Fetch internal schema + val internalSchemaStr = sharedConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA) + // Internal schema has to be pruned at this point + val querySchemaOption = SerDeHelper.fromJson(internalSchemaStr) + + var shouldUseInternalSchema = !isNullOrEmpty(internalSchemaStr) && querySchemaOption.isPresent + + val tablePath = sharedConf.get(SparkInternalSchemaConverter.HOODIE_TABLE_PATH) + val fileSchema = if (shouldUseInternalSchema) { + val commitInstantTime = FSUtils.getCommitTime(filePath.getName).toLong; + val validCommits = sharedConf.get(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST) + InternalSchemaCache.getInternalSchemaByVersionId(commitInstantTime, tablePath, sharedConf, if (validCommits == null) "" else validCommits) + } else { + null + } lazy val footerFileMetaData = ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData - val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( - footerFileMetaData.getKeyValueMetaData.get, - datetimeRebaseModeInRead) // Try to push down filters when filter push-down is enabled. val pushed = if (enableParquetFilterPushDown) { val parquetSchema = footerFileMetaData.getSchema - val parquetFilters = new ParquetFilters( - parquetSchema, - pushDownDate, - pushDownTimestamp, - pushDownDecimal, - pushDownStringPredicate, - pushDownInFilterThreshold, - isCaseSensitive, - datetimeRebaseSpec) - filters.map(schemaEvolutionUtils.rebuildFilterFromParquet) + val parquetFilters = if (HoodieSparkUtils.gteqSpark3_2_1) { + // NOTE: Below code could only be compiled against >= Spark 3.2.1, + // and unfortunately won't compile against Spark 3.2.0 + // However this code is runtime-compatible w/ both Spark 3.2.0 and >= Spark 3.2.1 + val datetimeRebaseSpec = + DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) + new ParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringStartWith, + pushDownInFilterThreshold, + isCaseSensitive, + datetimeRebaseSpec) + } else { + // Spark 3.2.0 + val datetimeRebaseMode = + Spark34DataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) + createParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringStartWith, + pushDownInFilterThreshold, + isCaseSensitive, + datetimeRebaseMode) + } + filters.map(rebuildFilterFromParquet(_, fileSchema, querySchemaOption.orElse(null))) // 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(_)) + .flatMap(parquetFilters.createFilter) .reduceOption(FilterApi.and) } else { None @@ -182,13 +235,35 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu None } - val int96RebaseSpec = DataSourceUtils.int96RebaseSpec( - footerFileMetaData.getKeyValueMetaData.get, - int96RebaseModeInRead) - val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) + + // Clone new conf + val hadoopAttemptConf = new Configuration(broadcastedHadoopConf.value.value) + val typeChangeInfos: java.util.Map[Integer, Pair[DataType, DataType]] = if (shouldUseInternalSchema) { + val mergedInternalSchema = new InternalSchemaMerger(fileSchema, querySchemaOption.get(), true, true).mergeSchema() + val mergedSchema = SparkInternalSchemaConverter.constructSparkSchemaFromInternalSchema(mergedInternalSchema) + + hadoopAttemptConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, mergedSchema.json) + + SparkInternalSchemaConverter.collectTypeChangedCols(querySchemaOption.get(), mergedInternalSchema) + } else { + val (implicitTypeChangeInfo, sparkRequestSchema) = HoodieParquetFileFormatHelper.buildImplicitSchemaChangeInfo(hadoopAttemptConf, footerFileMetaData, requiredSchema) + if (!implicitTypeChangeInfo.isEmpty) { + shouldUseInternalSchema = true + hadoopAttemptConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, sparkRequestSchema.json) + } + implicitTypeChangeInfo + } + + if (enableVectorizedReader && shouldUseInternalSchema && + !typeChangeInfos.values().forall(_.getLeft.isInstanceOf[AtomicType])) { + throw new IllegalArgumentException( + "Nested types with type changes(implicit or explicit) cannot be read in vectorized mode. " + + "To workaround this issue, set spark.sql.parquet.enableVectorizedReader=false.") + } + val hadoopAttemptContext = - new TaskAttemptContextImpl(schemaEvolutionUtils.getHadoopConfClone(footerFileMetaData, enableVectorizedReader), attemptId) + new TaskAttemptContextImpl(hadoopAttemptConf, attemptId) // Try to push down filters when filter push-down is enabled. // Notice: This push-down is RowGroups level, not individual records. @@ -197,14 +272,51 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu } val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { - val vectorizedReader = schemaEvolutionUtils.buildVectorizedReader( - convertTz.orNull, - datetimeRebaseSpec.mode.toString, - datetimeRebaseSpec.timeZone, - int96RebaseSpec.mode.toString, - int96RebaseSpec.timeZone, - enableOffHeapColumnVector && taskContext.isDefined, - capacity) + val vectorizedReader = + if (shouldUseInternalSchema) { + val int96RebaseSpec = + DataSourceUtils.int96RebaseSpec(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) + val datetimeRebaseSpec = + DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) + new Spark32PlusHoodieVectorizedParquetRecordReader( + convertTz.orNull, + datetimeRebaseSpec.mode.toString, + datetimeRebaseSpec.timeZone, + int96RebaseSpec.mode.toString, + int96RebaseSpec.timeZone, + enableOffHeapColumnVector && taskContext.isDefined, + capacity, + typeChangeInfos) + } else if (HoodieSparkUtils.gteqSpark3_2_1) { + // NOTE: Below code could only be compiled against >= Spark 3.2.1, + // and unfortunately won't compile against Spark 3.2.0 + // However this code is runtime-compatible w/ both Spark 3.2.0 and >= Spark 3.2.1 + val int96RebaseSpec = + DataSourceUtils.int96RebaseSpec(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) + val datetimeRebaseSpec = + DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) + new VectorizedParquetRecordReader( + convertTz.orNull, + datetimeRebaseSpec.mode.toString, + datetimeRebaseSpec.timeZone, + int96RebaseSpec.mode.toString, + int96RebaseSpec.timeZone, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) + } else { + // Spark 3.2.0 + val datetimeRebaseMode = + Spark34DataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) + val int96RebaseMode = + Spark34DataSourceUtils.int96RebaseMode(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) + createVectorizedParquetRecordReader( + convertTz.orNull, + datetimeRebaseMode.toString, + int96RebaseMode.toString, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) + } + // SPARK-37089: We cannot register a task completion listener to close this iterator here // because downstream exec nodes have already registered their listeners. Since listeners // are executed in reverse order of registration, a listener registered here would close the @@ -215,7 +327,16 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val iter = new RecordReaderIterator(vectorizedReader) try { vectorizedReader.initialize(split, hadoopAttemptContext) - vectorizedReader.initBatch(partitionSchema, file.partitionValues) + + // NOTE: We're making appending of the partitioned values to the rows read from the + // data file configurable + if (shouldAppendPartitionValues) { + logDebug(s"Appending $partitionSchema ${file.partitionValues}") + vectorizedReader.initBatch(partitionSchema, file.partitionValues) + } else { + vectorizedReader.initBatch(StructType(Nil), InternalRow.empty) + } + if (returningBatch) { vectorizedReader.enableReturningBatches() } @@ -230,26 +351,63 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu throw e } } else { - // ParquetRecordReader returns InternalRow - val readSupport = new ParquetReadSupport( - convertTz, - enableVectorizedReader = false, - datetimeRebaseSpec, - int96RebaseSpec) + logDebug(s"Falling back to parquet-mr") + val readSupport = if (HoodieSparkUtils.gteqSpark3_2_1) { + // ParquetRecordReader returns InternalRow + // NOTE: Below code could only be compiled against >= Spark 3.2.1, + // and unfortunately won't compile against Spark 3.2.0 + // However this code is runtime-compatible w/ both Spark 3.2.0 and >= Spark 3.2.1 + val int96RebaseSpec = + DataSourceUtils.int96RebaseSpec(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) + val datetimeRebaseSpec = + DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) + new ParquetReadSupport( + convertTz, + enableVectorizedReader = false, + datetimeRebaseSpec, + int96RebaseSpec) + } else { + val datetimeRebaseMode = + Spark34DataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) + val int96RebaseMode = + Spark34DataSourceUtils.int96RebaseMode(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) + createParquetReadSupport( + convertTz, + /* enableVectorizedReader = */ false, + datetimeRebaseMode, + int96RebaseMode) + } + val reader = if (pushed.isDefined && enableRecordFilter) { val parquetFilter = FilterCompat.get(pushed.get, null) new ParquetRecordReader[InternalRow](readSupport, parquetFilter) } else { new ParquetRecordReader[InternalRow](readSupport) } - val readerWithRowIndexes = ParquetRowIndexUtil.addRowIndexToRecordReaderIfNeeded(reader, - requiredSchema) - val iter = new RecordReaderIterator[InternalRow](readerWithRowIndexes) + val iter = new RecordReaderIterator[InternalRow](reader) try { - readerWithRowIndexes.initialize(split, hadoopAttemptContext) + reader.initialize(split, hadoopAttemptContext) val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes - val unsafeProjection = schemaEvolutionUtils.generateUnsafeProjection(fullSchema, timeZoneId) + val unsafeProjection = if (typeChangeInfos.isEmpty) { + GenerateUnsafeProjection.generate(fullSchema, fullSchema) + } else { + // find type changed. + val newFullSchema = new StructType(requiredSchema.fields.zipWithIndex.map { case (f, i) => + if (typeChangeInfos.containsKey(i)) { + StructField(f.name, typeChangeInfos.get(i).getRight, f.nullable, f.metadata) + } else f + }).toAttributes ++ partitionSchema.toAttributes + val castSchema = newFullSchema.zipWithIndex.map { case (attr, i) => + if (typeChangeInfos.containsKey(i)) { + val srcType = typeChangeInfos.get(i).getRight + val dstType = typeChangeInfos.get(i).getLeft + val needTimeZone = Cast.needsTimeZone(srcType, dstType) + Cast(attr, dstType, if (needTimeZone) timeZoneId else None) + } else attr + } + GenerateUnsafeProjection.generate(castSchema, newFullSchema) + } // NOTE: We're making appending of the partitioned values to the rows read from the // data file configurable @@ -271,3 +429,109 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu } } } + +object Spark34LegacyHoodieParquetFileFormat { + + /** + * NOTE: This method is specific to Spark 3.2.0 + */ + private def createParquetFilters(args: Any*): ParquetFilters = { + // NOTE: ParquetFilters ctor args contain Scala enum, therefore we can't look it + // up by arg types, and have to instead rely on the number of args based on individual class; + // the ctor order is not guaranteed + val ctor = classOf[ParquetFilters].getConstructors.maxBy(_.getParameterCount) + ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*) + .asInstanceOf[ParquetFilters] + } + + /** + * NOTE: This method is specific to Spark 3.2.0 + */ + private def createParquetReadSupport(args: Any*): ParquetReadSupport = { + // NOTE: ParquetReadSupport ctor args contain Scala enum, therefore we can't look it + // up by arg types, and have to instead rely on the number of args based on individual class; + // the ctor order is not guaranteed + val ctor = classOf[ParquetReadSupport].getConstructors.maxBy(_.getParameterCount) + ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*) + .asInstanceOf[ParquetReadSupport] + } + + /** + * NOTE: This method is specific to Spark 3.2.0 + */ + private def createVectorizedParquetRecordReader(args: Any*): VectorizedParquetRecordReader = { + // NOTE: ParquetReadSupport ctor args contain Scala enum, therefore we can't look it + // up by arg types, and have to instead rely on the number of args based on individual class; + // the ctor order is not guaranteed + val ctor = classOf[VectorizedParquetRecordReader].getConstructors.maxBy(_.getParameterCount) + ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*) + .asInstanceOf[VectorizedParquetRecordReader] + } + + def pruneInternalSchema(internalSchemaStr: String, requiredSchema: StructType): String = { + val querySchemaOption = SerDeHelper.fromJson(internalSchemaStr) + if (querySchemaOption.isPresent && requiredSchema.nonEmpty) { + val prunedSchema = SparkInternalSchemaConverter.convertAndPruneStructTypeToInternalSchema(requiredSchema, querySchemaOption.get()) + SerDeHelper.toJson(prunedSchema) + } else { + internalSchemaStr + } + } + + private def rebuildFilterFromParquet(oldFilter: Filter, fileSchema: InternalSchema, querySchema: InternalSchema): Filter = { + if (fileSchema == null || querySchema == null) { + oldFilter + } else { + oldFilter match { + case eq: EqualTo => + val newAttribute = InternalSchemaUtils.reBuildFilterName(eq.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else eq.copy(attribute = newAttribute) + case eqs: EqualNullSafe => + val newAttribute = InternalSchemaUtils.reBuildFilterName(eqs.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else eqs.copy(attribute = newAttribute) + case gt: GreaterThan => + val newAttribute = InternalSchemaUtils.reBuildFilterName(gt.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else gt.copy(attribute = newAttribute) + case gtr: GreaterThanOrEqual => + val newAttribute = InternalSchemaUtils.reBuildFilterName(gtr.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else gtr.copy(attribute = newAttribute) + case lt: LessThan => + val newAttribute = InternalSchemaUtils.reBuildFilterName(lt.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else lt.copy(attribute = newAttribute) + case lte: LessThanOrEqual => + val newAttribute = InternalSchemaUtils.reBuildFilterName(lte.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else lte.copy(attribute = newAttribute) + case i: In => + val newAttribute = InternalSchemaUtils.reBuildFilterName(i.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else i.copy(attribute = newAttribute) + case isn: IsNull => + val newAttribute = InternalSchemaUtils.reBuildFilterName(isn.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else isn.copy(attribute = newAttribute) + case isnn: IsNotNull => + val newAttribute = InternalSchemaUtils.reBuildFilterName(isnn.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else isnn.copy(attribute = newAttribute) + case And(left, right) => + And(rebuildFilterFromParquet(left, fileSchema, querySchema), rebuildFilterFromParquet(right, fileSchema, querySchema)) + case Or(left, right) => + Or(rebuildFilterFromParquet(left, fileSchema, querySchema), rebuildFilterFromParquet(right, fileSchema, querySchema)) + case Not(child) => + Not(rebuildFilterFromParquet(child, fileSchema, querySchema)) + case ssw: StringStartsWith => + val newAttribute = InternalSchemaUtils.reBuildFilterName(ssw.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else ssw.copy(attribute = newAttribute) + case ses: StringEndsWith => + val newAttribute = InternalSchemaUtils.reBuildFilterName(ses.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else ses.copy(attribute = newAttribute) + case sc: StringContains => + val newAttribute = InternalSchemaUtils.reBuildFilterName(sc.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else sc.copy(attribute = newAttribute) + case AlwaysTrue => + AlwaysTrue + case AlwaysFalse => + AlwaysFalse + case _ => + AlwaysTrue + } + } + } +} diff --git a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35LegacyHoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35LegacyHoodieParquetFileFormat.scala index e853e2788864..b6177b942fcf 100644 --- a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35LegacyHoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35LegacyHoodieParquetFileFormat.scala @@ -21,21 +21,36 @@ import org.apache.hadoop.conf.Configuration 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 +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.{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.JoinedRow -import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.catalyst.expressions.{Cast, JoinedRow} +import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.datasources.{DataSourceUtils, FileFormat, PartitionedFile, RecordReaderIterator} +import org.apache.spark.sql.execution.WholeStageCodegenExec +import org.apache.spark.sql.execution.datasources.parquet.Spark35LegacyHoodieParquetFileFormat._ +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.StructType +import org.apache.spark.sql.types.{AtomicType, DataType, StructField, StructType} import org.apache.spark.util.SerializableConfiguration +import scala.collection.convert.ImplicitConversions.`collection AsScalaIterable` + /** * This class is an extension of [[ParquetFileFormat]] overriding Spark-specific behavior * that's not possible to customize in any other way @@ -48,6 +63,15 @@ import org.apache.spark.util.SerializableConfiguration */ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValues: Boolean) extends ParquetFileFormat { + def supportsColumnar(sparkSession: SparkSession, schema: StructType): Boolean = { + val conf = sparkSession.sessionState.conf + // Only output columnar if there is WSCG to read it. + val requiredWholeStageCodegenSettings = + conf.wholeStageEnabled && !WholeStageCodegenExec.isTooManyFields(conf, schema) + requiredWholeStageCodegenSettings && + supportBatch(sparkSession, schema) + } + override def buildReaderWithPartitionValues(sparkSession: SparkSession, dataSchema: StructType, partitionSchema: StructType, @@ -72,6 +96,8 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu SQLConf.CASE_SENSITIVE.key, sparkSession.sessionState.conf.caseSensitiveAnalysis) + ParquetWriteSupport.setSchema(requiredSchema, hadoopConf) + // Sets flags for `ParquetToSparkSchemaConverter` hadoopConf.setBoolean( SQLConf.PARQUET_BINARY_AS_STRING.key, @@ -79,12 +105,23 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu hadoopConf.setBoolean( SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, sparkSession.sessionState.conf.isParquetINT96AsTimestamp) - hadoopConf.setBoolean( - SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key, - sparkSession.sessionState.conf.parquetInferTimestampNTZEnabled) + // Using string value of this conf to preserve compatibility across spark versions. hadoopConf.setBoolean( SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, - sparkSession.sessionState.conf.legacyParquetNanosAsLong) + sparkSession.sessionState.conf.getConfString( + SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, + SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.defaultValueString).toBoolean + ) + hadoopConf.setBoolean(SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key, sparkSession.sessionState.conf.parquetInferTimestampNTZEnabled) + hadoopConf.setBoolean(SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, sparkSession.sessionState.conf.legacyParquetNanosAsLong) + val internalSchemaStr = hadoopConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA) + // For Spark DataSource v1, there's no Physical Plan projection/schema pruning w/in Spark itself, + // therefore it's safe to do schema projection here + if (!isNullOrEmpty(internalSchemaStr)) { + val prunedInternalSchemaStr = + pruneInternalSchema(internalSchemaStr, requiredSchema) + hadoopConf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, prunedInternalSchemaStr) + } val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) @@ -95,8 +132,7 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val resultSchema = StructType(partitionSchema.fields ++ requiredSchema.fields) val sqlConf = sparkSession.sessionState.conf val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled - val enableVectorizedReader: Boolean = - ParquetUtils.isBatchReadSupportedForSchema(sqlConf, resultSchema) + val enableVectorizedReader: Boolean = supportBatch(sparkSession, resultSchema) val enableRecordFilter: Boolean = sqlConf.parquetRecordFilterEnabled val timestampConversion: Boolean = sqlConf.isParquetINT96TimestampConversion val capacity = sqlConf.parquetVectorizedReaderBatchSize @@ -104,7 +140,7 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val pushDownDate = sqlConf.parquetFilterPushDownDate val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal - val pushDownStringPredicate = sqlConf.parquetFilterPushDownStringPredicate + val pushDownStringStartWith = sqlConf.parquetFilterPushDownStringPredicate val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold val isCaseSensitive = sqlConf.caseSensitiveAnalysis val parquetOptions = new ParquetOptions(options, sparkSession.sessionState.conf) @@ -114,61 +150,72 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu // Should always be set by FileSourceScanExec creating this. // Check conf before checking option, to allow working around an issue by changing conf. val returningBatch = sparkSession.sessionState.conf.parquetVectorizedReaderEnabled && - options.getOrElse(FileFormat.OPTION_RETURNING_BATCH, - throw new IllegalArgumentException( - "OPTION_RETURNING_BATCH should always be set for ParquetFileFormat. " + - "To workaround this issue, set spark.sql.parquet.enableVectorizedReader=false.")) - .equals("true") - if (returningBatch) { - // If the passed option said that we are to return batches, we need to also be able to - // do this based on config and resultSchema. - assert(supportBatch(sparkSession, resultSchema)) - } + supportsColumnar(sparkSession, resultSchema).toString.equals("true") (file: PartitionedFile) => { assert(!shouldAppendPartitionValues || file.partitionValues.numFields == partitionSchema.size) - - val filePath = file.toPath + val filePath = file.filePath.toPath val split = new FileSplit(filePath, file.start, file.length, Array.empty[String]) val sharedConf = broadcastedHadoopConf.value.value - val schemaEvolutionUtils = new Spark32PlusParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) - val fileFooter = if (enableVectorizedReader) { - // When there are vectorized reads, we can avoid reading the footer twice by reading - // all row groups in advance and filter row groups according to filters that require - // push down (no need to read the footer metadata again). - ParquetFooterReader.readFooter(sharedConf, file, ParquetFooterReader.WITH_ROW_GROUPS) + // Fetch internal schema + val internalSchemaStr = sharedConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA) + // Internal schema has to be pruned at this point + val querySchemaOption = SerDeHelper.fromJson(internalSchemaStr) + + var shouldUseInternalSchema = !isNullOrEmpty(internalSchemaStr) && querySchemaOption.isPresent + + val tablePath = sharedConf.get(SparkInternalSchemaConverter.HOODIE_TABLE_PATH) + val fileSchema = if (shouldUseInternalSchema) { + val commitInstantTime = FSUtils.getCommitTime(filePath.getName).toLong; + val validCommits = sharedConf.get(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST) + InternalSchemaCache.getInternalSchemaByVersionId(commitInstantTime, tablePath, sharedConf, if (validCommits == null) "" else validCommits) } else { - ParquetFooterReader.readFooter(sharedConf, file, ParquetFooterReader.SKIP_ROW_GROUPS) + null } - val footerFileMetaData = fileFooter.getFileMetaData - val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( - footerFileMetaData.getKeyValueMetaData.get, - datetimeRebaseModeInRead) - val int96RebaseSpec = DataSourceUtils.int96RebaseSpec( - footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) - + lazy val footerFileMetaData = + ParquetFooterReader.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( - parquetSchema, - pushDownDate, - pushDownTimestamp, - pushDownDecimal, - pushDownStringPredicate, - pushDownInFilterThreshold, - isCaseSensitive, - datetimeRebaseSpec) - filters.map(schemaEvolutionUtils.rebuildFilterFromParquet) + val parquetFilters = if (HoodieSparkUtils.gteqSpark3_2_1) { + // NOTE: Below code could only be compiled against >= Spark 3.2.1, + // and unfortunately won't compile against Spark 3.2.0 + // However this code is runtime-compatible w/ both Spark 3.2.0 and >= Spark 3.2.1 + val datetimeRebaseSpec = + DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) + new ParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringStartWith, + pushDownInFilterThreshold, + isCaseSensitive, + datetimeRebaseSpec) + } else { + // Spark 3.2.0 + val datetimeRebaseMode = + Spark35DataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) + createParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringStartWith, + pushDownInFilterThreshold, + isCaseSensitive, + datetimeRebaseMode) + } + filters.map(rebuildFilterFromParquet(_, fileSchema, querySchemaOption.orElse(null))) // 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(_)) + .flatMap(parquetFilters.createFilter) .reduceOption(FilterApi.and) } else { None @@ -189,10 +236,35 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu None } - val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) + + // Clone new conf + val hadoopAttemptConf = new Configuration(broadcastedHadoopConf.value.value) + val typeChangeInfos: java.util.Map[Integer, Pair[DataType, DataType]] = if (shouldUseInternalSchema) { + val mergedInternalSchema = new InternalSchemaMerger(fileSchema, querySchemaOption.get(), true, true).mergeSchema() + val mergedSchema = SparkInternalSchemaConverter.constructSparkSchemaFromInternalSchema(mergedInternalSchema) + + hadoopAttemptConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, mergedSchema.json) + + SparkInternalSchemaConverter.collectTypeChangedCols(querySchemaOption.get(), mergedInternalSchema) + } else { + val (implicitTypeChangeInfo, sparkRequestSchema) = HoodieParquetFileFormatHelper.buildImplicitSchemaChangeInfo(hadoopAttemptConf, footerFileMetaData, requiredSchema) + if (!implicitTypeChangeInfo.isEmpty) { + shouldUseInternalSchema = true + hadoopAttemptConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, sparkRequestSchema.json) + } + implicitTypeChangeInfo + } + + if (enableVectorizedReader && shouldUseInternalSchema && + !typeChangeInfos.values().forall(_.getLeft.isInstanceOf[AtomicType])) { + throw new IllegalArgumentException( + "Nested types with type changes(implicit or explicit) cannot be read in vectorized mode. " + + "To workaround this issue, set spark.sql.parquet.enableVectorizedReader=false.") + } + val hadoopAttemptContext = - new TaskAttemptContextImpl(schemaEvolutionUtils.getHadoopConfClone(footerFileMetaData, enableVectorizedReader), attemptId) + new TaskAttemptContextImpl(hadoopAttemptConf, attemptId) // Try to push down filters when filter push-down is enabled. // Notice: This push-down is RowGroups level, not individual records. @@ -201,14 +273,51 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu } val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { - val vectorizedReader = schemaEvolutionUtils.buildVectorizedReader( - convertTz.orNull, - datetimeRebaseSpec.mode.toString, - datetimeRebaseSpec.timeZone, - int96RebaseSpec.mode.toString, - int96RebaseSpec.timeZone, - enableOffHeapColumnVector && taskContext.isDefined, - capacity) + val vectorizedReader = + if (shouldUseInternalSchema) { + val int96RebaseSpec = + DataSourceUtils.int96RebaseSpec(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) + val datetimeRebaseSpec = + DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) + new Spark32PlusHoodieVectorizedParquetRecordReader( + convertTz.orNull, + datetimeRebaseSpec.mode.toString, + datetimeRebaseSpec.timeZone, + int96RebaseSpec.mode.toString, + int96RebaseSpec.timeZone, + enableOffHeapColumnVector && taskContext.isDefined, + capacity, + typeChangeInfos) + } else if (HoodieSparkUtils.gteqSpark3_2_1) { + // NOTE: Below code could only be compiled against >= Spark 3.2.1, + // and unfortunately won't compile against Spark 3.2.0 + // However this code is runtime-compatible w/ both Spark 3.2.0 and >= Spark 3.2.1 + val int96RebaseSpec = + DataSourceUtils.int96RebaseSpec(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) + val datetimeRebaseSpec = + DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) + new VectorizedParquetRecordReader( + convertTz.orNull, + datetimeRebaseSpec.mode.toString, + datetimeRebaseSpec.timeZone, + int96RebaseSpec.mode.toString, + int96RebaseSpec.timeZone, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) + } else { + // Spark 3.2.0 + val datetimeRebaseMode = + Spark35DataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) + val int96RebaseMode = + Spark35DataSourceUtils.int96RebaseMode(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) + createVectorizedParquetRecordReader( + convertTz.orNull, + datetimeRebaseMode.toString, + int96RebaseMode.toString, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) + } + // SPARK-37089: We cannot register a task completion listener to close this iterator here // because downstream exec nodes have already registered their listeners. Since listeners // are executed in reverse order of registration, a listener registered here would close the @@ -218,8 +327,17 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu // Instead, we use FileScanRDD's task completion listener to close this iterator. val iter = new RecordReaderIterator(vectorizedReader) try { - vectorizedReader.initialize(split, hadoopAttemptContext, Option.apply(fileFooter)) - vectorizedReader.initBatch(partitionSchema, file.partitionValues) + vectorizedReader.initialize(split, hadoopAttemptContext) + + // NOTE: We're making appending of the partitioned values to the rows read from the + // data file configurable + if (shouldAppendPartitionValues) { + logDebug(s"Appending $partitionSchema ${file.partitionValues}") + vectorizedReader.initBatch(partitionSchema, file.partitionValues) + } else { + vectorizedReader.initBatch(StructType(Nil), InternalRow.empty) + } + if (returningBatch) { vectorizedReader.enableReturningBatches() } @@ -234,26 +352,64 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu throw e } } else { - // ParquetRecordReader returns InternalRow - val readSupport = new ParquetReadSupport( - convertTz, - enableVectorizedReader = false, - datetimeRebaseSpec, - int96RebaseSpec) + logDebug(s"Falling back to parquet-mr") + val readSupport = if (HoodieSparkUtils.gteqSpark3_2_1) { + // ParquetRecordReader returns InternalRow + // NOTE: Below code could only be compiled against >= Spark 3.2.1, + // and unfortunately won't compile against Spark 3.2.0 + // However this code is runtime-compatible w/ both Spark 3.2.0 and >= Spark 3.2.1 + val int96RebaseSpec = + DataSourceUtils.int96RebaseSpec(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) + val datetimeRebaseSpec = + DataSourceUtils.datetimeRebaseSpec(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) + new ParquetReadSupport( + convertTz, + enableVectorizedReader = false, + datetimeRebaseSpec, + int96RebaseSpec) + } else { + val datetimeRebaseMode = + Spark35DataSourceUtils.datetimeRebaseMode(footerFileMetaData.getKeyValueMetaData.get, datetimeRebaseModeInRead) + val int96RebaseMode = + Spark35DataSourceUtils.int96RebaseMode(footerFileMetaData.getKeyValueMetaData.get, int96RebaseModeInRead) + createParquetReadSupport( + convertTz, + /* enableVectorizedReader = */ false, + datetimeRebaseMode, + int96RebaseMode) + } + val reader = if (pushed.isDefined && enableRecordFilter) { val parquetFilter = FilterCompat.get(pushed.get, null) new ParquetRecordReader[InternalRow](readSupport, parquetFilter) } else { new ParquetRecordReader[InternalRow](readSupport) } - val readerWithRowIndexes = ParquetRowIndexUtil.addRowIndexToRecordReaderIfNeeded(reader, - requiredSchema) - val iter = new RecordReaderIterator[InternalRow](readerWithRowIndexes) + val iter = new RecordReaderIterator[InternalRow](reader) try { - readerWithRowIndexes.initialize(split, hadoopAttemptContext) + reader.initialize(split, hadoopAttemptContext) - val fullSchema = toAttributes(requiredSchema) ++ toAttributes(partitionSchema) - val unsafeProjection = schemaEvolutionUtils.generateUnsafeProjection(fullSchema, timeZoneId) + val fullSchema = DataTypeUtils.toAttributes(requiredSchema) ++ DataTypeUtils.toAttributes(partitionSchema) + val unsafeProjection = if (typeChangeInfos.isEmpty) { + GenerateUnsafeProjection.generate(fullSchema, fullSchema) + } else { + // find type changed. + val newSchema = new StructType(requiredSchema.fields.zipWithIndex.map { case (f, i) => + if (typeChangeInfos.containsKey(i)) { + StructField(f.name, typeChangeInfos.get(i).getRight, f.nullable, f.metadata) + } else f + }) + val newFullSchema = DataTypeUtils.toAttributes(newSchema) ++ DataTypeUtils.toAttributes(partitionSchema) + val castSchema = newFullSchema.zipWithIndex.map { case (attr, i) => + if (typeChangeInfos.containsKey(i)) { + val srcType = typeChangeInfos.get(i).getRight + val dstType = typeChangeInfos.get(i).getLeft + val needTimeZone = Cast.needsTimeZone(srcType, dstType) + Cast(attr, dstType, if (needTimeZone) timeZoneId else None) + } else attr + } + GenerateUnsafeProjection.generate(castSchema, newFullSchema) + } // NOTE: We're making appending of the partitioned values to the rows read from the // data file configurable @@ -276,3 +432,108 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu } } +object Spark35LegacyHoodieParquetFileFormat { + + /** + * NOTE: This method is specific to Spark 3.2.0 + */ + private def createParquetFilters(args: Any*): ParquetFilters = { + // NOTE: ParquetFilters ctor args contain Scala enum, therefore we can't look it + // up by arg types, and have to instead rely on the number of args based on individual class; + // the ctor order is not guaranteed + val ctor = classOf[ParquetFilters].getConstructors.maxBy(_.getParameterCount) + ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*) + .asInstanceOf[ParquetFilters] + } + + /** + * NOTE: This method is specific to Spark 3.2.0 + */ + private def createParquetReadSupport(args: Any*): ParquetReadSupport = { + // NOTE: ParquetReadSupport ctor args contain Scala enum, therefore we can't look it + // up by arg types, and have to instead rely on the number of args based on individual class; + // the ctor order is not guaranteed + val ctor = classOf[ParquetReadSupport].getConstructors.maxBy(_.getParameterCount) + ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*) + .asInstanceOf[ParquetReadSupport] + } + + /** + * NOTE: This method is specific to Spark 3.2.0 + */ + private def createVectorizedParquetRecordReader(args: Any*): VectorizedParquetRecordReader = { + // NOTE: ParquetReadSupport ctor args contain Scala enum, therefore we can't look it + // up by arg types, and have to instead rely on the number of args based on individual class; + // the ctor order is not guaranteed + val ctor = classOf[VectorizedParquetRecordReader].getConstructors.maxBy(_.getParameterCount) + ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*) + .asInstanceOf[VectorizedParquetRecordReader] + } + + def pruneInternalSchema(internalSchemaStr: String, requiredSchema: StructType): String = { + val querySchemaOption = SerDeHelper.fromJson(internalSchemaStr) + if (querySchemaOption.isPresent && requiredSchema.nonEmpty) { + val prunedSchema = SparkInternalSchemaConverter.convertAndPruneStructTypeToInternalSchema(requiredSchema, querySchemaOption.get()) + SerDeHelper.toJson(prunedSchema) + } else { + internalSchemaStr + } + } + + private def rebuildFilterFromParquet(oldFilter: Filter, fileSchema: InternalSchema, querySchema: InternalSchema): Filter = { + if (fileSchema == null || querySchema == null) { + oldFilter + } else { + oldFilter match { + case eq: EqualTo => + val newAttribute = InternalSchemaUtils.reBuildFilterName(eq.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else eq.copy(attribute = newAttribute) + case eqs: EqualNullSafe => + val newAttribute = InternalSchemaUtils.reBuildFilterName(eqs.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else eqs.copy(attribute = newAttribute) + case gt: GreaterThan => + val newAttribute = InternalSchemaUtils.reBuildFilterName(gt.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else gt.copy(attribute = newAttribute) + case gtr: GreaterThanOrEqual => + val newAttribute = InternalSchemaUtils.reBuildFilterName(gtr.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else gtr.copy(attribute = newAttribute) + case lt: LessThan => + val newAttribute = InternalSchemaUtils.reBuildFilterName(lt.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else lt.copy(attribute = newAttribute) + case lte: LessThanOrEqual => + val newAttribute = InternalSchemaUtils.reBuildFilterName(lte.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else lte.copy(attribute = newAttribute) + case i: In => + val newAttribute = InternalSchemaUtils.reBuildFilterName(i.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else i.copy(attribute = newAttribute) + case isn: IsNull => + val newAttribute = InternalSchemaUtils.reBuildFilterName(isn.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else isn.copy(attribute = newAttribute) + case isnn: IsNotNull => + val newAttribute = InternalSchemaUtils.reBuildFilterName(isnn.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else isnn.copy(attribute = newAttribute) + case And(left, right) => + And(rebuildFilterFromParquet(left, fileSchema, querySchema), rebuildFilterFromParquet(right, fileSchema, querySchema)) + case Or(left, right) => + Or(rebuildFilterFromParquet(left, fileSchema, querySchema), rebuildFilterFromParquet(right, fileSchema, querySchema)) + case Not(child) => + Not(rebuildFilterFromParquet(child, fileSchema, querySchema)) + case ssw: StringStartsWith => + val newAttribute = InternalSchemaUtils.reBuildFilterName(ssw.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else ssw.copy(attribute = newAttribute) + case ses: StringEndsWith => + val newAttribute = InternalSchemaUtils.reBuildFilterName(ses.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else ses.copy(attribute = newAttribute) + case sc: StringContains => + val newAttribute = InternalSchemaUtils.reBuildFilterName(sc.attribute, fileSchema, querySchema) + if (newAttribute.isEmpty) AlwaysTrue else sc.copy(attribute = newAttribute) + case AlwaysTrue => + AlwaysTrue + case AlwaysFalse => + AlwaysFalse + case _ => + AlwaysTrue + } + } + } +} From a08eacbe858b12a534a74ff70146947064fbe0cf Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Mon, 15 Apr 2024 14:09:00 -0400 Subject: [PATCH 65/86] update spark 3.2 --- .../parquet/Spark32ParquetReader.scala | 21 ++++------- .../Spark32ParquetSchemaEvolutionUtils.scala | 35 ------------------- 2 files changed, 6 insertions(+), 50 deletions(-) delete mode 100644 hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32ParquetSchemaEvolutionUtils.scala diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32ParquetReader.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32ParquetReader.scala index 6b9261572207..e2189e5cbdfb 100644 --- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32ParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32ParquetReader.scala @@ -82,16 +82,16 @@ class Spark32ParquetReader(enableVectorizedReader: Boolean, * @return iterator of rows read from the file output type says [[InternalRow]] but could be [[ColumnarBatch]] */ protected def doRead(file: PartitionedFile, - requiredSchema: StructType, - partitionSchema: StructType, - filters: Seq[Filter], - sharedConf: Configuration): Iterator[InternalRow] = { + requiredSchema: StructType, + partitionSchema: StructType, + filters: Seq[Filter], + sharedConf: Configuration): Iterator[InternalRow] = { assert(file.partitionValues.numFields == partitionSchema.size) val filePath = new Path(new URI(file.filePath)) val split = new FileSplit(filePath, file.start, file.length, Array.empty[String]) - val schemaEvolutionUtils = new Spark32ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) + val schemaEvolutionUtils = new Spark32PlusParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) lazy val footerFileMetaData = ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData @@ -170,15 +170,7 @@ class Spark32ParquetReader(enableVectorizedReader: Boolean, } val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { - val vectorizedReader = if (schemaEvolutionUtils.shouldUseInternalSchema) { - schemaEvolutionUtils.buildVectorizedReader(convertTz, - datetimeRebaseSpec, - int96RebaseSpec, - enableOffHeapColumnVector, - taskContext, - capacity) - } else { - new VectorizedParquetRecordReader( + val vectorizedReader = schemaEvolutionUtils.buildVectorizedReader( convertTz.orNull, datetimeRebaseSpec.mode.toString, datetimeRebaseSpec.timeZone, @@ -186,7 +178,6 @@ class Spark32ParquetReader(enableVectorizedReader: Boolean, int96RebaseSpec.timeZone, enableOffHeapColumnVector && taskContext.isDefined, capacity) - } // SPARK-37089: We cannot register a task completion listener to close this iterator here // because downstream exec nodes have already registered their listeners. Since listeners // are executed in reverse order of registration, a listener registered here would close the diff --git a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32ParquetSchemaEvolutionUtils.scala b/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32ParquetSchemaEvolutionUtils.scala deleted file mode 100644 index 43741e8193de..000000000000 --- a/hudi-spark-datasource/hudi-spark3.2.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32ParquetSchemaEvolutionUtils.scala +++ /dev/null @@ -1,35 +0,0 @@ -/* - * 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.spark.sql.catalyst.expressions.AttributeReference -import org.apache.spark.sql.types.StructType - -class Spark32ParquetSchemaEvolutionUtils(sharedConf: Configuration, - filePath: Path, - requiredSchema: StructType, - partitionSchema: StructType) extends - Spark32PlusParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema){ - override protected def toAttributes(schema: StructType): Seq[AttributeReference] = { - schema.toAttributes - } -} From 1edf6bf087ed906a67cdb9c6d949c7d751e5eb08 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Mon, 15 Apr 2024 14:13:58 -0400 Subject: [PATCH 66/86] update spark 3.1 --- .../parquet/Spark31ParquetReader.scala | 23 +++-------- .../Spark31ParquetSchemaEvolutionUtils.scala | 40 +++++++++---------- 2 files changed, 26 insertions(+), 37 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31ParquetReader.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31ParquetReader.scala index ef7ddf1b510b..a3e179fa184e 100644 --- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31ParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31ParquetReader.scala @@ -165,23 +165,12 @@ class Spark31ParquetReader(enableVectorizedReader: Boolean, } val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { - val vectorizedReader = if (schemaEvolutionUtils.shouldUseInternalSchema) { - schemaEvolutionUtils.buildVectorizedReader( - convertTz, - datetimeRebaseMode, - int96RebaseMode, - enableOffHeapColumnVector, - taskContext, - capacity) - } else { - new VectorizedParquetRecordReader( - convertTz.orNull, - datetimeRebaseMode.toString, - int96RebaseMode.toString, - enableOffHeapColumnVector && taskContext.isDefined, - capacity) - } - + val vectorizedReader = schemaEvolutionUtils.buildVectorizedReader( + convertTz.orNull, + datetimeRebaseMode.toString, + int96RebaseMode.toString, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) val iter = new RecordReaderIterator(vectorizedReader) // SPARK-23457 Register a task completion listener before `initialization`. taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31ParquetSchemaEvolutionUtils.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31ParquetSchemaEvolutionUtils.scala index 4ad8265ba524..d31efde4d290 100644 --- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31ParquetSchemaEvolutionUtils.scala +++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31ParquetSchemaEvolutionUtils.scala @@ -21,10 +21,7 @@ package org.apache.spark.sql.execution.datasources.parquet import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.TaskContext -import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.execution.datasources.Spark3ParquetSchemaEvolutionUtils -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import java.time.ZoneId @@ -35,23 +32,26 @@ class Spark31ParquetSchemaEvolutionUtils(sharedConf: Configuration, partitionSchema: StructType) extends Spark3ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) { - - override protected def toAttributes(schema: StructType): Seq[AttributeReference] = { - schema.toAttributes - } - - def buildVectorizedReader(convertTz: Option[ZoneId], - datetimeRebaseMode: SQLConf.LegacyBehaviorPolicy.Value, - int96RebaseMode: SQLConf.LegacyBehaviorPolicy.Value, - enableOffHeapColumnVector: Boolean, - taskContext: Option[TaskContext], + def buildVectorizedReader(convertTz: ZoneId, + datetimeRebaseMode: String, + int96RebaseMode: String, + useOffHeap: Boolean, capacity: Int): VectorizedParquetRecordReader = { - new Spark31HoodieVectorizedParquetRecordReader( - convertTz.orNull, - datetimeRebaseMode.toString, - int96RebaseMode.toString, - enableOffHeapColumnVector && taskContext.isDefined, - capacity, - typeChangeInfos) + if (shouldUseInternalSchema) { + new Spark31HoodieVectorizedParquetRecordReader( + convertTz, + datetimeRebaseMode, + int96RebaseMode, + useOffHeap, + capacity, + typeChangeInfos) + } else { + new VectorizedParquetRecordReader( + convertTz, + datetimeRebaseMode, + int96RebaseMode, + useOffHeap, + capacity) + } } } From eb58a1a3af3bda46cd0db910ac39e37efd744cdd Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Mon, 15 Apr 2024 14:17:14 -0400 Subject: [PATCH 67/86] update spark 3.0 --- .../parquet/Spark30ParquetReader.scala | 19 +++------- .../Spark30ParquetSchemaEvolutionUtils.scala | 35 +++++++++---------- 2 files changed, 22 insertions(+), 32 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30ParquetReader.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30ParquetReader.scala index c6c277fff3dc..a6e9a3c5fe02 100644 --- a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30ParquetReader.scala +++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30ParquetReader.scala @@ -149,20 +149,11 @@ class Spark30ParquetReader(enableVectorizedReader: Boolean, } val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { - val vectorizedReader = if (schemaEvolutionUtils.shouldUseInternalSchema) { - schemaEvolutionUtils.buildVectorizedReader( - convertTz, - datetimeRebaseMode, - enableOffHeapColumnVector, - taskContext, - capacity) - } else { - new VectorizedParquetRecordReader( - convertTz.orNull, - datetimeRebaseMode.toString, - enableOffHeapColumnVector && taskContext.isDefined, - capacity) - } + val vectorizedReader = schemaEvolutionUtils.buildVectorizedReader( + convertTz.orNull, + datetimeRebaseMode.toString, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) val iter = new RecordReaderIterator(vectorizedReader) // SPARK-23457 Register a task completion listener before `initialization`. taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) diff --git a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30ParquetSchemaEvolutionUtils.scala b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30ParquetSchemaEvolutionUtils.scala index 0ff2ede9ba7d..28acb6e0474d 100644 --- a/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30ParquetSchemaEvolutionUtils.scala +++ b/hudi-spark-datasource/hudi-spark3.0.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark30ParquetSchemaEvolutionUtils.scala @@ -21,10 +21,7 @@ package org.apache.spark.sql.execution.datasources.parquet import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.TaskContext -import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.execution.datasources.Spark3ParquetSchemaEvolutionUtils -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import java.time.ZoneId @@ -35,21 +32,23 @@ class Spark30ParquetSchemaEvolutionUtils(sharedConf: Configuration, partitionSchema: StructType) extends Spark3ParquetSchemaEvolutionUtils(sharedConf, filePath, requiredSchema, partitionSchema) { - - override protected def toAttributes(schema: StructType): Seq[AttributeReference] = { - schema.toAttributes - } - - def buildVectorizedReader(convertTz: Option[ZoneId], - datetimeRebaseMode: SQLConf.LegacyBehaviorPolicy.Value, - enableOffHeapColumnVector: Boolean, - taskContext: Option[TaskContext], + def buildVectorizedReader(convertTz: ZoneId, + datetimeRebaseMode: String, + useOffHeap: Boolean, capacity: Int): VectorizedParquetRecordReader = { - new Spark30HoodieVectorizedParquetRecordReader( - convertTz.orNull, - datetimeRebaseMode.toString, - enableOffHeapColumnVector && taskContext.isDefined, - capacity, - typeChangeInfos) + if (shouldUseInternalSchema) { + new Spark30HoodieVectorizedParquetRecordReader( + convertTz, + datetimeRebaseMode, + useOffHeap, + capacity, + typeChangeInfos) + } else { + new VectorizedParquetRecordReader( + convertTz, + datetimeRebaseMode, + useOffHeap, + capacity) + } } } From 72e09f67466fcfa61b0ec555fc2eecfa52fbb856 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 16 Apr 2024 10:59:46 -0400 Subject: [PATCH 68/86] remove some unnecessary changes to make it easier to review --- .../storage/HoodieSparkFileReaderFactory.java | 13 ++++---- ...rkFileFormatInternalRowReaderContext.scala | 33 ++++++++----------- .../common/engine/HoodieReaderContext.java | 7 ---- .../read/HoodieBaseFileGroupRecordBuffer.java | 2 +- .../table/read/HoodieFileGroupReader.java | 6 ++-- .../read/HoodieFileGroupReaderState.java | 1 + ...diePositionBasedFileGroupRecordBuffer.java | 2 +- .../HoodieUnmergedFileGroupRecordBuffer.java | 2 +- .../schema/action/InternalSchemaMerger.java | 1 + .../convert/AvroInternalSchemaConverter.java | 16 ++++----- .../read/TestHoodieFileGroupReaderBase.java | 2 +- .../HoodieFileGroupReaderTestUtils.java | 2 +- .../reader/HoodieTestReaderContext.java | 7 +--- .../scala/org/apache/hudi/DefaultSource.scala | 6 ++-- .../hudi/HoodieHadoopFsRelationFactory.scala | 26 +++++++-------- ...ileGroupReaderBasedParquetFileFormat.scala | 3 +- .../TestHoodieFileGroupReaderOnSpark.scala | 2 +- ...tSpark35RecordPositionMetadataColumn.scala | 3 +- 18 files changed, 59 insertions(+), 75 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReaderFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReaderFactory.java index 60ae86d9e122..d06b69139059 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReaderFactory.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReaderFactory.java @@ -33,16 +33,15 @@ public class HoodieSparkFileReaderFactory extends HoodieFileReaderFactory { @Override public HoodieFileReader newParquetFileReader(Configuration conf, Path path) { - Configuration configClone = new Configuration(conf); - configClone.setIfUnset(SQLConf.PARQUET_BINARY_AS_STRING().key(), SQLConf.PARQUET_BINARY_AS_STRING().defaultValueString()); - configClone.setIfUnset(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), SQLConf.PARQUET_INT96_AS_TIMESTAMP().defaultValueString()); - configClone.setIfUnset(SQLConf.CASE_SENSITIVE().key(), SQLConf.CASE_SENSITIVE().defaultValueString()); + conf.setIfUnset(SQLConf.PARQUET_BINARY_AS_STRING().key(), SQLConf.PARQUET_BINARY_AS_STRING().defaultValueString()); + conf.setIfUnset(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), SQLConf.PARQUET_INT96_AS_TIMESTAMP().defaultValueString()); + conf.setIfUnset(SQLConf.CASE_SENSITIVE().key(), SQLConf.CASE_SENSITIVE().defaultValueString()); // Using string value of this conf to preserve compatibility across spark versions. - configClone.setIfUnset("spark.sql.legacy.parquet.nanosAsLong", "false"); + conf.setIfUnset("spark.sql.legacy.parquet.nanosAsLong", "false"); // This is a required config since Spark 3.4.0: SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED // Using string value of this conf to preserve compatibility across spark versions. - configClone.setIfUnset("spark.sql.parquet.inferTimestampNTZ.enabled", "true"); - return new HoodieSparkParquetReader(configClone, path); + conf.setIfUnset("spark.sql.parquet.inferTimestampNTZ.enabled", "true"); + return new HoodieSparkParquetReader(conf, path); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala index 59213297b138..3847a1007fff 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala @@ -49,15 +49,15 @@ import scala.collection.mutable * * This uses Spark parquet reader to read parquet data files or parquet log blocks. * - * @param baseFileReader A reader that transforms a {@link PartitionedFile} to an iterator of + * @param parquetFileReader A reader that transforms a {@link PartitionedFile} to an iterator of * {@link InternalRow}. This is required for reading the base file and * not required for reading a file group with only log files. - * @param partitionValues The values for a partition in which the file group lives. + * @param recordKeyColumn column name for the recordkey + * @param filters spark filters that might be pushed down into the reader */ class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkParquetReader, recordKeyColumn: String, - filters: Seq[Filter], - shouldUseRecordPosition: Boolean) extends BaseSparkInternalRowReaderContext { + filters: Seq[Filter]) extends BaseSparkInternalRowReaderContext { lazy val sparkAdapter = SparkAdapterSupport.sparkAdapter lazy val sparkFileReaderFactory = new HoodieSparkFileReaderFactory val deserializerMap: mutable.Map[Schema, HoodieAvroDeserializer] = mutable.Map() @@ -94,12 +94,14 @@ class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkParquetRea } private def getSchemaAndFiltersForRead(structType: StructType): (StructType, Seq[Filter]) = { - (readerState.hasLogFiles.booleanValue(), readerState.needsBootstrapMerge.booleanValue(), shouldUseRecordPosition) match { + (readerState.hasLogFiles.booleanValue(), + readerState.needsBootstrapMerge.booleanValue(), + readerState.useRecordPosition.booleanValue()) match { case (false, false, _) => (structType, filters) - case (false, true, true) if shouldUseRecordPosition => + case (false, true, true) => (getAppliedRequiredSchema(structType), filters) - case (true, _, true) if shouldUseRecordPosition => + case (true, _, true) => (getAppliedRequiredSchema(structType), recordKeyFilters) case (_, _, _) => (structType, Seq.empty) @@ -133,7 +135,7 @@ class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkParquetRea skeletonRequiredSchema: Schema, dataFileIterator: ClosableIterator[Any], dataRequiredSchema: Schema): ClosableIterator[InternalRow] = { - if (shouldUseRecordPosition) { + if (readerState.useRecordPosition) { assert(AvroSchemaUtils.containsFieldInSchema(skeletonRequiredSchema, ROW_INDEX_TEMPORARY_COLUMN_NAME)) assert(AvroSchemaUtils.containsFieldInSchema(dataRequiredSchema, ROW_INDEX_TEMPORARY_COLUMN_NAME)) val javaSet = new java.util.HashSet[String]() @@ -141,7 +143,7 @@ class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkParquetRea val skeletonProjection = projectRecord(skeletonRequiredSchema, AvroSchemaUtils.removeFieldsFromSchema(skeletonRequiredSchema, javaSet)) //If we have log files, we will want to do position based merging with those as well, - //so leave a temporary column at the end + //so leave the row index column at the end val dataProjection = if (readerState.hasLogFiles) { getIdentityProjection } else { @@ -149,9 +151,12 @@ class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkParquetRea AvroSchemaUtils.removeFieldsFromSchema(dataRequiredSchema, javaSet)) } + //Always use internal row for positional merge because + //we need to iterate row by row when merging new CachingIterator[InternalRow] { val combinedRow = new JoinedRow() + //position column will always be at the end of the row private def getPos(row: InternalRow): Long = { row.getLong(row.numFields-1) } @@ -234,13 +239,7 @@ class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkParquetRea } }.asInstanceOf[ClosableIterator[InternalRow]] } - - } - - override def shouldUseRecordPositionMerging(): Boolean = { - shouldUseRecordPosition } - } object SparkFileFormatInternalRowReaderContext { @@ -257,10 +256,6 @@ object SparkFileFormatInternalRowReaderContext { filters.filter(f => f.references.exists(c => c.equalsIgnoreCase(recordKeyColumn))) } - def hasIndexTempColumn(structType: StructType): Boolean = { - structType.fields.exists(isIndexTempColumn) - } - def isIndexTempColumn(field: StructField): Boolean = { field.name.equals(ROW_INDEX_TEMPORARY_COLUMN_NAME) } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java index d7b9f0cf02cb..c0985b6b35cb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java @@ -234,13 +234,6 @@ public long extractRecordPosition(T record, Schema schema, String fieldName, lon return providedPositionIfNeeded; } - /** - * returns true if record position should be used for merging - */ - public boolean shouldUseRecordPositionMerging() { - return false; - } - /** * Constructs engine specific delete record. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java index a3aa8559fc56..c886f5dbbdef 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java @@ -383,4 +383,4 @@ protected static List extractRecordPositions(HoodieLogBlock logBlock) thro return blockPositions; } -} \ No newline at end of file +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java index b251b8ff52ef..cc287563bdd6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java @@ -111,17 +111,17 @@ public HoodieFileGroupReader(HoodieReaderContext readerContext, this.readerState.recordMerger = this.recordMerger; this.readerState.tablePath = tablePath; this.readerState.latestCommitTime = latestCommitTime; - boolean shouldMergeWithPosition = shouldUseRecordPosition && readerContext.shouldUseRecordPositionMerging(); + readerState.useRecordPosition = shouldUseRecordPosition; readerState.hasLogFiles = !this.logFiles.isEmpty(); readerState.hasBootstrapBaseFile = hoodieBaseFileOption.isPresent() && hoodieBaseFileOption.get().getBootstrapBaseFile().isPresent(); - readerState.schemaHandler = shouldMergeWithPosition + readerState.schemaHandler = shouldUseRecordPosition ? new HoodiePositionBasedSchemaHandler<>(readerContext, dataSchema, requestedSchema, internalSchemaOpt, tableConfig) : new HoodieFileGroupReaderSchemaHandler<>(readerContext, dataSchema, requestedSchema, internalSchemaOpt, tableConfig); this.schemaHandler = readerState.schemaHandler; this.outputConverter = schemaHandler.getOutputConverter(); this.recordBuffer = this.logFiles.isEmpty() ? null - : shouldMergeWithPosition + : shouldUseRecordPosition ? new HoodiePositionBasedFileGroupRecordBuffer<>(readerContext, hoodieTableMetaClient, Option.empty(), Option.empty(), recordMerger, props, maxMemorySizeInBytes, spillableMapBasePath, diskMapType, isBitCaskDiskMapCompressionEnabled) : new HoodieKeyBasedFileGroupRecordBuffer<>(readerContext, hoodieTableMetaClient, Option.empty(), diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderState.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderState.java index 5fbd9d5e6d7f..ea3f816b5497 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderState.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderState.java @@ -33,4 +33,5 @@ public class HoodieFileGroupReaderState { public Boolean hasLogFiles = null; public Boolean hasBootstrapBaseFile = null; public Boolean needsBootstrapMerge = null; + public Boolean useRecordPosition = null; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java index 623982b78d8b..04c30f4f4296 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java @@ -225,4 +225,4 @@ protected boolean doHasNext() throws IOException { return false; } -} \ No newline at end of file +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieUnmergedFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieUnmergedFileGroupRecordBuffer.java index 325f9ca4f93e..8338c96f5a0e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieUnmergedFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieUnmergedFileGroupRecordBuffer.java @@ -143,4 +143,4 @@ public void processNextDeletedRecord(DeleteRecord deleteRecord, Serializable ind public boolean containsLogRecord(String recordKey) { return records.containsKey(recordKey); } -} \ No newline at end of file +} diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java index fc3ba94ea594..9ba71a80d5ce 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java @@ -233,3 +233,4 @@ private Type buildPrimitiveType(Type.PrimitiveType typeFromQuerySchema, int curr } } } + diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java index fc28554ff9bc..4389e59627c9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java @@ -326,20 +326,20 @@ private static Type visitAvroPrimitiveToBuildInternalType(Schema primitive) { String name = logical.getName(); if (logical instanceof LogicalTypes.Decimal) { return Types.DecimalType.get( - ((LogicalTypes.Decimal) logical).getPrecision(), - ((LogicalTypes.Decimal) logical).getScale()); + ((LogicalTypes.Decimal) logical).getPrecision(), + ((LogicalTypes.Decimal) logical).getScale()); } else if (logical instanceof LogicalTypes.Date) { return Types.DateType.get(); } else if ( - logical instanceof LogicalTypes.TimeMillis - || logical instanceof LogicalTypes.TimeMicros) { + logical instanceof LogicalTypes.TimeMillis + || logical instanceof LogicalTypes.TimeMicros) { return Types.TimeType.get(); } else if ( - logical instanceof LogicalTypes.TimestampMillis - || logical instanceof LogicalTypes.TimestampMicros) { + logical instanceof LogicalTypes.TimestampMillis + || logical instanceof LogicalTypes.TimestampMicros) { return Types.TimestampType.get(); } else if (LogicalTypes.uuid().getName().equals(name)) { return Types.UUIDType.get(); @@ -567,7 +567,7 @@ private static Schema visitInternalPrimitiveToBuildAvroPrimitiveType(Type.Primit default: throw new UnsupportedOperationException( - "Unsupported type ID: " + primitive.typeId()); + "Unsupported type ID: " + primitive.typeId()); } } @@ -582,4 +582,4 @@ private static int computeMinBytesForPrecision(int precision) { } return numBytes; } -} \ No newline at end of file +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java index 8ca2517eade0..b89836a4c55b 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java @@ -196,4 +196,4 @@ private void validateOutputFromFileGroupReader(Configuration hadoopConf, validateRecordsInFileGroup(tablePath, actualRecordList, avroSchema, fileSlice.getFileId()); } -} \ No newline at end of file +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestUtils.java index e72a71a4c5ab..c032b192cef2 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestUtils.java @@ -132,4 +132,4 @@ public HoodieFileGroupReader build( false); } } -} \ No newline at end of file +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieTestReaderContext.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieTestReaderContext.java index 91138e6ee237..81691d27662c 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieTestReaderContext.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieTestReaderContext.java @@ -227,9 +227,4 @@ private Object getFieldValueFromIndexedRecord( int pos = field.pos(); return record.get(pos); } - - @Override - public boolean shouldUseRecordPositionMerging() { - return true; - } -} \ No newline at end of file +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala index 102aa2d0c135..bdd26f4b9daa 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala @@ -47,9 +47,9 @@ import scala.collection.JavaConversions.mapAsJavaMap import scala.collection.JavaConverters._ /** - * Hoodie Spark Datasource, for reading and writing hoodie tables - * - */ + * Hoodie Spark Datasource, for reading and writing hoodie tables + * + */ class DefaultSource extends RelationProvider with SchemaRelationProvider with CreatableRelationProvider diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala index 02b4efa37d00..d8c3d3776344 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala @@ -31,13 +31,12 @@ import org.apache.hudi.common.config.{ConfigProperty, HoodieMetadataConfig, Hood import org.apache.hudi.common.model.HoodieRecord import org.apache.hudi.common.table.timeline.HoodieTimeline import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver} -import org.apache.hudi.common.util.{ConfigUtils, StringUtils} import org.apache.hudi.common.util.ValidationUtils.checkState -import org.apache.hudi.config.HoodieBootstrapConfig.DATA_QUERIES_ONLY +import org.apache.hudi.common.util.{ConfigUtils, StringUtils} import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter import org.apache.hudi.internal.schema.utils.SerDeHelper -import org.apache.hudi.internal.schema.{HoodieSchemaException, InternalSchema} import org.apache.hudi.metadata.HoodieTableMetadataUtil import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.catalog.BucketSpec @@ -186,13 +185,13 @@ abstract class HoodieBaseHadoopFsRelationFactory(val sqlContext: SQLContext, metaClient.getCommitsAndCompactionTimeline.filterCompletedInstants protected def getConfigValue(config: ConfigProperty[String], - defaultValueOption: Option[String] = Option.empty): String = { + defaultValueOption: Option[String] = Option.empty): String = { optParams.getOrElse(config.key(), sqlContext.getConf(config.key(), defaultValueOption.getOrElse(config.defaultValue()))) } protected def checkIfAConfigurationEnabled(config: ConfigProperty[java.lang.Boolean], - defaultValueOption: Option[String] = Option.empty): Boolean = { + defaultValueOption: Option[String] = Option.empty): Boolean = { optParams.getOrElse(config.key(), sqlContext.getConf(config.key(), defaultValueOption.getOrElse(String.valueOf(config.defaultValue())))).toBoolean } @@ -295,10 +294,10 @@ class HoodieMergeOnReadIncrementalHadoopFsRelationFactory(override val sqlContex } class HoodieCopyOnWriteSnapshotHadoopFsRelationFactory(override val sqlContext: SQLContext, - override val metaClient: HoodieTableMetaClient, - override val options: Map[String, String], - override val schemaSpec: Option[StructType], - isBootstrap: Boolean) + override val metaClient: HoodieTableMetaClient, + override val options: Map[String, String], + override val schemaSpec: Option[StructType], + isBootstrap: Boolean) extends HoodieMergeOnReadSnapshotHadoopFsRelationFactory(sqlContext, metaClient, options, schemaSpec, isBootstrap) { override val mandatoryFields: Seq[String] = Seq.empty @@ -354,10 +353,10 @@ class HoodieCopyOnWriteIncrementalHadoopFsRelationFactory(override val sqlContex } class HoodieMergeOnReadCDCHadoopFsRelationFactory(override val sqlContext: SQLContext, - override val metaClient: HoodieTableMetaClient, - override val options: Map[String, String], - override val schemaSpec: Option[StructType], - isBootstrap: Boolean) + override val metaClient: HoodieTableMetaClient, + override val options: Map[String, String], + override val schemaSpec: Option[StructType], + isBootstrap: Boolean) extends HoodieMergeOnReadIncrementalHadoopFsRelationFactory(sqlContext, metaClient, options, schemaSpec, isBootstrap) { override val fileIndex = new HoodieCDCFileIndex( sparkSession, metaClient, schemaSpec, options, FileStatusCache.getOrCreate(sparkSession), true, true) @@ -381,3 +380,4 @@ class HoodieCopyOnWriteCDCHadoopFsRelationFactory(override val sqlContext: SQLCo override def buildPartitionSchema(): StructType = StructType(Nil) } + diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala index c1ad8b528aeb..9577f74af7a0 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala @@ -146,8 +146,7 @@ override def supportBatch(sparkSession: SparkSession, schema: StructType): Boole val hoodieBaseFile = fileSlice.getBaseFile.get() baseFileReader(createPartitionedFile(fileSliceMapping.getPartitionValues, hoodieBaseFile.getHadoopPath, 0, hoodieBaseFile.getFileLen)) } else { - val readerContext = new SparkFileFormatInternalRowReaderContext(parquetFileReader.value, - tableState.recordKeyField, filters, shouldUseRecordPosition) + val readerContext = new SparkFileFormatInternalRowReaderContext(parquetFileReader.value, tableState.recordKeyField, filters) val serializedHadoopConf = broadcastedHadoopConf.value.value val metaClient: HoodieTableMetaClient = HoodieTableMetaClient .builder().setConf(serializedHadoopConf).setBasePath(tableState.tablePath).build diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala index 11ccebc469a5..c889282e7f52 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala @@ -89,7 +89,7 @@ class TestHoodieFileGroupReaderOnSpark extends TestHoodieFileGroupReaderBase[Int checkState(keyFields.length == 1) keyFields.head } - new SparkFileFormatInternalRowReaderContext(reader, recordKeyField, Seq.empty, false) + new SparkFileFormatInternalRowReaderContext(reader, recordKeyField, Seq.empty) } override def commitToTable(recordList: util.List[String], operation: String, options: util.Map[String, String]): Unit = { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala index 6180cfbad820..d3c5d9116d94 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala @@ -90,8 +90,9 @@ class TestSpark35RecordPositionMetadataColumn extends SparkClientFunctionalTestH _spark.sparkContext.hadoopConfiguration, basePath) val allBaseFiles = HoodieTestTable.of(metaClient).listAllBaseFiles assertTrue(allBaseFiles.nonEmpty) - val readerContext = new SparkFileFormatInternalRowReaderContext(reader, HoodieRecord.RECORD_KEY_METADATA_FIELD, Seq.empty, true) + val readerContext = new SparkFileFormatInternalRowReaderContext(reader, HoodieRecord.RECORD_KEY_METADATA_FIELD, Seq.empty) val readerState = readerContext.getReaderState + readerState.useRecordPosition = true readerState.hasLogFiles = true readerState.needsBootstrapMerge = false readerState.hasBootstrapBaseFile = false From 24be89663d1b95cf7db83dd39378a675a54b98fc Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 18 Apr 2024 14:06:12 -0400 Subject: [PATCH 69/86] fix most review comments --- .../BaseSparkInternalRowReaderContext.java | 21 ++++++++----------- .../common/engine/HoodieReaderContext.java | 9 +++++--- .../read/HoodieBaseFileGroupRecordBuffer.java | 2 +- .../reader/HoodieTestReaderContext.java | 5 ++++- .../hudi/common/util/ValidationUtils.java | 2 +- .../spark/sql/hudi/ddl/TestSpark3DDL.scala | 3 --- ...oodieDeltaStreamerSchemaEvolutionBase.java | 1 - 7 files changed, 21 insertions(+), 22 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java index 552d18ccd9b5..5fa4103a7b3a 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java @@ -142,18 +142,15 @@ private Object getFieldValueFromInternalRow(InternalRow row, Schema recordSchema } @Override - public UnaryOperator projectRecord(Schema from, Schema to) { - UnsafeProjection projection = HoodieInternalRowUtils.generateUnsafeProjectionAlias(getCachedSchema(from), getCachedSchema(to)); - return projection::apply; - } - - @Override - public UnaryOperator projectRecordUnsafe(Schema from, Schema to, Map renamedColumns) { - StructType structType = HoodieInternalRowUtils.getCachedSchema(from); - StructType newStructType = HoodieInternalRowUtils.getCachedSchema(to); - Function1 unsafeRowWriter = - HoodieInternalRowUtils.getCachedUnsafeRowWriter(structType, newStructType, renamedColumns); - return row -> (InternalRow) unsafeRowWriter.apply(row); + public UnaryOperator projectRecord(Schema from, Schema to, Map renamedColumns) { + if (renamedColumns.isEmpty()) { + UnsafeProjection projection = HoodieInternalRowUtils.generateUnsafeProjectionAlias(getCachedSchema(from), getCachedSchema(to)); + return projection::apply; + } else { + Function1 unsafeRowWriter = + HoodieInternalRowUtils.getCachedUnsafeRowWriter(getCachedSchema(from), getCachedSchema(to), renamedColumns); + return row -> (InternalRow) unsafeRowWriter.apply(row); + } } protected UnaryOperator getIdentityProjection() { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java index c0985b6b35cb..5495415c4a5b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java @@ -33,6 +33,7 @@ import org.apache.hadoop.fs.Path; import java.io.IOException; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.function.UnaryOperator; @@ -217,12 +218,14 @@ public abstract ClosableIterator mergeBootstrapReaders(ClosableIterator sk * * @param from the schema of records to be passed into UnaryOperator * @param to the schema of records produced by UnaryOperator + * @param renamedColumns map of renamed columns where the key is the new name from the query and + * the value is the old name that exists in the file * @return a function that takes in a record and returns the record with reordered columns */ - public abstract UnaryOperator projectRecord(Schema from, Schema to); + public abstract UnaryOperator projectRecord(Schema from, Schema to, Map renamedColumns); - public UnaryOperator projectRecordUnsafe(Schema from, Schema to, Map renamedColumns) { - throw new UnsupportedOperationException("Schema on read is not supported for this reader."); + public final UnaryOperator projectRecord(Schema from, Schema to) { + return projectRecord(from, to, Collections.emptyMap()); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java index c886f5dbbdef..fac76c51cf8b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java @@ -289,7 +289,7 @@ protected Option, Schema>> composeEvolvedSchemaTransformer( true, false, false).mergeSchemaGetRenamed(); Schema mergedAvroSchema = AvroInternalSchemaConverter.convert(mergedInternalSchema.getLeft(), readerSchema.getFullName()); assert mergedAvroSchema.equals(readerSchema); - return Option.of(Pair.of(readerContext.projectRecordUnsafe(dataBlock.getSchema(), mergedAvroSchema, mergedInternalSchema.getRight()), mergedAvroSchema)); + return Option.of(Pair.of(readerContext.projectRecord(dataBlock.getSchema(), mergedAvroSchema, mergedInternalSchema.getRight()), mergedAvroSchema)); } /** diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieTestReaderContext.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieTestReaderContext.java index 81691d27662c..cd4db9fb6086 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieTestReaderContext.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieTestReaderContext.java @@ -172,7 +172,10 @@ public ClosableIterator mergeBootstrapReaders(ClosableIterator projectRecord(Schema from, Schema to) { + public UnaryOperator projectRecord(Schema from, Schema to, Map renamedColumns) { + if (!renamedColumns.isEmpty()) { + throw new UnsupportedOperationException("Schema evolution is not supported for the test reader context"); + } Map fromFields = IntStream.range(0, from.getFields().size()) .boxed() .collect(Collectors.toMap( diff --git a/hudi-io/src/main/java/org/apache/hudi/common/util/ValidationUtils.java b/hudi-io/src/main/java/org/apache/hudi/common/util/ValidationUtils.java index b12ec9c46757..e2ad3862e1b6 100644 --- a/hudi-io/src/main/java/org/apache/hudi/common/util/ValidationUtils.java +++ b/hudi-io/src/main/java/org/apache/hudi/common/util/ValidationUtils.java @@ -79,7 +79,7 @@ public static void checkState(final boolean expression, String errorMessage) { public static void checkNotNull(Object o) { if (o == null) { - throw new IllegalStateException(); + throw new IllegalStateException("Value is null when it should be initialized"); } } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestSpark3DDL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestSpark3DDL.scala index afa462960adc..b2b390f56327 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestSpark3DDL.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestSpark3DDL.scala @@ -709,8 +709,6 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { } test("Test schema auto evolution") { - //This test will be flakey for mor until [HUDI-6798] is landed and we can set the merge mode - spark.sql("set hoodie.file.group.reader.enabled=false") withTempDir { tmp => Seq("COPY_ON_WRITE", "MERGE_ON_READ").foreach { tableType => // for complex schema. @@ -809,7 +807,6 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { } } } - spark.sqlContext.conf.unsetConf("hoodie.file.group.reader.enabled"); } test("Test DATE to STRING conversions when vectorized reading is not enabled") { diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionBase.java index 120154fc9db6..c6bf08f8a7c8 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerSchemaEvolutionBase.java @@ -156,7 +156,6 @@ protected HoodieDeltaStreamer.Config getDeltaStreamerConfig(String[] transformer protected HoodieDeltaStreamer.Config getDeltaStreamerConfig(String[] transformerClasses, boolean nullForDeletedCols, TypedProperties extraProps) throws IOException { - extraProps.setProperty(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), "true"); extraProps.setProperty("hoodie.datasource.write.table.type", tableType); extraProps.setProperty("hoodie.datasource.write.row.writer.enable", rowWriterEnable.toString()); extraProps.setProperty(DataSourceWriteOptions.SET_NULL_FOR_MISSING_COLUMNS().key(), Boolean.toString(nullForDeletedCols)); From d1fbbf60dc8b22199a534a59c4cbeccc8f795079 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 18 Apr 2024 16:29:59 -0400 Subject: [PATCH 70/86] address review comments --- ...rkFileFormatInternalRowReaderContext.scala | 8 +- .../common/engine/HoodieReaderContext.java | 84 ++++++++++++++++++- .../table/log/BaseHoodieLogRecordReader.java | 11 +-- .../log/HoodieMergedLogRecordReader.java | 6 +- .../read/HoodieBaseFileGroupRecordBuffer.java | 6 +- .../table/read/HoodieFileGroupReader.java | 37 ++++---- .../HoodieFileGroupReaderSchemaHandler.java | 17 +--- .../read/HoodieFileGroupReaderState.java | 37 -------- .../HoodiePositionBasedSchemaHandler.java | 4 +- .../hudi/common/util/ValidationUtils.java | 6 -- .../hudi/HoodieHadoopFsRelationFactory.scala | 26 +++--- ...ileGroupReaderBasedParquetFileFormat.scala | 29 ++----- ...diePositionBasedFileGroupRecordBuffer.java | 15 ++-- .../TestHoodieFileGroupReaderOnSpark.scala | 12 +-- ...tSpark35RecordPositionMetadataColumn.scala | 9 +- 15 files changed, 145 insertions(+), 162 deletions(-) delete mode 100644 hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderState.java diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala index 3847a1007fff..0fe9ee8bd121 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala @@ -94,9 +94,7 @@ class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkParquetRea } private def getSchemaAndFiltersForRead(structType: StructType): (StructType, Seq[Filter]) = { - (readerState.hasLogFiles.booleanValue(), - readerState.needsBootstrapMerge.booleanValue(), - readerState.useRecordPosition.booleanValue()) match { + (getHasLogFiles, getNeedsBootstrapMerge, getUseRecordPosition) match { case (false, false, _) => (structType, filters) case (false, true, true) => @@ -135,7 +133,7 @@ class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkParquetRea skeletonRequiredSchema: Schema, dataFileIterator: ClosableIterator[Any], dataRequiredSchema: Schema): ClosableIterator[InternalRow] = { - if (readerState.useRecordPosition) { + if (getUseRecordPosition) { assert(AvroSchemaUtils.containsFieldInSchema(skeletonRequiredSchema, ROW_INDEX_TEMPORARY_COLUMN_NAME)) assert(AvroSchemaUtils.containsFieldInSchema(dataRequiredSchema, ROW_INDEX_TEMPORARY_COLUMN_NAME)) val javaSet = new java.util.HashSet[String]() @@ -144,7 +142,7 @@ class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkParquetRea AvroSchemaUtils.removeFieldsFromSchema(skeletonRequiredSchema, javaSet)) //If we have log files, we will want to do position based merging with those as well, //so leave the row index column at the end - val dataProjection = if (readerState.hasLogFiles) { + val dataProjection = if (getHasLogFiles) { getIdentityProjection } else { projectRecord(dataRequiredSchema, diff --git a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java index 5495415c4a5b..6ff5d59cb7f5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java @@ -22,7 +22,7 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordMerger; -import org.apache.hudi.common.table.read.HoodieFileGroupReaderState; +import org.apache.hudi.common.table.read.HoodieFileGroupReaderSchemaHandler; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ClosableIterator; @@ -50,10 +50,86 @@ * and {@code RowData} in Flink. */ public abstract class HoodieReaderContext { - protected HoodieFileGroupReaderState readerState = new HoodieFileGroupReaderState<>(); - public HoodieFileGroupReaderState getReaderState() { - return readerState; + private HoodieFileGroupReaderSchemaHandler schemaHandler = null; + private String tablePath = null; + private String latestCommitTime = null; + private HoodieRecordMerger recordMerger = null; + private Boolean hasLogFiles = null; + private Boolean hasBootstrapBaseFile = null; + private Boolean needsBootstrapMerge = null; + private Boolean useRecordPosition = null; + + // Getter and Setter for schemaHandler + public HoodieFileGroupReaderSchemaHandler getSchemaHandler() { + return schemaHandler; + } + + public void setSchemaHandler(HoodieFileGroupReaderSchemaHandler schemaHandler) { + this.schemaHandler = schemaHandler; + } + + public String getTablePath() { + if (tablePath == null) { + throw new IllegalStateException("Table path not set in reader context."); + } + return tablePath; + } + + public void setTablePath(String tablePath) { + this.tablePath = tablePath; + } + + public String getLatestCommitTime() { + return latestCommitTime; + } + + public void setLatestCommitTime(String latestCommitTime) { + this.latestCommitTime = latestCommitTime; + } + + public HoodieRecordMerger getRecordMerger() { + return recordMerger; + } + + public void setRecordMerger(HoodieRecordMerger recordMerger) { + this.recordMerger = recordMerger; + } + + // Getter and Setter for hasLogFiles + public boolean getHasLogFiles() { + return hasLogFiles; + } + + public void setHasLogFiles(boolean hasLogFiles) { + this.hasLogFiles = hasLogFiles; + } + + // Getter and Setter for hasBootstrapBaseFile + public boolean getHasBootstrapBaseFile() { + return hasBootstrapBaseFile; + } + + public void setHasBootstrapBaseFile(boolean hasBootstrapBaseFile) { + this.hasBootstrapBaseFile = hasBootstrapBaseFile; + } + + // Getter and Setter for needsBootstrapMerge + public boolean getNeedsBootstrapMerge() { + return needsBootstrapMerge; + } + + public void setNeedsBootstrapMerge(boolean needsBootstrapMerge) { + this.needsBootstrapMerge = needsBootstrapMerge; + } + + // Getter and Setter for useRecordPosition + public boolean getUseRecordPosition() { + return useRecordPosition; + } + + public void setUseRecordPosition(boolean useRecordPosition) { + this.useRecordPosition = useRecordPosition; } // These internal key names are only used in memory for record metadata and merging, diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java index dd084c94e519..3e09f29e18b1 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java @@ -31,7 +31,6 @@ import org.apache.hudi.common.table.log.block.HoodieDataBlock; import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock; -import org.apache.hudi.common.table.read.HoodieFileGroupReaderState; import org.apache.hudi.common.table.read.HoodieFileGroupRecordBuffer; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; @@ -86,7 +85,6 @@ public abstract class BaseHoodieLogRecordReader { // Log-Blocks belonging to inflight delta-instants are filtered-out using this high-watermark. private final String latestInstantTime; protected final HoodieReaderContext readerContext; - protected final HoodieFileGroupReaderState readerState; protected final HoodieTableMetaClient hoodieTableMetaClient; // Merge strategy to use when combining records from log private final String payloadClassFQN; @@ -152,10 +150,9 @@ protected BaseHoodieLogRecordReader(HoodieReaderContext readerContext, HoodieRecordMerger recordMerger, HoodieFileGroupRecordBuffer recordBuffer) { this.readerContext = readerContext; - this.readerState = readerContext.getReaderState(); - this.readerSchema = readerState.schemaHandler.getRequiredSchema(); - this.latestInstantTime = readerState.latestCommitTime; - this.hoodieTableMetaClient = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(readerState.tablePath).build(); + this.readerSchema = readerContext.getSchemaHandler().getRequiredSchema(); + this.latestInstantTime = readerContext.getLatestCommitTime(); + this.hoodieTableMetaClient = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(readerContext.getTablePath()).build(); // load class from the payload fully qualified class name HoodieTableConfig tableConfig = this.hoodieTableMetaClient.getTableConfig(); this.payloadClassFQN = tableConfig.getPayloadClass(); @@ -175,7 +172,7 @@ protected BaseHoodieLogRecordReader(HoodieReaderContext readerContext, this.instantRange = instantRange; this.withOperationField = withOperationField; this.forceFullScan = forceFullScan; - this.internalSchema = readerState.schemaHandler.getInternalSchema(); + this.internalSchema = readerContext.getSchemaHandler().getInternalSchema(); this.enableOptimizedLogBlocksScan = enableOptimizedLogBlocksScan; if (keyFieldOverride.isPresent()) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordReader.java index 2972fa29d998..dd0bbb3d6c8e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordReader.java @@ -24,7 +24,6 @@ import org.apache.hudi.common.model.HoodiePreCombineAvroRecordMerger; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordMerger; -import org.apache.hudi.common.table.read.HoodieFileGroupReaderState; import org.apache.hudi.common.table.read.HoodieFileGroupRecordBuffer; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.HoodieRecordUtils; @@ -213,10 +212,8 @@ public void close() { */ public static class Builder extends BaseHoodieLogRecordReader.Builder { private HoodieReaderContext readerContext; - private HoodieFileGroupReaderState readerState; private FileSystem fs; private List logFilePaths; - private String latestInstantTime; private boolean reverseReader; private int bufferSize; // specific configurations @@ -316,9 +313,8 @@ public HoodieMergedLogRecordReader build() { ValidationUtils.checkArgument(recordMerger != null); ValidationUtils.checkArgument(recordBuffer != null); ValidationUtils.checkArgument(readerContext != null); - this.readerState = readerContext.getReaderState(); if (this.partitionName == null && CollectionUtils.nonEmpty(this.logFilePaths)) { - this.partitionName = getRelativePartitionPath(new Path(readerState.tablePath), new Path(this.logFilePaths.get(0)).getParent()); + this.partitionName = getRelativePartitionPath(new Path(readerContext.getTablePath()), new Path(this.logFilePaths.get(0)).getParent()); } return new HoodieMergedLogRecordReader<>( diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java index fac76c51cf8b..08660906ee72 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java @@ -62,7 +62,6 @@ public abstract class HoodieBaseFileGroupRecordBuffer implements HoodieFileGroupRecordBuffer { protected final HoodieReaderContext readerContext; - protected final HoodieFileGroupReaderState readerState; protected final Schema readerSchema; protected final Option partitionNameOverrideOpt; protected final Option partitionPathFieldOpt; @@ -87,13 +86,12 @@ public HoodieBaseFileGroupRecordBuffer(HoodieReaderContext readerContext, ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled) { this.readerContext = readerContext; - this.readerState = readerContext.getReaderState(); - this.readerSchema = readerState.schemaHandler.getRequiredSchema(); + this.readerSchema = readerContext.getSchemaHandler().getRequiredSchema(); this.partitionNameOverrideOpt = partitionNameOverrideOpt; this.partitionPathFieldOpt = partitionPathFieldOpt; this.recordMerger = recordMerger; this.payloadProps = payloadProps; - this.internalSchema = readerState.schemaHandler.getInternalSchema(); + this.internalSchema = readerContext.getSchemaHandler().getInternalSchema(); this.hoodieTableMetaClient = hoodieTableMetaClient; try { // Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java index cc287563bdd6..fcf29aa77095 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java @@ -31,7 +31,6 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieMergedLogRecordReader; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.CachingIterator; import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.common.util.collection.EmptyIterator; @@ -75,11 +74,9 @@ public final class HoodieFileGroupReader implements Closeable { private final long length; // Core structure to store and process records. private final HoodieFileGroupRecordBuffer recordBuffer; - private final HoodieFileGroupReaderState readerState; private ClosableIterator baseFileIterator; private final HoodieRecordMerger recordMerger; private final Option> outputConverter; - private final HoodieFileGroupReaderSchemaHandler schemaHandler; public HoodieFileGroupReader(HoodieReaderContext readerContext, Configuration hadoopConf, @@ -100,7 +97,6 @@ public HoodieFileGroupReader(HoodieReaderContext readerContext, ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled) { this.readerContext = readerContext; - this.readerState = readerContext.getReaderState(); this.hadoopConf = hadoopConf; this.hoodieBaseFileOption = fileSlice.getBaseFile(); this.logFiles = fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); @@ -108,17 +104,16 @@ public HoodieFileGroupReader(HoodieReaderContext readerContext, this.start = start; this.length = length; this.recordMerger = readerContext.getRecordMerger(tableConfig.getRecordMergerStrategy()); - this.readerState.recordMerger = this.recordMerger; - this.readerState.tablePath = tablePath; - this.readerState.latestCommitTime = latestCommitTime; - readerState.useRecordPosition = shouldUseRecordPosition; - readerState.hasLogFiles = !this.logFiles.isEmpty(); - readerState.hasBootstrapBaseFile = hoodieBaseFileOption.isPresent() && hoodieBaseFileOption.get().getBootstrapBaseFile().isPresent(); - readerState.schemaHandler = shouldUseRecordPosition + readerContext.setRecordMerger(this.recordMerger); + readerContext.setTablePath(tablePath); + readerContext.setLatestCommitTime(latestCommitTime); + readerContext.setUseRecordPosition(shouldUseRecordPosition); + readerContext.setHasLogFiles(!this.logFiles.isEmpty()); + readerContext.setHasBootstrapBaseFile(hoodieBaseFileOption.isPresent() && hoodieBaseFileOption.get().getBootstrapBaseFile().isPresent()); + readerContext.setSchemaHandler(shouldUseRecordPosition ? new HoodiePositionBasedSchemaHandler<>(readerContext, dataSchema, requestedSchema, internalSchemaOpt, tableConfig) - : new HoodieFileGroupReaderSchemaHandler<>(readerContext, dataSchema, requestedSchema, internalSchemaOpt, tableConfig); - this.schemaHandler = readerState.schemaHandler; - this.outputConverter = schemaHandler.getOutputConverter(); + : new HoodieFileGroupReaderSchemaHandler<>(readerContext, dataSchema, requestedSchema, internalSchemaOpt, tableConfig)); + this.outputConverter = readerContext.getSchemaHandler().getOutputConverter(); this.recordBuffer = this.logFiles.isEmpty() ? null : shouldUseRecordPosition @@ -153,13 +148,14 @@ private ClosableIterator makeBaseFileIterator() throws IOException { } return readerContext.getFileRecordIterator(baseFile.getHadoopPath(), start, length, - schemaHandler.getDataSchema(), schemaHandler.getRequiredSchema(), hadoopConf); + readerContext.getSchemaHandler().getDataSchema(), + readerContext.getSchemaHandler().getRequiredSchema(), hadoopConf); } private ClosableIterator makeBootstrapBaseFileIterator(HoodieBaseFile baseFile) throws IOException { BaseFile dataFile = baseFile.getBootstrapBaseFile().get(); - Pair,List> requiredFields = schemaHandler.getBootstrapRequiredFields(); - Pair,List> allFields = schemaHandler.getBootstrapDataFields(); + Pair,List> requiredFields = readerContext.getSchemaHandler().getBootstrapRequiredFields(); + Pair,List> allFields = readerContext.getSchemaHandler().getBootstrapDataFields(); Option,Schema>> dataFileIterator = makeBootstrapBaseFileIteratorHelper(requiredFields.getRight(), allFields.getRight(), dataFile); Option,Schema>> skeletonFileIterator = @@ -182,9 +178,9 @@ private Option,Schema>> makeBootstrapBaseFileIteratorHe if (requiredFields.isEmpty()) { return Option.empty(); } - Schema requiredSchema = schemaHandler.createSchemaFromFields(requiredFields); + Schema requiredSchema = readerContext.getSchemaHandler().createSchemaFromFields(requiredFields); return Option.of(Pair.of(readerContext.getFileRecordIterator(file.getHadoopPath(), 0, file.getFileLen(), - schemaHandler.createSchemaFromFields(allFields), requiredSchema, hadoopConf), requiredSchema)); + readerContext.getSchemaHandler().createSchemaFromFields(allFields), requiredSchema, hadoopConf), requiredSchema)); } /** @@ -211,8 +207,7 @@ public T next() { } private void scanLogFiles() { - ValidationUtils.checkNotNull(readerState.tablePath); - String path = readerState.tablePath; + String path = readerContext.getTablePath(); FileSystem fs = readerContext.getFs(path, hadoopConf); HoodieMergedLogRecordReader logRecordReader = HoodieMergedLogRecordReader.newBuilder() diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java index e0273b26aacc..41bfd3bf1685 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java @@ -24,7 +24,6 @@ import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter; @@ -45,10 +44,6 @@ public class HoodieFileGroupReaderSchemaHandler { - protected HoodieFileGroupReaderState readerState; - - - protected final Schema dataSchema; // requestedSchema: the schema that the caller requests @@ -77,19 +72,15 @@ public HoodieFileGroupReaderSchemaHandler(HoodieReaderContext readerContext, Option internalSchemaOpt, HoodieTableConfig hoodieTableConfig) { this.readerContext = readerContext; - this.readerState = readerContext.getReaderState(); - ValidationUtils.checkNotNull(readerState.hasBootstrapBaseFile); - this.hasBootstrapBaseFile = readerState.hasBootstrapBaseFile; - ValidationUtils.checkNotNull(readerState.hasLogFiles); - this.needsMORMerge = readerState.hasLogFiles; - ValidationUtils.checkNotNull(readerState.recordMerger); - this.recordMerger = readerState.recordMerger; + this.hasBootstrapBaseFile = readerContext.getHasBootstrapBaseFile(); + this.needsMORMerge = readerContext.getHasLogFiles(); + this.recordMerger = readerContext.getRecordMerger(); this.dataSchema = dataSchema; this.requestedSchema = requestedSchema; this.hoodieTableConfig = hoodieTableConfig; this.requiredSchema = prepareSchema(); this.internalSchema = pruneInternalSchema(requiredSchema, internalSchemaOpt); - readerState.needsBootstrapMerge = this.needsBootstrapMerge; + readerContext.setNeedsBootstrapMerge(this.needsBootstrapMerge); } public Schema getDataSchema() { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderState.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderState.java deleted file mode 100644 index ea3f816b5497..000000000000 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderState.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.hudi.common.table.read; - -import org.apache.hudi.common.model.HoodieRecordMerger; - -/** - * A class holding the state that is needed by {@code HoodieFileGroupReader}, - * e.g., schema, merging strategy, etc. - */ -public class HoodieFileGroupReaderState { - public HoodieFileGroupReaderSchemaHandler schemaHandler = null; - public String tablePath = null; - public String latestCommitTime = null; - public HoodieRecordMerger recordMerger = null; - public Boolean hasLogFiles = null; - public Boolean hasBootstrapBaseFile = null; - public Boolean needsBootstrapMerge = null; - public Boolean useRecordPosition = null; -} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedSchemaHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedSchemaHandler.java index a4940ca45653..48dfe2f9dbd4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedSchemaHandler.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedSchemaHandler.java @@ -44,7 +44,7 @@ public HoodiePositionBasedSchemaHandler(HoodieReaderContext readerContext, @Override protected Schema prepareSchema() { Schema preMergeSchema = super.prepareSchema(); - return readerState.hasLogFiles + return readerContext.getHasLogFiles() ? addPositionalMergeCol(preMergeSchema) : preMergeSchema; } @@ -60,7 +60,7 @@ private Schema.Field getPositionalMergeField() { @Override public Schema createSchemaFromFields(List fields) { - if (readerState.hasLogFiles) { + if (readerContext.getHasLogFiles()) { fields.add(getPositionalMergeField()); } return super.createSchemaFromFields(fields); diff --git a/hudi-io/src/main/java/org/apache/hudi/common/util/ValidationUtils.java b/hudi-io/src/main/java/org/apache/hudi/common/util/ValidationUtils.java index e2ad3862e1b6..3350c9a86081 100644 --- a/hudi-io/src/main/java/org/apache/hudi/common/util/ValidationUtils.java +++ b/hudi-io/src/main/java/org/apache/hudi/common/util/ValidationUtils.java @@ -76,10 +76,4 @@ public static void checkState(final boolean expression, String errorMessage) { throw new IllegalStateException(errorMessage); } } - - public static void checkNotNull(Object o) { - if (o == null) { - throw new IllegalStateException("Value is null when it should be initialized"); - } - } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala index d8c3d3776344..10d254f3e4bb 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala @@ -25,7 +25,6 @@ import org.apache.hadoop.mapred.JobConf import org.apache.hudi.HoodieBaseRelation.{convertToAvroSchema, isSchemaEvolutionEnabledOnRead} import org.apache.hudi.HoodieConversionUtils.toScalaOption import org.apache.hudi.HoodieFileIndex.getConfigProperties -import org.apache.hudi.client.utils.SparkInternalSchemaConverter import org.apache.hudi.common.config.HoodieMetadataConfig.{DEFAULT_METADATA_ENABLE_FOR_READERS, ENABLE} import org.apache.hudi.common.config.{ConfigProperty, HoodieMetadataConfig, HoodieReaderConfig, TypedProperties} import org.apache.hudi.common.model.HoodieRecord @@ -36,7 +35,6 @@ import org.apache.hudi.common.util.{ConfigUtils, StringUtils} import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter -import org.apache.hudi.internal.schema.utils.SerDeHelper import org.apache.hudi.metadata.HoodieTableMetadataUtil import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.catalog.BucketSpec @@ -65,7 +63,7 @@ abstract class HoodieBaseHadoopFsRelationFactory(val sqlContext: SQLContext, val schemaSpec: Option[StructType] ) extends SparkAdapterSupport with HoodieHadoopFsRelationFactory { protected lazy val sparkSession: SparkSession = sqlContext.sparkSession - protected var optParams: Map[String, String] = options + protected lazy val optParams: Map[String, String] = options protected lazy val hadoopConfig: Configuration = new Configuration(sqlContext.sparkContext.hadoopConfiguration) protected lazy val jobConf = new JobConf(hadoopConfig) @@ -86,13 +84,7 @@ abstract class HoodieBaseHadoopFsRelationFactory(val sqlContext: SQLContext, specifiedQueryTimestamp.map(schemaResolver.getTableInternalSchemaFromCommitMetadata) .getOrElse(schemaResolver.getTableInternalSchemaFromCommitMetadata) } match { - case Success(internalSchemaOpt) => - if (internalSchemaOpt.isPresent) { - optParams = optParams + (SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA -> SerDeHelper.toJson(internalSchemaOpt.get())) - optParams = optParams + (SparkInternalSchemaConverter.HOODIE_TABLE_PATH -> basePath.toString) - optParams = optParams + (SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST -> timeline.getInstants.iterator.asScala.map(_.getFileName).mkString(",")) - } - toScalaOption(internalSchemaOpt) + case Success(internalSchemaOpt) => toScalaOption(internalSchemaOpt) case Failure(e) => None } @@ -115,6 +107,12 @@ abstract class HoodieBaseHadoopFsRelationFactory(val sqlContext: SQLContext, (avroSchema, internalSchemaOpt) } + protected lazy val validCommits: String = if (internalSchemaOpt.nonEmpty) { + timeline.getInstants.iterator.asScala.map(_.getFileName).mkString(",") + } else { + "" + } + protected lazy val tableStructSchema: StructType = { val converted = AvroConversionUtils.convertAvroSchemaToStructType(tableAvroSchema) val metaFieldMetadata = sparkAdapter.createCatalystMetadataForMetaField @@ -244,7 +242,7 @@ class HoodieMergeOnReadSnapshotHadoopFsRelationFactory(override val sqlContext: new HoodieFileGroupReaderBasedParquetFileFormat( tableState, HoodieTableSchema(tableStructSchema, tableAvroSchema.toString, internalSchemaOpt), metaClient.getTableConfig.getTableName, mergeType, mandatoryFields, - true, isBootstrap, false, shouldUseRecordPosition, Seq.empty) + true, false, validCommits, shouldUseRecordPosition, Seq.empty) } } @@ -288,7 +286,7 @@ class HoodieMergeOnReadIncrementalHadoopFsRelationFactory(override val sqlContex new HoodieFileGroupReaderBasedParquetFileFormat( tableState, HoodieTableSchema(tableStructSchema, tableAvroSchema.toString, internalSchemaOpt), metaClient.getTableConfig.getTableName, mergeType, mandatoryFields, - true, isBootstrap, true, shouldUseRecordPosition, fileIndex.getRequiredFilters) + true, true, validCommits, shouldUseRecordPosition, fileIndex.getRequiredFilters) } } } @@ -319,7 +317,7 @@ class HoodieCopyOnWriteSnapshotHadoopFsRelationFactory(override val sqlContext: new HoodieFileGroupReaderBasedParquetFileFormat( tableState, HoodieTableSchema(tableStructSchema, tableAvroSchema.toString, internalSchemaOpt), metaClient.getTableConfig.getTableName, mergeType, mandatoryFields, - false, isBootstrap, false, shouldUseRecordPosition, Seq.empty) + false, false, validCommits, shouldUseRecordPosition, Seq.empty) } } } @@ -347,7 +345,7 @@ class HoodieCopyOnWriteIncrementalHadoopFsRelationFactory(override val sqlContex new HoodieFileGroupReaderBasedParquetFileFormat( tableState, HoodieTableSchema(tableStructSchema, tableAvroSchema.toString, internalSchemaOpt), metaClient.getTableConfig.getTableName, mergeType, mandatoryFields, - false, isBootstrap, true, shouldUseRecordPosition, fileIndex.getRequiredFilters) + false, true, validCommits, shouldUseRecordPosition, fileIndex.getRequiredFilters) } } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala index 9577f74af7a0..5f3df644ee34 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala @@ -30,6 +30,7 @@ import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} import org.apache.hudi.common.util.FileIOUtils import org.apache.hudi.common.util.collection.ExternalSpillableMap.DiskMapType import org.apache.hudi.internal.schema.InternalSchema +import org.apache.hudi.internal.schema.utils.SerDeHelper import org.apache.hudi.{AvroConversionUtils, HoodieFileIndex, HoodiePartitionCDCFileGroupMapping, HoodiePartitionFileSliceMapping, HoodieTableSchema, HoodieTableState, SparkAdapterSupport, SparkFileFormatInternalRowReaderContext} import org.apache.spark.sql.HoodieCatalystExpressionUtils.generateUnsafeProjection import org.apache.spark.sql.SparkSession @@ -60,8 +61,8 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, mergeType: String, mandatoryFields: Seq[String], isMOR: Boolean, - isBootstrap: Boolean, isIncremental: Boolean, + validCommits: String, shouldUseRecordPosition: Boolean, requiredFilters: Seq[Filter] ) extends ParquetFileFormat with SparkAdapterSupport with HoodieFormatTrait { @@ -70,22 +71,6 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, private val sanitizedTableName = AvroSchemaUtils.getAvroRecordQualifiedName(tableName) - /** - * Support batch needs to remain consistent, even if one side of a bootstrap merge can support - * while the other side can't - */ - /* -private var supportBatchCalled = false -private var supportBatchResult = false - -override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { - if (!supportBatchCalled || supportBatchResult) { - supportBatchCalled = true - supportBatchResult = tableSchema.internalSchema.isEmpty && !isMOR && !isIncremental && !isBootstrap && super.supportBatch(sparkSession, schema) - } - supportBatchResult -} - */ override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = false private val supportBatchResult = false @@ -114,7 +99,7 @@ override def supportBatch(sparkSession: SparkSession, schema: StructType): Boole spark.conf.set("spark.sql.parquet.enableVectorizedReader", supportBatchResult) val isCount = requiredSchema.isEmpty && !isMOR && !isIncremental val augmentedHadoopConf = FSUtils.buildInlineConf(hadoopConf) - setSchemaEvolutionConfigs(augmentedHadoopConf, options) + setSchemaEvolutionConfigs(augmentedHadoopConf) val baseFileReader = super.buildReaderWithPartitionValues(spark, dataSchema, partitionSchema, requiredSchema, filters ++ requiredFilters, options, new Configuration(augmentedHadoopConf)) val cdcFileReader = super.buildReaderWithPartitionValues( @@ -195,11 +180,11 @@ override def supportBatch(sparkSession: SparkSession, schema: StructType): Boole } } - protected def setSchemaEvolutionConfigs(conf: Configuration, options: Map[String, String]): Unit = { + protected def setSchemaEvolutionConfigs(conf: Configuration): Unit = { if (internalSchemaOpt.isPresent) { - options.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA).foreach(s => conf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, s)) - options.get(SparkInternalSchemaConverter.HOODIE_TABLE_PATH).foreach(s => conf.set(SparkInternalSchemaConverter.HOODIE_TABLE_PATH, s)) - options.get(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST).foreach(s => conf.set(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST, s)) + conf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, SerDeHelper.toJson(internalSchemaOpt.get())) + conf.set(SparkInternalSchemaConverter.HOODIE_TABLE_PATH, tableState.tablePath) + conf.set(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST, validCommits) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodiePositionBasedFileGroupRecordBuffer.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodiePositionBasedFileGroupRecordBuffer.java index 53ddb4fc3a0f..0573d424204e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodiePositionBasedFileGroupRecordBuffer.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodiePositionBasedFileGroupRecordBuffer.java @@ -30,7 +30,6 @@ import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock; -import org.apache.hudi.common.table.read.HoodieFileGroupReaderState; import org.apache.hudi.common.table.read.HoodiePositionBasedFileGroupRecordBuffer; import org.apache.hudi.common.table.read.HoodiePositionBasedSchemaHandler; import org.apache.hudi.common.table.read.TestHoodieFileGroupReaderOnSpark; @@ -101,18 +100,18 @@ public void prepareBuffer(boolean useCustomMerger) throws Exception { ? Option.empty() : Option.of(partitionPaths[0]); HoodieReaderContext ctx = getHoodieReaderContext(getBasePath(), avroSchema, getHadoopConf()); - HoodieFileGroupReaderState state = ctx.getReaderState(); - state.hasBootstrapBaseFile = false; - state.hasLogFiles = true; - state.needsBootstrapMerge = false; - state.recordMerger = useCustomMerger ? new CustomMerger() : new HoodieSparkRecordMerger(); - state.schemaHandler = new HoodiePositionBasedSchemaHandler(ctx, avroSchema, avroSchema, Option.empty(), metaClient.getTableConfig()); + ctx.setHasBootstrapBaseFile(false); + ctx.setHasLogFiles(true); + ctx.setNeedsBootstrapMerge(false); + ctx.setRecordMerger(useCustomMerger ? new CustomMerger() : new HoodieSparkRecordMerger()); + ctx.setSchemaHandler(new HoodiePositionBasedSchemaHandler<>(ctx, avroSchema, avroSchema, + Option.empty(), metaClient.getTableConfig())); buffer = new HoodiePositionBasedFileGroupRecordBuffer<>( ctx, metaClient, partitionNameOpt, partitionFields, - state.recordMerger, + ctx.getRecordMerger(), new TypedProperties(), 1024 * 1024 * 1000, metaClient.getTempFolderPath(), diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala index c889282e7f52..68514a59fd63 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala @@ -24,10 +24,10 @@ import org.apache.hadoop.conf.Configuration import org.apache.hudi.common.config.HoodieReaderConfig.FILE_GROUP_READER_ENABLED import org.apache.hudi.common.engine.HoodieReaderContext import org.apache.hudi.common.fs.FSUtils -import org.apache.hudi.common.model.{HoodieRecord, WriteOperationType} +import org.apache.hudi.common.model.{HoodieRecord, HoodieRecordMerger, WriteOperationType} import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.util.ValidationUtils.checkState -import org.apache.hudi.{SparkAdapterSupport, SparkFileFormatInternalRowReaderContext} +import org.apache.hudi.{HoodieSparkRecordMerger, SparkAdapterSupport, SparkFileFormatInternalRowReaderContext} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.functions.col import org.apache.spark.sql.{Dataset, HoodieInternalRowUtils, HoodieUnsafeUtils, Row, SaveMode, SparkSession} @@ -82,13 +82,7 @@ class TestHoodieFileGroupReaderOnSpark extends TestHoodieFileGroupReaderBase[Int override def getHoodieReaderContext(tablePath: String, avroSchema: Schema, hadoopConf: Configuration): HoodieReaderContext[InternalRow] = { val reader = sparkAdapter.createParquetFileReader(vectorized = false, spark.sessionState.conf, Map.empty, hadoopConf) val metaClient = HoodieTableMetaClient.builder().setConf(getHadoopConf).setBasePath(tablePath).build - val recordKeyField = if (metaClient.getTableConfig.populateMetaFields()) { - HoodieRecord.RECORD_KEY_METADATA_FIELD - } else { - val keyFields = metaClient.getTableConfig.getRecordKeyFields.get() - checkState(keyFields.length == 1) - keyFields.head - } + val recordKeyField = new HoodieSparkRecordMerger().getMandatoryFieldsForMerging(metaClient.getTableConfig)(0) new SparkFileFormatInternalRowReaderContext(reader, recordKeyField, Seq.empty) } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala index d3c5d9116d94..42d182c29267 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala @@ -91,11 +91,10 @@ class TestSpark35RecordPositionMetadataColumn extends SparkClientFunctionalTestH val allBaseFiles = HoodieTestTable.of(metaClient).listAllBaseFiles assertTrue(allBaseFiles.nonEmpty) val readerContext = new SparkFileFormatInternalRowReaderContext(reader, HoodieRecord.RECORD_KEY_METADATA_FIELD, Seq.empty) - val readerState = readerContext.getReaderState - readerState.useRecordPosition = true - readerState.hasLogFiles = true - readerState.needsBootstrapMerge = false - readerState.hasBootstrapBaseFile = false + readerContext.setUseRecordPosition(true) + readerContext.setHasLogFiles(true) + readerContext.setNeedsBootstrapMerge(false) + readerContext.setHasBootstrapBaseFile(false) //dataschema param is set to null because it is not used val fileRecordIterator = readerContext.getFileRecordIterator(allBaseFiles.head.getPath, 0, allBaseFiles.head.getLen, null, sparkAdapter.getAvroSchemaConverters.toAvroType(dataSchema, nullable = true, "record"), hadoopConf) From 89078f34a2dafff26d47d8a201a59d8bf8a540ba Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 18 Apr 2024 18:23:44 -0400 Subject: [PATCH 71/86] use more complex projection --- .../hudi/BaseSparkInternalRowReaderContext.java | 15 ++++----------- .../apache/spark/sql/HoodieInternalRowUtils.scala | 7 ------- 2 files changed, 4 insertions(+), 18 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java index 13ae3aefa172..dabe9cbfd1f9 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java @@ -37,13 +37,10 @@ import org.apache.spark.sql.HoodieInternalRowUtils; import org.apache.spark.sql.HoodieUnsafeRowUtils; import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.expressions.UnsafeProjection; import org.apache.spark.sql.catalyst.expressions.UnsafeRow; import org.apache.spark.sql.types.StructType; -import java.util.Collections; import java.util.Map; -import java.util.function.Function; import java.util.function.UnaryOperator; import scala.Function1; @@ -143,14 +140,10 @@ private Object getFieldValueFromInternalRow(InternalRow row, Schema recordSchema @Override public UnaryOperator projectRecord(Schema from, Schema to, Map renamedColumns) { - if (renamedColumns.isEmpty()) { - UnsafeProjection projection = HoodieInternalRowUtils.generateUnsafeProjectionAlias(getCachedSchema(from), getCachedSchema(to)); - return projection::apply; - } else { - Function1 unsafeRowWriter = - HoodieInternalRowUtils.getCachedUnsafeRowWriter(getCachedSchema(from), getCachedSchema(to), renamedColumns); - return row -> (InternalRow) unsafeRowWriter.apply(row); - } + Function1 unsafeRowWriter = + HoodieInternalRowUtils.getCachedUnsafeRowWriter(getCachedSchema(from), getCachedSchema(to), renamedColumns); + return row -> (InternalRow) unsafeRowWriter.apply(row); + } protected UnaryOperator getIdentityProjection() { diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieInternalRowUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieInternalRowUtils.scala index afeb9969c5f7..d5831be7d916 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieInternalRowUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieInternalRowUtils.scala @@ -73,13 +73,6 @@ object HoodieInternalRowUtils { .getOrElseUpdate((from, to), generateUnsafeProjection(from, to)) } - /** - * due to scala2.11 and HoodieCatalystExpressionUtils is both an object and trait, - * we can't directly call generateUnsafeProjection from java code - */ - def generateUnsafeProjectionAlias(from: StructType, to: StructType): UnsafeProjection = { - generateUnsafeProjection(from, to) - } /** * Provides cached instance of [[UnsafeRowWriter]] transforming provided [[InternalRow]]s from * one [[StructType]] and into another [[StructType]] From 879e07c167692250636215e06e67b6c370496c03 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Mon, 29 Apr 2024 19:56:26 -0400 Subject: [PATCH 72/86] add batch support --- .../org/apache/hudi/HoodieFileIndex.scala | 5 ++ .../hudi/HoodieHadoopFsRelationFactory.scala | 8 ++-- ...ileGroupReaderBasedParquetFileFormat.scala | 48 +++++++++++++------ .../apache/hudi/TestHoodieParquetBloom.scala | 15 ++++-- 4 files changed, 53 insertions(+), 23 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index f4ecbc44cd85..443a34bc269f 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -112,6 +112,9 @@ case class HoodieFileIndex(spark: SparkSession, override def rootPaths: Seq[Path] = getQueryPaths.asScala.map(e => new Path(e.toUri)) + // if true, fg reader can attempt to enable batch reading + var canEnableBatch = false + /** * Returns the FileStatus for all the base files (excluding log files). This should be used only for * cases where Spark directly fetches the list of files via HoodieFileIndex or for read optimized query logic @@ -152,6 +155,7 @@ case class HoodieFileIndex(spark: SparkSession, * @return list of PartitionDirectory containing partition to base files mapping */ override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { + canEnableBatch = true val prunedPartitionsAndFilteredFileSlices = filterFileSlices(dataFilters, partitionFilters).map { case (partitionOpt, fileSlices) => if (shouldEmbedFileSlices) { @@ -170,6 +174,7 @@ case class HoodieFileIndex(spark: SparkSession, || (f.getBaseFile.isPresent && f.getBaseFile.get().getBootstrapBaseFile.isPresent)). foldLeft(Map[String, FileSlice]()) { (m, f) => m + (f.getFileId -> f) } if (c.nonEmpty) { + canEnableBatch = false sparkAdapter.getSparkPartitionedFileUtils.newPartitionDirectory( new HoodiePartitionFileSliceMapping(InternalRow.fromSeq(partitionOpt.get.values), c), baseFileStatusesAndLogFileOnly) } else { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala index d2b6d76e23df..7d5dcccb084f 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala @@ -244,7 +244,7 @@ class HoodieMergeOnReadSnapshotHadoopFsRelationFactory(override val sqlContext: new HoodieFileGroupReaderBasedParquetFileFormat( tableState, HoodieTableSchema(tableStructSchema, tableAvroSchema.toString, internalSchemaOpt), metaClient.getTableConfig.getTableName, mergeType, mandatoryFields, - true, false, validCommits, shouldUseRecordPosition, Seq.empty) + true, false, validCommits, shouldUseRecordPosition, Seq.empty, fileIndex) } } @@ -288,7 +288,7 @@ class HoodieMergeOnReadIncrementalHadoopFsRelationFactory(override val sqlContex new HoodieFileGroupReaderBasedParquetFileFormat( tableState, HoodieTableSchema(tableStructSchema, tableAvroSchema.toString, internalSchemaOpt), metaClient.getTableConfig.getTableName, mergeType, mandatoryFields, - true, true, validCommits, shouldUseRecordPosition, fileIndex.getRequiredFilters) + true, true, validCommits, shouldUseRecordPosition, fileIndex.getRequiredFilters, fileIndex) } } } @@ -319,7 +319,7 @@ class HoodieCopyOnWriteSnapshotHadoopFsRelationFactory(override val sqlContext: new HoodieFileGroupReaderBasedParquetFileFormat( tableState, HoodieTableSchema(tableStructSchema, tableAvroSchema.toString, internalSchemaOpt), metaClient.getTableConfig.getTableName, mergeType, mandatoryFields, - false, false, validCommits, shouldUseRecordPosition, Seq.empty) + false, false, validCommits, shouldUseRecordPosition, Seq.empty, fileIndex) } } } @@ -347,7 +347,7 @@ class HoodieCopyOnWriteIncrementalHadoopFsRelationFactory(override val sqlContex new HoodieFileGroupReaderBasedParquetFileFormat( tableState, HoodieTableSchema(tableStructSchema, tableAvroSchema.toString, internalSchemaOpt), metaClient.getTableConfig.getTableName, mergeType, mandatoryFields, - false, true, validCommits, shouldUseRecordPosition, fileIndex.getRequiredFilters) + false, true, validCommits, shouldUseRecordPosition, fileIndex.getRequiredFilters, fileIndex) } } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala index e9b7abc5d103..36c476ccfa5f 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala @@ -31,12 +31,13 @@ import org.apache.hudi.common.util.FileIOUtils import org.apache.hudi.common.util.collection.ExternalSpillableMap.DiskMapType import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.internal.schema.utils.SerDeHelper -import org.apache.hudi.{AvroConversionUtils, HoodieFileIndex, HoodiePartitionCDCFileGroupMapping, HoodiePartitionFileSliceMapping, HoodieTableSchema, HoodieTableState, SparkAdapterSupport, SparkFileFormatInternalRowReaderContext} +import org.apache.hudi.{AvroConversionUtils, HoodieCDCFileIndex, HoodieFileIndex, HoodiePartitionCDCFileGroupMapping, HoodiePartitionFileSliceMapping, HoodieTableSchema, HoodieTableState, SparkAdapterSupport, SparkFileFormatInternalRowReaderContext} import org.apache.spark.sql.HoodieCatalystExpressionUtils.generateUnsafeProjection import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.JoinedRow import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.internal.SQLConf.PARQUET_VECTORIZED_READER_ENABLED import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration @@ -64,16 +65,33 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, isIncremental: Boolean, validCommits: String, shouldUseRecordPosition: Boolean, - requiredFilters: Seq[Filter] + requiredFilters: Seq[Filter], + @transient hoodieFileIndex: HoodieFileIndex ) extends ParquetFileFormat with SparkAdapterSupport with HoodieFormatTrait { def getRequiredFilters: Seq[Filter] = requiredFilters private val sanitizedTableName = AvroSchemaUtils.getAvroRecordQualifiedName(tableName) - override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = false + private lazy val canSupportBatch = hoodieFileIndex.canEnableBatch + private lazy val isCDC = hoodieFileIndex.isInstanceOf[HoodieCDCFileIndex] - private val supportBatchResult = false + + /** + * Support batch needs to remain consistent, even if one side of a bootstrap merge can support + * while the other side can't + */ + private var supportBatchCalled = false + private var supportBatchResult = false + + override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { + if (!supportBatchCalled || supportBatchResult) { + supportBatchCalled = true + supportBatchResult = !isCDC && !isIncremental && !shouldUseRecordPosition && canSupportBatch && super.supportBatch(sparkSession, schema) + } + sparkSession.conf.set(PARQUET_VECTORIZED_READER_ENABLED.key, supportBatchResult) + supportBatchResult + } private lazy val internalSchemaOpt: org.apache.hudi.common.util.Option[InternalSchema] = if (tableSchema.internalSchema.isEmpty) { org.apache.hudi.common.util.Option.empty() @@ -96,20 +114,21 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, val partitionColumns = partitionSchema.fieldNames val dataSchema = StructType(tableSchema.structTypeSchema.fields.filterNot(f => partitionColumns.contains(f.name))) val outputSchema = StructType(requiredSchema.fields ++ partitionSchema.fields) - spark.conf.set("spark.sql.parquet.enableVectorizedReader", supportBatchResult) val isCount = requiredSchema.isEmpty && !isMOR && !isIncremental val augmentedHadoopConf = FSUtils.buildInlineConf(hadoopConf) setSchemaEvolutionConfigs(augmentedHadoopConf) val baseFileReader = super.buildReaderWithPartitionValues(spark, dataSchema, partitionSchema, requiredSchema, filters ++ requiredFilters, options, new Configuration(augmentedHadoopConf)) - val cdcFileReader = super.buildReaderWithPartitionValues( - spark, - tableSchema.structTypeSchema, - StructType(Nil), - tableSchema.structTypeSchema, - Nil, - options, - new Configuration(hadoopConf)) + val cdcFileReader = if (isCDC) { + super.buildReaderWithPartitionValues( + spark, + tableSchema.structTypeSchema, + StructType(Nil), + tableSchema.structTypeSchema, + Nil, + options, + new Configuration(hadoopConf)) + } val requestedAvroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(requiredSchema, sanitizedTableName) val dataAvroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(dataSchema, sanitizedTableName) @@ -172,7 +191,8 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, val fileSplits = hoodiePartitionCDCFileGroupSliceMapping.getFileSplits().toArray val fileGroupSplit: HoodieCDCFileGroupSplit = HoodieCDCFileGroupSplit(fileSplits) buildCDCRecordIterator( - fileGroupSplit, cdcFileReader, broadcastedHadoopConf.value.value, fileIndexProps, requiredSchema) + fileGroupSplit, cdcFileReader.asInstanceOf[PartitionedFile => Iterator[InternalRow]], + broadcastedHadoopConf.value.value, fileIndexProps, requiredSchema) case _ => parquetFileReader.value.read(file, requiredSchema, partitionSchema, filters, broadcastedHadoopConf.value.value) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieParquetBloom.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieParquetBloom.scala index 32b3ea0b66f1..a6f3a0e7368b 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieParquetBloom.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieParquetBloom.scala @@ -17,12 +17,17 @@ package org.apache.hudi -import org.apache.hudi.common.model.{HoodieTableType, WriteOperationType} -import org.apache.hudi.config.HoodieWriteConfig -import org.apache.hudi.testutils.HoodieSparkClientTestBase import org.apache.spark.sql._ +import org.apache.spark.sql.hudi.HoodieSparkSessionExtension import org.apache.spark.util.AccumulatorV2 +import org.apache.spark.SparkContext +import org.apache.hudi.testutils.HoodieClientTestUtils.getSparkConfForTest +import org.apache.hudi.DataSourceWriteOptions +import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.common.model.{HoodieTableType, WriteOperationType} +import org.apache.hudi.testutils.HoodieSparkClientTestBase import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.EnumSource @@ -60,11 +65,11 @@ class TestHoodieParquetBloomFilter extends HoodieSparkClientTestBase with ScalaA sparkSession.sparkContext.register(accu) // this one shall skip partition scanning thanks to bloom when spark >=3 - sparkSession.read.format("hudi").load(basePath).filter("bloom_col = '3'").foreachPartition((it: Iterator[Row]) => it.foreach(_ => accu.add(1))) + sparkSession.read.format("hudi").load(basePath).filter("bloom_col = '3'").foreachPartition((it: Iterator[Row]) => it.foreach(_ => accu.add(0))) assertEquals(if (currentSparkSupportParquetBloom()) 0 else 1, accu.value) // this one will trigger one partition scan - sparkSession.read.format("hudi").load(basePath).filter("bloom_col = '2'").foreachPartition((it: Iterator[Row]) => it.foreach(_ => accu.add(1))) + sparkSession.read.format("hudi").load(basePath).filter("bloom_col = '2'").foreachPartition((it: Iterator[Row]) => it.foreach(_ => accu.add(0))) assertEquals(1, accu.value) } From 9d1ac2a1bd9f2343174a0273437e7a240294eee4 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 30 Apr 2024 12:54:39 -0400 Subject: [PATCH 73/86] add comment and allow vector read when positional merge is used --- .../apache/hudi/SparkFileFormatInternalRowReaderContext.scala | 1 + .../parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala index cf06c5a76bd8..64593a3beb17 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala @@ -214,6 +214,7 @@ class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkParquetRea override def next(): Any = { (skeletonFileIterator.next(), dataFileIterator.next()) match { case (s: ColumnarBatch, d: ColumnarBatch) => + //This will not be used until [HUDI-7693] is implemented val numCols = s.numCols() + d.numCols() val vecs: Array[ColumnVector] = new Array[ColumnVector](numCols) for (i <- 0 until numCols) { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala index 30374d984351..2cdf50909aaa 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala @@ -88,7 +88,7 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { if (!supportBatchCalled || supportBatchResult) { supportBatchCalled = true - supportBatchResult = !isCDC && !isIncremental && !shouldUseRecordPosition && canSupportBatch && super.supportBatch(sparkSession, schema) + supportBatchResult = !isCDC && !isIncremental && canSupportBatch && super.supportBatch(sparkSession, schema) } sparkSession.conf.set(PARQUET_VECTORIZED_READER_ENABLED.key, supportBatchResult) supportBatchResult From 63737caa30a0ba2ccc66b05bbeb3005d185eb4b7 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 30 May 2024 11:45:03 -0400 Subject: [PATCH 74/86] don't unwrap copy, we need to original so that the hadoopconf gets modified by build --- .../parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala index 6dc63ab0204c..451709495924 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala @@ -134,7 +134,7 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, val requestedAvroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(requiredSchema, sanitizedTableName) val dataAvroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(dataSchema, sanitizedTableName) val parquetFileReader = spark.sparkContext.broadcast(sparkAdapter.createParquetFileReader(supportBatchResult, - spark.sessionState.conf, options, augmentedStorageConf.unwrapCopy())) + spark.sessionState.conf, options, augmentedStorageConf.unwrap())) val broadcastedStorageConf = spark.sparkContext.broadcast(augmentedStorageConf) val broadcastedDataSchema = spark.sparkContext.broadcast(dataAvroSchema) val broadcastedRequestedSchema = spark.sparkContext.broadcast(requestedAvroSchema) From 475a1bc220eaee04fa78ba46a922b434b8306047 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Thu, 30 May 2024 12:38:54 -0400 Subject: [PATCH 75/86] fix another case where hadoopconf copy isolates config changes --- .../table/read/TestSpark35RecordPositionMetadataColumn.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala index 8e0b252418b9..f2ca5af0151e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala @@ -88,7 +88,7 @@ class TestSpark35RecordPositionMetadataColumn extends SparkClientFunctionalTestH // Prepare the file and Parquet file reader. val metaClient = getHoodieMetaClient( - HadoopFSUtils.getStorageConfWithCopy(_spark.sparkContext.hadoopConfiguration), basePath) + HadoopFSUtils.getStorageConfWithCopy(hadoopConf), basePath) val allBaseFiles = HoodieTestTable.of(metaClient).listAllBaseFiles assertFalse(allBaseFiles.isEmpty) val readerContext = new SparkFileFormatInternalRowReaderContext(reader, HoodieRecord.RECORD_KEY_METADATA_FIELD, Seq.empty) From d504e37ab6cee7d80e53e6daf2df1ef95eea01b7 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Fri, 31 May 2024 15:03:23 -0400 Subject: [PATCH 76/86] disable problem test --- .../apache/hudi/functional/TestRecordLevelIndexWithSQL.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndexWithSQL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndexWithSQL.scala index 97fdc1e10b21..9d39dcc9778c 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndexWithSQL.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndexWithSQL.scala @@ -27,11 +27,12 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, E import org.apache.spark.sql.types.StringType import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.io.TempDir -import org.junit.jupiter.api.{Tag, Test} +import org.junit.jupiter.api.{Disabled, Tag, Test} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @Tag("functional") +@Disabled class TestRecordLevelIndexWithSQL extends RecordLevelIndexTestBase { val sqlTempTable = "tbl" From ec6fa62945094d548dce7d7e8e6ef2363ba0d05f Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Fri, 31 May 2024 19:03:50 -0400 Subject: [PATCH 77/86] disable vectorized read for mor and bootstrap --- .../org/apache/hudi/HoodieFileIndex.scala | 5 ----- .../hudi/HoodieHadoopFsRelationFactory.scala | 13 +++++++----- ...ileGroupReaderBasedParquetFileFormat.scala | 11 ++++------ .../TestRecordLevelIndexWithSQL.scala | 20 +++++++++---------- 4 files changed, 21 insertions(+), 28 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index 18590ebc3a92..416a7a95832d 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -115,9 +115,6 @@ case class HoodieFileIndex(spark: SparkSession, override def rootPaths: Seq[Path] = getQueryPaths.asScala.map(e => new Path(e.toUri)).toSeq - // if true, fg reader can attempt to enable batch reading - var canEnableBatch = false - /** * Returns the FileStatus for all the base files (excluding log files). This should be used only for * cases where Spark directly fetches the list of files via HoodieFileIndex or for read optimized query logic @@ -158,7 +155,6 @@ case class HoodieFileIndex(spark: SparkSession, * @return list of PartitionDirectory containing partition to base files mapping */ override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { - canEnableBatch = true val prunedPartitionsAndFilteredFileSlices = filterFileSlices(dataFilters, partitionFilters).map { case (partitionOpt, fileSlices) => if (shouldEmbedFileSlices) { @@ -177,7 +173,6 @@ case class HoodieFileIndex(spark: SparkSession, || (f.getBaseFile.isPresent && f.getBaseFile.get().getBootstrapBaseFile.isPresent)). foldLeft(Map[String, FileSlice]()) { (m, f) => m + (f.getFileId -> f) } if (c.nonEmpty) { - canEnableBatch = false sparkAdapter.getSparkPartitionedFileUtils.newPartitionDirectory( new HoodiePartitionFileSliceMapping(InternalRow.fromSeq(partitionOpt.get.values), c), baseFileStatusesAndLogFileOnly) } else { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala index 7d5dcccb084f..7464b72b3d10 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala @@ -243,8 +243,8 @@ class HoodieMergeOnReadSnapshotHadoopFsRelationFactory(override val sqlContext: } else { new HoodieFileGroupReaderBasedParquetFileFormat( tableState, HoodieTableSchema(tableStructSchema, tableAvroSchema.toString, internalSchemaOpt), - metaClient.getTableConfig.getTableName, mergeType, mandatoryFields, - true, false, validCommits, shouldUseRecordPosition, Seq.empty, fileIndex) + metaClient.getTableConfig.getTableName, mergeType, mandatoryFields, true, isBootstrap, + false, fileIndex.isInstanceOf[HoodieCDCFileIndex], validCommits, shouldUseRecordPosition, Seq.empty) } } @@ -288,7 +288,8 @@ class HoodieMergeOnReadIncrementalHadoopFsRelationFactory(override val sqlContex new HoodieFileGroupReaderBasedParquetFileFormat( tableState, HoodieTableSchema(tableStructSchema, tableAvroSchema.toString, internalSchemaOpt), metaClient.getTableConfig.getTableName, mergeType, mandatoryFields, - true, true, validCommits, shouldUseRecordPosition, fileIndex.getRequiredFilters, fileIndex) + true, isBootstrap, true, fileIndex.isInstanceOf[HoodieCDCFileIndex], + validCommits, shouldUseRecordPosition, fileIndex.getRequiredFilters) } } } @@ -319,7 +320,8 @@ class HoodieCopyOnWriteSnapshotHadoopFsRelationFactory(override val sqlContext: new HoodieFileGroupReaderBasedParquetFileFormat( tableState, HoodieTableSchema(tableStructSchema, tableAvroSchema.toString, internalSchemaOpt), metaClient.getTableConfig.getTableName, mergeType, mandatoryFields, - false, false, validCommits, shouldUseRecordPosition, Seq.empty, fileIndex) + false, isBootstrap, false, fileIndex.isInstanceOf[HoodieCDCFileIndex], validCommits, + shouldUseRecordPosition, Seq.empty) } } } @@ -347,7 +349,8 @@ class HoodieCopyOnWriteIncrementalHadoopFsRelationFactory(override val sqlContex new HoodieFileGroupReaderBasedParquetFileFormat( tableState, HoodieTableSchema(tableStructSchema, tableAvroSchema.toString, internalSchemaOpt), metaClient.getTableConfig.getTableName, mergeType, mandatoryFields, - false, true, validCommits, shouldUseRecordPosition, fileIndex.getRequiredFilters, fileIndex) + false, isBootstrap, true, fileIndex.isInstanceOf[HoodieCDCFileIndex], + validCommits, shouldUseRecordPosition, fileIndex.getRequiredFilters) } } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala index 451709495924..15df1049b628 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala @@ -63,21 +63,18 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, mergeType: String, mandatoryFields: Seq[String], isMOR: Boolean, + isBootstrap: Boolean, isIncremental: Boolean, + isCDC: Boolean, validCommits: String, shouldUseRecordPosition: Boolean, - requiredFilters: Seq[Filter], - @transient hoodieFileIndex: HoodieFileIndex + requiredFilters: Seq[Filter] ) extends ParquetFileFormat with SparkAdapterSupport with HoodieFormatTrait { def getRequiredFilters: Seq[Filter] = requiredFilters private val sanitizedTableName = AvroSchemaUtils.getAvroRecordQualifiedName(tableName) - private lazy val canSupportBatch = hoodieFileIndex.canEnableBatch - private lazy val isCDC = hoodieFileIndex.isInstanceOf[HoodieCDCFileIndex] - - /** * Support batch needs to remain consistent, even if one side of a bootstrap merge can support * while the other side can't @@ -88,7 +85,7 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { if (!supportBatchCalled || supportBatchResult) { supportBatchCalled = true - supportBatchResult = !isCDC && !isIncremental && canSupportBatch && super.supportBatch(sparkSession, schema) + supportBatchResult = !isCDC && !isIncremental && !isMOR && !isBootstrap && super.supportBatch(sparkSession, schema) } sparkSession.conf.set(PARQUET_VECTORIZED_READER_ENABLED.key, supportBatchResult) supportBatchResult diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndexWithSQL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndexWithSQL.scala index 9d39dcc9778c..18ae77412bda 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndexWithSQL.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndexWithSQL.scala @@ -26,13 +26,11 @@ import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Expression, GreaterThan, GreaterThanOrEqual, In, Literal, Or} import org.apache.spark.sql.types.StringType import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} -import org.junit.jupiter.api.io.TempDir -import org.junit.jupiter.api.{Disabled, Tag, Test} +import org.junit.jupiter.api.{Tag, Test} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @Tag("functional") -@Disabled class TestRecordLevelIndexWithSQL extends RecordLevelIndexTestBase { val sqlTempTable = "tbl" @@ -54,39 +52,39 @@ class TestRecordLevelIndexWithSQL extends RecordLevelIndexTestBase { validate = false) createTempTable(hudiOpts) - verifyInQuery(hudiOpts) - verifyEqualToQuery(hudiOpts) + //verifyInQuery(hudiOpts) + //verifyEqualToQuery(hudiOpts) verifyNegativeTestCases(hudiOpts) } private def verifyNegativeTestCases(hudiOpts: Map[String, String]): Unit = { val commonOpts = hudiOpts + ("path" -> basePath) metaClient = HoodieTableMetaClient.reload(metaClient) - val fileIndex = HoodieFileIndex(spark, metaClient, None, commonOpts, includeLogFiles = true) + //val fileIndex = HoodieFileIndex(spark, metaClient, None, commonOpts, includeLogFiles = true) // when no data filter is applied - assertEquals(getLatestDataFilesCount(commonOpts), fileIndex.listFiles(Seq.empty, Seq.empty).flatMap(s => s.files).size) + //assertEquals(getLatestDataFilesCount(commonOpts), fileIndex.listFiles(Seq.empty, Seq.empty).flatMap(s => s.files).size) assertEquals(6, spark.sql("select * from " + sqlTempTable).count()) // non existing entries in EqualTo query var dataFilter: Expression = EqualTo(attribute("_row_key"), Literal("xyz")) assertEquals(0, spark.sql("select * from " + sqlTempTable + " where " + dataFilter.sql).count()) - assertEquals(0, fileIndex.listFiles(Seq.empty, Seq(dataFilter)).flatMap(s => s.files).size) + //assertEquals(0, fileIndex.listFiles(Seq.empty, Seq(dataFilter)).flatMap(s => s.files).size) // non existing entries in IN query dataFilter = In(attribute("_row_key"), List.apply(Literal("xyz"), Literal("abc"))) assertEquals(0, spark.sql("select * from " + sqlTempTable + " where " + dataFilter.sql).count()) - assertEquals(0, fileIndex.listFiles(Seq.empty, Seq(dataFilter)).flatMap(s => s.files).size) + //assertEquals(0, fileIndex.listFiles(Seq.empty, Seq(dataFilter)).flatMap(s => s.files).size) // not supported GreaterThan query val reckey = mergedDfList.last.limit(2).collect().map(row => row.getAs("_row_key").toString) dataFilter = GreaterThan(attribute("_row_key"), Literal(reckey(0))) - assertTrue(fileIndex.listFiles(Seq.empty, Seq(dataFilter)).flatMap(s => s.files).size >= 3) + //assertTrue(fileIndex.listFiles(Seq.empty, Seq(dataFilter)).flatMap(s => s.files).size >= 3) // not supported OR query dataFilter = Or(EqualTo(attribute("_row_key"), Literal(reckey(0))), GreaterThanOrEqual(attribute("timestamp"), Literal(0))) assertEquals(6, spark.sql("select * from " + sqlTempTable + " where " + dataFilter.sql).count()) - assertTrue(fileIndex.listFiles(Seq.empty, Seq(dataFilter)).flatMap(s => s.files).size >= 3) + //assertTrue(fileIndex.listFiles(Seq.empty, Seq(dataFilter)).flatMap(s => s.files).size >= 3) } def verifyEqualToQuery(hudiOpts: Map[String, String]): Unit = { From 22846139475031d663fc6bb2b1a554dd1b2e637e Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Mon, 3 Jun 2024 09:43:41 -0400 Subject: [PATCH 78/86] revert test change --- .../TestRecordLevelIndexWithSQL.scala | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndexWithSQL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndexWithSQL.scala index 18ae77412bda..97fdc1e10b21 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndexWithSQL.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndexWithSQL.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Expression, GreaterThan, GreaterThanOrEqual, In, Literal, Or} import org.apache.spark.sql.types.StringType import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} +import org.junit.jupiter.api.io.TempDir import org.junit.jupiter.api.{Tag, Test} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -52,39 +53,39 @@ class TestRecordLevelIndexWithSQL extends RecordLevelIndexTestBase { validate = false) createTempTable(hudiOpts) - //verifyInQuery(hudiOpts) - //verifyEqualToQuery(hudiOpts) + verifyInQuery(hudiOpts) + verifyEqualToQuery(hudiOpts) verifyNegativeTestCases(hudiOpts) } private def verifyNegativeTestCases(hudiOpts: Map[String, String]): Unit = { val commonOpts = hudiOpts + ("path" -> basePath) metaClient = HoodieTableMetaClient.reload(metaClient) - //val fileIndex = HoodieFileIndex(spark, metaClient, None, commonOpts, includeLogFiles = true) + val fileIndex = HoodieFileIndex(spark, metaClient, None, commonOpts, includeLogFiles = true) // when no data filter is applied - //assertEquals(getLatestDataFilesCount(commonOpts), fileIndex.listFiles(Seq.empty, Seq.empty).flatMap(s => s.files).size) + assertEquals(getLatestDataFilesCount(commonOpts), fileIndex.listFiles(Seq.empty, Seq.empty).flatMap(s => s.files).size) assertEquals(6, spark.sql("select * from " + sqlTempTable).count()) // non existing entries in EqualTo query var dataFilter: Expression = EqualTo(attribute("_row_key"), Literal("xyz")) assertEquals(0, spark.sql("select * from " + sqlTempTable + " where " + dataFilter.sql).count()) - //assertEquals(0, fileIndex.listFiles(Seq.empty, Seq(dataFilter)).flatMap(s => s.files).size) + assertEquals(0, fileIndex.listFiles(Seq.empty, Seq(dataFilter)).flatMap(s => s.files).size) // non existing entries in IN query dataFilter = In(attribute("_row_key"), List.apply(Literal("xyz"), Literal("abc"))) assertEquals(0, spark.sql("select * from " + sqlTempTable + " where " + dataFilter.sql).count()) - //assertEquals(0, fileIndex.listFiles(Seq.empty, Seq(dataFilter)).flatMap(s => s.files).size) + assertEquals(0, fileIndex.listFiles(Seq.empty, Seq(dataFilter)).flatMap(s => s.files).size) // not supported GreaterThan query val reckey = mergedDfList.last.limit(2).collect().map(row => row.getAs("_row_key").toString) dataFilter = GreaterThan(attribute("_row_key"), Literal(reckey(0))) - //assertTrue(fileIndex.listFiles(Seq.empty, Seq(dataFilter)).flatMap(s => s.files).size >= 3) + assertTrue(fileIndex.listFiles(Seq.empty, Seq(dataFilter)).flatMap(s => s.files).size >= 3) // not supported OR query dataFilter = Or(EqualTo(attribute("_row_key"), Literal(reckey(0))), GreaterThanOrEqual(attribute("timestamp"), Literal(0))) assertEquals(6, spark.sql("select * from " + sqlTempTable + " where " + dataFilter.sql).count()) - //assertTrue(fileIndex.listFiles(Seq.empty, Seq(dataFilter)).flatMap(s => s.files).size >= 3) + assertTrue(fileIndex.listFiles(Seq.empty, Seq(dataFilter)).flatMap(s => s.files).size >= 3) } def verifyEqualToQuery(hudiOpts: Map[String, String]): Unit = { From 72eb6deed7d06841a7ba94b4f9b20fba09bea5b7 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Tue, 4 Jun 2024 13:07:32 +0530 Subject: [PATCH 79/86] Address self comments --- .../BaseSparkInternalRowReaderContext.java | 2 +- ...rkFileFormatInternalRowReaderContext.scala | 43 ++++++------- .../common/engine/HoodieReaderContext.java | 6 +- .../read/HoodieBaseFileGroupRecordBuffer.java | 64 +++++++++++++++---- .../HoodieFileGroupReaderSchemaHandler.java | 3 + .../HoodieKeyBasedFileGroupRecordBuffer.java | 28 +------- ...diePositionBasedFileGroupRecordBuffer.java | 49 ++------------ .../HoodiePositionBasedSchemaHandler.java | 3 + .../schema/action/InternalSchemaMerger.java | 4 +- .../HoodieFileGroupReaderTestUtils.java | 10 +-- .../schema/action/TestMergeSchema.java | 13 ++-- .../HoodieFileGroupReaderTestHarness.java | 3 +- ...ileGroupReaderBasedParquetFileFormat.scala | 2 +- 13 files changed, 111 insertions(+), 119 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java index a90bb207ddc9..528cab69dc4b 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java @@ -149,4 +149,4 @@ public UnaryOperator projectRecord(Schema from, Schema to, Map getIdentityProjection() { return row -> row; } -} \ No newline at end of file +} diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala index b2d6d8e1f170..5cbca31a6f90 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala @@ -22,10 +22,11 @@ package org.apache.hudi import org.apache.avro.Schema import org.apache.avro.generic.IndexedRecord import org.apache.hadoop.conf.Configuration -import org.apache.hudi.SparkFileFormatInternalRowReaderContext.{ROW_INDEX_TEMPORARY_COLUMN_NAME, getAppliedRequiredSchema} +import org.apache.hudi.SparkFileFormatInternalRowReaderContext.getAppliedRequiredSchema import org.apache.hudi.avro.AvroSchemaUtils import org.apache.hudi.common.engine.HoodieReaderContext import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.table.read.HoodiePositionBasedFileGroupRecordBuffer.ROW_INDEX_TEMPORARY_COLUMN_NAME import org.apache.hudi.common.util.ValidationUtils.checkState import org.apache.hudi.common.util.collection.{CachingIterator, ClosableIterator, CloseableMappingIterator} import org.apache.hudi.io.storage.{HoodieSparkFileReaderFactory, HoodieSparkParquetReader} @@ -37,6 +38,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, SparkParquetReader} +import org.apache.spark.sql.hudi.SparkAdapter import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.{LongType, MetadataBuilder, StructField, StructType} import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} @@ -44,23 +46,23 @@ import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch} import scala.collection.mutable /** - * Implementation of {@link HoodieReaderContext} to read {@link InternalRow}s with - * {@link ParquetFileFormat} on Spark. + * Implementation of [[HoodieReaderContext]] to read [[InternalRow]]s with + * [[ParquetFileFormat]] on Spark. * * This uses Spark parquet reader to read parquet data files or parquet log blocks. * - * @param parquetFileReader A reader that transforms a {@link PartitionedFile} to an iterator of - * {@link InternalRow}. This is required for reading the base file and - * not required for reading a file group with only log files. - * @param recordKeyColumn column name for the recordkey - * @param filters spark filters that might be pushed down into the reader + * @param parquetFileReader A reader that transforms a [[PartitionedFile]] to an iterator of + * [[InternalRow]]. This is required for reading the base file and + * not required for reading a file group with only log files. + * @param recordKeyColumn column name for the recordkey + * @param filters spark filters that might be pushed down into the reader */ class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkParquetReader, recordKeyColumn: String, filters: Seq[Filter]) extends BaseSparkInternalRowReaderContext { - lazy val sparkAdapter = SparkAdapterSupport.sparkAdapter - val deserializerMap: mutable.Map[Schema, HoodieAvroDeserializer] = mutable.Map() + lazy val sparkAdapter: SparkAdapter = SparkAdapterSupport.sparkAdapter lazy val recordKeyFilters: Seq[Filter] = filters.filter(f => f.references.exists(c => c.equalsIgnoreCase(recordKeyColumn))) + private val deserializerMap: mutable.Map[Schema, HoodieAvroDeserializer] = mutable.Map() override def getFileRecordIterator(filePath: StoragePath, start: Long, @@ -128,10 +130,10 @@ class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkParquetRea dataFileIterator.asInstanceOf[ClosableIterator[Any]], dataRequiredSchema) } - protected def doBootstrapMerge(skeletonFileIterator: ClosableIterator[Any], - skeletonRequiredSchema: Schema, - dataFileIterator: ClosableIterator[Any], - dataRequiredSchema: Schema): ClosableIterator[InternalRow] = { + private def doBootstrapMerge(skeletonFileIterator: ClosableIterator[Any], + skeletonRequiredSchema: Schema, + dataFileIterator: ClosableIterator[Any], + dataRequiredSchema: Schema): ClosableIterator[InternalRow] = { if (getUseRecordPosition) { assert(AvroSchemaUtils.containsFieldInSchema(skeletonRequiredSchema, ROW_INDEX_TEMPORARY_COLUMN_NAME)) assert(AvroSchemaUtils.containsFieldInSchema(dataRequiredSchema, ROW_INDEX_TEMPORARY_COLUMN_NAME)) @@ -241,10 +243,6 @@ class SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkParquetRea } object SparkFileFormatInternalRowReaderContext { - // From "ParquetFileFormat.scala": The names of the field for record position. - private val ROW_INDEX = "row_index" - private val ROW_INDEX_TEMPORARY_COLUMN_NAME = s"_tmp_metadata_$ROW_INDEX" - // From "namedExpressions.scala": Used to construct to record position field metadata. private val FILE_SOURCE_GENERATED_METADATA_COL_ATTR_KEY = "__file_source_generated_metadata_col" private val FILE_SOURCE_METADATA_COL_ATTR_KEY = "__file_source_metadata_col" @@ -254,10 +252,6 @@ object SparkFileFormatInternalRowReaderContext { filters.filter(f => f.references.exists(c => c.equalsIgnoreCase(recordKeyColumn))) } - def isIndexTempColumn(field: StructField): Boolean = { - field.name.equals(ROW_INDEX_TEMPORARY_COLUMN_NAME) - } - def getAppliedRequiredSchema(requiredSchema: StructType): StructType = { val metadata = new MetadataBuilder() .putString(METADATA_COL_ATTR_KEY, ROW_INDEX_TEMPORARY_COLUMN_NAME) @@ -267,4 +261,9 @@ object SparkFileFormatInternalRowReaderContext { val rowIndexField = StructField(ROW_INDEX_TEMPORARY_COLUMN_NAME, LongType, nullable = false, metadata) StructType(requiredSchema.fields.filterNot(isIndexTempColumn) :+ rowIndexField) } + + private def isIndexTempColumn(field: StructField): Boolean = { + field.name.equals(ROW_INDEX_TEMPORARY_COLUMN_NAME) + } + } \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java index 36b50036b33c..577478b6ae62 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java @@ -281,7 +281,7 @@ public Map updateSchemaAndResetOrderingValInMetadata(Map mergeBootstrapReaders(ClosableIterator skeletonFileIterator, @@ -293,8 +293,8 @@ public abstract ClosableIterator mergeBootstrapReaders(ClosableIterator sk * Creates a function that will reorder records of schema "from" to schema of "to" * all fields in "to" must be in "from", but not all fields in "from" must be in "to" * - * @param from the schema of records to be passed into UnaryOperator - * @param to the schema of records produced by UnaryOperator + * @param from the schema of records to be passed into UnaryOperator + * @param to the schema of records produced by UnaryOperator * @param renamedColumns map of renamed columns where the key is the new name from the query and * the value is the old name that exists in the file * @return a function that takes in a record and returns the record with reordered columns diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java index 08660906ee72..0d5750e29928 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java @@ -51,6 +51,7 @@ import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -250,19 +251,9 @@ protected Pair, Schema> getRecordsIterator(HoodieDataBlock d } else { blockRecordsIterator = dataBlock.getEngineRecordIterator(readerContext); } - Option, Schema>> schemaEvolutionTransformerOpt = - composeEvolvedSchemaTransformer(dataBlock); - - // In case when schema has been evolved original persisted records will have to be - // transformed to adhere to the new schema - Function transformer = - schemaEvolutionTransformerOpt.map(Pair::getLeft) - .orElse(Function.identity()); - - Schema schema = schemaEvolutionTransformerOpt.map(Pair::getRight) - .orElseGet(dataBlock::getSchema); - - return Pair.of(new CloseableMappingIterator<>(blockRecordsIterator, transformer), schema); + Pair, Schema> schemaTransformerWithEvolvedSchema = getSchemaTransformerWithEvolvedSchema(dataBlock); + return Pair.of(new CloseableMappingIterator<>( + blockRecordsIterator, schemaTransformerWithEvolvedSchema.getLeft()), schemaTransformerWithEvolvedSchema.getRight()); } /** @@ -381,4 +372,51 @@ protected static List extractRecordPositions(HoodieLogBlock logBlock) thro return blockPositions; } + + protected boolean hasNextBaseRecord(T baseRecord, Pair, Map> logRecordInfo) throws IOException { + Map metadata = readerContext.generateMetadataForRecord( + baseRecord, readerSchema); + + Option resultRecord = logRecordInfo != null + ? merge(Option.of(baseRecord), metadata, logRecordInfo.getLeft(), logRecordInfo.getRight()) + : merge(Option.empty(), Collections.emptyMap(), Option.of(baseRecord), metadata); + if (resultRecord.isPresent()) { + nextRecord = readerContext.seal(resultRecord.get()); + return true; + } + return false; + } + + protected boolean hasNextLogRecord() throws IOException { + if (logRecordIterator == null) { + logRecordIterator = records.values().iterator(); + } + + while (logRecordIterator.hasNext()) { + Pair, Map> nextRecordInfo = logRecordIterator.next(); + Option resultRecord; + resultRecord = merge(Option.empty(), Collections.emptyMap(), + nextRecordInfo.getLeft(), nextRecordInfo.getRight()); + if (resultRecord.isPresent()) { + nextRecord = readerContext.seal(resultRecord.get()); + return true; + } + } + return false; + } + + protected Pair, Schema> getSchemaTransformerWithEvolvedSchema(HoodieDataBlock dataBlock) { + Option, Schema>> schemaEvolutionTransformerOpt = + composeEvolvedSchemaTransformer(dataBlock); + + // In case when schema has been evolved original persisted records will have to be + // transformed to adhere to the new schema + Function transformer = + schemaEvolutionTransformerOpt.map(Pair::getLeft) + .orElse(Function.identity()); + + Schema evolvedSchema = schemaEvolutionTransformerOpt.map(Pair::getRight) + .orElseGet(dataBlock::getSchema); + return Pair.of(transformer, evolvedSchema); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java index 41bfd3bf1685..1a4c68e00202 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderSchemaHandler.java @@ -42,6 +42,9 @@ import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchemaDedupNested; import static org.apache.hudi.avro.AvroSchemaUtils.findNestedField; +/** + * This class is responsible for handling the schema for the file group reader. + */ public class HoodieFileGroupReaderSchemaHandler { protected final Schema dataSchema; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieKeyBasedFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieKeyBasedFileGroupRecordBuffer.java index 689e43ce1d83..96d4306afd40 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieKeyBasedFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieKeyBasedFileGroupRecordBuffer.java @@ -38,7 +38,6 @@ import java.io.IOException; import java.io.Serializable; import java.util.Arrays; -import java.util.Collections; import java.util.Iterator; import java.util.Map; @@ -137,33 +136,12 @@ protected boolean doHasNext() throws IOException { String recordKey = readerContext.getRecordKey(baseRecord, readerSchema); Pair, Map> logRecordInfo = records.remove(recordKey); - Map metadata = readerContext.generateMetadataForRecord( - baseRecord, readerSchema); - - Option resultRecord = logRecordInfo != null - ? merge(Option.of(baseRecord), metadata, logRecordInfo.getLeft(), logRecordInfo.getRight()) - : merge(Option.empty(), Collections.emptyMap(), Option.of(baseRecord), metadata); - if (resultRecord.isPresent()) { - nextRecord = readerContext.seal(resultRecord.get()); + if (hasNextBaseRecord(baseRecord, logRecordInfo)) { return true; } } // Handle records solely from log files. - if (logRecordIterator == null) { - logRecordIterator = records.values().iterator(); - } - - while (logRecordIterator.hasNext()) { - Pair, Map> nextRecordInfo = logRecordIterator.next(); - Option resultRecord; - resultRecord = merge(Option.empty(), Collections.emptyMap(), - nextRecordInfo.getLeft(), nextRecordInfo.getRight()); - if (resultRecord.isPresent()) { - nextRecord = readerContext.seal(resultRecord.get()); - return true; - } - } - return false; + return hasNextLogRecord(); } -} \ No newline at end of file +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java index 04c30f4f4296..29f05b015ed1 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedFileGroupRecordBuffer.java @@ -38,7 +38,6 @@ import java.io.IOException; import java.io.Serializable; import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -97,21 +96,10 @@ public void processDataBlock(HoodieDataBlock dataBlock, Option keySpecO // Extract positions from data block. List recordPositions = extractRecordPositions(dataBlock); + Pair, Schema> schemaTransformerWithEvolvedSchema = getSchemaTransformerWithEvolvedSchema(dataBlock); - Option, Schema>> schemaEvolutionTransformerOpt = - composeEvolvedSchemaTransformer(dataBlock); - - // In case when schema has been evolved original persisted records will have to be - // transformed to adhere to the new schema - Function transformer = - schemaEvolutionTransformerOpt.map(Pair::getLeft) - .orElse(Function.identity()); - - Schema evolvedSchema = schemaEvolutionTransformerOpt.map(Pair::getRight) - .orElseGet(dataBlock::getSchema); - - // TODO: return an iterator that can generate sequence number with the record. - // Then we can hide this logic into data block. + // TODO: Return an iterator that can generate sequence number with the record. + // Then we can hide this logic into data block. try (ClosableIterator recordIterator = dataBlock.getEngineRecordIterator(readerContext)) { int recordIndex = 0; while (recordIterator.hasNext()) { @@ -125,10 +113,10 @@ public void processDataBlock(HoodieDataBlock dataBlock, Option keySpecO long recordPosition = recordPositions.get(recordIndex++); - T evolvedNextRecord = transformer.apply(nextRecord); + T evolvedNextRecord = schemaTransformerWithEvolvedSchema.getLeft().apply(nextRecord); processNextDataRecord( evolvedNextRecord, - readerContext.generateMetadataForRecord(evolvedNextRecord, evolvedSchema), + readerContext.generateMetadataForRecord(evolvedNextRecord, schemaTransformerWithEvolvedSchema.getRight()), recordPosition ); } @@ -194,35 +182,12 @@ protected boolean doHasNext() throws IOException { T baseRecord = baseFileIterator.next(); nextRecordPosition = readerContext.extractRecordPosition(baseRecord, readerSchema, ROW_INDEX_COLUMN_NAME, nextRecordPosition); Pair, Map> logRecordInfo = records.remove(nextRecordPosition++); - - Map metadata = readerContext.generateMetadataForRecord( - baseRecord, readerSchema); - - Option resultRecord = logRecordInfo != null - ? merge(Option.of(baseRecord), metadata, logRecordInfo.getLeft(), logRecordInfo.getRight()) - : merge(Option.empty(), Collections.emptyMap(), Option.of(baseRecord), metadata); - if (resultRecord.isPresent()) { - nextRecord = readerContext.seal(resultRecord.get()); + if (hasNextBaseRecord(baseRecord, logRecordInfo)) { return true; } } // Handle records solely from log files. - if (logRecordIterator == null) { - logRecordIterator = records.values().iterator(); - } - - while (logRecordIterator.hasNext()) { - Pair, Map> nextRecordInfo = logRecordIterator.next(); - Option resultRecord; - resultRecord = merge(Option.empty(), Collections.emptyMap(), - nextRecordInfo.getLeft(), nextRecordInfo.getRight()); - if (resultRecord.isPresent()) { - nextRecord = readerContext.seal(resultRecord.get()); - return true; - } - } - - return false; + return hasNextLogRecord(); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedSchemaHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedSchemaHandler.java index 48dfe2f9dbd4..b001f5767dcc 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedSchemaHandler.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedSchemaHandler.java @@ -31,6 +31,9 @@ import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchemaDedupNested; +/** + * This class is responsible for handling the schema for the file group reader that supports positional merge. + */ public class HoodiePositionBasedSchemaHandler extends HoodieFileGroupReaderSchemaHandler { public HoodiePositionBasedSchemaHandler(HoodieReaderContext readerContext, Schema dataSchema, diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java index 9ba71a80d5ce..26c364c18004 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java @@ -92,9 +92,7 @@ public InternalSchema mergeSchema() { * @return read schema to read avro/parquet file. */ public Pair> mergeSchemaGetRenamed() { - Types.RecordType record = (Types.RecordType) mergeType(querySchema.getRecord(), 0); - InternalSchema internalSchema = new InternalSchema(record); - return Pair.of(internalSchema, renamedFields); + return Pair.of(mergeSchema(), renamedFields); } /** diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestUtils.java index 56cc5c2a5e95..944722e45f56 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestUtils.java @@ -44,7 +44,8 @@ public static HoodieFileGroupReader createFileGroupReader( TypedProperties properties, HoodieStorage storage, HoodieTableConfig tableConfig, - HoodieReaderContext readerContext + HoodieReaderContext readerContext, + HoodieTableMetaClient metaClient ) { assert (fileSliceOpt.isPresent()); return new HoodieFileGroupReaderBuilder() @@ -55,7 +56,7 @@ public static HoodieFileGroupReader createFileGroupReader( .withLength(length) .withProperties(properties) .withTableConfig(tableConfig) - .build(basePath, latestCommitTime, schema, shouldUseRecordPosition); + .build(basePath, latestCommitTime, schema, shouldUseRecordPosition, metaClient); } public static class HoodieFileGroupReaderBuilder { @@ -109,7 +110,8 @@ public HoodieFileGroupReader build( String basePath, String latestCommitTime, Schema schema, - boolean shouldUseRecordPosition + boolean shouldUseRecordPosition, + HoodieTableMetaClient metaClient ) { return new HoodieFileGroupReader<>( readerContext, @@ -120,7 +122,7 @@ public HoodieFileGroupReader build( schema, schema, Option.empty(), - null, + metaClient, props, tableConfig, start, diff --git a/hudi-common/src/test/java/org/apache/hudi/internal/schema/action/TestMergeSchema.java b/hudi-common/src/test/java/org/apache/hudi/internal/schema/action/TestMergeSchema.java index 5240179fb8c5..6b6bdf2d69ae 100644 --- a/hudi-common/src/test/java/org/apache/hudi/internal/schema/action/TestMergeSchema.java +++ b/hudi-common/src/test/java/org/apache/hudi/internal/schema/action/TestMergeSchema.java @@ -18,6 +18,7 @@ package org.apache.hudi.internal.schema.action; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.internal.schema.Types; import org.apache.hudi.internal.schema.utils.SchemaChangeUtils; @@ -25,6 +26,7 @@ import org.junit.jupiter.api.Test; import java.util.Arrays; +import java.util.Map; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -58,8 +60,8 @@ public void testPrimitiveMerge() { TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(newDeleteSchema); updateChange.updateColumnType("col2", Types.LongType.get()) .updateColumnComment("col2", "alter col2 comments") - .renameColumn("col2", "colx").addPositionChange("col2", - "col4", "after"); + .renameColumn("col2", "colx") // rename col2 to colx to assert later on renamed field + .addPositionChange("col2", "col4", "after"); InternalSchema updateSchema = SchemaChangeUtils.applyTableChanges2Schema(newDeleteSchema, updateChange); // add col1 again @@ -67,14 +69,17 @@ public void testPrimitiveMerge() { addChange1.addColumns("col1", Types.BooleanType.get(), "add new col1"); InternalSchema finalSchema = SchemaChangeUtils.applyTableChanges2Schema(updateSchema, addChange1); // merge schema by using columnType from query schema - InternalSchema mergeSchema = new InternalSchemaMerger(oldSchema, finalSchema, true, false).mergeSchema(); - + Pair> mergeSchemaWithRenamedField = new InternalSchemaMerger(oldSchema, finalSchema, true, false, false).mergeSchemaGetRenamed(); + InternalSchema mergeSchema = mergeSchemaWithRenamedField.getLeft(); + assertEquals("col2", mergeSchemaWithRenamedField.getRight().get("colx")); InternalSchema checkedSchema = new InternalSchema(Types.RecordType.get(Arrays.asList( Types.Field.get(4, true, "c1", Types.BooleanType.get(), "add c1 after col1"), Types.Field.get(5, true, "c2", Types.IntType.get(), "add c2 before col3"), Types.Field.get(3, true, "col4", Types.FloatType.get()), Types.Field.get(1, true, "col2", Types.LongType.get(), "alter col2 comments"), Types.Field.get(6, true, "col1suffix", Types.BooleanType.get(), "add new col1")))); + // merged schema without renamed fields + mergeSchema = new InternalSchemaMerger(oldSchema, finalSchema, true, false).mergeSchema(); assertEquals(mergeSchema, checkedSchema); // merge schema by using columnType from file schema diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestHarness.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestHarness.java index 5d50f885204f..a2ef6ae39068 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestHarness.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/testutils/reader/HoodieFileGroupReaderTestHarness.java @@ -120,7 +120,8 @@ protected ClosableIterator getFileGroupIterator(int numFiles) properties, new HoodieHadoopStorage(basePath, storageConf), tableConfig, - readerContext + readerContext, + metaClient ); fileGroupReader.initRecordIterators(); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala index 15df1049b628..a7b3b700dd3a 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala @@ -199,7 +199,7 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, } } - protected def setSchemaEvolutionConfigs(conf: StorageConfiguration[_]): Unit = { + private def setSchemaEvolutionConfigs(conf: StorageConfiguration[_]): Unit = { if (internalSchemaOpt.isPresent) { conf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, SerDeHelper.toJson(internalSchemaOpt.get())) conf.set(SparkInternalSchemaConverter.HOODIE_TABLE_PATH, tableState.tablePath) From 5f257f081549090f901803b237bebaaec8701798 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 4 Jun 2024 10:42:06 -0400 Subject: [PATCH 80/86] allow batch for cdc --- .../parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala index a7b3b700dd3a..0412d202148a 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala @@ -33,7 +33,7 @@ import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.internal.schema.utils.SerDeHelper import org.apache.hudi.storage.StorageConfiguration import org.apache.hudi.storage.hadoop.{HadoopStorageConfiguration, HoodieHadoopStorage} -import org.apache.hudi.{AvroConversionUtils, HoodieCDCFileIndex, HoodieFileIndex, HoodiePartitionCDCFileGroupMapping, HoodiePartitionFileSliceMapping, HoodieTableSchema, HoodieTableState, SparkAdapterSupport, SparkFileFormatInternalRowReaderContext} +import org.apache.hudi.{AvroConversionUtils, HoodieFileIndex, HoodiePartitionCDCFileGroupMapping, HoodiePartitionFileSliceMapping, HoodieTableSchema, HoodieTableState, SparkAdapterSupport, SparkFileFormatInternalRowReaderContext} import org.apache.spark.sql.HoodieCatalystExpressionUtils.generateUnsafeProjection import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow @@ -85,7 +85,7 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { if (!supportBatchCalled || supportBatchResult) { supportBatchCalled = true - supportBatchResult = !isCDC && !isIncremental && !isMOR && !isBootstrap && super.supportBatch(sparkSession, schema) + supportBatchResult = !isMOR && !isIncremental && !isBootstrap && super.supportBatch(sparkSession, schema) } sparkSession.conf.set(PARQUET_VECTORIZED_READER_ENABLED.key, supportBatchResult) supportBatchResult From 29c604c0249c7ec61987c0f944737a0c663903f1 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 4 Jun 2024 11:00:10 -0400 Subject: [PATCH 81/86] add comments --- .../scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala | 1 + .../table/read/TestSpark35RecordPositionMetadataColumn.scala | 1 + 2 files changed, 2 insertions(+) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala index 7464b72b3d10..be1f234f76c7 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala @@ -173,6 +173,7 @@ abstract class HoodieBaseHadoopFsRelationFactory(val sqlContext: SQLContext, protected lazy val mandatoryFieldsForMerging: Seq[String] = Seq(recordKeyField) ++ preCombineFieldOpt.map(Seq(_)).getOrElse(Seq()) + //feature added in spark 3.5 protected lazy val shouldUseRecordPosition: Boolean = checkIfAConfigurationEnabled(HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS) && HoodieSparkUtils.gteqSpark3_5 protected def queryTimestamp: Option[String] = diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala index f2ca5af0151e..5fab505df6de 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestSpark35RecordPositionMetadataColumn.scala @@ -64,6 +64,7 @@ class TestSpark35RecordPositionMetadataColumn extends SparkClientFunctionalTestH @Test def testRecordPositionColumn(): Unit = { + //feature is only supported in spark 3.5+ if (HoodieSparkUtils.gteqSpark3_5) { val _spark = spark // Prepare the schema From 4045388e89b8764c2c0e70a809fdc5bde4572e21 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 4 Jun 2024 14:41:22 -0400 Subject: [PATCH 82/86] add testing back/ add new testing --- .../convert/AvroInternalSchemaConverter.java | 19 +++-- .../apache/hudi/avro/AvroSchemaTestUtils.java | 66 ++++++++++++++ .../TestAvroInternalSchemaConverter.java | 85 +++++++++++++++++++ .../apache/hudi/TestHoodieSchemaUtils.java | 38 ++------- ...ileGroupReaderBasedParquetFileFormat.scala | 17 +++- 5 files changed, 182 insertions(+), 43 deletions(-) create mode 100644 hudi-common/src/test/java/org/apache/hudi/avro/AvroSchemaTestUtils.java create mode 100644 hudi-common/src/test/java/org/apache/hudi/internal/schema/convert/TestAvroInternalSchemaConverter.java diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java index 4389e59627c9..5a7ccd3069c1 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java @@ -20,6 +20,7 @@ import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.VisibleForTesting; import org.apache.hudi.exception.HoodieNullSchemaTypeException; import org.apache.hudi.internal.schema.HoodieSchemaException; import org.apache.hudi.internal.schema.InternalSchema; @@ -84,7 +85,8 @@ public static InternalSchema pruneAvroSchemaToInternalSchema(Schema schema, Inte * @param schema a avro schema. * @return leaf nodes full names. */ - private static List collectColNamesFromSchema(Schema schema) { + @VisibleForTesting + static List collectColNamesFromSchema(Schema schema) { List result = new ArrayList<>(); Deque visited = new LinkedList<>(); collectColNamesFromAvroSchema(schema, visited, result); @@ -99,7 +101,7 @@ private static void collectColNamesFromAvroSchema(Schema schema, Deque v visited.push(f.name()); collectColNamesFromAvroSchema(f.schema(), visited, resultSet); visited.pop(); - addFullName(f.schema(), f.name(), visited, resultSet); + addFullNameIfLeafNode(f.schema(), f.name(), visited, resultSet); } return; @@ -111,27 +113,26 @@ private static void collectColNamesFromAvroSchema(Schema schema, Deque v visited.push("element"); collectColNamesFromAvroSchema(schema.getElementType(), visited, resultSet); visited.pop(); - addFullName(schema.getElementType(), "element", visited, resultSet); + addFullNameIfLeafNode(schema.getElementType(), "element", visited, resultSet); return; case MAP: - addFullName(STRING, "key", visited, resultSet); + addFullNameIfLeafNode(STRING, "key", visited, resultSet); visited.push("value"); collectColNamesFromAvroSchema(schema.getValueType(), visited, resultSet); visited.pop(); - addFullName(schema.getValueType(), "value", visited, resultSet); + addFullNameIfLeafNode(schema.getValueType(), "value", visited, resultSet); return; default: - return; } } - private static void addFullName(Schema schema, String name, Deque visited, List resultSet) { - addFullName(AvroSchemaUtils.resolveNullableSchema(schema).getType(), name, visited, resultSet); + private static void addFullNameIfLeafNode(Schema schema, String name, Deque visited, List resultSet) { + addFullNameIfLeafNode(AvroSchemaUtils.resolveNullableSchema(schema).getType(), name, visited, resultSet); } - private static void addFullName(Schema.Type type, String name, Deque visited, List resultSet) { + private static void addFullNameIfLeafNode(Schema.Type type, String name, Deque visited, List resultSet) { switch (type) { case RECORD: case ARRAY: diff --git a/hudi-common/src/test/java/org/apache/hudi/avro/AvroSchemaTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/avro/AvroSchemaTestUtils.java new file mode 100644 index 000000000000..2935c98f6b96 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/avro/AvroSchemaTestUtils.java @@ -0,0 +1,66 @@ +/* + * 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.hudi.avro; + +import org.apache.avro.Schema; + +import java.util.Arrays; + +public class AvroSchemaTestUtils { + public static Schema.Field createNestedField(String name, Schema.Type type) { + return createNestedField(name, Schema.create(type)); + } + + public static Schema.Field createNestedField(String name, Schema schema) { + return new Schema.Field(name, createRecord(name, new Schema.Field("nested", schema, null, null)), null, null); + } + + public static Schema.Field createArrayField(String name, Schema.Type type) { + return createArrayField(name, Schema.create(type)); + } + + public static Schema.Field createArrayField(String name, Schema schema) { + return new Schema.Field(name, Schema.createArray(schema), null, null); + } + + public static Schema.Field createMapField(String name, Schema.Type type) { + return createMapField(name, Schema.create(type)); + } + + public static Schema.Field createMapField(String name, Schema schema) { + return new Schema.Field(name, Schema.createMap(schema), null, null); + } + + public static Schema.Field createPrimitiveField(String name, Schema.Type type) { + return new Schema.Field(name, Schema.create(type), null, null); + } + + public static Schema.Field createNullablePrimitiveField(String name, Schema.Type type) { + return new Schema.Field(name, AvroSchemaUtils.createNullableSchema(type), null, null); + } + + public static Schema createRecord(String name, Schema.Field... fields) { + return Schema.createRecord(name, null, null, false, Arrays.asList(fields)); + } + + public static Schema createNullableRecord(String name, Schema.Field... fields) { + return AvroSchemaUtils.createNullableSchema(Schema.createRecord(name, null, null, false, Arrays.asList(fields))); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/internal/schema/convert/TestAvroInternalSchemaConverter.java b/hudi-common/src/test/java/org/apache/hudi/internal/schema/convert/TestAvroInternalSchemaConverter.java new file mode 100644 index 000000000000..571d5e0bafb1 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/internal/schema/convert/TestAvroInternalSchemaConverter.java @@ -0,0 +1,85 @@ +/* + * 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.hudi.internal.schema.convert; + +import org.apache.hudi.avro.AvroSchemaTestUtils; + +import org.apache.avro.Schema; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; + +import static org.apache.hudi.avro.AvroSchemaTestUtils.createArrayField; +import static org.apache.hudi.avro.AvroSchemaTestUtils.createMapField; +import static org.apache.hudi.avro.AvroSchemaTestUtils.createNullablePrimitiveField; +import static org.apache.hudi.avro.AvroSchemaTestUtils.createNullableRecord; +import static org.apache.hudi.avro.AvroSchemaTestUtils.createPrimitiveField; +import static org.apache.hudi.avro.AvroSchemaTestUtils.createRecord; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestAvroInternalSchemaConverter { + + @Test + public void testCollectColumnNames() { + Schema simpleSchema = createRecord("simpleSchema", + createPrimitiveField("field1", Schema.Type.INT), + createPrimitiveField("field2", Schema.Type.STRING)); + + List fieldNames = AvroInternalSchemaConverter.collectColNamesFromSchema(simpleSchema); + List expectedOutput = Arrays.asList("field1", "field2"); + assertEquals(expectedOutput.size(), fieldNames.size()); + assertTrue(fieldNames.containsAll(expectedOutput)); + + + Schema simpleSchemaWithNullable = createRecord("simpleSchemaWithNullable", + createNullablePrimitiveField("field1", Schema.Type.INT), + createPrimitiveField("field2", Schema.Type.STRING)); + fieldNames = AvroInternalSchemaConverter.collectColNamesFromSchema(simpleSchemaWithNullable); + expectedOutput = Arrays.asList("field1", "field2"); + assertEquals(expectedOutput.size(), fieldNames.size()); + assertTrue(fieldNames.containsAll(expectedOutput)); + + Schema complexSchemaSingleLevel = createRecord("complexSchemaSingleLevel", + AvroSchemaTestUtils.createNestedField("field1", Schema.Type.INT), + createArrayField("field2", Schema.Type.STRING), + createMapField("field3", Schema.Type.DOUBLE)); + fieldNames = AvroInternalSchemaConverter.collectColNamesFromSchema(complexSchemaSingleLevel); + expectedOutput = Arrays.asList("field1.nested", "field2.element", "field3.key", "field3.value"); + assertEquals(expectedOutput.size(), fieldNames.size()); + assertTrue(fieldNames.containsAll(expectedOutput)); + + Schema deeplyNestedField = createRecord("deeplyNestedField", + createPrimitiveField("field1", Schema.Type.INT), + new Schema.Field("field2", + createRecord("field2nest", + createArrayField("field2nestarray", + createNullableRecord("field2nestarraynest", + createNullablePrimitiveField("field21", Schema.Type.INT), + createNullablePrimitiveField("field22", Schema.Type.INT))))), + createNullablePrimitiveField("field3", Schema.Type.INT)); + fieldNames = AvroInternalSchemaConverter.collectColNamesFromSchema(deeplyNestedField); + expectedOutput = Arrays.asList("field1", "field2.field2nestarray.element.field21", + "field2.field2nestarray.element.field22", "field3"); + assertEquals(expectedOutput.size(), fieldNames.size()); + assertTrue(fieldNames.containsAll(expectedOutput)); + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/TestHoodieSchemaUtils.java b/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/TestHoodieSchemaUtils.java index b10d0cfa9929..d884dd4a3f77 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/TestHoodieSchemaUtils.java +++ b/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/TestHoodieSchemaUtils.java @@ -32,10 +32,14 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; -import java.util.Arrays; import java.util.HashMap; import java.util.Map; +import static org.apache.hudi.avro.AvroSchemaTestUtils.createArrayField; +import static org.apache.hudi.avro.AvroSchemaTestUtils.createMapField; +import static org.apache.hudi.avro.AvroSchemaTestUtils.createNestedField; +import static org.apache.hudi.avro.AvroSchemaTestUtils.createPrimitiveField; +import static org.apache.hudi.avro.AvroSchemaTestUtils.createRecord; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -251,36 +255,4 @@ private static Schema deduceWriterSchema(Schema incomingSchema, Schema latestTab Option.empty(), TYPED_PROPERTIES); } - private static Schema.Field createNestedField(String name, Schema.Type type) { - return createNestedField(name, Schema.create(type)); - } - - private static Schema.Field createNestedField(String name, Schema schema) { - return new Schema.Field(name, createRecord(name, new Schema.Field("nested", schema, null, null)), null, null); - } - - private static Schema.Field createArrayField(String name, Schema.Type type) { - return createArrayField(name, Schema.create(type)); - } - - private static Schema.Field createArrayField(String name, Schema schema) { - return new Schema.Field(name, Schema.createArray(schema), null, null); - } - - private static Schema.Field createMapField(String name, Schema.Type type) { - return createMapField(name, Schema.create(type)); - } - - private static Schema.Field createMapField(String name, Schema schema) { - return new Schema.Field(name, Schema.createMap(schema), null, null); - } - - private static Schema.Field createPrimitiveField(String name, Schema.Type type) { - return new Schema.Field(name, Schema.create(type), null, null); - } - - private static Schema createRecord(String name, Schema.Field... fields) { - return Schema.createRecord(name, null, null, false, Arrays.asList(fields)); - } - } diff --git a/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/execution/datasources/parquet/TestHoodieFileGroupReaderBasedParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/execution/datasources/parquet/TestHoodieFileGroupReaderBasedParquetFileFormat.scala index 7c8b2d6108d7..00afbcd7b170 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/execution/datasources/parquet/TestHoodieFileGroupReaderBasedParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/execution/datasources/parquet/TestHoodieFileGroupReaderBasedParquetFileFormat.scala @@ -20,9 +20,11 @@ package org.apache.spark.execution.datasources.parquet import org.apache.hudi.SparkFileFormatInternalRowReaderContext +import org.apache.hudi.common.table.read.HoodiePositionBasedFileGroupRecordBuffer.ROW_INDEX_TEMPORARY_COLUMN_NAME import org.apache.hudi.testutils.SparkClientFunctionalTestHarness import org.apache.spark.sql.sources.{EqualTo, GreaterThan, IsNotNull} -import org.junit.jupiter.api.Assertions.assertEquals +import org.apache.spark.sql.types.{LongType, StringType, StructField, StructType} +import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.Test class TestHoodieFileGroupReaderBasedParquetFileFormat extends SparkClientFunctionalTestHarness { @@ -45,4 +47,17 @@ class TestHoodieFileGroupReaderBasedParquetFileFormat extends SparkClientFunctio assertEquals(1, filtersWithKeyColumn.size) assertEquals("key_column", filtersWithKeyColumn.head.references.head) } + + @Test + def testGetAppliedRequiredSchema(): Unit = { + val fields = Array( + StructField("column_a", LongType, nullable = false), + StructField("column_b", StringType, nullable = false)) + val requiredSchema = StructType(fields) + + val appliedSchema: StructType = SparkFileFormatInternalRowReaderContext.getAppliedRequiredSchema( + requiredSchema) + assertEquals(3, appliedSchema.fields.length) + assertTrue(appliedSchema.fields.map(f => f.name).contains(ROW_INDEX_TEMPORARY_COLUMN_NAME)) + } } From 575b20681ead4465eddb7ed71ee8994cb82031cd Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 4 Jun 2024 16:16:35 -0400 Subject: [PATCH 83/86] add spark test --- .../TestAvroInternalSchemaConverter.java | 69 +++++++++----- .../TestSparkInternalSchemaConverter.scala | 93 +++++++++++++++++++ 2 files changed, 141 insertions(+), 21 deletions(-) create mode 100644 hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/util/TestSparkInternalSchemaConverter.scala diff --git a/hudi-common/src/test/java/org/apache/hudi/internal/schema/convert/TestAvroInternalSchemaConverter.java b/hudi-common/src/test/java/org/apache/hudi/internal/schema/convert/TestAvroInternalSchemaConverter.java index 571d5e0bafb1..8992f0c27fbe 100644 --- a/hudi-common/src/test/java/org/apache/hudi/internal/schema/convert/TestAvroInternalSchemaConverter.java +++ b/hudi-common/src/test/java/org/apache/hudi/internal/schema/convert/TestAvroInternalSchemaConverter.java @@ -38,36 +38,35 @@ public class TestAvroInternalSchemaConverter { - @Test - public void testCollectColumnNames() { - Schema simpleSchema = createRecord("simpleSchema", + public static Schema getSimpleSchema() { + return createRecord("simpleSchema", createPrimitiveField("field1", Schema.Type.INT), createPrimitiveField("field2", Schema.Type.STRING)); + } - List fieldNames = AvroInternalSchemaConverter.collectColNamesFromSchema(simpleSchema); - List expectedOutput = Arrays.asList("field1", "field2"); - assertEquals(expectedOutput.size(), fieldNames.size()); - assertTrue(fieldNames.containsAll(expectedOutput)); - + public static List getSimpleSchemaExpectedColumnNames() { + return Arrays.asList("field1", "field2"); + } - Schema simpleSchemaWithNullable = createRecord("simpleSchemaWithNullable", + public static Schema getSimpleSchemaWithNullable() { + return createRecord("simpleSchemaWithNullable", createNullablePrimitiveField("field1", Schema.Type.INT), createPrimitiveField("field2", Schema.Type.STRING)); - fieldNames = AvroInternalSchemaConverter.collectColNamesFromSchema(simpleSchemaWithNullable); - expectedOutput = Arrays.asList("field1", "field2"); - assertEquals(expectedOutput.size(), fieldNames.size()); - assertTrue(fieldNames.containsAll(expectedOutput)); + } - Schema complexSchemaSingleLevel = createRecord("complexSchemaSingleLevel", + public static Schema getComplexSchemaSingleLevel() { + return createRecord("complexSchemaSingleLevel", AvroSchemaTestUtils.createNestedField("field1", Schema.Type.INT), createArrayField("field2", Schema.Type.STRING), createMapField("field3", Schema.Type.DOUBLE)); - fieldNames = AvroInternalSchemaConverter.collectColNamesFromSchema(complexSchemaSingleLevel); - expectedOutput = Arrays.asList("field1.nested", "field2.element", "field3.key", "field3.value"); - assertEquals(expectedOutput.size(), fieldNames.size()); - assertTrue(fieldNames.containsAll(expectedOutput)); + } + + public static List getComplexSchemaSingleLevelExpectedColumnNames() { + return Arrays.asList("field1.nested", "field2.element", "field3.key", "field3.value"); + } - Schema deeplyNestedField = createRecord("deeplyNestedField", + public static Schema getDeeplyNestedFieldSchema() { + return createRecord("deeplyNestedFieldSchema", createPrimitiveField("field1", Schema.Type.INT), new Schema.Field("field2", createRecord("field2nest", @@ -76,9 +75,37 @@ public void testCollectColumnNames() { createNullablePrimitiveField("field21", Schema.Type.INT), createNullablePrimitiveField("field22", Schema.Type.INT))))), createNullablePrimitiveField("field3", Schema.Type.INT)); - fieldNames = AvroInternalSchemaConverter.collectColNamesFromSchema(deeplyNestedField); - expectedOutput = Arrays.asList("field1", "field2.field2nestarray.element.field21", + } + + public static List getDeeplyNestedFieldSchemaExpectedColumnNames() { + return Arrays.asList("field1", "field2.field2nestarray.element.field21", "field2.field2nestarray.element.field22", "field3"); + } + + @Test + public void testCollectColumnNames() { + Schema simpleSchema = getSimpleSchema(); + List fieldNames = AvroInternalSchemaConverter.collectColNamesFromSchema(simpleSchema); + List expectedOutput = getSimpleSchemaExpectedColumnNames(); + assertEquals(expectedOutput.size(), fieldNames.size()); + assertTrue(fieldNames.containsAll(expectedOutput)); + + + Schema simpleSchemaWithNullable = getSimpleSchemaWithNullable(); + fieldNames = AvroInternalSchemaConverter.collectColNamesFromSchema(simpleSchemaWithNullable); + expectedOutput = getSimpleSchemaExpectedColumnNames(); + assertEquals(expectedOutput.size(), fieldNames.size()); + assertTrue(fieldNames.containsAll(expectedOutput)); + + Schema complexSchemaSingleLevel = getComplexSchemaSingleLevel(); + fieldNames = AvroInternalSchemaConverter.collectColNamesFromSchema(complexSchemaSingleLevel); + expectedOutput = getComplexSchemaSingleLevelExpectedColumnNames(); + assertEquals(expectedOutput.size(), fieldNames.size()); + assertTrue(fieldNames.containsAll(expectedOutput)); + + Schema deeplyNestedFieldSchema = getDeeplyNestedFieldSchema(); + fieldNames = AvroInternalSchemaConverter.collectColNamesFromSchema(deeplyNestedFieldSchema); + expectedOutput = getDeeplyNestedFieldSchemaExpectedColumnNames(); assertEquals(expectedOutput.size(), fieldNames.size()); assertTrue(fieldNames.containsAll(expectedOutput)); } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/util/TestSparkInternalSchemaConverter.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/util/TestSparkInternalSchemaConverter.scala new file mode 100644 index 000000000000..3419a25bd1a0 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/util/TestSparkInternalSchemaConverter.scala @@ -0,0 +1,93 @@ +/* + * 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.hudi.util + +import org.apache.avro.Schema +import org.apache.hudi.SparkAdapterSupport +import org.apache.hudi.client.utils.SparkInternalSchemaConverter.collectColNamesFromSparkStruct +import org.apache.hudi.internal.schema.convert.TestAvroInternalSchemaConverter._ +import org.apache.hudi.testutils.HoodieSparkClientTestHarness +import org.apache.spark.sql.types._ +import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} +import org.junit.jupiter.api.Test + +class TestSparkInternalSchemaConverter extends HoodieSparkClientTestHarness with SparkAdapterSupport { + + private def getStructType(schema: Schema): DataType = { + sparkAdapter.getAvroSchemaConverters.toSqlType(schema)._1 + } + + @Test + def testCollectColumnNames(): Unit = { + val simpleSchema = StructType(Seq( + StructField("field1", IntegerType, nullable = false), + StructField("field2", StringType, nullable = false))) + + assertEquals(getStructType(getSimpleSchema).json, simpleSchema.json) + var fieldNames = collectColNamesFromSparkStruct(simpleSchema) + var expectedOutput = getSimpleSchemaExpectedColumnNames() + assertEquals(expectedOutput.size(), fieldNames.size()) + assertTrue(fieldNames.containsAll(expectedOutput)) + + val simpleSchemaWithNullable = StructType(Seq( + StructField("field1", IntegerType, nullable = true), + StructField("field2", StringType, nullable = false))) + + assertEquals(getStructType(getSimpleSchemaWithNullable).json, simpleSchemaWithNullable.json) + fieldNames = collectColNamesFromSparkStruct(simpleSchemaWithNullable) + expectedOutput = getSimpleSchemaExpectedColumnNames() + assertEquals(expectedOutput.size(), fieldNames.size()) + assertTrue(fieldNames.containsAll(expectedOutput)) + + val complexSchemaSingleLevel = StructType(Seq( + StructField("field1", StructType(Seq( + StructField("nested", IntegerType, nullable = false) + )), nullable = false), + StructField("field2", ArrayType(StringType, containsNull = false), nullable = false), + StructField("field3", MapType(StringType, DoubleType, valueContainsNull = false), nullable = false) + )) + + assertEquals(getStructType(getComplexSchemaSingleLevel).json, complexSchemaSingleLevel.json) + fieldNames = collectColNamesFromSparkStruct(complexSchemaSingleLevel) + expectedOutput = getComplexSchemaSingleLevelExpectedColumnNames() + assertEquals(expectedOutput.size(), fieldNames.size()) + assertTrue(fieldNames.containsAll(expectedOutput)) + + val deeplyNestedField = StructType(Seq( + StructField("field1", IntegerType, nullable = false), + StructField("field2", StructType(Seq( + StructField("field2nestarray", + ArrayType( + StructType(Seq( + StructField("field21", IntegerType, nullable = true), + StructField("field22", IntegerType, nullable = true) + )), containsNull = true), + nullable = false) + )), nullable = false), + StructField("field3", IntegerType, nullable = true) + )) + + assertEquals(getStructType(getDeeplyNestedFieldSchema).json, deeplyNestedField.json) + fieldNames = collectColNamesFromSparkStruct(deeplyNestedField) + expectedOutput = getDeeplyNestedFieldSchemaExpectedColumnNames() + assertEquals(expectedOutput.size(), fieldNames.size()) + assertTrue(fieldNames.containsAll(expectedOutput)) + } +} From 36d0b151cbd361eb0dc6444e800ba65ccf4beaa7 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Tue, 4 Jun 2024 16:44:23 -0400 Subject: [PATCH 84/86] fix build errors --- .../TestAvroInternalSchemaConverter.java | 2 +- .../TestSparkInternalSchemaConverter.scala | 24 +++++++++---------- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/hudi-common/src/test/java/org/apache/hudi/internal/schema/convert/TestAvroInternalSchemaConverter.java b/hudi-common/src/test/java/org/apache/hudi/internal/schema/convert/TestAvroInternalSchemaConverter.java index 8992f0c27fbe..8bf7eabce029 100644 --- a/hudi-common/src/test/java/org/apache/hudi/internal/schema/convert/TestAvroInternalSchemaConverter.java +++ b/hudi-common/src/test/java/org/apache/hudi/internal/schema/convert/TestAvroInternalSchemaConverter.java @@ -73,7 +73,7 @@ public static Schema getDeeplyNestedFieldSchema() { createArrayField("field2nestarray", createNullableRecord("field2nestarraynest", createNullablePrimitiveField("field21", Schema.Type.INT), - createNullablePrimitiveField("field22", Schema.Type.INT))))), + createNullablePrimitiveField("field22", Schema.Type.INT)))), null, null), createNullablePrimitiveField("field3", Schema.Type.INT)); } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/util/TestSparkInternalSchemaConverter.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/util/TestSparkInternalSchemaConverter.scala index 3419a25bd1a0..d2ba8c3582c9 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/util/TestSparkInternalSchemaConverter.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/util/TestSparkInternalSchemaConverter.scala @@ -37,8 +37,8 @@ class TestSparkInternalSchemaConverter extends HoodieSparkClientTestHarness with @Test def testCollectColumnNames(): Unit = { val simpleSchema = StructType(Seq( - StructField("field1", IntegerType, nullable = false), - StructField("field2", StringType, nullable = false))) + StructField("field1", IntegerType, nullable = false, Metadata.empty), + StructField("field2", StringType, nullable = false, Metadata.empty))) assertEquals(getStructType(getSimpleSchema).json, simpleSchema.json) var fieldNames = collectColNamesFromSparkStruct(simpleSchema) @@ -47,8 +47,8 @@ class TestSparkInternalSchemaConverter extends HoodieSparkClientTestHarness with assertTrue(fieldNames.containsAll(expectedOutput)) val simpleSchemaWithNullable = StructType(Seq( - StructField("field1", IntegerType, nullable = true), - StructField("field2", StringType, nullable = false))) + StructField("field1", IntegerType, nullable = true, Metadata.empty), + StructField("field2", StringType, nullable = false, Metadata.empty))) assertEquals(getStructType(getSimpleSchemaWithNullable).json, simpleSchemaWithNullable.json) fieldNames = collectColNamesFromSparkStruct(simpleSchemaWithNullable) @@ -58,10 +58,10 @@ class TestSparkInternalSchemaConverter extends HoodieSparkClientTestHarness with val complexSchemaSingleLevel = StructType(Seq( StructField("field1", StructType(Seq( - StructField("nested", IntegerType, nullable = false) - )), nullable = false), - StructField("field2", ArrayType(StringType, containsNull = false), nullable = false), - StructField("field3", MapType(StringType, DoubleType, valueContainsNull = false), nullable = false) + StructField("nested", IntegerType, nullable = false, Metadata.empty) + )), nullable = false, Metadata.empty), + StructField("field2", ArrayType(StringType, containsNull = false), nullable = false, Metadata.empty), + StructField("field3", MapType(StringType, DoubleType, valueContainsNull = false), nullable = false, Metadata.empty) )) assertEquals(getStructType(getComplexSchemaSingleLevel).json, complexSchemaSingleLevel.json) @@ -71,17 +71,17 @@ class TestSparkInternalSchemaConverter extends HoodieSparkClientTestHarness with assertTrue(fieldNames.containsAll(expectedOutput)) val deeplyNestedField = StructType(Seq( - StructField("field1", IntegerType, nullable = false), + StructField("field1", IntegerType, nullable = false, Metadata.empty), StructField("field2", StructType(Seq( StructField("field2nestarray", ArrayType( StructType(Seq( - StructField("field21", IntegerType, nullable = true), - StructField("field22", IntegerType, nullable = true) + StructField("field21", IntegerType, nullable = true, Metadata.empty), + StructField("field22", IntegerType, nullable = true, Metadata.empty) )), containsNull = true), nullable = false) )), nullable = false), - StructField("field3", IntegerType, nullable = true) + StructField("field3", IntegerType, nullable = true, Metadata.empty) )) assertEquals(getStructType(getDeeplyNestedFieldSchema).json, deeplyNestedField.json) From e710020df011ae0e9aac4284126dbc226533e6d5 Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Wed, 5 Jun 2024 15:56:42 -0400 Subject: [PATCH 85/86] make default value -1 for position column --- .../common/table/read/HoodiePositionBasedSchemaHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedSchemaHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedSchemaHandler.java index b001f5767dcc..9d34c1d7e332 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedSchemaHandler.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodiePositionBasedSchemaHandler.java @@ -58,7 +58,7 @@ private Schema addPositionalMergeCol(Schema input) { private Schema.Field getPositionalMergeField() { return new Schema.Field(HoodiePositionBasedFileGroupRecordBuffer.ROW_INDEX_TEMPORARY_COLUMN_NAME, - Schema.create(Schema.Type.LONG), "", 0L); + Schema.create(Schema.Type.LONG), "", -1L); } @Override From a4f3d9a64cc59f67bda1b9f9e045774b29213d2c Mon Sep 17 00:00:00 2001 From: Jonathan Vexler <=> Date: Wed, 5 Jun 2024 17:52:56 -0400 Subject: [PATCH 86/86] replace use of parquet reader with fg reader in the cdc iterator --- .../hudi/cdc/CDCFileGroupIterator.scala | 83 ++++++++++--------- ...ileGroupReaderBasedParquetFileFormat.scala | 22 ++--- 2 files changed, 51 insertions(+), 54 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala index 6d11b9152b53..60758e840c9d 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCFileGroupIterator.scala @@ -19,11 +19,14 @@ package org.apache.hudi.cdc -import org.apache.hudi.HoodieBaseRelation.BaseFileReader +import org.apache.avro.Schema +import org.apache.avro.generic.{GenericData, GenericRecord, IndexedRecord} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path import org.apache.hudi.HoodieConversionUtils.toScalaOption import org.apache.hudi.HoodieDataSourceHelper.AvroDeserializerSupport import org.apache.hudi.avro.HoodieAvroUtils -import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties} +import org.apache.hudi.common.config.{HoodieCommonConfig, HoodieMemoryConfig, HoodieMetadataConfig, TypedProperties} import org.apache.hudi.common.model.{FileSlice, HoodieAvroRecordMerger, HoodieLogFile, HoodieRecord, HoodieRecordMerger, HoodieRecordPayload} import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.cdc.HoodieCDCInferenceCase._ @@ -31,28 +34,25 @@ import org.apache.hudi.common.table.cdc.HoodieCDCOperation._ import org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode._ import org.apache.hudi.common.table.cdc.{HoodieCDCFileSplit, HoodieCDCUtils} import org.apache.hudi.common.table.log.HoodieCDCLogRecordIterator +import org.apache.hudi.common.table.read.HoodieFileGroupReader +import org.apache.hudi.common.util.FileIOUtils import org.apache.hudi.common.util.ValidationUtils.checkState +import org.apache.hudi.common.util.collection.ExternalSpillableMap.DiskMapType import org.apache.hudi.config.HoodiePayloadConfig import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory import org.apache.hudi.storage.{StorageConfiguration, StoragePath} -import org.apache.hudi.{AvroConversionUtils, AvroProjection, HoodieMergeOnReadFileSplit, HoodieTableSchema, HoodieTableState, LogFileIterator, RecordMergingFileIterator, SparkAdapterSupport} - -import org.apache.avro.Schema -import org.apache.avro.generic.{GenericData, GenericRecord, IndexedRecord} -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path +import org.apache.hudi.{AvroConversionUtils, AvroProjection, HoodieMergeOnReadFileSplit, HoodieTableSchema, HoodieTableState, LogFileIterator, SparkAdapterSupport, SparkFileFormatInternalRowReaderContext} import org.apache.spark.sql.HoodieCatalystExpressionUtils.generateUnsafeProjection import org.apache.spark.sql.avro.HoodieAvroDeserializer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Projection -import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.execution.datasources.parquet.SparkParquetReader import org.apache.spark.sql.types.StructType import org.apache.spark.unsafe.types.UTF8String import java.io.Closeable -import java.util.Properties import java.util.stream.Collectors - +import java.util.{Locale, Properties} import scala.annotation.tailrec import scala.collection.JavaConverters._ import scala.collection.mutable @@ -60,11 +60,12 @@ import scala.collection.mutable class CDCFileGroupIterator(split: HoodieCDCFileGroupSplit, metaClient: HoodieTableMetaClient, conf: StorageConfiguration[_], - parquetReader: PartitionedFile => Iterator[InternalRow], + sparkParquetReader: SparkParquetReader, originTableSchema: HoodieTableSchema, cdcSchema: StructType, requiredCdcSchema: StructType, - props: TypedProperties) + props: TypedProperties, + options: Map[String, String]) extends Iterator[InternalRow] with SparkAdapterSupport with AvroDeserializerSupport with Closeable { @@ -376,7 +377,8 @@ class CDCFileGroupIterator(split: HoodieCDCFileGroupSplit, val pf = sparkPartitionedFileUtils.createPartitionedFile( InternalRow.empty, absCDCPath, 0, fileStatus.getLength) - recordIter = parquetReader(pf) + recordIter = sparkParquetReader.read(pf, structTypeSchema, new StructType(), Seq.empty, + storage.getConf.asInstanceOf[StorageConfiguration[Configuration]]) case BASE_FILE_DELETE => assert(currentCDCFileSplit.getBeforeFileSlice.isPresent) recordIter = loadFileSlice(currentCDCFileSplit.getBeforeFileSlice.get) @@ -479,32 +481,35 @@ class CDCFileGroupIterator(split: HoodieCDCFileGroupSplit, } private def loadFileSlice(fileSlice: FileSlice): Iterator[InternalRow] = { - val baseFileInfo = storage.getPathInfo(fileSlice.getBaseFile.get().getStoragePath) - val basePartitionedFile = sparkPartitionedFileUtils.createPartitionedFile( - InternalRow.empty, - baseFileInfo.getPath, + val readerContext = new SparkFileFormatInternalRowReaderContext(sparkParquetReader, recordKeyField, Seq.empty) + val reader = new HoodieFileGroupReader[InternalRow]( + readerContext, + storage, + tableState.tablePath, + tableState.latestCommitTimestamp.get, + fileSlice, + avroSchema, + avroSchema, + org.apache.hudi.common.util.Option.empty(), + metaClient, + metaClient.getTableConfig.getProps, + metaClient.getTableConfig, 0, - baseFileInfo.getLength - ) - val logFiles = fileSlice.getLogFiles - .sorted(HoodieLogFile.getLogFileComparator) - .collect(Collectors.toList[HoodieLogFile]) - .asScala.toList - .filterNot(_.getFileName.endsWith(HoodieCDCUtils.CDC_LOGFILE_SUFFIX)) - - if (logFiles.isEmpty) { - // no log files, just load the base parquet file - parquetReader(basePartitionedFile) - } else { - // use [[RecordMergingFileIterator]] to load both the base file and log files - val morSplit = HoodieMergeOnReadFileSplit(Some(basePartitionedFile), logFiles) - new RecordMergingFileIterator( - morSplit, - BaseFileReader(parquetReader, originTableSchema.structTypeSchema), - originTableSchema, - originTableSchema, - tableState, - conf.unwrapAs(classOf[Configuration])) + storage.getPathInfo(fileSlice.getBaseFile.get().getStoragePath).getLength, + false, + options.getOrElse(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE.key(), HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE.defaultValue() + "").toLong, + options.getOrElse(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH.key(), FileIOUtils.getDefaultSpillableMapBasePath), + DiskMapType.valueOf(options.getOrElse(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.key(), HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue().name()).toUpperCase(Locale.ROOT)), + options.getOrElse(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.key(), HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue().toString).toBoolean) + reader.initRecordIterators() + val iter = reader.getClosableIterator + + new Iterator[InternalRow] with Closeable { + override def hasNext: Boolean = iter.hasNext + + override def next(): InternalRow = iter.next() + + override def close(): Unit = iter.close() } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala index 0412d202148a..3dde68fb35de 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedParquetFileFormat.scala @@ -117,16 +117,6 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, setSchemaEvolutionConfigs(augmentedStorageConf) val baseFileReader = super.buildReaderWithPartitionValues(spark, dataSchema, partitionSchema, requiredSchema, filters ++ requiredFilters, options, augmentedStorageConf.unwrapCopy()) - val cdcFileReader = if (isCDC) { - super.buildReaderWithPartitionValues( - spark, - tableSchema.structTypeSchema, - StructType(Nil), - tableSchema.structTypeSchema, - Nil, - options, - new Configuration(hadoopConf)) - } val requestedAvroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(requiredSchema, sanitizedTableName) val dataAvroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(dataSchema, sanitizedTableName) @@ -190,8 +180,8 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, val fileSplits = hoodiePartitionCDCFileGroupSliceMapping.getFileSplits().toArray val fileGroupSplit: HoodieCDCFileGroupSplit = HoodieCDCFileGroupSplit(fileSplits) buildCDCRecordIterator( - fileGroupSplit, cdcFileReader.asInstanceOf[PartitionedFile => Iterator[InternalRow]], - broadcastedStorageConf.value, fileIndexProps, requiredSchema) + fileGroupSplit, parquetFileReader.value, broadcastedStorageConf.value, + fileIndexProps, options, requiredSchema) case _ => parquetFileReader.value.read(file, requiredSchema, partitionSchema, filters, broadcastedStorageConf.value) @@ -208,9 +198,10 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, } protected def buildCDCRecordIterator(cdcFileGroupSplit: HoodieCDCFileGroupSplit, - cdcFileReader: PartitionedFile => Iterator[InternalRow], + sparkParquetReader: SparkParquetReader, storageConf: StorageConfiguration[_], props: TypedProperties, + options: Map[String, String], requiredSchema: StructType): Iterator[InternalRow] = { props.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_KEY, tableName) val cdcSchema = CDCRelation.FULL_CDC_SPARK_SCHEMA @@ -220,11 +211,12 @@ class HoodieFileGroupReaderBasedParquetFileFormat(tableState: HoodieTableState, cdcFileGroupSplit, metaClient, storageConf, - cdcFileReader, + sparkParquetReader, tableSchema, cdcSchema, requiredSchema, - props) + props, + options) } private def appendPartitionAndProject(iter: HoodieFileGroupReader.HoodieFileGroupReaderIterator[InternalRow],