codope commented on code in PR #9761:
URL: https://github.com/apache/hudi/pull/9761#discussion_r1372534307


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/HoodieMultipleBaseFileFormat.scala:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.{FileStatus, Path}
+import org.apache.hadoop.mapreduce.Job
+import 
org.apache.hudi.DataSourceReadOptions.{REALTIME_PAYLOAD_COMBINE_OPT_VAL, 
REALTIME_SKIP_MERGE_OPT_VAL}
+import org.apache.hudi.MergeOnReadSnapshotRelation.createPartitionedFile
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.model.{FileSlice, HoodieLogFile}
+import org.apache.hudi.{HoodieBaseRelation, HoodieTableSchema, 
HoodieTableState, LogFileIterator, MergeOnReadSnapshotRelation, 
PartitionFileSliceMapping, RecordMergingFileIterator, SparkAdapterSupport}
+import org.apache.spark.broadcast.Broadcast
+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.orc.OrcFileFormat
+import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.{StructField, StructType}
+import org.apache.spark.util.SerializableConfiguration
+
+import scala.collection.mutable
+import scala.jdk.CollectionConverters.asScalaIteratorConverter
+
+/**
+ * File format that supports reading multiple base file formats in a table.
+ */
+class HoodieMultipleBaseFileFormat(tableState: Broadcast[HoodieTableState],
+                                   tableSchema: Broadcast[HoodieTableSchema],
+                                   tableName: String,
+                                   mergeType: String,
+                                   mandatoryFields: Seq[String],
+                                   isMOR: Boolean) extends FileFormat with 
SparkAdapterSupport {
+  private val parquetFormat = new ParquetFileFormat()
+  private val orcFormat = new OrcFileFormat()
+
+  override def inferSchema(sparkSession: SparkSession,
+                           options: Map[String, String],
+                           files: Seq[FileStatus]): Option[StructType] = {
+    // This is a simple heuristic assuming all files have the same extension.
+    val fileFormat = detectFileFormat(files.head.getPath.toString)
+
+    fileFormat match {
+      case "parquet" => parquetFormat.inferSchema(sparkSession, options, files)
+      case "orc" => orcFormat.inferSchema(sparkSession, options, files)
+      case _ => throw new UnsupportedOperationException(s"File format 
$fileFormat is not supported.")
+    }
+  }
+
+  override def isSplitable(sparkSession: SparkSession, options: Map[String, 
String], path: Path): Boolean = {
+    false
+  }
+
+  // Used so that the planner only projects once and does not stack overflow
+  var isProjected = 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) {
+      supportBatchCalled = true
+      supportBatchResult =
+        !isMOR && parquetFormat.supportBatch(sparkSession, schema) && 
orcFormat.supportBatch(sparkSession, schema)
+    }
+    supportBatchResult
+  }
+
+  override def prepareWrite(sparkSession: SparkSession,
+                            job: Job,
+                            options: Map[String, String],
+                            dataSchema: StructType): OutputWriterFactory = {
+    throw new UnsupportedOperationException("Write operations are not 
supported in this example.")
+  }
+
+  override def buildReaderWithPartitionValues(sparkSession: 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 requiredSchemaWithMandatory = if (!isMOR || 
MergeOnReadSnapshotRelation.isProjectionCompatible(tableState.value)) {
+      // add mandatory fields to required schema
+      val added: mutable.Buffer[StructField] = mutable.Buffer[StructField]()
+      for (field <- mandatoryFields) {
+        if (requiredSchema.getFieldIndex(field).isEmpty) {
+          val fieldToAdd = 
dataSchema.fields(dataSchema.getFieldIndex(field).get)
+          added.append(fieldToAdd)
+        }
+      }
+      val addedFields = StructType(added.toArray)
+      StructType(requiredSchema.toArray ++ addedFields.fields)
+    } else {
+      dataSchema
+    }
+
+    val (parquetBaseFileReader, orcBaseFileReader, 
preMergeParquetBaseFileReader, preMergeOrcBaseFileReader) = buildFileReaders(
+      sparkSession, dataSchema, partitionSchema, requiredSchema, filters, 
options, hadoopConf, requiredSchemaWithMandatory)
+
+    val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new 
SerializableConfiguration(hadoopConf))
+    (file: PartitionedFile) => {
+      val filePath = 
sparkAdapter.getSparkPartitionedFileUtils.getPathFromPartitionedFile(file)
+      val fileFormat = detectFileFormat(filePath.toString)
+      file.partitionValues match {
+        case fileSliceMapping: PartitionFileSliceMapping =>
+          if (FSUtils.isLogFile(filePath)) {
+            // no base file
+            val fileSlice = 
fileSliceMapping.getSlice(FSUtils.getFileId(filePath.getName).substring(1)).get
+            val logFiles = getLogFilesFromSlice(fileSlice)
+            val outputAvroSchema = 
HoodieBaseRelation.convertToAvroSchema(outputSchema, tableName)
+            new LogFileIterator(logFiles, filePath.getParent, 
tableSchema.value, outputSchema, outputAvroSchema,
+              tableState.value, broadcastedHadoopConf.value.value)
+          } else {
+            // We do not broadcast the slice if it has no log files
+            fileSliceMapping.getSlice(FSUtils.getFileId(filePath.getName)) 
match {
+              case Some(fileSlice) =>
+                val hoodieBaseFile = fileSlice.getBaseFile.get()
+                val baseFileFormat = 
detectFileFormat(hoodieBaseFile.getFileName)
+                val partitionValues = fileSliceMapping.getInternalRow
+                val logFiles = getLogFilesFromSlice(fileSlice)
+                if (requiredSchemaWithMandatory.isEmpty) {
+                  val baseFile = createPartitionedFile(partitionValues, 
hoodieBaseFile.getHadoopPath, 0, hoodieBaseFile.getFileLen)
+                  baseFileFormat match {
+                    case "parquet" => parquetBaseFileReader(baseFile)
+                    case "orc" => orcBaseFileReader(baseFile)

Review Comment:
   No, it will ultimately fallback to using the table config because the 
`HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT` has infer function - 
https://github.com/apache/hudi/blob/0ad4560f2a4de00e43814b0d6cef2886a8a38155/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java#L103



-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to