jonvex commented on code in PR #11415: URL: https://github.com/apache/hudi/pull/11415#discussion_r1633551290
########## hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPositionBasedMergingFallback.scala: ########## @@ -0,0 +1,192 @@ +/* + * 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.hadoop.fs.FileSystem +import org.apache.hudi.DataSourceWriteOptions +import org.apache.hudi.DataSourceWriteOptions.{OPERATION, PRECOMBINE_FIELD, RECORDKEY_FIELD, TABLE_TYPE} +import org.apache.hudi.HoodieConversionUtils.toJavaOption +import org.apache.hudi.common.config.{HoodieReaderConfig, HoodieStorageConfig} +import org.apache.hudi.common.model.HoodieRecordMerger +import org.apache.hudi.common.util +import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.testutils.HoodieSparkClientTestBase +import org.apache.hudi.util.JFunction +import org.apache.spark.sql.SaveMode.{Append, Overwrite} +import org.apache.spark.sql.SparkSessionExtensions +import org.apache.spark.sql.hudi.HoodieSparkSessionExtension +import org.apache.spark.sql.internal.SQLConf +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.{AfterEach, BeforeEach} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.{Arguments, MethodSource} + +import java.util.function.Consumer + +class TestPositionBasedMergingFallback extends HoodieSparkClientTestBase { + override def getSparkSessionExtensionsInjector: util.Option[Consumer[SparkSessionExtensions]] = + toJavaOption( + Some( + JFunction.toJavaConsumer((receiver: SparkSessionExtensions) => new HoodieSparkSessionExtension().apply(receiver))) + ) + + @BeforeEach override def setUp(): Unit = { + initPath() + initSparkContexts() + sparkSession.conf.set(SQLConf.PARQUET_RECORD_FILTER_ENABLED.key, "true") + initTestDataGenerator() + initHoodieStorage() + } + + @AfterEach override def tearDown(): Unit = { + cleanupSparkContexts() + cleanupTestDataGenerator() + cleanupFileSystem() + FileSystem.closeAll() + System.gc() + } + + @ParameterizedTest + @MethodSource(Array("testArgs")) + def testPositionFallback(updateWithRecordPositions: String, deleteWithRecordPositions: String, secondUpdateWithPositions: String): Unit = { + val columns = Seq("ts", "key", "name", "_hoodie_is_deleted") + val data = Seq( + (10, "1", "A", false), + (10, "2", "B", false), + (10, "3", "C", false), + (10, "4", "D", false), + (10, "5", "E", false)) + + val inserts = sparkSession.createDataFrame(data).toDF(columns: _*) + inserts.write.format("hudi"). + option(RECORDKEY_FIELD.key(), "key"). + option(PRECOMBINE_FIELD.key(), "ts"). + option("hoodie.table.name", "test_table"). + option(TABLE_TYPE.key(), "MERGE_ON_READ"). + option(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key(), "parquet"). + option(DataSourceWriteOptions.RECORD_MERGER_STRATEGY.key(), HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID). + option(DataSourceWriteOptions.RECORD_MERGER_IMPLS.key(), "org.apache.hudi.HoodieSparkRecordMerger"). + option(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), "true"). + option(HoodieWriteConfig.WRITE_RECORD_POSITIONS.key(), "true"). + mode(Overwrite). + save(basePath) + + val updateData = Seq((11, "1", "A_1", false), (9, "2", "B_1", false)) + + val updates = sparkSession.createDataFrame(updateData).toDF(columns: _*) + + updates.write.format("hudi"). + option(RECORDKEY_FIELD.key(), "key"). + option(PRECOMBINE_FIELD.key(), "ts"). + option("hoodie.table.name", "test_table"). + option(TABLE_TYPE.key(), "MERGE_ON_READ"). + option(OPERATION.key(), "upsert"). + option(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key(), "parquet"). + option(DataSourceWriteOptions.RECORD_MERGER_STRATEGY.key(), HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID). + option(DataSourceWriteOptions.RECORD_MERGER_IMPLS.key(), "org.apache.hudi.HoodieSparkRecordMerger"). + option(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), "true"). + option(HoodieWriteConfig.WRITE_RECORD_POSITIONS.key(), updateWithRecordPositions). + mode(Append). + save(basePath) + + val deletesData = Seq((10, "4", "D", true), (10, "3", "C", true)) + + val deletes = sparkSession.createDataFrame(deletesData).toDF(columns: _*) + deletes.write.format("hudi"). + option(RECORDKEY_FIELD.key(), "key"). + option(PRECOMBINE_FIELD.key(), "ts"). + option("hoodie.table.name", "test_table"). + option(TABLE_TYPE.key(), "MERGE_ON_READ"). + option(OPERATION.key(), "upsert"). + option(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key(), "parquet"). + option(DataSourceWriteOptions.RECORD_MERGER_STRATEGY.key(), HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID). + option(DataSourceWriteOptions.RECORD_MERGER_IMPLS.key(), "org.apache.hudi.HoodieSparkRecordMerger"). + option(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), "true"). + option(HoodieWriteConfig.WRITE_RECORD_POSITIONS.key(), deleteWithRecordPositions). + mode(Append). + save(basePath) + + + val secondUpdateData = Seq((14, "5", "E_3", false), (3, "3", "C_3", false)) + val secondUpdates = sparkSession.createDataFrame(secondUpdateData).toDF(columns: _*) + secondUpdates.write.format("hudi"). + option(RECORDKEY_FIELD.key(), "key"). + option(PRECOMBINE_FIELD.key(), "ts"). + option("hoodie.table.name", "test_table"). + option(TABLE_TYPE.key(), "MERGE_ON_READ"). + option(OPERATION.key(), "upsert"). + option(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key(), "parquet"). + option(DataSourceWriteOptions.RECORD_MERGER_STRATEGY.key(), HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID). + option(DataSourceWriteOptions.RECORD_MERGER_IMPLS.key(), "org.apache.hudi.HoodieSparkRecordMerger"). + option(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), "true"). + option(HoodieWriteConfig.WRITE_RECORD_POSITIONS.key(), secondUpdateWithPositions). + mode(Append). + save(basePath) + + val df = sparkSession.read.format("hudi"). + option(DataSourceWriteOptions.RECORD_MERGER_STRATEGY.key(), HoodieRecordMerger.DEFAULT_MERGER_STRATEGY_UUID). + option(DataSourceWriteOptions.RECORD_MERGER_IMPLS.key(), "org.apache.hudi.HoodieSparkRecordMerger"). + option(HoodieReaderConfig.FILE_GROUP_READER_ENABLED.key(), "true"). + option(HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS.key(), "true").load(basePath) + val finalDf = df.select("ts", "key", "name") + val finalColumns = Seq("ts", "key", "name") + + val finalExpectedData = Seq( + (11, "1", "A_1"), + (10, "2", "B"), + (14, "5", "E_3")) + + val expectedDf = sparkSession.createDataFrame(finalExpectedData).toDF(finalColumns: _*) + + assertEquals(0, finalDf.except(expectedDf).count()) + assertEquals(0, expectedDf.except(finalDf).count()) Review Comment: needToDoHybridStrategy is needed when we just read the positions from the delete block using the roaring bitmap. The bitmap only stores the positions, so even if the recordkey is not null in the record, we still don't have that information. -- 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]
