siddhantsangwan commented on a change in pull request #2441:
URL: https://github.com/apache/ozone/pull/2441#discussion_r678258702
##########
File path:
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java
##########
@@ -226,78 +256,188 @@ private boolean initializeIteration() {
}
Collections.reverse(underUtilizedNodes);
- long countDatanodesBalanced = 0;
- // count number of nodes that were balanced in previous iteration
- for (DatanodeUsageInfo node : unBalancedNodes) {
- if (!containsNode(overUtilizedNodes, node) &&
- !containsNode(underUtilizedNodes, node)) {
- countDatanodesBalanced += 1;
- }
- }
- // calculate total number of nodes that have been balanced so far
- countDatanodesBalanced =
- metrics.incrementDatanodesNumBalanced(countDatanodesBalanced);
-
unBalancedNodes = new ArrayList<>(
overUtilizedNodes.size() + underUtilizedNodes.size());
+ unBalancedNodes.addAll(overUtilizedNodes);
+ unBalancedNodes.addAll(underUtilizedNodes);
- if (countDatanodesBalanced + countDatanodesToBalance >
- maxDatanodesToBalance) {
- LOG.info("Approaching Max Datanodes To Balance limit in Container " +
- "Balancer. Stopping Balancer.");
+ if (unBalancedNodes.isEmpty()) {
+ LOG.info("Did not find any unbalanced Datanodes.");
return false;
- } else {
- unBalancedNodes.addAll(overUtilizedNodes);
- unBalancedNodes.addAll(underUtilizedNodes);
-
- //for now, we just sleep to simulate the execution of balancer
- //this if for acceptance test now. modify this later when balancer
- //if fully completed
- try {
- Thread.sleep(50);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- }
- /////////////////////////////
+ }
+
+ LOG.info("Container Balancer has identified Datanodes that need to be" +
+ " balanced.");
+
+ selectionCriteria = new ContainerBalancerSelectionCriteria(config,
+ nodeManager, replicationManager, containerManager);
+ sourceToTargetMap = new HashMap<>(overUtilizedNodes.size() +
+ withinThresholdUtilizedNodes.size());
+
+ // initialize maps to track how much size is leaving and entering datanodes
+ sizeLeavingNode = new HashMap<>(overUtilizedNodes.size() +
+ withinThresholdUtilizedNodes.size());
+ overUtilizedNodes.forEach(datanodeUsageInfo -> sizeLeavingNode
+ .put(datanodeUsageInfo.getDatanodeDetails(), 0L));
+ withinThresholdUtilizedNodes.forEach(datanodeUsageInfo -> sizeLeavingNode
+ .put(datanodeUsageInfo.getDatanodeDetails(), 0L));
+
+ sizeEnteringNode = new HashMap<>(underUtilizedNodes.size() +
+ withinThresholdUtilizedNodes.size());
+ underUtilizedNodes.forEach(datanodeUsageInfo -> sizeEnteringNode
+ .put(datanodeUsageInfo.getDatanodeDetails(), 0L));
+ withinThresholdUtilizedNodes.forEach(datanodeUsageInfo -> sizeEnteringNode
+ .put(datanodeUsageInfo.getDatanodeDetails(), 0L));
- if (unBalancedNodes.isEmpty()) {
- LOG.info("Did not find any unbalanced Datanodes.");
+ return true;
+ }
+
+ private boolean doIteration() {
+ List<DatanodeDetails> potentialTargets = getPotentialTargets();
+ Set<DatanodeDetails> selectedTargets =
+ new HashSet<>(potentialTargets.size());
+
+ // match each overUtilized node with a target
+ for (DatanodeUsageInfo datanodeUsageInfo : overUtilizedNodes) {
+ DatanodeDetails source = datanodeUsageInfo.getDatanodeDetails();
+ if (!checkConditionsForBalancing()) {
+ LOG.info("Conditions for balancing failed. Stopping Container " +
+ "Balancer...");
return false;
- } else {
- LOG.info("Container Balancer has identified Datanodes that need to be"
+
- " balanced.");
+ }
+
+ ContainerMoveSelection moveSelection =
+ matchSourceWithTarget(source, potentialTargets);
+
+ if (moveSelection != null) {
+ LOG.info("ContainerBalancer is trying to move container {} from " +
+ "source datanode {} to target datanode {}",
+ moveSelection.getContainerID().toString(), source.getUuidString(),
+ moveSelection.getTargetNode().getUuidString());
+
+ // move container
+ // if move successful, do the following
+ potentialTargets = updateTargetsAndSelectionCriteria(potentialTargets,
+ selectedTargets, moveSelection, source);
+ }
+ }
+
+ // if not all underUtilized nodes have been selected, try to match
+ // withinThresholdUtilized nodes with underUtilized nodes
+ if (selectedTargets.size() < underUtilizedNodes.size()) {
+ potentialTargets.removeAll(selectedTargets);
+ Collections.reverse(withinThresholdUtilizedNodes);
+
+ for (DatanodeUsageInfo datanodeUsageInfo : withinThresholdUtilizedNodes)
{
+ DatanodeDetails source = datanodeUsageInfo.getDatanodeDetails();
+ if (!checkConditionsForBalancing()) {
+ LOG.info("Conditions for balancing failed. Stopping Container " +
+ "Balancer...");
+ return false;
+ }
+
+ ContainerMoveSelection moveSelection = matchSourceWithTarget(source,
+ potentialTargets);
+
+ if (moveSelection != null) {
+ LOG.info("ContainerBalancer is trying to move container {} from " +
+ "source datanode {} to target datanode {}",
+ moveSelection.getContainerID().toString(),
+ source.getUuidString(),
+ moveSelection.getTargetNode().getUuidString());
+
+ // move container
+ // if move successful, do the following
+ potentialTargets =
updateTargetsAndSelectionCriteria(potentialTargets,
+ selectedTargets, moveSelection, source);
+ }
}
}
return true;
}
/**
- * Performs binary search to determine if the specified listToSearch
- * contains the specified node.
- *
- * @param listToSearch List of DatanodeUsageInfo to be searched.
- * @param node DatanodeUsageInfo to be searched for.
- * @return true if the specified node is present in listToSearch, otherwise
- * false.
+ * Match a source datanode with a target datanode and identify the container
+ * to move.
+ * @param potentialTargets List of potential targets to move container to
+ * @return ContainerMoveSelection containing the selected target and
container
*/
- private boolean containsNode(
- List<DatanodeUsageInfo> listToSearch, DatanodeUsageInfo node) {
- int index = 0;
- Comparator<DatanodeUsageInfo> comparator =
- DatanodeUsageInfo.getMostUsedByRemainingRatio();
- int size = listToSearch.size();
- if (size == 0) {
- return false;
+ private ContainerMoveSelection matchSourceWithTarget(DatanodeDetails source,
+ List<DatanodeDetails> potentialTargets) {
+
+ NavigableSet<ContainerID> candidateContainers =
+ selectionCriteria.getCandidateContainers(source);
+
+ if (candidateContainers.isEmpty()) {
+ LOG.info("ContainerBalancer could not find any candidate containers for"
+
+ " datanode {}", source.getUuidString());
+ return null;
+ }
+ LOG.info("ContainerBalancer is finding suitable target for source " +
+ "datanode {}", source.getUuidString());
+ ContainerMoveSelection moveSelection =
+ findTargetStrategy.findTargetForContainerMove(
+ source, potentialTargets, candidateContainers,
+ this::canSizeEnterTarget);
+
+ if (moveSelection == null) {
+ LOG.info("ContainerBalancer could not find a suitable target for " +
+ "source node {}.", source.getUuidString());
+ return null;
}
- if (comparator.compare(listToSearch.get(0),
- listToSearch.get(size - 1)) < 0) {
- index =
- Collections.binarySearch(listToSearch, node, comparator.reversed());
- } else {
- index = Collections.binarySearch(listToSearch, node, comparator);
+ LOG.info("ContainerBalancer matched source datanode {} with target " +
+ "datanode {} for container move.", source.getUuidString(),
+ moveSelection.getTargetNode().getUuidString());
+
+ return moveSelection;
+ }
+
+ /**
+ * Checks if limits maxDatanodesToBalance and maxSizeToMove have not been
hit.
+ * @return true if conditions pass and balancing can continue, else false
+ */
+ private boolean checkConditionsForBalancing() {
+ if (countDatanodesBalanced + 2 > maxDatanodesToBalance) {
+ LOG.info("Hit max datanodes to balance limit. {} datanodes have already"
+
+ " been balanced and the limit is {}.", countDatanodesBalanced,
+ maxDatanodesToBalance);
+ return false;
}
- return index >= 0 && listToSearch.get(index).equals(node);
+ if (totalSizeMoved + (long) ozoneConfiguration.getStorageSize(
Review comment:
Yes, if we check after selecting a target datanode and container. In
that case, some extra work would be performed if the check eventually fails.
--
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]