You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2020/01/27 23:41:43 UTC

[GitHub] [incubator-hudi] prashantwason opened a new pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

prashantwason opened a new pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287
 
 
   ## What is the purpose of the pull request
   
   Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
   
   ## Verify this pull request
   
   This change added tests and can be verified as follows:
   
   mvn test --also-make -DfailIfNoTests=false -Dtest=TestHoodieActiveTimeline -pl hudi-common
   
   ## Committer checklist
   
    - [ ] Has a corresponding JIRA in PR title & commit
    
    - [ ] Commit message is descriptive of the change
    
    - [ ] CI is green
   
    - [ ] Necessary doc changes done or have another open PR
          
    - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r372563070
 
 

 ##########
 File path: hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java
 ##########
 @@ -164,4 +173,242 @@ public void testTimelineOperations() {
     assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
     assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
   }
+
+  @Test
+  public void testTimelineGetOperations() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    BiConsumer<HoodieTimeline, Set<String>> checkTimeline = (HoodieTimeline timeline, Set<String> actions) -> {
+      sup.get().filter(i -> actions.contains(i.getAction())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !actions.contains(i.getAction())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    // Test that various types of getXXX operations from HoodieActiveTimeline
+    // return the correct set of Instant
+    checkTimeline.accept(timeline.getCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCommitsAndCompactionTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION));
+    checkTimeline.accept(timeline.getCommitTimeline(), Sets.newHashSet(HoodieTimeline.COMMIT_ACTION));
+
+    checkTimeline.accept(timeline.getDeltaCommitTimeline(), Sets.newHashSet(HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCleanerTimeline(), Sets.newHashSet(HoodieTimeline.CLEAN_ACTION));
+    checkTimeline.accept(timeline.getRollbackTimeline(), Sets.newHashSet(HoodieTimeline.ROLLBACK_ACTION));
+    checkTimeline.accept(timeline.getRestoreTimeline(), Sets.newHashSet(HoodieTimeline.RESTORE_ACTION));
+    checkTimeline.accept(timeline.getSavePointTimeline(), Sets.newHashSet(HoodieTimeline.SAVEPOINT_ACTION));
+    checkTimeline.accept(timeline.getAllCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION,
+                    HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION,
+                    HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION));
+
+    // Get some random Instants
+    Random rand = new Random();
+    Set<String> randomInstants = sup.get().filter(i -> rand.nextBoolean())
+                                          .map(i -> i.getAction())
+                                          .collect(Collectors.toSet());
+    checkTimeline.accept(timeline.getTimelineOfActions(randomInstants), randomInstants);
+  }
+
+  @Test
+  public void testTimelineInstantOperations() {
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    assertEquals("No instant present", timeline.countInstants(), 0);
+
+    // revertToInflight
+    HoodieInstant commit = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1");
+    timeline.createNewInstant(commit);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(commit));
+    HoodieInstant inflight = timeline.revertToInflight(commit);
+    // revert creates the .requested file
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deleteInflight
+    timeline.deleteInflight(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deletePending
+    timeline.createNewInstant(commit);
+    timeline.createNewInstant(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    timeline.deletePending(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertTrue(timeline.containsInstant(commit));
+
+    // deleteCompactionRequested
+    HoodieInstant compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "2");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 2);
+    timeline.deleteCompactionRequested(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(commit));
+
+    // transitionCompactionXXXtoYYY and revertCompactionXXXtoYYY
+    compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "3");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(inflight));
+    compaction = timeline.revertCompactionInflightToRequested(inflight);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    compaction = timeline.transitionCompactionInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // transitionCleanXXXtoYYY
+    HoodieInstant clean = new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "4");
+    timeline.saveToCleanRequested(clean, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    inflight = timeline.transitionCleanRequestedToInflight(clean, Option.empty());
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(clean));
+    assertTrue(timeline.containsInstant(inflight));
+    clean = timeline.transitionCleanInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // Various states of Instants
+    HoodieInstant srcInstant = new HoodieInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant otherInstant = HoodieTimeline.getRequestedInstant(srcInstant);
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "5"));
+
+    // containsOrBeforeTimelineStarts
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    timeline.setInstants(allInstants);
+    timeline.createNewInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "2"));
+    allInstants.stream().map(i -> i.getTimestamp()).forEach(s -> assertTrue(timeline.containsOrBeforeTimelineStarts(s)));
+    assertTrue(timeline.containsOrBeforeTimelineStarts("0"));
+    assertFalse(timeline.containsOrBeforeTimelineStarts(String.valueOf(System.currentTimeMillis() + 1000)));
+    assertFalse(timeline.getTimelineHash().isEmpty());
+
+    // Etc.
+    timeline.getInstantDetails(clean);
+    timeline.toString();
+    clean.toString();
+  }
+
+  @Test
+  public void testCreateInstants() {
 
 Review comment:
   Generally, for any test, you want to have an evaluation function, if you want to test createInstants for eg, then get the number of instants from getAllInstants() and check against the number provided by       timeline.createNewInstant(instant by asserting they are equal. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r374915701
 
 

 ##########
 File path: hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTimeline.java
 ##########
 @@ -111,21 +115,21 @@
    * view is constructed with this timeline so that file-slice after pending compaction-requested instant-time is also
    * considered valid. A RT file-system view for reading must then merge the file-slices before and after pending
    * compaction instant so that all delta-commits are read.
-   * 
+   *
 
 Review comment:
   Sure. will fix my setup.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r374916236
 
 

 ##########
 File path: hudi-hadoop-mr/pom.xml
 ##########
 @@ -114,6 +114,10 @@
         <groupId>org.apache.rat</groupId>
         <artifactId>apache-rat-plugin</artifactId>
       </plugin>
+      <plugin>
 
 Review comment:
   Sure

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r373217796
 
 

 ##########
 File path: hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java
 ##########
 @@ -164,4 +173,242 @@ public void testTimelineOperations() {
     assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
     assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
   }
+
+  @Test
+  public void testTimelineGetOperations() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    BiConsumer<HoodieTimeline, Set<String>> checkTimeline = (HoodieTimeline timeline, Set<String> actions) -> {
+      sup.get().filter(i -> actions.contains(i.getAction())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !actions.contains(i.getAction())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    // Test that various types of getXXX operations from HoodieActiveTimeline
+    // return the correct set of Instant
+    checkTimeline.accept(timeline.getCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCommitsAndCompactionTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION));
+    checkTimeline.accept(timeline.getCommitTimeline(), Sets.newHashSet(HoodieTimeline.COMMIT_ACTION));
+
+    checkTimeline.accept(timeline.getDeltaCommitTimeline(), Sets.newHashSet(HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCleanerTimeline(), Sets.newHashSet(HoodieTimeline.CLEAN_ACTION));
+    checkTimeline.accept(timeline.getRollbackTimeline(), Sets.newHashSet(HoodieTimeline.ROLLBACK_ACTION));
+    checkTimeline.accept(timeline.getRestoreTimeline(), Sets.newHashSet(HoodieTimeline.RESTORE_ACTION));
+    checkTimeline.accept(timeline.getSavePointTimeline(), Sets.newHashSet(HoodieTimeline.SAVEPOINT_ACTION));
+    checkTimeline.accept(timeline.getAllCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION,
+                    HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION,
+                    HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION));
+
+    // Get some random Instants
+    Random rand = new Random();
+    Set<String> randomInstants = sup.get().filter(i -> rand.nextBoolean())
+                                          .map(i -> i.getAction())
+                                          .collect(Collectors.toSet());
+    checkTimeline.accept(timeline.getTimelineOfActions(randomInstants), randomInstants);
+  }
+
+  @Test
+  public void testTimelineInstantOperations() {
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    assertEquals("No instant present", timeline.countInstants(), 0);
+
+    // revertToInflight
+    HoodieInstant commit = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1");
+    timeline.createNewInstant(commit);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(commit));
+    HoodieInstant inflight = timeline.revertToInflight(commit);
+    // revert creates the .requested file
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deleteInflight
+    timeline.deleteInflight(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deletePending
+    timeline.createNewInstant(commit);
+    timeline.createNewInstant(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    timeline.deletePending(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertTrue(timeline.containsInstant(commit));
+
+    // deleteCompactionRequested
+    HoodieInstant compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "2");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 2);
+    timeline.deleteCompactionRequested(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(commit));
+
+    // transitionCompactionXXXtoYYY and revertCompactionXXXtoYYY
+    compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "3");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(inflight));
+    compaction = timeline.revertCompactionInflightToRequested(inflight);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    compaction = timeline.transitionCompactionInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // transitionCleanXXXtoYYY
+    HoodieInstant clean = new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "4");
+    timeline.saveToCleanRequested(clean, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    inflight = timeline.transitionCleanRequestedToInflight(clean, Option.empty());
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(clean));
+    assertTrue(timeline.containsInstant(inflight));
+    clean = timeline.transitionCleanInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // Various states of Instants
+    HoodieInstant srcInstant = new HoodieInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant otherInstant = HoodieTimeline.getRequestedInstant(srcInstant);
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "5"));
+
+    // containsOrBeforeTimelineStarts
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    timeline.setInstants(allInstants);
+    timeline.createNewInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "2"));
+    allInstants.stream().map(i -> i.getTimestamp()).forEach(s -> assertTrue(timeline.containsOrBeforeTimelineStarts(s)));
+    assertTrue(timeline.containsOrBeforeTimelineStarts("0"));
+    assertFalse(timeline.containsOrBeforeTimelineStarts(String.valueOf(System.currentTimeMillis() + 1000)));
+    assertFalse(timeline.getTimelineHash().isEmpty());
+
+    // Etc.
+    timeline.getInstantDetails(clean);
 
 Review comment:
   removed.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r372562333
 
 

 ##########
 File path: hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java
 ##########
 @@ -164,4 +173,242 @@ public void testTimelineOperations() {
     assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
     assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
   }
