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

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


The following commit(s) were added to refs/heads/master by this push:
     new adb385f02a HDDS-13002. Use DatanodeID in ContainerBalancerTask (#8421)
adb385f02a is described below

commit adb385f02a7dc2b944dced63f1019ed1d8540459
Author: Nandakumar Vadivelu <[email protected]>
AuthorDate: Sun May 11 11:10:48 2025 +0530

    HDDS-13002. Use DatanodeID in ContainerBalancerTask (#8421)
---
 .../scm/container/balancer/ContainerBalancerTask.java  | 18 +++++++++---------
 .../ContainerBalancerTaskIterationStatusInfo.java      |  8 ++++----
 .../hdds/scm/container/balancer/DataMoveInfo.java      | 14 +++++++-------
 .../balancer/TestContainerBalancerStatusInfo.java      |  4 ++--
 4 files changed, 22 insertions(+), 22 deletions(-)

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
index 98e64b6c00..9b0e16fc3f 100644
--- 
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
@@ -37,7 +37,6 @@
 import java.util.Objects;
 import java.util.Queue;
 import java.util.Set;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentLinkedQueue;
@@ -49,6 +48,7 @@
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.DatanodeID;
 import org.apache.hadoop.hdds.scm.PlacementPolicyValidateProxy;
 import org.apache.hadoop.hdds.scm.container.ContainerID;
 import org.apache.hadoop.hdds.scm.container.ContainerInfo;
@@ -329,9 +329,9 @@ private ContainerBalancerTaskIterationStatusInfo 
getIterationStatistic(Integer i
                                                                          
IterationResult currentIterationResult,
                                                                          long 
iterationDuration) {
     String currentIterationResultName = currentIterationResult == null ? null 
: currentIterationResult.name();
-    Map<UUID, Long> sizeEnteringDataToNodes =
+    Map<DatanodeID, Long> sizeEnteringDataToNodes =
         convertToNodeIdToTrafficMap(findTargetStrategy.getSizeEnteringNodes());
-    Map<UUID, Long> sizeLeavingDataFromNodes =
+    Map<DatanodeID, Long> sizeLeavingDataFromNodes =
         convertToNodeIdToTrafficMap(findSourceStrategy.getSizeLeavingNodes());
     IterationInfo iterationInfo = new IterationInfo(
         iterationNumber,
@@ -345,8 +345,8 @@ private ContainerBalancerTaskIterationStatusInfo 
getIterationStatistic(Integer i
     return new ContainerBalancerTaskIterationStatusInfo(iterationInfo, 
containerMoveInfo, dataMoveInfo);
   }
 
-  private DataMoveInfo getDataMoveInfo(String currentIterationResultName, 
Map<UUID, Long> sizeEnteringDataToNodes,
-                                       Map<UUID, Long> 
sizeLeavingDataFromNodes) {
+  private DataMoveInfo getDataMoveInfo(String currentIterationResultName, 
Map<DatanodeID, Long> sizeEnteringDataToNodes,
+                                       Map<DatanodeID, Long> 
sizeLeavingDataFromNodes) {
     if (currentIterationResultName == null) {
       // For unfinished iteration
       return new DataMoveInfo(
@@ -366,7 +366,7 @@ private DataMoveInfo getDataMoveInfo(String 
currentIterationResultName, Map<UUID
     }
   }
 
-  private Map<UUID, Long> convertToNodeIdToTrafficMap(Map<DatanodeDetails, 
Long> nodeTrafficMap) {
+  private Map<DatanodeID, Long> 
convertToNodeIdToTrafficMap(Map<DatanodeDetails, Long> nodeTrafficMap) {
     return nodeTrafficMap
         .entrySet()
         .stream()
@@ -374,7 +374,7 @@ private Map<UUID, Long> 
convertToNodeIdToTrafficMap(Map<DatanodeDetails, Long> n
         .filter(datanodeDetailsLongEntry -> 
datanodeDetailsLongEntry.getValue() > 0)
         .collect(
             Collectors.toMap(
-                entry -> entry.getKey().getUuid(),
+                entry -> entry.getKey().getID(),
                 Map.Entry::getValue
             )
         );
@@ -556,13 +556,13 @@ private boolean initializeIteration() {
       overUtilizedNodes.forEach(entry -> {
         LOG.debug("Datanode {} {} is Over-Utilized.",
             entry.getDatanodeDetails().getHostName(),
-            entry.getDatanodeDetails().getUuid());
+            entry.getDatanodeDetails().getID());
       });
 
       underUtilizedNodes.forEach(entry -> {
         LOG.debug("Datanode {} {} is Under-Utilized.",
             entry.getDatanodeDetails().getHostName(),
-            entry.getDatanodeDetails().getUuid());
+            entry.getDatanodeDetails().getID());
       });
     }
 
diff --git 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java
 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java
index 6e1085429c..f16f3cfe7e 100644
--- 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java
+++ 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java
@@ -20,8 +20,8 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
-import java.util.UUID;
 import java.util.stream.Collectors;
+import org.apache.hadoop.hdds.protocol.DatanodeID;
 import 
org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos;
 
 /**
@@ -110,7 +110,7 @@ public long getContainerMovesTimeout() {
    * Get a map of the node IDs and the corresponding data sizes moved to each 
node.
    * @return nodeId to size entering from node map
    */
-  public Map<UUID, Long> getSizeEnteringNodes() {
+  public Map<DatanodeID, Long> getSizeEnteringNodes() {
     return dataMoveInfo.getSizeEnteringNodes();
   }
 
@@ -118,7 +118,7 @@ public Map<UUID, Long> getSizeEnteringNodes() {
    * Get a map of the node IDs and the corresponding data sizes moved from 
each node.
    * @return nodeId to size leaving from node map
    */
-  public Map<UUID, Long> getSizeLeavingNodes() {
+  public Map<DatanodeID, Long> getSizeLeavingNodes() {
     return dataMoveInfo.getSizeLeavingNodes();
   }
 
@@ -160,7 +160,7 @@ public 
StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStat
    * @return node transfer info proto representation
    */
   private List<StorageContainerLocationProtocolProtos.NodeTransferInfoProto> 
mapToProtoNodeTransferInfo(
-      Map<UUID, Long> nodes
+      Map<DatanodeID, Long> nodes
   ) {
     return nodes.entrySet()
         .stream()
diff --git 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/DataMoveInfo.java
 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/DataMoveInfo.java
index 455e1310a8..a090ad465f 100644
--- 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/DataMoveInfo.java
+++ 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/DataMoveInfo.java
@@ -18,7 +18,7 @@
 package org.apache.hadoop.hdds.scm.container.balancer;
 
 import java.util.Map;
-import java.util.UUID;
+import org.apache.hadoop.hdds.protocol.DatanodeID;
 
 /**
  * Information about the process of moving data.
@@ -26,14 +26,14 @@
 public class DataMoveInfo {
   private final long sizeScheduledForMove;
   private final long dataSizeMoved;
-  private final Map<UUID, Long> sizeEnteringNodes;
-  private final Map<UUID, Long> sizeLeavingNodes;
+  private final Map<DatanodeID, Long> sizeEnteringNodes;
+  private final Map<DatanodeID, Long> sizeLeavingNodes;
 
   public DataMoveInfo(
       long sizeScheduledForMove,
       long dataSizeMoved,
-      Map<UUID, Long> sizeEnteringNodes,
-      Map<UUID, Long> sizeLeavingNodes) {
+      Map<DatanodeID, Long> sizeEnteringNodes,
+      Map<DatanodeID, Long> sizeLeavingNodes) {
     this.sizeScheduledForMove = sizeScheduledForMove;
     this.dataSizeMoved = dataSizeMoved;
     this.sizeEnteringNodes = sizeEnteringNodes;
@@ -48,11 +48,11 @@ public long getDataSizeMoved() {
     return dataSizeMoved;
   }
 
-  public Map<UUID, Long> getSizeEnteringNodes() {
+  public Map<DatanodeID, Long> getSizeEnteringNodes() {
     return sizeEnteringNodes;
   }
 
-  public Map<UUID, Long> getSizeLeavingNodes() {
+  public Map<DatanodeID, Long> getSizeLeavingNodes() {
     return sizeLeavingNodes;
   }
 }
diff --git 
a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java
 
b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java
index 3f8bae946b..bb6f0c4627 100644
--- 
a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java
+++ 
b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java
@@ -25,10 +25,10 @@
 
 import java.util.List;
 import java.util.Map;
-import java.util.UUID;
 import org.apache.commons.math3.util.ArithmeticUtils;
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeID;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.ozone.test.LambdaTestUtils;
@@ -229,7 +229,7 @@ private void verifyStartedEmptyIteration(
     assertTrue(iteration.getSizeLeavingNodes().isEmpty());
   }
 
-  private static Long getTotalMovedData(Map<UUID, Long> iteration) {
+  private static Long getTotalMovedData(Map<DatanodeID, Long> iteration) {
     return iteration.values().stream().reduce(0L, 
ArithmeticUtils::addAndCheck);
   }
 


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

Reply via email to