siddhantsangwan commented on code in PR #7139:
URL: https://github.com/apache/ozone/pull/7139#discussion_r1856616197


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java:
##########
@@ -306,83 +323,123 @@ private void balance() {
     tryStopWithSaveConfiguration("Completed all iterations.");
   }
 
-  private void saveIterationStatistic(Integer iterationNumber, IterationResult 
iR) {
+  private void saveIterationStatistic(Integer iterationNumber, IterationResult 
currentIterationResult) {
+    long iterationDuration = now().toEpochSecond() - 
currentIterationStarted.toEpochSecond();
+    Map<UUID, Long> sizeEnteringDataToNodes =
+        convertToNodeIdToTrafficMap(findTargetStrategy.getSizeEnteringNodes());
+    Map<UUID, Long> sizeLeavingDataFromNodes =
+        convertToNodeIdToTrafficMap(findSourceStrategy.getSizeLeavingNodes());
+    IterationInfo iterationInfo = new IterationInfo(
+        iterationNumber,
+        currentIterationResult.name(),
+        iterationDuration
+    );
+    ContainerMoveInfo containerMoveInfo = new ContainerMoveInfo(metrics);
+    DataMoveInfo dataMoveInfo = new DataMoveInfo(
+        getSizeScheduledForMoveInLatestIteration(),
+        metrics.getDataSizeMovedInLatestIteration(),
+        sizeEnteringDataToNodes,
+        sizeLeavingDataFromNodes
+    );
     ContainerBalancerTaskIterationStatusInfo iterationStatistic = new 
ContainerBalancerTaskIterationStatusInfo(
-            iterationNumber,
-            iR.name(),
-            getSizeScheduledForMoveInLatestIteration() / OzoneConsts.GB,
-            metrics.getDataSizeMovedGBInLatestIteration(),
-            metrics.getNumContainerMovesScheduledInLatestIteration(),
-            metrics.getNumContainerMovesCompletedInLatestIteration(),
-            metrics.getNumContainerMovesFailedInLatestIteration(),
-            metrics.getNumContainerMovesTimeoutInLatestIteration(),
-            findTargetStrategy.getSizeEnteringNodes()
-                    .entrySet()
-                    .stream()
-                    .filter(Objects::nonNull)
-                    .filter(datanodeDetailsLongEntry -> 
datanodeDetailsLongEntry.getValue() > 0)
-                    .collect(
-                            Collectors.toMap(
-                                    entry -> entry.getKey().getUuid(),
-                                    entry -> entry.getValue() / OzoneConsts.GB
-                            )
-                    ),
-            findSourceStrategy.getSizeLeavingNodes()
-                    .entrySet()
-                    .stream()
-                    .filter(Objects::nonNull)
-                    .filter(datanodeDetailsLongEntry -> 
datanodeDetailsLongEntry.getValue() > 0)
-                    .collect(
-                            Collectors.toMap(
-                                    entry -> entry.getKey().getUuid(),
-                                    entry -> entry.getValue() / OzoneConsts.GB
-                            )
-                    )
+        iterationInfo,
+        containerMoveInfo,
+        dataMoveInfo
     );
     iterationsStatistic.add(iterationStatistic);
   }
 
+  private Map<UUID, Long> convertToNodeIdToTrafficMap(Map<DatanodeDetails, 
Long> nodeTrafficMap) {
+    return nodeTrafficMap
+        .entrySet()
+        .stream()
+        .filter(Objects::nonNull)
+        .filter(datanodeDetailsLongEntry -> 
datanodeDetailsLongEntry.getValue() > 0)
+        .collect(
+            Collectors.toMap(
+                entry -> entry.getKey().getUuid(),
+                Map.Entry::getValue
+            )
+        );
+  }
+
+  /**
+   * Get current iteration statistics.
+   * @return current iteration statistic
+   */
   public List<ContainerBalancerTaskIterationStatusInfo> 
getCurrentIterationsStatistic() {
+    List<ContainerBalancerTaskIterationStatusInfo> resultList = new 
ArrayList<>(iterationsStatistic);
+    ContainerBalancerTaskIterationStatusInfo currentIterationStatistic = 
createCurrentIterationStatistic();
+    resultList.add(currentIterationStatistic);
+    return resultList;
+  }
+
+  private ContainerBalancerTaskIterationStatusInfo 
createCurrentIterationStatistic() {
+    int lastIterationNumber = iterationsStatistic.isEmpty()
+        ? 0
+        : iterationsStatistic.get(iterationsStatistic.size() - 
1).getIterationNumber();
+    long iterationDuration = getCurrentIterationDuration();
 
-    int lastIterationNumber = iterationsStatistic.stream()
-        .mapToInt(ContainerBalancerTaskIterationStatusInfo::getIterationNumber)
-        .max()
-        .orElse(0);
+    if (isCurrentIterationInProgress.get()) {
+      return getFilledCurrentIterationStatistic(lastIterationNumber, 
iterationDuration);
+    } else {
+      return getEmptyCurrentIterationStatistic(iterationDuration);
+    }
+  }
 
-    ContainerBalancerTaskIterationStatusInfo currentIterationStatistic = new 
ContainerBalancerTaskIterationStatusInfo(
+  private static ContainerBalancerTaskIterationStatusInfo 
getEmptyCurrentIterationStatistic(
+      long iterationDuration) {
+    ContainerMoveInfo containerMoveInfo = new ContainerMoveInfo(0, 0, 0, 0);
+    DataMoveInfo dataMoveInfo = new DataMoveInfo(
+        0,
+        0,
+        emptyMap(),
+        emptyMap()
+    );
+    IterationInfo iterationInfo = new IterationInfo(
+        0,
+        null,
+        iterationDuration
+    );
+    return new ContainerBalancerTaskIterationStatusInfo(
+        iterationInfo,
+        containerMoveInfo,
+        dataMoveInfo
+    );
+  }
+
+  private ContainerBalancerTaskIterationStatusInfo 
getFilledCurrentIterationStatistic(int lastIterationNumber,

Review Comment:
   It's doable with a little refactoring. I hope we can reuse it so that we can 
keep things simple and increase maintainability.
   
   We can start from something like this:
   ```
   Subject: [PATCH] reuse
   ---
   Index: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java
   IDEA additional info:
   Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
   <+>UTF-8
   ===================================================================
   diff --git 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java
 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java
   --- 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java
  (revision 6acad2b4fac6cf58e6d1c930e83bc86c14ccec10)
   +++ 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java
  (date 1732537488628)
   @@ -264,7 +264,7 @@
          }
    
          IterationResult iR = doIteration();
   -      saveIterationStatistic(i + 1, iR);
   +      iterationsStatistic.offer(getIterationStatistic(i + 1, iR.name(), 
getCurrentIterationDuration()));
    
          isCurrentIterationInProgress.compareAndSet(true, false);
    
   @@ -314,15 +314,16 @@
        tryStopWithSaveConfiguration("Completed all iterations.");
      }
    
   -  private void saveIterationStatistic(Integer iterationNumber, 
IterationResult currentIterationResult) {
   -    long iterationDuration = now().toEpochSecond() - 
currentIterationStarted.toEpochSecond();
   +  private ContainerBalancerTaskIterationStatusInfo 
getIterationStatistic(Integer iterationNumber,
   +                                                                            
   String currentIterationResult,
   +                                                                            
   long iterationDuration) {
        Map<UUID, Long> sizeEnteringDataToNodes =
            
convertToNodeIdToTrafficMap(findTargetStrategy.getSizeEnteringNodes());
        Map<UUID, Long> sizeLeavingDataFromNodes =
            
convertToNodeIdToTrafficMap(findSourceStrategy.getSizeLeavingNodes());
        IterationInfo iterationInfo = new IterationInfo(
            iterationNumber,
   -        currentIterationResult.name(),
   +        currentIterationResult,
            iterationDuration
        );
        ContainerMoveInfo containerMoveInfo = new ContainerMoveInfo(metrics);
   @@ -332,12 +333,8 @@
            sizeEnteringDataToNodes,
            sizeLeavingDataFromNodes
        );
   -    ContainerBalancerTaskIterationStatusInfo iterationStatistic = new 
ContainerBalancerTaskIterationStatusInfo(
   -        iterationInfo,
   -        containerMoveInfo,
   -        dataMoveInfo
   -    );
   -    iterationsStatistic.offer(iterationStatistic);
   +    //    iterationsStatistic.offer(iterationStatistic);
   +    return new ContainerBalancerTaskIterationStatusInfo(iterationInfo, 
containerMoveInfo, dataMoveInfo);
      }
    
      private Map<UUID, Long> convertToNodeIdToTrafficMap(Map<DatanodeDetails, 
Long> nodeTrafficMap) {
   @@ -375,7 +372,7 @@
        long iterationDuration = getCurrentIterationDuration();
    
        if (isCurrentIterationInProgress.get()) {
   -      return getFilledCurrentIterationStatistic(lastIterationNumber, 
iterationDuration);
   +      return getIterationStatistic(lastIterationNumber + 1, null, 
iterationDuration);
        } else {
          return getEmptyCurrentIterationStatistic(iterationDuration);
        }
   
   ```
   One test is failing with this change - I have not checked why, but I think 
this patch can be a good starting point and help you.



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