You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by vi...@apache.org on 2020/09/19 09:55:37 UTC

[hudi] branch master updated: [HUDI-995] Migrate HoodieTestUtils APIs to HoodieTestTable (#2094)

This is an automated email from the ASF dual-hosted git repository.

vinoyang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/master by this push:
     new 7c45894  [HUDI-995] Migrate HoodieTestUtils APIs to HoodieTestTable (#2094)
7c45894 is described below

commit 7c45894f432387463b05c096bfe15d196e596ef5
Author: Raymond Xu <27...@users.noreply.github.com>
AuthorDate: Sat Sep 19 02:55:24 2020 -0700

    [HUDI-995] Migrate HoodieTestUtils APIs to HoodieTestTable (#2094)
    
    Migrate deprecated APIs in HoodieTestUtils to HoodieTestTable for test classes
    - TestClientRollback
    - TestCopyOnWriteRollbackActionExecutor
    
    Use FileCreateUtils APIs in CompactionTestUtils.
    
    Then remove unused deprecated APIs after migration.
---
 .../org/apache/hudi/client/TestClientRollback.java | 216 +++++++++++----------
 .../TestCopyOnWriteRollbackActionExecutor.java     |  97 ++++-----
 .../hudi/common/testutils/CompactionTestUtils.java |  52 ++---
 .../common/testutils/HoodieTestDataGenerator.java  |  10 +-
 .../hudi/common/testutils/HoodieTestTable.java     |  63 ++++--
 .../hudi/common/testutils/HoodieTestUtils.java     |  30 ---
 .../hudi/common/util/TestCompactionUtils.java      |  10 +-
 7 files changed, 243 insertions(+), 235 deletions(-)

diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestClientRollback.java b/hudi-client/src/test/java/org/apache/hudi/client/TestClientRollback.java
index 943a1ea..da4c002 100644
--- a/hudi-client/src/test/java/org/apache/hudi/client/TestClientRollback.java
+++ b/hudi-client/src/test/java/org/apache/hudi/client/TestClientRollback.java
@@ -26,7 +26,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
 import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
-import org.apache.hudi.common.testutils.HoodieTestUtils;
+import org.apache.hudi.common.testutils.HoodieTestTable;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
@@ -38,8 +38,9 @@ import org.apache.hudi.testutils.HoodieClientTestBase;
 import org.apache.spark.api.java.JavaRDD;
 import org.junit.jupiter.api.Test;
 
-import java.io.File;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.stream.Collectors;
 
 import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
@@ -161,37 +162,46 @@ public class TestClientRollback extends HoodieClientTestBase {
   @Test
   public void testRollbackCommit() throws Exception {
     // Let's create some commit files and parquet files
-    String commitTime1 = "20160501010101";
-    String commitTime2 = "20160502020601";
-    String commitTime3 = "20160506030611";
-    new File(basePath + "/.hoodie").mkdirs();
-    HoodieTestDataGenerator.writePartitionMetadata(fs, new String[] {"2016/05/01", "2016/05/02", "2016/05/06"},
-        basePath);
-
-    // Only first two have commit files
-    HoodieTestUtils.createCommitFiles(basePath, commitTime1, commitTime2);
-    // Third one has a .inflight intermediate commit file
-    HoodieTestUtils.createInflightCommitFiles(basePath, commitTime3);
-
-    // Make commit1
-    String file11 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime1, "id11");
-    String file12 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime1, "id12");
-    String file13 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime1, "id13");
-
-    // Make commit2
-    String file21 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime2, "id21");
-    String file22 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime2, "id22");
-    String file23 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime2, "id23");
-
-    // Make commit3
-    String file31 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime3, "id31");
-    String file32 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime3, "id32");
-    String file33 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime3, "id33");
+    final String p1 = "2016/05/01";
+    final String p2 = "2016/05/02";
+    final String p3 = "2016/05/06";
+    final String commitTime1 = "20160501010101";
+    final String commitTime2 = "20160502020601";
+    final String commitTime3 = "20160506030611";
+    Map<String, String> partitionAndFileId1 = new HashMap<String, String>() {
+      {
+        put(p1, "id11");
+        put(p2, "id12");
+        put(p3, "id13");
+      }
+    };
+    Map<String, String> partitionAndFileId2 = new HashMap<String, String>() {
+      {
+        put(p1, "id21");
+        put(p2, "id22");
+        put(p3, "id23");
+      }
+    };
+    Map<String, String> partitionAndFileId3 = new HashMap<String, String>() {
+      {
+        put(p1, "id31");
+        put(p2, "id32");
+        put(p3, "id33");
+      }
+    };
+    HoodieTestTable testTable = HoodieTestTable.of(metaClient)
+        .withPartitionMetaFiles(p1, p2, p3)
+        .addCommit(commitTime1)
+        .withBaseFilesInPartitions(partitionAndFileId1)
+        .addCommit(commitTime2)
+        .withBaseFilesInPartitions(partitionAndFileId2)
+        .addInflightCommit(commitTime3)
+        .withBaseFilesInPartitions(partitionAndFileId3);
 
     HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
         .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
 
