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


##########
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);
+    }

Review Comment:
   I think it is not critical. Maybe makes sense. But it leads to another 
output behaviour. Perhaps it would be better to move it to a separate task? Not 
to blow this 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.

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