jonvex commented on code in PR #9276:
URL: https://github.com/apache/hudi/pull/9276#discussion_r1284863793


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/NewHoodieParquetFileFormat.scala:
##########
@@ -0,0 +1,353 @@
+/*
+ * 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.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.{BaseFile, FileSlice, HoodieLogFile, 
HoodieRecord}
+import org.apache.hudi.common.util.ValidationUtils.checkState
+import org.apache.hudi.{HoodieBaseRelation, HoodieSparkUtils, 
HoodieTableSchema, HoodieTableState, LogFileIterator, 
MergeOnReadSnapshotRelation, PartitionFileSliceMapping, 
RecordMergingFileIterator, SkipMergeIterator, 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.PartitionedFile
+import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isMetaField
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.{StructField, StructType}
+import org.apache.spark.sql.vectorized.{ColumnVector, ColumnarBatch}
+import org.apache.spark.util.SerializableConfiguration
+
+import scala.collection.mutable
+import scala.jdk.CollectionConverters.asScalaIteratorConverter
+
+class NewHoodieParquetFileFormat(tableState: Broadcast[HoodieTableState],
+                                 tableSchema: Broadcast[HoodieTableSchema],
+                                 tableName: String,
+                                 mergeType: String,
+                                 mandatoryFields: Seq[String],
+                                 isMOR: Boolean,
+                                 isBootstrap: Boolean) extends 
ParquetFileFormat with SparkAdapterSupport {
+
+  //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 && super.supportBatch(sparkSession, schema)
+    }
+    supportBatchResult
+  }
+
+  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 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 needMetaCols = requiredMeta.nonEmpty
+    val needDataCols = requiredWithoutMeta.nonEmpty
+    // note: this is only the output of the bootstrap merge if isMOR. If it is 
only bootstrap then the
+    // output will just be outputSchema
+    val bootstrapReaderOutput = StructType(requiredMeta.fields ++ 
requiredWithoutMeta.fields)
+
+    val skeletonReaderAppend = needMetaCols && isBootstrap && !(needDataCols 
|| isMOR) && partitionSchema.nonEmpty
+    val bootstrapBaseAppend = needDataCols && isBootstrap && !isMOR && 
partitionSchema.nonEmpty
+
+    val (baseFileReader, preMergeBaseFileReader, skeletonReader, 
bootstrapBaseReader) = buildFileReaders(sparkSession,
+      dataSchema, partitionSchema, requiredSchema, filters, options, 
hadoopConf, requiredSchemaWithMandatory,
+      requiredWithoutMeta, requiredMeta)
+
+    val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new 
SerializableConfiguration(hadoopConf))
+    (file: PartitionedFile) => {
+      file.partitionValues match {
+        case broadcast: PartitionFileSliceMapping =>

Review Comment:
   So there are 2 things going on here:
   the first is that you are seeing 
   `isBootstrap && bootstrapFileOpt.isPresent` and `isMOR && logFiles.nonEmpty` 
the isBootstrap and isMore are not necessary because if the second condition is 
true then the first has to be true.
   
   The second thing, that I don't think should be changed is 
   ```
                     (isMOR, logFiles.nonEmpty) match {
                       case (true, true) => 
buildMergeOnReadIterator(bootstrapIterator, logFiles, filePath.getParent,
                         bootstrapReaderOutput, requiredSchemaWithMandatory, 
outputSchema, partitionSchema, partitionValues,
                         broadcastedHadoopConf.value.value)
                       case (true, false) => 
appendPartitionAndProject(bootstrapIterator, bootstrapReaderOutput,
                         partitionSchema, outputSchema, partitionValues)
                       case (false, false) => bootstrapIterator
                       case (false, true) => throw new 
IllegalStateException("should not be log files if not mor table")
   ```
   The reasoning for this is that 
   if it's mor and it has log files then we need to merge the log files, then 
append the partition path, then project away any mandatory fields for merging 
(recordkey and precombine) that aren't required
   
   if it's mor and doesn't have log files, then we need to append the 
partitionpath 
   
   if it's not mor, then the partitionpath has already been appended by the 
reader itself, so we just return it.
   
   The final edge case to bring up is if  (requiredSchemaWithMandatory.isEmpty) 
   That means that it is a df.count() so we just use the base file reader 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to