-    try (HoodieWriteClient client = getHoodieWriteClient(config, false);) {
+    try (HoodieWriteClient client = getHoodieWriteClient(config, false)) {
 
       // Rollback commit 1 (this should fail, since commit2 is still around)
       assertThrows(HoodieRollbackException.class, () -> {
@@ -200,45 +210,40 @@ public class TestClientRollback extends HoodieClientTestBase {
 
       // Rollback commit3
       client.rollback(commitTime3);
-      assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
-      assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31)
-          || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32)
-          || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33));
+      assertFalse(testTable.inflightCommitExists(commitTime3));
+      assertFalse(testTable.baseFilesExist(partitionAndFileId3, commitTime3));
+      assertTrue(testTable.baseFilesExist(partitionAndFileId2, commitTime2));
+      assertTrue(testTable.baseFilesExist(partitionAndFileId1, commitTime1));
 
       // simulate partial failure, where .inflight was not deleted, but data files were.
-      HoodieTestUtils.createInflightCommitFiles(basePath, commitTime3);
+      testTable.addInflightCommit(commitTime3);
       client.rollback(commitTime3);
-      assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
+      assertFalse(testTable.inflightCommitExists(commitTime3));
+      assertTrue(testTable.baseFilesExist(partitionAndFileId2, commitTime2));
+      assertTrue(testTable.baseFilesExist(partitionAndFileId1, commitTime1));
 
       // Rollback commit2
       client.rollback(commitTime2);
-      assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime2));
-      assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
-      assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21)
-          || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22)
-          || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23));
+      assertFalse(testTable.commitExists(commitTime2));
+      assertFalse(testTable.inflightCommitExists(commitTime2));
+      assertFalse(testTable.baseFilesExist(partitionAndFileId2, commitTime2));
+      assertTrue(testTable.baseFilesExist(partitionAndFileId1, commitTime1));
 
       // simulate partial failure, where only .commit => .inflight renaming succeeded, leaving a
       // .inflight commit and a bunch of data files around.
-      HoodieTestUtils.createInflightCommitFiles(basePath, commitTime2);
-      file21 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime2, "id21");
-      file22 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime2, "id22");
-      file23 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime2, "id23");
+      testTable.addInflightCommit(commitTime2).withBaseFilesInPartitions(partitionAndFileId2);
 
       client.rollback(commitTime2);
-      assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime2));
-      assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
-      assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21)
-          || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22)
-          || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23));
+      assertFalse(testTable.commitExists(commitTime2));
+      assertFalse(testTable.inflightCommitExists(commitTime2));
+      assertFalse(testTable.baseFilesExist(partitionAndFileId2, commitTime2));
+      assertTrue(testTable.baseFilesExist(partitionAndFileId1, commitTime1));
 
       // Let's rollback commit1, Check results
       client.rollback(commitTime1);
-      assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
-      assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime1));
-      assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11)
-          || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12)
-          || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13));
+      assertFalse(testTable.commitExists(commitTime1));
+      assertFalse(testTable.inflightCommitExists(commitTime1));
+      assertFalse(testTable.baseFilesExist(partitionAndFileId1, commitTime1));
     }
   }
 
