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