You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by si...@apache.org on 2022/01/19 23:08:23 UTC

[hudi] 02/06: [HUDI-3194] fix MOR snapshot query during compaction (#4540)

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 <yw...@gmail.com>
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()))