This is an automated email from the ASF dual-hosted git repository. sivabalan pushed a commit to branch release-0.10.1 in repository https://gitbox.apache.org/repos/asf/hudi.git
commit a8ee57f3ea1f9a47d42d92ab508025aa968e26e7 Author: Yuwei XIAO <[email protected]> AuthorDate: Tue Jan 18 06:24:24 2022 +0800 [HUDI-3194] fix MOR snapshot query during compaction (#4540) --- .../TestHoodieSparkMergeOnReadTableCompaction.java | 125 +++++++++++++++++++++ .../utils/HoodieRealtimeInputFormatUtils.java | 8 +- 2 files changed, 129 insertions(+), 4 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableCompaction.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableCompaction.java new file mode 100644 index 0000000..13903bf --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableCompaction.java @@ -0,0 +1,125 @@ +/* + * 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.table.functional; + +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieLayoutConfig; +import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.table.action.commit.SparkBucketIndexPartitioner; +import org.apache.hudi.table.storage.HoodieStorageLayout; +import org.apache.hudi.testutils.HoodieMergeOnReadTestUtils; +import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; + +import org.apache.spark.api.java.JavaRDD; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; +import static org.apache.hudi.config.HoodieWriteConfig.AUTO_COMMIT_ENABLE; + +@Tag("functional") +public class TestHoodieSparkMergeOnReadTableCompaction extends SparkClientFunctionalTestHarness { + + private HoodieTestDataGenerator dataGen; + private SparkRDDWriteClient client; + private HoodieTableMetaClient metaClient; + + @BeforeEach + public void setup() { + dataGen = new HoodieTestDataGenerator(); + } + + @Test + public void testWriteDuringCompaction() throws IOException { + HoodieWriteConfig config = HoodieWriteConfig.newBuilder() + .forTable("test-trip-table") + .withPath(basePath()) + .withSchema(TRIP_EXAMPLE_SCHEMA) + .withParallelism(2, 2) + .withAutoCommit(false) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withMaxNumDeltaCommitsBeforeCompaction(1).build()) + .withStorageConfig(HoodieStorageConfig.newBuilder() + .parquetMaxFileSize(1024).build()) + .withLayoutConfig(HoodieLayoutConfig.newBuilder() + .withLayoutType(HoodieStorageLayout.LayoutType.BUCKET.name()) + .withLayoutPartitioner(SparkBucketIndexPartitioner.class.getName()).build()) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BUCKET).withBucketNum("1").build()).build(); + metaClient = getHoodieMetaClient(HoodieTableType.MERGE_ON_READ, config.getProps()); + client = getHoodieWriteClient(config); + + // write data and commit + writeData(HoodieActiveTimeline.createNewInstantTime(), 100, true); + // write data again, and in the case of bucket index, all records will go into log files (we use a small max_file_size) + writeData(HoodieActiveTimeline.createNewInstantTime(), 100, true); + Assertions.assertEquals(200, readTableTotalRecordsNum()); + // schedule compaction + String compactionTime = (String) client.scheduleCompaction(Option.empty()).get(); + // write data, and do not commit. those records should not visible to reader + String insertTime = HoodieActiveTimeline.createNewInstantTime(); + List<WriteStatus> writeStatuses = writeData(insertTime, 100, false); + Assertions.assertEquals(200, readTableTotalRecordsNum()); + // commit the write. The records should be visible now even though the compaction does not complete. + client.commitStats(insertTime, writeStatuses.stream().map(WriteStatus::getStat).collect(Collectors.toList()), Option.empty(), metaClient.getCommitActionType()); + Assertions.assertEquals(300, readTableTotalRecordsNum()); + // after the compaction, total records should remain the same + config.setValue(AUTO_COMMIT_ENABLE, "true"); + client.compact(compactionTime); + Assertions.assertEquals(300, readTableTotalRecordsNum()); + } + + private long readTableTotalRecordsNum() { + return HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf(), + Arrays.stream(dataGen.getPartitionPaths()).map(p -> Paths.get(basePath(), p).toString()).collect(Collectors.toList()), basePath()).size(); + } + + private List<WriteStatus> writeData(String instant, int numRecords, boolean doCommit) { + metaClient = HoodieTableMetaClient.reload(metaClient); + JavaRDD records = jsc().parallelize(dataGen.generateInserts(instant, numRecords), 2); + metaClient = HoodieTableMetaClient.reload(metaClient); + client.startCommitWithTime(instant); + List<WriteStatus> writeStatues = client.upsert(records, instant).collect(); + org.apache.hudi.testutils.Assertions.assertNoWriteErrors(writeStatues); + if (doCommit) { + Assertions.assertTrue(client.commitStats(instant, writeStatues.stream().map(WriteStatus::getStat).collect(Collectors.toList()), + Option.empty(), metaClient.getCommitActionType())); + } + metaClient = HoodieTableMetaClient.reload(metaClient); + return writeStatues; + } +} \ No newline at end of file diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java index 09338d3..6718642 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java @@ -69,7 +69,7 @@ public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils { private static final Logger LOG = LogManager.getLogger(HoodieRealtimeInputFormatUtils.class); - public static InputSplit[] getRealtimeSplits(Configuration conf, Stream<FileSplit> fileSplits) throws IOException { + public static InputSplit[] getRealtimeSplits(Configuration conf, Stream<FileSplit> fileSplits) { Map<Path, List<FileSplit>> partitionsToParquetSplits = fileSplits.collect(Collectors.groupingBy(split -> split.getPath().getParent())); // TODO(vc): Should we handle also non-hoodie splits here? @@ -94,8 +94,8 @@ public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils { HoodieTableMetaClient metaClient = partitionsToMetaClient.get(partitionPath); if (!fsCache.containsKey(metaClient)) { HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(conf); - HoodieTableFileSystemView fsView = FileSystemViewManager.createInMemoryFileSystemView(engineContext, - metaClient, HoodieInputFormatUtils.buildMetadataConfig(conf)); + HoodieTableFileSystemView fsView = FileSystemViewManager.createInMemoryFileSystemViewWithTimeline(engineContext, + metaClient, HoodieInputFormatUtils.buildMetadataConfig(conf), metaClient.getActiveTimeline()); fsCache.put(metaClient, fsView); } HoodieTableFileSystemView fsView = fsCache.get(metaClient); @@ -103,7 +103,7 @@ public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils { String relPartitionPath = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), partitionPath); // Both commit and delta-commits are included - pick the latest completed one Option<HoodieInstant> latestCompletedInstant = - metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant(); + metaClient.getActiveTimeline().getWriteTimeline().filterCompletedInstants().lastInstant(); Stream<FileSlice> latestFileSlices = latestCompletedInstant .map(instant -> fsView.getLatestMergedFileSlicesBeforeOrOn(relPartitionPath, instant.getTimestamp()))