@@ -248,71 +253,68 @@ public class TestClientRollback extends HoodieClientTestBase {
   @Test
   public void testAutoRollbackInflightCommit() throws Exception {
     // Let's create some commit files and parquet files
-    String commitTime1 = "20160501010101";
-    String commitTime2 = "20160502020601";
-    String commitTime3 = "20160506030611";
-    String commitTime4 = "20160506030621";
-    String commitTime5 = "20160506030631";
-    new File(basePath + "/.hoodie").mkdirs();
-    HoodieTestDataGenerator.writePartitionMetadata(fs, new String[] {"2016/05/01", "2016/05/02", "2016/05/06"},
-        basePath);
-
-    // One good commit
-    HoodieTestUtils.createCommitFiles(basePath, commitTime1);
-    // Two inflight commits
-    HoodieTestUtils.createInflightCommitFiles(basePath, commitTime2, commitTime3);
-
-    // Make commit1
-    String file11 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime1, "id11");
-    String file12 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime1, "id12");
-    String file13 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime1, "id13");
-
-    // Make commit2
-    String file21 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime2, "id21");
-    String file22 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime2, "id22");
-    String file23 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime2, "id23");
-
-    // Make commit3
-    String file31 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime3, "id31");
-    String file32 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime3, "id32");
-    String file33 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime3, "id33");
+    final String p1 = "2016/05/01";
+    final String p2 = "2016/05/02";
+    final String p3 = "2016/05/06";
+    final String commitTime1 = "20160501010101";
+    final String commitTime2 = "20160502020601";
+    final String commitTime3 = "20160506030611";
+    Map<String, String> partitionAndFileId1 = new HashMap<String, String>() {
+      {
+        put(p1, "id11");
+        put(p2, "id12");
+        put(p3, "id13");
+      }
+    };
+    Map<String, String> partitionAndFileId2 = new HashMap<String, String>() {
+      {
+        put(p1, "id21");
+        put(p2, "id22");
+        put(p3, "id23");
+      }
+    };
+    Map<String, String> partitionAndFileId3 = new HashMap<String, String>() {
+      {
+        put(p1, "id31");
+        put(p2, "id32");
+        put(p3, "id33");
+      }
+    };
+    HoodieTestTable testTable = HoodieTestTable.of(metaClient)
+        .withPartitionMetaFiles(p1, p2, p3)
+        .addCommit(commitTime1)
+        .withBaseFilesInPartitions(partitionAndFileId1)
+        .addInflightCommit(commitTime2)
+        .withBaseFilesInPartitions(partitionAndFileId2)
+        .addInflightCommit(commitTime3)
+        .withBaseFilesInPartitions(partitionAndFileId3);
 
     // Turn auto rollback off
     HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
         .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
 
-    try (HoodieWriteClient client = getHoodieWriteClient(config, false);) {
+    final String commitTime4 = "20160506030621";
+    try (HoodieWriteClient client = getHoodieWriteClient(config, false)) {
       client.startCommitWithTime(commitTime4);
       // Check results, nothing changed
-      assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
-      assertTrue(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
-      assertTrue(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
-      assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31)
-          && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32)
-          && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33));
-      assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21)
-          && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22)
-          && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23));
-      assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11)
-          && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12)
-          && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13));
+      assertTrue(testTable.commitExists(commitTime1));
+      assertTrue(testTable.inflightCommitExists(commitTime2));
+      assertTrue(testTable.inflightCommitExists(commitTime3));
+      assertTrue(testTable.baseFilesExist(partitionAndFileId1, commitTime1));
+      assertTrue(testTable.baseFilesExist(partitionAndFileId2, commitTime2));
+      assertTrue(testTable.baseFilesExist(partitionAndFileId3, commitTime3));
     }
 
     // Turn auto rollback on
+    final String commitTime5 = "20160506030631";
     try (HoodieWriteClient client = getHoodieWriteClient(config, true)) {
       client.startCommitWithTime(commitTime5);
-      assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
-      assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
-      assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
-      assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31)
-          || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32)
-          || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33));
-      assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21)
-          || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22)
-          || HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23));
-      assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11)
-          && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12)
-          && HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13));
+      assertTrue(testTable.commitExists(commitTime1));
+      assertFalse(testTable.inflightCommitExists(commitTime2));
+      assertFalse(testTable.inflightCommitExists(commitTime3));
+      assertTrue(testTable.baseFilesExist(partitionAndFileId1, commitTime1));
+      assertFalse(testTable.baseFilesExist(partitionAndFileId2, commitTime2));
+      assertFalse(testTable.baseFilesExist(partitionAndFileId3, commitTime3));
     }
   }
 }
diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java b/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java
index 26a85e1..a5bf032 100644
--- a/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java
+++ b/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java
@@ -24,9 +24,7 @@ import org.apache.hudi.common.model.FileSlice;
 import org.apache.hudi.common.model.HoodieFileGroup;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.table.timeline.HoodieTimeline;
-import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
-import org.apache.hudi.common.testutils.HoodieTestUtils;
-import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.testutils.HoodieTestTable;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.MarkerFiles;
@@ -37,7 +35,6 @@ import org.junit.jupiter.api.Test;
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.ValueSource;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -50,6 +47,7 @@ import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_S
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
 public class TestCopyOnWriteRollbackActionExecutor extends HoodieClientRollbackTestBase {
   @BeforeEach
@@ -66,24 +64,21 @@ public class TestCopyOnWriteRollbackActionExecutor extends HoodieClientRollbackT
   }
 
   @Test