+
+  @Test
+  public void testTimelineGetOperations() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    BiConsumer<HoodieTimeline, Set<String>> checkTimeline = (HoodieTimeline timeline, Set<String> actions) -> {
+      sup.get().filter(i -> actions.contains(i.getAction())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !actions.contains(i.getAction())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    // Test that various types of getXXX operations from HoodieActiveTimeline
+    // return the correct set of Instant
+    checkTimeline.accept(timeline.getCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCommitsAndCompactionTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION));
+    checkTimeline.accept(timeline.getCommitTimeline(), Sets.newHashSet(HoodieTimeline.COMMIT_ACTION));
+
+    checkTimeline.accept(timeline.getDeltaCommitTimeline(), Sets.newHashSet(HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCleanerTimeline(), Sets.newHashSet(HoodieTimeline.CLEAN_ACTION));
+    checkTimeline.accept(timeline.getRollbackTimeline(), Sets.newHashSet(HoodieTimeline.ROLLBACK_ACTION));
+    checkTimeline.accept(timeline.getRestoreTimeline(), Sets.newHashSet(HoodieTimeline.RESTORE_ACTION));
+    checkTimeline.accept(timeline.getSavePointTimeline(), Sets.newHashSet(HoodieTimeline.SAVEPOINT_ACTION));
+    checkTimeline.accept(timeline.getAllCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION,
+                    HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION,
+                    HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION));
+
+    // Get some random Instants
+    Random rand = new Random();
+    Set<String> randomInstants = sup.get().filter(i -> rand.nextBoolean())
+                                          .map(i -> i.getAction())
+                                          .collect(Collectors.toSet());
+    checkTimeline.accept(timeline.getTimelineOfActions(randomInstants), randomInstants);
+  }
+
+  @Test
+  public void testTimelineInstantOperations() {
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    assertEquals("No instant present", timeline.countInstants(), 0);
+
+    // revertToInflight
+    HoodieInstant commit = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1");
+    timeline.createNewInstant(commit);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(commit));
+    HoodieInstant inflight = timeline.revertToInflight(commit);
+    // revert creates the .requested file
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deleteInflight
+    timeline.deleteInflight(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deletePending
+    timeline.createNewInstant(commit);
+    timeline.createNewInstant(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    timeline.deletePending(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertTrue(timeline.containsInstant(commit));
+
+    // deleteCompactionRequested
+    HoodieInstant compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "2");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 2);
+    timeline.deleteCompactionRequested(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(commit));
+
+    // transitionCompactionXXXtoYYY and revertCompactionXXXtoYYY
+    compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "3");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(inflight));
+    compaction = timeline.revertCompactionInflightToRequested(inflight);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    compaction = timeline.transitionCompactionInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // transitionCleanXXXtoYYY
+    HoodieInstant clean = new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "4");
+    timeline.saveToCleanRequested(clean, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    inflight = timeline.transitionCleanRequestedToInflight(clean, Option.empty());
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(clean));
+    assertTrue(timeline.containsInstant(inflight));
+    clean = timeline.transitionCleanInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // Various states of Instants
+    HoodieInstant srcInstant = new HoodieInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant otherInstant = HoodieTimeline.getRequestedInstant(srcInstant);
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "5"));
+
+    // containsOrBeforeTimelineStarts
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    timeline.setInstants(allInstants);
+    timeline.createNewInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "2"));
+    allInstants.stream().map(i -> i.getTimestamp()).forEach(s -> assertTrue(timeline.containsOrBeforeTimelineStarts(s)));
+    assertTrue(timeline.containsOrBeforeTimelineStarts("0"));
+    assertFalse(timeline.containsOrBeforeTimelineStarts(String.valueOf(System.currentTimeMillis() + 1000)));
+    assertFalse(timeline.getTimelineHash().isEmpty());
+
+    // Etc.
+    timeline.getInstantDetails(clean);
 
 Review comment:
   Are these following lines testing anything ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r372563701
 
 

 ##########
 File path: hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java
 ##########
 @@ -164,4 +173,242 @@ public void testTimelineOperations() {
     assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
     assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
   }
