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
deleted file mode 100644
index 660452b2d8b0..000000000000
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/AbstractFindTargetGreedy.java
+++ /dev/null
@@ -1,281 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdds.scm.container.balancer;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
-import org.apache.hadoop.hdds.scm.PlacementPolicyValidateProxy;
-import org.apache.hadoop.hdds.scm.container.ContainerID;
-import org.apache.hadoop.hdds.scm.container.ContainerInfo;
-import org.apache.hadoop.hdds.scm.container.ContainerManager;
-import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
-import org.apache.hadoop.hdds.scm.container.ContainerReplica;
-import org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo;
-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.stream.Collectors;
-
-/**
- * Find a target for a source datanode with greedy strategy.
- */
-public abstract class AbstractFindTargetGreedy implements FindTargetStrategy {
- private Logger logger;
- private ContainerManager containerManager;
- private PlacementPolicyValidateProxy placementPolicyValidateProxy;
- private Map sizeEnteringNode;
- private NodeManager nodeManager;
- private ContainerBalancerConfiguration config;
- private Double upperLimit;
- private Collection potentialTargets;
-
- protected AbstractFindTargetGreedy(
- ContainerManager containerManager,
- PlacementPolicyValidateProxy placementPolicyValidateProxy,
- NodeManager nodeManager) {
- sizeEnteringNode = new HashMap<>();
- this.containerManager = containerManager;
- this.placementPolicyValidateProxy = placementPolicyValidateProxy;
- this.nodeManager = nodeManager;
- }
-
- protected void setLogger(Logger log) {
- logger = log;
- }
-
- protected void setPotentialTargets(Collection pt) {
- potentialTargets = pt;
- }
-
- private void setUpperLimit(Double upperLimit) {
- this.upperLimit = upperLimit;
- }
-
- protected int compareByUsage(DatanodeUsageInfo a, DatanodeUsageInfo b) {
- double currentUsageOfA = a.calculateUtilization(
- sizeEnteringNode.get(a.getDatanodeDetails()));
- double currentUsageOfB = b.calculateUtilization(
- sizeEnteringNode.get(b.getDatanodeDetails()));
- int ret = Double.compare(currentUsageOfA, currentUsageOfB);
- if (ret != 0) {
- return ret;
- }
- UUID uuidA = a.getDatanodeDetails().getUuid();
- UUID uuidB = b.getDatanodeDetails().getUuid();
- return uuidA.compareTo(uuidB);
- }
-
- private void setConfiguration(ContainerBalancerConfiguration conf) {
- config = conf;
- }
-
- /**
- * Find a {@link ContainerMoveSelection} consisting of a target and
- * container to move for a source datanode. Favours more under-utilized nodes.
- * @param source Datanode to find a target for
- * @param candidateContainers Set of candidate containers satisfying
- * selection criteria
- * {@link ContainerBalancerSelectionCriteria}
- * (DatanodeDetails, Long) method returns true if the size specified in the
- * second argument can enter the specified DatanodeDetails node
- * @return Found target and container
- */
- @Override
- public ContainerMoveSelection findTargetForContainerMove(
- DatanodeDetails source, Set candidateContainers) {
- sortTargetForSource(source);
- for (DatanodeUsageInfo targetInfo : potentialTargets) {
- DatanodeDetails target = targetInfo.getDatanodeDetails();
- for (ContainerID container : candidateContainers) {
- Set replicas;
- ContainerInfo containerInfo;
- try {
- replicas = containerManager.getContainerReplicas(container);
- containerInfo = containerManager.getContainer(container);
- } catch (ContainerNotFoundException e) {
- logger.warn("Could not get Container {} from Container Manager for " +
- "obtaining replicas in Container Balancer.", container, e);
- continue;
- }
-
- if (replicas.stream().noneMatch(
- replica -> replica.getDatanodeDetails().equals(target)) &&
- containerMoveSatisfiesPlacementPolicy(container, replicas, source,
- target) &&
- canSizeEnterTarget(target, containerInfo.getUsedBytes())) {
- return new ContainerMoveSelection(target, container);
- }
- }
- }
- logger.info("Container Balancer could not find a target for " +
- "source datanode {}", source.getUuidString());
- return null;
- }
-
- /**
- * Checks if container being present in target instead of source satisfies
- * the placement policy.
- * @param containerID Container to be moved from source to target
- * @param replicas Set of replicas of the given container
- * @param source Source datanode for container move
- * @param target Target datanode for container move
- * @return true if placement policy is satisfied, otherwise false
- */
- private boolean containerMoveSatisfiesPlacementPolicy(
- ContainerID containerID, Set replicas,
- DatanodeDetails source, DatanodeDetails target) {
- ContainerInfo containerInfo;
- try {
- containerInfo = containerManager.getContainer(containerID);
- } catch (ContainerNotFoundException e) {
- logger.warn("Could not get Container {} from Container Manager while " +
- "checking if container move satisfies placement policy in " +
- "Container Balancer.", containerID.toString(), e);
- return false;
- }
- List replicaList =
- replicas.stream()
- .map(ContainerReplica::getDatanodeDetails)
- .filter(datanodeDetails -> !datanodeDetails.equals(source))
- .collect(Collectors.toList());
- replicaList.add(target);
- ContainerPlacementStatus placementStatus = placementPolicyValidateProxy
- .validateContainerPlacement(replicaList, containerInfo);
-
- boolean isPolicySatisfied = placementStatus.isPolicySatisfied();
- if (!isPolicySatisfied) {
- logger.debug("Moving container {} from source {} to target {} will not " +
- "satisfy placement policy.", containerID, source.getUuidString(),
- target.getUuidString());
- }
- return isPolicySatisfied;
- }
-
- /**
- * Checks if specified size can enter specified target datanode
- * according to {@link ContainerBalancerConfiguration}
- * "size.entering.target.max".
- *
- * @param target target datanode in which size is entering
- * @param size size in bytes
- * @return true if size can enter target, else false
- */
- private boolean canSizeEnterTarget(DatanodeDetails target, long size) {
- if (sizeEnteringNode.containsKey(target)) {
- long sizeEnteringAfterMove = sizeEnteringNode.get(target) + size;
- //size can be moved into target datanode only when the following
- //two condition are met.
- //1 sizeEnteringAfterMove does not succeed the configured
- // MaxSizeEnteringTarget
- //2 current usage of target datanode plus sizeEnteringAfterMove
- // is smaller than or equal to upperLimit
- if (sizeEnteringAfterMove > config.getMaxSizeEnteringTarget()) {
- logger.debug("{} bytes cannot enter datanode {} because 'size" +
- ".entering.target.max' limit is {} and {} bytes have already " +
- "entered.", size, target.getUuidString(),
- config.getMaxSizeEnteringTarget(),
- sizeEnteringNode.get(target));
- return false;
- }
- if (Double.compare(nodeManager.getUsageInfo(target)
- .calculateUtilization(sizeEnteringAfterMove), upperLimit) > 0) {
- logger.debug("{} bytes cannot enter datanode {} because its " +
- "utilization will exceed the upper limit of {}.", size,
- target.getUuidString(), upperLimit);
- return false;
- }
- return true;
- }
-
- logger.warn("No record of how much size has entered datanode {}",
- target.getUuidString());
- return false;
- }
-
- /**
- * increase the Entering size of a candidate target data node.
- */
- @Override
- public void increaseSizeEntering(DatanodeDetails target, long size) {
- if (sizeEnteringNode.containsKey(target)) {
- long totalEnteringSize = sizeEnteringNode.get(target) + size;
- sizeEnteringNode.put(target, totalEnteringSize);
- potentialTargets.removeIf(
- c -> c.getDatanodeDetails().equals(target));
- if (totalEnteringSize < config.getMaxSizeEnteringTarget()) {
- //reorder
- potentialTargets.add(nodeManager.getUsageInfo(target));
- }
- return;
- }
- logger.warn("Cannot find {} in the candidates target nodes",
- target.getUuid());
- }
-
- /**
- * reInitialize FindTargetStrategy with the given new parameters.
- */
- @Override
- public void reInitialize(List potentialDataNodes,
- ContainerBalancerConfiguration conf,
- Double upLimit) {
- setConfiguration(conf);
- setUpperLimit(upLimit);
- sizeEnteringNode.clear();
- resetTargets(potentialDataNodes);
- }
-
- @VisibleForTesting
- public Collection getPotentialTargets() {
- return potentialTargets;
- }
-
- /**
- * sort potentialTargets for specified source datanode according to
- * network topology if enabled.
- * @param source the specified source datanode
- */
- @VisibleForTesting
- public abstract void sortTargetForSource(DatanodeDetails source);
-
- /**
- * Resets the collection of potential target datanodes that are considered
- * to identify a target for a source.
- * @param targets potential targets
- */
- void resetTargets(Collection targets) {
- potentialTargets.clear();
- targets.forEach(datanodeUsageInfo -> {
- sizeEnteringNode.putIfAbsent(datanodeUsageInfo.getDatanodeDetails(), 0L);
- potentialTargets.add(datanodeUsageInfo);
- });
- }
-
- NodeManager getNodeManager() {
- return nodeManager;
- }
-
-}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java
index 7b5cbe9f21fc..a9720c4ebd62 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java
@@ -30,55 +30,48 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.ReentrantLock;
/**
- * Container balancer is a service in SCM to move containers between over- and
- * under-utilized datanodes.
+ * Container balancer is a service in SCM to move containers between over- and under-utilized datanodes.
*/
public class ContainerBalancer extends StatefulService {
-
+ public static final Logger LOG = LoggerFactory.getLogger(ContainerBalancer.class);
private static final AtomicInteger ID = new AtomicInteger();
-
- public static final Logger LOG =
- LoggerFactory.getLogger(ContainerBalancer.class);
-
- private StorageContainerManager scm;
+ private final StorageContainerManager scm;
private final SCMContext scmContext;
- private OzoneConfiguration ozoneConfiguration;
+ private final OzoneConfiguration ozoneConfiguration;
private ContainerBalancerConfiguration config;
- private ContainerBalancerMetrics metrics;
+ private final ContainerBalancerMetrics metrics;
private volatile Thread currentBalancingThread;
private volatile ContainerBalancerTask task = null;
- private ReentrantLock lock;
+ private final ReentrantLock lock;
/**
- * Constructs ContainerBalancer with the specified arguments. Initializes
- * ContainerBalancerMetrics. Container Balancer does not start on
- * construction.
+ * Constructs ContainerBalancer with the specified arguments.
+ * Initializes ContainerBalancerMetrics. Container Balancer does not start on construction.
*
* @param scm the storage container manager
*/
- public ContainerBalancer(StorageContainerManager scm) {
+ public ContainerBalancer(@Nonnull StorageContainerManager scm) {
super(scm.getStatefulServiceStateManager());
this.scm = scm;
this.ozoneConfiguration = scm.getConfiguration();
- this.config = ozoneConfiguration.getObject(
- ContainerBalancerConfiguration.class);
+ this.config = ozoneConfiguration.getObject(ContainerBalancerConfiguration.class);
this.scmContext = scm.getScmContext();
this.metrics = ContainerBalancerMetrics.create();
-
this.lock = new ReentrantLock();
scm.getSCMServiceManager().register(this);
}
/**
* Receives a notification for raft or safe mode related status changes.
- * Stops ContainerBalancer if it's running and the current SCM becomes a
- * follower or enters safe mode. Starts ContainerBalancer if the current
- * SCM becomes leader, is out of safe mode and balancer should run.
+ * Stops ContainerBalancer if it's running and the current SCM becomes a follower or enters safe mode.
+ * Starts ContainerBalancer if the current SCM becomes leader, is out of safe mode and balancer should run.
*/
@Override
public void notifyStatusChanged() {
@@ -102,17 +95,14 @@ public void notifyStatusChanged() {
// else check for start
boolean shouldRun = shouldRun();
if (shouldRun && !canBalancerStart()) {
- LOG.warn("Could not start ContainerBalancer on notify," +
- " might be stopped");
+ LOG.warn("Could not start ContainerBalancer on notify, might be stopped");
}
if (shouldRun && canBalancerStart()) {
LOG.info("Starting ContainerBalancer in this scm on status change");
try {
start();
- } catch (IllegalContainerBalancerStateException |
- InvalidContainerBalancerConfigurationException e) {
- LOG.warn("Could not start ContainerBalancer on raft/safe-mode " +
- "status change.", e);
+ } catch (IllegalContainerBalancerStateException | InvalidContainerBalancerConfigurationException e) {
+ LOG.warn("Could not start ContainerBalancer on raft/safe-mode status change.", e);
}
}
} finally {
@@ -121,26 +111,23 @@ public void notifyStatusChanged() {
}
/**
- * Checks if ContainerBalancer should start (after a leader change, restart
- * etc.) by reading persisted state.
+ * Checks if ContainerBalancer should start (after a leader change, restart etc.) by reading persisted state.
+ *
* @return true if the persisted state is true, otherwise false
*/
@Override
public boolean shouldRun() {
try {
- ContainerBalancerConfigurationProto proto =
- readConfiguration(ContainerBalancerConfigurationProto.class);
+ ContainerBalancerConfigurationProto proto = readConfiguration(ContainerBalancerConfigurationProto.class);
if (proto == null) {
- LOG.warn("Could not find persisted configuration for {} when checking" +
- " if ContainerBalancer should run. ContainerBalancer should not " +
- "run now.", this.getServiceName());
+ LOG.warn("Could not find persisted configuration for {} when checking if ContainerBalancer should run." +
+ " ContainerBalancer should not run now.", this.getServiceName());
return false;
}
return proto.getShouldRun();
} catch (IOException e) {
- LOG.warn("Could not read persisted configuration for checking if " +
- "ContainerBalancer should start. ContainerBalancer should not start" +
- " now.", e);
+ LOG.warn("Could not read persisted configuration for checking if ContainerBalancer should start." +
+ " ContainerBalancer should not start now.", e);
return false;
}
}
@@ -151,8 +138,7 @@ public boolean shouldRun() {
* @return true if balancer started, otherwise false
*/
public boolean isBalancerRunning() {
- return (null != task
- && task.getBalancerStatus() == ContainerBalancerTask.Status.RUNNING);
+ return (null != task && task.getBalancerStatus() == ContainerBalancerTask.Status.RUNNING);
}
/**
@@ -161,18 +147,16 @@ public boolean isBalancerRunning() {
* @return true if balancer can be started, otherwise false
*/
private boolean canBalancerStart() {
- return (null == task
- || task.getBalancerStatus() == ContainerBalancerTask.Status.STOPPED);
+ return (null == task || task.getBalancerStatus() == ContainerBalancerTask.Status.STOPPED);
}
/**
* get the Container Balancer state.
*
- * @return true if balancer started, otherwise false
+ * @return ContainerBalancerTask.Status if balancer started, otherwise false
*/
- public ContainerBalancerTask.Status getBalancerStatus() {
- return null != task ? task.getBalancerStatus()
- : ContainerBalancerTask.Status.STOPPED;
+ public @Nonnull ContainerBalancerTask.Status getBalancerStatus() {
+ return null != task ? task.getBalancerStatus() : ContainerBalancerTask.Status.STOPPED;
}
/**
@@ -193,17 +177,15 @@ public String getServiceName() {
}
/**
- * Starts ContainerBalancer as an SCMService. Validates state, reads and
- * validates persisted configuration, and then starts the balancing
- * thread.
- * @throws IllegalContainerBalancerStateException if balancer should not
- * run according to persisted configuration
- * @throws InvalidContainerBalancerConfigurationException if failed to
- * retrieve persisted configuration, or the configuration is null
+ * Starts ContainerBalancer as an SCMService. Validates state, reads and validates persisted configuration,
+ * and then starts the balancing thread.
+ *
+ * @throws IllegalContainerBalancerStateException if balancer should not run according to persisted configuration
+ * @throws InvalidContainerBalancerConfigurationException if failed to retrieve persisted configuration,
+ * or the configuration is null
*/
@Override
- public void start() throws IllegalContainerBalancerStateException,
- InvalidContainerBalancerConfigurationException {
+ public void start() throws IllegalContainerBalancerStateException, InvalidContainerBalancerConfigurationException {
lock.lock();
try {
// should be leader-ready, out of safe mode, and not running already
@@ -212,22 +194,20 @@ public void start() throws IllegalContainerBalancerStateException,
try {
proto = readConfiguration(ContainerBalancerConfigurationProto.class);
} catch (IOException e) {
- throw new InvalidContainerBalancerConfigurationException("Could not " +
- "retrieve persisted configuration while starting " +
- "Container Balancer as an SCMService. Will not start now.", e);
+ throw new InvalidContainerBalancerConfigurationException(
+ "Could not retrieve persisted configuration while starting Container Balancer as an SCMService. " +
+ "Will not start now.", e);
}
if (proto == null) {
- throw new InvalidContainerBalancerConfigurationException("Persisted " +
- "configuration for ContainerBalancer is null during start. Will " +
- "not start now.");
+ throw new InvalidContainerBalancerConfigurationException(
+ "Persisted configuration for ContainerBalancer is null during start. Will not start now.");
}
if (!proto.getShouldRun()) {
- throw new IllegalContainerBalancerStateException("According to " +
- "persisted configuration, ContainerBalancer should not run.");
+ throw new IllegalContainerBalancerStateException(
+ "According to persisted configuration, ContainerBalancer should not run.");
}
ContainerBalancerConfiguration configuration =
- ContainerBalancerConfiguration.fromProtobuf(proto,
- ozoneConfiguration);
+ ContainerBalancerConfiguration.fromProtobuf(proto, ozoneConfiguration);
validateConfiguration(configuration);
this.config = configuration;
startBalancingThread(proto.getNextIterationIndex(), true);
@@ -237,20 +217,15 @@ public void start() throws IllegalContainerBalancerStateException,
}
/**
- * Starts Container Balancer after checking its state and validating
- * configuration.
+ * Starts Container Balancer after checking its state and validating configuration.
*
- * @throws IllegalContainerBalancerStateException if ContainerBalancer is
- * not in a start-appropriate state
- * @throws InvalidContainerBalancerConfigurationException if
- * {@link ContainerBalancerConfiguration} config file is incorrectly
- * configured
- * @throws IOException on failure to persist
- * {@link ContainerBalancerConfiguration}
+ * @throws IllegalContainerBalancerStateException if ContainerBalancer is not in a start-appropriate state
+ * @throws InvalidContainerBalancerConfigurationException if {@link ContainerBalancerConfiguration} config file
+ * is incorrectly configured
+ * @throws IOException on failure to persist {@link ContainerBalancerConfiguration}
*/
public void startBalancer(ContainerBalancerConfiguration configuration)
- throws IllegalContainerBalancerStateException,
- InvalidContainerBalancerConfigurationException, IOException {
+ throws IllegalContainerBalancerStateException, InvalidContainerBalancerConfigurationException, IOException {
lock.lock();
try {
// validates state, config, and then saves config
@@ -269,12 +244,10 @@ public void startBalancer(ContainerBalancerConfiguration configuration)
/**
* Starts a new balancing thread asynchronously.
*/
- private void startBalancingThread(int nextIterationIndex,
- boolean delayStart) {
+ private void startBalancingThread(int nextIterationIndex, boolean delayStart) {
String prefix = scmContext.threadNamePrefix();
- task = new ContainerBalancerTask(scm, nextIterationIndex, this, metrics,
- config, delayStart);
- Thread thread = new Thread(task);
+ task = new ContainerBalancerTask(scm, this, config);
+ Thread thread = new Thread(() -> task.run(nextIterationIndex, delayStart));
thread.setName(prefix + "ContainerBalancerTask-" + ID.incrementAndGet());
thread.setDaemon(true);
thread.start();
@@ -286,18 +259,15 @@ private void startBalancingThread(int nextIterationIndex,
* Validates balancer's state based on the specified expectedRunning.
* Confirms SCM is leader-ready and out of safe mode.
*
- * @param expectedRunning true if ContainerBalancer is expected to be
- * running, else false
- * @throws IllegalContainerBalancerStateException if SCM is not
- * leader-ready, is in safe mode, or state does not match the specified
- * expected state
+ * @param expectedRunning true if ContainerBalancer is expected to be running, else false
+ *
+ * @throws IllegalContainerBalancerStateException if SCM is not leader-ready, is in safe mode, or state
+ * does not match the specified expected state
*/
- private void validateState(boolean expectedRunning)
- throws IllegalContainerBalancerStateException {
+ private void validateState(boolean expectedRunning) throws IllegalContainerBalancerStateException {
if (!scmContext.isLeaderReady()) {
LOG.warn("SCM is not leader ready");
- throw new IllegalContainerBalancerStateException("SCM is not leader " +
- "ready");
+ throw new IllegalContainerBalancerStateException("SCM is not leader ready");
}
if (scmContext.isInSafeMode()) {
LOG.warn("SCM is in safe mode");
@@ -305,13 +275,11 @@ private void validateState(boolean expectedRunning)
}
if (!expectedRunning && !canBalancerStart()) {
throw new IllegalContainerBalancerStateException(
- "Expect ContainerBalancer as not running state" +
- ", but running state is actually " + getBalancerStatus());
+ "Expect ContainerBalancer as not running state, but running state is actually " + getBalancerStatus());
}
if (expectedRunning && !canBalancerStop()) {
throw new IllegalContainerBalancerStateException(
- "Expect ContainerBalancer as running state" +
- ", but running state is actually " + getBalancerStatus());
+ "Expect ContainerBalancer as running state, but running state is actually " + getBalancerStatus());
}
}
@@ -324,8 +292,7 @@ public void stop() {
Thread balancingThread;
try {
if (!canBalancerStop()) {
- LOG.warn("Cannot stop Container Balancer because it's not running or " +
- "stopping");
+ LOG.warn("Cannot stop Container Balancer because it's not running or stopping");
return;
}
LOG.info("Trying to stop ContainerBalancer in this SCM.");
@@ -338,9 +305,8 @@ public void stop() {
blockTillTaskStop(balancingThread);
}
- private static void blockTillTaskStop(Thread balancingThread) {
- // NOTE: join should be called outside the lock in hierarchy
- // to avoid locking others waiting
+ private static void blockTillTaskStop(@Nonnull Thread balancingThread) {
+ // NOTE: join should be called outside the lock in hierarchy to avoid locking others waiting
// wait for balancingThread to die with interrupt
balancingThread.interrupt();
LOG.info("Container Balancer waiting for {} to stop", balancingThread);
@@ -353,12 +319,10 @@ private static void blockTillTaskStop(Thread balancingThread) {
}
/**
- * Stops ContainerBalancer gracefully. Persists state such that
- * {@link ContainerBalancer#shouldRun()} will return false. This is the
- * "stop" command.
+ * Stops ContainerBalancer gracefully.
+ * Persists state such that {@link ContainerBalancer#shouldRun()} will return false. This is the "stop" command.
*/
- public void stopBalancer()
- throws IOException, IllegalContainerBalancerStateException {
+ public void stopBalancer() throws IOException, IllegalContainerBalancerStateException {
Thread balancingThread;
lock.lock();
try {
@@ -373,68 +337,62 @@ public void stopBalancer()
blockTillTaskStop(balancingThread);
}
- public void saveConfiguration(ContainerBalancerConfiguration configuration,
- boolean shouldRun, int index)
+ public void saveConfiguration(@Nonnull ContainerBalancerConfiguration configuration, boolean shouldRun, int index)
throws IOException {
config = configuration;
- saveConfiguration(configuration.toProtobufBuilder()
+ saveConfiguration(configuration
+ .toProtobufBuilder()
.setShouldRun(shouldRun)
.setNextIterationIndex(index)
.build());
}
- private void validateConfiguration(ContainerBalancerConfiguration conf)
+ private void validateConfiguration(@Nonnull ContainerBalancerConfiguration conf)
throws InvalidContainerBalancerConfigurationException {
- // maxSizeEnteringTarget and maxSizeLeavingSource should by default be
- // greater than container size
+ // maxSizeEnteringTarget and maxSizeLeavingSource should by default be greater than container size
long size = (long) ozoneConfiguration.getStorageSize(
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES);
if (conf.getMaxSizeEnteringTarget() <= size) {
- LOG.warn("hdds.container.balancer.size.entering.target.max {} should " +
- "be greater than ozone.scm.container.size {}",
+ LOG.warn("hdds.container.balancer.size.entering.target.max {} should be greater than ozone.scm.container.size {}",
conf.getMaxSizeEnteringTarget(), size);
throw new InvalidContainerBalancerConfigurationException(
- "hdds.container.balancer.size.entering.target.max should be greater" +
- " than ozone.scm.container.size");
+ "hdds.container.balancer.size.entering.target.max should be greater than ozone.scm.container.size");
}
if (conf.getMaxSizeLeavingSource() <= size) {
- LOG.warn("hdds.container.balancer.size.leaving.source.max {} should " +
- "be greater than ozone.scm.container.size {}",
+ LOG.warn("hdds.container.balancer.size.leaving.source.max {} should be greater than ozone.scm.container.size {}",
conf.getMaxSizeLeavingSource(), size);
throw new InvalidContainerBalancerConfigurationException(
- "hdds.container.balancer.size.leaving.source.max should be greater" +
- " than ozone.scm.container.size");
+ "hdds.container.balancer.size.leaving.source.max should be greater than ozone.scm.container.size");
}
// balancing interval should be greater than DUFactory refresh period
DUFactory.Conf duConf = ozoneConfiguration.getObject(DUFactory.Conf.class);
long refreshPeriod = duConf.getRefreshPeriod().toMillis();
if (conf.getBalancingInterval().toMillis() <= refreshPeriod) {
- LOG.warn("hdds.container.balancer.balancing.iteration.interval {} " +
- "should be greater than hdds.datanode.du.refresh.period {}",
+ LOG.warn("hdds.container.balancer.balancing.iteration.interval {} should be greater than " +
+ "hdds.datanode.du.refresh.period {}",
conf.getBalancingInterval().toMillis(), refreshPeriod);
}
// "move.replication.timeout" should be lesser than "move.timeout"
- if (conf.getMoveReplicationTimeout().toMillis() >=
- conf.getMoveTimeout().toMillis()) {
- LOG.warn("hdds.container.balancer.move.replication.timeout {} should " +
- "be less than hdds.container.balancer.move.timeout {}.",
+ if (conf.getMoveReplicationTimeout().toMillis() >= conf.getMoveTimeout().toMillis()) {
+ LOG.warn("hdds.container.balancer.move.replication.timeout {} should be less than " +
+ "hdds.container.balancer.move.timeout {}.",
conf.getMoveReplicationTimeout().toMinutes(),
conf.getMoveTimeout().toMinutes());
throw new InvalidContainerBalancerConfigurationException(
- "hdds.container.balancer.move.replication.timeout should " +
- "be less than hdds.container.balancer.move.timeout.");
+ "hdds.container.balancer.move.replication.timeout should be less than hdds.container.balancer.move.timeout.");
}
}
- public ContainerBalancerMetrics getMetrics() {
+ public @Nonnull ContainerBalancerMetrics getMetrics() {
return metrics;
}
@VisibleForTesting
+ @Nullable
Thread getCurrentBalancingThread() {
return currentBalancingThread;
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerConfiguration.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerConfiguration.java
index 8f9332e2d3ca..7b6182c2c92f 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerConfiguration.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerConfiguration.java
@@ -30,6 +30,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import javax.annotation.Nonnull;
import java.time.Duration;
import java.util.Arrays;
import java.util.Collections;
@@ -60,6 +61,26 @@ public final class ContainerBalancerConfiguration {
"that can be involved in balancing in one iteration.")
private int maxDatanodesPercentageToInvolvePerIteration = 20;
+ @Config(key = "adapt.balance.when.close.to.limit",
+ type = ConfigType.BOOLEAN,
+ defaultValue = "true",
+ tags = {ConfigTag.BALANCER},
+ description = "Allow to reset potential target datanodes if balancer " +
+ "is one datanode away from " +
+ "datanodes.involved.max.percentage.per.iteration limit"
+ )
+ private boolean adaptBalanceWhenCloseToLimit = true;
+
+ @Config(key = "adapt.balance.when.reach.the.limit",
+ type = ConfigType.BOOLEAN,
+ defaultValue = "true",
+ tags = {ConfigTag.BALANCER},
+ description = "lAlow to reset potential source and target datanodes " +
+ "if balancer has reached " +
+ "datanodes.involved.max.percentage.per.iteration limit"
+ )
+ private boolean adaptBalanceWhenReachTheLimit = true;
+
@Config(key = "size.moved.max.per.iteration", type = ConfigType.SIZE,
defaultValue = "500GB", tags = {ConfigTag.BALANCER},
description = "The maximum size of data in bytes that will be moved " +
@@ -169,13 +190,12 @@ public void setThreshold(double threshold) {
}
/**
- * Gets the iteration count for Container Balancer. A value of -1 means
- * infinite number of iterations.
+ * Gets the iteration count for Container Balancer. A value of -1 means infinite number of iterations.
*
- * @return a value greater than 0, or -1
+ * @return a value greater than 0, or Integer.MAX_VALUE that means number of iterations when values is -1
*/
public int getIterations() {
- return iterations;
+ return (iterations != -1) ? iterations : Integer.MAX_VALUE;
}
/**
@@ -286,6 +306,22 @@ public void setMaxSizeToMovePerIteration(long maxSizeToMovePerIteration) {
this.maxSizeToMovePerIteration = maxSizeToMovePerIteration;
}
+ public boolean adaptBalanceWhenCloseToLimits() {
+ return adaptBalanceWhenCloseToLimit;
+ }
+
+ public void setAdaptBalanceWhenCloseToLimit(boolean value) {
+ adaptBalanceWhenCloseToLimit = value;
+ }
+
+ public boolean adaptBalanceWhenReachTheLimits() {
+ return adaptBalanceWhenReachTheLimit;
+ }
+
+ public void setAdaptBalanceWhenReachTheLimit(boolean value) {
+ adaptBalanceWhenReachTheLimit = value;
+ }
+
public long getMaxSizeEnteringTarget() {
return maxSizeEnteringTarget;
}
@@ -302,7 +338,7 @@ public void setMaxSizeLeavingSource(long maxSizeLeavingSource) {
this.maxSizeLeavingSource = maxSizeLeavingSource;
}
- public Set getExcludeContainers() {
+ public @Nonnull Set getExcludeContainers() {
if (excludeContainers.isEmpty()) {
return new HashSet<>();
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java
index 09558d3a6d4f..86ef3e45fc27 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java
@@ -28,77 +28,41 @@
/**
* Metrics related to Container Balancer running in SCM.
*/
-@Metrics(name = "ContainerBalancer Metrics", about = "Metrics related to " +
- "Container Balancer running in SCM", context = "SCM")
+@Metrics(name = "ContainerBalancer Metrics",
+ about = "Metrics related toContainer Balancer running in SCM", context = "SCM")
public final class ContainerBalancerMetrics {
- public static final String NAME =
- ContainerBalancerMetrics.class.getSimpleName();
+ public static final String NAME = ContainerBalancerMetrics.class.getSimpleName();
private final MetricsSystem ms;
- @Metric(about = "Amount of Gigabytes that Container Balancer moved" +
- " in the latest iteration.")
- private MutableCounterLong dataSizeMovedGBInLatestIteration;
-
- @Metric(about = "Number of completed container moves performed by " +
- "Container Balancer in the latest iteration.")
- private MutableCounterLong numContainerMovesCompletedInLatestIteration;
-
- @Metric(about = "Number of timeout container moves performed by " +
- "Container Balancer in the latest iteration.")
- private MutableCounterLong numContainerMovesTimeoutInLatestIteration;
-
@Metric(about = "Number of iterations that Container Balancer has run for.")
private MutableCounterLong numIterations;
- @Metric(about = "Number of datanodes that were involved in balancing in the" +
- " latest iteration.")
- private MutableCounterLong numDatanodesInvolvedInLatestIteration;
-
- @Metric(about = "Amount of data in Gigabytes that is causing unbalance.")
- private MutableCounterLong dataSizeUnbalancedGB;
-
- @Metric(about = "Number of unbalanced datanodes.")
- private MutableCounterLong numDatanodesUnbalanced;
-
- @Metric(about = "Total number of completed container moves across all " +
- "iterations of Container Balancer.")
+ @Metric(about = "Total number of completed container moves across all iterations of Container Balancer.")
private MutableCounterLong numContainerMovesCompleted;
- @Metric(about = "Total number of timeout container moves across " +
- "all iterations of Container Balancer.")
+ @Metric(about = "Total number of timeout container moves across all iterations of Container Balancer.")
private MutableCounterLong numContainerMovesTimeout;
- @Metric(about = "Total data size in GB moved across all iterations of " +
- "Container Balancer.")
- private MutableCounterLong dataSizeMovedGB;
+ @Metric(about = "Total data size in Bytes moved across all iterations of Container Balancer.")
+ private MutableCounterLong dataSizeMovedBytes;
- @Metric(about = "Total number container for which moves failed " +
- "exceptionally across all iterations of Container Balancer.")
+ @Metric(about = "Total number container for which moves failed exceptionally across" +
+ " all iterations of Container Balancer.")
private MutableCounterLong numContainerMovesFailed;
- @Metric(about = "Total number container for which moves failed " +
- "exceptionally in latest iteration of Container Balancer.")
- private MutableCounterLong numContainerMovesFailedInLatestIteration;
-
- @Metric(about = "Number of container moves that were scheduled in the " +
- "latest iteration of Container Balancer.")
- private MutableCounterLong numContainerMovesScheduledInLatestIteration;
-
@Metric(about = "Total number of container moves that were scheduled across" +
" all iterations of Container Balancer.")
private MutableCounterLong numContainerMovesScheduled;
/**
- * Create and register metrics named {@link ContainerBalancerMetrics#NAME}
- * for {@link ContainerBalancer}.
+ * Create and register metrics named {@link ContainerBalancerMetrics#NAME} for {@link ContainerBalancer}.
*
* @return {@link ContainerBalancerMetrics}
*/
public static ContainerBalancerMetrics create() {
MetricsSystem ms = DefaultMetricsSystem.instance();
- return ms.register(NAME, "Container Balancer Metrics",
- new ContainerBalancerMetrics(ms));
+ return ms.register(NAME, "Container Balancer Metrics", new ContainerBalancerMetrics(ms));
}
private ContainerBalancerMetrics(MetricsSystem ms) {
@@ -106,123 +70,17 @@ private ContainerBalancerMetrics(MetricsSystem ms) {
}
/**
- * Gets the number of container moves scheduled across all iterations of
- * Container Balancer.
+ * Gets the number of container moves scheduled across all iterations of Container Balancer.
* @return number of moves
*/
public long getNumContainerMovesScheduled() {
return numContainerMovesScheduled.value();
}
- void incrementNumContainerMovesScheduled(long valueToAdd) {
+ public void incrementNumContainerMovesScheduled(long valueToAdd) {
this.numContainerMovesScheduled.incr(valueToAdd);
}
- /**
- * Gets the number of container moves scheduled in the latest iteration of
- * Container Balancer.
- * @return number of moves
- */
- public long getNumContainerMovesScheduledInLatestIteration() {
- return numContainerMovesScheduledInLatestIteration.value();
- }
-
- void incrementNumContainerMovesScheduledInLatestIteration(long valueToAdd) {
- this.numContainerMovesScheduledInLatestIteration.incr(valueToAdd);
- }
-
- /**
- * Gets the amount of data moved by Container Balancer in the latest
- * iteration.
- * @return size in GB
- */
- public long getDataSizeMovedGBInLatestIteration() {
- return dataSizeMovedGBInLatestIteration.value();
- }
-
- public void incrementDataSizeMovedGBInLatestIteration(long valueToAdd) {
- this.dataSizeMovedGBInLatestIteration.incr(valueToAdd);
- }
-
- public void resetDataSizeMovedGBInLatestIteration() {
- dataSizeMovedGBInLatestIteration.incr(
- -getDataSizeMovedGBInLatestIteration());
- }
-
- /**
- * Gets the number of container moves performed by Container Balancer in the
- * latest iteration.
- * @return number of container moves
- */
- public long getNumContainerMovesCompletedInLatestIteration() {
- return numContainerMovesCompletedInLatestIteration.value();
- }
-
- public void incrementNumContainerMovesCompletedInLatestIteration(
- long valueToAdd) {
- this.numContainerMovesCompletedInLatestIteration.incr(valueToAdd);
- }
-
- public void incrementCurrentIterationContainerMoveMetric(
- MoveManager.MoveResult result, long valueToAdd) {
- if (result == null) {
- return;
- }
- switch (result) {
- case COMPLETED:
- this.numContainerMovesCompletedInLatestIteration.incr(valueToAdd);
- break;
- case REPLICATION_FAIL_TIME_OUT:
- case DELETION_FAIL_TIME_OUT:
- this.numContainerMovesTimeoutInLatestIteration.incr(valueToAdd);
- break;
- // TODO: Add metrics for other errors that need to be tracked.
- case FAIL_LEADER_NOT_READY:
- case REPLICATION_FAIL_INFLIGHT_REPLICATION:
- case REPLICATION_FAIL_NOT_EXIST_IN_SOURCE:
- case REPLICATION_FAIL_EXIST_IN_TARGET:
- case REPLICATION_FAIL_CONTAINER_NOT_CLOSED:
- case REPLICATION_FAIL_INFLIGHT_DELETION:
- case REPLICATION_FAIL_NODE_NOT_IN_SERVICE:
- case DELETION_FAIL_NODE_NOT_IN_SERVICE:
- case REPLICATION_FAIL_NODE_UNHEALTHY:
- case DELETION_FAIL_NODE_UNHEALTHY:
- case DELETE_FAIL_POLICY:
- case REPLICATION_NOT_HEALTHY_BEFORE_MOVE:
- case REPLICATION_NOT_HEALTHY_AFTER_MOVE:
- case FAIL_CONTAINER_ALREADY_BEING_MOVED:
- case FAIL_UNEXPECTED_ERROR:
- incrementNumContainerMovesFailedInLatestIteration(valueToAdd);
- break;
- default:
- break;
- }
- }
-
- public void resetNumContainerMovesCompletedInLatestIteration() {
- numContainerMovesCompletedInLatestIteration.incr(
- -getNumContainerMovesCompletedInLatestIteration());
- }
-
- /**
- * Gets the number of timeout container moves performed by
- * Container Balancer in the latest iteration.
- * @return number of timeout container moves
- */
- public long getNumContainerMovesTimeoutInLatestIteration() {
- return numContainerMovesTimeoutInLatestIteration.value();
- }
-
- public void incrementNumContainerMovesTimeoutInLatestIteration(
- long valueToAdd) {
- this.numContainerMovesTimeoutInLatestIteration.incr(valueToAdd);
- }
-
- public void resetNumContainerMovesTimeoutInLatestIteration() {
- numContainerMovesTimeoutInLatestIteration.incr(
- -getNumContainerMovesTimeoutInLatestIteration());
- }
-
/**
* Gets the number of iterations that Container Balancer has run for.
* @return number of iterations
@@ -235,56 +93,6 @@ public void incrementNumIterations(long valueToAdd) {
numIterations.incr(valueToAdd);
}
- /**
- * Gets number of datanodes that were involved in balancing in the latest
- * iteration.
- * @return number of datanodes
- */
- public long getNumDatanodesInvolvedInLatestIteration() {
- return numDatanodesInvolvedInLatestIteration.value();
- }
-
- public void incrementNumDatanodesInvolvedInLatestIteration(long valueToAdd) {
- numDatanodesInvolvedInLatestIteration.incr(valueToAdd);
- }
-
- public void resetNumDatanodesInvolvedInLatestIteration() {
- numDatanodesInvolvedInLatestIteration.incr(
- -getNumDatanodesInvolvedInLatestIteration());
- }
-
- /**
- * Gets the amount of data in Gigabytes that is causing unbalance.
- * @return size of data as a long value
- */
- public long getDataSizeUnbalancedGB() {
- return dataSizeUnbalancedGB.value();
- }
-
- public void incrementDataSizeUnbalancedGB(long valueToAdd) {
- dataSizeUnbalancedGB.incr(valueToAdd);
- }
-
- public void resetDataSizeUnbalancedGB() {
- dataSizeUnbalancedGB.incr(-getDataSizeUnbalancedGB());
- }
-
- /**
- * Gets the number of datanodes that are unbalanced.
- * @return long value
- */
- public long getNumDatanodesUnbalanced() {
- return numDatanodesUnbalanced.value();
- }
-
- public void incrementNumDatanodesUnbalanced(long valueToAdd) {
- numDatanodesUnbalanced.incr(valueToAdd);
- }
-
- public void resetNumDatanodesUnbalanced() {
- numDatanodesUnbalanced.incr(-getNumDatanodesUnbalanced());
- }
-
public long getNumContainerMovesCompleted() {
return numContainerMovesCompleted.value();
}
@@ -301,12 +109,12 @@ public void incrementNumContainerMovesTimeout(long valueToAdd) {
numContainerMovesTimeout.incr(valueToAdd);
}
- public long getDataSizeMovedGB() {
- return dataSizeMovedGB.value();
+ public long getDataSizeMovedBytes() {
+ return dataSizeMovedBytes.value();
}
- public void incrementDataSizeMovedGB(long valueToAdd) {
- dataSizeMovedGB.incr(valueToAdd);
+ public void incrementDataSizeMovedBytes(long valueToAdd) {
+ dataSizeMovedBytes.incr(valueToAdd);
}
public long getNumContainerMovesFailed() {
@@ -316,17 +124,4 @@ public long getNumContainerMovesFailed() {
public void incrementNumContainerMovesFailed(long valueToAdd) {
numContainerMovesFailed.incr(valueToAdd);
}
-
- public long getNumContainerMovesFailedInLatestIteration() {
- return numContainerMovesFailedInLatestIteration.value();
- }
-
- public void incrementNumContainerMovesFailedInLatestIteration(
- long valueToAdd) {
- numContainerMovesFailedInLatestIteration.incr(valueToAdd);
- }
- public void resetNumContainerMovesFailedInLatestIteration() {
- numContainerMovesFailedInLatestIteration.incr(
- -getNumContainerMovesFailedInLatestIteration());
- }
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerSelectionCriteria.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerSelectionCriteria.java
deleted file mode 100644
index ec0e493c28f9..000000000000
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerSelectionCriteria.java
+++ /dev/null
@@ -1,248 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.hadoop.hdds.scm.container.balancer;
-
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
-import org.apache.hadoop.hdds.scm.container.ContainerID;
-import org.apache.hadoop.hdds.scm.container.ContainerInfo;
-import org.apache.hadoop.hdds.scm.container.ContainerManager;
-import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
-import org.apache.hadoop.hdds.scm.container.ContainerReplica;
-import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
-import org.apache.hadoop.hdds.scm.node.NodeManager;
-import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.NavigableSet;
-import java.util.Set;
-import java.util.TreeSet;
-
-/**
- * The selection criteria for selecting containers that will be moved and
- * selecting datanodes that containers will move to.
- */
-public class ContainerBalancerSelectionCriteria {
- private static final Logger LOG =
- LoggerFactory.getLogger(ContainerBalancerSelectionCriteria.class);
-
- private ContainerBalancerConfiguration balancerConfiguration;
- private NodeManager nodeManager;
- private ReplicationManager replicationManager;
- private ContainerManager containerManager;
- private Set selectedContainers;
- private Set excludeContainers;
- private FindSourceStrategy findSourceStrategy;
-
- public ContainerBalancerSelectionCriteria(
- ContainerBalancerConfiguration balancerConfiguration,
- NodeManager nodeManager,
- ReplicationManager replicationManager,
- ContainerManager containerManager,
- FindSourceStrategy findSourceStrategy) {
- this.balancerConfiguration = balancerConfiguration;
- this.nodeManager = nodeManager;
- this.replicationManager = replicationManager;
- this.containerManager = containerManager;
- selectedContainers = new HashSet<>();
- excludeContainers = balancerConfiguration.getExcludeContainers();
- this.findSourceStrategy = findSourceStrategy;
- }
-
- /**
- * Checks whether container is currently undergoing replication or deletion.
- *
- * @param containerID Container to check.
- * @return true if container is replicating or deleting, otherwise false.
- */
- private boolean isContainerReplicatingOrDeleting(ContainerID containerID) {
- return replicationManager.isContainerReplicatingOrDeleting(containerID);
- }
-
- /**
- * Gets containers that are suitable for moving based on the following
- * required criteria:
- * 1. Container must not be undergoing replication.
- * 2. Container must not already be selected for balancing.
- * 3. Container size should be closer to 5GB.
- * 4. Container must not be in the configured exclude containers list.
- * 5. Container should be closed.
- * 6. Container should not be an EC container
- * //TODO Temporarily not considering EC containers as candidates
- * @see
- * HDDS-6940
- *
- * @param node DatanodeDetails for which to find candidate containers.
- * @return NavigableSet of candidate containers that satisfy the criteria.
- */
- public NavigableSet getCandidateContainers(
- DatanodeDetails node, long sizeMovedAlready) {
- NavigableSet containerIDSet =
- new TreeSet<>(orderContainersByUsedBytes().reversed());
- try {
- containerIDSet.addAll(nodeManager.getContainers(node));
- } catch (NodeNotFoundException e) {
- LOG.warn("Could not find Datanode {} while selecting candidate " +
- "containers for Container Balancer.", node.toString(), e);
- return containerIDSet;
- }
- if (excludeContainers != null) {
- containerIDSet.removeAll(excludeContainers);
- }
- if (selectedContainers != null) {
- containerIDSet.removeAll(selectedContainers);
- }
-
- containerIDSet.removeIf(
- containerID -> shouldBeExcluded(containerID, node, sizeMovedAlready));
- return containerIDSet;
- }
-
- /**
- * Checks if the first container has more used space than second.
- * @param first first container to compare
- * @param second second container to compare
- * @return An integer greater than 0 if first is more used, 0 if they're
- * the same containers or a container is not found, and a value less than 0
- * if first is less used than second. If both containers have equal used
- * space, they're compared using {@link ContainerID#compareTo(ContainerID)}.
- */
- private int isContainerMoreUsed(ContainerID first,
- ContainerID second) {
- if (first.equals(second)) {
- return 0;
- }
- try {
- ContainerInfo firstInfo = containerManager.getContainer(first);
- ContainerInfo secondInfo = containerManager.getContainer(second);
- if (firstInfo.getUsedBytes() > secondInfo.getUsedBytes()) {
- return 1;
- } else if (firstInfo.getUsedBytes() < secondInfo.getUsedBytes()) {
- return -1;
- } else {
- return first.compareTo(second);
- }
- } catch (ContainerNotFoundException e) {
- LOG.warn("Could not retrieve ContainerInfo from container manager for " +
- "comparison.", e);
- return 0;
- }
- }
-
- /**
- * Compares containers on the basis of used space.
- * @return First container is more used if it has used space greater than
- * second container.
- */
- private Comparator orderContainersByUsedBytes() {
- return this::isContainerMoreUsed;
- }
-
- /**
- * Checks whether a Container has the ReplicationType
- * {@link HddsProtos.ReplicationType#EC}.
- * @param container container to check
- * @return true if the ReplicationType is EC and "hdds.scm.replication
- * .enable.legacy" is true, else false
- */
- private boolean isECContainer(ContainerInfo container) {
- return container.getReplicationType().equals(HddsProtos.ReplicationType.EC)
- && replicationManager.getConfig().isLegacyEnabled();
- }
-
- private boolean shouldBeExcluded(ContainerID containerID,
- DatanodeDetails node, long sizeMovedAlready) {
- ContainerInfo container;
- try {
- container = containerManager.getContainer(containerID);
- } catch (ContainerNotFoundException e) {
- LOG.warn("Could not find Container {} to check if it should be a " +
- "candidate container. Excluding it.", containerID);
- return true;
- }
- return !isContainerClosed(container, node) || isECContainer(container) ||
- isContainerReplicatingOrDeleting(containerID) ||
- !findSourceStrategy.canSizeLeaveSource(node, container.getUsedBytes())
- || breaksMaxSizeToMoveLimit(container.containerID(),
- container.getUsedBytes(), sizeMovedAlready);
- }
-
- /**
- * Checks whether specified container is closed. Also checks if the replica
- * on the specified datanode is CLOSED. Assumes that there will only be one
- * replica of a container on a particular Datanode.
- * @param container container to check
- * @param datanodeDetails datanode on which a replica of the container is
- * present
- * @return true if container LifeCycleState is
- * {@link HddsProtos.LifeCycleState#CLOSED} and its replica on the
- * specified datanode is CLOSED, else false
- */
- private boolean isContainerClosed(ContainerInfo container,
- DatanodeDetails datanodeDetails) {
- if (!container.getState().equals(HddsProtos.LifeCycleState.CLOSED)) {
- return false;
- }
-
- // also check that the replica on the specified DN is closed
- Set replicas;
- try {
- replicas = containerManager.getContainerReplicas(container.containerID());
- } catch (ContainerNotFoundException e) {
- LOG.warn("Container {} does not exist in ContainerManager. Skipping " +
- "this container.", container.getContainerID(), e);
- return false;
- }
- for (ContainerReplica replica : replicas) {
- if (replica.getDatanodeDetails().equals(datanodeDetails)) {
- // don't consider replica if it's not closed
- // assumption: there's only one replica of this container on this DN
- return replica.getState().equals(ContainerReplicaProto.State.CLOSED);
- }
- }
-
- return false;
- }
-
- private boolean breaksMaxSizeToMoveLimit(ContainerID containerID,
- long usedBytes,
- long sizeMovedAlready) {
- // check max size to move per iteration limit
- if (sizeMovedAlready + usedBytes >
- balancerConfiguration.getMaxSizeToMovePerIteration()) {
- LOG.debug("Removing container {} because it fails max size " +
- "to move per iteration check.", containerID);
- return true;
- }
- return false;
- }
-
- public void setExcludeContainers(
- Set excludeContainers) {
- this.excludeContainers = excludeContainers;
- }
-
- public void setSelectedContainers(
- Set selectedContainers) {
- this.selectedContainers = selectedContainers;
- }
-
-}
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 6541d75d2793..7ea7462fa811 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
@@ -20,169 +20,80 @@
import com.google.common.annotations.VisibleForTesting;
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.scm.PlacementPolicyValidateProxy;
import org.apache.hadoop.hdds.scm.container.ContainerID;
-import org.apache.hadoop.hdds.scm.container.ContainerInfo;
-import org.apache.hadoop.hdds.scm.container.ContainerManager;
-import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
-import org.apache.hadoop.hdds.scm.container.ContainerReplicaNotFoundException;
-import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
-import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
-import org.apache.hadoop.hdds.scm.ha.SCMContext;
-import org.apache.hadoop.hdds.scm.net.NetworkTopology;
+import org.apache.hadoop.hdds.scm.container.balancer.iteration.ContainerBalanceIteration;
+import org.apache.hadoop.hdds.scm.container.balancer.iteration.IterationResult;
+import org.apache.hadoop.hdds.scm.container.balancer.iteration.IterationMetrics;
import org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo;
-import org.apache.hadoop.hdds.scm.node.NodeManager;
-import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
-import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.util.StringUtils;
+import org.jetbrains.annotations.Nullable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import javax.annotation.Nonnull;
import java.io.IOException;
import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
import java.util.List;
import java.util.Map;
-import java.util.NavigableSet;
import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_NODE_REPORT_INTERVAL;
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_NODE_REPORT_INTERVAL_DEFAULT;
/**
- * Container balancer task performs move of containers between over- and
- * under-utilized datanodes.
+ * Container balancer task performs move of containers between over- and under-utilized datanodes.
*/
-public class ContainerBalancerTask implements Runnable {
+public class ContainerBalancerTask {
- public static final Logger LOG =
- LoggerFactory.getLogger(ContainerBalancerTask.class);
+ public static final Logger LOG = LoggerFactory.getLogger(ContainerBalancerTask.class);
- private NodeManager nodeManager;
- private ContainerManager containerManager;
- private ReplicationManager replicationManager;
- private MoveManager moveManager;
- private OzoneConfiguration ozoneConfiguration;
- private ContainerBalancer containerBalancer;
- private final SCMContext scmContext;
- private double threshold;
- private int totalNodesInCluster;
- private double maxDatanodesRatioToInvolvePerIteration;
- private long maxSizeToMovePerIteration;
- private int countDatanodesInvolvedPerIteration;
- private long sizeScheduledForMoveInLatestIteration;
- // count actual size moved in bytes
- private long sizeActuallyMovedInLatestIteration;
- private int iterations;
- private List unBalancedNodes;
- private List overUtilizedNodes;
- private List underUtilizedNodes;
- private List withinThresholdUtilizedNodes;
- private Set excludeNodes;
- private Set includeNodes;
- private ContainerBalancerConfiguration config;
- private ContainerBalancerMetrics metrics;
- private long clusterCapacity;
- private long clusterRemaining;
- private double clusterAvgUtilisation;
- private PlacementPolicyValidateProxy placementPolicyValidateProxy;
- private NetworkTopology networkTopology;
- private double upperLimit;
- private double lowerLimit;
- private ContainerBalancerSelectionCriteria selectionCriteria;
- private volatile Status taskStatus = Status.RUNNING;
-
- /*
- Since a container can be selected only once during an iteration, these maps
- use it as a primary key to track source to target pairings.
- */
- private final Map containerToSourceMap;
- private final Map containerToTargetMap;
-
- private Set selectedTargets;
- private Set selectedSources;
- private FindTargetStrategy findTargetStrategy;
- private FindSourceStrategy findSourceStrategy;
- private Map>
- moveSelectionToFutureMap;
- private IterationResult iterationResult;
- private int nextIterationIndex;
- private boolean delayStart;
+ private final ContainerBalancer containerBalancer;
+ private final StorageContainerManager scm;
+ private final ContainerBalancerConfiguration config;
+ private final AtomicReference taskStatus = new AtomicReference<>(Status.RUNNING);
+ private ContainerBalanceIteration it;
+ private IterationResult iterationResult = IterationResult.ITERATION_COMPLETED;
/**
* Constructs ContainerBalancerTask with the specified arguments.
*
- * @param scm the storage container manager
- * @param nextIterationIndex next iteration index for continue
- * @param containerBalancer the container balancer
- * @param metrics the metrics
- * @param config the config
+ * @param scm the storage container manager
+ * @param containerBalancer the container balancer
+ * @param config the config
*/
- public ContainerBalancerTask(StorageContainerManager scm,
- int nextIterationIndex,
- ContainerBalancer containerBalancer,
- ContainerBalancerMetrics metrics,
- ContainerBalancerConfiguration config,
- boolean delayStart) {
- this.nodeManager = scm.getScmNodeManager();
- this.containerManager = scm.getContainerManager();
- this.replicationManager = scm.getReplicationManager();
- this.moveManager = scm.getMoveManager();
- this.moveManager.setMoveTimeout(config.getMoveTimeout().toMillis());
- this.moveManager.setReplicationTimeout(
- config.getMoveReplicationTimeout().toMillis());
- this.delayStart = delayStart;
- this.ozoneConfiguration = scm.getConfiguration();
+ public ContainerBalancerTask(
+ @Nonnull StorageContainerManager scm,
+ @Nonnull ContainerBalancer containerBalancer,
+ @Nonnull ContainerBalancerConfiguration config
+ ) {
+ this.scm = scm;
this.containerBalancer = containerBalancer;
this.config = config;
- this.metrics = metrics;
- this.scmContext = scm.getScmContext();
- this.overUtilizedNodes = new ArrayList<>();
- this.underUtilizedNodes = new ArrayList<>();
- this.withinThresholdUtilizedNodes = new ArrayList<>();
- this.unBalancedNodes = new ArrayList<>();
- this.placementPolicyValidateProxy = scm.getPlacementPolicyValidateProxy();
- this.networkTopology = scm.getClusterMap();
- this.nextIterationIndex = nextIterationIndex;
- this.containerToSourceMap = new HashMap<>();
- this.containerToTargetMap = new HashMap<>();
- this.selectedSources = new HashSet<>();
- this.selectedTargets = new HashSet<>();
- findSourceStrategy = new FindSourceGreedy(nodeManager);
}
/**
* Run the container balancer task.
*/
- public void run() {
+ public void run(int nextIterationIndex, boolean delayStart) {
try {
if (delayStart) {
- long delayDuration = ozoneConfiguration.getTimeDuration(
+ long delayDuration = scm.getConfiguration().getTimeDuration(
HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT,
HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT_DEFAULT,
TimeUnit.SECONDS);
- LOG.info("ContainerBalancer will sleep for {} seconds before starting" +
- " balancing.", delayDuration);
+ LOG.info("ContainerBalancer will sleep for {} seconds before starting balancing.", delayDuration);
Thread.sleep(Duration.ofSeconds(delayDuration).toMillis());
}
- balance();
+ balance(nextIterationIndex, config.getIterations());
} catch (Exception e) {
LOG.error("Container Balancer is stopped abnormally, ", e);
} finally {
- synchronized (this) {
- taskStatus = Status.STOPPED;
- }
+ taskStatus.set(Status.STOPPED);
}
}
@@ -190,88 +101,49 @@ public void run() {
* Changes the status from RUNNING to STOPPING.
*/
public void stop() {
- synchronized (this) {
- if (taskStatus == Status.RUNNING) {
- taskStatus = Status.STOPPING;
- }
- }
+ taskStatus.compareAndSet(Status.RUNNING, Status.STOPPING);
}
- private void balance() {
- this.iterations = config.getIterations();
- if (this.iterations == -1) {
- //run balancer infinitely
- this.iterations = Integer.MAX_VALUE;
- }
-
- // nextIterationIndex is the iteration that balancer should start from on
- // leader change or restart
- int i = nextIterationIndex;
- for (; i < iterations && isBalancerRunning(); i++) {
- // reset some variables and metrics for this iteration
- resetState();
- if (config.getTriggerDuEnable()) {
- // before starting a new iteration, we trigger all the datanode
- // to run `du`. this is an aggressive action, with which we can
- // get more precise usage info of all datanodes before moving.
- // this is helpful for container balancer to make more appropriate
- // decisions. this will increase the disk io load of data nodes, so
- // please enable it with caution.
- nodeManager.refreshAllHealthyDnUsageInfo();
- try {
- long nodeReportInterval =
- ozoneConfiguration.getTimeDuration(HDDS_NODE_REPORT_INTERVAL,
- HDDS_NODE_REPORT_INTERVAL_DEFAULT, TimeUnit.MILLISECONDS);
- // one for sending command , one for running du, and one for
- // reporting back make it like this for now, a more suitable
- // value. can be set in the future if needed
- long sleepTime = 3 * nodeReportInterval;
- LOG.info("ContainerBalancer will sleep for {} ms while waiting " +
- "for updated usage information from Datanodes.", sleepTime);
- Thread.sleep(sleepTime);
- } catch (InterruptedException e) {
- LOG.info("Container Balancer was interrupted while waiting for" +
- "datanodes refreshing volume usage info");
- Thread.currentThread().interrupt();
- return;
- }
+ private void balance(int nextIterationIndex, int iterationCount) {
+ ContainerBalancerMetrics metrics = containerBalancer.getMetrics();
+ // NextIterationIndex is the iteration that balancer should start from on leader change or restart.
+ for (int i = nextIterationIndex; i < iterationCount && isBalancerRunning(); ++i) {
+ if (!balancerIsOk()) {
+ return;
}
- if (!isBalancerRunning()) {
+ List datanodeUsageInfos = getDatanodeUsageInfos();
+ if (datanodeUsageInfos == null) {
return;
}
- // initialize this iteration. stop balancing on initialization failure
- if (!initializeIteration()) {
- // just return if the reason for initialization failure is that
- // balancer has been stopped in another thread
- if (!isBalancerRunning()) {
- return;
- }
- // otherwise, try to stop balancer
- tryStopWithSaveConfiguration("Could not initialize " +
- "ContainerBalancer's iteration number " + i);
+ it = new ContainerBalanceIteration(config, scm, datanodeUsageInfos);
+
+ // Initialize this iteration. stop balancing on initialization failure.
+ if (!it.findUnBalancedNodes(this::isBalancerRunning, datanodeUsageInfos)) {
+ // Try to stop balancer.
+ tryStopWithSaveConfiguration("Could not initialize ContainerBalancer's iteration number " + i);
return;
}
- IterationResult iR = doIteration();
+ iterationResult = it.doIteration(this::isBalancerRunning, config);
+ LOG.info("Result of this iteration of Container Balancer: {}", iterationResult);
metrics.incrementNumIterations(1);
- LOG.info("Result of this iteration of Container Balancer: {}", iR);
+ collectMetrics(metrics, it.getMetrics());
- // if no new move option is generated, it means the cluster cannot be
- // balanced anymore; so just stop balancer
- if (iR == IterationResult.CAN_NOT_BALANCE_ANY_MORE) {
- tryStopWithSaveConfiguration(iR.toString());
+ // if no new move option is generated, it means the cluster cannot be balanced anymore; so just stop balancer
+ if (iterationResult == IterationResult.CAN_NOT_BALANCE_ANY_MORE) {
+ tryStopWithSaveConfiguration(iterationResult.toString());
return;
}
// persist next iteration index
- if (iR == IterationResult.ITERATION_COMPLETED) {
+ if (iterationResult == IterationResult.ITERATION_COMPLETED) {
try {
saveConfiguration(config, true, i + 1);
} catch (IOException | TimeoutException e) {
- LOG.warn("Could not persist next iteration index value for " +
- "ContainerBalancer after completing an iteration", e);
+ LOG.warn("Could not persist next iteration index value for ContainerBalancer after completing an iteration",
+ e);
}
}
@@ -280,843 +152,208 @@ private void balance() {
return;
}
- // wait for configured time before starting next iteration, unless
- // this was the final iteration
- if (i != iterations - 1) {
+ // wait for configured time before starting next iteration, unless this was the final iteration
+ if (i != iterationCount - 1) {
try {
Thread.sleep(config.getBalancingInterval().toMillis());
} catch (InterruptedException e) {
- LOG.info("Container Balancer was interrupted while waiting for" +
- " next iteration.");
+ LOG.info("Container Balancer was interrupted while waiting for next iteration.");
Thread.currentThread().interrupt();
return;
}
}
}
-
- tryStopWithSaveConfiguration("Completed all iterations.");
- }
- /**
- * Logs the reason for stop and save configuration and stop the task.
- *
- * @param stopReason a string specifying the reason for stop
- */
- private void tryStopWithSaveConfiguration(String stopReason) {
- synchronized (this) {
- try {
- LOG.info("Save Configuration for stopping. Reason: {}", stopReason);
- saveConfiguration(config, false, 0);
- stop();
- } catch (IOException | TimeoutException e) {
- LOG.warn("Save configuration failed. Reason for " +
- "stopping: {}", stopReason, e);
- }
- }
+ tryStopWithSaveConfiguration("Completed all iterations.");
}
- private void saveConfiguration(ContainerBalancerConfiguration configuration,
- boolean shouldRun, int index)
- throws IOException, TimeoutException {
- if (!isValidSCMState()) {
- LOG.warn("Save configuration is not allowed as not in valid State.");
- return;
- }
- synchronized (this) {
- if (isBalancerRunning()) {
- containerBalancer.saveConfiguration(configuration, shouldRun, index);
- }
- }
+ private static void collectMetrics(@Nonnull ContainerBalancerMetrics metrics, @Nonnull IterationMetrics itMetrics) {
+ metrics.incrementNumContainerMovesScheduled(itMetrics.getScheduledContainerMovesCount());
+ metrics.incrementNumContainerMovesCompleted(itMetrics.getCompletedContainerMovesCount());
+ metrics.incrementNumContainerMovesTimeout(itMetrics.getTimeoutContainerMovesCount());
+ metrics.incrementDataSizeMovedBytes(itMetrics.getMovedBytesCount());
+ metrics.incrementNumContainerMovesFailed(itMetrics.getFailedContainerMovesCount());
+ LOG.info("Iteration Summary. Number of Datanodes involved: {}." +
+ " Size moved: {} ({} Bytes)." +
+ " Number of Container moves completed: {}.",
+ itMetrics.getInvolvedDatanodeCount(),
+ StringUtils.byteDesc(itMetrics.getMovedBytesCount()),
+ itMetrics.getMovedBytesCount(),
+ itMetrics.getCompletedContainerMovesCount());
}
- /**
- * Initializes an iteration during balancing. Recognizes over, under, and
- * within threshold utilized nodes. Decides whether balancing needs to
- * continue or should be stopped.
- *
- * @return true if successfully initialized, otherwise false.
- */
- private boolean initializeIteration() {
- if (!isValidSCMState()) {
- return false;
- }
+ private @Nullable List getDatanodeUsageInfos() {
// sorted list in order from most to least used
- List datanodeUsageInfos =
- nodeManager.getMostOrLeastUsedDatanodes(true);
+ List datanodeUsageInfos = scm.getScmNodeManager().getMostOrLeastUsedDatanodes(true);
if (datanodeUsageInfos.isEmpty()) {
- LOG.warn("Received an empty list of datanodes from Node Manager when " +
- "trying to identify which nodes to balance");
- return false;
- }
-
- this.threshold = config.getThresholdAsRatio();
- this.maxDatanodesRatioToInvolvePerIteration =
- config.getMaxDatanodesRatioToInvolvePerIteration();
- this.maxSizeToMovePerIteration = config.getMaxSizeToMovePerIteration();
- if (config.getNetworkTopologyEnable()) {
- findTargetStrategy = new FindTargetGreedyByNetworkTopology(
- containerManager, placementPolicyValidateProxy,
- nodeManager, networkTopology);
- } else {
- findTargetStrategy = new FindTargetGreedyByUsageInfo(containerManager,
- placementPolicyValidateProxy, nodeManager);
+ LOG.warn("Received an empty list of datanodes from Node Manager when trying to identify which nodes to balance");
+ return null;
}
- this.excludeNodes = config.getExcludeNodes();
- this.includeNodes = config.getIncludeNodes();
// include/exclude nodes from balancing according to configs
- datanodeUsageInfos.removeIf(datanodeUsageInfo -> shouldExcludeDatanode(
- datanodeUsageInfo.getDatanodeDetails()));
-
- this.totalNodesInCluster = datanodeUsageInfos.size();
-
- clusterAvgUtilisation = calculateAvgUtilization(datanodeUsageInfos);
- if (LOG.isDebugEnabled()) {
- LOG.debug("Average utilization of the cluster is {}",
- clusterAvgUtilisation);
- }
-
- // over utilized nodes have utilization(that is, used / capacity) greater
- // than upper limit
- this.upperLimit = clusterAvgUtilisation + threshold;
- // under utilized nodes have utilization(that is, used / capacity) less
- // than lower limit
- this.lowerLimit = clusterAvgUtilisation - threshold;
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("Lower limit for utilization is {} and Upper limit for " +
- "utilization is {}", lowerLimit, upperLimit);
- }
-
- long totalOverUtilizedBytes = 0L, totalUnderUtilizedBytes = 0L;
- // find over and under utilized nodes
- for (DatanodeUsageInfo datanodeUsageInfo : datanodeUsageInfos) {
- if (!isBalancerRunning()) {
- return false;
- }
- double utilization = datanodeUsageInfo.calculateUtilization();
- if (LOG.isDebugEnabled()) {
- LOG.debug("Utilization for node {} with capacity {}B, used {}B, and " +
- "remaining {}B is {}",
- datanodeUsageInfo.getDatanodeDetails().getUuidString(),
- datanodeUsageInfo.getScmNodeStat().getCapacity().get(),
- datanodeUsageInfo.getScmNodeStat().getScmUsed().get(),
- datanodeUsageInfo.getScmNodeStat().getRemaining().get(),
- utilization);
- }
- if (Double.compare(utilization, upperLimit) > 0) {
- overUtilizedNodes.add(datanodeUsageInfo);
- metrics.incrementNumDatanodesUnbalanced(1);
-
- // amount of bytes greater than upper limit in this node
- long overUtilizedBytes = ratioToBytes(
- datanodeUsageInfo.getScmNodeStat().getCapacity().get(),
- utilization) - ratioToBytes(
- datanodeUsageInfo.getScmNodeStat().getCapacity().get(),
- upperLimit);
- totalOverUtilizedBytes += overUtilizedBytes;
- } else if (Double.compare(utilization, lowerLimit) < 0) {
- underUtilizedNodes.add(datanodeUsageInfo);
- metrics.incrementNumDatanodesUnbalanced(1);
-
- // amount of bytes lesser than lower limit in this node
- long underUtilizedBytes = ratioToBytes(
- datanodeUsageInfo.getScmNodeStat().getCapacity().get(),
- lowerLimit) - ratioToBytes(
- datanodeUsageInfo.getScmNodeStat().getCapacity().get(),
- utilization);
- totalUnderUtilizedBytes += underUtilizedBytes;
- } else {
- withinThresholdUtilizedNodes.add(datanodeUsageInfo);
- }
- }
- metrics.incrementDataSizeUnbalancedGB(
- Math.max(totalOverUtilizedBytes, totalUnderUtilizedBytes) /
- OzoneConsts.GB);
- Collections.reverse(underUtilizedNodes);
-
- unBalancedNodes = new ArrayList<>(
- overUtilizedNodes.size() + underUtilizedNodes.size());
- unBalancedNodes.addAll(overUtilizedNodes);
- unBalancedNodes.addAll(underUtilizedNodes);
-
- if (unBalancedNodes.isEmpty()) {
- LOG.info("Did not find any unbalanced Datanodes.");
- return false;
- }
-
- LOG.info("Container Balancer has identified {} Over-Utilized and {} " +
- "Under-Utilized Datanodes that need to be balanced.",
- overUtilizedNodes.size(), underUtilizedNodes.size());
-
- if (LOG.isDebugEnabled()) {
- overUtilizedNodes.forEach(entry -> {
- LOG.debug("Datanode {} {} is Over-Utilized.",
- entry.getDatanodeDetails().getHostName(),
- entry.getDatanodeDetails().getUuid());
- });
-
- underUtilizedNodes.forEach(entry -> {
- LOG.debug("Datanode {} {} is Under-Utilized.",
- entry.getDatanodeDetails().getHostName(),
- entry.getDatanodeDetails().getUuid());
- });
- }
-
- selectionCriteria = new ContainerBalancerSelectionCriteria(config,
- nodeManager, replicationManager, containerManager, findSourceStrategy);
- return true;
- }
-
- private boolean isValidSCMState() {
- if (scmContext.isInSafeMode()) {
- LOG.error("Container Balancer cannot operate while SCM is in Safe Mode.");
- return false;
- }
- if (!scmContext.isLeaderReady()) {
- LOG.warn("Current SCM is not the leader.");
- return false;
- }
- return true;
- }
-
- private IterationResult doIteration() {
- // note that potential and selected targets are updated in the following
- // loop
- //TODO(jacksonyao): take withinThresholdUtilizedNodes as candidate for both
- // source and target
- List potentialTargets = getPotentialTargets();
- findTargetStrategy.reInitialize(potentialTargets, config, upperLimit);
- findSourceStrategy.reInitialize(getPotentialSources(), config, lowerLimit);
-
- moveSelectionToFutureMap = new HashMap<>(unBalancedNodes.size());
- boolean isMoveGeneratedInThisIteration = false;
- iterationResult = IterationResult.ITERATION_COMPLETED;
- boolean canAdaptWhenNearingLimits = true;
- boolean canAdaptOnReachingLimits = true;
-
- // match each source node with a target
- while (true) {
- if (!isBalancerRunning()) {
- iterationResult = IterationResult.ITERATION_INTERRUPTED;
- break;
- }
-
- // break out if we've reached max size to move limit
- if (reachedMaxSizeToMovePerIteration()) {
- break;
- }
-
- /* if balancer is approaching the iteration limits for max datanodes to
- involve, take some action in adaptWhenNearingIterationLimits()
- */
- if (canAdaptWhenNearingLimits) {
- if (adaptWhenNearingIterationLimits()) {
- canAdaptWhenNearingLimits = false;
- }
- }
- if (canAdaptOnReachingLimits) {
- // check if balancer has hit the iteration limits and take some action
- if (adaptOnReachingIterationLimits()) {
- canAdaptOnReachingLimits = false;
- canAdaptWhenNearingLimits = false;
- }
- }
-
- DatanodeDetails source =
- findSourceStrategy.getNextCandidateSourceDataNode();
- if (source == null) {
- // no more source DNs are present
- break;
- }
-
- ContainerMoveSelection moveSelection = matchSourceWithTarget(source);
- if (moveSelection != null) {
- if (processMoveSelection(source, moveSelection)) {
- isMoveGeneratedInThisIteration = true;
- }
- } else {
- // can not find any target for this source
- findSourceStrategy.removeCandidateSourceDataNode(source);
- }
- }
-
- checkIterationResults(isMoveGeneratedInThisIteration);
- return iterationResult;
+ datanodeUsageInfos.removeIf(dnUsageInfo -> shouldExcludeDatanode(config, dnUsageInfo.getDatanodeDetails()));
+ return datanodeUsageInfos;
}
- private boolean processMoveSelection(DatanodeDetails source,
- ContainerMoveSelection moveSelection) {
- ContainerID containerID = moveSelection.getContainerID();
- if (containerToSourceMap.containsKey(containerID) ||
- containerToTargetMap.containsKey(containerID)) {
- LOG.warn("Container {} has already been selected for move from source " +
- "{} to target {} earlier. Not moving this container again.",
- containerID,
- containerToSourceMap.get(containerID),
- containerToTargetMap.get(containerID));
- return false;
- }
-
- ContainerInfo containerInfo;
- try {
- containerInfo =
- containerManager.getContainer(containerID);
- } catch (ContainerNotFoundException e) {
- LOG.warn("Could not get container {} from Container Manager before " +
- "starting a container move", containerID, e);
- return false;
- }
- LOG.info("ContainerBalancer is trying to move container {} with size " +
- "{}B from source datanode {} to target datanode {}",
- containerID.toString(),
- containerInfo.getUsedBytes(),
- source.getUuidString(),
- moveSelection.getTargetNode().getUuidString());
-
- if (moveContainer(source, moveSelection)) {
- // consider move successful for now, and update selection criteria
- updateTargetsAndSelectionCriteria(moveSelection, source);
- }
- return true;
- }
-
- /**
- * Check the iteration results. Result can be:
- * ITERATION_INTERRUPTED if balancing was stopped
- * CAN_NOT_BALANCE_ANY_MORE if no move was generated during this iteration
- *
- * ITERATION_COMPLETED
- * @param isMoveGeneratedInThisIteration whether a move was generated during
- * the iteration
- */
- private void checkIterationResults(boolean isMoveGeneratedInThisIteration) {
- if (!isMoveGeneratedInThisIteration) {
- /*
- If no move was generated during this iteration then we don't need to
- check the move results
- */
- iterationResult = IterationResult.CAN_NOT_BALANCE_ANY_MORE;
- } else {
- checkIterationMoveResults();
- }
+ private boolean balancerIsOk() {
+ boolean result = isBalancerRunning();
+ result = config.getTriggerDuEnable() ? runCommandDU() : result;
+ return result & scmStateIsValid();
}
- /**
- * Checks the results of all move operations when exiting an iteration.
- */
- private void checkIterationMoveResults() {
- this.countDatanodesInvolvedPerIteration = 0;
- CompletableFuture allFuturesResult = CompletableFuture.allOf(
- moveSelectionToFutureMap.values()
- .toArray(new CompletableFuture[moveSelectionToFutureMap.size()]));
+ private boolean runCommandDU() {
+ // Before starting a new iteration, we trigger all the datanode to run `du`.
+ // This is an aggressive action, with which we can get more precise usage info of all datanodes before moving.
+ // This is helpful for container balancer to make more appropriate decisions.
+ // This will increase the disk io load of data nodes, so please enable it with caution.
+ scm.getScmNodeManager().refreshAllHealthyDnUsageInfo();
try {
- allFuturesResult.get(config.getMoveTimeout().toMillis(),
- TimeUnit.MILLISECONDS);
+ long nodeReportInterval = scm.getConfiguration().getTimeDuration(
+ HDDS_NODE_REPORT_INTERVAL,
+ HDDS_NODE_REPORT_INTERVAL_DEFAULT,
+ TimeUnit.MILLISECONDS
+ );
+ // One for sending command, one for running du, and one for reporting back make it like this for now,
+ // a more suitable value. can be set in the future if needed
+ long sleepTime = 3 * nodeReportInterval;
+ LOG.info("ContainerBalancer will sleep for {} ms while waiting for updated usage information from Datanodes.",
+ sleepTime);
+ Thread.sleep(sleepTime);
+ return true;
} catch (InterruptedException e) {
- LOG.warn("Container balancer is interrupted");
+ LOG.info("Container Balancer was interrupted while waiting for datanodes refreshing volume usage info");
Thread.currentThread().interrupt();
- } catch (TimeoutException e) {
- long timeoutCounts = cancelMovesThatExceedTimeoutDuration();
- LOG.warn("{} Container moves are canceled.", timeoutCounts);
- metrics.incrementNumContainerMovesTimeoutInLatestIteration(timeoutCounts);
- } catch (ExecutionException e) {
- LOG.error("Got exception while checkIterationMoveResults", e);
+ return false;
}
-
- countDatanodesInvolvedPerIteration =
- selectedSources.size() + selectedTargets.size();
- metrics.incrementNumDatanodesInvolvedInLatestIteration(
- countDatanodesInvolvedPerIteration);
- metrics.incrementNumContainerMovesScheduled(
- metrics.getNumContainerMovesScheduledInLatestIteration());
- metrics.incrementNumContainerMovesCompleted(
- metrics.getNumContainerMovesCompletedInLatestIteration());
- metrics.incrementNumContainerMovesTimeout(
- metrics.getNumContainerMovesTimeoutInLatestIteration());
- metrics.incrementDataSizeMovedGBInLatestIteration(
- sizeActuallyMovedInLatestIteration / OzoneConsts.GB);
- metrics.incrementDataSizeMovedGB(
- metrics.getDataSizeMovedGBInLatestIteration());
- metrics.incrementNumContainerMovesFailed(
- metrics.getNumContainerMovesFailedInLatestIteration());
- LOG.info("Iteration Summary. Number of Datanodes involved: {}. Size " +
- "moved: {} ({} Bytes). Number of Container moves completed: {}.",
- countDatanodesInvolvedPerIteration,
- StringUtils.byteDesc(sizeActuallyMovedInLatestIteration),
- sizeActuallyMovedInLatestIteration,
- metrics.getNumContainerMovesCompletedInLatestIteration());
}
/**
- * Cancels container moves that are not yet done. Note that if a move
- * command has already been sent out to a Datanode, we don't yet have the
- * capability to cancel it. However, those commands in the DN should time out
- * if they haven't been processed yet.
+ * Logs the reason for stop and save configuration and stop the task.
*
- * @return number of moves that did not complete (timed out) and were
- * cancelled.
+ * @param stopReason a string specifying the reason for stop
*/
- private long cancelMovesThatExceedTimeoutDuration() {
- Set>>
- entries = moveSelectionToFutureMap.entrySet();
- Iterator>>
- iterator = entries.iterator();
-
- int numCancelled = 0;
- // iterate through all moves and cancel ones that aren't done yet
- while (iterator.hasNext()) {
- Map.Entry>
- entry = iterator.next();
- if (!entry.getValue().isDone()) {
- LOG.warn("Container move timed out for container {} from source {}" +
- " to target {}.", entry.getKey().getContainerID(),
- containerToSourceMap.get(entry.getKey().getContainerID())
- .getUuidString(),
- entry.getKey().getTargetNode().getUuidString());
-
- entry.getValue().cancel(true);
- numCancelled += 1;
- }
+ private void tryStopWithSaveConfiguration(@Nonnull String stopReason) {
+ try {
+ LOG.info("Save Configuration for stopping. Reason: {}", stopReason);
+ saveConfiguration(config, false, 0);
+ stop();
+ } catch (IOException | TimeoutException e) {
+ LOG.warn("Save configuration failed. Reason for stopping: {}", stopReason, e);
}
-
- return numCancelled;
}
- /**
- * Match a source datanode with a target datanode and identify the container
- * to move.
- *
- * @return ContainerMoveSelection containing the selected target and container
- */
- private ContainerMoveSelection matchSourceWithTarget(DatanodeDetails source) {
- NavigableSet candidateContainers =
- selectionCriteria.getCandidateContainers(source,
- sizeScheduledForMoveInLatestIteration);
-
- if (candidateContainers.isEmpty()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("ContainerBalancer could not find any candidate containers " +
- "for datanode {}", source.getUuidString());
- }
- return null;
- }
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("ContainerBalancer is finding suitable target for source " +
- "datanode {}", source.getUuidString());
+ private void saveConfiguration(@Nonnull ContainerBalancerConfiguration configuration, boolean shouldRun, int index)
+ throws IOException, TimeoutException {
+ if (!scmStateIsValid()) {
+ LOG.warn("Save configuration is not allowed as not in valid State.");
+ return;
}
- ContainerMoveSelection moveSelection =
- findTargetStrategy.findTargetForContainerMove(
- source, candidateContainers);
-
- if (moveSelection == null) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("ContainerBalancer could not find a suitable target for " +
- "source node {}.", source.getUuidString());
+ synchronized (this) {
+ if (isBalancerRunning()) {
+ containerBalancer.saveConfiguration(configuration, shouldRun, index);
}
- return null;
- }
- LOG.info("ContainerBalancer matched source datanode {} with target " +
- "datanode {} for container move.", source.getUuidString(),
- moveSelection.getTargetNode().getUuidString());
-
- return moveSelection;
- }
-
- private boolean reachedMaxSizeToMovePerIteration() {
- // since candidate containers in ContainerBalancerSelectionCriteria are
- // filtered out according to this limit, balancer should not have crossed it
- if (sizeScheduledForMoveInLatestIteration >= maxSizeToMovePerIteration) {
- LOG.warn("Reached max size to move limit. {} bytes have already been" +
- " scheduled for balancing and the limit is {} bytes.",
- sizeScheduledForMoveInLatestIteration, maxSizeToMovePerIteration);
- return true;
- }
- return false;
- }
-
- /**
- * Restricts potential target datanodes to nodes that have
- * already been selected if balancer is one datanode away from
- * "datanodes.involved.max.percentage.per.iteration" limit.
- * @return true if potential targets were restricted, else false
- */
- private boolean adaptWhenNearingIterationLimits() {
- // check if we're nearing max datanodes to involve
- int maxDatanodesToInvolve =
- (int) (maxDatanodesRatioToInvolvePerIteration * totalNodesInCluster);
- if (countDatanodesInvolvedPerIteration + 1 == maxDatanodesToInvolve) {
- /* We're one datanode away from reaching the limit. Restrict potential
- targets to targets that have already been selected.
- */
- findTargetStrategy.resetPotentialTargets(selectedTargets);
- LOG.debug("Approaching max datanodes to involve limit. {} datanodes " +
- "have already been selected for balancing and the limit is " +
- "{}. Only already selected targets can be selected as targets" +
- " now.",
- countDatanodesInvolvedPerIteration, maxDatanodesToInvolve);
- return true;
}
-
- // return false if we didn't adapt
- return false;
- }
-
- /**
- * Restricts potential source and target datanodes to nodes that have
- * already been selected if balancer has reached
- * "datanodes.involved.max.percentage.per.iteration" limit.
- * @return true if potential sources and targets were restricted, else false
- */
- private boolean adaptOnReachingIterationLimits() {
- // check if we've reached max datanodes to involve limit
- int maxDatanodesToInvolve =
- (int) (maxDatanodesRatioToInvolvePerIteration * totalNodesInCluster);
- if (countDatanodesInvolvedPerIteration == maxDatanodesToInvolve) {
- // restrict both to already selected sources and targets
- findTargetStrategy.resetPotentialTargets(selectedTargets);
- findSourceStrategy.resetPotentialSources(selectedSources);
- LOG.debug("Reached max datanodes to involve limit. {} datanodes " +
- "have already been selected for balancing and the limit " +
- "is {}. Only already selected sources and targets can be " +
- "involved in balancing now.",
- countDatanodesInvolvedPerIteration, maxDatanodesToInvolve);
- return true;
- }
-
- // return false if we didn't adapt
- return false;
}
- /**
- * Asks {@link ReplicationManager} or {@link MoveManager} to move the
- * specified container from source to target.
- *
- * @param source the source datanode
- * @param moveSelection the selected container to move and target datanode
- * @return false if an exception occurred or the move completed with a
- * result other than MoveManager.MoveResult.COMPLETED. Returns true
- * if the move completed with MoveResult.COMPLETED or move is not yet done
- */
- private boolean moveContainer(DatanodeDetails source,
- ContainerMoveSelection moveSelection) {
- ContainerID containerID = moveSelection.getContainerID();
- CompletableFuture future;
- try {
- ContainerInfo containerInfo = containerManager.getContainer(containerID);
-
- /*
- If LegacyReplicationManager is enabled, ReplicationManager will
- redirect to it. Otherwise, use MoveManager.
- */
- if (replicationManager.getConfig().isLegacyEnabled()) {
- future = replicationManager
- .move(containerID, source, moveSelection.getTargetNode());
- } else {
- future = moveManager.move(containerID, source,
- moveSelection.getTargetNode());
- }
- metrics.incrementNumContainerMovesScheduledInLatestIteration(1);
-
- future = future.whenComplete((result, ex) -> {
- metrics.incrementCurrentIterationContainerMoveMetric(result, 1);
- if (ex != null) {
- LOG.info("Container move for container {} from source {} to " +
- "target {} failed with exceptions.",
- containerID.toString(),
- source.getUuidString(),
- moveSelection.getTargetNode().getUuidString(), ex);
- metrics.incrementNumContainerMovesFailedInLatestIteration(1);
- } else {
- if (result == MoveManager.MoveResult.COMPLETED) {
- sizeActuallyMovedInLatestIteration +=
- containerInfo.getUsedBytes();
- LOG.debug("Container move completed for container {} from " +
- "source {} to target {}", containerID,
- source.getUuidString(),
- moveSelection.getTargetNode().getUuidString());
- } else {
- LOG.warn(
- "Container move for container {} from source {} to target" +
- " {} failed: {}",
- moveSelection.getContainerID(), source.getUuidString(),
- moveSelection.getTargetNode().getUuidString(), result);
- }
- }
- });
- } catch (ContainerNotFoundException e) {
- LOG.warn("Could not find Container {} for container move",
- containerID, e);
- metrics.incrementNumContainerMovesFailedInLatestIteration(1);
- return false;
- } catch (NodeNotFoundException | TimeoutException |
- ContainerReplicaNotFoundException e) {
- LOG.warn("Container move failed for container {}", containerID, e);
- metrics.incrementNumContainerMovesFailedInLatestIteration(1);
+ private boolean scmStateIsValid() {
+ if (scm.getScmContext().isInSafeMode()) {
+ LOG.error("Container Balancer cannot operate while SCM is in Safe Mode.");
return false;
}
-
- /*
- If the future hasn't failed yet, put it in moveSelectionToFutureMap for
- processing later
- */
- if (future.isDone()) {
- if (future.isCompletedExceptionally()) {
- return false;
- } else {
- MoveManager.MoveResult result = future.join();
- moveSelectionToFutureMap.put(moveSelection, future);
- return result == MoveManager.MoveResult.COMPLETED;
- }
- } else {
- moveSelectionToFutureMap.put(moveSelection, future);
- return true;
- }
- }
-
- /**
- * Update targets, sources, and selection criteria after a move.
- *
- * @param moveSelection latest selected target datanode and container
- * @param source the source datanode
- */
- private void updateTargetsAndSelectionCriteria(
- ContainerMoveSelection moveSelection, DatanodeDetails source) {
- ContainerID containerID = moveSelection.getContainerID();
- DatanodeDetails target = moveSelection.getTargetNode();
-
- // count source if it has not been involved in move earlier
- if (!selectedSources.contains(source)) {
- countDatanodesInvolvedPerIteration += 1;
- }
- // count target if it has not been involved in move earlier
- if (!selectedTargets.contains(target)) {
- countDatanodesInvolvedPerIteration += 1;
- }
-
- incSizeSelectedForMoving(source, moveSelection);
- containerToSourceMap.put(containerID, source);
- containerToTargetMap.put(containerID, target);
- selectedTargets.add(target);
- selectedSources.add(source);
- selectionCriteria.setSelectedContainers(
- new HashSet<>(containerToSourceMap.keySet()));
- }
-
- /**
- * Calculates the number of used bytes given capacity and utilization ratio.
- *
- * @param nodeCapacity capacity of the node.
- * @param utilizationRatio used space by capacity ratio of the node.
- * @return number of bytes
- */
- private long ratioToBytes(Long nodeCapacity, double utilizationRatio) {
- return (long) (nodeCapacity * utilizationRatio);
- }
-
- /**
- * Calculates the average utilization for the specified nodes.
- * Utilization is (capacity - remaining) divided by capacity.
- *
- * @param nodes List of DatanodeUsageInfo to find the average utilization for
- * @return Average utilization value
- */
- @VisibleForTesting
- double calculateAvgUtilization(List nodes) {
- if (nodes.size() == 0) {
- LOG.warn("No nodes to calculate average utilization for in " +
- "ContainerBalancer.");
- return 0;
- }
- SCMNodeStat aggregatedStats = new SCMNodeStat(
- 0, 0, 0);
- for (DatanodeUsageInfo node : nodes) {
- aggregatedStats.add(node.getScmNodeStat());
+ if (!scm.getScmContext().isLeaderReady()) {
+ LOG.warn("Current SCM is not the leader.");
+ return false;
}
- clusterCapacity = aggregatedStats.getCapacity().get();
- clusterRemaining = aggregatedStats.getRemaining().get();
-
- return (clusterCapacity - clusterRemaining) / (double) clusterCapacity;
- }
-
- /**
- * Get potential targets for container move. Potential targets are under
- * utilized and within threshold utilized nodes.
- *
- * @return A list of potential target DatanodeUsageInfo.
- */
- private List getPotentialTargets() {
- //TODO(jacksonyao): take withinThresholdUtilizedNodes as candidate for both
- // source and target
- return underUtilizedNodes;
+ return true;
}
/**
- * Get potential sourecs for container move. Potential sourecs are over
- * utilized and within threshold utilized nodes.
+ * Consults the configurations
+ * {@link ContainerBalancerConfiguration#includeNodes} and
+ * {@link ContainerBalancerConfiguration#excludeNodes} to check
+ * if the specified Datanode should be excluded from balancing.
*
- * @return A list of potential source DatanodeUsageInfo.
- */
- private List getPotentialSources() {
- //TODO(jacksonyao): take withinThresholdUtilizedNodes as candidate for both
- // source and target
- return overUtilizedNodes;
- }
-
- /**
- * Consults the configurations {@link ContainerBalancerTask#includeNodes} and
- * {@link ContainerBalancerTask#excludeNodes} to check if the specified
- * Datanode should be excluded from balancing.
+ * @param config
* @param datanode DatanodeDetails to check
+ *
* @return true if Datanode should be excluded, else false
*/
- private boolean shouldExcludeDatanode(DatanodeDetails datanode) {
- if (excludeNodes.contains(datanode.getHostName()) ||
- excludeNodes.contains(datanode.getIpAddress())) {
+ private static boolean shouldExcludeDatanode(
+ @Nonnull ContainerBalancerConfiguration config,
+ @Nonnull DatanodeDetails datanode
+ ) {
+ Set excludeNodes = config.getExcludeNodes();
+ String hostName = datanode.getHostName();
+ String ipAddress = datanode.getIpAddress();
+ if (excludeNodes.contains(hostName) || excludeNodes.contains(ipAddress)) {
return true;
- } else if (!includeNodes.isEmpty()) {
- return !includeNodes.contains(datanode.getHostName()) &&
- !includeNodes.contains(datanode.getIpAddress());
- }
- return false;
- }
-
- /**
- * Updates conditions for balancing, such as total size moved by balancer,
- * total size that has entered a datanode, and total size that has left a
- * datanode in this iteration.
- *
- * @param source source datanode
- * @param moveSelection selected target datanode and container
- */
- private void incSizeSelectedForMoving(DatanodeDetails source,
- ContainerMoveSelection moveSelection) {
- DatanodeDetails target = moveSelection.getTargetNode();
- ContainerInfo container;
- try {
- container =
- containerManager.getContainer(moveSelection.getContainerID());
- } catch (ContainerNotFoundException e) {
- LOG.warn("Could not find Container {} while matching source and " +
- "target nodes in ContainerBalancer",
- moveSelection.getContainerID(), e);
- return;
+ } else {
+ Set includeNodes = config.getIncludeNodes();
+ return !includeNodes.isEmpty() && !includeNodes.contains(hostName) && !includeNodes.contains(ipAddress);
}
- long size = container.getUsedBytes();
- sizeScheduledForMoveInLatestIteration += size;
-
- // update sizeLeavingNode map with the recent moveSelection
- findSourceStrategy.increaseSizeLeaving(source, size);
-
- // update sizeEnteringNode map with the recent moveSelection
- findTargetStrategy.increaseSizeEntering(target, size);
- }
-
- /**
- * Resets some variables and metrics for this iteration.
- */
- private void resetState() {
- moveManager.resetState();
- this.clusterCapacity = 0L;
- this.clusterRemaining = 0L;
- this.overUtilizedNodes.clear();
- this.underUtilizedNodes.clear();
- this.unBalancedNodes.clear();
- this.containerToSourceMap.clear();
- this.containerToTargetMap.clear();
- this.selectedSources.clear();
- this.selectedTargets.clear();
- this.countDatanodesInvolvedPerIteration = 0;
- this.sizeScheduledForMoveInLatestIteration = 0;
- this.sizeActuallyMovedInLatestIteration = 0;
- metrics.resetDataSizeMovedGBInLatestIteration();
- metrics.resetNumContainerMovesCompletedInLatestIteration();
- metrics.resetNumContainerMovesTimeoutInLatestIteration();
- metrics.resetNumDatanodesInvolvedInLatestIteration();
- metrics.resetDataSizeUnbalancedGB();
- metrics.resetNumDatanodesUnbalanced();
- metrics.resetNumContainerMovesFailedInLatestIteration();
}
/**
- * Checks if ContainerBalancerTask is currently running.
+ * Gets the list of under utilized nodes in the cluster.
*
- * @return true if the status is RUNNING, otherwise false
+ * @return List of DatanodeUsageInfo containing under utilized nodes.
*/
- private boolean isBalancerRunning() {
- return taskStatus == Status.RUNNING;
+ @VisibleForTesting
+ List getUnderUtilizedNodes() {
+ return it.getUnderUtilizedNodes();
}
/**
- * Gets the list of unBalanced nodes, that is, the over and under utilized
- * nodes in the cluster.
+ * Gets the list of over utilized nodes in the cluster.
*
- * @return List of DatanodeUsageInfo containing unBalanced nodes.
+ * @return List of DatanodeUsageInfo containing over utilized nodes.
*/
@VisibleForTesting
- List getUnBalancedNodes() {
- return unBalancedNodes;
+ List getOverUtilizedNodes() {
+ return it.getOverUtilizedNodes();
}
/**
* Gets a map with selected containers and their source datanodes.
+ *
* @return map with mappings from {@link ContainerID} to
* {@link DatanodeDetails}
*/
@VisibleForTesting
Map getContainerToSourceMap() {
- return containerToSourceMap;
+ return it.getContainerToSourceMap();
}
/**
* Gets a map with selected containers and target datanodes.
- * @return map with mappings from {@link ContainerID} to
- * {@link DatanodeDetails}.
+ *
+ * @return map with mappings from {@link ContainerID} to {@link DatanodeDetails}.
*/
@VisibleForTesting
Map getContainerToTargetMap() {
- return containerToTargetMap;
+ return it.getContainerToTargetMap();
}
@VisibleForTesting
Set getSelectedTargets() {
- return selectedTargets;
+ return it.getSelectedTargets();
}
@VisibleForTesting
- int getCountDatanodesInvolvedPerIteration() {
- return countDatanodesInvolvedPerIteration;
+ public IterationMetrics getIterationMetrics() {
+ return it.getMetrics();
}
@VisibleForTesting
- public long getSizeScheduledForMoveInLatestIteration() {
- return sizeScheduledForMoveInLatestIteration;
- }
-
- public ContainerBalancerMetrics getMetrics() {
- return metrics;
- }
-
- @VisibleForTesting
- IterationResult getIterationResult() {
+ public IterationResult getIterationResult() {
return iterationResult;
}
- @VisibleForTesting
- void setConfig(ContainerBalancerConfiguration config) {
- this.config = config;
- }
-
- @VisibleForTesting
- void setTaskStatus(Status taskStatus) {
- this.taskStatus = taskStatus;
+ private boolean isBalancerRunning() {
+ return taskStatus.get() == Status.RUNNING;
}
public Status getBalancerStatus() {
- return taskStatus;
+ return taskStatus.get();
}
@Override
@@ -1127,19 +364,10 @@ public String toString() {
return status + config.toString();
}
- /**
- * The result of {@link ContainerBalancerTask#doIteration()}.
- */
- enum IterationResult {
- ITERATION_COMPLETED,
- ITERATION_INTERRUPTED,
- CAN_NOT_BALANCE_ANY_MORE
- }
-
/**
* The status of {@link ContainerBalancerTask}.
*/
- enum Status {
+ public enum Status {
RUNNING,
STOPPING,
STOPPED
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerMoveSelection.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerMoveSelection.java
index 1a8cbee09d0a..e0d916542a8f 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerMoveSelection.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerMoveSelection.java
@@ -20,17 +20,20 @@
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.container.ContainerID;
+import javax.annotation.Nonnull;
+
/**
* This class represents a target datanode and the container to be moved from
* a source to that target.
*/
public class ContainerMoveSelection {
- private DatanodeDetails targetNode;
- private ContainerID containerID;
+ private final DatanodeDetails targetNode;
+ private final ContainerID containerID;
public ContainerMoveSelection(
- DatanodeDetails targetNode,
- ContainerID containerID) {
+ @Nonnull DatanodeDetails targetNode,
+ @Nonnull ContainerID containerID
+ ) {
this.targetNode = targetNode;
this.containerID = containerID;
}
@@ -39,17 +42,7 @@ public DatanodeDetails getTargetNode() {
return targetNode;
}
- public void setTargetNode(
- DatanodeDetails targetNode) {
- this.targetNode = targetNode;
- }
-
public ContainerID getContainerID() {
return containerID;
}
-
- public void setContainerID(
- ContainerID containerID) {
- this.containerID = containerID;
- }
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindTargetGreedyByNetworkTopology.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindTargetGreedyByNetworkTopology.java
deleted file mode 100644
index 87feae4981d5..000000000000
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindTargetGreedyByNetworkTopology.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdds.scm.container.balancer;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.scm.PlacementPolicyValidateProxy;
-import org.apache.hadoop.hdds.scm.container.ContainerManager;
-import org.apache.hadoop.hdds.scm.net.NetworkTopology;
-import org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo;
-import org.apache.hadoop.hdds.scm.node.NodeManager;
-import org.jetbrains.annotations.NotNull;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.LinkedList;
-import java.util.List;
-
-/**
- * an implementation of FindTargetGreedy, which will always select the
- * target with the shortest distance according to network topology
- * distance to the give source datanode.
- */
-public class FindTargetGreedyByNetworkTopology
- extends AbstractFindTargetGreedy {
-
- private NetworkTopology networkTopology;
- private List potentialTargets;
-
- public FindTargetGreedyByNetworkTopology(
- ContainerManager containerManager,
- PlacementPolicyValidateProxy placementPolicyValidateProxy,
- NodeManager nodeManager,
- NetworkTopology networkTopology) {
- super(containerManager, placementPolicyValidateProxy, nodeManager);
- setLogger(LoggerFactory.getLogger(FindTargetGreedyByNetworkTopology.class));
- potentialTargets = new LinkedList<>();
- setPotentialTargets(potentialTargets);
- this.networkTopology = networkTopology;
- }
-
- /**
- * sort potentialTargets for specified source datanode according to
- * network topology.
- * @param source the specified source datanode
- */
- @VisibleForTesting
- public void sortTargetForSource(DatanodeDetails source) {
- Collections.sort(potentialTargets,
- (DatanodeUsageInfo da, DatanodeUsageInfo db) -> {
- DatanodeDetails a = da.getDatanodeDetails();
- DatanodeDetails b = db.getDatanodeDetails();
- // sort by network topology first
- int distanceToA = networkTopology.getDistanceCost(source, a);
- int distanceToB = networkTopology.getDistanceCost(source, b);
- if (distanceToA != distanceToB) {
- return distanceToA - distanceToB;
- }
- // if distance to source is equal , sort by usage
- return compareByUsage(da, db);
- });
- }
-
- /**
- * Resets the collection of target datanode usage info that will be
- * considered for balancing. Gets the latest usage info from node manager.
- * @param targets collection of target {@link DatanodeDetails} that
- * containers can move to
- */
- @Override
- public void resetPotentialTargets(
- @NotNull Collection targets) {
- // create DatanodeUsageInfo from DatanodeDetails
- List usageInfos = new ArrayList<>(targets.size());
- targets.forEach(datanodeDetails -> usageInfos.add(
- getNodeManager().getUsageInfo(datanodeDetails)));
-
- super.resetTargets(usageInfos);
- }
-
-}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindTargetGreedyByUsageInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindTargetGreedyByUsageInfo.java
deleted file mode 100644
index 71a338db9229..000000000000
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindTargetGreedyByUsageInfo.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdds.scm.container.balancer;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.scm.PlacementPolicyValidateProxy;
-import org.apache.hadoop.hdds.scm.container.ContainerManager;
-import org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo;
-import org.apache.hadoop.hdds.scm.node.NodeManager;
-import org.jetbrains.annotations.NotNull;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.TreeSet;
-
-/**
- * an implementation of FindTargetGreedy, which will always select the
- * target with the lowest space usage.
- */
-public class FindTargetGreedyByUsageInfo extends AbstractFindTargetGreedy {
- public FindTargetGreedyByUsageInfo(
- ContainerManager containerManager,
- PlacementPolicyValidateProxy placementPolicyValidateProxy,
- NodeManager nodeManager) {
- super(containerManager, placementPolicyValidateProxy, nodeManager);
- setLogger(LoggerFactory.getLogger(FindTargetGreedyByUsageInfo.class));
- setPotentialTargets(new TreeSet<>((a, b) -> compareByUsage(a, b)));
- }
-
- /**
- * do nothing , since TreeSet is ordered itself.
- */
- @VisibleForTesting
- public void sortTargetForSource(DatanodeDetails source) {
- //noop, Treeset is naturally sorted.
- return;
- }
-
- /**
- * Resets the collection of target datanode usage info that will be
- * considered for balancing. Gets the latest usage info from node manager.
- * @param targets collection of target {@link DatanodeDetails} that
- * containers can move to
- */
- @Override
- public void resetPotentialTargets(
- @NotNull Collection targets) {
- // create DatanodeUsageInfo from DatanodeDetails
- List usageInfos = new ArrayList<>(targets.size());
- targets.forEach(datanodeDetails -> usageInfos.add(
- getNodeManager().getUsageInfo(datanodeDetails)));
-
- super.resetTargets(usageInfos);
- }
-
-}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/MoveManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/MoveManager.java
index 0d7be7fbd096..b6b79d6ceea0 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/MoveManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/MoveManager.java
@@ -138,7 +138,7 @@ Pair, MoveDataNodePair>> getPendingMove() {
return pendingMoves;
}
- void resetState() {
+ public void resetState() {
pendingMoves.clear();
}
@@ -205,7 +205,7 @@ private void startMove(
* @param src source datanode
* @param tgt target datanode
*/
- CompletableFuture move(
+ public CompletableFuture move(
ContainerID cid, DatanodeDetails src, DatanodeDetails tgt)
throws ContainerNotFoundException, NodeNotFoundException,
ContainerReplicaNotFoundException {
@@ -496,11 +496,11 @@ public void opCompleted(ContainerReplicaOp op, ContainerID containerID,
}
}
- void setMoveTimeout(long moveTimeout) {
+ public void setMoveTimeout(long moveTimeout) {
this.moveTimeout = moveTimeout;
}
- void setReplicationTimeout(long replicationTimeout) {
+ public void setReplicationTimeout(long replicationTimeout) {
this.replicationTimeout = replicationTimeout;
}
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/AbstractFindTargetGreedy.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/AbstractFindTargetGreedy.java
new file mode 100644
index 000000000000..a2609cf6e509
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/AbstractFindTargetGreedy.java
@@ -0,0 +1,247 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.scm.container.balancer.iteration;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
+import org.apache.hadoop.hdds.scm.PlacementPolicyValidateProxy;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
+import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerConfiguration;
+import org.apache.hadoop.hdds.scm.container.balancer.ContainerMoveSelection;
+import org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+
+/**
+ * Find a target for a source datanode with greedy strategy.
+ */
+abstract class AbstractFindTargetGreedy implements FindTargetStrategy {
+ private final Logger logger;
+ private final ContainerManager containerManager;
+ private final PlacementPolicyValidateProxy placementPolicyValidateProxy;
+ private final Map sizeEnteringNode = new HashMap<>();
+ private final NodeManager nodeManager;
+
+ protected AbstractFindTargetGreedy(@Nonnull StorageContainerManager scm, @Nonnull Class> findTargetClazz) {
+ containerManager = scm.getContainerManager();
+ placementPolicyValidateProxy = scm.getPlacementPolicyValidateProxy();
+ nodeManager = scm.getScmNodeManager();
+ logger = LoggerFactory.getLogger(findTargetClazz);
+ }
+
+ protected int compareByUsage(@Nonnull DatanodeUsageInfo a, @Nonnull DatanodeUsageInfo b) {
+ double currentUsageOfA = a.calculateUtilization(sizeEnteringNode.get(a.getDatanodeDetails()));
+ double currentUsageOfB = b.calculateUtilization(sizeEnteringNode.get(b.getDatanodeDetails()));
+ int ret = Double.compare(currentUsageOfA, currentUsageOfB);
+ if (ret != 0) {
+ return ret;
+ }
+ UUID uuidA = a.getDatanodeDetails().getUuid();
+ UUID uuidB = b.getDatanodeDetails().getUuid();
+ return uuidA.compareTo(uuidB);
+ }
+
+ @Override
+ public @Nullable ContainerMoveSelection findTargetForContainerMove(
+ @Nonnull DatanodeDetails source,
+ @Nonnull Set candidateContainers,
+ long maxSizeEnteringTarget,
+ double upperLimit
+ ) {
+ sortTargetForSource(source);
+ for (DatanodeUsageInfo targetInfo : getPotentialTargets()) {
+ DatanodeDetails target = targetInfo.getDatanodeDetails();
+ for (ContainerID container : candidateContainers) {
+ Set replicatedDnDetails = new HashSet<>();
+ ContainerInfo containerInfo;
+ try {
+ containerManager.getContainerReplicas(container).forEach(
+ replica -> replicatedDnDetails.add(replica.getDatanodeDetails())
+ );
+ containerInfo = containerManager.getContainer(container);
+ } catch (ContainerNotFoundException e) {
+ logger.warn("Could not get Container {} from Container Manager for obtaining replicas in Container Balancer.",
+ container, e);
+ continue;
+ }
+
+ boolean noTargetsInReplicas = !replicatedDnDetails.contains(target);
+ if (noTargetsInReplicas &&
+ canSizeEnterTarget(target, containerInfo.getUsedBytes(), maxSizeEnteringTarget, upperLimit) &&
+ containerMoveSatisfiesPlacementPolicy(containerInfo, replicatedDnDetails, source, target)
+ ) {
+ return new ContainerMoveSelection(target, container);
+ }
+ }
+ }
+ logger.info("Container Balancer could not find a target for source datanode {}", source.getUuidString());
+ return null;
+ }
+
+ /**
+ * Checks if container being present in target instead of source satisfies the placement policy.
+ *
+ * @param containerInfo info about container to be moved from
+ * source to target
+ * @param replicatedDnDetails set of replicas of the given container
+ * @param source source datanode for container move
+ * @param target target datanode for container move
+ * @return true if placement policy is satisfied, otherwise false
+ */
+ private boolean containerMoveSatisfiesPlacementPolicy(
+ @Nonnull ContainerInfo containerInfo,
+ @Nonnull Set replicatedDnDetails,
+ @Nonnull DatanodeDetails source,
+ @Nonnull DatanodeDetails target
+ ) {
+ List targetDnDetails = new ArrayList<>();
+ replicatedDnDetails.forEach(datanodeDetails -> {
+ if (!datanodeDetails.equals(source)) {
+ targetDnDetails.add(datanodeDetails);
+ }
+ });
+ targetDnDetails.add(target);
+
+ ContainerPlacementStatus placementStatus = placementPolicyValidateProxy
+ .validateContainerPlacement(targetDnDetails, containerInfo);
+
+ boolean isPolicySatisfied = placementStatus.isPolicySatisfied();
+ if (!isPolicySatisfied) {
+ logger.debug("Moving container {} from source {} to target {} will not satisfy placement policy.",
+ containerInfo.getContainerID(),
+ source.getUuidString(),
+ target.getUuidString());
+ }
+ return isPolicySatisfied;
+ }
+
+ /**
+ * Checks if specified size can enter specified target datanode according to {@link ContainerBalancerConfiguration}
+ * "size.entering.target.max".
+ *
+ * @param target target datanode in which size is entering
+ * @param size size in bytes
+ * @return true if size can enter target, else false
+ */
+ private boolean canSizeEnterTarget(
+ @Nonnull DatanodeDetails target,
+ long size,
+ long maxSizeEnteringTarget,
+ double upperLimit
+ ) {
+ if (sizeEnteringNode.containsKey(target)) {
+ long sizeEnteringAfterMove = sizeEnteringNode.get(target) + size;
+ // Size can be moved into target datanode only when the following two condition are met.
+ // 1. SizeEnteringAfterMove does not succeed the configured MaxSizeEnteringTarget
+ // 2. Current usage of target datanode plus sizeEnteringAfterMove is smaller than or equal to upperLimit
+ if (sizeEnteringAfterMove > maxSizeEnteringTarget) {
+ logger.debug("{} bytes cannot enter datanode {} because 'size.entering.target.max' limit is {} and {} bytes " +
+ "have already entered.",
+ size,
+ target.getUuidString(),
+ maxSizeEnteringTarget,
+ sizeEnteringNode.get(target));
+ return false;
+ }
+ DatanodeUsageInfo usageInfo = nodeManager.getUsageInfo(target);
+ if (Double.compare(usageInfo.calculateUtilization(sizeEnteringAfterMove), upperLimit) > 0) {
+ logger.debug("{} bytes cannot enter datanode {} because its utilization will exceed the upper limit of {}.",
+ size, target.getUuidString(), upperLimit);
+ return false;
+ }
+ return true;
+ }
+
+ logger.warn("No record of how much size has entered datanode {}", target.getUuidString());
+ return false;
+ }
+
+ @Override
+ public void increaseSizeEntering(
+ @Nonnull DatanodeDetails target,
+ long size,
+ long maxSizeEnteringTarget
+ ) {
+ if (sizeEnteringNode.containsKey(target)) {
+ long totalEnteringSize = sizeEnteringNode.get(target) + size;
+ sizeEnteringNode.put(target, totalEnteringSize);
+ if (totalEnteringSize >= maxSizeEnteringTarget) {
+ getPotentialTargets().removeIf(c -> c.getDatanodeDetails().equals(target));
+ }
+ } else {
+ logger.warn("Cannot find {} in the candidates target nodes", target.getUuid());
+ }
+ }
+
+ @Override
+ public void reInitialize(@Nonnull List potentialDataNodes) {
+ sizeEnteringNode.clear();
+ Collection potentialTargets = getPotentialTargets();
+ potentialTargets.clear();
+ potentialDataNodes.forEach(datanodeUsageInfo -> {
+ sizeEnteringNode.putIfAbsent(datanodeUsageInfo.getDatanodeDetails(), 0L);
+ potentialTargets.add(datanodeUsageInfo);
+ });
+ }
+
+ @VisibleForTesting
+ protected abstract Collection getPotentialTargets();
+
+ /**
+ * sort potentialTargets for specified source datanode according to network topology if enabled.
+ *
+ * @param source the specified source datanode
+ */
+ @VisibleForTesting
+ public abstract void sortTargetForSource(@Nonnull DatanodeDetails source);
+
+ /**
+ * Resets the collection of target datanode usage info that will be considered for balancing.
+ * Gets the latest usage info from node manager.
+ *
+ * @param targets collection of target {@link DatanodeDetails} that containers can move to
+ */
+ @Override
+ public final void resetPotentialTargets(@Nonnull Collection targets) {
+ Collection potentialTargets = getPotentialTargets();
+ potentialTargets.clear();
+ targets.forEach(datanodeDetails -> {
+ DatanodeUsageInfo usageInfo = nodeManager.getUsageInfo(datanodeDetails);
+ sizeEnteringNode.putIfAbsent(datanodeDetails, 0L);
+ potentialTargets.add(usageInfo);
+ });
+ }
+}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/ContainerBalanceIteration.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/ContainerBalanceIteration.java
new file mode 100644
index 000000000000..ed73d12680c6
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/ContainerBalanceIteration.java
@@ -0,0 +1,644 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.scm.container.balancer.iteration;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
+import org.apache.hadoop.hdds.scm.container.ContainerReplicaNotFoundException;
+import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerConfiguration;
+import org.apache.hadoop.hdds.scm.container.balancer.ContainerMoveSelection;
+import org.apache.hadoop.hdds.scm.container.balancer.MoveManager;
+import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
+
+/**
+ * A single step in container balance operation.
+ * The step is responsible for moving containers for source to target nodes
+ */
+public class ContainerBalanceIteration {
+ private static final Logger LOGGER = LoggerFactory.getLogger(ContainerBalanceIteration.class);
+ private final int maxDatanodeCountToUseInIteration;
+ private final long maxSizeToMovePerIteration;
+ private final double upperLimit;
+ private final double lowerLimit;
+ private final long maxSizeEnteringTarget;
+ private final List overUtilizedNodes = new ArrayList<>();
+ private final List underUtilizedNodes = new ArrayList<>();
+ private final List withinThresholdUtilizedNodes = new ArrayList<>();
+ private final FindTargetStrategy findTargetStrategy;
+ private final FindSourceStrategy findSourceStrategy;
+ private final ContainerSelectionCriteria selectionCriteria;
+
+ /*
+ Since a container can be selected only once during an iteration, these maps
+ use it as a primary key to track source to target pairings.
+ */
+ private final Map containerToSourceMap = new HashMap<>();
+ private final Map containerToTargetMap = new HashMap<>();
+
+ private final Set selectedTargets = new HashSet<>();
+ private final Set selectedSources = new HashSet<>();
+ private final ArrayList moveStateList = new ArrayList<>();
+ private final StorageContainerManager scm;
+ private final IterationMetrics metrics = new IterationMetrics();
+
+ public ContainerBalanceIteration(
+ @Nonnull ContainerBalancerConfiguration config,
+ @Nonnull StorageContainerManager scm,
+ @Nonnull List datanodeUsageInfos
+ ) {
+ this.scm = scm;
+ maxSizeToMovePerIteration = config.getMaxSizeToMovePerIteration();
+ int totalNodesInCluster = datanodeUsageInfos.size();
+ maxDatanodeCountToUseInIteration = (int) (config.getMaxDatanodesRatioToInvolvePerIteration() * totalNodesInCluster);
+
+ resetMoveManager(config, scm);
+
+ double clusterAvgUtilisation = calculateAvgUtilization(datanodeUsageInfos);
+ LOGGER.debug("Average utilization of the cluster is {}", clusterAvgUtilisation);
+
+ double threshold = config.getThresholdAsRatio();
+ // over utilized nodes have utilization greater than upper limit
+ upperLimit = clusterAvgUtilisation + threshold;
+ // under utilized nodes have utilization less than lower limit
+ lowerLimit = clusterAvgUtilisation - threshold;
+
+ findSourceStrategy = new FindSourceGreedy(scm.getScmNodeManager());
+ findTargetStrategy = FindTargetStrategyFactory.create(scm, config.getNetworkTopologyEnable());
+
+ LOGGER.debug("Lower limit for utilization is {} and Upper limit for utilization is {}", lowerLimit, upperLimit);
+
+ selectionCriteria = new ContainerSelectionCriteria(config, scm);
+ maxSizeEnteringTarget = config.getMaxSizeEnteringTarget();
+ }
+
+ private static void resetMoveManager(@Nonnull ContainerBalancerConfiguration config,
+ @Nonnull StorageContainerManager scm
+ ) {
+ MoveManager moveManager = scm.getMoveManager();
+ moveManager.resetState();
+ moveManager.setMoveTimeout(config.getMoveTimeout().toMillis());
+ moveManager.setReplicationTimeout(config.getMoveReplicationTimeout().toMillis());
+ }
+
+ public boolean findUnBalancedNodes(
+ @Nonnull Supplier isTaskRunning,
+ @Nonnull List datanodeUsageInfos
+ ) {
+ long totalOverUtilizedBytes = 0L, totalUnderUtilizedBytes = 0L;
+ // find over and under utilized nodes
+ for (DatanodeUsageInfo datanodeUsageInfo : datanodeUsageInfos) {
+ if (!isTaskRunning.get()) {
+ return false;
+ }
+ double utilization = datanodeUsageInfo.calculateUtilization();
+ SCMNodeStat scmNodeStat = datanodeUsageInfo.getScmNodeStat();
+ long capacity = scmNodeStat.getCapacity().get();
+
+ LOGGER.debug("Utilization for node {} with capacity {}B, used {}B, and remaining {}B is {}",
+ datanodeUsageInfo.getDatanodeDetails().getUuidString(),
+ capacity,
+ scmNodeStat.getScmUsed().get(),
+ scmNodeStat.getRemaining().get(),
+ utilization);
+ if (Double.compare(utilization, upperLimit) > 0) {
+ overUtilizedNodes.add(datanodeUsageInfo);
+ metrics.addToUnbalancedDatanodeCount(1);
+
+ // amount of bytes greater than upper limit in this node
+ long overUtilizedBytes = ratioToBytes(capacity, utilization) - ratioToBytes(capacity, upperLimit);
+ totalOverUtilizedBytes += overUtilizedBytes;
+ } else if (Double.compare(utilization, lowerLimit) < 0) {
+ underUtilizedNodes.add(datanodeUsageInfo);
+ metrics.addToUnbalancedDatanodeCount(1);
+
+ // amount of bytes lesser than lower limit in this node
+ long underUtilizedBytes = ratioToBytes(capacity, lowerLimit) - ratioToBytes(capacity, utilization);
+ totalUnderUtilizedBytes += underUtilizedBytes;
+ } else {
+ withinThresholdUtilizedNodes.add(datanodeUsageInfo);
+ }
+ }
+ metrics.addToUnbalancedDataSizeInBytes(Math.max(totalOverUtilizedBytes, totalUnderUtilizedBytes));
+
+ if (overUtilizedNodes.isEmpty() && underUtilizedNodes.isEmpty()) {
+ LOGGER.info("Did not find any unbalanced Datanodes.");
+ return false;
+ }
+
+ LOGGER.info("Container Balancer has identified {} Over-Utilized and {} Under-Utilized Datanodes " +
+ "that need to be balanced.",
+ overUtilizedNodes.size(), underUtilizedNodes.size());
+
+ if (LOGGER.isDebugEnabled()) {
+ overUtilizedNodes.forEach(entry -> {
+ LOGGER.debug("Datanode {} {} is Over-Utilized.",
+ entry.getDatanodeDetails().getHostName(),
+ entry.getDatanodeDetails().getUuid());
+ });
+
+ underUtilizedNodes.forEach(entry -> {
+ LOGGER.debug("Datanode {} {} is Under-Utilized.",
+ entry.getDatanodeDetails().getHostName(),
+ entry.getDatanodeDetails().getUuid());
+ });
+ }
+
+ return true;
+ }
+
+ public @Nonnull IterationResult doIteration(
+ @Nonnull Supplier isTaskRunning,
+ @Nonnull ContainerBalancerConfiguration config
+ ) {
+ // potential and selected targets are updated in the following loop
+ // TODO(jacksonyao): take withinThresholdUtilizedNodes as candidate for both source and target
+ findTargetStrategy.reInitialize(getPotentialTargets());
+ findSourceStrategy.reInitialize(getPotentialSources());
+
+ boolean isMoveGeneratedInThisIteration = false;
+ boolean canAdaptWhenNearingLimits = config.adaptBalanceWhenCloseToLimits();
+ boolean canAdaptOnReachingLimits = config.adaptBalanceWhenReachTheLimits();
+ IterationResult currentState = IterationResult.ITERATION_COMPLETED;
+
+ // match each source node with a target
+ while (true) {
+ if (!isTaskRunning.get()) {
+ currentState = IterationResult.ITERATION_INTERRUPTED;
+ break;
+ }
+
+ // break out if we've reached max size to move limit
+ if (reachedMaxSizeToMovePerIteration()) {
+ break;
+ }
+
+ // If balancer is approaching the iteration limits for max datanodes to involve,
+ // take some action in adaptWhenNearingIterationLimits()
+ if (canAdaptWhenNearingLimits && adaptWhenNearingIterationLimits()) {
+ canAdaptWhenNearingLimits = false;
+ }
+ // check if balancer has hit the iteration limits and take some action
+ if (canAdaptOnReachingLimits && adaptOnReachingIterationLimits()) {
+ canAdaptOnReachingLimits = false;
+ canAdaptWhenNearingLimits = false;
+ }
+
+ DatanodeDetails source = findSourceStrategy.getNextCandidateSourceDataNode();
+ if (source == null) {
+ // no more source DNs are present
+ break;
+ }
+
+ ContainerMoveSelection moveSelection = matchSourceWithTarget(source);
+ if (moveSelection != null && processMoveSelection(source, moveSelection)) {
+ isMoveGeneratedInThisIteration = true;
+ } else {
+ // can not find any target for this source
+ findSourceStrategy.removeCandidateSourceDataNode(source);
+ }
+ }
+
+ if (isMoveGeneratedInThisIteration) {
+ checkIterationMoveResults(config.getMoveTimeout().toMillis());
+ metrics.setInvolvedDatanodeCount(selectedSources.size() + selectedTargets.size());
+ } else {
+ // If no move was generated during this iteration then we don't need to check the move results
+ currentState = IterationResult.CAN_NOT_BALANCE_ANY_MORE;
+ }
+ return currentState;
+ }
+
+ /**
+ * Match a source datanode with a target datanode and identify the container to move.
+ *
+ * @return ContainerMoveSelection containing the selected target and container
+ */
+ private @Nullable ContainerMoveSelection matchSourceWithTarget(@Nonnull DatanodeDetails source) {
+ Set candidateContainers = selectionCriteria.getCandidateContainers(
+ source, findSourceStrategy, metrics.getMovedBytesCount(), lowerLimit);
+
+ if (candidateContainers.isEmpty()) {
+ LOGGER.debug("ContainerBalancer could not find any candidate containers for datanode {}", source.getUuidString());
+ return null;
+ }
+
+ LOGGER.debug("ContainerBalancer is finding suitable target for source datanode {}", source.getUuidString());
+
+ ContainerMoveSelection moveSelection = findTargetStrategy.findTargetForContainerMove(
+ source, candidateContainers, maxSizeEnteringTarget, upperLimit);
+
+ if (moveSelection != null) {
+ LOGGER.info("ContainerBalancer matched source datanode {} with target datanode {} for container move.",
+ source.getUuidString(),
+ moveSelection.getTargetNode().getUuidString());
+ } else {
+ LOGGER.debug("ContainerBalancer could not find a suitable target for source node {}.", source.getUuidString());
+ }
+
+ return moveSelection;
+ }
+
+ private boolean reachedMaxSizeToMovePerIteration() {
+ // since candidate containers in ContainerBalancerSelectionCriteria are
+ // filtered out according to this limit, balancer should not have crossed it
+ long bytesMovedInLatestIteration = metrics.getMovedBytesCount();
+ if (bytesMovedInLatestIteration >= maxSizeToMovePerIteration) {
+ LOGGER.warn("Reached max size to move limit. {} bytes have already been scheduled for balancing and " +
+ "the limit is {} bytes.",
+ bytesMovedInLatestIteration,
+ maxSizeToMovePerIteration);
+ return true;
+ }
+ return false;
+ }
+
+ /**
+ * Restricts potential target datanodes to nodes that have already been selected if balancer is one datanode away from
+ * "datanodes.involved.max.percentage.per.iteration" limit.
+ *
+ * @return true if potential targets were restricted, else false
+ */
+ private boolean adaptWhenNearingIterationLimits() {
+ // check if we're nearing max datanodes to involve
+ if (metrics.getInvolvedDatanodeCount() + 1 == maxDatanodeCountToUseInIteration) {
+ // We're one datanode away from reaching the limit. Restrict potential targets to targets
+ // that have already been selected.
+ findTargetStrategy.resetPotentialTargets(selectedTargets);
+ LOGGER.debug("Approaching max datanodes to involve limit. {} datanodes have already been selected for " +
+ "balancing and the limit is {}. Only already selected targets can be selected as targets now.",
+ metrics.getInvolvedDatanodeCount(), maxDatanodeCountToUseInIteration);
+ return true;
+ }
+ // return false if we didn't adapt
+ return false;
+ }
+
+ /**
+ * Restricts potential source and target datanodes to nodes that have already been selected if balancer has reached
+ * "datanodes.involved.max.percentage.per.iteration" limit.
+ * @return true if potential sources and targets were restricted, else false
+ */
+ private boolean adaptOnReachingIterationLimits() {
+ // check if we've reached max datanodes to involve limit
+ if (metrics.getInvolvedDatanodeCount() == maxDatanodeCountToUseInIteration) {
+ // restrict both to already selected sources and targets
+ findTargetStrategy.resetPotentialTargets(selectedTargets);
+ findSourceStrategy.resetPotentialSources(selectedSources);
+ LOGGER.debug("Reached max datanodes to involve limit. {} datanodes have already been selected for balancing " +
+ "and the limit is {}. Only already selected sources and targets can be involved in balancing now.",
+ metrics.getInvolvedDatanodeCount(), maxDatanodeCountToUseInIteration);
+ return true;
+ }
+ // return false if we didn't adapt
+ return false;
+ }
+
+ private boolean processMoveSelection(
+ @Nonnull DatanodeDetails source,
+ @Nonnull ContainerMoveSelection moveSelection
+ ) {
+ ContainerID containerID = moveSelection.getContainerID();
+ if (containerToSourceMap.containsKey(containerID) || containerToTargetMap.containsKey(containerID)) {
+ LOGGER.warn("Container {} has already been selected for move from source {} to target {} earlier. " +
+ "Not moving this container again.",
+ containerID,
+ containerToSourceMap.get(containerID),
+ containerToTargetMap.get(containerID));
+ return false;
+ }
+
+ ContainerInfo containerInfo;
+ try {
+ containerInfo = scm.getContainerManager().getContainer(containerID);
+ } catch (ContainerNotFoundException e) {
+ LOGGER.warn("Could not get container {} from Container Manager before starting a container move", containerID, e);
+ return false;
+ }
+ LOGGER.info("ContainerBalancer is trying to move container {} with size {}B from source datanode {} " +
+ "to target datanode {}",
+ containerID.toString(),
+ containerInfo.getUsedBytes(),
+ source.getUuidString(),
+ moveSelection.getTargetNode().getUuidString());
+
+ if (moveContainer(source, moveSelection)) {
+ // consider move successful for now, and update selection criteria
+ updateTargetsAndSelectionCriteria(moveSelection, source);
+ }
+ return true;
+ }
+
+ /**
+ * Asks {@link ReplicationManager} or {@link MoveManager} to move the specified container from source to target.
+ *
+ * @param sourceDnDetails the source datanode
+ * @param moveSelection the selected container to move and target datanode
+ * @return false if an exception occurred or the move completed with a result other than
+ * MoveManager.MoveResult.COMPLETED.
+ * Returns true if the move completed with MoveResult.COMPLETED or move is not yet done
+ */
+ private boolean moveContainer(
+ @Nonnull DatanodeDetails sourceDnDetails,
+ @Nonnull ContainerMoveSelection moveSelection
+ ) {
+ ContainerID containerID = moveSelection.getContainerID();
+ CompletableFuture future;
+ try {
+ ContainerInfo containerInfo = scm.getContainerManager().getContainer(containerID);
+ /*
+ If LegacyReplicationManager is enabled, ReplicationManager will redirect to it. Otherwise, use MoveManager.
+ */
+ ReplicationManager replicationManager = scm.getReplicationManager();
+ DatanodeDetails targetDnDetails = moveSelection.getTargetNode();
+ if (replicationManager.getConfig().isLegacyEnabled()) {
+ future = replicationManager.move(containerID, sourceDnDetails, targetDnDetails);
+ } else {
+ future = scm.getMoveManager().move(containerID, sourceDnDetails, targetDnDetails);
+ }
+ metrics.addToScheduledContainerMovesCount(1);
+
+ future = future.whenComplete((result, ex) -> {
+ if (result != null) {
+ metrics.addToContainerMoveMetrics(result, 1);
+ }
+ String sourceUUID = sourceDnDetails.getUuidString();
+ String targetUUUID = targetDnDetails.getUuidString();
+ if (ex != null) {
+ LOGGER.info("Container move for container {} from source {} to target {} failed with exceptions.",
+ containerID.toString(), sourceUUID, targetUUUID, ex);
+ metrics.addToFailedContainerMovesCount(1);
+ } else {
+ if (result == MoveManager.MoveResult.COMPLETED) {
+ metrics.addToMovedBytesCount(containerInfo.getUsedBytes());
+ LOGGER.debug("Container move completed for container {} from source {} to target {}",
+ containerID, sourceUUID, targetUUUID);
+ } else {
+ LOGGER.warn("Container move for container {} from source {} to target {} failed: {}",
+ containerInfo, sourceUUID, targetUUUID, result);
+ }
+ }
+ });
+ } catch (ContainerNotFoundException e) {
+ LOGGER.warn("Could not find Container {} for container move", containerID, e);
+ metrics.addToFailedContainerMovesCount(1);
+ return false;
+ } catch (NodeNotFoundException | TimeoutException | ContainerReplicaNotFoundException e) {
+ LOGGER.warn("Container move failed for container {}", containerID, e);
+ metrics.addToFailedContainerMovesCount(1);
+ return false;
+ }
+
+ /*
+ If the future hasn't failed yet, put it in moveSelectionToFutureMap for processing later
+ */
+ if (future.isDone()) {
+ if (future.isCompletedExceptionally()) {
+ return false;
+ } else {
+ MoveManager.MoveResult result = future.join();
+ moveStateList.add(new MoveState(moveSelection, future));
+ return result == MoveManager.MoveResult.COMPLETED;
+ }
+ } else {
+ moveStateList.add(new MoveState(moveSelection, future));
+ return true;
+ }
+ }
+
+ /**
+ * Update targets, sources, and selection criteria after a move.
+ *
+ * @param moveSelection latest selected target datanode and container
+ * @param source the source datanode
+ */
+ private void updateTargetsAndSelectionCriteria(
+ @Nonnull ContainerMoveSelection moveSelection,
+ @Nonnull DatanodeDetails source
+ ) {
+ ContainerID containerID = moveSelection.getContainerID();
+ DatanodeDetails target = moveSelection.getTargetNode();
+
+ // count target if it has not been involved in move earlier
+ metrics.addToInvolvedDatanodeCount(selectedSources.contains(source) ? 0 : 1);
+ metrics.addToInvolvedDatanodeCount(selectedTargets.contains(target) ? 0 : 1);
+
+ incSizeSelectedForMoving(source, moveSelection);
+ containerToSourceMap.put(containerID, source);
+ containerToTargetMap.put(containerID, target);
+ selectedSources.add(source);
+ selectedTargets.add(target);
+ selectionCriteria.updateSelectedContainer(containerToSourceMap.keySet());
+ }
+
+ /**
+ * Updates conditions for balancing, such as total size moved by balancer, total size that has entered a datanode,
+ * and total size that has left a datanode in this iteration.
+ *
+ * @param source source datanode
+ * @param moveSelection selected target datanode and container
+ */
+ private void incSizeSelectedForMoving(
+ @Nonnull DatanodeDetails source,
+ @Nonnull ContainerMoveSelection moveSelection
+ ) {
+ try {
+ ContainerInfo container = scm.getContainerManager().getContainer(moveSelection.getContainerID());
+ long usedBytes = container.getUsedBytes();
+
+ // update sizeLeavingNode map with the recent moveSelection
+ findSourceStrategy.increaseSizeLeaving(source, usedBytes);
+
+ // update sizeEnteringNode map with the recent moveSelection
+ findTargetStrategy.increaseSizeEntering(moveSelection.getTargetNode(), usedBytes, maxSizeEnteringTarget);
+ } catch (ContainerNotFoundException e) {
+ LOGGER.warn("Could not find Container {} while matching source and target nodes in ContainerBalancer",
+ moveSelection.getContainerID(), e);
+ }
+ }
+
+ /**
+ * Checks the results of all move operations when exiting an iteration.
+ */
+ @SuppressWarnings("rawtypes")
+ private void checkIterationMoveResults(long timeoutInMillis) {
+ CompletableFuture[] futureArray = new CompletableFuture[moveStateList.size()];
+ for (int i = 0; i < moveStateList.size(); ++i) {
+ futureArray[i] = moveStateList.get(i).result;
+ }
+ CompletableFuture allFuturesResult = CompletableFuture.allOf(futureArray);
+ try {
+ allFuturesResult.get(timeoutInMillis, TimeUnit.MILLISECONDS);
+ } catch (InterruptedException e) {
+ LOGGER.warn("Container balancer is interrupted");
+ Thread.currentThread().interrupt();
+ } catch (TimeoutException e) {
+ long timeoutCounts = cancelMovesThatExceedTimeoutDuration();
+ LOGGER.warn("{} Container moves are canceled.", timeoutCounts);
+ metrics.addToTimeoutContainerMovesCount(timeoutCounts);
+ } catch (ExecutionException e) {
+ LOGGER.error("Got exception while checkIterationMoveResults", e);
+ }
+ }
+
+ /**
+ * Cancels container moves that are not yet done. Note that if a move command has already been sent out to a Datanode,
+ * we don't yet have the capability to cancel it.
+ * However, those commands in the DN should time out if they haven't been processed yet.
+ *
+ * @return number of moves that did not complete (timed out) and were cancelled.
+ */
+ private long cancelMovesThatExceedTimeoutDuration() {
+ int numCancelled = 0;
+ // iterate through all moves and cancel ones that aren't done yet
+ for (MoveState state : moveStateList) {
+ CompletableFuture future = state.result;
+ if (!future.isDone()) {
+ ContainerMoveSelection moveSelection = state.moveSelection;
+ ContainerID containerID = moveSelection.getContainerID();
+ LOGGER.warn("Container move timed out for container {} from source {} to target {}.",
+ containerID,
+ containerToSourceMap.get(containerID).getUuidString(),
+ moveSelection.getTargetNode().getUuidString()
+ );
+
+ future.cancel(true);
+ numCancelled += 1;
+ }
+ }
+
+ return numCancelled;
+ }
+
+ /**
+ * Get potential targets for container move. Potential targets are under utilized and within threshold utilized nodes.
+ *
+ * @return A list of potential target DatanodeUsageInfo.
+ */
+ private @Nonnull List getPotentialTargets() {
+ // TODO(jacksonyao): take withinThresholdUtilizedNodes as candidate for both source and target
+ return getUnderUtilizedNodes();
+ }
+
+ /**
+ * Get potential sourecs for container move. Potential sourecs are over utilized and within threshold utilized nodes.
+ *
+ * @return A list of potential source DatanodeUsageInfo.
+ */
+ private @Nonnull List getPotentialSources() {
+ // TODO(jacksonyao): take withinThresholdUtilizedNodes as candidate for both source and target
+ return getOverUtilizedNodes();
+ }
+
+ /**
+ * Calculates the average utilization for the specified nodes.
+ * Utilization is (capacity - remaining) divided by capacity.
+ *
+ * @param nodes List of DatanodeUsageInfo to find the average utilization for
+ * @return Average utilization value
+ */
+ @VisibleForTesting
+ public static double calculateAvgUtilization(@Nonnull List nodes) {
+ if (nodes.isEmpty()) {
+ LOGGER.warn("No nodes to calculate average utilization for ContainerBalancer.");
+ return 0;
+ }
+ SCMNodeStat aggregatedStats = new SCMNodeStat(0, 0, 0);
+ for (DatanodeUsageInfo node : nodes) {
+ aggregatedStats.add(node.getScmNodeStat());
+ }
+ long clusterCapacity = aggregatedStats.getCapacity().get();
+ long clusterRemaining = aggregatedStats.getRemaining().get();
+
+ return (clusterCapacity - clusterRemaining) / (double) clusterCapacity;
+ }
+
+ /**
+ * Calculates the number of used bytes given capacity and utilization ratio.
+ *
+ * @param nodeCapacity capacity of the node.
+ * @param utilizationRatio used space by capacity ratio of the node.
+ * @return number of bytes
+ */
+ private long ratioToBytes(long nodeCapacity, double utilizationRatio) {
+ return (long) (nodeCapacity * utilizationRatio);
+ }
+
+ public @Nonnull List getOverUtilizedNodes() {
+ return overUtilizedNodes;
+ }
+
+ public @Nonnull List getUnderUtilizedNodes() {
+ return underUtilizedNodes;
+ }
+
+ public @Nonnull Set getSelectedTargets() {
+ return selectedTargets;
+ }
+
+ public @Nonnull Map getContainerToTargetMap() {
+ return containerToTargetMap;
+ }
+
+ public @Nonnull Map getContainerToSourceMap() {
+ return containerToSourceMap;
+ }
+
+ public IterationMetrics getMetrics() {
+ return metrics;
+ }
+
+ /**
+ * Class represents the current move state of iteration.
+ */
+ private static final class MoveState {
+ private final ContainerMoveSelection moveSelection;
+ private final CompletableFuture result;
+
+ private MoveState(
+ @Nonnull ContainerMoveSelection moveSelection,
+ @Nonnull CompletableFuture future
+ ) {
+ this.moveSelection = moveSelection;
+ this.result = future;
+ }
+ }
+}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/ContainerSelectionCriteria.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/ContainerSelectionCriteria.java
new file mode 100644
index 000000000000..94e4665d9fac
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/ContainerSelectionCriteria.java
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.hdds.scm.container.balancer.iteration;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerConfiguration;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * The selection criteria for selecting containers that will be moved and selecting datanodes
+ * that containers will move to.
+ */
+public class ContainerSelectionCriteria {
+ private static final Logger LOG = LoggerFactory.getLogger(ContainerSelectionCriteria.class);
+
+ private final ContainerBalancerConfiguration config;
+ private final NodeManager nodeManager;
+ private final ReplicationManager replicationManager;
+ private final ContainerManager containerManager;
+ private final Set selectedContainers = new HashSet<>();
+ private final Set excludeContainers;
+
+ public ContainerSelectionCriteria(
+ @Nonnull ContainerBalancerConfiguration balancerConfig,
+ @Nonnull StorageContainerManager scm
+ ) {
+ config = balancerConfig;
+ nodeManager = scm.getScmNodeManager();
+ replicationManager = scm.getReplicationManager();
+ containerManager = scm.getContainerManager();
+ excludeContainers = balancerConfig.getExcludeContainers();
+ }
+
+ /**
+ * Checks whether container is currently undergoing replication or deletion.
+ *
+ * @param containerID Container to check.
+ *
+ * @return true if container is replicating or deleting, otherwise false.
+ */
+ private boolean isContainerReplicatingOrDeleting(@Nonnull ContainerID containerID) {
+ return replicationManager.isContainerReplicatingOrDeleting(containerID);
+ }
+
+ /**
+ * Gets containers that are suitable for moving based on the following required criteria:
+ * 1. Container must not be undergoing replication.
+ * 2. Container must not already be selected for balancing.
+ * 3. Container size should be closer to 5GB.
+ * 4. Container must not be in the configured exclude containers list.
+ * 5. Container should be closed.
+ * 6. Container should not be an EC container
+ * //TODO Temporarily not considering EC containers as candidates
+ *
+ * @param node datanodeDetails for which to find candidate containers
+ * @param lowerLimit the value of lower limit for node utilization: clusterAvgUtilisation - threshold
+ *
+ * @return Set of candidate containers that satisfy the criteria.
+ * @see HDDS-6940
+ */
+ public @Nonnull Set getCandidateContainers(
+ @Nonnull DatanodeDetails node,
+ @Nonnull FindSourceStrategy sourceStrategy,
+ long sizeMovedAlready,
+ double lowerLimit
+ ) {
+ Set containerIDSet = new TreeSet<>(orderContainersByUsedBytes().reversed());
+ try {
+ containerIDSet.addAll(nodeManager.getContainers(node));
+ } catch (NodeNotFoundException e) {
+ LOG.warn("Could not find Datanode {} while selecting candidate containers for Container Balancer.", node, e);
+ return containerIDSet;
+ }
+ containerIDSet.removeAll(excludeContainers);
+ containerIDSet.removeAll(selectedContainers);
+
+ containerIDSet.removeIf(
+ containerID -> shouldBeExcluded(containerID, node, sourceStrategy, sizeMovedAlready, lowerLimit)
+ );
+ return containerIDSet;
+ }
+
+ /**
+ * Checks if the first container has more used space than second.
+ *
+ * @param first first container to compare
+ * @param second second container to compare
+ *
+ * @return An integer greater than 0 if first is more used, 0 if they're the same containers or a container
+ * is not found, and a value less than 0 if first is less used than second. If both containers have equal used space,
+ * they're compared using {@link ContainerID#compareTo(ContainerID)}.
+ */
+ private int isContainerMoreUsed(@Nonnull ContainerID first, @Nonnull ContainerID second) {
+ if (first.equals(second)) {
+ return 0;
+ }
+ try {
+ long firstUsedBytes = containerManager.getContainer(first).getUsedBytes();
+ long secondUsedBytes = containerManager.getContainer(second).getUsedBytes();
+ if (firstUsedBytes > secondUsedBytes) {
+ return 1;
+ } else if (firstUsedBytes < secondUsedBytes) {
+ return -1;
+ } else {
+ return first.compareTo(second);
+ }
+ } catch (ContainerNotFoundException e) {
+ LOG.warn("Could not retrieve ContainerInfo from container manager for comparison.", e);
+ return 0;
+ }
+ }
+
+ /**
+ * Compares containers on the basis of used space.
+ *
+ * @return First container is more used if it has used space greater than second container.
+ */
+ private @Nonnull Comparator orderContainersByUsedBytes() {
+ return this::isContainerMoreUsed;
+ }
+
+ /**
+ * Checks whether a Container has the ReplicationType {@link HddsProtos.ReplicationType#EC}.
+ *
+ * @param container container to check
+ *
+ * @return true if the ReplicationType is EC and "hdds.scm.replication.enable.legacy" is true, else false
+ */
+ private boolean isECContainer(@Nonnull ContainerInfo container) {
+ return container.getReplicationType().equals(HddsProtos.ReplicationType.EC) &&
+ replicationManager.getConfig().isLegacyEnabled();
+ }
+
+ private boolean shouldBeExcluded(
+ @Nonnull ContainerID containerID,
+ @Nonnull DatanodeDetails node,
+ @Nonnull FindSourceStrategy sourceStrategy,
+ long sizeMovedAlready,
+ double lowerLimit
+ ) {
+ ContainerInfo container;
+ try {
+ container = containerManager.getContainer(containerID);
+ } catch (ContainerNotFoundException e) {
+ LOG.warn("Could not find Container {} to check if it should be a candidate container. Excluding it.",
+ containerID);
+ return true;
+ }
+ long usedBytes = container.getUsedBytes();
+ return !isContainerClosed(container, node) ||
+ isECContainer(container) ||
+ isContainerReplicatingOrDeleting(containerID) ||
+ !sourceStrategy.canSizeLeaveSource(node, usedBytes, config.getMaxSizeLeavingSource(), lowerLimit) ||
+ breaksMaxSizeToMoveLimit(container.containerID(), usedBytes, sizeMovedAlready);
+ }
+
+ /**
+ * Checks whether specified container is closed. Also checks if the replica on the specified datanode is CLOSED.
+ * Assumes that there will only be one replica of a container on a particular Datanode.
+ *
+ * @param container container to check
+ * @param datanodeDetails datanode on which a replica of the container is present
+ *
+ * @return true if container LifeCycleState is {@link HddsProtos.LifeCycleState#CLOSED} and
+ * its replica on the specified datanode is CLOSED, else false
+ */
+ private boolean isContainerClosed(@Nonnull ContainerInfo container, @Nonnull DatanodeDetails datanodeDetails) {
+ if (!container.getState().equals(HddsProtos.LifeCycleState.CLOSED)) {
+ return false;
+ }
+
+ // also check that the replica on the specified DN is closed
+ Set replicas;
+ try {
+ replicas = containerManager.getContainerReplicas(container.containerID());
+ } catch (ContainerNotFoundException e) {
+ LOG.warn("Container {} does not exist in ContainerManager. Skipping this container.",
+ container.getContainerID(), e);
+ return false;
+ }
+ for (ContainerReplica replica : replicas) {
+ if (replica.getDatanodeDetails().equals(datanodeDetails)) {
+ // don't consider replica if it's not closed assumption: there's only one replica of this container on this DN
+ return replica.getState().equals(ContainerReplicaProto.State.CLOSED);
+ }
+ }
+
+ return false;
+ }
+
+ private boolean breaksMaxSizeToMoveLimit(
+ @Nonnull ContainerID containerID,
+ long usedBytes,
+ long sizeMovedAlready
+ ) {
+ // check max size to move per iteration limit
+ if (sizeMovedAlready + usedBytes > config.getMaxSizeToMovePerIteration()) {
+ LOG.debug("Removing container {} because it fails max size to move per iteration check.", containerID);
+ return true;
+ }
+ return false;
+ }
+
+ public void updateSelectedContainer(@Nonnull Set newSelectedContainers) {
+ selectedContainers.clear();
+ selectedContainers.addAll(newSelectedContainers);
+ }
+}
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/iteration/FindSourceGreedy.java
similarity index 51%
rename from hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindSourceGreedy.java
rename to hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/FindSourceGreedy.java
index 4f5868f2456e..63b197add1f2 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/iteration/FindSourceGreedy.java
@@ -15,16 +15,17 @@
* the License.
*/
-package org.apache.hadoop.hdds.scm.container.balancer;
+package org.apache.hadoop.hdds.scm.container.balancer.iteration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerConfiguration;
import org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo;
import org.apache.hadoop.hdds.scm.node.NodeManager;
-import org.jetbrains.annotations.NotNull;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.util.ArrayList;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
@@ -37,21 +38,16 @@
* which will be moved out.
*/
public class FindSourceGreedy implements FindSourceStrategy {
- private static final Logger LOG =
- LoggerFactory.getLogger(FindSourceGreedy.class);
- private Map sizeLeavingNode;
- private PriorityQueue potentialSources;
- private NodeManager nodeManager;
- private ContainerBalancerConfiguration config;
- private Double lowerLimit;
+ private static final Logger LOG = LoggerFactory.getLogger(FindSourceGreedy.class);
+ private final Map sizeLeavingNode = new HashMap<>();
+ private final PriorityQueue potentialSources;
+ private final NodeManager nodeManager;
- FindSourceGreedy(NodeManager nodeManager) {
- sizeLeavingNode = new HashMap<>();
+ public FindSourceGreedy(@Nonnull NodeManager nodeManager) {
+ this.nodeManager = nodeManager;
potentialSources = new PriorityQueue<>((a, b) -> {
- double currentUsageOfA = a.calculateUtilization(
- -sizeLeavingNode.get(a.getDatanodeDetails()));
- double currentUsageOfB = b.calculateUtilization(
- -sizeLeavingNode.get(b.getDatanodeDetails()));
+ double currentUsageOfA = a.calculateUtilization(-sizeLeavingNode.get(a.getDatanodeDetails()));
+ double currentUsageOfB = b.calculateUtilization(-sizeLeavingNode.get(b.getDatanodeDetails()));
//in descending order
int ret = Double.compare(currentUsageOfB, currentUsageOfA);
if (ret != 0) {
@@ -61,48 +57,42 @@ public class FindSourceGreedy implements FindSourceStrategy {
UUID uuidB = b.getDatanodeDetails().getUuid();
return uuidA.compareTo(uuidB);
});
- this.nodeManager = nodeManager;
- }
-
- private void setLowerLimit(Double lowerLimit) {
- this.lowerLimit = lowerLimit;
- }
-
- /**
- * {@inheritDoc}
- */
- public void resetPotentialSources(
- @NotNull Collection sources) {
- List usageInfos = new ArrayList<>(sources.size());
- sources.forEach(source -> usageInfos.add(nodeManager.getUsageInfo(source)));
- resetSources(usageInfos);
}
/**
- * Resets the collection of source datanodes that are considered when
- * selecting the next source datanode.
+ * Resets the collection of source datanodes that are considered when selecting the next source datanode.
+ *
* @param sources potential sources
*/
- private void resetSources(Collection sources) {
+ public void resetPotentialSources(@Nonnull Collection sources) {
potentialSources.clear();
- /* since sizeLeavingNode map is being used to track how much data has
- left a DN in an iteration, put keys only if they don't already exist
+ /* since sizeLeavingNode map is being used to track how much data has left a DN in an iteration,
+ put keys only if they don't already exist
*/
sources.forEach(source -> {
- sizeLeavingNode.putIfAbsent(source.getDatanodeDetails(), 0L);
- potentialSources.add(source);
+ DatanodeUsageInfo usageInfo = nodeManager.getUsageInfo(source);
+ resetSources(usageInfo);
});
}
- private void setConfiguration(ContainerBalancerConfiguration conf) {
- this.config = conf;
+ /**
+ * Resets the collection of source datanodes that are considered when selecting the next source datanode.
+ *
+ * @param source potential source
+ */
+ private void resetSources(@Nonnull DatanodeUsageInfo source) {
+ /* since sizeLeavingNode map is being used to track how much data has left a DN in an iteration,
+ put keys only if they don't already exist
+ */
+ sizeLeavingNode.putIfAbsent(source.getDatanodeDetails(), 0L);
+ potentialSources.add(source);
}
/**
* increase the Leaving size of a candidate source data node.
*/
@Override
- public void increaseSizeLeaving(DatanodeDetails dui, long size) {
+ public void increaseSizeLeaving(@Nonnull DatanodeDetails dui, long size) {
Long currentSize = sizeLeavingNode.get(dui);
if (currentSize != null) {
sizeLeavingNode.put(dui, currentSize + size);
@@ -110,18 +100,16 @@ public void increaseSizeLeaving(DatanodeDetails dui, long size) {
potentialSources.add(nodeManager.getUsageInfo(dui));
return;
}
- LOG.warn("Cannot find datanode {} in candidate source datanodes",
- dui.getUuid());
+ LOG.warn("Cannot find datanode {} in candidate source datanodes", dui.getUuid());
}
/**
- * get the next candidate source data node according to
- * the strategy.
+ * get the next candidate source data node according to the strategy.
*
* @return the nex candidate source data node.
*/
@Override
- public DatanodeDetails getNextCandidateSourceDataNode() {
+ public @Nullable DatanodeDetails getNextCandidateSourceDataNode() {
if (potentialSources.isEmpty()) {
LOG.info("no more candidate source data node");
return null;
@@ -130,45 +118,48 @@ public DatanodeDetails getNextCandidateSourceDataNode() {
}
/**
- * remove the specified data node from candidate source
- * data nodes.
+ * remove the specified data node from candidate source data nodes.
*/
@Override
- public void removeCandidateSourceDataNode(DatanodeDetails dui) {
- potentialSources.removeIf(a -> a.getDatanodeDetails().equals(dui));
+ public void removeCandidateSourceDataNode(@Nonnull DatanodeDetails dnDetails) {
+ potentialSources.removeIf(a -> a.getDatanodeDetails().equals(dnDetails));
}
/**
- * Checks if specified size can leave a specified target datanode
- * according to {@link ContainerBalancerConfiguration}
+ * Checks if specified size can leave a specified target datanode according to {@link ContainerBalancerConfiguration}
* "size.entering.target.max".
*
- * @param source target datanode in which size is entering
- * @param size size in bytes
+ * @param source target datanode in which size is entering
+ * @param size size in bytes
+ * @param lowerLimit the value of lower limit for node utilization: clusterAvgUtilisation - threshold
+ *
* @return true if size can leave, else false
*/
@Override
- public boolean canSizeLeaveSource(DatanodeDetails source, long size) {
+ public boolean canSizeLeaveSource(
+ @Nonnull DatanodeDetails source,
+ long size,
+ long maxSizeLeavingSource,
+ double lowerLimit
+ ) {
if (sizeLeavingNode.containsKey(source)) {
long sizeLeavingAfterMove = sizeLeavingNode.get(source) + size;
- //size can be moved out of source datanode only when the following
- //two condition are met.
- //1 sizeLeavingAfterMove does not succeed the configured
- // MaxSizeLeavingTarget
- //2 after subtracting sizeLeavingAfterMove, the usage is bigger
- // than or equal to lowerLimit
- if (sizeLeavingAfterMove > config.getMaxSizeLeavingSource()) {
- LOG.debug("{} bytes cannot leave datanode {} because 'size.leaving" +
- ".source.max' limit is {} and {} bytes have already left.",
- size, source.getUuidString(), config.getMaxSizeLeavingSource(),
+ // Size can be moved out of source datanode only when the following two condition are met.
+ // 1. SizeLeavingAfterMove does not succeed the configured MaxSizeLeavingTarget
+ // 2. After subtracting sizeLeavingAfterMove, the usage is bigger than or equal to lowerLimit
+ if (sizeLeavingAfterMove > maxSizeLeavingSource) {
+ LOG.debug("{} bytes cannot leave datanode {} because 'size.leaving.source.max' limit is {} and " +
+ "{} bytes have already left.",
+ size,
+ source.getUuidString(),
+ maxSizeLeavingSource,
sizeLeavingNode.get(source));
return false;
}
- if (Double.compare(nodeManager.getUsageInfo(source)
- .calculateUtilization(-sizeLeavingAfterMove), lowerLimit) < 0) {
- LOG.debug("{} bytes cannot leave datanode {} because its utilization " +
- "will drop below the lower limit of {}.", size,
- source.getUuidString(), lowerLimit);
+ DatanodeUsageInfo usageInfo = nodeManager.getUsageInfo(source);
+ if (Double.compare(usageInfo.calculateUtilization(-sizeLeavingAfterMove), lowerLimit) < 0) {
+ LOG.debug("{} bytes cannot leave datanode {} because its utilization will drop below the lower limit of {}.",
+ size, source.getUuidString(), lowerLimit);
return false;
}
return true;
@@ -182,12 +173,8 @@ public boolean canSizeLeaveSource(DatanodeDetails source, long size) {
* reInitialize FindSourceStrategy.
*/
@Override
- public void reInitialize(List potentialDataNodes,
- ContainerBalancerConfiguration conf,
- Double lowLimit) {
- setConfiguration(conf);
- setLowerLimit(lowLimit);
+ public void reInitialize(@Nonnull List potentialDataNodes) {
sizeLeavingNode.clear();
- resetSources(potentialDataNodes);
+ potentialDataNodes.forEach(this::resetSources);
}
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindSourceStrategy.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/FindSourceStrategy.java
similarity index 55%
rename from hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindSourceStrategy.java
rename to hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/FindSourceStrategy.java
index e27163048390..c3a2b20e1d9a 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindSourceStrategy.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/FindSourceStrategy.java
@@ -16,63 +16,60 @@
* limitations under the License.
*/
-package org.apache.hadoop.hdds.scm.container.balancer;
+package org.apache.hadoop.hdds.scm.container.balancer.iteration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerConfiguration;
import org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo;
import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
import java.util.Collection;
import java.util.List;
/**
- * This interface can be used to implement strategies to get a
- * source datanode.
+ * This interface can be used to implement strategies to get a source datanode.
*/
-public interface FindSourceStrategy {
+interface FindSourceStrategy {
/**
- * get the next candidate source data node according to
- * the strategy.
+ * get the next candidate source data node according to the strategy.
*
* @return the nex candidate source data node.
*/
- DatanodeDetails getNextCandidateSourceDataNode();
+ @Nullable DatanodeDetails getNextCandidateSourceDataNode();
/**
- * remove the specified data node from candidate source
- * data nodes.
+ * remove the specified data node from candidate source data nodes.
*/
- void removeCandidateSourceDataNode(DatanodeDetails dui);
+ void removeCandidateSourceDataNode(@Nonnull DatanodeDetails dui);
/**
* increase the Leaving size of a candidate source data node.
*/
- void increaseSizeLeaving(DatanodeDetails dui, long size);
+ void increaseSizeLeaving(@Nonnull DatanodeDetails dui, long size);
/**
- * Checks if specified size can leave a specified source datanode
- * according to {@link ContainerBalancerConfiguration}
+ * Checks if specified size can leave a specified source datanode according to {@link ContainerBalancerConfiguration}
* "size.entering.target.max".
*
- * @param source target datanode in which size is entering
- * @param size size in bytes
+ * @param source target datanode in which size is entering
+ * @param size size in bytes
+ * @param lowerLimit the value of lower limit for node utilization:
+ * clusterAvgUtilisation - threshold
* @return true if size can leave, else false
*/
- boolean canSizeLeaveSource(DatanodeDetails source, long size);
+ boolean canSizeLeaveSource(@Nonnull DatanodeDetails source, long size, long maxSizeLeavingSource, double lowerLimit);
/**
* reInitialize FindSourceStrategy.
*/
- void reInitialize(List potentialDataNodes,
- ContainerBalancerConfiguration config, Double lowerLimit);
+ void reInitialize(@Nonnull List potentialDataNodes);
/**
- * Resets the collection of source {@link DatanodeUsageInfo} that can be
- * selected for balancing.
+ * Resets the collection of source {@link DatanodeUsageInfo} that can be selected for balancing.
*
- * @param sources collection of source
- * {@link DatanodeDetails} that containers can move from
+ * @param sources collection of source {@link DatanodeDetails} that containers can move from
*/
void resetPotentialSources(@Nonnull Collection sources);
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/FindTargetGreedyByNetworkTopology.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/FindTargetGreedyByNetworkTopology.java
new file mode 100644
index 000000000000..e42eee31d19d
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/FindTargetGreedyByNetworkTopology.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.scm.container.balancer.iteration;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.net.NetworkTopology;
+import org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo;
+import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
+
+import javax.annotation.Nonnull;
+import java.util.ArrayList;
+import java.util.Collection;
+
+/**
+ * An implementation of FindTargetGreedy, which will always select the target with the shortest distance according
+ * to network topology distance to the give source datanode.
+ */
+class FindTargetGreedyByNetworkTopology extends AbstractFindTargetGreedy {
+ private final NetworkTopology networkTopology;
+ private final ArrayList potentialTargets;
+
+ FindTargetGreedyByNetworkTopology(@Nonnull StorageContainerManager scm) {
+ super(scm, FindTargetGreedyByNetworkTopology.class);
+ networkTopology = scm.getClusterMap();
+ potentialTargets = new ArrayList<>();
+ }
+
+ @Override
+ public @Nonnull Collection getPotentialTargets() {
+ return potentialTargets;
+ }
+
+ /**
+ * Sort potentialTargets for specified source datanode according to network topology.
+ *
+ * @param source the specified source datanode
+ */
+ @VisibleForTesting
+ public void sortTargetForSource(@Nonnull DatanodeDetails source) {
+ potentialTargets.sort((DatanodeUsageInfo da, DatanodeUsageInfo db) -> {
+ DatanodeDetails a = da.getDatanodeDetails();
+ DatanodeDetails b = db.getDatanodeDetails();
+ // sort by network topology first
+ int distanceToA = networkTopology.getDistanceCost(source, a);
+ int distanceToB = networkTopology.getDistanceCost(source, b);
+ if (distanceToA != distanceToB) {
+ return distanceToA - distanceToB;
+ }
+ // if distance to source is equal , sort by usage
+ return compareByUsage(da, db);
+ });
+ }
+}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/FindTargetGreedyByUsageInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/FindTargetGreedyByUsageInfo.java
new file mode 100644
index 000000000000..299950c4556d
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/FindTargetGreedyByUsageInfo.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.scm.container.balancer.iteration;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo;
+import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
+
+import javax.annotation.Nonnull;
+import java.util.Collection;
+import java.util.TreeSet;
+
+/**
+ * An implementation of FindTargetGreedy, which will always select the target with the lowest space usage.
+ */
+class FindTargetGreedyByUsageInfo extends AbstractFindTargetGreedy {
+ private final TreeSet potentialTargets;
+
+ FindTargetGreedyByUsageInfo(@Nonnull StorageContainerManager scm) {
+ super(scm, FindTargetGreedyByUsageInfo.class);
+ potentialTargets = new TreeSet<>(this::compareByUsage);
+ }
+
+ @Override
+ protected Collection getPotentialTargets() {
+ return potentialTargets;
+ }
+
+ @Override
+ public void sortTargetForSource(@Nonnull DatanodeDetails source) {
+ // noop, Treeset is naturally sorted.
+ }
+}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindTargetStrategy.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/FindTargetStrategy.java
similarity index 54%
rename from hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindTargetStrategy.java
rename to hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/FindTargetStrategy.java
index 17f6aa329dcb..6dba5d2a361b 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindTargetStrategy.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/FindTargetStrategy.java
@@ -16,57 +16,58 @@
* limitations under the License.
*/
-package org.apache.hadoop.hdds.scm.container.balancer;
+package org.apache.hadoop.hdds.scm.container.balancer.iteration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.balancer.ContainerMoveSelection;
import org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo;
import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
import java.util.Collection;
import java.util.List;
import java.util.Set;
/**
- * This interface can be used to implement strategies to find a target for a
- * source.
+ * This interface can be used to implement strategies to find a target for a source.
*/
-public interface FindTargetStrategy {
+interface FindTargetStrategy {
/**
- * Finds target for a source for container move, given a collection of
- * potential target datanodes, a set of candidate containers, and a
- * functional interface with a method that returns true if a given size can
- * enter a potential target.
+ * Finds target for a source for container move, given a collection of potential target datanodes,
+ * a set of candidate containers, and a functional interface with a method that returns true
+ * if a given size can enter a potential target.
*
- * @param source Datanode to find a target for
- * @param candidateContainers Set of candidate containers satisfying
- * selection criteria
- * {@link ContainerBalancerSelectionCriteria}
- * (DatanodeDetails, Long) method returns true if the size specified in the
- * second argument can enter the specified DatanodeDetails node
- * @return {@link ContainerMoveSelection} containing the target node and
- * selected container
+ * @param source datanode to find a target for
+ * @param containersToMove set of candidate containers satisfying selection criteria
+ * {@link ContainerSelectionCriteria}
+ * @param maxSizeEnteringTarget the maximum size that can enter a target datanode in each iteration while balancing.
+ * @param upperLimit the value of upper limit for node utilization:
+ * clusterAvgUtilisation + threshold
+ * @return {@link ContainerMoveSelection} containing the target node and selected container
*/
- ContainerMoveSelection findTargetForContainerMove(
- DatanodeDetails source, Set candidateContainers);
+ @Nullable ContainerMoveSelection findTargetForContainerMove(
+ @Nonnull DatanodeDetails source,
+ @Nonnull Set containersToMove,
+ long maxSizeEnteringTarget,
+ double upperLimit
+ );
/**
* increase the Entering size of a candidate target data node.
*/
- void increaseSizeEntering(DatanodeDetails target, long size);
+ void increaseSizeEntering(@Nonnull DatanodeDetails target, long size, long maxSizeEnteringTarget);
/**
* reInitialize FindTargetStrategy.
*/
- void reInitialize(List potentialDataNodes,
- ContainerBalancerConfiguration config, Double upperLimit);
+ void reInitialize(@Nonnull List potentialDataNodes);
/**
- * Resets the collection of target {@link DatanodeUsageInfo} that can be
- * selected for balancing.
- * @param targets collection of target {@link DatanodeDetails}
- * that containers can be moved to
+ * Resets the collection of target {@link DatanodeUsageInfo} that can be selected for balancing.
+ *
+ * @param targets collection of target {@link DatanodeDetails} that containers can be moved to
*/
void resetPotentialTargets(@Nonnull Collection targets);
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/FindTargetStrategyFactory.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/FindTargetStrategyFactory.java
new file mode 100644
index 000000000000..cd53e93936d1
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/FindTargetStrategyFactory.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.scm.container.balancer.iteration;
+
+import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
+import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerConfiguration;
+
+import javax.annotation.Nonnull;
+
+/**
+ * Class using for building strategy for looking target nodes.
+ */
+final class FindTargetStrategyFactory {
+ private FindTargetStrategyFactory() { }
+
+ /**
+ * @param scm StorageContainerManager instance.
+ * @param isNetworkTopologyEnabled Value from {@link ContainerBalancerConfiguration}.
+ * Specifies the instance of FindTargetStrategy that will be created
+ * @return the instance of FindTargetStrategy
+ */
+ public static @Nonnull FindTargetStrategy create(
+ @Nonnull StorageContainerManager scm,
+ boolean isNetworkTopologyEnabled
+ ) {
+ if (isNetworkTopologyEnabled) {
+ return new FindTargetGreedyByNetworkTopology(scm);
+ } else {
+ return new FindTargetGreedyByUsageInfo(scm);
+ }
+ }
+}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/IterationMetrics.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/IterationMetrics.java
new file mode 100644
index 000000000000..7f3ece1ea29b
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/IterationMetrics.java
@@ -0,0 +1,137 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.scm.container.balancer.iteration;
+
+import org.apache.hadoop.hdds.scm.container.balancer.MoveManager;
+
+/**
+ * Metrics related to Container Balancer Iteration running in SCM.
+ * Collected inside one thread, doesn't require atomic read|write
+ */
+
+public final class IterationMetrics {
+ private long completedContainerMovesCount;
+ private long timeoutContainerMovesCount;
+ private long failedContainerMovesCount;
+ private long scheduledContainerMovesCount;
+ private long involvedDatanodeCount;
+ private long movedBytesCount;
+ private long unbalancedDataSizeInBytes;
+ private int unbalancedDatanodeCount;
+
+ public long getTimeoutContainerMovesCount() {
+ return timeoutContainerMovesCount;
+ }
+
+ public long getInvolvedDatanodeCount() {
+ return involvedDatanodeCount;
+ }
+
+ public long getMovedBytesCount() {
+ return movedBytesCount;
+ }
+
+ public long getCompletedContainerMovesCount() {
+ return completedContainerMovesCount;
+ }
+
+ public long getUnbalancedDataSizeInBytes() {
+ return unbalancedDataSizeInBytes;
+ }
+
+ public long getUnbalancedDatanodeCount() {
+ return unbalancedDatanodeCount;
+ }
+
+ public long getFailedContainerMovesCount() {
+ return failedContainerMovesCount;
+ }
+
+ public long getScheduledContainerMovesCount() {
+ return scheduledContainerMovesCount;
+ }
+
+ void addToTimeoutContainerMovesCount(long value) {
+ timeoutContainerMovesCount += value;
+ }
+
+ void addToInvolvedDatanodeCount(long value) {
+ involvedDatanodeCount += value;
+ }
+
+ void setInvolvedDatanodeCount(long value) {
+ involvedDatanodeCount = value;
+ }
+
+ void addToMovedBytesCount(long value) {
+ movedBytesCount += value;
+ }
+
+ void addToCompletedContainerMovesCount(long value) {
+ completedContainerMovesCount += value;
+ }
+
+ void addToUnbalancedDataSizeInBytes(long value) {
+ unbalancedDataSizeInBytes = value;
+ }
+
+ void addToUnbalancedDatanodeCount(int value) {
+ unbalancedDatanodeCount += value;
+ }
+
+ void addToFailedContainerMovesCount(long value) {
+ failedContainerMovesCount += value;
+ }
+
+ void addToScheduledContainerMovesCount(long value) {
+ scheduledContainerMovesCount += value;
+ }
+
+ void addToContainerMoveMetrics(MoveManager.MoveResult result, long value) {
+ switch (result) {
+ case COMPLETED:
+ addToCompletedContainerMovesCount(value);
+ break;
+ case REPLICATION_FAIL_TIME_OUT:
+ case DELETION_FAIL_TIME_OUT:
+ addToTimeoutContainerMovesCount(value);
+ break;
+ // TODO: Add metrics for other errors that need to be tracked.
+ case FAIL_LEADER_NOT_READY:
+ case REPLICATION_FAIL_INFLIGHT_REPLICATION:
+ case REPLICATION_FAIL_NOT_EXIST_IN_SOURCE:
+ case REPLICATION_FAIL_EXIST_IN_TARGET:
+ case REPLICATION_FAIL_CONTAINER_NOT_CLOSED:
+ case REPLICATION_FAIL_INFLIGHT_DELETION:
+ case REPLICATION_FAIL_NODE_NOT_IN_SERVICE:
+ case DELETION_FAIL_NODE_NOT_IN_SERVICE:
+ case REPLICATION_FAIL_NODE_UNHEALTHY:
+ case DELETION_FAIL_NODE_UNHEALTHY:
+ case DELETE_FAIL_POLICY:
+ case REPLICATION_NOT_HEALTHY_BEFORE_MOVE:
+ case REPLICATION_NOT_HEALTHY_AFTER_MOVE:
+ case FAIL_CONTAINER_ALREADY_BEING_MOVED:
+ case FAIL_UNEXPECTED_ERROR:
+ addToFailedContainerMovesCount(value);
+ break;
+ default:
+ break;
+ }
+ }
+}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/IterationResult.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/IterationResult.java
new file mode 100644
index 000000000000..ed2183d1470b
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/IterationResult.java
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.scm.container.balancer.iteration;
+
+/**
+ * The result of {@link ContainerBalanceIteration}.
+ */
+public enum IterationResult {
+ ITERATION_COMPLETED,
+ ITERATION_INTERRUPTED,
+ CAN_NOT_BALANCE_ANY_MORE
+}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/package-info.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/package-info.java
new file mode 100644
index 000000000000..4bac3cdcd955
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+/**
+ * This package contains classes related to Container Balancer Iteration.
+ */
+package org.apache.hadoop.hdds.scm.container.balancer.iteration;
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java
new file mode 100644
index 000000000000..272c6ed40b83
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java
@@ -0,0 +1,374 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.scm.container.balancer;
+
+import com.google.protobuf.ByteString;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.PlacementPolicyValidateProxy;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
+import org.apache.hadoop.hdds.scm.container.ContainerReplicaNotFoundException;
+import org.apache.hadoop.hdds.scm.container.MockNodeManager;
+import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementPolicyFactory;
+import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementMetrics;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.exceptions.SCMException;
+import org.apache.hadoop.hdds.scm.ha.SCMContext;
+import org.apache.hadoop.hdds.scm.ha.SCMService;
+import org.apache.hadoop.hdds.scm.ha.SCMServiceManager;
+import org.apache.hadoop.hdds.scm.ha.StatefulServiceStateManager;
+import org.apache.hadoop.hdds.scm.ha.StatefulServiceStateManagerImpl;
+import org.apache.hadoop.hdds.scm.net.NetworkTopology;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
+import org.apache.hadoop.hdds.server.events.EventPublisher;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.mockito.Mockito;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.time.Clock;
+import java.time.ZoneId;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeoutException;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Class for test used for setting up testable StorageContainerManager.
+ * Provides an access to {@link TestableCluster} and to necessary
+ * mocked instances
+ */
+public final class MockedSCM {
+ public static final long STORAGE_UNIT = OzoneConsts.GB;
+ private final StorageContainerManager scm;
+ private final TestableCluster cluster;
+ private final ContainerBalancerConfiguration balancerCfg;
+ private final OzoneConfiguration ozoneCfg;
+ private final MockNodeManager mockNodeManager;
+ private ContainerBalancer containerBalancer;
+ private MockedReplicationManager mockedReplicaManager;
+ private MoveManager moveManager;
+ private ContainerManager containerManager;
+
+ private MockedPlacementPolicies mockedPlacementPolicies;
+
+ private MockedSCM(@Nonnull TestableCluster testableCluster,
+ @Nonnull OzoneConfiguration ozoneConfig,
+ @Nonnull ContainerBalancerConfiguration balancerConfig
+ ) {
+ scm = mock(StorageContainerManager.class);
+ cluster = testableCluster;
+ mockNodeManager = new MockNodeManager(cluster.getDatanodeToContainersMap());
+ ozoneCfg = ozoneConfig;
+ balancerCfg = balancerConfig;
+ }
+
+ /**
+ * Mock some instances that will be used for MockedStorageContainerManager.
+ */
+ private void doMock() throws IOException, NodeNotFoundException, TimeoutException {
+ containerManager = mockContainerManager(cluster);
+ mockedReplicaManager = MockedReplicationManager.doMock();
+ moveManager = mockMoveManager();
+ StatefulServiceStateManager stateManager = MockedServiceStateManager.doMock();
+ SCMServiceManager scmServiceManager = mockSCMServiceManger();
+
+ mockedPlacementPolicies = MockedPlacementPolicies.doMock(ozoneCfg, mockNodeManager);
+
+ when(scm.getConfiguration()).thenReturn(ozoneCfg);
+ when(scm.getMoveManager()).thenReturn(moveManager);
+ when(scm.getScmNodeManager()).thenReturn(mockNodeManager);
+ when(scm.getContainerManager()).thenReturn(containerManager);
+ when(scm.getReplicationManager()).thenReturn(mockedReplicaManager.manager);
+ when(scm.getContainerPlacementPolicy()).thenReturn(mockedPlacementPolicies.placementPolicy);
+ when(scm.getPlacementPolicyValidateProxy()).thenReturn(mockedPlacementPolicies.validateProxyPolicy);
+ when(scm.getSCMServiceManager()).thenReturn(scmServiceManager);
+ when(scm.getScmContext()).thenReturn(SCMContext.emptyContext());
+ when(scm.getClusterMap()).thenReturn(null);
+ when(scm.getEventQueue()).thenReturn(mock(EventPublisher.class));
+ when(scm.getStatefulServiceStateManager()).thenReturn(stateManager);
+ }
+
+ private static @Nonnull ContainerBalancerConfiguration createBalancerCfg(@Nonnull OzoneConfiguration ozoneCfg) {
+ ContainerBalancerConfiguration balancerCfg = ozoneCfg.getObject(ContainerBalancerConfiguration.class);
+ balancerCfg.setThreshold(10);
+ balancerCfg.setIterations(1);
+ balancerCfg.setMaxDatanodesPercentageToInvolvePerIteration(100);
+ balancerCfg.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
+ balancerCfg.setMaxSizeEnteringTarget(50 * STORAGE_UNIT);
+ ozoneCfg.setFromObject(balancerCfg);
+ return balancerCfg;
+ }
+
+ public static @Nonnull MockedSCM getMockedSCM(int datanodeCount) {
+ OzoneConfiguration ozoneCfg = new OzoneConfiguration();
+ ContainerBalancerConfiguration balancerCfg = createBalancerCfg(ozoneCfg);
+ return getMockedSCM(datanodeCount, ozoneCfg, balancerCfg);
+ }
+
+ public static @Nonnull MockedSCM getMockedSCM(
+ int datanodeCount,
+ @Nonnull OzoneConfiguration ozoneCfg,
+ @Nonnull ContainerBalancerConfiguration balancerCfg
+ ) {
+ TestableCluster cluster = new TestableCluster(datanodeCount, STORAGE_UNIT);
+ MockedSCM mockedSCM = new MockedSCM(cluster, ozoneCfg, balancerCfg);
+ try {
+ mockedSCM.doMock();
+ mockedSCM.initContainerBalancer();
+
+ } catch (IOException | NodeNotFoundException | TimeoutException e) {
+ throw new RuntimeException("Can't initialize TestOzoneHDDS: ", e);
+ }
+ return mockedSCM;
+ }
+
+ private void initContainerBalancer() {
+ containerBalancer = new ContainerBalancer(scm);
+ }
+
+ @Override
+ public String toString() {
+ return cluster.toString();
+ }
+
+ public @Nonnull ContainerBalancer createContainerBalancer() {
+ return new ContainerBalancer(scm);
+ }
+
+ public @Nonnull ContainerBalancerTask startBalancerTask(
+ @Nonnull ContainerBalancer balancer,
+ @Nonnull ContainerBalancerConfiguration config
+ ) {
+ ContainerBalancerTask task = new ContainerBalancerTask(scm, balancer, config);
+ task.run(0, false);
+ return task;
+ }
+
+ public @Nonnull ContainerBalancerTask startBalancerTask(@Nonnull ContainerBalancerConfiguration config) {
+ return startBalancerTask(new ContainerBalancer(scm), config);
+ }
+
+ public void enableLegacyReplicationManager() {
+ mockedReplicaManager.conf.setEnableLegacy(true);
+ }
+
+ public void disableLegacyReplicationManager() {
+ mockedReplicaManager.conf.setEnableLegacy(false);
+ }
+
+ public @Nonnull MoveManager getMoveManager() {
+ return moveManager;
+ }
+
+ public @Nonnull ReplicationManager getReplicationManager() {
+ return mockedReplicaManager.manager;
+ }
+
+ public @Nonnull ContainerBalancerConfiguration getBalancerConfig() {
+ return balancerCfg;
+ }
+
+ public @Nonnull MockNodeManager getNodeManager() {
+ return mockNodeManager;
+ }
+
+ public @Nonnull OzoneConfiguration getOzoneConfig() {
+ return ozoneCfg;
+ }
+
+ public @Nonnull ContainerBalancerConfiguration getBalancerConfigByOzoneConfig(@Nonnull OzoneConfiguration config) {
+ return config.getObject(ContainerBalancerConfiguration.class);
+ }
+
+ public @Nonnull StorageContainerManager getStorageContainerManager() {
+ return scm;
+ }
+
+ public @Nonnull TestableCluster getCluster() {
+ return cluster;
+ }
+
+ public @Nonnull ContainerBalancer getContainerBalancer() {
+ return containerBalancer;
+ }
+
+ public @Nonnull ContainerManager getContainerManager() {
+ return containerManager;
+ }
+
+ public @Nonnull PlacementPolicy getPlacementPolicy() {
+ return mockedPlacementPolicies.placementPolicy;
+ }
+
+ public @Nonnull PlacementPolicy getEcPlacementPolicy() {
+ return mockedPlacementPolicies.ecPlacementPolicy;
+ }
+ private static @Nonnull ContainerManager mockContainerManager(@Nonnull TestableCluster cluster)
+ throws ContainerNotFoundException {
+ ContainerManager containerManager = mock(ContainerManager.class);
+ Mockito
+ .when(containerManager.getContainerReplicas(any(ContainerID.class)))
+ .thenAnswer(invocationOnMock -> {
+ ContainerID cid = (ContainerID) invocationOnMock.getArguments()[0];
+ return cluster.getCidToReplicasMap().get(cid);
+ });
+
+ Mockito
+ .when(containerManager.getContainer(any(ContainerID.class)))
+ .thenAnswer(invocationOnMock -> {
+ ContainerID cid = (ContainerID) invocationOnMock.getArguments()[0];
+ return cluster.getCidToInfoMap().get(cid);
+ });
+
+ Mockito
+ .when(containerManager.getContainers())
+ .thenReturn(new ArrayList<>(cluster.getCidToInfoMap().values()));
+ return containerManager;
+ }
+
+ private static @Nonnull SCMServiceManager mockSCMServiceManger() {
+ SCMServiceManager scmServiceManager = mock(SCMServiceManager.class);
+
+ Mockito
+ .doNothing()
+ .when(scmServiceManager)
+ .register(Mockito.any(SCMService.class));
+
+ return scmServiceManager;
+ }
+
+ private static @Nonnull MoveManager mockMoveManager()
+ throws NodeNotFoundException, ContainerReplicaNotFoundException, ContainerNotFoundException {
+ MoveManager moveManager = mock(MoveManager.class);
+ Mockito
+ .when(moveManager
+ .move(
+ any(ContainerID.class),
+ any(DatanodeDetails.class),
+ any(DatanodeDetails.class)))
+ .thenReturn(
+ CompletableFuture.completedFuture(MoveManager.MoveResult.COMPLETED)
+ );
+ return moveManager;
+ }
+
+ private static final class MockedReplicationManager {
+ private final ReplicationManager manager;
+ private final ReplicationManager.ReplicationManagerConfiguration conf;
+
+ private MockedReplicationManager() {
+ manager = mock(ReplicationManager.class);
+ conf = new ReplicationManager.ReplicationManagerConfiguration();
+ /*
+ Disable LegacyReplicationManager. This means balancer should select RATIS as well as EC containers for balancing.
+ Also, MoveManager will be used.
+ */
+ conf.setEnableLegacy(false);
+ }
+
+ private static @Nonnull MockedReplicationManager doMock()
+ throws NodeNotFoundException, ContainerNotFoundException, TimeoutException {
+ MockedReplicationManager mockedManager = new MockedReplicationManager();
+
+ Mockito
+ .when(mockedManager.manager.getConfig())
+ .thenReturn(mockedManager.conf);
+
+ Mockito
+ .when(mockedManager.manager.isContainerReplicatingOrDeleting(Mockito.any(ContainerID.class)))
+ .thenReturn(false);
+
+ Mockito
+ .when(mockedManager.manager.move(
+ Mockito.any(ContainerID.class),
+ Mockito.any(DatanodeDetails.class),
+ Mockito.any(DatanodeDetails.class)))
+ .thenReturn(CompletableFuture.completedFuture(MoveManager.MoveResult.COMPLETED));
+
+ Mockito
+ .when(mockedManager.manager.getClock())
+ .thenReturn(Clock.system(ZoneId.systemDefault()));
+
+ return mockedManager;
+ }
+ }
+
+ private static final class MockedServiceStateManager {
+ private final Map serviceToConfigMap = new HashMap<>();
+ private final StatefulServiceStateManager serviceStateManager = Mockito.mock(StatefulServiceStateManagerImpl.class);
+
+ private static @Nonnull StatefulServiceStateManager doMock() throws IOException {
+ MockedServiceStateManager manager = new MockedServiceStateManager();
+ // When StatefulServiceStateManager#saveConfiguration is called, save to in-memory serviceToConfigMap instead.
+ Map map = manager.serviceToConfigMap;
+
+ StatefulServiceStateManager stateManager = manager.serviceStateManager;
+ Mockito
+ .doAnswer(i -> {
+ map.put(i.getArgument(0, String.class), i.getArgument(1, ByteString.class));
+ return null;
+ })
+ .when(stateManager)
+ .saveConfiguration(Mockito.any(String.class), Mockito.any(ByteString.class));
+
+ // When StatefulServiceStateManager#readConfiguration is called, read from serviceToConfigMap instead.
+ Mockito
+ .when(stateManager.readConfiguration(Mockito.anyString()))
+ .thenAnswer(i -> map.get(i.getArgument(0, String.class)));
+ return stateManager;
+ }
+ }
+
+ private static final class MockedPlacementPolicies {
+ private final PlacementPolicy placementPolicy;
+ private final PlacementPolicy ecPlacementPolicy;
+ private final PlacementPolicyValidateProxy validateProxyPolicy;
+
+ private MockedPlacementPolicies(@Nonnull PlacementPolicy placementPolicy, @Nonnull PlacementPolicy ecPolicy) {
+ this.placementPolicy = placementPolicy;
+ ecPlacementPolicy = ecPolicy;
+ validateProxyPolicy = new PlacementPolicyValidateProxy(this.placementPolicy, ecPlacementPolicy);
+ }
+
+ private static @Nonnull MockedPlacementPolicies doMock(
+ @Nonnull OzoneConfiguration ozoneConfig,
+ @Nonnull NodeManager nodeManager
+ ) throws SCMException {
+ NetworkTopology clusterMap = nodeManager.getClusterNetworkTopologyMap();
+ PlacementPolicy policy = ContainerPlacementPolicyFactory.getPolicy(
+ ozoneConfig, nodeManager, clusterMap, true,
+ SCMContainerPlacementMetrics.create());
+ PlacementPolicy ecPolicy = ContainerPlacementPolicyFactory.getECPolicy(
+ ozoneConfig, nodeManager, clusterMap, true,
+ SCMContainerPlacementMetrics.create());
+ return new MockedPlacementPolicies(policy, ecPolicy);
+ }
+ }
+}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancer.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancer.java
index d5ae7457db43..f30b94f9cdff 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancer.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancer.java
@@ -18,140 +18,98 @@
package org.apache.hadoop.hdds.scm.container.balancer;
-import com.google.protobuf.ByteString;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.scm.ha.SCMContext;
-import org.apache.hadoop.hdds.scm.ha.SCMServiceManager;
-import org.apache.hadoop.hdds.scm.ha.StatefulServiceStateManager;
-import org.apache.hadoop.hdds.scm.ha.StatefulServiceStateManagerImpl;
-import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
import org.apache.ozone.test.GenericTestUtils;
import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
-import org.mockito.Mockito;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.slf4j.event.Level;
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_NODE_REPORT_INTERVAL;
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
/**
* Tests for {@link ContainerBalancer}.
*/
@Timeout(60)
public class TestContainerBalancer {
- private static final Logger LOG =
- LoggerFactory.getLogger(TestContainerBalancer.class);
+ private static final Logger LOG = LoggerFactory.getLogger(TestContainerBalancer.class);
- private ContainerBalancer containerBalancer;
- private StorageContainerManager scm;
- private ContainerBalancerConfiguration balancerConfiguration;
- private Map serviceToConfigMap = new HashMap<>();
- private StatefulServiceStateManager serviceStateManager;
- private OzoneConfiguration conf;
+ private MockedSCM mockedScm;
+ @BeforeAll
+ public static void setupAll() {
+ GenericTestUtils.setLogLevel(ContainerBalancer.LOG, Level.DEBUG);
+ }
/**
* Sets up configuration values and creates a mock cluster.
*/
@BeforeEach
- public void setup() throws IOException, NodeNotFoundException,
- TimeoutException {
- conf = new OzoneConfiguration();
- conf.setTimeDuration(HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT,
- 5, TimeUnit.SECONDS);
+ public void setup() throws IOException, NodeNotFoundException, TimeoutException {
+ OzoneConfiguration conf = new OzoneConfiguration();
+ conf.setTimeDuration(HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT, 5, TimeUnit.SECONDS);
conf.setTimeDuration(HDDS_NODE_REPORT_INTERVAL, 2, TimeUnit.SECONDS);
- scm = Mockito.mock(StorageContainerManager.class);
- serviceStateManager = Mockito.mock(StatefulServiceStateManagerImpl.class);
- balancerConfiguration =
- conf.getObject(ContainerBalancerConfiguration.class);
+
+ ContainerBalancerConfiguration balancerConfiguration = conf.getObject(ContainerBalancerConfiguration.class);
balancerConfiguration.setThreshold(10);
balancerConfiguration.setIterations(10);
- // Note: this will make container balancer task to wait for running
- // for 6 sec as default and ensure below test case have sufficient
- // time to verify, and interrupt when stop.
+ // Note: this will make container balancer task to wait for running for 6 sec as default and
+ // ensure below test case have sufficient time to verify, and interrupt when stop.
balancerConfiguration.setTriggerDuEnable(true);
conf.setFromObject(balancerConfiguration);
- GenericTestUtils.setLogLevel(ContainerBalancer.LOG, Level.DEBUG);
-
- when(scm.getScmNodeManager()).thenReturn(mock(NodeManager.class));
- when(scm.getScmContext()).thenReturn(SCMContext.emptyContext());
- when(scm.getConfiguration()).thenReturn(conf);
- when(scm.getStatefulServiceStateManager()).thenReturn(serviceStateManager);
- when(scm.getSCMServiceManager()).thenReturn(mock(SCMServiceManager.class));
- when(scm.getMoveManager()).thenReturn(Mockito.mock(MoveManager.class));
- /*
- When StatefulServiceStateManager#saveConfiguration is called, save to
- in-memory serviceToConfigMap and read from same.
- */
- Mockito.doAnswer(i -> {
- serviceToConfigMap.put(i.getArgument(0, String.class), i.getArgument(1,
- ByteString.class));
- return null;
- }).when(serviceStateManager).saveConfiguration(
- Mockito.any(String.class),
- Mockito.any(ByteString.class));
- when(serviceStateManager.readConfiguration(Mockito.anyString())).thenAnswer(
- i -> serviceToConfigMap.get(i.getArgument(0, String.class)));
-
- containerBalancer = new ContainerBalancer(scm);
+ mockedScm = MockedSCM.getMockedSCM(10, conf, balancerConfiguration);
}
@Test
public void testShouldRun() throws Exception {
+ ContainerBalancer containerBalancer = mockedScm.getContainerBalancer();
boolean doRun = containerBalancer.shouldRun();
Assertions.assertFalse(doRun);
- containerBalancer.saveConfiguration(balancerConfiguration, true, 0);
+ containerBalancer.saveConfiguration(mockedScm.getBalancerConfig(), true, 0);
doRun = containerBalancer.shouldRun();
Assertions.assertTrue(doRun);
- containerBalancer.saveConfiguration(balancerConfiguration, false, 0);
+ containerBalancer.saveConfiguration(mockedScm.getBalancerConfig(), false, 0);
doRun = containerBalancer.shouldRun();
Assertions.assertFalse(doRun);
}
@Test
public void testStartBalancerStop() throws Exception {
- startBalancer(balancerConfiguration);
+ ContainerBalancer containerBalancer = mockedScm.getContainerBalancer();
+ containerBalancer.startBalancer(mockedScm.getBalancerConfig());
try {
- containerBalancer.startBalancer(balancerConfiguration);
- Assertions.assertTrue(false,
- "Exception should be thrown when startBalancer again");
+ containerBalancer.startBalancer(mockedScm.getBalancerConfig());
+ Assertions.fail("Exception should be thrown when startBalancer again");
} catch (IllegalContainerBalancerStateException e) {
// start failed again, valid case
}
try {
containerBalancer.start();
- Assertions.assertTrue(false,
- "Exception should be thrown when start again");
+ Assertions.fail("Exception should be thrown when start again");
} catch (IllegalContainerBalancerStateException e) {
// start failed again, valid case
}
- Assertions.assertTrue(containerBalancer.getBalancerStatus()
- == ContainerBalancerTask.Status.RUNNING);
+ Assertions.assertSame(containerBalancer.getBalancerStatus(), ContainerBalancerTask.Status.RUNNING);
stopBalancer();
- Assertions.assertTrue(containerBalancer.getBalancerStatus()
- == ContainerBalancerTask.Status.STOPPED);
+ Assertions.assertSame(containerBalancer.getBalancerStatus(), ContainerBalancerTask.Status.STOPPED);
try {
containerBalancer.stopBalancer();
- Assertions.assertTrue(false,
- "Exception should be thrown when stop again");
+ Assertions.fail("Exception should be thrown when stop again");
} catch (Exception e) {
// stop failed as already stopped, valid case
}
@@ -159,129 +117,104 @@ public void testStartBalancerStop() throws Exception {
@Test
public void testStartStopSCMCalls() throws Exception {
- containerBalancer.saveConfiguration(balancerConfiguration, true, 0);
+ ContainerBalancer containerBalancer = mockedScm.getContainerBalancer();
+ containerBalancer.saveConfiguration(mockedScm.getBalancerConfig(), true, 0);
containerBalancer.start();
- Assertions.assertTrue(containerBalancer.getBalancerStatus()
- == ContainerBalancerTask.Status.RUNNING);
+ Assertions.assertSame(containerBalancer.getBalancerStatus(), ContainerBalancerTask.Status.RUNNING);
containerBalancer.notifyStatusChanged();
try {
containerBalancer.start();
- Assertions.assertTrue(false,
- "Exception should be thrown when start again");
+ Assertions.fail("Exception should be thrown when start again");
} catch (IllegalContainerBalancerStateException e) {
// start failed when triggered again, valid case
}
- Assertions.assertTrue(containerBalancer.getBalancerStatus()
- == ContainerBalancerTask.Status.RUNNING);
+ Assertions.assertSame(containerBalancer.getBalancerStatus(), ContainerBalancerTask.Status.RUNNING);
containerBalancer.stop();
- Assertions.assertTrue(containerBalancer.getBalancerStatus()
- == ContainerBalancerTask.Status.STOPPED);
- containerBalancer.saveConfiguration(balancerConfiguration, false, 0);
+ Assertions.assertSame(containerBalancer.getBalancerStatus(), ContainerBalancerTask.Status.STOPPED);
+ containerBalancer.saveConfiguration(mockedScm.getBalancerConfig(), false, 0);
}
@Test
public void testNotifyStateChangeStopStart() throws Exception {
- containerBalancer.startBalancer(balancerConfiguration);
+ ContainerBalancer containerBalancer = mockedScm.getContainerBalancer();
+ containerBalancer.startBalancer(mockedScm.getBalancerConfig());
+ StorageContainerManager scm = mockedScm.getStorageContainerManager();
scm.getScmContext().updateLeaderAndTerm(false, 1);
- Assertions.assertTrue(containerBalancer.getBalancerStatus()
- == ContainerBalancerTask.Status.RUNNING);
+ Assertions.assertSame(containerBalancer.getBalancerStatus(), ContainerBalancerTask.Status.RUNNING);
containerBalancer.notifyStatusChanged();
- Assertions.assertTrue(containerBalancer.getBalancerStatus()
- == ContainerBalancerTask.Status.STOPPED);
+ Assertions.assertSame(containerBalancer.getBalancerStatus(), ContainerBalancerTask.Status.STOPPED);
scm.getScmContext().updateLeaderAndTerm(true, 2);
scm.getScmContext().setLeaderReady();
containerBalancer.notifyStatusChanged();
- Assertions.assertTrue(containerBalancer.getBalancerStatus()
- == ContainerBalancerTask.Status.RUNNING);
+ Assertions.assertSame(containerBalancer.getBalancerStatus(), ContainerBalancerTask.Status.RUNNING);
containerBalancer.stop();
- Assertions.assertTrue(containerBalancer.getBalancerStatus()
- == ContainerBalancerTask.Status.STOPPED);
+ Assertions.assertSame(containerBalancer.getBalancerStatus(), ContainerBalancerTask.Status.STOPPED);
}
/**
- * This tests that ContainerBalancer rejects certain invalid configurations
- * while starting. It should fail to start in some cases.
+ * This tests that ContainerBalancer rejects certain invalid configurations while starting.
+ * It should fail to start in some cases.
*/
@Test
public void testValidationOfConfigurations() {
- conf = new OzoneConfiguration();
+ OzoneConfiguration conf = mockedScm.getOzoneConfig();
+ conf.setTimeDuration("hdds.container.balancer.move.replication.timeout", 60, TimeUnit.MINUTES);
+ conf.setTimeDuration("hdds.container.balancer.move.timeout", 59, TimeUnit.MINUTES);
- conf.setTimeDuration(
- "hdds.container.balancer.move.replication.timeout", 60,
- TimeUnit.MINUTES);
-
- conf.setTimeDuration("hdds.container.balancer.move.timeout", 59,
- TimeUnit.MINUTES);
-
- balancerConfiguration =
- conf.getObject(ContainerBalancerConfiguration.class);
+ ContainerBalancerConfiguration balancerConfiguration = mockedScm.getBalancerConfigByOzoneConfig(conf);
+ ContainerBalancer containerBalancer = mockedScm.getContainerBalancer();
Assertions.assertThrowsExactly(
InvalidContainerBalancerConfigurationException.class,
() -> containerBalancer.startBalancer(balancerConfiguration),
- "hdds.container.balancer.move.replication.timeout should " +
- "be less than hdds.container.balancer.move.timeout.");
+ "hdds.container.balancer.move.replication.timeout should be less than hdds.container.balancer.move.timeout.");
}
/**
- * Tests that ContainerBalancerTask starts with a delay of
- * "hdds.scm.wait.time.after.safemode.exit" when ContainerBalancer receives
- * status change notification in
- * {@link ContainerBalancer#notifyStatusChanged()}.
+ * Tests that ContainerBalancerTask starts with a delay of "hdds.scm.wait.time.after.safemode.exit"
+ * when ContainerBalancer receives status change notification in {@link ContainerBalancer#notifyStatusChanged()}.
*/
@Test
public void testDelayedStartOnSCMStatusChange()
- throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException, TimeoutException,
- InterruptedException {
- long delayDuration = conf.getTimeDuration(
- HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT, 10, TimeUnit.SECONDS);
- balancerConfiguration =
- conf.getObject(ContainerBalancerConfiguration.class);
+ throws IllegalContainerBalancerStateException, IOException, InvalidContainerBalancerConfigurationException,
+ TimeoutException, InterruptedException {
+ OzoneConfiguration ozoneCfg = mockedScm.getOzoneConfig();
+ long delayDuration = ozoneCfg.getTimeDuration(HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT, 10, TimeUnit.SECONDS);
+ ContainerBalancerConfiguration balancerConfiguration = mockedScm.getBalancerConfigByOzoneConfig(ozoneCfg);
+ ContainerBalancer containerBalancer = mockedScm.getContainerBalancer();
// Start the ContainerBalancer service.
containerBalancer.startBalancer(balancerConfiguration);
- GenericTestUtils.waitFor(() -> containerBalancer.isBalancerRunning(), 1,
- 20);
+ GenericTestUtils.waitFor(() -> containerBalancer.isBalancerRunning(), 1, 20);
Assertions.assertTrue(containerBalancer.isBalancerRunning());
- // Balancer should stop the current balancing thread when it receives a
- // status change notification
+ StorageContainerManager scm = mockedScm.getStorageContainerManager();
+ // Balancer should stop the current balancing thread when it receives a status change notification
scm.getScmContext().updateLeaderAndTerm(false, 1);
containerBalancer.notifyStatusChanged();
Assertions.assertFalse(containerBalancer.isBalancerRunning());
- GenericTestUtils.LogCapturer logCapturer =
- GenericTestUtils.LogCapturer.captureLogs(ContainerBalancerTask.LOG);
- String expectedLog = "ContainerBalancer will sleep for " + delayDuration +
- " seconds before starting balancing.";
- /*
- Send a status change notification again and check whether balancer
- starts balancing. We're actually just checking for the expected log
- line here.
- */
+ GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer.captureLogs(ContainerBalancerTask.LOG);
+ String expectedLog = "ContainerBalancer will sleep for " + delayDuration + " seconds before starting balancing.";
+
+ // Send a status change notification again and check whether balancer starts balancing.
+ // We're actually just checking for the expected log line here.
scm.getScmContext().updateLeaderAndTerm(true, 2);
scm.getScmContext().setLeaderReady();
containerBalancer.notifyStatusChanged();
Assertions.assertTrue(containerBalancer.isBalancerRunning());
Thread balancingThread = containerBalancer.getCurrentBalancingThread();
- GenericTestUtils.waitFor(
- () -> balancingThread.getState() == Thread.State.TIMED_WAITING, 2, 20);
+ GenericTestUtils.waitFor(() -> balancingThread.getState() == Thread.State.TIMED_WAITING, 2, 20);
Assertions.assertTrue(logCapturer.getOutput().contains(expectedLog));
stopBalancer();
}
- private void startBalancer(ContainerBalancerConfiguration config)
- throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException, TimeoutException {
- containerBalancer.startBalancer(config);
- }
-
private void stopBalancer() {
try {
+ ContainerBalancer containerBalancer = mockedScm.getContainerBalancer();
if (containerBalancer.isBalancerRunning()) {
containerBalancer.stopBalancer();
}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerTask.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerTask.java
index 4bc3cf43cf6e..8988fe80c911 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerTask.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerTask.java
@@ -18,489 +18,374 @@
package org.apache.hadoop.hdds.scm.container.balancer;
-import com.google.protobuf.ByteString;
-import org.apache.hadoop.conf.StorageUnit;
-import org.apache.hadoop.hdds.client.ECReplicationConfig;
-import org.apache.hadoop.hdds.client.RatisReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
import org.apache.hadoop.hdds.scm.PlacementPolicy;
-import org.apache.hadoop.hdds.scm.PlacementPolicyValidateProxy;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
-import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
import org.apache.hadoop.hdds.scm.container.ContainerReplica;
import org.apache.hadoop.hdds.scm.container.MockNodeManager;
-import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementPolicyFactory;
-import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementMetrics;
-import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
-import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
-import org.apache.hadoop.hdds.scm.ha.SCMContext;
-import org.apache.hadoop.hdds.scm.ha.SCMService;
-import org.apache.hadoop.hdds.scm.ha.SCMServiceManager;
-import org.apache.hadoop.hdds.scm.ha.StatefulServiceStateManager;
-import org.apache.hadoop.hdds.scm.ha.StatefulServiceStateManagerImpl;
-import org.apache.hadoop.hdds.scm.net.NetworkTopology;
+import org.apache.hadoop.hdds.scm.container.balancer.iteration.ContainerBalanceIteration;
+import org.apache.hadoop.hdds.scm.container.balancer.iteration.IterationResult;
+import org.apache.hadoop.hdds.scm.container.balancer.iteration.IterationMetrics;
import org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo;
import org.apache.hadoop.hdds.scm.node.NodeStatus;
import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
-import org.apache.hadoop.hdds.server.events.EventPublisher;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.ozone.test.GenericTestUtils;
import org.apache.ozone.test.tag.Unhealthy;
import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
import org.mockito.Mockito;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import org.slf4j.event.Level;
+import javax.annotation.Nonnull;
import java.io.IOException;
-import java.time.Clock;
import java.time.Duration;
-import java.time.ZoneId;
import java.util.ArrayList;
-import java.util.HashMap;
+import java.util.Arrays;
+import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
+import java.util.stream.Stream;
-import static org.apache.hadoop.hdds.scm.container.replication.ReplicationManager.ReplicationManagerConfiguration;
+import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.atLeastOnce;
-import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.when;
/**
* Tests for {@link ContainerBalancer}.
*/
public class TestContainerBalancerTask {
-
- private static final Logger LOG =
- LoggerFactory.getLogger(TestContainerBalancerTask.class);
-
- private ReplicationManager replicationManager;
- private MoveManager moveManager;
- private ContainerManager containerManager;
- private ContainerBalancerTask containerBalancerTask;
- private MockNodeManager mockNodeManager;
- private StorageContainerManager scm;
- private OzoneConfiguration conf;
- private ReplicationManagerConfiguration rmConf;
- private PlacementPolicy placementPolicy;
- private PlacementPolicy ecPlacementPolicy;
- private PlacementPolicyValidateProxy placementPolicyValidateProxy;
- private ContainerBalancerConfiguration balancerConfiguration;
- private List nodesInCluster;
- private List nodeUtilizations;
- private double averageUtilization;
- private int numberOfNodes;
- private Map> cidToReplicasMap =
- new HashMap<>();
- private Map cidToInfoMap = new HashMap<>();
- private Map> datanodeToContainersMap =
- new HashMap<>();
- private Map serviceToConfigMap = new HashMap<>();
- private static final ThreadLocalRandom RANDOM = ThreadLocalRandom.current();
-
- private StatefulServiceStateManager serviceStateManager;
- private static final long STORAGE_UNIT = OzoneConsts.GB;
-
- /**
- * Sets up configuration values and creates a mock cluster.
- */
- @BeforeEach
- public void setup() throws IOException, NodeNotFoundException,
- TimeoutException {
- conf = new OzoneConfiguration();
- rmConf = new ReplicationManagerConfiguration();
- scm = Mockito.mock(StorageContainerManager.class);
- containerManager = Mockito.mock(ContainerManager.class);
- replicationManager = Mockito.mock(ReplicationManager.class);
- serviceStateManager = Mockito.mock(StatefulServiceStateManagerImpl.class);
- SCMServiceManager scmServiceManager = Mockito.mock(SCMServiceManager.class);
- moveManager = Mockito.mock(MoveManager.class);
- Mockito.when(moveManager.move(any(ContainerID.class),
- any(DatanodeDetails.class), any(DatanodeDetails.class)))
- .thenReturn(CompletableFuture.completedFuture(
- MoveManager.MoveResult.COMPLETED));
-
- /*
- Disable LegacyReplicationManager. This means balancer should select RATIS
- as well as EC containers for balancing. Also, MoveManager will be used.
- */
- Mockito.when(replicationManager.getConfig()).thenReturn(rmConf);
- rmConf.setEnableLegacy(false);
- // these configs will usually be specified in each test
- balancerConfiguration =
- conf.getObject(ContainerBalancerConfiguration.class);
- balancerConfiguration.setThreshold(10);
- balancerConfiguration.setIterations(1);
- balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
- balancerConfiguration.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
- balancerConfiguration.setMaxSizeEnteringTarget(50 * STORAGE_UNIT);
- conf.setFromObject(balancerConfiguration);
+ @BeforeAll
+ public static void setup() {
GenericTestUtils.setLogLevel(ContainerBalancerTask.LOG, Level.DEBUG);
-
- averageUtilization = createCluster();
- mockNodeManager = new MockNodeManager(datanodeToContainersMap);
-
- NetworkTopology clusterMap = mockNodeManager.getClusterNetworkTopologyMap();
-
- placementPolicy = ContainerPlacementPolicyFactory
- .getPolicy(conf, mockNodeManager, clusterMap, true,
- SCMContainerPlacementMetrics.create());
- ecPlacementPolicy = ContainerPlacementPolicyFactory.getECPolicy(
- conf, mockNodeManager, clusterMap,
- true, SCMContainerPlacementMetrics.create());
- placementPolicyValidateProxy = new PlacementPolicyValidateProxy(
- placementPolicy, ecPlacementPolicy);
-
- Mockito.when(replicationManager
- .isContainerReplicatingOrDeleting(Mockito.any(ContainerID.class)))
- .thenReturn(false);
-
- Mockito.when(replicationManager.move(Mockito.any(ContainerID.class),
- Mockito.any(DatanodeDetails.class),
- Mockito.any(DatanodeDetails.class)))
- .thenReturn(CompletableFuture.
- completedFuture(MoveManager.MoveResult.COMPLETED));
-
- Mockito.when(replicationManager.getClock())
- .thenReturn(Clock.system(ZoneId.systemDefault()));
-
- when(containerManager.getContainerReplicas(Mockito.any(ContainerID.class)))
- .thenAnswer(invocationOnMock -> {
- ContainerID cid = (ContainerID) invocationOnMock.getArguments()[0];
- return cidToReplicasMap.get(cid);
- });
-
- when(containerManager.getContainer(Mockito.any(ContainerID.class)))
- .thenAnswer(invocationOnMock -> {
- ContainerID cid = (ContainerID) invocationOnMock.getArguments()[0];
- return cidToInfoMap.get(cid);
- });
-
- when(containerManager.getContainers())
- .thenReturn(new ArrayList<>(cidToInfoMap.values()));
-
- when(scm.getScmNodeManager()).thenReturn(mockNodeManager);
- when(scm.getContainerPlacementPolicy()).thenReturn(placementPolicy);
- when(scm.getContainerManager()).thenReturn(containerManager);
- when(scm.getReplicationManager()).thenReturn(replicationManager);
- when(scm.getScmContext()).thenReturn(SCMContext.emptyContext());
- when(scm.getClusterMap()).thenReturn(null);
- when(scm.getEventQueue()).thenReturn(mock(EventPublisher.class));
- when(scm.getConfiguration()).thenReturn(conf);
- when(scm.getStatefulServiceStateManager()).thenReturn(serviceStateManager);
- when(scm.getSCMServiceManager()).thenReturn(scmServiceManager);
- when(scm.getPlacementPolicyValidateProxy())
- .thenReturn(placementPolicyValidateProxy);
- when(scm.getMoveManager()).thenReturn(moveManager);
-
- /*
- When StatefulServiceStateManager#saveConfiguration is called, save to
- in-memory serviceToConfigMap instead.
- */
- Mockito.doAnswer(i -> {
- serviceToConfigMap.put(i.getArgument(0, String.class), i.getArgument(1,
- ByteString.class));
- return null;
- }).when(serviceStateManager).saveConfiguration(
- Mockito.any(String.class),
- Mockito.any(ByteString.class));
-
- /*
- When StatefulServiceStateManager#readConfiguration is called, read from
- serviceToConfigMap instead.
- */
- when(serviceStateManager.readConfiguration(Mockito.anyString())).thenAnswer(
- i -> serviceToConfigMap.get(i.getArgument(0, String.class)));
-
- Mockito.doNothing().when(scmServiceManager)
- .register(Mockito.any(SCMService.class));
- ContainerBalancer sb = new ContainerBalancer(scm);
- containerBalancerTask = new ContainerBalancerTask(scm, 0, sb,
- sb.getMetrics(), balancerConfiguration, false);
}
- @Test
- public void testCalculationOfUtilization() {
- Assertions.assertEquals(nodesInCluster.size(), nodeUtilizations.size());
- for (int i = 0; i < nodesInCluster.size(); i++) {
- Assertions.assertEquals(nodeUtilizations.get(i),
- nodesInCluster.get(i).calculateUtilization(), 0.0001);
+ private static Stream createMockedSCMs() {
+ return Stream.of(
+ Arguments.of(MockedSCM.getMockedSCM(4)),
+ Arguments.of(MockedSCM.getMockedSCM(5)),
+ Arguments.of(MockedSCM.getMockedSCM(6)),
+ Arguments.of(MockedSCM.getMockedSCM(7)),
+ Arguments.of(MockedSCM.getMockedSCM(8)),
+ Arguments.of(MockedSCM.getMockedSCM(9)),
+ Arguments.of(MockedSCM.getMockedSCM(10)),
+ Arguments.of(MockedSCM.getMockedSCM(17)),
+ Arguments.of(MockedSCM.getMockedSCM(30)));
+ }
+
+ private static Stream createMockedSCMWithDatanodeLimits() {
+ return Stream.of(
+ // Doesn't make sense to set limits formit for maximum datanode count
+ // per iteration for cluster of 4 nodes
+ Arguments.of(MockedSCM.getMockedSCM(4), false),
+ Arguments.of(MockedSCM.getMockedSCM(5), true),
+ Arguments.of(MockedSCM.getMockedSCM(5), false),
+ Arguments.of(MockedSCM.getMockedSCM(6), true),
+ Arguments.of(MockedSCM.getMockedSCM(6), false),
+ Arguments.of(MockedSCM.getMockedSCM(7), true),
+ Arguments.of(MockedSCM.getMockedSCM(7), false),
+ Arguments.of(MockedSCM.getMockedSCM(8), true),
+ Arguments.of(MockedSCM.getMockedSCM(8), false),
+ Arguments.of(MockedSCM.getMockedSCM(9), true),
+ Arguments.of(MockedSCM.getMockedSCM(9), false),
+ Arguments.of(MockedSCM.getMockedSCM(10), true),
+ Arguments.of(MockedSCM.getMockedSCM(10), false),
+ Arguments.of(MockedSCM.getMockedSCM(11), true),
+ Arguments.of(MockedSCM.getMockedSCM(11), false),
+ Arguments.of(MockedSCM.getMockedSCM(12), true),
+ Arguments.of(MockedSCM.getMockedSCM(12), false),
+ Arguments.of(MockedSCM.getMockedSCM(15), true),
+ Arguments.of(MockedSCM.getMockedSCM(15), false),
+ Arguments.of(MockedSCM.getMockedSCM(19), true),
+ Arguments.of(MockedSCM.getMockedSCM(19), false),
+ Arguments.of(MockedSCM.getMockedSCM(20), true),
+ Arguments.of(MockedSCM.getMockedSCM(20), false));
+ }
+
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}")
+ @MethodSource("createMockedSCMs")
+ public void testCalculationOfUtilization(@Nonnull MockedSCM mockedSCM) {
+ TestableCluster cluster = mockedSCM.getCluster();
+ DatanodeUsageInfo[] nodesInCluster = cluster.getNodesInCluster();
+ double[] nodeUtilizationList = cluster.getNodeUtilizationList();
+ assertEquals(nodesInCluster.length, nodeUtilizationList.length);
+ for (int i = 0; i < nodesInCluster.length; i++) {
+ assertEquals(nodeUtilizationList[i], nodesInCluster[i].calculateUtilization(), 0.0001);
}
-
- // should be equal to average utilization of the cluster
- Assertions.assertEquals(averageUtilization,
- containerBalancerTask.calculateAvgUtilization(nodesInCluster), 0.0001);
+ // Should be equal to average utilization of the cluster
+ assertEquals(cluster.getAverageUtilization(),
+ ContainerBalanceIteration.calculateAvgUtilization(Arrays.asList(nodesInCluster)), 0.0001);
}
/**
- * Checks whether ContainerBalancer is correctly updating the list of
- * unBalanced nodes with varying values of Threshold.
+ * Checks whether ContainerBalancer is correctly updating the list of unBalanced nodes with
+ * varying values of Threshold.
*/
- @Test
- public void
- initializeIterationShouldUpdateUnBalancedNodesWhenThresholdChanges() {
- List expectedUnBalancedNodes;
- List unBalancedNodesAccordingToBalancer;
-
- // check for random threshold values
- ContainerBalancer sb = new ContainerBalancer(scm);
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource("createMockedSCMWithDatanodeLimits")
+ public void testBalancerTaskAfterChangingThresholdValue(@Nonnull MockedSCM mockedSCM, boolean ignored) {
+ // Check for random threshold values
+ ContainerBalancer balancer = mockedSCM.createContainerBalancer();
+ TestableCluster cluster = mockedSCM.getCluster();
for (int i = 0; i < 50; i++) {
- double randomThreshold = RANDOM.nextDouble() * 100;
+ double randomThreshold = TestableCluster.RANDOM.nextDouble() * 100;
- expectedUnBalancedNodes =
- determineExpectedUnBalancedNodes(randomThreshold);
+ List expectedUnBalancedNodes = cluster.getUnBalancedNodes(randomThreshold);
+ // Sort unbalanced nodes as it is done inside ContainerBalancerTask:337 in descending order by node utilization
+ // (most used node)
+ expectedUnBalancedNodes.sort(
+ (d1, d2) -> Double.compare(d2.calculateUtilization(), d1.calculateUtilization())
+ );
- balancerConfiguration.setThreshold(randomThreshold);
- containerBalancerTask = new ContainerBalancerTask(scm, 0, sb,
- sb.getMetrics(), balancerConfiguration, false);
- containerBalancerTask.run();
+ ContainerBalancerConfiguration config = mockedSCM.getBalancerConfig();
+ config.setThreshold(randomThreshold);
+ ContainerBalancerTask task = mockedSCM.startBalancerTask(balancer, config);
- unBalancedNodesAccordingToBalancer =
- containerBalancerTask.getUnBalancedNodes();
+ List actualUnBalancedNodes = new ArrayList<>();
+ actualUnBalancedNodes.addAll(task.getOverUtilizedNodes());
+ actualUnBalancedNodes.addAll(task.getUnderUtilizedNodes());
- Assertions.assertEquals(
- expectedUnBalancedNodes.size(),
- unBalancedNodesAccordingToBalancer.size());
+ assertEquals(expectedUnBalancedNodes.size(), actualUnBalancedNodes.size());
for (int j = 0; j < expectedUnBalancedNodes.size(); j++) {
- Assertions.assertEquals(
+ assertEquals(
expectedUnBalancedNodes.get(j).getDatanodeDetails(),
- unBalancedNodesAccordingToBalancer.get(j).getDatanodeDetails());
+ actualUnBalancedNodes.get(j).getDatanodeDetails());
}
}
}
- @Test
- public void testBalancerWithMoveManager()
- throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException, TimeoutException,
- NodeNotFoundException {
- rmConf.setEnableLegacy(false);
- startBalancer(balancerConfiguration);
- Mockito.verify(moveManager, atLeastOnce())
- .move(Mockito.any(ContainerID.class),
- Mockito.any(DatanodeDetails.class),
- Mockito.any(DatanodeDetails.class));
-
- Mockito.verify(replicationManager, times(0))
- .move(Mockito.any(ContainerID.class), Mockito.any(
- DatanodeDetails.class), Mockito.any(DatanodeDetails.class));
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource("createMockedSCMWithDatanodeLimits")
+ public void testBalancerWithMoveManager(@Nonnull MockedSCM mockedSCM, boolean ignored)
+ throws IOException, TimeoutException, NodeNotFoundException {
+ mockedSCM.disableLegacyReplicationManager();
+ mockedSCM.startBalancerTask(mockedSCM.getBalancerConfig());
+
+ Mockito
+ .verify(mockedSCM.getMoveManager(), atLeastOnce())
+ .move(
+ any(ContainerID.class),
+ any(DatanodeDetails.class),
+ any(DatanodeDetails.class));
+ Mockito
+ .verify(mockedSCM.getReplicationManager(), times(0))
+ .move(
+ any(ContainerID.class),
+ any(DatanodeDetails.class),
+ any(DatanodeDetails.class));
}
/**
- * Checks whether the list of unBalanced nodes is empty when the cluster is
- * balanced.
+ * Checks whether the list of unBalanced nodes is empty when the cluster is balanced.
*/
- @Test
- public void unBalancedNodesListShouldBeEmptyWhenClusterIsBalanced()
- throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException, TimeoutException {
- balancerConfiguration.setThreshold(99.99);
- startBalancer(balancerConfiguration);
-
-
- stopBalancer();
- ContainerBalancerMetrics metrics = containerBalancerTask.getMetrics();
- Assertions.assertEquals(0, containerBalancerTask.getUnBalancedNodes()
- .size());
- Assertions.assertEquals(0, metrics.getNumDatanodesUnbalanced());
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource("createMockedSCMWithDatanodeLimits")
+ public void unBalancedNodesListShouldBeEmptyWhenClusterIsBalanced(@Nonnull MockedSCM mockedSCM, boolean ignored) {
+ ContainerBalancerConfiguration config = mockedSCM.getBalancerConfig();
+ config.setThreshold(99.99);
+ ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
+
+ IterationMetrics metrics = task.getIterationMetrics();
+ assertFalse(stillHaveUnbalancedNodes(task));
+ assertEquals(0, metrics.getUnbalancedDatanodeCount());
}
/**
- * ContainerBalancer should not involve more datanodes than the
- * maxDatanodesRatioToInvolvePerIteration limit.
+ * ContainerBalancer should not involve more datanodes than the maxDatanodesRatioToInvolvePerIteration limit.
*/
- @Test
- public void containerBalancerShouldObeyMaxDatanodesToInvolveLimit()
- throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException, TimeoutException {
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource("createMockedSCMWithDatanodeLimits")
+ public void containerBalancerShouldObeyMaxDatanodesToInvolveLimit(
+ @Nonnull MockedSCM mockedSCM,
+ boolean useDatanodeLimits
+ ) {
+ ContainerBalancerConfiguration config = mockedSCM.getBalancerConfig();
int percent = 40;
- balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(
- percent);
- balancerConfiguration.setMaxSizeToMovePerIteration(100 * STORAGE_UNIT);
- balancerConfiguration.setThreshold(1);
- balancerConfiguration.setIterations(1);
- startBalancer(balancerConfiguration);
-
- int number = percent * numberOfNodes / 100;
- ContainerBalancerMetrics metrics = containerBalancerTask.getMetrics();
- Assertions.assertFalse(
- containerBalancerTask.getCountDatanodesInvolvedPerIteration() > number);
- Assertions.assertTrue(
- metrics.getNumDatanodesInvolvedInLatestIteration() > 0);
- Assertions.assertFalse(
- metrics.getNumDatanodesInvolvedInLatestIteration() > number);
- stopBalancer();
+ config.setMaxDatanodesPercentageToInvolvePerIteration(percent);
+ config.setMaxSizeToMovePerIteration(100 * MockedSCM.STORAGE_UNIT);
+ config.setThreshold(1);
+ config.setIterations(1);
+ config.setAdaptBalanceWhenCloseToLimit(useDatanodeLimits);
+ config.setAdaptBalanceWhenReachTheLimit(useDatanodeLimits);
+
+ ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
+ IterationMetrics metrics = task.getIterationMetrics();
+ assertTrue(metrics.getInvolvedDatanodeCount() > 0);
+ if (useDatanodeLimits) {
+ int number = percent * mockedSCM.getCluster().getNodeCount() / 100;
+ assertTrue(metrics.getInvolvedDatanodeCount() <= number);
+ }
}
- @Test
- public void containerBalancerShouldSelectOnlyClosedContainers()
- throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException, TimeoutException {
- // make all containers open, balancer should not select any of them
- for (ContainerInfo containerInfo : cidToInfoMap.values()) {
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource("createMockedSCMWithDatanodeLimits")
+ public void containerBalancerShouldSelectOnlyClosedContainers(@Nonnull MockedSCM mockedSCM, boolean ignored) {
+ // Make all containers open, balancer should not select any of them
+ Collection containers = mockedSCM.getCluster().getCidToInfoMap().values();
+ for (ContainerInfo containerInfo : containers) {
containerInfo.setState(HddsProtos.LifeCycleState.OPEN);
}
- balancerConfiguration.setThreshold(10);
- startBalancer(balancerConfiguration);
- stopBalancer();
-
- // balancer should have identified unbalanced nodes
- Assertions.assertFalse(containerBalancerTask.getUnBalancedNodes()
- .isEmpty());
- // no container should have been selected
- Assertions.assertTrue(containerBalancerTask.getContainerToSourceMap()
- .isEmpty());
- /*
- Iteration result should be CAN_NOT_BALANCE_ANY_MORE because no container
- move is generated
- */
- Assertions.assertEquals(
- ContainerBalancerTask.IterationResult.CAN_NOT_BALANCE_ANY_MORE,
- containerBalancerTask.getIterationResult());
-
- // now, close all containers
- for (ContainerInfo containerInfo : cidToInfoMap.values()) {
+
+ ContainerBalancerConfiguration config = mockedSCM.getBalancerConfig();
+ config.setThreshold(10);
+ ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
+
+ // Balancer should have identified unbalanced nodes
+ assertTrue(stillHaveUnbalancedNodes(task));
+ // No container should have been selected
+ assertTrue(task.getContainerToSourceMap().isEmpty());
+ // Iteration result should be CAN_NOT_BALANCE_ANY_MORE because no container move is generated
+ assertTrue(canNotBalanceAnyMore(task));
+
+ // Now, close all containers
+ for (ContainerInfo containerInfo : containers) {
containerInfo.setState(HddsProtos.LifeCycleState.CLOSED);
}
- startBalancer(balancerConfiguration);
- stopBalancer();
-
- // check whether all selected containers are closed
- for (ContainerID cid:
- containerBalancerTask.getContainerToSourceMap().keySet()) {
- Assertions.assertSame(
- cidToInfoMap.get(cid).getState(), HddsProtos.LifeCycleState.CLOSED);
+ mockedSCM.startBalancerTask(config);
+
+ // Check whether all selected containers are closed
+ for (ContainerInfo cid : containers) {
+ Assertions.assertSame(cid.getState(), HddsProtos.LifeCycleState.CLOSED);
}
}
/**
* Container Balancer should not select a non-CLOSED replica for moving.
*/
- @Test
- public void balancerShouldNotSelectNonClosedContainerReplicas()
- throws IOException, IllegalContainerBalancerStateException,
- InvalidContainerBalancerConfigurationException, TimeoutException {
-
- // let's mock such that all replicas have CLOSING state
- when(containerManager.getContainerReplicas(Mockito.any(ContainerID.class)))
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource("createMockedSCMWithDatanodeLimits")
+ public void balancerShouldNotSelectNonClosedContainerReplicas(@Nonnull MockedSCM mockedSCM, boolean ignored)
+ throws IOException {
+ // Let's mock such that all replicas have CLOSING state
+ ContainerManager containerManager = mockedSCM.getContainerManager();
+ Map> cidToReplicasMap = mockedSCM.getCluster().getCidToReplicasMap();
+ Mockito
+ .when(containerManager.getContainerReplicas(any(ContainerID.class)))
.thenAnswer(invocationOnMock -> {
ContainerID cid = (ContainerID) invocationOnMock.getArguments()[0];
Set replicas = cidToReplicasMap.get(cid);
- Set replicasToReturn =
- new HashSet<>(replicas.size());
+ Set replicasToReturn = new HashSet<>(replicas.size());
for (ContainerReplica replica : replicas) {
- ContainerReplica newReplica =
- replica.toBuilder().setContainerState(
- ContainerReplicaProto.State.CLOSING).build();
+ ContainerReplica newReplica = replica.toBuilder()
+ .setContainerState(ContainerReplicaProto.State.CLOSING)
+ .build();
replicasToReturn.add(newReplica);
}
-
return replicasToReturn;
});
- balancerConfiguration.setThreshold(10);
- balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
- balancerConfiguration.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
- balancerConfiguration.setMaxSizeEnteringTarget(50 * STORAGE_UNIT);
-
- startBalancer(balancerConfiguration);
- stopBalancer();
-
- // balancer should have identified unbalanced nodes
- Assertions.assertFalse(containerBalancerTask.getUnBalancedNodes()
- .isEmpty());
- // no container should have moved because all replicas are CLOSING
- Assertions.assertTrue(
- containerBalancerTask.getContainerToSourceMap().isEmpty());
+ ContainerBalancerConfiguration config = mockedSCM.getBalancerConfig();
+ config.setThreshold(10);
+ config.setMaxDatanodesPercentageToInvolvePerIteration(100);
+ config.setMaxSizeToMovePerIteration(50 * MockedSCM.STORAGE_UNIT);
+ config.setMaxSizeEnteringTarget(50 * MockedSCM.STORAGE_UNIT);
+
+ ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
+
+ // Balancer should have identified unbalanced nodes
+ assertTrue(stillHaveUnbalancedNodes(task));
+ // No container should have moved because all replicas are CLOSING
+ assertTrue(task.getContainerToSourceMap().isEmpty());
+ }
+
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource("createMockedSCMWithDatanodeLimits")
+ public void containerBalancerShouldObeyMaxSizeToMoveLimit(
+ @Nonnull MockedSCM mockedSCM, boolean useDatanodeLimits
+ ) {
+ ContainerBalancerConfiguration config = mockedSCM.getBalancerConfig();
+ config.setThreshold(1);
+ config.setMaxSizeToMovePerIteration(10 * MockedSCM.STORAGE_UNIT);
+ config.setIterations(1);
+ config.setMaxDatanodesPercentageToInvolvePerIteration(20);
+ config.setAdaptBalanceWhenCloseToLimit(useDatanodeLimits);
+ config.setAdaptBalanceWhenReachTheLimit(useDatanodeLimits);
+
+ ContainerBalancer balancer = mockedSCM.createContainerBalancer();
+ ContainerBalancerTask task = mockedSCM.startBalancerTask(balancer, config);
+
+ // Balancer should not have moved more size than the limit
+ long sizeForMove = balancer.getMetrics().getDataSizeMovedBytes();
+ assertFalse(sizeForMove > 10 * MockedSCM.STORAGE_UNIT);
+
+ long size = task.getIterationMetrics().getMovedBytesCount();
+ if (sizeForMove == 0) {
+ assertEquals(0, size);
+ } else {
+ assertTrue(size > 0);
+ assertFalse(size > 10 * MockedSCM.STORAGE_UNIT);
+ }
}
- @Test
- public void containerBalancerShouldObeyMaxSizeToMoveLimit()
- throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException, TimeoutException {
- balancerConfiguration.setThreshold(1);
- balancerConfiguration.setMaxSizeToMovePerIteration(10 * STORAGE_UNIT);
- balancerConfiguration.setIterations(1);
- startBalancer(balancerConfiguration);
-
- // balancer should not have moved more size than the limit
- Assertions.assertFalse(
- containerBalancerTask.getSizeScheduledForMoveInLatestIteration() >
- 10 * STORAGE_UNIT);
-
- long size = containerBalancerTask.getMetrics()
- .getDataSizeMovedGBInLatestIteration();
- Assertions.assertTrue(size > 0);
- Assertions.assertFalse(size > 10);
- stopBalancer();
- }
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource("createMockedSCMWithDatanodeLimits")
+ public void targetDatanodeShouldNotAlreadyContainSelectedContainer(@Nonnull MockedSCM mockedSCM, boolean ignored) {
+ ContainerBalancerConfiguration config = mockedSCM.getBalancerConfig();
+ config.setThreshold(10);
+ config.setMaxSizeToMovePerIteration(100 * MockedSCM.STORAGE_UNIT);
+ config.setMaxDatanodesPercentageToInvolvePerIteration(100);
+ ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
- @Test
- public void targetDatanodeShouldNotAlreadyContainSelectedContainer()
- throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException, TimeoutException {
- balancerConfiguration.setThreshold(10);
- balancerConfiguration.setMaxSizeToMovePerIteration(100 * STORAGE_UNIT);
- balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
- startBalancer(balancerConfiguration);
-
- stopBalancer();
- Map map =
- containerBalancerTask.getContainerToTargetMap();
+ Map> cidToReplicasMap = mockedSCM.getCluster().getCidToReplicasMap();
+ Map map = task.getContainerToTargetMap();
for (Map.Entry entry : map.entrySet()) {
ContainerID container = entry.getKey();
DatanodeDetails target = entry.getValue();
- Assertions.assertTrue(cidToReplicasMap.get(container)
+ assertTrue(cidToReplicasMap.get(container)
.stream()
.map(ContainerReplica::getDatanodeDetails)
.noneMatch(target::equals));
}
}
- @Test
- public void containerMoveSelectionShouldFollowPlacementPolicy()
- throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException, TimeoutException {
- balancerConfiguration.setThreshold(10);
- balancerConfiguration.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
- balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
- balancerConfiguration.setIterations(1);
- startBalancer(balancerConfiguration);
-
- stopBalancer();
- Map containerFromSourceMap =
- containerBalancerTask.getContainerToSourceMap();
- Map containerToTargetMap =
- containerBalancerTask.getContainerToTargetMap();
-
- // for each move selection, check if {replicas - source + target}
- // satisfies placement policy
- for (Map.Entry entry :
- containerFromSourceMap.entrySet()) {
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource("createMockedSCMWithDatanodeLimits")
+ public void containerMoveSelectionShouldFollowPlacementPolicy(@Nonnull MockedSCM mockedSCM, boolean ignored) {
+ ContainerBalancerConfiguration config = mockedSCM.getBalancerConfig();
+ config.setThreshold(10);
+ config.setMaxSizeToMovePerIteration(50 * MockedSCM.STORAGE_UNIT);
+ config.setMaxDatanodesPercentageToInvolvePerIteration(100);
+ config.setIterations(1);
+
+
+ TestableCluster cluster = mockedSCM.getCluster();
+ Map cidToInfoMap = cluster.getCidToInfoMap();
+ Map> cidToReplicasMap = cluster.getCidToReplicasMap();
+
+ PlacementPolicy policy = mockedSCM.getPlacementPolicy();
+ PlacementPolicy ecPolicy = mockedSCM.getEcPlacementPolicy();
+
+ ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
+ Map containerToTargetMap = task.getContainerToTargetMap();
+ Set> cnToDnDetailsMap = task.getContainerToSourceMap().entrySet();
+ // For each move selection, check if {replicas - source + target} satisfies placement policy
+ for (Map.Entry entry : cnToDnDetailsMap) {
ContainerID container = entry.getKey();
DatanodeDetails source = entry.getValue();
- List replicas = cidToReplicasMap.get(container)
+ List replicas = cidToReplicasMap
+ .get(container)
.stream()
.map(ContainerReplica::getDatanodeDetails)
.collect(Collectors.toList());
@@ -508,834 +393,355 @@ public void containerMoveSelectionShouldFollowPlacementPolicy()
replicas.remove(source);
replicas.add(containerToTargetMap.get(container));
- ContainerInfo containerInfo = cidToInfoMap.get(container);
- ContainerPlacementStatus placementStatus;
- if (containerInfo.getReplicationType() ==
- HddsProtos.ReplicationType.RATIS) {
- placementStatus = placementPolicy.validateContainerPlacement(replicas,
- containerInfo.getReplicationConfig().getRequiredNodes());
- } else {
- placementStatus =
- ecPlacementPolicy.validateContainerPlacement(replicas,
- containerInfo.getReplicationConfig().getRequiredNodes());
- }
- Assertions.assertTrue(placementStatus.isPolicySatisfied());
+ ContainerInfo info = cidToInfoMap.get(container);
+ int requiredNodes = info.getReplicationConfig().getRequiredNodes();
+ ContainerPlacementStatus status = (info.getReplicationType() == HddsProtos.ReplicationType.RATIS)
+ ? policy.validateContainerPlacement(replicas, requiredNodes)
+ : ecPolicy.validateContainerPlacement(replicas, requiredNodes);
+
+ assertTrue(status.isPolicySatisfied());
}
}
- @Test
- public void targetDatanodeShouldBeInServiceHealthy()
- throws NodeNotFoundException, IllegalContainerBalancerStateException,
- IOException, InvalidContainerBalancerConfigurationException,
- TimeoutException {
- balancerConfiguration.setThreshold(10);
- balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
- balancerConfiguration.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
- balancerConfiguration.setMaxSizeEnteringTarget(50 * STORAGE_UNIT);
- balancerConfiguration.setIterations(1);
- startBalancer(balancerConfiguration);
-
- stopBalancer();
- for (DatanodeDetails target : containerBalancerTask.getSelectedTargets()) {
- NodeStatus status = mockNodeManager.getNodeStatus(target);
- Assertions.assertSame(HddsProtos.NodeOperationalState.IN_SERVICE,
- status.getOperationalState());
- Assertions.assertTrue(status.isHealthy());
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource("createMockedSCMWithDatanodeLimits")
+ public void targetDatanodeShouldBeInServiceHealthy(@Nonnull MockedSCM mockedSCM, boolean ignored)
+ throws NodeNotFoundException {
+ ContainerBalancerConfiguration config = mockedSCM.getBalancerConfig();
+ config.setThreshold(10);
+ config.setMaxDatanodesPercentageToInvolvePerIteration(100);
+ config.setMaxSizeToMovePerIteration(50 * MockedSCM.STORAGE_UNIT);
+ config.setMaxSizeEnteringTarget(50 * MockedSCM.STORAGE_UNIT);
+ config.setIterations(1);
+ ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
+
+ MockNodeManager nodeManager = mockedSCM.getNodeManager();
+ for (DatanodeDetails target : task.getSelectedTargets()) {
+ NodeStatus status = nodeManager.getNodeStatus(target);
+ Assertions.assertSame(HddsProtos.NodeOperationalState.IN_SERVICE, status.getOperationalState());
+ assertTrue(status.isHealthy());
}
}
- @Test
- public void selectedContainerShouldNotAlreadyHaveBeenSelected()
- throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException, NodeNotFoundException,
- TimeoutException {
- balancerConfiguration.setThreshold(10);
- balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
- balancerConfiguration.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
- balancerConfiguration.setMaxSizeEnteringTarget(50 * STORAGE_UNIT);
- balancerConfiguration.setIterations(1);
- rmConf.setEnableLegacy(true);
-
- startBalancer(balancerConfiguration);
-
- stopBalancer();
-
- int numContainers = containerBalancerTask.getContainerToTargetMap().size();
-
- /*
- Assuming move is called exactly once for each unique container, number of
- calls to move should equal number of unique containers. If number of
- calls to move is more than number of unique containers, at least one
- container has been re-selected. It's expected that number of calls to
- move should equal number of unique, selected containers (from
- containerToTargetMap).
- */
- Mockito.verify(replicationManager, times(numContainers))
- .move(any(ContainerID.class), any(DatanodeDetails.class),
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource("createMockedSCMWithDatanodeLimits")
+ public void selectedContainerShouldNotAlreadyHaveBeenSelected(@Nonnull MockedSCM mockedSCM, boolean ignored)
+ throws IOException, NodeNotFoundException, TimeoutException {
+ ContainerBalancerConfiguration config = mockedSCM.getBalancerConfig();
+ config.setThreshold(10);
+ config.setMaxDatanodesPercentageToInvolvePerIteration(100);
+ config.setMaxSizeToMovePerIteration(50 * MockedSCM.STORAGE_UNIT);
+ config.setMaxSizeEnteringTarget(50 * MockedSCM.STORAGE_UNIT);
+ config.setIterations(1);
+
+ mockedSCM.enableLegacyReplicationManager();
+ ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
+
+ int numContainers = task.getContainerToTargetMap().size();
+
+ // Assuming move is called exactly once for each unique container, number of calls to move should equal
+ // number of unique containers. If number of calls to move is more than number of unique containers, at least one
+ // container has been re-selected. It's expected that number of calls to move should equal number of unique,
+ // selected containers (from containerToTargetMap).
+ Mockito
+ .verify(mockedSCM.getReplicationManager(), times(numContainers))
+ .move(
+ any(ContainerID.class),
+ any(DatanodeDetails.class),
any(DatanodeDetails.class));
- /*
- Try the same test by disabling LegacyReplicationManager so that
- MoveManager is used.
- */
- rmConf.setEnableLegacy(false);
- startBalancer(balancerConfiguration);
- stopBalancer();
- numContainers = containerBalancerTask.getContainerToTargetMap().size();
- Mockito.verify(moveManager, times(numContainers))
- .move(any(ContainerID.class), any(DatanodeDetails.class),
+ // Try the same test by disabling LegacyReplicationManager so that MoveManager is used.
+ mockedSCM.disableLegacyReplicationManager();
+ task = mockedSCM.startBalancerTask(config);
+
+ numContainers = task.getContainerToTargetMap().size();
+ Mockito
+ .verify(mockedSCM.getMoveManager(), times(numContainers))
+ .move(
+ any(ContainerID.class),
+ any(DatanodeDetails.class),
any(DatanodeDetails.class));
}
- @Test
- public void balancerShouldNotSelectConfiguredExcludeContainers()
- throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException, TimeoutException {
- balancerConfiguration.setThreshold(10);
- balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
- balancerConfiguration.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
- balancerConfiguration.setMaxSizeEnteringTarget(50 * STORAGE_UNIT);
- balancerConfiguration.setExcludeContainers("1, 4, 5");
-
- startBalancer(balancerConfiguration);
-
- stopBalancer();
- Set excludeContainers =
- balancerConfiguration.getExcludeContainers();
- for (ContainerID container :
- containerBalancerTask.getContainerToSourceMap().keySet()) {
- Assertions.assertFalse(excludeContainers.contains(container));
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource("createMockedSCMWithDatanodeLimits")
+ public void balancerShouldNotSelectConfiguredExcludeContainers(@Nonnull MockedSCM mockedSCM, boolean ignored) {
+ ContainerBalancerConfiguration config = mockedSCM.getBalancerConfig();
+ config.setThreshold(10);
+ config.setMaxDatanodesPercentageToInvolvePerIteration(100);
+ config.setMaxSizeToMovePerIteration(50 * MockedSCM.STORAGE_UNIT);
+ config.setMaxSizeEnteringTarget(50 * MockedSCM.STORAGE_UNIT);
+ config.setExcludeContainers("1, 4, 5");
+
+ ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
+
+ Set excludeContainers = config.getExcludeContainers();
+ for (ContainerID container : task.getContainerToSourceMap().keySet()) {
+ assertFalse(excludeContainers.contains(container));
}
}
- @Test
- public void balancerShouldObeyMaxSizeEnteringTargetLimit()
- throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException, TimeoutException {
- conf.set("ozone.scm.container.size", "1MB");
- balancerConfiguration =
- conf.getObject(ContainerBalancerConfiguration.class);
- balancerConfiguration.setThreshold(10);
- balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
- balancerConfiguration.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
-
- // no containers should be selected when the limit is just 2 MB
- balancerConfiguration.setMaxSizeEnteringTarget(2 * OzoneConsts.MB);
- startBalancer(balancerConfiguration);
-
- Assertions.assertFalse(containerBalancerTask.getUnBalancedNodes()
- .isEmpty());
- Assertions.assertTrue(containerBalancerTask.getContainerToSourceMap()
- .isEmpty());
- stopBalancer();
-
- // some containers should be selected when using default values
- OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
- ContainerBalancerConfiguration cbc = ozoneConfiguration.
- getObject(ContainerBalancerConfiguration.class);
- cbc.setBalancingInterval(1);
- ContainerBalancer sb = new ContainerBalancer(scm);
- containerBalancerTask = new ContainerBalancerTask(scm, 0, sb,
- sb.getMetrics(), cbc, false);
- containerBalancerTask.run();
-
- stopBalancer();
- // balancer should have identified unbalanced nodes
- Assertions.assertFalse(containerBalancerTask.getUnBalancedNodes()
- .isEmpty());
- Assertions.assertFalse(containerBalancerTask.getContainerToSourceMap()
- .isEmpty());
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource("createMockedSCMWithDatanodeLimits")
+ public void balancerShouldObeyMaxSizeEnteringTargetLimit(@Nonnull MockedSCM mockedSCM, boolean useDatanodeLimits) {
+ OzoneConfiguration ozoneConfig = mockedSCM.getOzoneConfig();
+ ozoneConfig.set("ozone.scm.container.size", "1MB");
+ ContainerBalancerConfiguration config = mockedSCM.getBalancerConfigByOzoneConfig(ozoneConfig);
+ config.setThreshold(10);
+ config.setMaxDatanodesPercentageToInvolvePerIteration(100);
+ config.setMaxSizeToMovePerIteration(50 * MockedSCM.STORAGE_UNIT);
+ config.setAdaptBalanceWhenCloseToLimit(useDatanodeLimits);
+ config.setAdaptBalanceWhenReachTheLimit(useDatanodeLimits);
+
+ // No containers should be selected when the limit is just 2 MB
+ config.setMaxSizeEnteringTarget(2 * OzoneConsts.MB);
+ ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
+
+ assertTrue(stillHaveUnbalancedNodes(task));
+ assertTrue(task.getContainerToSourceMap().isEmpty());
+
+ // Some containers should be selected when using default values
+ ContainerBalancerConfiguration balancerConfig = mockedSCM.getBalancerConfigByOzoneConfig(new OzoneConfiguration());
+ balancerConfig.setBalancingInterval(1);
+ balancerConfig.setAdaptBalanceWhenCloseToLimit(useDatanodeLimits);
+ balancerConfig.setAdaptBalanceWhenReachTheLimit(useDatanodeLimits);
+
+ task = mockedSCM.startBalancerTask(balancerConfig);
+
+ // Balancer should have identified unbalanced nodes
+ assertTrue(stillHaveUnbalancedNodes(task));
+ if (canNotBalanceAnyMore(task)) {
+ assertTrue(task.getContainerToSourceMap().isEmpty());
+ } else {
+ assertFalse(task.getContainerToSourceMap().isEmpty());
+ }
}
- @Test
- public void balancerShouldObeyMaxSizeLeavingSourceLimit()
- throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException, TimeoutException {
- conf.set("ozone.scm.container.size", "1MB");
- balancerConfiguration =
- conf.getObject(ContainerBalancerConfiguration.class);
- balancerConfiguration.setThreshold(10);
- balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
- balancerConfiguration.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
-
- // no source containers should be selected when the limit is just 2 MB
- balancerConfiguration.setMaxSizeLeavingSource(2 * OzoneConsts.MB);
- startBalancer(balancerConfiguration);
-
- Assertions.assertFalse(containerBalancerTask.getUnBalancedNodes()
- .isEmpty());
- Assertions.assertTrue(containerBalancerTask.getContainerToSourceMap()
- .isEmpty());
- stopBalancer();
-
- // some containers should be selected when using default values
- OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
- ContainerBalancerConfiguration cbc = ozoneConfiguration.
- getObject(ContainerBalancerConfiguration.class);
- cbc.setBalancingInterval(1);
- ContainerBalancer sb = new ContainerBalancer(scm);
- containerBalancerTask = new ContainerBalancerTask(scm, 0, sb,
- sb.getMetrics(), cbc, false);
- containerBalancerTask.run();
-
- stopBalancer();
- // balancer should have identified unbalanced nodes
- Assertions.assertFalse(containerBalancerTask.getUnBalancedNodes()
- .isEmpty());
- Assertions.assertFalse(containerBalancerTask.getContainerToSourceMap()
- .isEmpty());
- Assertions.assertTrue(0 !=
- containerBalancerTask.getSizeScheduledForMoveInLatestIteration());
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource("createMockedSCMWithDatanodeLimits")
+ public void balancerShouldObeyMaxSizeLeavingSourceLimit(@Nonnull MockedSCM mockedSCM, boolean useDatanodeLimits) {
+ OzoneConfiguration ozoneConfig = mockedSCM.getOzoneConfig();
+ ozoneConfig.set("ozone.scm.container.size", "1MB");
+ ContainerBalancerConfiguration config = mockedSCM.getBalancerConfigByOzoneConfig(ozoneConfig);
+ config.setThreshold(10);
+ config.setMaxDatanodesPercentageToInvolvePerIteration(100);
+ config.setMaxSizeToMovePerIteration(50 * MockedSCM.STORAGE_UNIT);
+ config.setAdaptBalanceWhenCloseToLimit(useDatanodeLimits);
+ config.setAdaptBalanceWhenReachTheLimit(useDatanodeLimits);
+
+ // No source containers should be selected when the limit is just 2 MB
+ config.setMaxSizeLeavingSource(2 * OzoneConsts.MB);
+ ContainerBalancer balancer = mockedSCM.createContainerBalancer();
+ ContainerBalancerTask task = mockedSCM.startBalancerTask(balancer, config);
+
+ assertTrue(stillHaveUnbalancedNodes(task));
+ assertTrue(task.getContainerToSourceMap().isEmpty());
+
+ // Some containers should be selected when using default values
+ ContainerBalancerConfiguration newBalancerConfig =
+ mockedSCM.getBalancerConfigByOzoneConfig(new OzoneConfiguration());
+ newBalancerConfig.setBalancingInterval(1);
+ newBalancerConfig.setAdaptBalanceWhenCloseToLimit(useDatanodeLimits);
+ newBalancerConfig.setAdaptBalanceWhenReachTheLimit(useDatanodeLimits);
+
+ task = mockedSCM.startBalancerTask(balancer, newBalancerConfig);
+
+ // Balancer should have identified unbalanced nodes
+ assertTrue(stillHaveUnbalancedNodes(task));
+
+ if (canNotBalanceAnyMore(task)) {
+ assertTrue(task.getContainerToSourceMap().isEmpty());
+ assertEquals(0, balancer.getMetrics().getDataSizeMovedBytes());
+ } else {
+ assertFalse(task.getContainerToSourceMap().isEmpty());
+ assertTrue(0 != balancer.getMetrics().getDataSizeMovedBytes());
+ }
}
- @Test
- public void testMetrics()
- throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException, TimeoutException,
- NodeNotFoundException {
- conf.set("hdds.datanode.du.refresh.period", "1ms");
- balancerConfiguration.setBalancingInterval(Duration.ofMillis(2));
- balancerConfiguration.setThreshold(10);
- balancerConfiguration.setIterations(1);
- balancerConfiguration.setMaxSizeEnteringTarget(6 * STORAGE_UNIT);
- // deliberately set max size per iteration to a low value, 6 GB
- balancerConfiguration.setMaxSizeToMovePerIteration(6 * STORAGE_UNIT);
- balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
- Mockito.when(moveManager.move(any(), any(), any()))
- .thenReturn(CompletableFuture.completedFuture(
- MoveManager.MoveResult.REPLICATION_FAIL_NODE_UNHEALTHY))
- .thenReturn(CompletableFuture.completedFuture(
- MoveManager.MoveResult.COMPLETED));
-
- startBalancer(balancerConfiguration);
- stopBalancer();
-
- ContainerBalancerMetrics metrics = containerBalancerTask.getMetrics();
- Assertions.assertEquals(determineExpectedUnBalancedNodes(
- balancerConfiguration.getThreshold()).size(),
- metrics.getNumDatanodesUnbalanced());
- Assertions.assertTrue(metrics.getDataSizeMovedGBInLatestIteration() <= 6);
- Assertions.assertTrue(metrics.getDataSizeMovedGB() > 0);
- Assertions.assertEquals(1, metrics.getNumIterations());
- Assertions.assertTrue(
- metrics.getNumContainerMovesScheduledInLatestIteration() > 0);
- Assertions.assertEquals(metrics.getNumContainerMovesScheduled(),
- metrics.getNumContainerMovesScheduledInLatestIteration());
- Assertions.assertEquals(metrics.getNumContainerMovesScheduled(),
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource("createMockedSCMWithDatanodeLimits")
+ public void testMetrics(@Nonnull MockedSCM mockedSCM, boolean ignored) throws IOException, NodeNotFoundException {
+ OzoneConfiguration ozoneConfig = mockedSCM.getOzoneConfig();
+ ozoneConfig.set("ozone.scm.container.size", "1MB");
+ ContainerBalancerConfiguration config = mockedSCM.getBalancerConfigByOzoneConfig(ozoneConfig);
+ config.setBalancingInterval(Duration.ofMillis(2));
+ config.setThreshold(10);
+ config.setIterations(1);
+ config.setMaxSizeEnteringTarget(6 * MockedSCM.STORAGE_UNIT);
+ // Deliberately set max size per iteration to a low value, 6 GB
+ config.setMaxSizeToMovePerIteration(6 * MockedSCM.STORAGE_UNIT);
+ config.setMaxDatanodesPercentageToInvolvePerIteration(100);
+
+ Mockito
+ .when(mockedSCM.getMoveManager().move(any(), any(), any()))
+ .thenReturn(CompletableFuture.completedFuture(MoveManager.MoveResult.REPLICATION_FAIL_NODE_UNHEALTHY))
+ .thenReturn(CompletableFuture.completedFuture(MoveManager.MoveResult.COMPLETED));
+
+ ContainerBalancer balancer = mockedSCM.createContainerBalancer();
+ ContainerBalancerTask task = mockedSCM.startBalancerTask(balancer, config);
+
+ IterationMetrics itMetrics = task.getIterationMetrics();
+ assertEquals(
+ mockedSCM.getCluster().getUnBalancedNodes(config.getThreshold()).size(),
+ itMetrics.getUnbalancedDatanodeCount());
+ assertTrue(itMetrics.getMovedBytesCount() <= 6 * MockedSCM.STORAGE_UNIT);
+ assertTrue(itMetrics.getScheduledContainerMovesCount() > 0);
+
+ ContainerBalancerMetrics metrics = balancer.getMetrics();
+ assertEquals(metrics.getNumContainerMovesScheduled(), itMetrics.getScheduledContainerMovesCount());
+ assertTrue(metrics.getDataSizeMovedBytes() > 0);
+ assertEquals(1, metrics.getNumIterations());
+ assertEquals(metrics.getNumContainerMovesScheduled(),
metrics.getNumContainerMovesCompleted() +
metrics.getNumContainerMovesFailed() +
metrics.getNumContainerMovesTimeout());
- Assertions.assertEquals(0, metrics.getNumContainerMovesTimeout());
- Assertions.assertEquals(1, metrics.getNumContainerMovesFailed());
+ assertEquals(0, metrics.getNumContainerMovesTimeout());
+ assertEquals(1, metrics.getNumContainerMovesFailed());
}
/**
- * Tests if {@link ContainerBalancer} follows the includeNodes and
- * excludeNodes configurations in {@link ContainerBalancerConfiguration}.
- * If the includeNodes configuration is not empty, only the specified
- * includeNodes should be included in balancing. excludeNodes should be
- * excluded from balancing. If a datanode is specified in both include and
- * exclude configurations, then it should be excluded.
+ * Tests if {@link ContainerBalancer} follows the includeNodes and excludeNodes configurations in
+ * {@link ContainerBalancerConfiguration}. If the includeNodes configuration is not empty, only the specified
+ * includeNodes should be included in balancing. excludeNodes should be excluded from balancing.
+ * If a datanode is specified in both include and exclude configurations, then it should be excluded.
*/
- @Test
- public void balancerShouldFollowExcludeAndIncludeDatanodesConfigurations()
- throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException, TimeoutException {
- balancerConfiguration.setThreshold(10);
- balancerConfiguration.setIterations(1);
- balancerConfiguration.setMaxSizeEnteringTarget(10 * STORAGE_UNIT);
- balancerConfiguration.setMaxSizeToMovePerIteration(100 * STORAGE_UNIT);
- balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
-
- // only these nodes should be included
- // the ones also specified in excludeNodes should be excluded
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource("createMockedSCMWithDatanodeLimits")
+ public void balancerShouldFollowExcludeAndIncludeDatanodesConfigurations(
+ @Nonnull MockedSCM mockedSCM,
+ boolean ignored
+ ) {
+ ContainerBalancerConfiguration config = mockedSCM.getBalancerConfig();
+ config.setThreshold(10);
+ config.setIterations(1);
+ config.setMaxSizeEnteringTarget(10 * MockedSCM.STORAGE_UNIT);
+ config.setMaxSizeToMovePerIteration(100 * MockedSCM.STORAGE_UNIT);
+ config.setMaxDatanodesPercentageToInvolvePerIteration(100);
+
+ DatanodeUsageInfo[] nodesInCluster = mockedSCM.getCluster().getNodesInCluster();
+
+ // Only these nodes should be included the ones also specified in excludeNodes should be excluded
int firstIncludeIndex = 0, secondIncludeIndex = 1;
- int thirdIncludeIndex = nodesInCluster.size() - 2;
- int fourthIncludeIndex = nodesInCluster.size() - 1;
- String includeNodes =
- nodesInCluster.get(firstIncludeIndex).getDatanodeDetails()
- .getIpAddress() + ", " +
- nodesInCluster.get(secondIncludeIndex).getDatanodeDetails()
- .getIpAddress() + ", " +
- nodesInCluster.get(thirdIncludeIndex).getDatanodeDetails()
- .getHostName() + ", " +
- nodesInCluster.get(fourthIncludeIndex).getDatanodeDetails()
- .getHostName();
-
- // these nodes should be excluded
- int firstExcludeIndex = 0, secondExcludeIndex = nodesInCluster.size() - 1;
- String excludeNodes =
- nodesInCluster.get(firstExcludeIndex).getDatanodeDetails()
- .getIpAddress() + ", " +
- nodesInCluster.get(secondExcludeIndex).getDatanodeDetails()
- .getHostName();
-
- balancerConfiguration.setExcludeNodes(excludeNodes);
- balancerConfiguration.setIncludeNodes(includeNodes);
- startBalancer(balancerConfiguration);
- stopBalancer();
-
- // finally, these should be the only nodes included in balancing
- // (included - excluded)
- DatanodeDetails dn1 =
- nodesInCluster.get(secondIncludeIndex).getDatanodeDetails();
- DatanodeDetails dn2 =
- nodesInCluster.get(thirdIncludeIndex).getDatanodeDetails();
- Map containerFromSourceMap =
- containerBalancerTask.getContainerToSourceMap();
- Map containerToTargetMap =
- containerBalancerTask.getContainerToTargetMap();
- for (Map.Entry entry :
- containerFromSourceMap.entrySet()) {
+ int thirdIncludeIndex = nodesInCluster.length - 2;
+ int fourthIncludeIndex = nodesInCluster.length - 1;
+ String includeNodes = String.join(", ",
+ nodesInCluster[firstIncludeIndex].getDatanodeDetails().getIpAddress(),
+ nodesInCluster[secondIncludeIndex].getDatanodeDetails().getIpAddress(),
+ nodesInCluster[thirdIncludeIndex].getDatanodeDetails().getHostName(),
+ nodesInCluster[fourthIncludeIndex].getDatanodeDetails().getHostName());
+
+ // These nodes should be excluded
+ int firstExcludeIndex = 0, secondExcludeIndex = nodesInCluster.length - 1;
+ String excludeNodes = String.join(", ",
+ nodesInCluster[firstExcludeIndex].getDatanodeDetails().getIpAddress(),
+ nodesInCluster[secondExcludeIndex].getDatanodeDetails().getHostName());
+
+ config.setExcludeNodes(excludeNodes);
+ config.setIncludeNodes(includeNodes);
+ ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
+
+ // Finally, these should be the only nodes included in balancing (included - excluded)
+ DatanodeDetails dn1 = nodesInCluster[secondIncludeIndex].getDatanodeDetails();
+ DatanodeDetails dn2 = nodesInCluster[thirdIncludeIndex].getDatanodeDetails();
+ Map containerFromSourceMap = task.getContainerToSourceMap();
+ Map containerToTargetMap = task.getContainerToTargetMap();
+ for (Map.Entry entry : containerFromSourceMap.entrySet()) {
DatanodeDetails source = entry.getValue();
DatanodeDetails target = containerToTargetMap.get(entry.getKey());
- Assertions.assertTrue(source.equals(dn1) || source.equals(dn2));
- Assertions.assertTrue(target.equals(dn1) || target.equals(dn2));
+ assertTrue(source.equals(dn1) || source.equals(dn2));
+ assertTrue(target.equals(dn1) || target.equals(dn2));
}
}
- @Test
- public void testContainerBalancerConfiguration() {
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource("createMockedSCMWithDatanodeLimits")
+ public void testContainerBalancerConfiguration(@Nonnull MockedSCM mockedSCM, boolean ignored) {
OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
ozoneConfiguration.set("ozone.scm.container.size", "5GB");
- ozoneConfiguration.setDouble(
- "hdds.container.balancer.utilization.threshold", 1);
+ ozoneConfiguration.setDouble("hdds.container.balancer.utilization.threshold", 1);
long maxSizeLeavingSource = 26;
- ozoneConfiguration.setStorageSize(
- "hdds.container.balancer.size.leaving.source.max", maxSizeLeavingSource,
+ ozoneConfiguration.setStorageSize("hdds.container.balancer.size.leaving.source.max", maxSizeLeavingSource,
StorageUnit.GB);
long moveTimeout = 90;
- ozoneConfiguration.setTimeDuration("hdds.container.balancer.move.timeout",
- moveTimeout, TimeUnit.MINUTES);
+ ozoneConfiguration.setTimeDuration("hdds.container.balancer.move.timeout", moveTimeout, TimeUnit.MINUTES);
long replicationTimeout = 60;
- ozoneConfiguration.setTimeDuration(
- "hdds.container.balancer.move.replication.timeout",
- replicationTimeout, TimeUnit.MINUTES);
+ ozoneConfiguration.setTimeDuration("hdds.container.balancer.move.replication.timeout", replicationTimeout,
+ TimeUnit.MINUTES);
- ContainerBalancerConfiguration cbConf =
- ozoneConfiguration.getObject(ContainerBalancerConfiguration.class);
- Assertions.assertEquals(1, cbConf.getThreshold(), 0.001);
+ ContainerBalancerConfiguration cbConf = mockedSCM.getBalancerConfigByOzoneConfig(ozoneConfiguration);
+ assertEquals(1, cbConf.getThreshold(), 0.001);
// Expected is 26 GB
- Assertions.assertEquals(maxSizeLeavingSource * 1024 * 1024 * 1024,
- cbConf.getMaxSizeLeavingSource());
- Assertions.assertEquals(moveTimeout, cbConf.getMoveTimeout().toMinutes());
- Assertions.assertEquals(replicationTimeout,
- cbConf.getMoveReplicationTimeout().toMinutes());
- }
-
- @Test
- public void checkIterationResult()
- throws NodeNotFoundException, IOException,
- IllegalContainerBalancerStateException,
- InvalidContainerBalancerConfigurationException,
- TimeoutException {
- balancerConfiguration.setThreshold(10);
- balancerConfiguration.setIterations(1);
- balancerConfiguration.setMaxSizeEnteringTarget(10 * STORAGE_UNIT);
- balancerConfiguration.setMaxSizeToMovePerIteration(100 * STORAGE_UNIT);
- balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
- rmConf.setEnableLegacy(true);
-
- startBalancer(balancerConfiguration);
-
- /*
- According to the setup and configurations, this iteration's result should
- be ITERATION_COMPLETED.
- */
- Assertions.assertEquals(
- ContainerBalancerTask.IterationResult.ITERATION_COMPLETED,
- containerBalancerTask.getIterationResult());
- stopBalancer();
-
- /*
- Now, limit maxSizeToMovePerIteration but fail all container moves. The
- result should still be ITERATION_COMPLETED.
- */
- Mockito.when(replicationManager.move(Mockito.any(ContainerID.class),
- Mockito.any(DatanodeDetails.class),
- Mockito.any(DatanodeDetails.class)))
- .thenReturn(CompletableFuture.completedFuture(
- MoveManager.MoveResult.REPLICATION_FAIL_NODE_UNHEALTHY));
- balancerConfiguration.setMaxSizeToMovePerIteration(10 * STORAGE_UNIT);
-
- startBalancer(balancerConfiguration);
-
- Assertions.assertEquals(
- ContainerBalancerTask.IterationResult.ITERATION_COMPLETED,
- containerBalancerTask.getIterationResult());
- stopBalancer();
-
- /*
- Try the same but use MoveManager for container move instead of legacy RM.
- */
- rmConf.setEnableLegacy(false);
- startBalancer(balancerConfiguration);
- Assertions.assertEquals(
- ContainerBalancerTask.IterationResult.ITERATION_COMPLETED,
- containerBalancerTask.getIterationResult());
- stopBalancer();
- }
-
- /**
- * Tests the situation where some container moves time out because they
- * take longer than "move.timeout".
- */
- @Test
- public void checkIterationResultTimeout()
- throws NodeNotFoundException, IOException,
- IllegalContainerBalancerStateException,
- InvalidContainerBalancerConfigurationException,
- TimeoutException {
-
- CompletableFuture completedFuture =
- CompletableFuture.completedFuture(MoveManager.MoveResult.COMPLETED);
- Mockito.when(replicationManager.move(Mockito.any(ContainerID.class),
- Mockito.any(DatanodeDetails.class),
- Mockito.any(DatanodeDetails.class)))
- .thenReturn(completedFuture)
- .thenAnswer(invocation -> genCompletableFuture(2000));
-
- balancerConfiguration.setThreshold(10);
- balancerConfiguration.setIterations(1);
- balancerConfiguration.setMaxSizeEnteringTarget(10 * STORAGE_UNIT);
- balancerConfiguration.setMaxSizeToMovePerIteration(100 * STORAGE_UNIT);
- balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
- balancerConfiguration.setMoveTimeout(Duration.ofMillis(500));
- rmConf.setEnableLegacy(true);
- startBalancer(balancerConfiguration);
-
- /*
- According to the setup and configurations, this iteration's result should
- be ITERATION_COMPLETED.
- */
- Assertions.assertEquals(
- ContainerBalancerTask.IterationResult.ITERATION_COMPLETED,
- containerBalancerTask.getIterationResult());
- Assertions.assertEquals(1,
- containerBalancerTask.getMetrics()
- .getNumContainerMovesCompletedInLatestIteration());
- Assertions.assertTrue(containerBalancerTask.getMetrics()
- .getNumContainerMovesTimeoutInLatestIteration() > 1);
- stopBalancer();
-
- /*
- Test the same but use MoveManager instead of LegacyReplicationManager.
- The first move being 10ms falls within the timeout duration of 500ms. It
- should be successful. The rest should fail.
- */
- rmConf.setEnableLegacy(false);
- Mockito.when(moveManager.move(Mockito.any(ContainerID.class),
- Mockito.any(DatanodeDetails.class),
- Mockito.any(DatanodeDetails.class)))
- .thenReturn(completedFuture)
- .thenAnswer(invocation -> genCompletableFuture(2000));
-
- startBalancer(balancerConfiguration);
- Assertions.assertEquals(
- ContainerBalancerTask.IterationResult.ITERATION_COMPLETED,
- containerBalancerTask.getIterationResult());
- Assertions.assertEquals(1,
- containerBalancerTask.getMetrics()
- .getNumContainerMovesCompletedInLatestIteration());
- Assertions.assertTrue(containerBalancerTask.getMetrics()
- .getNumContainerMovesTimeoutInLatestIteration() > 1);
- stopBalancer();
- }
-
- @Test
- public void checkIterationResultTimeoutFromReplicationManager()
- throws NodeNotFoundException, IOException,
- IllegalContainerBalancerStateException,
- InvalidContainerBalancerConfigurationException, TimeoutException {
- CompletableFuture future
- = CompletableFuture.supplyAsync(() ->
- MoveManager.MoveResult.REPLICATION_FAIL_TIME_OUT);
- CompletableFuture future2
- = CompletableFuture.supplyAsync(() ->
- MoveManager.MoveResult.DELETION_FAIL_TIME_OUT);
- Mockito.when(replicationManager.move(Mockito.any(ContainerID.class),
- Mockito.any(DatanodeDetails.class),
- Mockito.any(DatanodeDetails.class)))
- .thenReturn(future, future2);
-
- balancerConfiguration.setThreshold(10);
- balancerConfiguration.setIterations(1);
- balancerConfiguration.setMaxSizeEnteringTarget(10 * STORAGE_UNIT);
- balancerConfiguration.setMaxSizeToMovePerIteration(100 * STORAGE_UNIT);
- balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
- balancerConfiguration.setMoveTimeout(Duration.ofMillis(500));
- rmConf.setEnableLegacy(true);
- startBalancer(balancerConfiguration);
-
- Assertions.assertTrue(containerBalancerTask.getMetrics()
- .getNumContainerMovesTimeoutInLatestIteration() > 0);
- Assertions.assertEquals(0, containerBalancerTask.getMetrics()
- .getNumContainerMovesCompletedInLatestIteration());
- stopBalancer();
-
- /*
- Try the same test with MoveManager instead of LegacyReplicationManager.
- */
- Mockito.when(moveManager.move(Mockito.any(ContainerID.class),
- Mockito.any(DatanodeDetails.class),
- Mockito.any(DatanodeDetails.class)))
- .thenReturn(future).thenAnswer(invocation -> future2);
-
- rmConf.setEnableLegacy(false);
- startBalancer(balancerConfiguration);
- Assertions.assertTrue(containerBalancerTask.getMetrics()
- .getNumContainerMovesTimeoutInLatestIteration() > 0);
- Assertions.assertEquals(0, containerBalancerTask.getMetrics()
- .getNumContainerMovesCompletedInLatestIteration());
- stopBalancer();
- }
-
- @Test
- public void checkIterationResultException()
- throws NodeNotFoundException, IOException,
- IllegalContainerBalancerStateException,
- InvalidContainerBalancerConfigurationException,
- TimeoutException {
-
- CompletableFuture future =
- new CompletableFuture<>();
- future.completeExceptionally(new RuntimeException("Runtime Exception"));
- Mockito.when(replicationManager.move(Mockito.any(ContainerID.class),
- Mockito.any(DatanodeDetails.class),
- Mockito.any(DatanodeDetails.class)))
- .thenReturn(CompletableFuture.supplyAsync(() -> {
- try {
- Thread.sleep(1);
- } catch (Exception ignored) {
- }
- throw new RuntimeException("Runtime Exception after doing work");
- }))
- .thenThrow(new ContainerNotFoundException("Test Container not found"))
- .thenReturn(future);
-
- balancerConfiguration.setThreshold(10);
- balancerConfiguration.setIterations(1);
- balancerConfiguration.setMaxSizeEnteringTarget(10 * STORAGE_UNIT);
- balancerConfiguration.setMaxSizeToMovePerIteration(100 * STORAGE_UNIT);
- balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
- balancerConfiguration.setMoveTimeout(Duration.ofMillis(500));
- rmConf.setEnableLegacy(true);
-
- startBalancer(balancerConfiguration);
-
- Assertions.assertEquals(
- ContainerBalancerTask.IterationResult.ITERATION_COMPLETED,
- containerBalancerTask.getIterationResult());
- Assertions.assertTrue(
- containerBalancerTask.getMetrics()
- .getNumContainerMovesFailed() >= 3);
- stopBalancer();
-
- /*
- Try the same test but with MoveManager instead of ReplicationManager.
- */
- Mockito.when(moveManager.move(Mockito.any(ContainerID.class),
- Mockito.any(DatanodeDetails.class),
- Mockito.any(DatanodeDetails.class)))
- .thenReturn(CompletableFuture.supplyAsync(() -> {
- try {
- Thread.sleep(1);
- } catch (Exception ignored) {
- }
- throw new RuntimeException("Runtime Exception after doing work");
- }))
- .thenThrow(new ContainerNotFoundException("Test Container not found"))
- .thenReturn(future);
-
- rmConf.setEnableLegacy(false);
- startBalancer(balancerConfiguration);
- Assertions.assertEquals(
- ContainerBalancerTask.IterationResult.ITERATION_COMPLETED,
- containerBalancerTask.getIterationResult());
- Assertions.assertTrue(
- containerBalancerTask.getMetrics()
- .getNumContainerMovesFailed() >= 3);
- stopBalancer();
+ assertEquals(maxSizeLeavingSource * 1024 * 1024 * 1024, cbConf.getMaxSizeLeavingSource());
+ assertEquals(moveTimeout, cbConf.getMoveTimeout().toMinutes());
+ assertEquals(replicationTimeout, cbConf.getMoveReplicationTimeout().toMinutes());
}
@Unhealthy("HDDS-8941")
- @Test
- public void testDelayedStart() throws InterruptedException, TimeoutException {
- conf.setTimeDuration("hdds.scm.wait.time.after.safemode.exit", 10,
- TimeUnit.SECONDS);
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource("createMockedSCMWithDatanodeLimits")
+ public void testDelayedStart(@Nonnull MockedSCM mockedSCM, boolean ignored)
+ throws InterruptedException, TimeoutException {
+ OzoneConfiguration ozoneConfig = mockedSCM.getOzoneConfig();
+ ozoneConfig.setTimeDuration("hdds.scm.wait.time.after.safemode.exit", 10, TimeUnit.SECONDS);
+ ContainerBalancerConfiguration config = mockedSCM.getBalancerConfigByOzoneConfig(ozoneConfig);
+
+ StorageContainerManager scm = mockedSCM.getStorageContainerManager();
ContainerBalancer balancer = new ContainerBalancer(scm);
- containerBalancerTask = new ContainerBalancerTask(scm, 2, balancer,
- balancer.getMetrics(), balancerConfiguration, true);
- Thread balancingThread = new Thread(containerBalancerTask);
- // start the thread and assert that balancer is RUNNING
+ ContainerBalancerTask task = new ContainerBalancerTask(scm, balancer, config);
+ Thread balancingThread = new Thread(() -> task.run(2, true));
+
+ // Start the thread and assert that balancer is RUNNING
balancingThread.start();
- Assertions.assertEquals(ContainerBalancerTask.Status.RUNNING,
- containerBalancerTask.getBalancerStatus());
-
- /*
- Wait for the thread to start sleeping and assert that it's sleeping.
- This is the delay before it starts balancing.
- */
- GenericTestUtils.waitFor(
- () -> balancingThread.getState() == Thread.State.TIMED_WAITING, 1, 20);
- Assertions.assertEquals(Thread.State.TIMED_WAITING,
- balancingThread.getState());
-
- // interrupt the thread from its sleep, wait and assert that balancer has
- // STOPPED
+ assertEquals(ContainerBalancerTask.Status.RUNNING, task.getBalancerStatus());
+
+ // Wait for the thread to start sleeping and assert that it's sleeping.
+ // This is the delay before it starts balancing.
+ GenericTestUtils.waitFor(() -> balancingThread.getState() == Thread.State.TIMED_WAITING, 1, 20);
+ assertEquals(Thread.State.TIMED_WAITING, balancingThread.getState());
+
+ // Interrupt the thread from its sleep, wait and assert that balancer has STOPPED
balancingThread.interrupt();
- GenericTestUtils.waitFor(() -> containerBalancerTask.getBalancerStatus() ==
- ContainerBalancerTask.Status.STOPPED, 1, 20);
- Assertions.assertEquals(ContainerBalancerTask.Status.STOPPED,
- containerBalancerTask.getBalancerStatus());
+ GenericTestUtils.waitFor(() -> task.getBalancerStatus() == ContainerBalancerTask.Status.STOPPED, 1, 20);
+ assertEquals(ContainerBalancerTask.Status.STOPPED, task.getBalancerStatus());
- // ensure the thread dies
+ // Ensure the thread dies
GenericTestUtils.waitFor(() -> !balancingThread.isAlive(), 1, 20);
- Assertions.assertFalse(balancingThread.isAlive());
+ assertFalse(balancingThread.isAlive());
}
/**
- * The expectation is that only RATIS containers should be selected for
- * balancing when LegacyReplicationManager is enabled. This is because
- * LegacyReplicationManager does not support moving EC containers.
+ * The expectation is that only RATIS containers should be selected for balancing when LegacyReplicationManager
+ * is enabled. This is because LegacyReplicationManager does not support moving EC containers.
*/
- @Test
- public void balancerShouldExcludeECContainersWhenLegacyRmIsEnabled()
- throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException, TimeoutException {
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource("createMockedSCMWithDatanodeLimits")
+ public void balancerShouldExcludeECContainersWhenLegacyRmIsEnabled(@Nonnull MockedSCM mockedSCM, boolean ignored) {
// Enable LegacyReplicationManager
- rmConf.setEnableLegacy(true);
- balancerConfiguration.setThreshold(10);
- balancerConfiguration.setIterations(1);
- balancerConfiguration.setMaxSizeEnteringTarget(10 * STORAGE_UNIT);
- balancerConfiguration.setMaxSizeToMovePerIteration(100 * STORAGE_UNIT);
- balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
-
- startBalancer(balancerConfiguration);
-
- /*
- Get all containers that were selected by balancer and assert none of
- them is an EC container.
- */
- Map containerToSource =
- containerBalancerTask.getContainerToSourceMap();
- Assertions.assertFalse(containerToSource.isEmpty());
- for (Map.Entry entry :
- containerToSource.entrySet()) {
- ContainerInfo containerInfo = cidToInfoMap.get(entry.getKey());
- Assertions.assertNotSame(HddsProtos.ReplicationType.EC,
- containerInfo.getReplicationType());
- }
- }
-
- /**
- * Determines unBalanced nodes, that is, over and under utilized nodes,
- * according to the generated utilization values for nodes and the threshold.
- *
- * @param threshold A percentage in the range 0 to 100
- * @return List of DatanodeUsageInfo containing the expected(correct)
- * unBalanced nodes.
- */
- private List determineExpectedUnBalancedNodes(
- double threshold) {
- threshold /= 100;
- double lowerLimit = averageUtilization - threshold;
- double upperLimit = averageUtilization + threshold;
-
- // use node utilizations to determine over and under utilized nodes
- List expectedUnBalancedNodes = new ArrayList<>();
- for (int i = 0; i < numberOfNodes; i++) {
- if (nodeUtilizations.get(numberOfNodes - i - 1) > upperLimit) {
- expectedUnBalancedNodes.add(nodesInCluster.get(numberOfNodes - i - 1));
- }
- }
- for (int i = 0; i < numberOfNodes; i++) {
- if (nodeUtilizations.get(i) < lowerLimit) {
- expectedUnBalancedNodes.add(nodesInCluster.get(i));
- }
+ mockedSCM.enableLegacyReplicationManager();
+ ContainerBalancerConfiguration config = mockedSCM.getBalancerConfig();
+ config.setThreshold(10);
+ config.setIterations(1);
+ config.setMaxSizeEnteringTarget(10 * MockedSCM.STORAGE_UNIT);
+ config.setMaxSizeToMovePerIteration(100 * MockedSCM.STORAGE_UNIT);
+ config.setMaxDatanodesPercentageToInvolvePerIteration(100);
+
+ ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
+
+ // Get all containers that were selected by balancer and assert none of them is an EC container.
+ Map containerToSource = task.getContainerToSourceMap();
+ assertFalse(containerToSource.isEmpty());
+ Map cidToInfoMap = mockedSCM.getCluster().getCidToInfoMap();
+ for (ContainerID containerID : containerToSource.keySet()) {
+ ContainerInfo containerInfo = cidToInfoMap.get(containerID);
+ Assertions.assertNotSame(HddsProtos.ReplicationType.EC, containerInfo.getReplicationType());
}
- return expectedUnBalancedNodes;
}
- /**
- * Generates a range of equally spaced utilization(that is, used / capacity)
- * values from 0 to 1.
- *
- * @param count Number of values to generate. Count must be greater than or
- * equal to 1.
- * @throws IllegalArgumentException If the value of the parameter count is
- * less than 1.
- */
- private void generateUtilizations(int count) throws IllegalArgumentException {
- if (count < 1) {
- LOG.warn("The value of argument count is {}. However, count must be " +
- "greater than 0.", count);
- throw new IllegalArgumentException();
- }
- nodeUtilizations = new ArrayList<>(count);
- for (int i = 0; i < count; i++) {
- nodeUtilizations.add(i / (double) count);
- }
+ private static boolean canNotBalanceAnyMore(@Nonnull ContainerBalancerTask task) {
+ return IterationResult.CAN_NOT_BALANCE_ANY_MORE == task.getIterationResult();
}
-
- /**
- * Create an unbalanced cluster by generating some data. Nodes in the
- * cluster have utilization values determined by generateUtilizations method.
- * @return average utilization (used space / capacity) of the cluster
- */
- private double createCluster() {
- generateData();
- createReplicasForContainers();
- long clusterCapacity = 0, clusterUsedSpace = 0;
-
- // for each node utilization, calculate that datanode's used space and
- // capacity
- for (int i = 0; i < nodeUtilizations.size(); i++) {
- long datanodeUsedSpace = 0, datanodeCapacity = 0;
- Set containerIDSet =
- datanodeToContainersMap.get(nodesInCluster.get(i));
-
- for (ContainerID containerID : containerIDSet) {
- datanodeUsedSpace += cidToInfoMap.get(containerID).getUsedBytes();
- }
-
- // use node utilization and used space to determine node capacity
- if (nodeUtilizations.get(i) == 0) {
- datanodeCapacity = STORAGE_UNIT * RANDOM.nextInt(10, 60);
- } else {
- datanodeCapacity = (long) (datanodeUsedSpace / nodeUtilizations.get(i));
- }
- SCMNodeStat stat = new SCMNodeStat(datanodeCapacity, datanodeUsedSpace,
- datanodeCapacity - datanodeUsedSpace);
- nodesInCluster.get(i).setScmNodeStat(stat);
- clusterUsedSpace += datanodeUsedSpace;
- clusterCapacity += datanodeCapacity;
- }
- return (double) clusterUsedSpace / clusterCapacity;
- }
-
- /**
- * Create some datanodes and containers for each node.
- */
- private void generateData() {
- this.numberOfNodes = 10;
- generateUtilizations(numberOfNodes);
- nodesInCluster = new ArrayList<>(nodeUtilizations.size());
-
- // create datanodes and add containers to them
- for (int i = 0; i < numberOfNodes; i++) {
- Set containerIDSet = new HashSet<>();
- DatanodeUsageInfo usageInfo =
- new DatanodeUsageInfo(MockDatanodeDetails.randomDatanodeDetails(),
- new SCMNodeStat());
-
- // create containers with varying used space
- int sizeMultiple = 0;
- for (int j = 0; j < i; j++) {
- sizeMultiple %= 5;
- sizeMultiple++;
- ContainerInfo container =
- createContainer((long) i * i + j, sizeMultiple);
-
- cidToInfoMap.put(container.containerID(), container);
- containerIDSet.add(container.containerID());
-
- // create initial replica for this container and add it
- Set containerReplicaSet = new HashSet<>();
- containerReplicaSet.add(createReplica(container.containerID(),
- usageInfo.getDatanodeDetails(), container.getUsedBytes()));
- cidToReplicasMap.put(container.containerID(), containerReplicaSet);
- }
- nodesInCluster.add(usageInfo);
- datanodeToContainersMap.put(usageInfo, containerIDSet);
- }
- }
-
- private ContainerInfo createContainer(long id, int multiple) {
- ContainerInfo.Builder builder = new ContainerInfo.Builder()
- .setContainerID(id)
- .setState(HddsProtos.LifeCycleState.CLOSED)
- .setOwner("TestContainerBalancer")
- .setUsedBytes(STORAGE_UNIT * multiple);
-
- /*
- Make it a RATIS container if id is even, else make it an EC container
- */
- if (id % 2 == 0) {
- builder.setReplicationConfig(RatisReplicationConfig
- .getInstance(HddsProtos.ReplicationFactor.THREE));
- } else {
- builder.setReplicationConfig(new ECReplicationConfig(3, 2));
- }
-
- return builder.build();
- }
-
- /**
- * Create the required number of replicas for each container. Note that one
- * replica already exists and nodes with utilization value 0 should not
- * have any replicas.
- */
- private void createReplicasForContainers() {
- for (ContainerInfo container : cidToInfoMap.values()) {
-
- // one replica already exists; create the remaining ones
- for (int i = 0;
- i < container.getReplicationConfig().getRequiredNodes() - 1; i++) {
-
- // randomly pick a datanode for this replica
- int datanodeIndex = RANDOM.nextInt(0, numberOfNodes);
- // don't put replicas in DNs that are supposed to have 0 utilization
- if (Math.abs(nodeUtilizations.get(datanodeIndex) - 0.0d) > 0.00001) {
- DatanodeDetails node =
- nodesInCluster.get(datanodeIndex).getDatanodeDetails();
- Set replicas =
- cidToReplicasMap.get(container.containerID());
- replicas.add(createReplica(container.containerID(), node,
- container.getUsedBytes()));
- cidToReplicasMap.put(container.containerID(), replicas);
- datanodeToContainersMap.get(nodesInCluster.get(datanodeIndex))
- .add(container.containerID());
- }
- }
- }
- }
-
- private ContainerReplica createReplica(ContainerID containerID,
- DatanodeDetails datanodeDetails,
- long usedBytes) {
- return ContainerReplica.newBuilder()
- .setContainerID(containerID)
- .setContainerState(ContainerReplicaProto.State.CLOSED)
- .setDatanodeDetails(datanodeDetails)
- .setOriginNodeId(datanodeDetails.getUuid())
- .setSequenceId(1000L)
- .setBytesUsed(usedBytes)
- .build();
- }
-
- private void startBalancer(ContainerBalancerConfiguration config)
- throws IllegalContainerBalancerStateException, IOException,
- InvalidContainerBalancerConfigurationException, TimeoutException {
- containerBalancerTask.setConfig(config);
- containerBalancerTask.setTaskStatus(ContainerBalancerTask.Status.RUNNING);
- containerBalancerTask.run();
- }
-
- private void stopBalancer() {
- // do nothing as testcase is not threaded
- }
-
- private CompletableFuture
- genCompletableFuture(int sleepMilSec) {
- return CompletableFuture.supplyAsync(() -> {
- try {
- Thread.sleep(sleepMilSec);
- } catch (InterruptedException e) {
- e.printStackTrace();
- }
- return MoveManager.MoveResult.COMPLETED;
- });
+ private static boolean stillHaveUnbalancedNodes(@Nonnull ContainerBalancerTask task) {
+ return !task.getOverUtilizedNodes().isEmpty() || !task.getUnderUtilizedNodes().isEmpty();
}
}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestFindTargetStrategy.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestFindTargetStrategy.java
deleted file mode 100644
index 7e734042d883..000000000000
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestFindTargetStrategy.java
+++ /dev/null
@@ -1,231 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdds.scm.container.balancer;
-
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
-import org.apache.hadoop.hdds.scm.container.MockNodeManager;
-import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
-import org.apache.hadoop.hdds.scm.net.NetworkTopology;
-import org.apache.hadoop.hdds.scm.net.NetworkTopologyImpl;
-import org.apache.hadoop.hdds.scm.net.NodeSchema;
-import org.apache.hadoop.hdds.scm.net.NodeSchemaManager;
-import org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.Assertions;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import static org.apache.hadoop.hdds.scm.net.NetConstants.LEAF_SCHEMA;
-import static org.apache.hadoop.hdds.scm.net.NetConstants.NODEGROUP_SCHEMA;
-import static org.apache.hadoop.hdds.scm.net.NetConstants.RACK_SCHEMA;
-import static org.apache.hadoop.hdds.scm.net.NetConstants.ROOT_SCHEMA;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-
-/**
- * Tests for all the implementations of FindTargetStrategy.
- */
-public class TestFindTargetStrategy {
- /**
- * Checks whether FindTargetGreedyByUsage always choose target
- * for a given source by Usage.
- */
- @Test
- public void testFindTargetGreedyByUsage() {
- FindTargetGreedyByUsageInfo findTargetStrategyByUsageInfo =
- new FindTargetGreedyByUsageInfo(null, null, null);
- List overUtilizedDatanodes = new ArrayList<>();
-
- //create three datanodes with different usageinfo
- DatanodeUsageInfo dui1 = new DatanodeUsageInfo(MockDatanodeDetails
- .randomDatanodeDetails(), new SCMNodeStat(100, 0, 40));
- DatanodeUsageInfo dui2 = new DatanodeUsageInfo(MockDatanodeDetails
- .randomDatanodeDetails(), new SCMNodeStat(100, 0, 60));
- DatanodeUsageInfo dui3 = new DatanodeUsageInfo(MockDatanodeDetails
- .randomDatanodeDetails(), new SCMNodeStat(100, 0, 80));
-
- //insert in ascending order
- overUtilizedDatanodes.add(dui1);
- overUtilizedDatanodes.add(dui2);
- overUtilizedDatanodes.add(dui3);
- findTargetStrategyByUsageInfo.reInitialize(
- overUtilizedDatanodes, null, null);
-
- //no need to set the datanode usage for source.
- findTargetStrategyByUsageInfo.sortTargetForSource(
- MockDatanodeDetails.randomDatanodeDetails());
-
- Collection potentialTargets =
- findTargetStrategyByUsageInfo.getPotentialTargets();
-
- Object[] sortedPotentialTargetArray = potentialTargets.toArray();
-
- Assertions.assertEquals(sortedPotentialTargetArray.length, 3);
-
- //make sure after sorting target for source, the potentialTargets is
- //sorted in descending order of usage
- Assertions.assertEquals(((DatanodeUsageInfo)sortedPotentialTargetArray[0])
- .getDatanodeDetails(), dui3.getDatanodeDetails());
- Assertions.assertEquals(((DatanodeUsageInfo)sortedPotentialTargetArray[1])
- .getDatanodeDetails(), dui2.getDatanodeDetails());
- Assertions.assertEquals(((DatanodeUsageInfo)sortedPotentialTargetArray[2])
- .getDatanodeDetails(), dui1.getDatanodeDetails());
-
- }
-
- /**
- * Tests {@link FindTargetStrategy#resetPotentialTargets(Collection)}.
- */
- @Test
- public void testResetPotentialTargets() {
- // create three datanodes with different usage infos
- DatanodeUsageInfo dui1 = new DatanodeUsageInfo(MockDatanodeDetails
- .randomDatanodeDetails(), new SCMNodeStat(100, 30, 70));
- DatanodeUsageInfo dui2 = new DatanodeUsageInfo(MockDatanodeDetails
- .randomDatanodeDetails(), new SCMNodeStat(100, 20, 80));
- DatanodeUsageInfo dui3 = new DatanodeUsageInfo(MockDatanodeDetails
- .randomDatanodeDetails(), new SCMNodeStat(100, 10, 90));
-
- List potentialTargets = new ArrayList<>();
- potentialTargets.add(dui1);
- potentialTargets.add(dui2);
- potentialTargets.add(dui3);
- MockNodeManager mockNodeManager = new MockNodeManager(potentialTargets);
-
- FindTargetGreedyByUsageInfo findTargetGreedyByUsageInfo =
- new FindTargetGreedyByUsageInfo(null, null, mockNodeManager);
- findTargetGreedyByUsageInfo.reInitialize(potentialTargets, null, null);
-
- // now, reset potential targets to only the first datanode
- List newPotentialTargets = new ArrayList<>(1);
- newPotentialTargets.add(dui1.getDatanodeDetails());
- findTargetGreedyByUsageInfo.resetPotentialTargets(newPotentialTargets);
- Assertions.assertEquals(1,
- findTargetGreedyByUsageInfo.getPotentialTargets().size());
- Assertions.assertEquals(dui1,
- findTargetGreedyByUsageInfo.getPotentialTargets().iterator().next());
- }
-
- /**
- * Checks whether FindTargetGreedyByNetworkTopology always choose target
- * for a given source by network topology distance.
- */
- @Test
- public void testFindTargetGreedyByNetworkTopology() {
- // network topology with default cost
- List schemas = new ArrayList<>();
- schemas.add(ROOT_SCHEMA);
- schemas.add(RACK_SCHEMA);
- schemas.add(NODEGROUP_SCHEMA);
- schemas.add(LEAF_SCHEMA);
-
- NodeSchemaManager manager = NodeSchemaManager.getInstance();
- manager.init(schemas.toArray(new NodeSchema[0]), true);
- NetworkTopology newCluster =
- new NetworkTopologyImpl(manager);
-
- DatanodeDetails source =
- MockDatanodeDetails.createDatanodeDetails("1.1.1.1", "/r1/ng1");
- //create one target in the same rack and same node group
- DatanodeDetails target1 =
- MockDatanodeDetails.createDatanodeDetails("2.2.2.2", "/r1/ng1");
- //create tree targets in the same rack but different node group
- DatanodeDetails target2 =
- MockDatanodeDetails.createDatanodeDetails("3.3.3.3", "/r1/ng2");
- DatanodeDetails target3 =
- MockDatanodeDetails.createDatanodeDetails("4.4.4.4", "/r1/ng2");
- DatanodeDetails target4 =
- MockDatanodeDetails.createDatanodeDetails("5.5.5.5", "/r1/ng2");
- //create one target in different rack
- DatanodeDetails target5 =
- MockDatanodeDetails.createDatanodeDetails("6.6.6.6", "/r2/ng1");
-
- //add all datanode to cluster map
- newCluster.add(source);
- newCluster.add(target1);
- newCluster.add(target2);
- newCluster.add(target3);
- newCluster.add(target4);
- newCluster.add(target5);
-
- //make sure targets have different network topology distance to source
- assertEquals(2, newCluster.getDistanceCost(source, target1));
- assertEquals(4, newCluster.getDistanceCost(source, target2));
- assertEquals(4, newCluster.getDistanceCost(source, target3));
- assertEquals(4, newCluster.getDistanceCost(source, target4));
- assertEquals(6, newCluster.getDistanceCost(source, target5));
-
-
-
- //insert in ascending order of network topology distance
- List overUtilizedDatanodes = new ArrayList<>();
- //set the farthest target with the lowest usage info
- overUtilizedDatanodes.add(
- new DatanodeUsageInfo(target5, new SCMNodeStat(100, 0, 90)));
- //set the tree targets, which have the same network topology distance
- //to source , with different usage info
- overUtilizedDatanodes.add(
- new DatanodeUsageInfo(target2, new SCMNodeStat(100, 0, 20)));
- overUtilizedDatanodes.add(
- new DatanodeUsageInfo(target3, new SCMNodeStat(100, 0, 40)));
- overUtilizedDatanodes.add(
- new DatanodeUsageInfo(target4, new SCMNodeStat(100, 0, 60)));
- //set the nearest target with the highest usage info
- overUtilizedDatanodes.add(
- new DatanodeUsageInfo(target1, new SCMNodeStat(100, 0, 10)));
-
-
- FindTargetGreedyByNetworkTopology findTargetGreedyByNetworkTopology =
- new FindTargetGreedyByNetworkTopology(
- null, null, null, newCluster);
-
- findTargetGreedyByNetworkTopology.reInitialize(
- overUtilizedDatanodes, null, null);
-
- findTargetGreedyByNetworkTopology.sortTargetForSource(source);
-
- Collection potentialTargets =
- findTargetGreedyByNetworkTopology.getPotentialTargets();
-
- Object[] sortedPotentialTargetArray = potentialTargets.toArray();
- Assertions.assertEquals(sortedPotentialTargetArray.length, 5);
-
- // although target1 has the highest usage, it has the nearest network
- // topology distance to source, so it should be at the head of the
- // sorted PotentialTargetArray
- Assertions.assertEquals(((DatanodeUsageInfo)sortedPotentialTargetArray[0])
- .getDatanodeDetails(), target1);
-
- // these targets have same network topology distance to source,
- // so they should be sorted by usage
- Assertions.assertEquals(((DatanodeUsageInfo)sortedPotentialTargetArray[1])
- .getDatanodeDetails(), target4);
- Assertions.assertEquals(((DatanodeUsageInfo)sortedPotentialTargetArray[2])
- .getDatanodeDetails(), target3);
- Assertions.assertEquals(((DatanodeUsageInfo)sortedPotentialTargetArray[3])
- .getDatanodeDetails(), target2);
-
- //target5 has the lowest usage , but it has the farthest distance to source
- //so it should be at the tail of the sorted PotentialTargetArray
- Assertions.assertEquals(((DatanodeUsageInfo)sortedPotentialTargetArray[4])
- .getDatanodeDetails(), target5);
- }
-}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestableCluster.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestableCluster.java
new file mode 100644
index 000000000000..032943372219
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestableCluster.java
@@ -0,0 +1,252 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.hdds.scm.container.balancer;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
+import org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+
+/**
+ * Class is used for creating test cluster with a required number of datanodes.
+ * 1. Fill the cluster by generating some data.
+ * 2. Nodes in the cluster have utilization values determined by
+ * generateUtilization method.
+ */
+public final class TestableCluster {
+ static final ThreadLocalRandom RANDOM = ThreadLocalRandom.current();
+ private static final Logger LOG = LoggerFactory.getLogger(TestableCluster.class);
+ private final int nodeCount;
+ private final double[] nodeUtilizationList;
+ private final DatanodeUsageInfo[] nodesInCluster;
+ private final Map cidToInfoMap = new HashMap<>();
+ private final Map> cidToReplicasMap = new HashMap<>();
+ private final Map> dnUsageToContainersMap = new HashMap<>();
+ private final double averageUtilization;
+
+ TestableCluster(int numberOfNodes, long storageUnit) {
+ nodeCount = numberOfNodes;
+ nodeUtilizationList = createUtilizationList(nodeCount);
+ nodesInCluster = new DatanodeUsageInfo[nodeCount];
+
+ generateData(storageUnit);
+ createReplicasForContainers();
+ long clusterCapacity = 0, clusterUsedSpace = 0;
+
+ // For each node utilization, calculate that datanode's used space and capacity
+ for (int i = 0; i < nodeUtilizationList.length; i++) {
+ Set containerIDSet = dnUsageToContainersMap.get(nodesInCluster[i]);
+ long datanodeUsedSpace = 0;
+ for (ContainerID containerID : containerIDSet) {
+ datanodeUsedSpace += cidToInfoMap.get(containerID).getUsedBytes();
+ }
+ // Use node utilization and used space to determine node capacity
+ long datanodeCapacity = (nodeUtilizationList[i] == 0)
+ ? storageUnit * RANDOM.nextInt(10, 60)
+ : (long) (datanodeUsedSpace / nodeUtilizationList[i]);
+
+ SCMNodeStat stat = new SCMNodeStat(datanodeCapacity, datanodeUsedSpace, datanodeCapacity - datanodeUsedSpace);
+ nodesInCluster[i].setScmNodeStat(stat);
+ clusterUsedSpace += datanodeUsedSpace;
+ clusterCapacity += datanodeCapacity;
+ }
+
+ averageUtilization = (double) clusterUsedSpace / clusterCapacity;
+ }
+
+ @Override
+ public String toString() {
+ return "cluster of " + nodeCount + " nodes";
+ }
+
+ @Nonnull Map> getDatanodeToContainersMap() {
+ return dnUsageToContainersMap;
+ }
+
+ @Nonnull Map getCidToInfoMap() {
+ return cidToInfoMap;
+ }
+
+ public int getNodeCount() {
+ return nodeCount;
+ }
+
+ double getAverageUtilization() {
+ return averageUtilization;
+ }
+
+ @Nonnull DatanodeUsageInfo[] getNodesInCluster() {
+ return nodesInCluster;
+ }
+
+ double[] getNodeUtilizationList() {
+ return nodeUtilizationList;
+ }
+
+ @Nonnull Map> getCidToReplicasMap() {
+ return cidToReplicasMap;
+ }
+
+ /**
+ * Determines unBalanced nodes, that is, over and under utilized nodes, according to the generated utilization values
+ * for nodes and the threshold.
+ *
+ * @param threshold a percentage in the range 0 to 100
+ * @return list of DatanodeUsageInfo containing the expected(correct) unBalanced nodes.
+ */
+ @Nonnull List getUnBalancedNodes(double threshold) {
+ threshold /= 100;
+ double lowerLimit = averageUtilization - threshold;
+ double upperLimit = averageUtilization + threshold;
+
+ // use node utilizations to determine over and under utilized nodes
+ List expectedUnBalancedNodes = new ArrayList<>();
+ for (int i = 0; i < nodeCount; i++) {
+ double nodeUtilization = nodeUtilizationList[i];
+ if (nodeUtilization < lowerLimit || nodeUtilization > upperLimit) {
+ expectedUnBalancedNodes.add(nodesInCluster[i]);
+ }
+ }
+ return expectedUnBalancedNodes;
+ }
+
+ /**
+ * Create some datanodes and containers for each node.
+ */
+ private void generateData(long storageUnit) {
+ // Create datanodes and add containers to them
+ for (int i = 0; i < nodeCount; i++) {
+ DatanodeUsageInfo usageInfo =
+ new DatanodeUsageInfo(MockDatanodeDetails.randomDatanodeDetails(), new SCMNodeStat());
+ nodesInCluster[i] = usageInfo;
+
+ // Create containers with varying used space
+ Set containerIDSet = new HashSet<>();
+ int sizeMultiple = 0;
+ for (int j = 0; j < i; j++) {
+ sizeMultiple %= 5;
+ sizeMultiple++;
+ ContainerInfo container = createContainer((long) i * i + j, storageUnit * sizeMultiple);
+
+ cidToInfoMap.put(container.containerID(), container);
+ containerIDSet.add(container.containerID());
+
+ // Create initial replica for this container and add it
+ Set containerReplicaSet = new HashSet<>();
+ containerReplicaSet.add(createReplica(container.containerID(),
+ usageInfo.getDatanodeDetails(), container.getUsedBytes()));
+ cidToReplicasMap.put(container.containerID(), containerReplicaSet);
+ }
+ dnUsageToContainersMap.put(usageInfo, containerIDSet);
+ }
+ }
+
+ private @Nonnull ContainerInfo createContainer(long id, long usedBytes) {
+ ContainerInfo.Builder builder = new ContainerInfo.Builder()
+ .setContainerID(id)
+ .setState(HddsProtos.LifeCycleState.CLOSED)
+ .setOwner("TestContainerBalancer")
+ .setUsedBytes(usedBytes);
+
+ /* Make it a RATIS container if id is even, else make it an EC container */
+ ReplicationConfig config = (id % 2 == 0)
+ ? RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE)
+ : new ECReplicationConfig(3, 2);
+
+ builder.setReplicationConfig(config);
+ return builder.build();
+ }
+
+ /**
+ * Create the required number of replicas for each container. Note that one replica already exists and
+ * nodes with utilization value 0 should not have any replicas.
+ */
+ private void createReplicasForContainers() {
+ for (ContainerInfo container : cidToInfoMap.values()) {
+ // one replica already exists; create the remaining ones
+ ReplicationConfig replicationConfig = container.getReplicationConfig();
+ ContainerID key = container.containerID();
+ for (int i = 0; i < replicationConfig.getRequiredNodes() - 1; i++) {
+ // randomly pick a datanode for this replica
+ int dnIndex = RANDOM.nextInt(0, nodeCount);
+ // don't put replicas in DNs that are supposed to have 0 utilization
+ if (Math.abs(nodeUtilizationList[dnIndex] - 0.0d) > 0.00001) {
+ DatanodeDetails node = nodesInCluster[dnIndex].getDatanodeDetails();
+ Set replicas = cidToReplicasMap.get(key);
+ replicas.add(createReplica(key, node, container.getUsedBytes()));
+ cidToReplicasMap.put(key, replicas);
+ dnUsageToContainersMap.get(nodesInCluster[dnIndex]).add(key);
+ }
+ }
+ }
+ }
+
+ /**
+ * Generates a range of equally spaced utilization(that is, used / capacity) values from 0 to 1.
+ *
+ * @param count Number of values to generate. Count must be greater than or equal to 1.
+ * @return double array of node utilization values
+ * @throws IllegalArgumentException If the value of the parameter count is less than 1.
+ */
+ private static double[] createUtilizationList(int count) throws IllegalArgumentException {
+ if (count < 1) {
+ LOG.warn("The value of argument count is {}. However, count must be greater than 0.", count);
+ throw new IllegalArgumentException();
+ }
+ double[] result = new double[count];
+ for (int i = 0; i < count; i++) {
+ result[i] = (i / (double) count);
+ }
+ return result;
+ }
+
+ private @Nonnull ContainerReplica createReplica(
+ @Nonnull ContainerID containerID,
+ @Nonnull DatanodeDetails datanodeDetails,
+ long usedBytes
+ ) {
+ return ContainerReplica.newBuilder()
+ .setContainerID(containerID)
+ .setContainerState(ContainerReplicaProto.State.CLOSED)
+ .setDatanodeDetails(datanodeDetails)
+ .setOriginNodeId(datanodeDetails.getUuid())
+ .setSequenceId(1000L)
+ .setBytesUsed(usedBytes)
+ .build();
+ }
+}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/TestContainerBalancerIteration.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/TestContainerBalancerIteration.java
new file mode 100644
index 000000000000..ee3d32f6d3ff
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/TestContainerBalancerIteration.java
@@ -0,0 +1,264 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.scm.container.balancer.iteration;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
+import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancer;
+import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerConfiguration;
+import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerMetrics;
+import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerTask;
+import org.apache.hadoop.hdds.scm.container.balancer.MockedSCM;
+import org.apache.hadoop.hdds.scm.container.balancer.MoveManager;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.mockito.Mockito;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeoutException;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+
+/**
+ * Tests for {@link ContainerBalancer}.
+ */
+public class TestContainerBalancerIteration {
+ /**
+ * Tests the situation where some container moves time out because they take longer than "move.timeout".
+ */
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource(
+ "org.apache.hadoop.hdds.scm.container.balancer.TestContainerBalancerTask#createMockedSCMWithDatanodeLimits")
+ public void checkIterationResultTimeout(@Nonnull MockedSCM mockedSCM, boolean useDatanodeLimits)
+ throws NodeNotFoundException, IOException, TimeoutException {
+ CompletableFuture completedFuture =
+ CompletableFuture.completedFuture(MoveManager.MoveResult.COMPLETED);
+ Mockito
+ .when(mockedSCM.getReplicationManager()
+ .move(
+ any(ContainerID.class),
+ any(DatanodeDetails.class),
+ any(DatanodeDetails.class)))
+ .thenReturn(completedFuture)
+ .thenAnswer(invocation -> genCompletableFuture(2000, false));
+
+ ContainerBalancerConfiguration config = mockedSCM.getBalancerConfig();
+ config.setThreshold(10);
+ config.setIterations(1);
+ config.setMaxSizeEnteringTarget(10 * MockedSCM.STORAGE_UNIT);
+ config.setMaxSizeToMovePerIteration(100 * MockedSCM.STORAGE_UNIT);
+ config.setMaxDatanodesPercentageToInvolvePerIteration(100);
+ config.setMoveTimeout(Duration.ofMillis(500));
+ config.setAdaptBalanceWhenCloseToLimit(useDatanodeLimits);
+ config.setAdaptBalanceWhenReachTheLimit(useDatanodeLimits);
+
+ mockedSCM.enableLegacyReplicationManager();
+ ContainerBalancer balancer = mockedSCM.createContainerBalancer();
+ ContainerBalancerTask task = mockedSCM.startBalancerTask(balancer, config);
+
+ // According to the setup and configurations, this iteration's result should be ITERATION_COMPLETED.
+ assertEquals(IterationResult.ITERATION_COMPLETED, task.getIterationResult());
+
+ ContainerBalancerMetrics metrics = balancer.getMetrics();
+ assertEquals(1, metrics.getNumContainerMovesCompleted());
+ assertTrue(metrics.getNumContainerMovesTimeout() >= 1);
+
+ // Test the same but use MoveManager instead of LegacyReplicationManager.
+ // The first move being 10ms falls within the timeout duration of 500ms.
+ // It should be successful. The rest should fail.
+ mockedSCM.disableLegacyReplicationManager();
+ Mockito
+ .when(mockedSCM.getMoveManager()
+ .move(
+ any(ContainerID.class),
+ any(DatanodeDetails.class),
+ any(DatanodeDetails.class)))
+ .thenReturn(completedFuture)
+ .thenAnswer(invocation -> genCompletableFuture(2000, false));
+
+ assertEquals(IterationResult.ITERATION_COMPLETED, task.getIterationResult());
+ assertEquals(1, metrics.getNumContainerMovesCompleted());
+ assertTrue(metrics.getNumContainerMovesTimeout() >= 1);
+ }
+
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource(
+ "org.apache.hadoop.hdds.scm.container.balancer.TestContainerBalancerTask#createMockedSCMWithDatanodeLimits")
+ public void checkIterationResultTimeoutFromReplicationManager(@Nonnull MockedSCM mockedSCM, boolean ignored)
+ throws NodeNotFoundException, IOException, TimeoutException {
+ CompletableFuture future = CompletableFuture
+ .supplyAsync(() -> MoveManager.MoveResult.REPLICATION_FAIL_TIME_OUT);
+ CompletableFuture future2 = CompletableFuture
+ .supplyAsync(() -> MoveManager.MoveResult.DELETION_FAIL_TIME_OUT);
+
+ Mockito
+ .when(mockedSCM.getReplicationManager()
+ .move(
+ any(ContainerID.class),
+ any(DatanodeDetails.class),
+ any(DatanodeDetails.class)))
+ .thenReturn(future, future2);
+
+ ContainerBalancerConfiguration config = mockedSCM.getBalancerConfig();
+ config.setThreshold(10);
+ config.setIterations(1);
+ config.setMaxSizeEnteringTarget(10 * MockedSCM.STORAGE_UNIT);
+ config.setMaxSizeToMovePerIteration(100 * MockedSCM.STORAGE_UNIT);
+ config.setMaxDatanodesPercentageToInvolvePerIteration(100);
+ config.setMoveTimeout(Duration.ofMillis(500));
+ mockedSCM.enableLegacyReplicationManager();
+ ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
+
+ IterationMetrics metrics = task.getIterationMetrics();
+ assertTrue(metrics.getTimeoutContainerMovesCount() > 0);
+ assertEquals(0, metrics.getCompletedContainerMovesCount());
+
+ // Try the same test with MoveManager instead of LegacyReplicationManager.
+ Mockito
+ .when(mockedSCM.getMoveManager()
+ .move(
+ any(ContainerID.class),
+ any(DatanodeDetails.class),
+ any(DatanodeDetails.class)))
+ .thenReturn(future)
+ .thenAnswer(invocation -> future2);
+
+ mockedSCM.disableLegacyReplicationManager();
+
+ assertTrue(metrics.getTimeoutContainerMovesCount() > 0);
+ assertEquals(0, metrics.getCompletedContainerMovesCount());
+ }
+
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource(
+ "org.apache.hadoop.hdds.scm.container.balancer.TestContainerBalancerTask#createMockedSCMWithDatanodeLimits")
+ public void checkIterationResultException(@Nonnull MockedSCM mockedSCM, boolean useDatanodeLimits)
+ throws NodeNotFoundException, IOException, TimeoutException {
+ CompletableFuture future = new CompletableFuture<>();
+ future.completeExceptionally(new RuntimeException("Runtime Exception"));
+ Mockito
+ .when(mockedSCM.getReplicationManager()
+ .move(
+ any(ContainerID.class),
+ any(DatanodeDetails.class),
+ any(DatanodeDetails.class)))
+ .thenReturn(genCompletableFuture(1, true))
+ .thenThrow(new ContainerNotFoundException("Test Container not found"))
+ .thenReturn(future);
+
+ ContainerBalancerConfiguration config = mockedSCM.getBalancerConfig();
+ config.setThreshold(10);
+ config.setIterations(1);
+ config.setMaxSizeEnteringTarget(10 * MockedSCM.STORAGE_UNIT);
+ config.setMaxSizeToMovePerIteration(100 * MockedSCM.STORAGE_UNIT);
+ config.setMaxDatanodesPercentageToInvolvePerIteration(100);
+ config.setMoveTimeout(Duration.ofMillis(500));
+ config.setAdaptBalanceWhenCloseToLimit(useDatanodeLimits);
+ config.setAdaptBalanceWhenReachTheLimit(useDatanodeLimits);
+
+ mockedSCM.enableLegacyReplicationManager();
+ ContainerBalancer balancer = mockedSCM.createContainerBalancer();
+ ContainerBalancerTask task = mockedSCM.startBalancerTask(balancer, config);
+ assertEquals(IterationResult.ITERATION_COMPLETED, task.getIterationResult());
+
+ int nodeCount = mockedSCM.getCluster().getNodeCount();
+ int expectedMovesFailed = (nodeCount > 6) ? 3 : 1;
+ assertTrue(balancer.getMetrics().getNumContainerMovesFailed() >= expectedMovesFailed);
+
+ // Try the same test but with MoveManager instead of ReplicationManager.
+ Mockito
+ .when(mockedSCM.getMoveManager()
+ .move(
+ any(ContainerID.class),
+ any(DatanodeDetails.class),
+ any(DatanodeDetails.class)))
+ .thenReturn(genCompletableFuture(1, true))
+ .thenThrow(new ContainerNotFoundException("Test Container not found"))
+ .thenReturn(future);
+
+ mockedSCM.disableLegacyReplicationManager();
+ task = mockedSCM.startBalancerTask(config);
+
+ assertEquals(IterationResult.ITERATION_COMPLETED, task.getIterationResult());
+ assertTrue(balancer.getMetrics().getNumContainerMovesFailed() >= expectedMovesFailed);
+ }
+
+ @ParameterizedTest(name = "MockedSCM #{index}: {0}; use datanode limits: {1}")
+ @MethodSource(
+ "org.apache.hadoop.hdds.scm.container.balancer.TestContainerBalancerTask#createMockedSCMWithDatanodeLimits")
+ public void checkIterationResult(@Nonnull MockedSCM mockedSCM, boolean ignored)
+ throws NodeNotFoundException, IOException, TimeoutException {
+ ContainerBalancerConfiguration config = mockedSCM.getBalancerConfig();
+ config.setThreshold(10);
+ config.setIterations(1);
+ config.setMaxSizeEnteringTarget(10 * MockedSCM.STORAGE_UNIT);
+ config.setMaxSizeToMovePerIteration(100 * MockedSCM.STORAGE_UNIT);
+ config.setMaxDatanodesPercentageToInvolvePerIteration(100);
+
+ mockedSCM.enableLegacyReplicationManager();
+ ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
+
+ // According to the setup and configurations, this iteration's result should be ITERATION_COMPLETED.
+ assertEquals(IterationResult.ITERATION_COMPLETED, task.getIterationResult());
+
+ // Now, limit maxSizeToMovePerIteration but fail all container moves.
+ // The result should still be ITERATION_COMPLETED.
+ Mockito
+ .when(mockedSCM.getReplicationManager()
+ .move(
+ any(ContainerID.class),
+ any(DatanodeDetails.class),
+ any(DatanodeDetails.class)))
+ .thenReturn(CompletableFuture.completedFuture(MoveManager.MoveResult.REPLICATION_FAIL_NODE_UNHEALTHY));
+
+ config.setMaxSizeToMovePerIteration(10 * MockedSCM.STORAGE_UNIT);
+ mockedSCM.startBalancerTask(config);
+ assertEquals(IterationResult.ITERATION_COMPLETED, task.getIterationResult());
+
+ // Try the same but use MoveManager for container move instead of legacy RM.
+ mockedSCM.disableLegacyReplicationManager();
+ mockedSCM.startBalancerTask(config);
+ assertEquals(IterationResult.ITERATION_COMPLETED, task.getIterationResult());
+ }
+
+ private static @Nonnull CompletableFuture genCompletableFuture(
+ int sleepMilSec,
+ boolean doThrowException
+ ) {
+ return CompletableFuture.supplyAsync(() -> {
+ try {
+ Thread.sleep(sleepMilSec);
+ } catch (InterruptedException e) {
+ if (doThrowException) {
+ throw new RuntimeException("Runtime Exception after doing work");
+ } else {
+ e.printStackTrace();
+ }
+ }
+ return MoveManager.MoveResult.COMPLETED;
+ });
+ }
+}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/TestFindTargetStrategy.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/TestFindTargetStrategy.java
new file mode 100644
index 000000000000..f9a2dbf7511a
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/iteration/TestFindTargetStrategy.java
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.scm.container.balancer.iteration;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.scm.container.MockNodeManager;
+import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
+import org.apache.hadoop.hdds.scm.net.NetworkTopology;
+import org.apache.hadoop.hdds.scm.net.NetworkTopologyImpl;
+import org.apache.hadoop.hdds.scm.net.NodeSchema;
+import org.apache.hadoop.hdds.scm.net.NodeSchemaManager;
+import org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo;
+import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.hadoop.hdds.scm.net.NetConstants.LEAF_SCHEMA;
+import static org.apache.hadoop.hdds.scm.net.NetConstants.NODEGROUP_SCHEMA;
+import static org.apache.hadoop.hdds.scm.net.NetConstants.RACK_SCHEMA;
+import static org.apache.hadoop.hdds.scm.net.NetConstants.ROOT_SCHEMA;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for all the implementations of FindTargetStrategy.
+ */
+public class TestFindTargetStrategy {
+ /**
+ * Checks whether FindTargetGreedyByUsage always choose target for a given source by Usage.
+ */
+ @Test
+ public void testFindTargetGreedyByUsage() {
+ StorageContainerManager scm = Mockito.mock(StorageContainerManager.class);
+ FindTargetGreedyByUsageInfo findTargetStrategyByUsageInfo = new FindTargetGreedyByUsageInfo(scm);
+ List overUtilizedDatanodes = new ArrayList<>();
+
+ //create three datanodes with different usageinfo
+ DatanodeUsageInfo dui1 = new DatanodeUsageInfo(
+ MockDatanodeDetails.randomDatanodeDetails(), new SCMNodeStat(100, 0, 40));
+ DatanodeUsageInfo dui2 = new DatanodeUsageInfo(
+ MockDatanodeDetails.randomDatanodeDetails(), new SCMNodeStat(100, 0, 60));
+ DatanodeUsageInfo dui3 = new DatanodeUsageInfo(
+ MockDatanodeDetails.randomDatanodeDetails(), new SCMNodeStat(100, 0, 80));
+
+ //insert in ascending order
+ overUtilizedDatanodes.add(dui1);
+ overUtilizedDatanodes.add(dui2);
+ overUtilizedDatanodes.add(dui3);
+ findTargetStrategyByUsageInfo.reInitialize(overUtilizedDatanodes);
+
+ //no need to set the datanode usage for source.
+ findTargetStrategyByUsageInfo.sortTargetForSource(MockDatanodeDetails.randomDatanodeDetails());
+
+ Collection potentialTargets = findTargetStrategyByUsageInfo.getPotentialTargets();
+ Object[] sortedPotentialTargetArray = potentialTargets.toArray();
+ assertEquals(sortedPotentialTargetArray.length, 3);
+
+ // Make sure after sorting target for source, the potentialTargets is sorted in descending order of usage
+ assertEquals(((DatanodeUsageInfo) sortedPotentialTargetArray[0]).getDatanodeDetails(), dui3.getDatanodeDetails());
+ assertEquals(((DatanodeUsageInfo) sortedPotentialTargetArray[1]).getDatanodeDetails(), dui2.getDatanodeDetails());
+ assertEquals(((DatanodeUsageInfo) sortedPotentialTargetArray[2]).getDatanodeDetails(), dui1.getDatanodeDetails());
+
+ }
+
+ /**
+ * Tests {@link FindTargetStrategy#resetPotentialTargets(Collection)}.
+ */
+ @Test
+ public void testResetPotentialTargets() {
+ // create three datanodes with different usage infos
+ DatanodeUsageInfo dui1 = new DatanodeUsageInfo(
+ MockDatanodeDetails.randomDatanodeDetails(), new SCMNodeStat(100, 30, 70));
+ DatanodeUsageInfo dui2 = new DatanodeUsageInfo(
+ MockDatanodeDetails.randomDatanodeDetails(), new SCMNodeStat(100, 20, 80));
+ DatanodeUsageInfo dui3 = new DatanodeUsageInfo(
+ MockDatanodeDetails.randomDatanodeDetails(), new SCMNodeStat(100, 10, 90));
+
+ List potentialTargets = new ArrayList<>();
+ potentialTargets.add(dui1);
+ potentialTargets.add(dui2);
+ potentialTargets.add(dui3);
+ MockNodeManager mockNodeManager = new MockNodeManager(potentialTargets);
+
+ StorageContainerManager scm = Mockito.mock(StorageContainerManager.class);
+ when(scm.getScmNodeManager()).thenReturn(mockNodeManager);
+
+ FindTargetGreedyByUsageInfo findTargetGreedyByUsageInfo = new FindTargetGreedyByUsageInfo(scm);
+ findTargetGreedyByUsageInfo.reInitialize(potentialTargets);
+
+ // now, reset potential targets to only the first datanode
+ List newPotentialTargets = new ArrayList<>(1);
+ newPotentialTargets.add(dui1.getDatanodeDetails());
+ findTargetGreedyByUsageInfo.resetPotentialTargets(newPotentialTargets);
+ assertEquals(1, findTargetGreedyByUsageInfo.getPotentialTargets().size());
+ assertEquals(dui1, findTargetGreedyByUsageInfo.getPotentialTargets().iterator().next());
+ }
+
+ /**
+ * Checks whether FindTargetGreedyByNetworkTopology always choose target for a given source
+ * by network topology distance.
+ */
+ @Test
+ public void testFindTargetGreedyByNetworkTopology() {
+ // Network topology with default cost
+ List schemas = new ArrayList<>();
+ schemas.add(ROOT_SCHEMA);
+ schemas.add(RACK_SCHEMA);
+ schemas.add(NODEGROUP_SCHEMA);
+ schemas.add(LEAF_SCHEMA);
+
+ NodeSchemaManager manager = NodeSchemaManager.getInstance();
+ manager.init(schemas.toArray(new NodeSchema[0]), true);
+ NetworkTopology newCluster = new NetworkTopologyImpl(manager);
+
+ DatanodeDetails source = MockDatanodeDetails.createDatanodeDetails("1.1.1.1", "/r1/ng1");
+ // Create one target in the same rack and same node group
+ DatanodeDetails target1 = MockDatanodeDetails.createDatanodeDetails("2.2.2.2", "/r1/ng1");
+ // Create tree targets in the same rack but different node group
+ DatanodeDetails target2 = MockDatanodeDetails.createDatanodeDetails("3.3.3.3", "/r1/ng2");
+ DatanodeDetails target3 = MockDatanodeDetails.createDatanodeDetails("4.4.4.4", "/r1/ng2");
+ DatanodeDetails target4 = MockDatanodeDetails.createDatanodeDetails("5.5.5.5", "/r1/ng2");
+ // Create one target in different rack
+ DatanodeDetails target5 = MockDatanodeDetails.createDatanodeDetails("6.6.6.6", "/r2/ng1");
+
+ // Add all datanode to cluster map
+ newCluster.add(source);
+ newCluster.add(target1);
+ newCluster.add(target2);
+ newCluster.add(target3);
+ newCluster.add(target4);
+ newCluster.add(target5);
+
+ // Make sure targets have different network topology distance to source
+ assertEquals(2, newCluster.getDistanceCost(source, target1));
+ assertEquals(4, newCluster.getDistanceCost(source, target2));
+ assertEquals(4, newCluster.getDistanceCost(source, target3));
+ assertEquals(4, newCluster.getDistanceCost(source, target4));
+ assertEquals(6, newCluster.getDistanceCost(source, target5));
+
+
+ // Insert in ascending order of network topology distance
+ List overUtilizedDatanodes = new ArrayList<>();
+ // Set the farthest target with the lowest usage info
+ overUtilizedDatanodes.add(new DatanodeUsageInfo(target5, new SCMNodeStat(100, 0, 90)));
+ // Set the tree targets, which have the same network topology distance to source , with different usage info
+ overUtilizedDatanodes.add(new DatanodeUsageInfo(target2, new SCMNodeStat(100, 0, 20)));
+ overUtilizedDatanodes.add(new DatanodeUsageInfo(target3, new SCMNodeStat(100, 0, 40)));
+ overUtilizedDatanodes.add(new DatanodeUsageInfo(target4, new SCMNodeStat(100, 0, 60)));
+ // Set the nearest target with the highest usage info
+ overUtilizedDatanodes.add(new DatanodeUsageInfo(target1, new SCMNodeStat(100, 0, 10)));
+
+ StorageContainerManager scm = Mockito.mock(StorageContainerManager.class);
+ when(scm.getClusterMap()).thenReturn(newCluster);
+
+ FindTargetGreedyByNetworkTopology strategy = new FindTargetGreedyByNetworkTopology(scm);
+ strategy.reInitialize(overUtilizedDatanodes);
+ strategy.sortTargetForSource(source);
+
+ Collection potentialTargets = strategy.getPotentialTargets();
+ Object[] sortedPotentialTargetArray = potentialTargets.toArray();
+ Assertions.assertEquals(sortedPotentialTargetArray.length, 5);
+
+ // Although target1 has the highest usage, it has the nearest network topology distance to source,
+ // so it should be at the head of the sorted PotentialTargetArray
+ assertEquals(((DatanodeUsageInfo) sortedPotentialTargetArray[0]).getDatanodeDetails(), target1);
+
+ // These targets have same network topology distance to source, so they should be sorted by usage
+ assertEquals(((DatanodeUsageInfo) sortedPotentialTargetArray[1]).getDatanodeDetails(), target4);
+ assertEquals(((DatanodeUsageInfo) sortedPotentialTargetArray[2]).getDatanodeDetails(), target3);
+ assertEquals(((DatanodeUsageInfo) sortedPotentialTargetArray[3]).getDatanodeDetails(), target2);
+
+ // Target5 has the lowest usage , but it has the farthest distance to source so
+ // it should be at the tail of the sorted PotentialTargetArray
+ assertEquals(((DatanodeUsageInfo) sortedPotentialTargetArray[4]).getDatanodeDetails(), target5);
+ }
+}