-  public void testCopyOnWriteRollbackActionExecutorForFileListingAsGenerateFile() throws IOException {
+  public void testCopyOnWriteRollbackActionExecutorForFileListingAsGenerateFile() throws Exception {
+    final String p1 = "2015/03/16";
+    final String p2 = "2015/03/17";
+    final String p3 = "2016/03/15";
     // Let's create some commit files and parquet files
-    String commitTime1 = "001";
-    String commitTime2 = "002";
-    new File(basePath + "/.hoodie").mkdirs();
-    HoodieTestDataGenerator.writePartitionMetadata(fs, new String[]{"2015/03/16", "2015/03/17", "2016/03/15"},
-        basePath);
-    HoodieTestUtils.createCommitFiles(basePath, commitTime1, commitTime2);
-
-    // Make commit1
-    String file11 = HoodieTestUtils.createDataFile(basePath, "2015/03/16", commitTime1, "id11");
-    HoodieTestUtils.createNewLogFile(fs, basePath, "2015/03/16",
-        commitTime1, "id11", Option.of(3));
-    String file12 = HoodieTestUtils.createDataFile(basePath, "2015/03/17", commitTime1, "id12");
-
-    // Make commit2
-    String file21 = HoodieTestUtils.createDataFile(basePath, "2015/03/16", commitTime2, "id21");
-    String file22 = HoodieTestUtils.createDataFile(basePath, "2015/03/17", commitTime2, "id22");
+    HoodieTestTable testTable = HoodieTestTable.of(metaClient)
+        .withPartitionMetaFiles(p1, p2, p3)
+        .addCommit("001")
+        .withBaseFilesInPartition(p1, "id11")
+        .withBaseFilesInPartition(p2, "id12")
+        .withLogFile(p1, "id11", 3)
+        .addCommit("002")
+        .withBaseFilesInPartition(p1, "id21")
+        .withBaseFilesInPartition(p2, "id22");
+
     HoodieTable table = this.getHoodieTable(metaClient, getConfig());
     HoodieInstant needRollBackInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "002");
 
@@ -94,34 +89,40 @@ public class TestCopyOnWriteRollbackActionExecutor extends HoodieClientRollbackT
 
     // assert hoodieRollbackStats
     assertEquals(hoodieRollbackStats.size(), 3);
-    hoodieRollbackStats.forEach(stat -> {
-      if (stat.getPartitionPath().equals("2015/03/16")) {
-        assertEquals(1, stat.getSuccessDeleteFiles().size());
-        assertEquals(0, stat.getFailedDeleteFiles().size());
-        assertEquals(Collections.EMPTY_MAP, stat.getCommandBlocksCount());
-        assertEquals("file:" + HoodieTestUtils.getDataFilePath(basePath, "2015/03/16", commitTime2, file21),
-            stat.getSuccessDeleteFiles().get(0));
-      } else if (stat.getPartitionPath().equals("2015/03/17")) {
-        assertEquals(1, stat.getSuccessDeleteFiles().size());
-        assertEquals(0, stat.getFailedDeleteFiles().size());
-        assertEquals(Collections.EMPTY_MAP, stat.getCommandBlocksCount());
-        assertEquals("file:" + HoodieTestUtils.getDataFilePath(basePath, "2015/03/17", commitTime2, file22),
-            stat.getSuccessDeleteFiles().get(0));
-      } else if (stat.getPartitionPath().equals("2016/03/15")) {
-        assertEquals(0, stat.getSuccessDeleteFiles().size());
-        assertEquals(0, stat.getFailedDeleteFiles().size());
-        assertEquals(Collections.EMPTY_MAP, stat.getCommandBlocksCount());
+    for (HoodieRollbackStat stat : hoodieRollbackStats) {
+      switch (stat.getPartitionPath()) {
+        case p1:
+          assertEquals(1, stat.getSuccessDeleteFiles().size());
+          assertEquals(0, stat.getFailedDeleteFiles().size());
+          assertEquals(Collections.EMPTY_MAP, stat.getCommandBlocksCount());
+          assertEquals(testTable.forCommit("002").getBaseFilePath(p1, "id21").toString(),
+              stat.getSuccessDeleteFiles().get(0));
+          break;
+        case p2:
+          assertEquals(1, stat.getSuccessDeleteFiles().size());
+          assertEquals(0, stat.getFailedDeleteFiles().size());
+          assertEquals(Collections.EMPTY_MAP, stat.getCommandBlocksCount());
+          assertEquals(testTable.forCommit("002").getBaseFilePath(p2, "id22").toString(),
+              stat.getSuccessDeleteFiles().get(0));
+          break;
+        case p3:
+          assertEquals(0, stat.getSuccessDeleteFiles().size());
+          assertEquals(0, stat.getFailedDeleteFiles().size());
+          assertEquals(Collections.EMPTY_MAP, stat.getCommandBlocksCount());
+          break;
+        default:
+          fail("Unexpected partition: " + stat.getPartitionPath());
       }
-    });
-
-    assertTrue(HoodieTestUtils.doesCommitExist(basePath, "001"));
-    assertTrue(HoodieTestUtils.doesInflightExist(basePath, "001"));
-    assertFalse(HoodieTestUtils.doesCommitExist(basePath, "002"));
-    assertFalse(HoodieTestUtils.doesInflightExist(basePath, "002"));
-    assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2015/03/16", commitTime1, file11)
-        && HoodieTestUtils.doesDataFileExist(basePath, "2015/03/17", commitTime1, file12));
-    assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2015/03/16", commitTime2, file21)
-        || HoodieTestUtils.doesDataFileExist(basePath, "2015/03/17", commitTime2, file22));
+    }
+
+    assertTrue(testTable.inflightCommitExists("001"));
+    assertTrue(testTable.commitExists("001"));
+    assertTrue(testTable.baseFileExists(p1, "001", "id11"));
+    assertTrue(testTable.baseFileExists(p2, "001", "id12"));
+    assertFalse(testTable.inflightCommitExists("002"));
+    assertFalse(testTable.commitExists("002"));
+    assertFalse(testTable.baseFileExists(p1, "002", "id21"));
+    assertFalse(testTable.baseFileExists(p2, "002", "id22"));
   }
 
   @ParameterizedTest
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/CompactionTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/CompactionTestUtils.java
index 7d9110b..44e3da0 100644
--- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/CompactionTestUtils.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/CompactionTestUtils.java
@@ -20,7 +20,6 @@ package org.apache.hudi.common.testutils;
 
 import org.apache.hudi.avro.model.HoodieCompactionOperation;
 import org.apache.hudi.avro.model.HoodieCompactionPlan;
