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

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


The following commit(s) were added to refs/heads/master by this push:
     new 5e2b1732986 [HUDI-6674] Add rollback info from metadata table in 
timeline commands (#9411)
5e2b1732986 is described below

commit 5e2b17329865514206c19021abd0266be983a16e
Author: Y Ethan Guo <ethan.guoyi...@gmail.com>
AuthorDate: Sat Aug 12 18:29:50 2023 -0700

    [HUDI-6674] Add rollback info from metadata table in timeline commands 
(#9411)
---
 .../apache/hudi/cli/HoodieTableHeaderFields.java   |  1 -
 .../apache/hudi/cli/commands/TimelineCommand.java  | 99 ++++++++++------------
 2 files changed, 47 insertions(+), 53 deletions(-)

diff --git 
a/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieTableHeaderFields.java 
b/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieTableHeaderFields.java
index 20829251ee2..e1e4ea7c168 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieTableHeaderFields.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieTableHeaderFields.java
@@ -180,7 +180,6 @@ public class HoodieTableHeaderFields {
   public static final String HEADER_REQUESTED_TIME = "Requested\nTime";
   public static final String HEADER_INFLIGHT_TIME = "Inflight\nTime";
   public static final String HEADER_COMPLETED_TIME = "Completed\nTime";
-  public static final String HEADER_ROLLBACK_INFO = "Rollback Info";
   public static final String HEADER_MT_PREFIX = "MT\n";
   public static final String HEADER_MT_ACTION = HEADER_MT_PREFIX + 
HEADER_ACTION;
   public static final String HEADER_MT_STATE = HEADER_MT_PREFIX + HEADER_STATE;
diff --git 
a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TimelineCommand.java 
b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TimelineCommand.java
index 0d63c8a40cd..2b89175293d 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TimelineCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TimelineCommand.java
@@ -208,41 +208,29 @@ public class TimelineCommand {
       Map<String, Map<HoodieInstant.State, HoodieInstantWithModTime>> 
instantInfoMap,
       Integer limit, String sortByField, boolean descending, boolean 
headerOnly, boolean withRowNo,
       boolean showTimeSeconds, boolean showRollbackInfo) {
-    Map<String, List<String>> rollbackInfo = 
getRolledBackInstantInfo(timeline);
+    Map<String, List<String>> rollbackInfoMap = 
getRolledBackInstantInfo(timeline);
     final List<Comparable[]> rows = timeline.getInstantsAsStream().map(instant 
-> {
-      int numColumns = showRollbackInfo ? 7 : 6;
-      Comparable[] row = new Comparable[numColumns];
+      Comparable[] row = new Comparable[6];
       String instantTimestamp = instant.getTimestamp();
+      String rollbackInfoString = showRollbackInfo
+          ? getRollbackInfoString(Option.of(instant), timeline, 
rollbackInfoMap) : "";
+
       row[0] = instantTimestamp;
-      row[1] = instant.getAction();
+      row[1] = instant.getAction() + rollbackInfoString;
       row[2] = instant.getState();
-      if (showRollbackInfo) {
-        if 
(HoodieTimeline.ROLLBACK_ACTION.equalsIgnoreCase(instant.getAction())) {
-          row[3] = "Rolls back\n" + getInstantToRollback(timeline, instant);
-        } else {
-          if (rollbackInfo.containsKey(instantTimestamp)) {
-            row[3] = "Rolled back by\n" + String.join(",\n", 
rollbackInfo.get(instantTimestamp));
-          } else {
-            row[3] = "-";
-          }
-        }
-      }
-      row[numColumns - 3] = getFormattedDate(
+      row[3] = getFormattedDate(
           instantTimestamp, HoodieInstant.State.REQUESTED, instantInfoMap, 
showTimeSeconds);
-      row[numColumns - 2] = getFormattedDate(
+      row[4] = getFormattedDate(
           instantTimestamp, HoodieInstant.State.INFLIGHT, instantInfoMap, 
showTimeSeconds);
-      row[numColumns - 1] = getFormattedDate(
+      row[5] = getFormattedDate(
           instantTimestamp, HoodieInstant.State.COMPLETED, instantInfoMap, 
showTimeSeconds);
       return row;
     }).collect(Collectors.toList());
     TableHeader header = new TableHeader()
         .addTableHeaderField(HoodieTableHeaderFields.HEADER_INSTANT)
         .addTableHeaderField(HoodieTableHeaderFields.HEADER_ACTION)
-        .addTableHeaderField(HoodieTableHeaderFields.HEADER_STATE);
-    if (showRollbackInfo) {
-      header.addTableHeaderField(HoodieTableHeaderFields.HEADER_ROLLBACK_INFO);
-    }
-    header.addTableHeaderField(HoodieTableHeaderFields.HEADER_REQUESTED_TIME)
+        .addTableHeaderField(HoodieTableHeaderFields.HEADER_STATE)
+        .addTableHeaderField(HoodieTableHeaderFields.HEADER_REQUESTED_TIME)
         .addTableHeaderField(HoodieTableHeaderFields.HEADER_INFLIGHT_TIME)
         .addTableHeaderField(HoodieTableHeaderFields.HEADER_COMPLETED_TIME);
     return HoodiePrintHelper.print(
@@ -259,52 +247,42 @@ public class TimelineCommand {
     instantTimeSet.addAll(mtInstantInfoMap.keySet());
     List<String> instantTimeList = instantTimeSet.stream()
         .sorted(new 
HoodieInstantTimeComparator()).collect(Collectors.toList());
-    Map<String, List<String>> dtRollbackInfo = 
getRolledBackInstantInfo(dtTimeline);
+    Map<String, List<String>> dtRollbackInfoMap = 
getRolledBackInstantInfo(dtTimeline);
+    Map<String, List<String>> mtRollbackInfoMap = 
getRolledBackInstantInfo(mtTimeline);
 
     final List<Comparable[]> rows = 
instantTimeList.stream().map(instantTimestamp -> {
-      int numColumns = showRollbackInfo ? 12 : 11;
       Option<HoodieInstant> dtInstant = getInstant(dtTimeline, 
instantTimestamp);
       Option<HoodieInstant> mtInstant = getInstant(mtTimeline, 
instantTimestamp);
-      Comparable[] row = new Comparable[numColumns];
+      Comparable[] row = new Comparable[11];
       row[0] = instantTimestamp;
-      row[1] = dtInstant.isPresent() ? dtInstant.get().getAction() : "-";
+      String dtRollbackInfoString = showRollbackInfo
+          ? getRollbackInfoString(dtInstant, dtTimeline, dtRollbackInfoMap) : 
"";
+      row[1] = (dtInstant.isPresent() ? dtInstant.get().getAction() : "-") + 
dtRollbackInfoString;
       row[2] = dtInstant.isPresent() ? dtInstant.get().getState() : "-";
-      if (showRollbackInfo) {
-        if (dtInstant.isPresent()
-            && 
HoodieTimeline.ROLLBACK_ACTION.equalsIgnoreCase(dtInstant.get().getAction())) {
-          row[3] = "Rolls back\n" + getInstantToRollback(dtTimeline, 
dtInstant.get());
-        } else {
-          if (dtRollbackInfo.containsKey(instantTimestamp)) {
-            row[3] = "Rolled back by\n" + String.join(",\n", 
dtRollbackInfo.get(instantTimestamp));
-          } else {
-            row[3] = "-";
-          }
-        }
-      }
-      row[numColumns - 8] = getFormattedDate(
+      row[3] = getFormattedDate(
           instantTimestamp, HoodieInstant.State.REQUESTED, dtInstantInfoMap, 
showTimeSeconds);
-      row[numColumns - 7] = getFormattedDate(
+      row[4] = getFormattedDate(
           instantTimestamp, HoodieInstant.State.INFLIGHT, dtInstantInfoMap, 
showTimeSeconds);
-      row[numColumns - 6] = getFormattedDate(
+      row[5] = getFormattedDate(
           instantTimestamp, HoodieInstant.State.COMPLETED, dtInstantInfoMap, 
showTimeSeconds);
-      row[numColumns - 5] = mtInstant.isPresent() ? 
mtInstant.get().getAction() : "-";
-      row[numColumns - 4] = mtInstant.isPresent() ? mtInstant.get().getState() 
: "-";
-      row[numColumns - 3] = getFormattedDate(
+
+      String mtRollbackInfoString = showRollbackInfo
+          ? getRollbackInfoString(mtInstant, mtTimeline, mtRollbackInfoMap) : 
"";
+      row[6] = (mtInstant.isPresent() ? mtInstant.get().getAction() : "-") + 
mtRollbackInfoString;
+      row[7] = mtInstant.isPresent() ? mtInstant.get().getState() : "-";
+      row[8] = getFormattedDate(
           instantTimestamp, HoodieInstant.State.REQUESTED, mtInstantInfoMap, 
showTimeSeconds);
-      row[numColumns - 2] = getFormattedDate(
+      row[9] = getFormattedDate(
           instantTimestamp, HoodieInstant.State.INFLIGHT, mtInstantInfoMap, 
showTimeSeconds);
-      row[numColumns - 1] = getFormattedDate(
+      row[10] = getFormattedDate(
           instantTimestamp, HoodieInstant.State.COMPLETED, mtInstantInfoMap, 
showTimeSeconds);
       return row;
     }).collect(Collectors.toList());
     TableHeader header = new TableHeader()
         .addTableHeaderField(HoodieTableHeaderFields.HEADER_INSTANT)
         .addTableHeaderField(HoodieTableHeaderFields.HEADER_ACTION)
-        .addTableHeaderField(HoodieTableHeaderFields.HEADER_STATE);
-    if (showRollbackInfo) {
-      header.addTableHeaderField(HoodieTableHeaderFields.HEADER_ROLLBACK_INFO);
-    }
-    header.addTableHeaderField(HoodieTableHeaderFields.HEADER_REQUESTED_TIME)
+        .addTableHeaderField(HoodieTableHeaderFields.HEADER_STATE)
+        .addTableHeaderField(HoodieTableHeaderFields.HEADER_REQUESTED_TIME)
         .addTableHeaderField(HoodieTableHeaderFields.HEADER_INFLIGHT_TIME)
         .addTableHeaderField(HoodieTableHeaderFields.HEADER_COMPLETED_TIME)
         .addTableHeaderField(HoodieTableHeaderFields.HEADER_MT_ACTION)
@@ -370,6 +348,23 @@ public class TimelineCommand {
     return rollbackInfoMap;
   }
 
+  private String getRollbackInfoString(Option<HoodieInstant> instant,
+                                       HoodieTimeline timeline,
+                                       Map<String, List<String>> 
rollbackInfoMap) {
+    String rollbackInfoString = "";
+    if (instant.isPresent()) {
+      if 
(HoodieTimeline.ROLLBACK_ACTION.equalsIgnoreCase(instant.get().getAction())) {
+        rollbackInfoString = "\nRolls back\n" + getInstantToRollback(timeline, 
instant.get());
+      } else {
+        String instantTimestamp = instant.get().getTimestamp();
+        if (rollbackInfoMap.containsKey(instantTimestamp)) {
+          rollbackInfoString = "\nRolled back by\n" + String.join(",\n", 
rollbackInfoMap.get(instantTimestamp));
+        }
+      }
+    }
+    return rollbackInfoString;
+  }
+
   static class HoodieInstantWithModTime extends HoodieInstant {
 
     private final long modificationTimeMs;

Reply via email to