This is an automated email from the ASF dual-hosted git repository.
sumitagrawal 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 20c4cfa349 HDDS-11386. Multithreading bug in ContainerBalancerTask
(#7339)
20c4cfa349 is described below
commit 20c4cfa349bce090f00251eda0ec149d93e977bd
Author: Sarveksha Yeshavantha Raju
<[email protected]>
AuthorDate: Mon Nov 25 12:26:24 2024 +0530
HDDS-11386. Multithreading bug in ContainerBalancerTask (#7339)
---
.../balancer/AbstractFindTargetGreedy.java | 4 +-
.../container/balancer/ContainerBalancerTask.java | 76 +++++++++++-----------
.../scm/container/balancer/FindSourceGreedy.java | 4 +-
3 files changed, 41 insertions(+), 43 deletions(-)
diff --git
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/AbstractFindTargetGreedy.java
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/AbstractFindTargetGreedy.java
index dd2d1c5789..88657047a0 100644
---
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/AbstractFindTargetGreedy.java
+++
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/AbstractFindTargetGreedy.java
@@ -32,11 +32,11 @@ import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.slf4j.Logger;
import java.util.Collection;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
import java.util.stream.Collectors;
/**
@@ -56,7 +56,7 @@ public abstract class AbstractFindTargetGreedy implements
FindTargetStrategy {
ContainerManager containerManager,
PlacementPolicyValidateProxy placementPolicyValidateProxy,
NodeManager nodeManager) {
- sizeEnteringNode = new HashMap<>();
+ sizeEnteringNode = new ConcurrentHashMap<>();
this.containerManager = containerManager;
this.placementPolicyValidateProxy = placementPolicyValidateProxy;
this.nodeManager = nodeManager;
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 c3b76dc449..2c113e8e6a 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
@@ -51,10 +51,11 @@ import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
-import java.util.Objects;
+import java.util.Queue;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
@@ -117,7 +118,7 @@ public class ContainerBalancerTask implements Runnable {
private IterationResult iterationResult;
private int nextIterationIndex;
private boolean delayStart;
- private List<ContainerBalancerTaskIterationStatusInfo> iterationsStatistic;
+ private Queue<ContainerBalancerTaskIterationStatusInfo> iterationsStatistic;
/**
* Constructs ContainerBalancerTask with the specified arguments.
@@ -166,7 +167,7 @@ public class ContainerBalancerTask implements Runnable {
findTargetStrategy = new FindTargetGreedyByUsageInfo(containerManager,
placementPolicyValidateProxy, nodeManager);
}
- this.iterationsStatistic = new ArrayList<>();
+ this.iterationsStatistic = new ConcurrentLinkedQueue<>();
}
/**
@@ -308,43 +309,42 @@ public class ContainerBalancerTask implements Runnable {
private void saveIterationStatistic(Integer iterationNumber, IterationResult
iR) {
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
- )
- )
+ iterationNumber,
+ iR.name(),
+ getSizeScheduledForMoveInLatestIteration() / OzoneConsts.GB,
+ metrics.getDataSizeMovedGBInLatestIteration(),
+ metrics.getNumContainerMovesScheduledInLatestIteration(),
+ metrics.getNumContainerMovesCompletedInLatestIteration(),
+ metrics.getNumContainerMovesFailedInLatestIteration(),
+ metrics.getNumContainerMovesTimeoutInLatestIteration(),
+ findTargetStrategy.getSizeEnteringNodes()
+ .entrySet()
+ .stream()
+ .filter(datanodeDetailsLongEntry ->
datanodeDetailsLongEntry.getValue() > 0)
+ .collect(
+ Collectors.toMap(
+ entry -> entry.getKey().getUuid(),
+ entry -> entry.getValue() / OzoneConsts.GB
+ )
+ ),
+ findSourceStrategy.getSizeLeavingNodes()
+ .entrySet()
+ .stream()
+ .filter(datanodeDetailsLongEntry ->
datanodeDetailsLongEntry.getValue() > 0)
+ .collect(
+ Collectors.toMap(
+ entry -> entry.getKey().getUuid(),
+ entry -> entry.getValue() / OzoneConsts.GB
+ )
+ )
);
- iterationsStatistic.add(iterationStatistic);
+ iterationsStatistic.offer(iterationStatistic);
}
public List<ContainerBalancerTaskIterationStatusInfo>
getCurrentIterationsStatistic() {
+ List<ContainerBalancerTaskIterationStatusInfo> resultList = new
ArrayList<>(iterationsStatistic);
- int lastIterationNumber = iterationsStatistic.stream()
+ int lastIterationNumber = resultList.stream()
.mapToInt(ContainerBalancerTaskIterationStatusInfo::getIterationNumber)
.max()
.orElse(0);
@@ -361,9 +361,9 @@ public class ContainerBalancerTask implements Runnable {
findTargetStrategy.getSizeEnteringNodes()
.entrySet()
.stream()
- .filter(Objects::nonNull)
.filter(datanodeDetailsLongEntry ->
datanodeDetailsLongEntry.getValue() > 0)
- .collect(Collectors.toMap(
+ .collect(
+ Collectors.toMap(
entry -> entry.getKey().getUuid(),
entry -> entry.getValue() / OzoneConsts.GB
)
@@ -371,7 +371,6 @@ public class ContainerBalancerTask implements Runnable {
findSourceStrategy.getSizeLeavingNodes()
.entrySet()
.stream()
- .filter(Objects::nonNull)
.filter(datanodeDetailsLongEntry ->
datanodeDetailsLongEntry.getValue() > 0)
.collect(
Collectors.toMap(
@@ -380,7 +379,6 @@ public class ContainerBalancerTask implements Runnable {
)
)
);
- List<ContainerBalancerTaskIterationStatusInfo> resultList = new
ArrayList<>(iterationsStatistic);
resultList.add(currentIterationStatistic);
return resultList;
}
diff --git
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindSourceGreedy.java
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindSourceGreedy.java
index 435cc9859a..57cc8b32b9 100644
---
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindSourceGreedy.java
+++
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindSourceGreedy.java
@@ -26,11 +26,11 @@ import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collection;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.PriorityQueue;
import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
/**
* The selection criteria for selecting source datanodes , the containers of
@@ -46,7 +46,7 @@ public class FindSourceGreedy implements FindSourceStrategy {
private Double lowerLimit;
FindSourceGreedy(NodeManager nodeManager) {
- sizeLeavingNode = new HashMap<>();
+ sizeLeavingNode = new ConcurrentHashMap<>();
potentialSources = new PriorityQueue<>((a, b) -> {
double currentUsageOfA = a.calculateUtilization(
-sizeLeavingNode.get(a.getDatanodeDetails()));
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]