-import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.FileSlice;
 import org.apache.hudi.common.model.HoodieBaseFile;
 import org.apache.hudi.common.model.HoodieFileGroupId;
@@ -33,11 +32,12 @@ import org.apache.hudi.common.util.CollectionUtils;
 import org.apache.hudi.common.util.CompactionUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.exception.HoodieException;
 
 import org.apache.hadoop.fs.Path;
 
 import java.io.IOException;
+import java.nio.file.Paths;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
@@ -49,6 +49,10 @@ import java.util.stream.Stream;
 
 import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION;
 import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION;
+import static org.apache.hudi.common.testutils.FileCreateUtils.baseFileName;
+import static org.apache.hudi.common.testutils.FileCreateUtils.createBaseFile;
+import static org.apache.hudi.common.testutils.FileCreateUtils.createLogFile;
+import static org.apache.hudi.common.testutils.FileCreateUtils.logFileName;
 import static org.apache.hudi.common.testutils.HoodieTestUtils.DEFAULT_PARTITION_PATHS;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNull;
@@ -152,40 +156,36 @@ public class CompactionTestUtils {
         .transitionCompactionRequestedToInflight(new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime));
   }
 
-  public static HoodieCompactionPlan createCompactionPlan(HoodieTableMetaClient metaClient, String instantId,
-      String compactionInstantId, int numFileIds, boolean createDataFile, boolean deltaCommitsAfterCompactionRequests) {
+  public static HoodieCompactionPlan createCompactionPlan(HoodieTableMetaClient metaClient, String instantTime,
+      String compactionInstantTime, int numFileIds, boolean createDataFile, boolean deltaCommitsAfterCompactionRequests) {
     List<HoodieCompactionOperation> ops = IntStream.range(0, numFileIds).boxed().map(idx -> {
       try {
-        String fileId = UUID.randomUUID().toString();
+        final String basePath = metaClient.getBasePath();
+        final String partition = DEFAULT_PARTITION_PATHS[0];
+        final String fileId = UUID.randomUUID().toString();
         if (createDataFile) {
-          HoodieTestUtils.createDataFile(metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0], instantId, fileId);
+          createBaseFile(basePath, partition, instantTime, fileId);
         }
-        HoodieTestUtils.createNewLogFile(metaClient.getFs(), metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0],
-            instantId, fileId, Option.of(1));
-        HoodieTestUtils.createNewLogFile(metaClient.getFs(), metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0],
-            instantId, fileId, Option.of(2));
-        FileSlice slice = new FileSlice(DEFAULT_PARTITION_PATHS[0], instantId, fileId);
+        createLogFile(basePath, partition, instantTime, fileId, 1);
+        createLogFile(basePath, partition, instantTime, fileId, 2);
+        FileSlice slice = new FileSlice(partition, instantTime, fileId);
         if (createDataFile) {
-          slice.setBaseFile(new TestHoodieBaseFile(metaClient.getBasePath() + "/" + DEFAULT_PARTITION_PATHS[0] + "/"
-              + FSUtils.makeDataFileName(instantId, TEST_WRITE_TOKEN, fileId)));
+          slice.setBaseFile(new DummyHoodieBaseFile(Paths.get(basePath, partition,
+              baseFileName(instantTime, fileId)).toString()));
         }
-        String logFilePath1 = HoodieTestUtils.getLogFilePath(metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0],
-            instantId, fileId, Option.of(1));
-        String logFilePath2 = HoodieTestUtils.getLogFilePath(metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0],
-            instantId, fileId, Option.of(2));
+        String logFilePath1 = Paths.get(basePath, partition, logFileName(instantTime, fileId, 1)).toString();
+        String logFilePath2 = Paths.get(basePath, partition, logFileName(instantTime, fileId, 2)).toString();
         slice.addLogFile(new HoodieLogFile(new Path(logFilePath1)));
         slice.addLogFile(new HoodieLogFile(new Path(logFilePath2)));
         HoodieCompactionOperation op =