+
+  @Test
+  public void testTimelineGetOperations() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    BiConsumer<HoodieTimeline, Set<String>> checkTimeline = (HoodieTimeline timeline, Set<String> actions) -> {
+      sup.get().filter(i -> actions.contains(i.getAction())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !actions.contains(i.getAction())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    // Test that various types of getXXX operations from HoodieActiveTimeline
+    // return the correct set of Instant
+    checkTimeline.accept(timeline.getCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCommitsAndCompactionTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION));
+    checkTimeline.accept(timeline.getCommitTimeline(), Sets.newHashSet(HoodieTimeline.COMMIT_ACTION));
+
+    checkTimeline.accept(timeline.getDeltaCommitTimeline(), Sets.newHashSet(HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCleanerTimeline(), Sets.newHashSet(HoodieTimeline.CLEAN_ACTION));
+    checkTimeline.accept(timeline.getRollbackTimeline(), Sets.newHashSet(HoodieTimeline.ROLLBACK_ACTION));
+    checkTimeline.accept(timeline.getRestoreTimeline(), Sets.newHashSet(HoodieTimeline.RESTORE_ACTION));
+    checkTimeline.accept(timeline.getSavePointTimeline(), Sets.newHashSet(HoodieTimeline.SAVEPOINT_ACTION));
+    checkTimeline.accept(timeline.getAllCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION,
+                    HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION,
+                    HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION));
+
+    // Get some random Instants
+    Random rand = new Random();
+    Set<String> randomInstants = sup.get().filter(i -> rand.nextBoolean())
+                                          .map(i -> i.getAction())
+                                          .collect(Collectors.toSet());
+    checkTimeline.accept(timeline.getTimelineOfActions(randomInstants), randomInstants);
+  }
+
+  @Test
+  public void testTimelineInstantOperations() {
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    assertEquals("No instant present", timeline.countInstants(), 0);
+
+    // revertToInflight
+    HoodieInstant commit = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1");
+    timeline.createNewInstant(commit);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(commit));
+    HoodieInstant inflight = timeline.revertToInflight(commit);
+    // revert creates the .requested file
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deleteInflight
+    timeline.deleteInflight(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deletePending
+    timeline.createNewInstant(commit);
+    timeline.createNewInstant(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    timeline.deletePending(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertTrue(timeline.containsInstant(commit));
+
+    // deleteCompactionRequested
+    HoodieInstant compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "2");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 2);
+    timeline.deleteCompactionRequested(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(commit));
+
+    // transitionCompactionXXXtoYYY and revertCompactionXXXtoYYY
+    compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "3");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(inflight));
+    compaction = timeline.revertCompactionInflightToRequested(inflight);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    compaction = timeline.transitionCompactionInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // transitionCleanXXXtoYYY
+    HoodieInstant clean = new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "4");
+    timeline.saveToCleanRequested(clean, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    inflight = timeline.transitionCleanRequestedToInflight(clean, Option.empty());
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(clean));
+    assertTrue(timeline.containsInstant(inflight));
+    clean = timeline.transitionCleanInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // Various states of Instants
+    HoodieInstant srcInstant = new HoodieInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant otherInstant = HoodieTimeline.getRequestedInstant(srcInstant);
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "5"));
+
+    // containsOrBeforeTimelineStarts
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    timeline.setInstants(allInstants);
+    timeline.createNewInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "2"));
+    allInstants.stream().map(i -> i.getTimestamp()).forEach(s -> assertTrue(timeline.containsOrBeforeTimelineStarts(s)));
+    assertTrue(timeline.containsOrBeforeTimelineStarts("0"));
+    assertFalse(timeline.containsOrBeforeTimelineStarts(String.valueOf(System.currentTimeMillis() + 1000)));
+    assertFalse(timeline.getTimelineHash().isEmpty());
+
+    // Etc.
+    timeline.getInstantDetails(clean);
+    timeline.toString();
+    clean.toString();
+  }
+
+  @Test
+  public void testCreateInstants() {
+    for (HoodieInstant instant : getAllInstants()) {
+      timeline.createNewInstant(instant);
+    }
+  }
+
+  @Test
+  public void testInstantFilenameOperations() {
+    HoodieInstant instantRequested = new HoodieInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant instantInflight = new HoodieInstant(State.INFLIGHT, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant instantComplete = new HoodieInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5");
+    assertEquals(HoodieTimeline.getCommitFromCommitFile(instantRequested.getFileName()), "5");
+    assertEquals(HoodieTimeline.getCommitFromCommitFile(instantInflight.getFileName()), "5");
+    assertEquals(HoodieTimeline.getCommitFromCommitFile(instantComplete.getFileName()), "5");
+
+    assertEquals(HoodieTimeline.makeFileNameAsComplete(instantInflight.getFileName()),
+            instantComplete.getFileName());
+
+    assertEquals(HoodieTimeline.makeFileNameAsInflight(instantComplete.getFileName()),
+            instantInflight.getFileName());
+  }
+
+  @Test
+  public void testFiltering() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+
+    timeline = new HoodieActiveTimeline(metaClient);
+    timeline.setInstants(allInstants);
+
+    // getReverseOrderedInstants
+    Stream<HoodieInstant> instants = timeline.getReverseOrderedInstants();
+    List<HoodieInstant> v1 = instants.collect(Collectors.toList());
+    List<HoodieInstant> v2 = sup.get().collect(Collectors.toList());
+    Collections.reverse(v2);
+    assertEquals(v1, v2);
+
+    BiConsumer<HoodieTimeline, Set<State>> checkFilter = (HoodieTimeline timeline, Set<State> states) -> {
+      sup.get().filter(i -> states.contains(i.getState())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !states.contains(i.getState())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    checkFilter.accept(timeline.filter(i -> false), Sets.newHashSet());
+    checkFilter.accept(timeline.filterInflights(), Sets.newHashSet(State.INFLIGHT));
+    checkFilter.accept(timeline.filterInflightsAndRequested(),
+            Sets.newHashSet(State.INFLIGHT, State.REQUESTED));
+
+    // filterCompletedAndCompactionInstants
+    // This cannot be done using checkFilter as it involves both states and actions
+    final HoodieTimeline t1 = timeline.filterCompletedAndCompactionInstants();
+    final Set<State> states = Sets.newHashSet(State.REQUESTED, State.COMPLETED);
+    final Set<String> actions = Sets.newHashSet(HoodieTimeline.COMPACTION_ACTION);
+    sup.get().filter(i -> states.contains(i.getState()) || actions.contains(i.getAction()))
+        .forEach(i -> assertTrue(t1.containsInstant(i)));
+    sup.get().filter(i -> !(states.contains(i.getState()) || actions.contains(i.getAction())))
+        .forEach(i -> assertFalse(t1.containsInstant(i)));
+
+    // filterPendingCompactionTimeline
+    final HoodieTimeline t2 = timeline.filterPendingCompactionTimeline();
+    sup.get().filter(i -> i.getAction() == HoodieTimeline.COMPACTION_ACTION)
+        .forEach(i -> assertTrue(t2.containsInstant(i)));
+    sup.get().filter(i -> i.getAction() != HoodieTimeline.COMPACTION_ACTION)
+        .forEach(i -> assertFalse(t2.containsInstant(i)));
+  }
+
+  private Vector<HoodieInstant> getAllInstants() {
+    timeline = new HoodieActiveTimeline(metaClient);
+    Vector<HoodieInstant> allInstants = new Vector<HoodieInstant>();
+    long startTime = System.currentTimeMillis();
+    for (State state : State.values()) {
+      if (state == State.INVALID) {
+        continue;
+      }
+      for (String action : new String[] {HoodieTimeline.COMMIT_ACTION,
+                                         HoodieTimeline.DELTA_COMMIT_ACTION,
+                                         HoodieTimeline.CLEAN_ACTION,
+                                         HoodieTimeline.SAVEPOINT_ACTION,
+                                         HoodieTimeline.RESTORE_ACTION,
+                                         HoodieTimeline.ROLLBACK_ACTION,
+                                         HoodieTimeline.COMPACTION_ACTION,
+          }) {
+        allInstants.add(new HoodieInstant(state, action, String.valueOf(startTime++)));
 
 Review comment:
   Please use a valid commitTime here or standardize with "000", "001" used in other tests.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] prashantwason commented on issue #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on issue #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#issuecomment-582143133
 
 
   I have addressed all the comments.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r374444682
 
 

 ##########
 File path: hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java
 ##########
 @@ -164,4 +173,269 @@ public void testTimelineOperations() {
     assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
     assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
   }
+
+  @Test
+  public void testTimelineGetOperations() {
+    List<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    /**
+     * Helper function to check HoodieTimeline only contains some type of Instant actions.
+     * @param timeline The HoodieTimeline to check
+     * @param actions The actions that should be present in the timeline being checked
+     */
+    BiConsumer<HoodieTimeline, Set<String>> checkTimeline = (HoodieTimeline timeline, Set<String> actions) -> {
+      sup.get().filter(i -> actions.contains(i.getAction())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !actions.contains(i.getAction())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    // Test that various types of getXXX operations from HoodieActiveTimeline
+    // return the correct set of Instant
+    checkTimeline.accept(timeline.getCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCommitsAndCompactionTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION));
+    checkTimeline.accept(timeline.getCommitTimeline(), Sets.newHashSet(HoodieTimeline.COMMIT_ACTION));
+
+    checkTimeline.accept(timeline.getDeltaCommitTimeline(), Sets.newHashSet(HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCleanerTimeline(), Sets.newHashSet(HoodieTimeline.CLEAN_ACTION));
+    checkTimeline.accept(timeline.getRollbackTimeline(), Sets.newHashSet(HoodieTimeline.ROLLBACK_ACTION));
+    checkTimeline.accept(timeline.getRestoreTimeline(), Sets.newHashSet(HoodieTimeline.RESTORE_ACTION));
+    checkTimeline.accept(timeline.getSavePointTimeline(), Sets.newHashSet(HoodieTimeline.SAVEPOINT_ACTION));
+    checkTimeline.accept(timeline.getAllCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION,
+                    HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION,
+                    HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION));
+
+    // Get some random Instants
+    Random rand = new Random();
+    Set<String> randomInstants = sup.get().filter(i -> rand.nextBoolean())
+                                          .map(i -> i.getAction())
+                                          .collect(Collectors.toSet());
+    checkTimeline.accept(timeline.getTimelineOfActions(randomInstants), randomInstants);
+  }
+
+  @Test
+  public void testTimelineInstantOperations() {
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    assertEquals("No instant present", timeline.countInstants(), 0);
+
+    // revertToInflight
+    HoodieInstant commit = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1");
+    timeline.createNewInstant(commit);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(commit));
+    HoodieInstant inflight = timeline.revertToInflight(commit);
+    // revert creates the .requested file
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deleteInflight
+    timeline.deleteInflight(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deletePending
+    timeline.createNewInstant(commit);
+    timeline.createNewInstant(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    timeline.deletePending(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertTrue(timeline.containsInstant(commit));
+
+    // deleteCompactionRequested
+    HoodieInstant compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "2");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 2);
+    timeline.deleteCompactionRequested(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(commit));
+
+    // transitionCompactionXXXtoYYY and revertCompactionXXXtoYYY
+    compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "3");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(inflight));
+    compaction = timeline.revertCompactionInflightToRequested(inflight);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    compaction = timeline.transitionCompactionInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // transitionCleanXXXtoYYY
+    HoodieInstant clean = new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "4");
+    timeline.saveToCleanRequested(clean, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    inflight = timeline.transitionCleanRequestedToInflight(clean, Option.empty());
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(clean));
+    assertTrue(timeline.containsInstant(inflight));
+    clean = timeline.transitionCleanInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // Various states of Instants
+    HoodieInstant srcInstant = new HoodieInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant otherInstant = HoodieTimeline.getRequestedInstant(srcInstant);
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "5"));
+
+    // containsOrBeforeTimelineStarts
+    List<HoodieInstant> allInstants = getAllInstants();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    timeline.setInstants(allInstants);
+    timeline.createNewInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "2"));
+    allInstants.stream().map(i -> i.getTimestamp()).forEach(s -> assertTrue(timeline.containsOrBeforeTimelineStarts(s)));
+    assertTrue(timeline.containsOrBeforeTimelineStarts("0"));
+    assertFalse(timeline.containsOrBeforeTimelineStarts(String.valueOf(System.currentTimeMillis() + 1000)));
+    assertFalse(timeline.getTimelineHash().isEmpty());
+  }
+
+  @Test
+  public void testCreateInstants() {
+    List<HoodieInstant> allInstants = getAllInstants();
+    for (HoodieInstant instant : allInstants) {
+      timeline.createNewInstant(instant);
+    }
+
+    timeline = timeline.reload();
+    for (HoodieInstant instant : allInstants) {
+      assertTrue(timeline.containsInstant(instant));
+    }
+  }
+
+  @Test
+  public void testInstantFilenameOperations() {
+    HoodieInstant instantRequested = new HoodieInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant instantInflight = new HoodieInstant(State.INFLIGHT, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant instantComplete = new HoodieInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5");
+    assertEquals(HoodieTimeline.getCommitFromCommitFile(instantRequested.getFileName()), "5");
+    assertEquals(HoodieTimeline.getCommitFromCommitFile(instantInflight.getFileName()), "5");
+    assertEquals(HoodieTimeline.getCommitFromCommitFile(instantComplete.getFileName()), "5");
+
+    assertEquals(HoodieTimeline.makeFileNameAsComplete(instantInflight.getFileName()),
+            instantComplete.getFileName());
+
+    assertEquals(HoodieTimeline.makeFileNameAsInflight(instantComplete.getFileName()),
+            instantInflight.getFileName());
+  }
+
+  @Test
+  public void testFiltering() {
+    List<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+
+    timeline = new HoodieActiveTimeline(metaClient);
+    timeline.setInstants(allInstants);
+
+    // getReverseOrderedInstants
+    Stream<HoodieInstant> instants = timeline.getReverseOrderedInstants();
+    List<HoodieInstant> v1 = instants.collect(Collectors.toList());
+    List<HoodieInstant> v2 = sup.get().collect(Collectors.toList());
+    Collections.reverse(v2);
+    assertEquals(v1, v2);
+
+    /**
+     * Helper function to check HoodieTimeline only contains some type of Instant states.
+     * @param timeline The HoodieTimeline to check
+     * @param states The states that should be present in the timeline being checked
+     */
+    BiConsumer<HoodieTimeline, Set<State>> checkFilter = (HoodieTimeline timeline, Set<State> states) -> {
+      sup.get().filter(i -> states.contains(i.getState())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !states.contains(i.getState())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    checkFilter.accept(timeline.filter(i -> false), Sets.newHashSet());
+    checkFilter.accept(timeline.filterInflights(), Sets.newHashSet(State.INFLIGHT));
+    checkFilter.accept(timeline.filterInflightsAndRequested(),
+            Sets.newHashSet(State.INFLIGHT, State.REQUESTED));
+
+    // filterCompletedAndCompactionInstants
+    // This cannot be done using checkFilter as it involves both states and actions
+    final HoodieTimeline t1 = timeline.filterCompletedAndCompactionInstants();
+    final Set<State> states = Sets.newHashSet(State.REQUESTED, State.COMPLETED);
+    final Set<String> actions = Sets.newHashSet(HoodieTimeline.COMPACTION_ACTION);
+    sup.get().filter(i -> states.contains(i.getState()) || actions.contains(i.getAction()))
+        .forEach(i -> assertTrue(t1.containsInstant(i)));
+    sup.get().filter(i -> !(states.contains(i.getState()) || actions.contains(i.getAction())))
+        .forEach(i -> assertFalse(t1.containsInstant(i)));
+
+    // filterPendingCompactionTimeline
+    final HoodieTimeline t2 = timeline.filterPendingCompactionTimeline();
+    sup.get().filter(i -> i.getAction() == HoodieTimeline.COMPACTION_ACTION)
+        .forEach(i -> assertTrue(t2.containsInstant(i)));
+    sup.get().filter(i -> i.getAction() != HoodieTimeline.COMPACTION_ACTION)
+        .forEach(i -> assertFalse(t2.containsInstant(i)));
+  }
+
+  /**
+   * Returns an exhaustive list of all possible HoodieInstant.
+   * @return list of HoodieInstant
+   */
+  private List<HoodieInstant> getAllInstants() {
+    timeline = new HoodieActiveTimeline(metaClient);
+    Vector<HoodieInstant> allInstants = new Vector<HoodieInstant>();
 
 Review comment:
   Can we just use List since that's consistently used across the project ? (unless there is a need for vector)

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r373214581
 
 

 ##########
 File path: hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java
 ##########
 @@ -164,4 +173,242 @@ public void testTimelineOperations() {
     assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
     assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
   }
+
+  @Test
+  public void testTimelineGetOperations() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    BiConsumer<HoodieTimeline, Set<String>> checkTimeline = (HoodieTimeline timeline, Set<String> actions) -> {
+      sup.get().filter(i -> actions.contains(i.getAction())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !actions.contains(i.getAction())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    // Test that various types of getXXX operations from HoodieActiveTimeline
+    // return the correct set of Instant
+    checkTimeline.accept(timeline.getCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCommitsAndCompactionTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION));
+    checkTimeline.accept(timeline.getCommitTimeline(), Sets.newHashSet(HoodieTimeline.COMMIT_ACTION));
+
+    checkTimeline.accept(timeline.getDeltaCommitTimeline(), Sets.newHashSet(HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCleanerTimeline(), Sets.newHashSet(HoodieTimeline.CLEAN_ACTION));
+    checkTimeline.accept(timeline.getRollbackTimeline(), Sets.newHashSet(HoodieTimeline.ROLLBACK_ACTION));
+    checkTimeline.accept(timeline.getRestoreTimeline(), Sets.newHashSet(HoodieTimeline.RESTORE_ACTION));
+    checkTimeline.accept(timeline.getSavePointTimeline(), Sets.newHashSet(HoodieTimeline.SAVEPOINT_ACTION));
+    checkTimeline.accept(timeline.getAllCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION,
+                    HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION,
+                    HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION));
+
+    // Get some random Instants
+    Random rand = new Random();
+    Set<String> randomInstants = sup.get().filter(i -> rand.nextBoolean())
+                                          .map(i -> i.getAction())
+                                          .collect(Collectors.toSet());
+    checkTimeline.accept(timeline.getTimelineOfActions(randomInstants), randomInstants);
+  }
+
+  @Test
+  public void testTimelineInstantOperations() {
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    assertEquals("No instant present", timeline.countInstants(), 0);
+
+    // revertToInflight
+    HoodieInstant commit = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1");
+    timeline.createNewInstant(commit);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(commit));
+    HoodieInstant inflight = timeline.revertToInflight(commit);
+    // revert creates the .requested file
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deleteInflight
+    timeline.deleteInflight(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deletePending
+    timeline.createNewInstant(commit);
+    timeline.createNewInstant(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    timeline.deletePending(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertTrue(timeline.containsInstant(commit));
+
+    // deleteCompactionRequested
+    HoodieInstant compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "2");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 2);
+    timeline.deleteCompactionRequested(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(commit));
+
+    // transitionCompactionXXXtoYYY and revertCompactionXXXtoYYY
+    compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "3");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(inflight));
+    compaction = timeline.revertCompactionInflightToRequested(inflight);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    compaction = timeline.transitionCompactionInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // transitionCleanXXXtoYYY
+    HoodieInstant clean = new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "4");
+    timeline.saveToCleanRequested(clean, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    inflight = timeline.transitionCleanRequestedToInflight(clean, Option.empty());
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(clean));
+    assertTrue(timeline.containsInstant(inflight));
+    clean = timeline.transitionCleanInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // Various states of Instants
+    HoodieInstant srcInstant = new HoodieInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant otherInstant = HoodieTimeline.getRequestedInstant(srcInstant);
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "5"));
+
+    // containsOrBeforeTimelineStarts
+    Vector<HoodieInstant> allInstants = getAllInstants();
 
 Review comment:
   List is fine too.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r374444581
 
 

 ##########
 File path: hudi-hadoop-mr/pom.xml
 ##########
 @@ -114,6 +114,10 @@
         <groupId>org.apache.rat</groupId>
         <artifactId>apache-rat-plugin</artifactId>
       </plugin>
