yihua commented on code in PR #9276: URL: https://github.com/apache/hudi/pull/9276#discussion_r1284822235
########## 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: I feel like the branching here can be further simplified based on the split or file group type without having to specify `isMOR` or `isBootstrap`: (1) base file only, (2) base file + log files, (3) log files only, (4) bootstrap skeleton file + original file, (5) bootstrap skeleton file + original file + log files. Then we may apply optimization like predicate push down per split type. We can improve this part in a follow-up, along with aligning logic in different query types (e.g., schema handling, partition path handling, etc.). ########## hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala: ########## @@ -143,8 +146,8 @@ class Spark2Adapter extends SparkAdapter { partitions.toSeq } - override def createHoodieParquetFileFormat(appendPartitionValues: Boolean): Option[ParquetFileFormat] = { - Some(new Spark24HoodieParquetFileFormat(appendPartitionValues)) + override def createLegacyHoodieParquetFileFormat(appendPartitionValues: Boolean): Option[ParquetFileFormat] = { + Some(new Spark24LegacyHoodieParquetFileFormat(appendPartitionValues)) Review Comment: So this is used by `BaseFileOnlyRelation` for COW and MOR read-optimized queries. I assume the new file format can also be applied by COW and MOR read-optimized queries too. We should follow up here in a separate PR. The goal is to get rid of Spark-version-specific file format classes and make the Hudi Spark integration easier to maintain. ########## hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestNewHoodieParquetFileFormat.java: ########## @@ -0,0 +1,137 @@ +/* + * 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.DataSourceReadOptions; +import org.apache.hudi.common.model.HoodieTableType; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.util.Arrays; +import java.util.Map; +import java.util.stream.Stream; + +import static org.apache.hudi.common.model.HoodieTableType.COPY_ON_WRITE; +import static org.apache.hudi.common.model.HoodieTableType.MERGE_ON_READ; +import static org.junit.jupiter.api.Assertions.assertEquals; + +@Tag("functional") +public class TestNewHoodieParquetFileFormat extends TestBootstrapReadBase { + + private static Stream<Arguments> testArgs() { + Stream.Builder<Arguments> b = Stream.builder(); + HoodieTableType[] tableType = {COPY_ON_WRITE, MERGE_ON_READ}; + Integer[] nPartitions = {0, 1, 2}; + for (HoodieTableType tt : tableType) { + for (Integer n : nPartitions) { + b.add(Arguments.of(tt, n)); + } + } + return b.build(); + } + + @ParameterizedTest + @MethodSource("testArgs") + public void runTests(HoodieTableType tableType, Integer nPartitions) { + this.bootstrapType = nPartitions == 0 ? "metadata" : "mixed"; + this.dashPartitions = true; + this.tableType = tableType; + this.nPartitions = nPartitions; + setupDirs(); + + //do bootstrap + Map<String, String> options = setBootstrapOptions(); + Dataset<Row> bootstrapDf = sparkSession.emptyDataFrame(); + bootstrapDf.write().format("hudi") + .options(options) + .mode(SaveMode.Overwrite) + .save(bootstrapTargetPath); + runComparisons(); + + options = basicOptions(); + doUpdate(options, "001"); + runComparisons(); + + doInsert(options, "002"); + runComparisons(); + + doDelete(options, "003"); + runComparisons(); + } + + protected void runComparisons() { + if (tableType.equals(MERGE_ON_READ)) { + runComparison(hudiBasePath); + } + runComparison(bootstrapTargetPath); + } + + protected void runComparison(String tableBasePath) { + testCount(tableBasePath); + runIndividualComparison(tableBasePath); + runIndividualComparison(tableBasePath, "partition_path"); + runIndividualComparison(tableBasePath, "_hoodie_record_key", "_hoodie_commit_time", "_hoodie_partition_path"); + runIndividualComparison(tableBasePath, "_hoodie_commit_time", "_hoodie_commit_seqno"); + runIndividualComparison(tableBasePath, "_hoodie_commit_time", "_hoodie_commit_seqno", "partition_path"); + runIndividualComparison(tableBasePath, "_row_key", "_hoodie_commit_seqno", "_hoodie_record_key", "_hoodie_partition_path"); + runIndividualComparison(tableBasePath, "_row_key", "partition_path", "_hoodie_is_deleted", "begin_lon"); + } + + protected void testCount(String tableBasePath) { + Dataset<Row> legacyDf = sparkSession.read().format("hudi") + .option(DataSourceReadOptions.USE_LEGACY_HUDI_PARQUET_FILE_FORMAT().key(),"true").load(tableBasePath); + Dataset<Row> fileFormatDf = sparkSession.read().format("hudi") + .option(DataSourceReadOptions.USE_LEGACY_HUDI_PARQUET_FILE_FORMAT().key(),"false").load(tableBasePath); + assertEquals(legacyDf.count(), fileFormatDf.count()); + } + + protected scala.collection.Seq<String> seq(String... a) { + return scala.collection.JavaConverters.asScalaBufferConverter(Arrays.asList(a)).asScala().toSeq(); + } + + protected void runIndividualComparison(String tableBasePath) { + runIndividualComparison(tableBasePath, ""); + } + + protected void runIndividualComparison(String tableBasePath, String firstColumn, String... columns) { + Dataset<Row> legacyDf = sparkSession.read().format("hudi") + .option(DataSourceReadOptions.USE_LEGACY_HUDI_PARQUET_FILE_FORMAT().key(),"true").load(tableBasePath); + Dataset<Row> fileFormatDf = sparkSession.read().format("hudi") + .option(DataSourceReadOptions.USE_LEGACY_HUDI_PARQUET_FILE_FORMAT().key(),"false").load(tableBasePath); + if (firstColumn.isEmpty()) { + legacyDf = legacyDf.drop("city_to_state"); + fileFormatDf = fileFormatDf.drop("city_to_state"); Review Comment: Why dropping the `city_to_state` column here? -- 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]