-            CompactionUtils.buildFromFileSlice(DEFAULT_PARTITION_PATHS[0], slice, Option.empty());
+            CompactionUtils.buildFromFileSlice(partition, slice, Option.empty());
         if (deltaCommitsAfterCompactionRequests) {
-          HoodieTestUtils.createNewLogFile(metaClient.getFs(), metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0],
-              compactionInstantId, fileId, Option.of(1));
-          HoodieTestUtils.createNewLogFile(metaClient.getFs(), metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0],
-              compactionInstantId, fileId, Option.of(2));
+          createLogFile(basePath, partition, compactionInstantTime, fileId, 1);
+          createLogFile(basePath, partition, compactionInstantTime, fileId, 2);
         }
         return op;
-      } catch (IOException e) {
-        throw new HoodieIOException(e.getMessage(), e);
+      } catch (Exception e) {
+        throw new HoodieException(e.getMessage(), e);
       }
     }).collect(Collectors.toList());
     return new HoodieCompactionPlan(ops.isEmpty() ? null : ops, new HashMap<>(),
@@ -195,11 +195,11 @@ public class CompactionTestUtils {
   /**
    * The hoodie data file for testing.
    */
-  public static class TestHoodieBaseFile extends HoodieBaseFile {
+  public static class DummyHoodieBaseFile extends HoodieBaseFile {
 
     private final String path;
 
-    public TestHoodieBaseFile(String path) {
+    public DummyHoodieBaseFile(String path) {
       super(path);
       this.path = path;
     }
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java
index 68b66b0..34d5926 100644
--- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java
@@ -19,9 +19,6 @@
 
 package org.apache.hudi.common.testutils;
 
-import org.apache.avro.Conversions;
-import org.apache.avro.LogicalTypes;
-import org.apache.avro.generic.GenericFixed;
 import org.apache.hudi.avro.HoodieAvroUtils;
 import org.apache.hudi.avro.model.HoodieCompactionPlan;
 import org.apache.hudi.common.fs.FSUtils;
@@ -37,9 +34,12 @@ import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.exception.HoodieIOException;
 
+import org.apache.avro.Conversions;
+import org.apache.avro.LogicalTypes;
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericArray;
 import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericFixed;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -157,6 +157,10 @@ public class HoodieTestDataGenerator {
     numKeysBySchema = new HashMap<>();
   }
 
+  /**
+   * @implNote {@link HoodieTestDataGenerator} is supposed to just generate records with schemas. Leave HoodieTable files (metafile, basefile, logfile, etc) to {@link HoodieTestTable}.
+   * @deprecated Use {@link HoodieTestTable#withPartitionMetaFiles(java.lang.String...)} instead.
+   */
   public static void writePartitionMetadata(FileSystem fs, String[] partitionPaths, String basePath) {
     for (String partitionPath : partitionPaths) {
       new HoodiePartitionMetadata(fs, "000", new Path(basePath), new Path(basePath, partitionPath)).trySave(0);
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java
index b80b732..02ed8e0 100644
--- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java
@@ -218,6 +218,13 @@ public class HoodieTestTable {
     return partitionFileIdMap;
   }
 
+  public HoodieTestTable withBaseFilesInPartitions(Map<String, String> partitionAndFileId) throws Exception {
+    for (Map.Entry<String, String> pair : partitionAndFileId.entrySet()) {
+      withBaseFilesInPartition(pair.getKey(), pair.getValue());
+    }
+    return this;
+  }
+
   public HoodieTestTable withBaseFilesInPartition(String partition, String... fileIds) throws Exception {
     for (String f : fileIds) {
       FileCreateUtils.createBaseFile(basePath, partition, currentInstantTime, f);
@@ -248,6 +255,30 @@ public class HoodieTestTable {
     return this;
   }
 
+  public boolean inflightCommitsExist(String... instantTime) {
+    return Arrays.stream(instantTime).allMatch(this::inflightCommitExists);
+  }
+
+  public boolean inflightCommitExists(String instantTime) {
+    try {
+      return fs.exists(getInflightCommitFilePath(instantTime));
+    } catch (IOException e) {
+      throw new HoodieTestTableException(e);
+    }
+  }
+
+  public boolean commitsExist(String... instantTime) {
+    return Arrays.stream(instantTime).allMatch(this::commitExists);
+  }
+
+  public boolean commitExists(String instantTime) {
+    try {
+      return fs.exists(getCommitFilePath(instantTime));
+    } catch (IOException e) {
+      throw new HoodieTestTableException(e);
+    }
+  }
+
   public boolean baseFilesExist(Map<String, String> partitionAndFileId, String instantTime) {
     return partitionAndFileId.entrySet().stream().allMatch(entry -> {
       String partition = entry.getKey();
@@ -268,16 +299,16 @@ public class HoodieTestTable {
     }
   }
 
-  public Path getPartitionPath(String partition) {
-    return new Path(Paths.get(basePath, partition).toUri());
-  }
-
-  public String getBaseFileNameById(String fileId) {
-    return baseFileName(currentInstantTime, fileId);
+  public boolean logFilesExist(String partition, String instantTime, String fileId, int... versions) {
+    return Arrays.stream(versions).allMatch(v -> logFileExists(partition, instantTime, fileId, v));
   }
 
-  public Path getBaseFilePath(String partition, String fileId) {
-    return new Path(Paths.get(basePath, partition, getBaseFileNameById(fileId)).toUri());
+  public boolean logFileExists(String partition, String instantTime, String fileId, int version) {
+    try {
+      return fs.exists(new Path(Paths.get(basePath, partition, logFileName(instantTime, fileId, version)).toString()));
+    } catch (IOException e) {
+      throw new HoodieTestTableException(e);
+    }
   }
 
   public Path getInflightCommitFilePath(String instantTime) {
@@ -292,16 +323,16 @@ public class HoodieTestTable {
     return new Path(Paths.get(basePath, HoodieTableMetaClient.AUXILIARYFOLDER_NAME, instantTime + HoodieTimeline.REQUESTED_COMPACTION_EXTENSION).toUri());
   }
 
-  public boolean logFilesExist(String partition, String instantTime, String fileId, int... versions) {
-    return Arrays.stream(versions).allMatch(v -> logFileExists(partition, instantTime, fileId, v));
+  public Path getPartitionPath(String partition) {
+    return new Path(Paths.get(basePath, partition).toUri());
   }
 
-  public boolean logFileExists(String partition, String instantTime, String fileId, int version) {
-    try {
-      return fs.exists(new Path(Paths.get(basePath, partition, logFileName(instantTime, fileId, version)).toString()));
-    } catch (IOException e) {
-      throw new HoodieTestTableException(e);
-    }
+  public Path getBaseFilePath(String partition, String fileId) {
+    return new Path(Paths.get(basePath, partition, getBaseFileNameById(fileId)).toUri());
+  }
+
+  public String getBaseFileNameById(String fileId) {
+    return baseFileName(currentInstantTime, fileId);
   }
 
   public List<FileStatus> listAllFiles(String partitionPath) throws IOException {
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java
index 84bfe22..7f00f5a 100644
--- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java
@@ -260,19 +260,6 @@ public class HoodieTestUtils {
   /**
    * @deprecated Use {@link HoodieTestTable} instead.
    */
-  public static String getDataFilePath(String basePath, String partitionPath, String instantTime, String fileID) {
-    return basePath + "/" + partitionPath + "/" + FSUtils.makeDataFileName(instantTime, DEFAULT_WRITE_TOKEN, fileID);
-  }
-
-  /**
-   * @deprecated Use {@link HoodieTestTable} instead.
-   */
-  public static String getLogFilePath(String basePath, String partitionPath, String instantTime, String fileID,
-      Option<Integer> version) {
-    return basePath + "/" + partitionPath + "/" + FSUtils.makeLogFileName(fileID, ".log", instantTime,
-        version.orElse(DEFAULT_LOG_VERSION), HoodieLogFormat.UNKNOWN_WRITE_TOKEN);
-  }
-
   public static String getCommitFilePath(String basePath, String instantTime) {
     return basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + instantTime + HoodieTimeline.COMMIT_EXTENSION;
   }
@@ -280,29 +267,12 @@ public class HoodieTestUtils {
   /**
    * @deprecated Use {@link HoodieTestTable} instead.
    */
-  public static boolean doesDataFileExist(String basePath, String partitionPath, String instantTime,
-      String fileID) {
-    return new File(getDataFilePath(basePath, partitionPath, instantTime, fileID)).exists();
-  }
-
-  /**
-   * @deprecated Use {@link HoodieTestTable} instead.
-   */
   public static boolean doesCommitExist(String basePath, String instantTime) {
     return new File(
         basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + instantTime + HoodieTimeline.COMMIT_EXTENSION)
             .exists();
   }
 
-  /**
-   * @deprecated Use {@link HoodieTestTable} instead.
-   */
-  public static boolean doesInflightExist(String basePath, String instantTime) {
-    return new File(
-        basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + instantTime + HoodieTimeline.INFLIGHT_EXTENSION)
-            .exists();
-  }
-
   public static void createCleanFiles(HoodieTableMetaClient metaClient, String basePath,
       String instantTime, Configuration configuration)
       throws IOException {
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestCompactionUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestCompactionUtils.java
index 5e722fd..35ff4cb 100644
--- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestCompactionUtils.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestCompactionUtils.java
@@ -28,7 +28,7 @@ import org.apache.hudi.common.model.HoodieLogFile;
 import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.timeline.versioning.compaction.CompactionPlanMigrator;
-import org.apache.hudi.common.testutils.CompactionTestUtils.TestHoodieBaseFile;
+import org.apache.hudi.common.testutils.CompactionTestUtils.DummyHoodieBaseFile;
 import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
 import org.apache.hudi.common.util.collection.Pair;
 
@@ -106,7 +106,7 @@ public class TestCompactionUtils extends HoodieCommonTestHarness {
 
     // File Slice with data-file but no log files
     FileSlice noLogFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "noLog1");
-    noLogFileSlice.setBaseFile(new TestHoodieBaseFile("/tmp/noLog_1_000.parquet"));
+    noLogFileSlice.setBaseFile(new DummyHoodieBaseFile("/tmp/noLog_1_000.parquet"));
     op = CompactionUtils.buildFromFileSlice(DEFAULT_PARTITION_PATHS[0], noLogFileSlice, Option.of(metricsCaptureFn));
     testFileSliceCompactionOpEquality(noLogFileSlice, op, DEFAULT_PARTITION_PATHS[0],
         LATEST_COMPACTION_METADATA_VERSION);
@@ -122,7 +122,7 @@ public class TestCompactionUtils extends HoodieCommonTestHarness {
 
     // File Slice with data-file and log files present
     FileSlice fileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "noData1");
-    fileSlice.setBaseFile(new TestHoodieBaseFile("/tmp/noLog_1_000.parquet"));
+    fileSlice.setBaseFile(new DummyHoodieBaseFile("/tmp/noLog_1_000.parquet"));
     fileSlice.addLogFile(
         new HoodieLogFile(new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN))));
     fileSlice.addLogFile(
@@ -138,13 +138,13 @@ public class TestCompactionUtils extends HoodieCommonTestHarness {
     Path fullPartitionPath = new Path(new Path(metaClient.getBasePath()), DEFAULT_PARTITION_PATHS[0]);
     FileSlice emptyFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "empty1");
     FileSlice fileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "noData1");
-    fileSlice.setBaseFile(new TestHoodieBaseFile(fullPartitionPath.toString() + "/data1_1_000.parquet"));
+    fileSlice.setBaseFile(new DummyHoodieBaseFile(fullPartitionPath.toString() + "/data1_1_000.parquet"));
     fileSlice.addLogFile(new HoodieLogFile(
         new Path(fullPartitionPath, new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN)))));
     fileSlice.addLogFile(new HoodieLogFile(
         new Path(fullPartitionPath, new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 2, TEST_WRITE_TOKEN)))));
     FileSlice noLogFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "noLog1");
-    noLogFileSlice.setBaseFile(new TestHoodieBaseFile(fullPartitionPath.toString() + "/noLog_1_000.parquet"));
+    noLogFileSlice.setBaseFile(new DummyHoodieBaseFile(fullPartitionPath.toString() + "/noLog_1_000.parquet"));
     FileSlice noDataFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0], "000", "noData1");
     noDataFileSlice.addLogFile(new HoodieLogFile(
         new Path(fullPartitionPath, new Path(FSUtils.makeLogFileName("noData1", ".log", "000", 1, TEST_WRITE_TOKEN)))));