sajjad-moradi commented on code in PR #13803:
URL: https://github.com/apache/pinot/pull/13803#discussion_r1723989636


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java:
##########
@@ -130,7 +134,10 @@
 public class PinotLLCRealtimeSegmentManager {
 
   // simple field in Ideal State representing pause status for the table
+  // Deprecated in favour of PAUSE_STATUS
+  @Deprecated
   public static final String IS_TABLE_PAUSED = "isTablePaused";
+  public static final String PAUSE_STATUS = "pauseStatus";

Review Comment:
   To be consistent, let's rename to pauseState.



##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java:
##########
@@ -963,10 +970,29 @@ void updateIdealStateOnSegmentCompletion(String 
realtimeTableName, String commit
     }, RetryPolicies.exponentialBackoffRetryPolicy(10, 1000L, 1.2f));
   }
 
-  private boolean isTablePaused(IdealState idealState) {
+  public static boolean isTablePaused(IdealState idealState) {
+    PauseState pauseStatus = extractTablePauseStatus(idealState);
+    if (pauseStatus != null) {
+      return pauseStatus.isPaused();
+    }
+    // backward compatibility
+    // TODO : remove this handling after next release.
+    //  Expectation is that all table IS are migrated to the newer 
representation.
     return 
Boolean.parseBoolean(idealState.getRecord().getSimpleField(IS_TABLE_PAUSED));
   }
 
+  private static PauseState extractTablePauseStatus(IdealState idealState) {
+    String pauseStatusStr = 
idealState.getRecord().getSimpleField(PinotLLCRealtimeSegmentManager.PAUSE_STATUS);

Review Comment:
   Both in code and the comments.



##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java:
##########
@@ -963,10 +970,29 @@ void updateIdealStateOnSegmentCompletion(String 
realtimeTableName, String commit
     }, RetryPolicies.exponentialBackoffRetryPolicy(10, 1000L, 1.2f));
   }
 
-  private boolean isTablePaused(IdealState idealState) {
+  public static boolean isTablePaused(IdealState idealState) {
+    PauseState pauseStatus = extractTablePauseStatus(idealState);
+    if (pauseStatus != null) {
+      return pauseStatus.isPaused();
+    }
+    // backward compatibility
+    // TODO : remove this handling after next release.
+    //  Expectation is that all table IS are migrated to the newer 
representation.
     return 
Boolean.parseBoolean(idealState.getRecord().getSimpleField(IS_TABLE_PAUSED));
   }
 
+  private static PauseState extractTablePauseStatus(IdealState idealState) {
+    String pauseStatusStr = 
idealState.getRecord().getSimpleField(PinotLLCRealtimeSegmentManager.PAUSE_STATUS);

Review Comment:
   Ditto: pauseStateStr?
   Could you please update all instances of "pause status" to "pause state"?



##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java:
##########
@@ -963,10 +970,29 @@ void updateIdealStateOnSegmentCompletion(String 
realtimeTableName, String commit
     }, RetryPolicies.exponentialBackoffRetryPolicy(10, 1000L, 1.2f));
   }
 
-  private boolean isTablePaused(IdealState idealState) {
+  public static boolean isTablePaused(IdealState idealState) {
+    PauseState pauseStatus = extractTablePauseStatus(idealState);

Review Comment:
   nit: pauseState?



##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java:
##########
@@ -1719,11 +1755,17 @@ private Set<String> findConsumingSegments(IdealState 
idealState) {
    *   - the value of `isTablePaused` flag in ideal state
    *   - list of consuming segments
    */
-  public PauseStatus getPauseStatus(String tableNameWithType) {
+  public PauseStatusDetails getPauseStatus(String tableNameWithType) {
     IdealState idealState = getIdealState(tableNameWithType);
-    String isTablePausedStr = 
idealState.getRecord().getSimpleField(IS_TABLE_PAUSED);
     Set<String> consumingSegments = findConsumingSegments(idealState);
-    return new PauseStatus(Boolean.parseBoolean(isTablePausedStr), 
consumingSegments, null);
+    PauseState pauseStatus = extractTablePauseStatus(idealState);
+    if (pauseStatus != null) {
+      return new PauseStatusDetails(pauseStatus.isPaused(), consumingSegments, 
pauseStatus.getReasonCode(),
+          pauseStatus.getComment(), pauseStatus.getTimeInMillis());
+    }
+    String isTablePausedStr = 
idealState.getRecord().getSimpleField(IS_TABLE_PAUSED);
+    return new PauseStatusDetails(Boolean.parseBoolean(isTablePausedStr), 
consumingSegments,
+        PauseState.ReasonCode.ADMINISTRATIVE, null, new 
Timestamp(System.currentTimeMillis()).toString());

Review Comment:
   Let's use an empty timestamp for the old pauses (using isTablePaused flag) 
so it's not misleading. 



-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to