+      <plugin>
 
 Review comment:
   Please move this to a different PR

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r374916739
 
 

 ##########
 File path: hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java
 ##########
 @@ -164,4 +173,269 @@ public void testTimelineOperations() {
     assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
     assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
   }
+
+  @Test
+  public void testTimelineGetOperations() {
+    List<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    /**
+     * Helper function to check HoodieTimeline only contains some type of Instant actions.
+     * @param timeline The HoodieTimeline to check
+     * @param actions The actions that should be present in the timeline being checked
+     */
+    BiConsumer<HoodieTimeline, Set<String>> checkTimeline = (HoodieTimeline timeline, Set<String> actions) -> {
+      sup.get().filter(i -> actions.contains(i.getAction())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !actions.contains(i.getAction())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    // Test that various types of getXXX operations from HoodieActiveTimeline
+    // return the correct set of Instant
+    checkTimeline.accept(timeline.getCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCommitsAndCompactionTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION));
+    checkTimeline.accept(timeline.getCommitTimeline(), Sets.newHashSet(HoodieTimeline.COMMIT_ACTION));
+
+    checkTimeline.accept(timeline.getDeltaCommitTimeline(), Sets.newHashSet(HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCleanerTimeline(), Sets.newHashSet(HoodieTimeline.CLEAN_ACTION));
+    checkTimeline.accept(timeline.getRollbackTimeline(), Sets.newHashSet(HoodieTimeline.ROLLBACK_ACTION));
+    checkTimeline.accept(timeline.getRestoreTimeline(), Sets.newHashSet(HoodieTimeline.RESTORE_ACTION));
+    checkTimeline.accept(timeline.getSavePointTimeline(), Sets.newHashSet(HoodieTimeline.SAVEPOINT_ACTION));
+    checkTimeline.accept(timeline.getAllCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION,
+                    HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION,
+                    HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION));
+
+    // Get some random Instants
+    Random rand = new Random();
+    Set<String> randomInstants = sup.get().filter(i -> rand.nextBoolean())
+                                          .map(i -> i.getAction())
+                                          .collect(Collectors.toSet());
+    checkTimeline.accept(timeline.getTimelineOfActions(randomInstants), randomInstants);
+  }
+
+  @Test
+  public void testTimelineInstantOperations() {
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    assertEquals("No instant present", timeline.countInstants(), 0);
+
+    // revertToInflight
+    HoodieInstant commit = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1");
+    timeline.createNewInstant(commit);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(commit));
+    HoodieInstant inflight = timeline.revertToInflight(commit);
+    // revert creates the .requested file
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deleteInflight
+    timeline.deleteInflight(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deletePending
+    timeline.createNewInstant(commit);
+    timeline.createNewInstant(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    timeline.deletePending(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertTrue(timeline.containsInstant(commit));
+
+    // deleteCompactionRequested
+    HoodieInstant compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "2");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 2);
+    timeline.deleteCompactionRequested(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(commit));
+
+    // transitionCompactionXXXtoYYY and revertCompactionXXXtoYYY
+    compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "3");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(inflight));
+    compaction = timeline.revertCompactionInflightToRequested(inflight);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    compaction = timeline.transitionCompactionInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // transitionCleanXXXtoYYY
+    HoodieInstant clean = new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "4");
+    timeline.saveToCleanRequested(clean, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    inflight = timeline.transitionCleanRequestedToInflight(clean, Option.empty());
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(clean));
+    assertTrue(timeline.containsInstant(inflight));
+    clean = timeline.transitionCleanInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // Various states of Instants
+    HoodieInstant srcInstant = new HoodieInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant otherInstant = HoodieTimeline.getRequestedInstant(srcInstant);
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "5"));
+
+    // containsOrBeforeTimelineStarts
+    List<HoodieInstant> allInstants = getAllInstants();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    timeline.setInstants(allInstants);
+    timeline.createNewInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "2"));
+    allInstants.stream().map(i -> i.getTimestamp()).forEach(s -> assertTrue(timeline.containsOrBeforeTimelineStarts(s)));
+    assertTrue(timeline.containsOrBeforeTimelineStarts("0"));
+    assertFalse(timeline.containsOrBeforeTimelineStarts(String.valueOf(System.currentTimeMillis() + 1000)));
+    assertFalse(timeline.getTimelineHash().isEmpty());
+  }
+
+  @Test
+  public void testCreateInstants() {
+    List<HoodieInstant> allInstants = getAllInstants();
+    for (HoodieInstant instant : allInstants) {
+      timeline.createNewInstant(instant);
+    }
+
+    timeline = timeline.reload();
+    for (HoodieInstant instant : allInstants) {
+      assertTrue(timeline.containsInstant(instant));
+    }
+  }
+
+  @Test
+  public void testInstantFilenameOperations() {
+    HoodieInstant instantRequested = new HoodieInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant instantInflight = new HoodieInstant(State.INFLIGHT, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant instantComplete = new HoodieInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5");
+    assertEquals(HoodieTimeline.getCommitFromCommitFile(instantRequested.getFileName()), "5");
+    assertEquals(HoodieTimeline.getCommitFromCommitFile(instantInflight.getFileName()), "5");
+    assertEquals(HoodieTimeline.getCommitFromCommitFile(instantComplete.getFileName()), "5");
+
+    assertEquals(HoodieTimeline.makeFileNameAsComplete(instantInflight.getFileName()),
+            instantComplete.getFileName());
+
+    assertEquals(HoodieTimeline.makeFileNameAsInflight(instantComplete.getFileName()),
+            instantInflight.getFileName());
+  }
+
+  @Test
+  public void testFiltering() {
+    List<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+
+    timeline = new HoodieActiveTimeline(metaClient);
+    timeline.setInstants(allInstants);
+
+    // getReverseOrderedInstants
+    Stream<HoodieInstant> instants = timeline.getReverseOrderedInstants();
+    List<HoodieInstant> v1 = instants.collect(Collectors.toList());
+    List<HoodieInstant> v2 = sup.get().collect(Collectors.toList());
+    Collections.reverse(v2);
+    assertEquals(v1, v2);
+
+    /**
+     * Helper function to check HoodieTimeline only contains some type of Instant states.
+     * @param timeline The HoodieTimeline to check
+     * @param states The states that should be present in the timeline being checked
+     */
+    BiConsumer<HoodieTimeline, Set<State>> checkFilter = (HoodieTimeline timeline, Set<State> states) -> {
+      sup.get().filter(i -> states.contains(i.getState())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !states.contains(i.getState())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    checkFilter.accept(timeline.filter(i -> false), Sets.newHashSet());
+    checkFilter.accept(timeline.filterInflights(), Sets.newHashSet(State.INFLIGHT));
+    checkFilter.accept(timeline.filterInflightsAndRequested(),
+            Sets.newHashSet(State.INFLIGHT, State.REQUESTED));
+
+    // filterCompletedAndCompactionInstants
+    // This cannot be done using checkFilter as it involves both states and actions
+    final HoodieTimeline t1 = timeline.filterCompletedAndCompactionInstants();
+    final Set<State> states = Sets.newHashSet(State.REQUESTED, State.COMPLETED);
+    final Set<String> actions = Sets.newHashSet(HoodieTimeline.COMPACTION_ACTION);
+    sup.get().filter(i -> states.contains(i.getState()) || actions.contains(i.getAction()))
+        .forEach(i -> assertTrue(t1.containsInstant(i)));
+    sup.get().filter(i -> !(states.contains(i.getState()) || actions.contains(i.getAction())))
+        .forEach(i -> assertFalse(t1.containsInstant(i)));
+
+    // filterPendingCompactionTimeline
+    final HoodieTimeline t2 = timeline.filterPendingCompactionTimeline();
+    sup.get().filter(i -> i.getAction() == HoodieTimeline.COMPACTION_ACTION)
+        .forEach(i -> assertTrue(t2.containsInstant(i)));
+    sup.get().filter(i -> i.getAction() != HoodieTimeline.COMPACTION_ACTION)
+        .forEach(i -> assertFalse(t2.containsInstant(i)));
+  }
+
+  /**
+   * Returns an exhaustive list of all possible HoodieInstant.
+   * @return list of HoodieInstant
+   */
+  private List<HoodieInstant> getAllInstants() {
+    timeline = new HoodieActiveTimeline(metaClient);
+    Vector<HoodieInstant> allInstants = new Vector<HoodieInstant>();
 
 Review comment:
   No real need for vector.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r372561723
 
 

 ##########
 File path: hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java
 ##########
 @@ -164,4 +173,242 @@ public void testTimelineOperations() {
     assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
     assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
   }
+
+  @Test
+  public void testTimelineGetOperations() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    BiConsumer<HoodieTimeline, Set<String>> checkTimeline = (HoodieTimeline timeline, Set<String> actions) -> {
+      sup.get().filter(i -> actions.contains(i.getAction())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !actions.contains(i.getAction())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    // Test that various types of getXXX operations from HoodieActiveTimeline
+    // return the correct set of Instant
+    checkTimeline.accept(timeline.getCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCommitsAndCompactionTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION));
+    checkTimeline.accept(timeline.getCommitTimeline(), Sets.newHashSet(HoodieTimeline.COMMIT_ACTION));
+
+    checkTimeline.accept(timeline.getDeltaCommitTimeline(), Sets.newHashSet(HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCleanerTimeline(), Sets.newHashSet(HoodieTimeline.CLEAN_ACTION));
+    checkTimeline.accept(timeline.getRollbackTimeline(), Sets.newHashSet(HoodieTimeline.ROLLBACK_ACTION));
+    checkTimeline.accept(timeline.getRestoreTimeline(), Sets.newHashSet(HoodieTimeline.RESTORE_ACTION));
+    checkTimeline.accept(timeline.getSavePointTimeline(), Sets.newHashSet(HoodieTimeline.SAVEPOINT_ACTION));
+    checkTimeline.accept(timeline.getAllCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION,
+                    HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION,
+                    HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION));
+
+    // Get some random Instants
+    Random rand = new Random();
+    Set<String> randomInstants = sup.get().filter(i -> rand.nextBoolean())
+                                          .map(i -> i.getAction())
+                                          .collect(Collectors.toSet());
+    checkTimeline.accept(timeline.getTimelineOfActions(randomInstants), randomInstants);
+  }
+
+  @Test
+  public void testTimelineInstantOperations() {
+    timeline = new HoodieActiveTimeline(metaClient, true);
 
 Review comment:
   In general, if you're not checking against the "commitTime" of the hoodie instant, can you simply use the ActiveTime.getInstant() to get a valid commit time instead of passing "1", "2" etc. One of the reasons is to standardize this across all tests - all other tests use commit times in the following format -> "000", "001" if they want to check against a deterministic commit time in the test..

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r372561723
 
 

 ##########
 File path: hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java
 ##########
 @@ -164,4 +173,242 @@ public void testTimelineOperations() {
     assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
     assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
   }
+
+  @Test
+  public void testTimelineGetOperations() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    BiConsumer<HoodieTimeline, Set<String>> checkTimeline = (HoodieTimeline timeline, Set<String> actions) -> {
+      sup.get().filter(i -> actions.contains(i.getAction())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !actions.contains(i.getAction())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    // Test that various types of getXXX operations from HoodieActiveTimeline
+    // return the correct set of Instant
+    checkTimeline.accept(timeline.getCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCommitsAndCompactionTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION));
+    checkTimeline.accept(timeline.getCommitTimeline(), Sets.newHashSet(HoodieTimeline.COMMIT_ACTION));
+
+    checkTimeline.accept(timeline.getDeltaCommitTimeline(), Sets.newHashSet(HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCleanerTimeline(), Sets.newHashSet(HoodieTimeline.CLEAN_ACTION));
+    checkTimeline.accept(timeline.getRollbackTimeline(), Sets.newHashSet(HoodieTimeline.ROLLBACK_ACTION));
+    checkTimeline.accept(timeline.getRestoreTimeline(), Sets.newHashSet(HoodieTimeline.RESTORE_ACTION));
+    checkTimeline.accept(timeline.getSavePointTimeline(), Sets.newHashSet(HoodieTimeline.SAVEPOINT_ACTION));
+    checkTimeline.accept(timeline.getAllCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION,
+                    HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION,
+                    HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION));
+
+    // Get some random Instants
+    Random rand = new Random();
+    Set<String> randomInstants = sup.get().filter(i -> rand.nextBoolean())
+                                          .map(i -> i.getAction())
+                                          .collect(Collectors.toSet());
+    checkTimeline.accept(timeline.getTimelineOfActions(randomInstants), randomInstants);
+  }
+
+  @Test
+  public void testTimelineInstantOperations() {
+    timeline = new HoodieActiveTimeline(metaClient, true);
 
 Review comment:
   In general, if you're not checking against the "commitTime" of the hoodie instant, can you simply use the ActiveTime.getInstant() to get a valid commit time instead of passing "1", "2" etc. One of the reasons is to standardize this across all tests - all other tests use commit times in the following format -> "000", "001"..

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r372558557
 
 

 ##########
 File path: pom.xml
 ##########
 @@ -110,6 +110,7 @@
     <spark.bundle.hive.shade.prefix></spark.bundle.hive.shade.prefix>
     <utilities.bundle.hive.scope>provided</utilities.bundle.hive.scope>
     <utilities.bundle.hive.shade.prefix></utilities.bundle.hive.shade.prefix>
+    <argLine>-Xmx1024m -XX:MaxPermSize=256m</argLine>
 
 Review comment:
   We need more memory only for integration tests, any reason to increase memory 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r372560120
 
 

 ##########
 File path: hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java
 ##########
 @@ -164,4 +173,242 @@ public void testTimelineOperations() {
     assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
     assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
   }
+
+  @Test
+  public void testTimelineGetOperations() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
 
 Review comment:
   What is the need to use a Vector here ? Does list not suffice ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] prashantwason commented on issue #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on issue #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#issuecomment-582205972
 
 
   Pushed a merged commit. The test which failed on travis (some timeout) succeeds on my local setup. Hopefully it was a transient error and works this time.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r373212552
 
 

 ##########
 File path: hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java
 ##########
 @@ -164,4 +173,242 @@ public void testTimelineOperations() {
     assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
     assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
   }
+
+  @Test
+  public void testTimelineGetOperations() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    BiConsumer<HoodieTimeline, Set<String>> checkTimeline = (HoodieTimeline timeline, Set<String> actions) -> {
+      sup.get().filter(i -> actions.contains(i.getAction())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !actions.contains(i.getAction())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    // Test that various types of getXXX operations from HoodieActiveTimeline
+    // return the correct set of Instant
+    checkTimeline.accept(timeline.getCommitsTimeline(),
 
 Review comment:
   This is the same way except I have grouped it into a function to reduce the number of lines of test code. Each checkTimline.accept(...) call is 2 lines of repeated code.
   
   If any of them fails then it is similar to a single junit Assert failing. 
   
   I will added a comment describing the function.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r373214384
 
 

 ##########
 File path: hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java
 ##########
 @@ -164,4 +173,242 @@ public void testTimelineOperations() {
     assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
     assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
   }
+
+  @Test
+  public void testTimelineGetOperations() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
 
 Review comment:
   List is fine too. I used Vector as that is the data structure allocated within getAllInstants(). 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r372565360
 
 

 ##########
 File path: hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java
 ##########
 @@ -164,4 +173,242 @@ public void testTimelineOperations() {
     assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
     assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
   }
+
+  @Test
+  public void testTimelineGetOperations() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    BiConsumer<HoodieTimeline, Set<String>> checkTimeline = (HoodieTimeline timeline, Set<String> actions) -> {
+      sup.get().filter(i -> actions.contains(i.getAction())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !actions.contains(i.getAction())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    // Test that various types of getXXX operations from HoodieActiveTimeline
+    // return the correct set of Instant
+    checkTimeline.accept(timeline.getCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCommitsAndCompactionTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION));
+    checkTimeline.accept(timeline.getCommitTimeline(), Sets.newHashSet(HoodieTimeline.COMMIT_ACTION));
+
+    checkTimeline.accept(timeline.getDeltaCommitTimeline(), Sets.newHashSet(HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCleanerTimeline(), Sets.newHashSet(HoodieTimeline.CLEAN_ACTION));
+    checkTimeline.accept(timeline.getRollbackTimeline(), Sets.newHashSet(HoodieTimeline.ROLLBACK_ACTION));
+    checkTimeline.accept(timeline.getRestoreTimeline(), Sets.newHashSet(HoodieTimeline.RESTORE_ACTION));
+    checkTimeline.accept(timeline.getSavePointTimeline(), Sets.newHashSet(HoodieTimeline.SAVEPOINT_ACTION));
+    checkTimeline.accept(timeline.getAllCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION,
+                    HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION,
+                    HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION));
+
+    // Get some random Instants
+    Random rand = new Random();
+    Set<String> randomInstants = sup.get().filter(i -> rand.nextBoolean())
+                                          .map(i -> i.getAction())
+                                          .collect(Collectors.toSet());
+    checkTimeline.accept(timeline.getTimelineOfActions(randomInstants), randomInstants);
+  }
+
+  @Test
+  public void testTimelineInstantOperations() {
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    assertEquals("No instant present", timeline.countInstants(), 0);
+
+    // revertToInflight
+    HoodieInstant commit = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1");
+    timeline.createNewInstant(commit);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(commit));
+    HoodieInstant inflight = timeline.revertToInflight(commit);
+    // revert creates the .requested file
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deleteInflight
+    timeline.deleteInflight(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deletePending
+    timeline.createNewInstant(commit);
+    timeline.createNewInstant(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    timeline.deletePending(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertTrue(timeline.containsInstant(commit));
+
+    // deleteCompactionRequested
+    HoodieInstant compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "2");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 2);
+    timeline.deleteCompactionRequested(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(commit));
+
+    // transitionCompactionXXXtoYYY and revertCompactionXXXtoYYY
+    compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "3");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(inflight));
+    compaction = timeline.revertCompactionInflightToRequested(inflight);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    compaction = timeline.transitionCompactionInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // transitionCleanXXXtoYYY
+    HoodieInstant clean = new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "4");
+    timeline.saveToCleanRequested(clean, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    inflight = timeline.transitionCleanRequestedToInflight(clean, Option.empty());
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(clean));
+    assertTrue(timeline.containsInstant(inflight));
+    clean = timeline.transitionCleanInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // Various states of Instants
+    HoodieInstant srcInstant = new HoodieInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant otherInstant = HoodieTimeline.getRequestedInstant(srcInstant);
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "5"));
+
+    // containsOrBeforeTimelineStarts
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    timeline.setInstants(allInstants);
+    timeline.createNewInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "2"));
+    allInstants.stream().map(i -> i.getTimestamp()).forEach(s -> assertTrue(timeline.containsOrBeforeTimelineStarts(s)));
+    assertTrue(timeline.containsOrBeforeTimelineStarts("0"));
+    assertFalse(timeline.containsOrBeforeTimelineStarts(String.valueOf(System.currentTimeMillis() + 1000)));
+    assertFalse(timeline.getTimelineHash().isEmpty());
+
+    // Etc.
+    timeline.getInstantDetails(clean);
+    timeline.toString();
+    clean.toString();
+  }
+
+  @Test
+  public void testCreateInstants() {
+    for (HoodieInstant instant : getAllInstants()) {
+      timeline.createNewInstant(instant);
+    }
+  }
+
+  @Test
+  public void testInstantFilenameOperations() {
+    HoodieInstant instantRequested = new HoodieInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant instantInflight = new HoodieInstant(State.INFLIGHT, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant instantComplete = new HoodieInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5");
+    assertEquals(HoodieTimeline.getCommitFromCommitFile(instantRequested.getFileName()), "5");
+    assertEquals(HoodieTimeline.getCommitFromCommitFile(instantInflight.getFileName()), "5");
+    assertEquals(HoodieTimeline.getCommitFromCommitFile(instantComplete.getFileName()), "5");
+
+    assertEquals(HoodieTimeline.makeFileNameAsComplete(instantInflight.getFileName()),
+            instantComplete.getFileName());
+
+    assertEquals(HoodieTimeline.makeFileNameAsInflight(instantComplete.getFileName()),
+            instantInflight.getFileName());
+  }
+
+  @Test
+  public void testFiltering() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+
+    timeline = new HoodieActiveTimeline(metaClient);
+    timeline.setInstants(allInstants);
+
+    // getReverseOrderedInstants
+    Stream<HoodieInstant> instants = timeline.getReverseOrderedInstants();
+    List<HoodieInstant> v1 = instants.collect(Collectors.toList());
+    List<HoodieInstant> v2 = sup.get().collect(Collectors.toList());
+    Collections.reverse(v2);
+    assertEquals(v1, v2);
+
+    BiConsumer<HoodieTimeline, Set<State>> checkFilter = (HoodieTimeline timeline, Set<State> states) -> {
+      sup.get().filter(i -> states.contains(i.getState())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !states.contains(i.getState())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    checkFilter.accept(timeline.filter(i -> false), Sets.newHashSet());
+    checkFilter.accept(timeline.filterInflights(), Sets.newHashSet(State.INFLIGHT));
+    checkFilter.accept(timeline.filterInflightsAndRequested(),
+            Sets.newHashSet(State.INFLIGHT, State.REQUESTED));
+
+    // filterCompletedAndCompactionInstants
+    // This cannot be done using checkFilter as it involves both states and actions
+    final HoodieTimeline t1 = timeline.filterCompletedAndCompactionInstants();
+    final Set<State> states = Sets.newHashSet(State.REQUESTED, State.COMPLETED);
+    final Set<String> actions = Sets.newHashSet(HoodieTimeline.COMPACTION_ACTION);
+    sup.get().filter(i -> states.contains(i.getState()) || actions.contains(i.getAction()))
+        .forEach(i -> assertTrue(t1.containsInstant(i)));
+    sup.get().filter(i -> !(states.contains(i.getState()) || actions.contains(i.getAction())))
+        .forEach(i -> assertFalse(t1.containsInstant(i)));
+
+    // filterPendingCompactionTimeline
+    final HoodieTimeline t2 = timeline.filterPendingCompactionTimeline();
+    sup.get().filter(i -> i.getAction() == HoodieTimeline.COMPACTION_ACTION)
+        .forEach(i -> assertTrue(t2.containsInstant(i)));
+    sup.get().filter(i -> i.getAction() != HoodieTimeline.COMPACTION_ACTION)
+        .forEach(i -> assertFalse(t2.containsInstant(i)));
+  }
+
+  private Vector<HoodieInstant> getAllInstants() {
+    timeline = new HoodieActiveTimeline(metaClient);
+    Vector<HoodieInstant> allInstants = new Vector<HoodieInstant>();
+    long startTime = System.currentTimeMillis();
+    for (State state : State.values()) {
+      if (state == State.INVALID) {
+        continue;
+      }
+      for (String action : new String[] {HoodieTimeline.COMMIT_ACTION,
 
 Review comment:
   May be introduce a variable VALID_ACTION_IN_TIMELINE which is a set in the HoodieActiveTimeline class. Tomorrow, if a new action is added, it might be easier to evolve the test ? 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r373324751
 
 

 ##########
 File path: hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java
 ##########
 @@ -164,4 +173,242 @@ public void testTimelineOperations() {
     assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
     assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
   }
+
+  @Test
+  public void testTimelineGetOperations() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    BiConsumer<HoodieTimeline, Set<String>> checkTimeline = (HoodieTimeline timeline, Set<String> actions) -> {
+      sup.get().filter(i -> actions.contains(i.getAction())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !actions.contains(i.getAction())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    // Test that various types of getXXX operations from HoodieActiveTimeline
+    // return the correct set of Instant
+    checkTimeline.accept(timeline.getCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCommitsAndCompactionTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION));
+    checkTimeline.accept(timeline.getCommitTimeline(), Sets.newHashSet(HoodieTimeline.COMMIT_ACTION));
+
+    checkTimeline.accept(timeline.getDeltaCommitTimeline(), Sets.newHashSet(HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCleanerTimeline(), Sets.newHashSet(HoodieTimeline.CLEAN_ACTION));
+    checkTimeline.accept(timeline.getRollbackTimeline(), Sets.newHashSet(HoodieTimeline.ROLLBACK_ACTION));
+    checkTimeline.accept(timeline.getRestoreTimeline(), Sets.newHashSet(HoodieTimeline.RESTORE_ACTION));
+    checkTimeline.accept(timeline.getSavePointTimeline(), Sets.newHashSet(HoodieTimeline.SAVEPOINT_ACTION));
+    checkTimeline.accept(timeline.getAllCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION,
+                    HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION,
+                    HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION));
+
+    // Get some random Instants
+    Random rand = new Random();
+    Set<String> randomInstants = sup.get().filter(i -> rand.nextBoolean())
+                                          .map(i -> i.getAction())
+                                          .collect(Collectors.toSet());
+    checkTimeline.accept(timeline.getTimelineOfActions(randomInstants), randomInstants);
+  }
+
+  @Test
+  public void testTimelineInstantOperations() {
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    assertEquals("No instant present", timeline.countInstants(), 0);
+
+    // revertToInflight
+    HoodieInstant commit = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1");
+    timeline.createNewInstant(commit);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(commit));
+    HoodieInstant inflight = timeline.revertToInflight(commit);
+    // revert creates the .requested file
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deleteInflight
+    timeline.deleteInflight(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deletePending
+    timeline.createNewInstant(commit);
+    timeline.createNewInstant(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    timeline.deletePending(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertTrue(timeline.containsInstant(commit));
+
+    // deleteCompactionRequested
+    HoodieInstant compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "2");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 2);
+    timeline.deleteCompactionRequested(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(commit));
+
+    // transitionCompactionXXXtoYYY and revertCompactionXXXtoYYY
+    compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "3");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(inflight));
+    compaction = timeline.revertCompactionInflightToRequested(inflight);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    compaction = timeline.transitionCompactionInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // transitionCleanXXXtoYYY
+    HoodieInstant clean = new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "4");
+    timeline.saveToCleanRequested(clean, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    inflight = timeline.transitionCleanRequestedToInflight(clean, Option.empty());
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(clean));
+    assertTrue(timeline.containsInstant(inflight));
+    clean = timeline.transitionCleanInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // Various states of Instants
+    HoodieInstant srcInstant = new HoodieInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant otherInstant = HoodieTimeline.getRequestedInstant(srcInstant);
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "5"));
+
+    // containsOrBeforeTimelineStarts
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    timeline.setInstants(allInstants);
+    timeline.createNewInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "2"));
+    allInstants.stream().map(i -> i.getTimestamp()).forEach(s -> assertTrue(timeline.containsOrBeforeTimelineStarts(s)));
+    assertTrue(timeline.containsOrBeforeTimelineStarts("0"));
+    assertFalse(timeline.containsOrBeforeTimelineStarts(String.valueOf(System.currentTimeMillis() + 1000)));
+    assertFalse(timeline.getTimelineHash().isEmpty());
+
+    // Etc.
+    timeline.getInstantDetails(clean);
+    timeline.toString();
+    clean.toString();
+  }
+
+  @Test
+  public void testCreateInstants() {
+    for (HoodieInstant instant : getAllInstants()) {
+      timeline.createNewInstant(instant);
+    }
+  }
+
+  @Test
+  public void testInstantFilenameOperations() {
+    HoodieInstant instantRequested = new HoodieInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant instantInflight = new HoodieInstant(State.INFLIGHT, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant instantComplete = new HoodieInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5");
+    assertEquals(HoodieTimeline.getCommitFromCommitFile(instantRequested.getFileName()), "5");
+    assertEquals(HoodieTimeline.getCommitFromCommitFile(instantInflight.getFileName()), "5");
+    assertEquals(HoodieTimeline.getCommitFromCommitFile(instantComplete.getFileName()), "5");
+
+    assertEquals(HoodieTimeline.makeFileNameAsComplete(instantInflight.getFileName()),
+            instantComplete.getFileName());
+
+    assertEquals(HoodieTimeline.makeFileNameAsInflight(instantComplete.getFileName()),
+            instantInflight.getFileName());
+  }
+
+  @Test
+  public void testFiltering() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+
+    timeline = new HoodieActiveTimeline(metaClient);
+    timeline.setInstants(allInstants);
+
+    // getReverseOrderedInstants
+    Stream<HoodieInstant> instants = timeline.getReverseOrderedInstants();
+    List<HoodieInstant> v1 = instants.collect(Collectors.toList());
+    List<HoodieInstant> v2 = sup.get().collect(Collectors.toList());
+    Collections.reverse(v2);
+    assertEquals(v1, v2);
+
+    BiConsumer<HoodieTimeline, Set<State>> checkFilter = (HoodieTimeline timeline, Set<State> states) -> {
+      sup.get().filter(i -> states.contains(i.getState())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !states.contains(i.getState())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    checkFilter.accept(timeline.filter(i -> false), Sets.newHashSet());
+    checkFilter.accept(timeline.filterInflights(), Sets.newHashSet(State.INFLIGHT));
+    checkFilter.accept(timeline.filterInflightsAndRequested(),
+            Sets.newHashSet(State.INFLIGHT, State.REQUESTED));
+
+    // filterCompletedAndCompactionInstants
+    // This cannot be done using checkFilter as it involves both states and actions
+    final HoodieTimeline t1 = timeline.filterCompletedAndCompactionInstants();
+    final Set<State> states = Sets.newHashSet(State.REQUESTED, State.COMPLETED);
+    final Set<String> actions = Sets.newHashSet(HoodieTimeline.COMPACTION_ACTION);
+    sup.get().filter(i -> states.contains(i.getState()) || actions.contains(i.getAction()))
+        .forEach(i -> assertTrue(t1.containsInstant(i)));
+    sup.get().filter(i -> !(states.contains(i.getState()) || actions.contains(i.getAction())))
+        .forEach(i -> assertFalse(t1.containsInstant(i)));
+
+    // filterPendingCompactionTimeline
+    final HoodieTimeline t2 = timeline.filterPendingCompactionTimeline();
+    sup.get().filter(i -> i.getAction() == HoodieTimeline.COMPACTION_ACTION)
+        .forEach(i -> assertTrue(t2.containsInstant(i)));
+    sup.get().filter(i -> i.getAction() != HoodieTimeline.COMPACTION_ACTION)
+        .forEach(i -> assertFalse(t2.containsInstant(i)));
+  }
+
+  private Vector<HoodieInstant> getAllInstants() {
+    timeline = new HoodieActiveTimeline(metaClient);
+    Vector<HoodieInstant> allInstants = new Vector<HoodieInstant>();
+    long startTime = System.currentTimeMillis();
+    for (State state : State.values()) {
+      if (state == State.INVALID) {
+        continue;
+      }
+      for (String action : new String[] {HoodieTimeline.COMMIT_ACTION,
 
 Review comment:
   Sounds good.
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r372559935
 
 

 ##########
 File path: hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java
 ##########
 @@ -164,4 +173,242 @@ public void testTimelineOperations() {
     assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
     assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
   }
+
+  @Test
+  public void testTimelineGetOperations() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    BiConsumer<HoodieTimeline, Set<String>> checkTimeline = (HoodieTimeline timeline, Set<String> actions) -> {
+      sup.get().filter(i -> actions.contains(i.getAction())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !actions.contains(i.getAction())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    // Test that various types of getXXX operations from HoodieActiveTimeline
+    // return the correct set of Instant
+    checkTimeline.accept(timeline.getCommitsTimeline(),
 
 Review comment:
   In all other parts of the code we use junit Assert, what is the reason to choose a different way of asserting the value of these get operations ? Also, if one of them fails, what is the exception expected ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r374443245
 
 

 ##########
 File path: hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTimeline.java
 ##########
 @@ -111,21 +115,21 @@
    * view is constructed with this timeline so that file-slice after pending compaction-requested instant-time is also
    * considered valid. A RT file-system view for reading must then merge the file-slices before and after pending
    * compaction instant so that all delta-commits are read.
-   * 
+   *
 
 Review comment:
   Can we avoid these changes ? Might be due to your intellij setup, please check

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r372558215
 
 

 ##########
 File path: pom.xml
 ##########
 @@ -257,11 +257,6 @@
               <configuration>
                 <!-- Sets the path to the file which contains the execution data. -->
                 <destFile>${project.build.directory}/coverage-reports/jacoco-ut.exec</destFile>
-                <!--
 
 Review comment:
   What is the reason for this change ? Is this related to test cases ? 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r373207891
 
 

 ##########
 File path: pom.xml
 ##########
 @@ -257,11 +257,6 @@
               <configuration>
                 <!-- Sets the path to the file which contains the execution data. -->
                 <destFile>${project.build.directory}/coverage-reports/jacoco-ut.exec</destFile>
-                <!--
 
 Review comment:
   This is an unrelated change but is required to run jacoco. Currently jacococ plugin does not run on master branch.
   
   I will separate this into another PR.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r372560120
 
 

 ##########
 File path: hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java
 ##########
 @@ -164,4 +173,242 @@ public void testTimelineOperations() {
     assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
     assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
   }
+
+  @Test
+  public void testTimelineGetOperations() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
 
 Review comment:
   What is the need to use a Vector 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] n3nash merged pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
n3nash merged pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287
 
 
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r373218682
 
 

 ##########
 File path: hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java
 ##########
 @@ -164,4 +173,242 @@ public void testTimelineOperations() {
     assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
     assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
   }
+
+  @Test
+  public void testTimelineGetOperations() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    BiConsumer<HoodieTimeline, Set<String>> checkTimeline = (HoodieTimeline timeline, Set<String> actions) -> {
+      sup.get().filter(i -> actions.contains(i.getAction())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !actions.contains(i.getAction())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    // Test that various types of getXXX operations from HoodieActiveTimeline
+    // return the correct set of Instant
+    checkTimeline.accept(timeline.getCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCommitsAndCompactionTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION));
+    checkTimeline.accept(timeline.getCommitTimeline(), Sets.newHashSet(HoodieTimeline.COMMIT_ACTION));
+
+    checkTimeline.accept(timeline.getDeltaCommitTimeline(), Sets.newHashSet(HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCleanerTimeline(), Sets.newHashSet(HoodieTimeline.CLEAN_ACTION));
+    checkTimeline.accept(timeline.getRollbackTimeline(), Sets.newHashSet(HoodieTimeline.ROLLBACK_ACTION));
+    checkTimeline.accept(timeline.getRestoreTimeline(), Sets.newHashSet(HoodieTimeline.RESTORE_ACTION));
+    checkTimeline.accept(timeline.getSavePointTimeline(), Sets.newHashSet(HoodieTimeline.SAVEPOINT_ACTION));
+    checkTimeline.accept(timeline.getAllCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION,
+                    HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION,
+                    HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION));
+
+    // Get some random Instants
+    Random rand = new Random();
+    Set<String> randomInstants = sup.get().filter(i -> rand.nextBoolean())
+                                          .map(i -> i.getAction())
+                                          .collect(Collectors.toSet());
+    checkTimeline.accept(timeline.getTimelineOfActions(randomInstants), randomInstants);
+  }
+
+  @Test
+  public void testTimelineInstantOperations() {
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    assertEquals("No instant present", timeline.countInstants(), 0);
+
+    // revertToInflight
+    HoodieInstant commit = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1");
+    timeline.createNewInstant(commit);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(commit));
+    HoodieInstant inflight = timeline.revertToInflight(commit);
+    // revert creates the .requested file
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deleteInflight
+    timeline.deleteInflight(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deletePending
+    timeline.createNewInstant(commit);
+    timeline.createNewInstant(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    timeline.deletePending(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertTrue(timeline.containsInstant(commit));
+
+    // deleteCompactionRequested
+    HoodieInstant compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "2");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 2);
+    timeline.deleteCompactionRequested(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(commit));
+
+    // transitionCompactionXXXtoYYY and revertCompactionXXXtoYYY
+    compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "3");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(inflight));
+    compaction = timeline.revertCompactionInflightToRequested(inflight);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    compaction = timeline.transitionCompactionInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // transitionCleanXXXtoYYY
+    HoodieInstant clean = new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "4");
+    timeline.saveToCleanRequested(clean, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    inflight = timeline.transitionCleanRequestedToInflight(clean, Option.empty());
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(clean));
+    assertTrue(timeline.containsInstant(inflight));
+    clean = timeline.transitionCleanInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // Various states of Instants
+    HoodieInstant srcInstant = new HoodieInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant otherInstant = HoodieTimeline.getRequestedInstant(srcInstant);
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "5"));
+
+    // containsOrBeforeTimelineStarts
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    timeline.setInstants(allInstants);
+    timeline.createNewInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "2"));
+    allInstants.stream().map(i -> i.getTimestamp()).forEach(s -> assertTrue(timeline.containsOrBeforeTimelineStarts(s)));
+    assertTrue(timeline.containsOrBeforeTimelineStarts("0"));
+    assertFalse(timeline.containsOrBeforeTimelineStarts(String.valueOf(System.currentTimeMillis() + 1000)));
+    assertFalse(timeline.getTimelineHash().isEmpty());
+
+    // Etc.
+    timeline.getInstantDetails(clean);
+    timeline.toString();
+    clean.toString();
+  }
+
+  @Test
+  public void testCreateInstants() {
 
 Review comment:
   Test added.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r372560927
 
 

 ##########
 File path: hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java
 ##########
 @@ -164,4 +173,242 @@ public void testTimelineOperations() {
     assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
     assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
   }
+
+  @Test
+  public void testTimelineGetOperations() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    BiConsumer<HoodieTimeline, Set<String>> checkTimeline = (HoodieTimeline timeline, Set<String> actions) -> {
+      sup.get().filter(i -> actions.contains(i.getAction())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !actions.contains(i.getAction())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    // Test that various types of getXXX operations from HoodieActiveTimeline
+    // return the correct set of Instant
+    checkTimeline.accept(timeline.getCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCommitsAndCompactionTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION));
+    checkTimeline.accept(timeline.getCommitTimeline(), Sets.newHashSet(HoodieTimeline.COMMIT_ACTION));
+
+    checkTimeline.accept(timeline.getDeltaCommitTimeline(), Sets.newHashSet(HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCleanerTimeline(), Sets.newHashSet(HoodieTimeline.CLEAN_ACTION));
+    checkTimeline.accept(timeline.getRollbackTimeline(), Sets.newHashSet(HoodieTimeline.ROLLBACK_ACTION));
+    checkTimeline.accept(timeline.getRestoreTimeline(), Sets.newHashSet(HoodieTimeline.RESTORE_ACTION));
+    checkTimeline.accept(timeline.getSavePointTimeline(), Sets.newHashSet(HoodieTimeline.SAVEPOINT_ACTION));
+    checkTimeline.accept(timeline.getAllCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION,
+                    HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION,
+                    HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION));
+
+    // Get some random Instants
+    Random rand = new Random();
+    Set<String> randomInstants = sup.get().filter(i -> rand.nextBoolean())
+                                          .map(i -> i.getAction())
+                                          .collect(Collectors.toSet());
+    checkTimeline.accept(timeline.getTimelineOfActions(randomInstants), randomInstants);
+  }
+
+  @Test
+  public void testTimelineInstantOperations() {
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    assertEquals("No instant present", timeline.countInstants(), 0);
+
+    // revertToInflight
+    HoodieInstant commit = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1");
+    timeline.createNewInstant(commit);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(commit));
+    HoodieInstant inflight = timeline.revertToInflight(commit);
+    // revert creates the .requested file
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deleteInflight
+    timeline.deleteInflight(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deletePending
+    timeline.createNewInstant(commit);
+    timeline.createNewInstant(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    timeline.deletePending(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertTrue(timeline.containsInstant(commit));
+
+    // deleteCompactionRequested
+    HoodieInstant compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "2");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 2);
+    timeline.deleteCompactionRequested(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(commit));
+
+    // transitionCompactionXXXtoYYY and revertCompactionXXXtoYYY
+    compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "3");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(inflight));
+    compaction = timeline.revertCompactionInflightToRequested(inflight);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    compaction = timeline.transitionCompactionInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // transitionCleanXXXtoYYY
+    HoodieInstant clean = new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "4");
+    timeline.saveToCleanRequested(clean, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    inflight = timeline.transitionCleanRequestedToInflight(clean, Option.empty());
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(clean));
+    assertTrue(timeline.containsInstant(inflight));
+    clean = timeline.transitionCleanInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // Various states of Instants
+    HoodieInstant srcInstant = new HoodieInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant otherInstant = HoodieTimeline.getRequestedInstant(srcInstant);
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "5"));
+
+    // containsOrBeforeTimelineStarts
+    Vector<HoodieInstant> allInstants = getAllInstants();
 
 Review comment:
   Again, what is the use of vector 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r374443111
 
 

 ##########
 File path: hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTimeline.java
 ##########
 @@ -56,6 +56,10 @@
   String REQUESTED_EXTENSION = ".requested";
   String RESTORE_ACTION = "restore";
 
