jonvex commented on code in PR #12622:
URL: https://github.com/apache/hudi/pull/12622#discussion_r2085369094
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/HoodieFileGroupReaderBasedFileFormat.scala:
##########
Review Comment:
This seems pretty much like a copy of parquet file format. So we should
probably just either combine the two or dedup the code
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/SparkReaderContextFactory.java:
##########
@@ -109,11 +114,19 @@ public HoodieReaderContext<InternalRow> getContext() {
throw new HoodieException("Table config broadcast is not initialized.");
}
- SparkParquetReader sparkParquetReader = parquetReaderBroadcast.getValue();
- if (sparkParquetReader != null) {
+ SparkFileReader parquetFileReader = parquetReaderBroadcast.getValue();
+ SparkFileReader orcFileReader = orcReaderBroadcast.getValue();
+ Map<String, SparkFileReader> fileReaders = new HashMap<>();
+ if (orcFileReader != null) {
+ fileReaders.put("orc", orcFileReader);
Review Comment:
Maybe use HoodieFileFormat instead of just strings inline?
##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala:
##########
@@ -21,7 +21,6 @@ package org.apache.spark.sql.hudi
import org.apache.hudi.client.utils.SparkRowSerDe
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.storage.StoragePath
Review Comment:
I think the extra line between hudi and non-hudi is supposed to be here
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/HoodieFileGroupReaderBasedFileFormat.scala:
##########
@@ -0,0 +1,381 @@
+/*
+ * 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.hudi.{AvroConversionUtils, HoodieFileIndex,
HoodiePartitionCDCFileGroupMapping, HoodiePartitionFileSliceMapping,
HoodieTableSchema, SparkAdapterSupport, SparkFileFormatInternalRowReaderContext}
+import org.apache.hudi.avro.AvroSchemaUtils
+import org.apache.hudi.cdc.{CDCFileGroupIterator, CDCRelation,
HoodieCDCFileGroupSplit}
+import org.apache.hudi.client.utils.SparkInternalSchemaConverter
+import org.apache.hudi.common.config.TypedProperties
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
+import org.apache.hudi.common.table.read.HoodieFileGroupReader
+import org.apache.hudi.internal.schema.InternalSchema
+import org.apache.hudi.storage.StorageConfiguration
+import org.apache.hudi.storage.hadoop.{HadoopStorageConfiguration,
HoodieHadoopStorage}
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileStatus, Path}
+import org.apache.hadoop.mapreduce.Job
+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.HoodieFileGroupReaderBasedFileFormat.{ORC_FILE_EXTENSION,
PARQUET_FILE_EXTENSION}
+import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat
+import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat,
SparkFileReader}
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector,
OnHeapColumnVector}
+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.vectorized.{ColumnarBatch, ColumnarBatchUtils}
+import org.apache.spark.util.SerializableConfiguration
+
+import java.io.Closeable
+
+trait HoodieFormatTrait {
+
+ // Used so that the planner only projects once and does not stack overflow
+ var isProjected: Boolean = false
+ def getRequiredFilters: Seq[Filter]
+}
+
+/**
+ * This class utilizes {@link HoodieFileGroupReader} and its related classes
to support reading
+ * from Parquet formatted base files and their log files.
+ */
+class HoodieFileGroupReaderBasedFileFormat(tablePath: String,
+ tableSchema: HoodieTableSchema,
+ tableName: String,
+ queryTimestamp: String,
+ mandatoryFields: Seq[String],
+ isMOR: Boolean,
+ isBootstrap: Boolean,
+ isIncremental: Boolean,
+ validCommits: String,
+ shouldUseRecordPosition: Boolean,
+ requiredFilters: Seq[Filter])
+ extends FileFormat with SparkAdapterSupport with HoodieFormatTrait with
Serializable {
+ protected var supportBatchCalled = false
+ protected var supportBatchResult = false
+
+ def getRequiredFilters: Seq[Filter] = requiredFilters
+
+ /**
+ * Conditions to support batch:
+ * 1. Parent file format supports batch,
+ * 2. This read is not for a MOR table,
+ * 3. This read is not an incremental read, and
+ * 4. This read is not a bootstrap read.
+ * TODO: remove query type constraint constraints.
+ * TODO: refactor the logic to be more clear.
+ */
+ override def supportBatch(sparkSession: SparkSession, schema: StructType):
Boolean = {
+ if (!supportBatchCalled || supportBatchResult) {
+ supportBatchCalled = true
+ supportBatchResult = !isMOR && !isIncremental && !isBootstrap &&
super.supportBatch(sparkSession, schema)
+ }
+ supportBatchResult
+ }
+
+ override def isSplitable(sparkSession: SparkSession,
+ options: Map[String, String],
+ path: Path): Boolean = false
+
+ // For partition columns that we read from the file, we don't want them to
be constant column vectors so we
+ // modify the vector types in this scenario
+ override def vectorTypes(requiredSchema: StructType,
+ partitionSchema: StructType,
+ sqlConf: SQLConf): Option[Seq[String]] = {
+ val originalVectorTypes = super.vectorTypes(requiredSchema,
partitionSchema, sqlConf)
+ if (mandatoryFields.isEmpty) {
+ originalVectorTypes
+ } else {
+ val regularVectorType = if (!sqlConf.offHeapColumnVectorEnabled) {
+ classOf[OnHeapColumnVector].getName
+ } else {
+ classOf[OffHeapColumnVector].getName
+ }
+ originalVectorTypes.map {
+ o: Seq[String] => o.zipWithIndex.map(a => {
+ if (a._2 >= requiredSchema.length && mandatoryFields.contains(
+ partitionSchema.fields(a._2 - requiredSchema.length).name)) {
+ regularVectorType
+ } else {
+ a._1
+ }
+ })
+ }
+ }
+ }
+
+ protected val sanitizedTableName =
AvroSchemaUtils.getAvroRecordQualifiedName(tableName)
+
+ protected 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 buildReaderWithPartitionValues(spark: SparkSession,
+ dataSchema: StructType,
+ partitionSchema: StructType,
+ requiredSchema: StructType,
+ filters: Seq[Filter],
+ options: Map[String, String],
+ hadoopConf: Configuration):
PartitionedFile => Iterator[InternalRow] = {
+ val outputSchema = StructType(requiredSchema.fields ++
partitionSchema.fields)
+ val isCount = requiredSchema.isEmpty && !isMOR && !isIncremental
+ val augmentedStorageConf = new
HadoopStorageConfiguration(hadoopConf).getInline
+ setSchemaEvolutionConfigs(augmentedStorageConf)
+ val (remainingPartitionSchemaArr, fixedPartitionIndexesArr) =
+ partitionSchema.fields.toSeq.zipWithIndex.filter(p =>
!mandatoryFields.contains(p._1.name)).unzip
+
+ // The schema of the partition cols we want to append the value instead of
reading from the file
+ val remainingPartitionSchema = StructType(remainingPartitionSchemaArr)
+
+ // index positions of the remainingPartitionSchema fields in
partitionSchema
+ val fixedPartitionIndexes = fixedPartitionIndexesArr.toSet
+
+ // schema that we want fg reader to output to us
+ val requestedSchema = StructType(requiredSchema.fields ++
partitionSchema.fields.filter(f => mandatoryFields.contains(f.name)))
+ val requestedAvroSchema =
AvroConversionUtils.convertStructTypeToAvroSchema(requestedSchema,
sanitizedTableName)
+ val dataAvroSchema =
AvroConversionUtils.convertStructTypeToAvroSchema(dataSchema,
sanitizedTableName)
+ val broadcastedParquetFileReader = spark.sparkContext.broadcast(
+ sparkAdapter.createParquetFileReader(supportBatchResult,
spark.sessionState.conf, options, augmentedStorageConf.unwrap()))
+ val broadcastedOrcFileReader = spark.sparkContext.broadcast(
+ sparkAdapter.createOrcFileReader(supportBatchCalled,
spark.sessionState.conf, options, augmentedStorageConf.unwrap()))
+ val broadcastedStorageConf = spark.sparkContext.broadcast(new
SerializableConfiguration(augmentedStorageConf.unwrap()))
+ val fileIndexProps: TypedProperties =
HoodieFileIndex.getConfigProperties(spark, options, null)
+
+ (file: PartitionedFile) => {
+ val storageConf = new
HadoopStorageConfiguration(broadcastedStorageConf.value.value)
+ file.partitionValues match {
+ // Snapshot or incremental queries.
+ case fileSliceMapping: HoodiePartitionFileSliceMapping =>
+ val filegroupName = FSUtils.getFileIdFromFilePath(
+
sparkAdapter.getSparkPartitionedFileUtils.getPathFromPartitionedFile(file))
+ fileSliceMapping.getSlice(filegroupName) match {
+ case Some(fileSlice) if !isCount && (requiredSchema.nonEmpty ||
fileSlice.getLogFiles.findAny().isPresent) =>
+ val fileReaders = new java.util.HashMap[String,
SparkFileReader]()
+ fileReaders.put(ORC_FILE_EXTENSION,
broadcastedOrcFileReader.value)
+ fileReaders.put(PARQUET_FILE_EXTENSION,
broadcastedParquetFileReader.value)
+ val metaClient: HoodieTableMetaClient = HoodieTableMetaClient
+ .builder().setConf(storageConf).setBasePath(tablePath).build
+ val readerContext = new SparkFileFormatInternalRowReaderContext(
+ fileReaders, filters, requiredFilters, storageConf,
metaClient.getTableConfig)
+ val props = metaClient.getTableConfig.getProps
+ options.foreach(kv => props.setProperty(kv._1, kv._2))
+ val reader = new HoodieFileGroupReader[InternalRow](
+ readerContext,
+ new HoodieHadoopStorage(metaClient.getBasePath, storageConf),
+ tablePath,
+ queryTimestamp,
+ fileSlice,
+ dataAvroSchema,
+ requestedAvroSchema,
+ internalSchemaOpt,
+ metaClient,
+ props,
+ file.start,
+ file.length,
+ shouldUseRecordPosition)
+ reader.initRecordIterators()
+ // Append partition values to rows and project to output schema
+ appendPartitionAndProject(
+ reader.getClosableIterator,
+ requestedSchema,
+ remainingPartitionSchema,
+ outputSchema,
+ fileSliceMapping.getPartitionValues,
+ fixedPartitionIndexes)
+
+ case _ =>
+ val filePath =
sparkAdapter.getSparkPartitionedFileUtils.getPathFromPartitionedFile(file)
+ val baseFileFormat = detectFileFormat(filePath.toString)
+ baseFileFormat match {
+ case PARQUET_FILE_EXTENSION => readBaseFile(file,
broadcastedParquetFileReader.value, requestedSchema, remainingPartitionSchema,
fixedPartitionIndexes,
+ requiredSchema, partitionSchema, outputSchema, filters,
storageConf)
+ case ORC_FILE_EXTENSION => readBaseFile(file,
broadcastedOrcFileReader.value, requestedSchema, remainingPartitionSchema,
fixedPartitionIndexes,
+ requiredSchema, partitionSchema, outputSchema, filters,
storageConf)
+ }
+ }
+ // CDC queries.
+ case hoodiePartitionCDCFileGroupSliceMapping:
HoodiePartitionCDCFileGroupMapping =>
+ buildCDCRecordIterator(hoodiePartitionCDCFileGroupSliceMapping,
broadcastedParquetFileReader.value, storageConf, fileIndexProps, requiredSchema)
+
+ case _ =>
+ val filePath =
sparkAdapter.getSparkPartitionedFileUtils.getPathFromPartitionedFile(file)
+ val baseFileFormat = detectFileFormat(filePath.toString)
+ baseFileFormat match {
+ case PARQUET_FILE_EXTENSION => readBaseFile(file,
broadcastedParquetFileReader.value, requestedSchema, remainingPartitionSchema,
fixedPartitionIndexes,
+ requiredSchema, partitionSchema, outputSchema, filters,
storageConf)
+ case ORC_FILE_EXTENSION => readBaseFile(file,
broadcastedOrcFileReader.value, requestedSchema, remainingPartitionSchema,
fixedPartitionIndexes,
+ requiredSchema, partitionSchema, outputSchema, filters,
storageConf)
+ }
+ }
+ }
Review Comment:
Need to add in CloseableIteratorListener.addListener() if we don't unify
this with parquet file format
##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala:
##########
@@ -97,7 +97,8 @@ class
SparkFileFormatInternalRowReaderContext(parquetFileReader: SparkParquetRea
val fileInfo = sparkAdapter.getSparkPartitionedFileUtils
.createPartitionedFile(InternalRow.empty, filePath, start, length)
val (readSchema, readFilters) = getSchemaAndFiltersForRead(structType,
hasRowIndexField)
- new CloseableInternalRowIterator(parquetFileReader.read(fileInfo,
+ val fileReader = if (fileInfo.filePath.toString.endsWith("orc"))
fileReaders.get("orc") else fileReaders.get("parquet")
Review Comment:
Yeah definitely seems like we should be using the enum instead of strings
##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkFileReader.scala:
##########
@@ -29,7 +29,7 @@ import
org.apache.spark.sql.execution.datasources.PartitionedFile
import org.apache.spark.sql.sources.Filter
import org.apache.spark.sql.types.StructType
-trait SparkParquetReader extends Serializable {
+trait SparkFileReader extends Serializable {
/**
* Read an individual parquet file
Review Comment:
update docs
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala:
##########
@@ -290,9 +290,10 @@ class
HoodieMergeOnReadSnapshotHadoopFsRelationFactory(override val sqlContext:
override def buildFileFormat(): FileFormat = {
if (metaClient.getTableConfig.isMultipleBaseFileFormatsEnabled &&
!isBootstrap) {
- new
HoodieMultipleBaseFileFormat(sparkSession.sparkContext.broadcast(tableState),
-
sparkSession.sparkContext.broadcast(HoodieTableSchema(tableStructSchema,
tableAvroSchema.toString, internalSchemaOpt)),
- metaClient.getTableConfig.getTableName, mergeType, mandatoryFields,
true, false, Seq.empty)
+ new HoodieFileGroupReaderBasedFileFormat(
Review Comment:
Consider if we should just unify the two formats
##########
hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/Spark33OrcReader.scala:
##########
Review Comment:
I looked at spark 3.3.0 and 3.3.4 and this doesn't really seem to resemble
either of them. This will make maintaining a lot more difficult if lots of
changes are required for each new reader
spark 3.3.0:
https://github.com/apache/spark/blob/v3.3.0/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala
spark 3.3.4:
https://github.com/apache/spark/blob/v3.3.4/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]