+  String[] VALID_ACTION_IN_TIMELINE = {COMMIT_ACTION, DELTA_COMMIT_ACTION,
 
 Review comment:
   nit : s/VALID_ACTION_IN_TIMELINE/VALID_ACTIONS_IN_TIMELINE/

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r373208014
 
 

 ##########
 File path: pom.xml
 ##########
 @@ -110,6 +110,7 @@
     <spark.bundle.hive.shade.prefix></spark.bundle.hive.shade.prefix>
     <utilities.bundle.hive.scope>provided</utilities.bundle.hive.scope>
     <utilities.bundle.hive.shade.prefix></utilities.bundle.hive.shade.prefix>
+    <argLine>-Xmx1024m -XX:MaxPermSize=256m</argLine>
 
 Review comment:
   This is an unrelated change but is required to run jacoco. Currently jacococ plugin does not run on master branch.
   
   I will separate this into another PR.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r373333246
 
 

 ##########
 File path: hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java
 ##########
 @@ -164,4 +173,242 @@ public void testTimelineOperations() {
     assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
     assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
   }
+
+  @Test
+  public void testTimelineGetOperations() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    BiConsumer<HoodieTimeline, Set<String>> checkTimeline = (HoodieTimeline timeline, Set<String> actions) -> {
+      sup.get().filter(i -> actions.contains(i.getAction())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !actions.contains(i.getAction())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    // Test that various types of getXXX operations from HoodieActiveTimeline
+    // return the correct set of Instant
+    checkTimeline.accept(timeline.getCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCommitsAndCompactionTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION));
+    checkTimeline.accept(timeline.getCommitTimeline(), Sets.newHashSet(HoodieTimeline.COMMIT_ACTION));
+
+    checkTimeline.accept(timeline.getDeltaCommitTimeline(), Sets.newHashSet(HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCleanerTimeline(), Sets.newHashSet(HoodieTimeline.CLEAN_ACTION));
+    checkTimeline.accept(timeline.getRollbackTimeline(), Sets.newHashSet(HoodieTimeline.ROLLBACK_ACTION));
+    checkTimeline.accept(timeline.getRestoreTimeline(), Sets.newHashSet(HoodieTimeline.RESTORE_ACTION));
+    checkTimeline.accept(timeline.getSavePointTimeline(), Sets.newHashSet(HoodieTimeline.SAVEPOINT_ACTION));
+    checkTimeline.accept(timeline.getAllCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION,
+                    HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION,
+                    HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION));
+
+    // Get some random Instants
+    Random rand = new Random();
+    Set<String> randomInstants = sup.get().filter(i -> rand.nextBoolean())
+                                          .map(i -> i.getAction())
+                                          .collect(Collectors.toSet());
+    checkTimeline.accept(timeline.getTimelineOfActions(randomInstants), randomInstants);
+  }
+
+  @Test
+  public void testTimelineInstantOperations() {
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    assertEquals("No instant present", timeline.countInstants(), 0);
+
+    // revertToInflight
+    HoodieInstant commit = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1");
+    timeline.createNewInstant(commit);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(commit));
+    HoodieInstant inflight = timeline.revertToInflight(commit);
+    // revert creates the .requested file
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertTrue(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deleteInflight
+    timeline.deleteInflight(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(commit));
+
+    // deletePending
+    timeline.createNewInstant(commit);
+    timeline.createNewInstant(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    timeline.deletePending(inflight);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertTrue(timeline.containsInstant(commit));
+
+    // deleteCompactionRequested
+    HoodieInstant compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "2");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 2);
+    timeline.deleteCompactionRequested(compaction);
+    timeline = timeline.reload();
+    assertEquals(timeline.countInstants(), 1);
+    assertFalse(timeline.containsInstant(inflight));
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(commit));
+
+    // transitionCompactionXXXtoYYY and revertCompactionXXXtoYYY
+    compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "3");
+    timeline.createNewInstant(compaction);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(compaction));
+    assertTrue(timeline.containsInstant(inflight));
+    compaction = timeline.revertCompactionInflightToRequested(inflight);
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+    inflight = timeline.transitionCompactionRequestedToInflight(compaction);
+    compaction = timeline.transitionCompactionInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(compaction));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // transitionCleanXXXtoYYY
+    HoodieInstant clean = new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "4");
+    timeline.saveToCleanRequested(clean, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    inflight = timeline.transitionCleanRequestedToInflight(clean, Option.empty());
+    timeline = timeline.reload();
+    assertFalse(timeline.containsInstant(clean));
+    assertTrue(timeline.containsInstant(inflight));
+    clean = timeline.transitionCleanInflightToComplete(inflight, Option.empty());
+    timeline = timeline.reload();
+    assertTrue(timeline.containsInstant(clean));
+    assertFalse(timeline.containsInstant(inflight));
+
+    // Various states of Instants
+    HoodieInstant srcInstant = new HoodieInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant otherInstant = HoodieTimeline.getRequestedInstant(srcInstant);
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCleanInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionRequestedInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "5"));
+    otherInstant = HoodieTimeline.getCompactionInflightInstant("5");
+    assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "5"));
+
+    // containsOrBeforeTimelineStarts
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    timeline.setInstants(allInstants);
+    timeline.createNewInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "2"));
+    allInstants.stream().map(i -> i.getTimestamp()).forEach(s -> assertTrue(timeline.containsOrBeforeTimelineStarts(s)));
+    assertTrue(timeline.containsOrBeforeTimelineStarts("0"));
+    assertFalse(timeline.containsOrBeforeTimelineStarts(String.valueOf(System.currentTimeMillis() + 1000)));
+    assertFalse(timeline.getTimelineHash().isEmpty());
+
+    // Etc.
+    timeline.getInstantDetails(clean);
+    timeline.toString();
+    clean.toString();
+  }
+
+  @Test
+  public void testCreateInstants() {
+    for (HoodieInstant instant : getAllInstants()) {
+      timeline.createNewInstant(instant);
+    }
+  }
+
+  @Test
+  public void testInstantFilenameOperations() {
+    HoodieInstant instantRequested = new HoodieInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant instantInflight = new HoodieInstant(State.INFLIGHT, HoodieTimeline.RESTORE_ACTION, "5");
+    HoodieInstant instantComplete = new HoodieInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5");
+    assertEquals(HoodieTimeline.getCommitFromCommitFile(instantRequested.getFileName()), "5");
+    assertEquals(HoodieTimeline.getCommitFromCommitFile(instantInflight.getFileName()), "5");
+    assertEquals(HoodieTimeline.getCommitFromCommitFile(instantComplete.getFileName()), "5");
+
+    assertEquals(HoodieTimeline.makeFileNameAsComplete(instantInflight.getFileName()),
+            instantComplete.getFileName());
+
+    assertEquals(HoodieTimeline.makeFileNameAsInflight(instantComplete.getFileName()),
+            instantInflight.getFileName());
+  }
+
+  @Test
+  public void testFiltering() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+
+    timeline = new HoodieActiveTimeline(metaClient);
+    timeline.setInstants(allInstants);
+
+    // getReverseOrderedInstants
+    Stream<HoodieInstant> instants = timeline.getReverseOrderedInstants();
+    List<HoodieInstant> v1 = instants.collect(Collectors.toList());
+    List<HoodieInstant> v2 = sup.get().collect(Collectors.toList());
+    Collections.reverse(v2);
+    assertEquals(v1, v2);
+
+    BiConsumer<HoodieTimeline, Set<State>> checkFilter = (HoodieTimeline timeline, Set<State> states) -> {
+      sup.get().filter(i -> states.contains(i.getState())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !states.contains(i.getState())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    checkFilter.accept(timeline.filter(i -> false), Sets.newHashSet());
+    checkFilter.accept(timeline.filterInflights(), Sets.newHashSet(State.INFLIGHT));
+    checkFilter.accept(timeline.filterInflightsAndRequested(),
+            Sets.newHashSet(State.INFLIGHT, State.REQUESTED));
+
+    // filterCompletedAndCompactionInstants
+    // This cannot be done using checkFilter as it involves both states and actions
+    final HoodieTimeline t1 = timeline.filterCompletedAndCompactionInstants();
+    final Set<State> states = Sets.newHashSet(State.REQUESTED, State.COMPLETED);
+    final Set<String> actions = Sets.newHashSet(HoodieTimeline.COMPACTION_ACTION);
+    sup.get().filter(i -> states.contains(i.getState()) || actions.contains(i.getAction()))
+        .forEach(i -> assertTrue(t1.containsInstant(i)));
+    sup.get().filter(i -> !(states.contains(i.getState()) || actions.contains(i.getAction())))
+        .forEach(i -> assertFalse(t1.containsInstant(i)));
+
+    // filterPendingCompactionTimeline
+    final HoodieTimeline t2 = timeline.filterPendingCompactionTimeline();
+    sup.get().filter(i -> i.getAction() == HoodieTimeline.COMPACTION_ACTION)
+        .forEach(i -> assertTrue(t2.containsInstant(i)));
+    sup.get().filter(i -> i.getAction() != HoodieTimeline.COMPACTION_ACTION)
+        .forEach(i -> assertFalse(t2.containsInstant(i)));
+  }
+
+  private Vector<HoodieInstant> getAllInstants() {
+    timeline = new HoodieActiveTimeline(metaClient);
+    Vector<HoodieInstant> allInstants = new Vector<HoodieInstant>();
+    long startTime = System.currentTimeMillis();
+    for (State state : State.values()) {
+      if (state == State.INVALID) {
+        continue;
+      }
+      for (String action : new String[] {HoodieTimeline.COMMIT_ACTION,
+                                         HoodieTimeline.DELTA_COMMIT_ACTION,
+                                         HoodieTimeline.CLEAN_ACTION,
+                                         HoodieTimeline.SAVEPOINT_ACTION,
+                                         HoodieTimeline.RESTORE_ACTION,
+                                         HoodieTimeline.ROLLBACK_ACTION,
+                                         HoodieTimeline.COMPACTION_ACTION,
+          }) {
+        allInstants.add(new HoodieInstant(state, action, String.valueOf(startTime++)));
 
 Review comment:
   Done

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-hudi] prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.

Posted by GitBox <gi...@apache.org>.
prashantwason commented on a change in pull request #1287: [HUDI-566] Added new test cases for class HoodieTimeline, HoodieDefaultTimeline and HoodieActiveTimeline.
URL: https://github.com/apache/incubator-hudi/pull/1287#discussion_r373217546
 
 

 ##########
 File path: hudi-common/src/test/java/org/apache/hudi/common/table/string/TestHoodieActiveTimeline.java
 ##########
 @@ -164,4 +173,242 @@ public void testTimelineOperations() {
     assertFalse("", activeCommitTimeline.isBeforeTimelineStarts("02"));
     assertTrue("", activeCommitTimeline.isBeforeTimelineStarts("00"));
   }
+
+  @Test
+  public void testTimelineGetOperations() {
+    Vector<HoodieInstant> allInstants = getAllInstants();
+    Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
+    timeline = new HoodieActiveTimeline(metaClient, true);
+    timeline.setInstants(allInstants);
+
+    BiConsumer<HoodieTimeline, Set<String>> checkTimeline = (HoodieTimeline timeline, Set<String> actions) -> {
+      sup.get().filter(i -> actions.contains(i.getAction())).forEach(i -> assertTrue(timeline.containsInstant(i)));
+      sup.get().filter(i -> !actions.contains(i.getAction())).forEach(i -> assertFalse(timeline.containsInstant(i)));
+    };
+
+    // Test that various types of getXXX operations from HoodieActiveTimeline
+    // return the correct set of Instant
+    checkTimeline.accept(timeline.getCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCommitsAndCompactionTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION));
+    checkTimeline.accept(timeline.getCommitTimeline(), Sets.newHashSet(HoodieTimeline.COMMIT_ACTION));
+
+    checkTimeline.accept(timeline.getDeltaCommitTimeline(), Sets.newHashSet(HoodieTimeline.DELTA_COMMIT_ACTION));
+    checkTimeline.accept(timeline.getCleanerTimeline(), Sets.newHashSet(HoodieTimeline.CLEAN_ACTION));
+    checkTimeline.accept(timeline.getRollbackTimeline(), Sets.newHashSet(HoodieTimeline.ROLLBACK_ACTION));
+    checkTimeline.accept(timeline.getRestoreTimeline(), Sets.newHashSet(HoodieTimeline.RESTORE_ACTION));
+    checkTimeline.accept(timeline.getSavePointTimeline(), Sets.newHashSet(HoodieTimeline.SAVEPOINT_ACTION));
+    checkTimeline.accept(timeline.getAllCommitsTimeline(),
+            Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION,
+                    HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION,
+                    HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION));
+
+    // Get some random Instants
+    Random rand = new Random();
+    Set<String> randomInstants = sup.get().filter(i -> rand.nextBoolean())
+                                          .map(i -> i.getAction())
+                                          .collect(Collectors.toSet());
+    checkTimeline.accept(timeline.getTimelineOfActions(randomInstants), randomInstants);
+  }
+
+  @Test
+  public void testTimelineInstantOperations() {
+    timeline = new HoodieActiveTimeline(metaClient, true);
 
 Review comment:
   I think you mean to use the createNewInstantTime() method to get an instant time. (I did not find any getInstant()).
   
   createNewInstantTime() is not suitable for unit tests like this as it has a mandatory delay of 1 sec between two invocations. So creating many instants times increases the test time.
   
   I can switch to "00X" format.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services