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

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


The following commit(s) were added to refs/heads/master by this push:
     new 0bde3a293f HDDS-11410. Refactoring more tests from 
TestContainerBalancerTask (#7156)
0bde3a293f is described below

commit 0bde3a293fd2698e1ef620130e28c59ef685fd85
Author: Andrei Mikhalev <[email protected]>
AuthorDate: Tue Dec 3 22:08:01 2024 +0300

    HDDS-11410. Refactoring more tests from TestContainerBalancerTask (#7156)
---
 .../balancer/ContainerBalancerConfigBuilder.java   |  46 +++
 .../hdds/scm/container/balancer/MockedSCM.java     |   4 +
 .../TestContainerBalancerDatanodeNodeLimit.java    | 330 ++++++++++++---------
 .../balancer/TestContainerBalancerTask.java        | 239 +--------------
 4 files changed, 247 insertions(+), 372 deletions(-)

diff --git 
a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerConfigBuilder.java
 
b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerConfigBuilder.java
new file mode 100644
index 0000000000..fc4bc9fb05
--- /dev/null
+++ 
b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerConfigBuilder.java
@@ -0,0 +1,46 @@
+/**
+ * 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.conf.OzoneConfiguration;
+
+class ContainerBalancerConfigBuilder {
+  private static final int DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER = 15;
+
+  private final ContainerBalancerConfiguration config;
+
+  ContainerBalancerConfigBuilder(int nodeCount) {
+    this(new OzoneConfiguration(), nodeCount);
+  }
+
+  ContainerBalancerConfigBuilder(OzoneConfiguration ozoneConfig, int 
nodeCount) {
+    config = ozoneConfig.getObject(ContainerBalancerConfiguration.class);
+    config.setIterations(1);
+    config.setThreshold(10);
+    config.setMaxSizeToMovePerIteration(50 * 
TestContainerBalancerTask.STORAGE_UNIT);
+    config.setMaxSizeEnteringTarget(50 * 
TestContainerBalancerTask.STORAGE_UNIT);
+    if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) {
+      config.setMaxDatanodesPercentageToInvolvePerIteration(100);
+    }
+  }
+
+  ContainerBalancerConfiguration build() {
+    return config;
+  }
+}
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
index 0972e57df6..1c5c7749a4 100644
--- 
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
@@ -142,6 +142,10 @@ public final class MockedSCM {
     return startBalancerTask(new ContainerBalancer(scm), config);
   }
 
+  public int getNodeCount() {
+    return cluster.getNodeCount();
+  }
+
   public void enableLegacyReplicationManager() {
     mockedReplicaManager.conf.setEnableLegacy(true);
   }
diff --git 
a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerDatanodeNodeLimit.java
 
b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerDatanodeNodeLimit.java
index 7a8f655f06..813acc9c39 100644
--- 
a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerDatanodeNodeLimit.java
+++ 
b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerDatanodeNodeLimit.java
@@ -73,7 +73,6 @@ import static org.mockito.Mockito.when;
  */
 public class TestContainerBalancerDatanodeNodeLimit {
   private static final long STORAGE_UNIT = OzoneConsts.GB;
-  private static final int DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER = 15;
 
   @BeforeAll
   public static void setup() {
@@ -103,13 +102,8 @@ public class TestContainerBalancerDatanodeNodeLimit {
   @ParameterizedTest(name = "MockedSCM #{index}: {0}")
   @MethodSource("createMockedSCMs")
   public void containerBalancerShouldObeyMaxDatanodesToInvolveLimit(@Nonnull 
MockedSCM mockedSCM) {
-    ContainerBalancerConfiguration config = new 
OzoneConfiguration().getObject(ContainerBalancerConfiguration.class);
-    int nodeCount = mockedSCM.getCluster().getNodeCount();
-    if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) {
-      config.setMaxDatanodesPercentageToInvolvePerIteration(100);
-    }
-    config.setIterations(1);
-    config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
+    int nodeCount = mockedSCM.getNodeCount();
+    ContainerBalancerConfiguration config = new 
ContainerBalancerConfigBuilder(nodeCount).build();
 
     ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
     ContainerBalancerMetrics metrics = task.getMetrics();
@@ -129,12 +123,8 @@ public class TestContainerBalancerDatanodeNodeLimit {
   public void balancerShouldObeyMaxSizeEnteringTargetLimit(@Nonnull MockedSCM 
mockedSCM) {
     OzoneConfiguration ozoneConfig = new OzoneConfiguration();
     ozoneConfig.set("ozone.scm.container.size", "1MB");
-    ContainerBalancerConfiguration config = 
balancerConfigByOzoneConfig(ozoneConfig);
-    if (mockedSCM.getCluster().getNodeCount() < 
DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) {
-      config.setMaxDatanodesPercentageToInvolvePerIteration(100);
-    }
-    config.setIterations(1);
-    config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
+    int nodeCount = mockedSCM.getNodeCount();
+    ContainerBalancerConfiguration config = new 
ContainerBalancerConfigBuilder(ozoneConfig, nodeCount).build();
     // No containers should be selected when the limit is just 2 MB.
     config.setMaxSizeEnteringTarget(2 * OzoneConsts.MB);
 
@@ -147,11 +137,7 @@ public class TestContainerBalancerDatanodeNodeLimit {
     assertEquals(0, task.getSizeScheduledForMoveInLatestIteration());
 
     // Some containers should be selected when using default values.
-    ContainerBalancerConfiguration balancerConfig = 
balancerConfigByOzoneConfig(new OzoneConfiguration());
-    if (mockedSCM.getCluster().getNodeCount() < 
DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) {
-      balancerConfig.setMaxDatanodesPercentageToInvolvePerIteration(100);
-    }
-    balancerConfig.setIterations(1);
+    ContainerBalancerConfiguration balancerConfig = new 
ContainerBalancerConfigBuilder(nodeCount).build();
 
     task = mockedSCM.startBalancerTask(balancerConfig);
     // Balancer should have identified unbalanced nodes.
@@ -167,13 +153,9 @@ public class TestContainerBalancerDatanodeNodeLimit {
   public void balancerShouldObeyMaxSizeLeavingSourceLimit(@Nonnull MockedSCM 
mockedSCM) {
     OzoneConfiguration ozoneConfig = new OzoneConfiguration();
     ozoneConfig.set("ozone.scm.container.size", "1MB");
-    ContainerBalancerConfiguration config = 
balancerConfigByOzoneConfig(ozoneConfig);
-    if (mockedSCM.getCluster().getNodeCount() < 
DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) {
-      config.setMaxDatanodesPercentageToInvolvePerIteration(100);
-    }
-    config.setIterations(1);
-    config.setMaxSizeEnteringTarget(50 * STORAGE_UNIT);
-    config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
+    int nodeCount = mockedSCM.getNodeCount();
+    ContainerBalancerConfiguration config = new 
ContainerBalancerConfigBuilder(ozoneConfig, nodeCount).build();
+
     // No source containers should be selected when the limit is just 2 MB.
     config.setMaxSizeLeavingSource(2 * OzoneConsts.MB);
 
@@ -186,13 +168,9 @@ public class TestContainerBalancerDatanodeNodeLimit {
     assertEquals(0, task.getSizeScheduledForMoveInLatestIteration());
 
     // Some containers should be selected when using default values.
-    ContainerBalancerConfiguration newBalancerConfig = 
balancerConfigByOzoneConfig(new OzoneConfiguration());
-    if (mockedSCM.getCluster().getNodeCount() < 
DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) {
-      newBalancerConfig.setMaxDatanodesPercentageToInvolvePerIteration(100);
-    }
-    newBalancerConfig.setIterations(1);
+    ContainerBalancerConfiguration balancerConfig = new 
ContainerBalancerConfigBuilder(ozoneConfig, nodeCount).build();
 
-    task = mockedSCM.startBalancerTask(newBalancerConfig);
+    task = mockedSCM.startBalancerTask(balancerConfig);
     // Balancer should have identified unbalanced nodes.
     assertTrue(stillHaveUnbalancedNodes(task));
     // ContainerToSourceMap is not empty due to some containers should be 
selected
@@ -208,18 +186,10 @@ public class TestContainerBalancerDatanodeNodeLimit {
   @ParameterizedTest(name = "MockedSCM #{index}: {0}")
   @MethodSource("createMockedSCMs")
   public void 
initializeIterationShouldUpdateUnBalancedNodesWhenThresholdChanges(@Nonnull 
MockedSCM mockedSCM) {
-    ContainerBalancerConfiguration config = new 
OzoneConfiguration().getObject(ContainerBalancerConfiguration.class);
-    int nodeCount = mockedSCM.getCluster().getNodeCount();
-    if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) {
-      config.setMaxDatanodesPercentageToInvolvePerIteration(100);
-    }
-    config.setThreshold(10);
-    config.setIterations(1);
-    config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
-    config.setMaxSizeEnteringTarget(50 * STORAGE_UNIT);
+    ContainerBalancerConfiguration config = new 
ContainerBalancerConfigBuilder(mockedSCM.getNodeCount()).build();
 
     // check for random threshold values
-    for (int i = 0; i < 50; i++) {
+    for (int i = 0; i < 10; i++) {
       double randomThreshold = RANDOM.nextDouble() * 100;
       List<DatanodeUsageInfo> expectedUnBalancedNodes = 
mockedSCM.getCluster().getUnBalancedNodes(randomThreshold);
       config.setThreshold(randomThreshold);
@@ -256,15 +226,7 @@ public class TestContainerBalancerDatanodeNodeLimit {
   @MethodSource("createMockedSCMs")
   public void testBalancerWithMoveManager(@Nonnull MockedSCM mockedSCM)
       throws IOException, NodeNotFoundException, TimeoutException {
-    ContainerBalancerConfiguration config = new 
OzoneConfiguration().getObject(ContainerBalancerConfiguration.class);
-    int nodeCount = mockedSCM.getCluster().getNodeCount();
-    if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) {
-      config.setMaxDatanodesPercentageToInvolvePerIteration(100);
-    }
-    config.setThreshold(10);
-    config.setIterations(1);
-    config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
-    config.setMaxSizeEnteringTarget(50 * STORAGE_UNIT);
+    ContainerBalancerConfiguration config = new 
ContainerBalancerConfigBuilder(mockedSCM.getNodeCount()).build();
 
     mockedSCM.disableLegacyReplicationManager();
     mockedSCM.startBalancerTask(config);
@@ -279,15 +241,7 @@ public class TestContainerBalancerDatanodeNodeLimit {
   @ParameterizedTest(name = "MockedSCM #{index}: {0}")
   @MethodSource("createMockedSCMs")
   public void unBalancedNodesListShouldBeEmptyWhenClusterIsBalanced(@Nonnull 
MockedSCM mockedSCM) {
-    ContainerBalancerConfiguration config = new 
OzoneConfiguration().getObject(ContainerBalancerConfiguration.class);
-    int nodeCount = mockedSCM.getCluster().getNodeCount();
-    if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) {
-      config.setMaxDatanodesPercentageToInvolvePerIteration(100);
-    }
-    config.setThreshold(10);
-    config.setIterations(1);
-    config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
-    config.setMaxSizeEnteringTarget(50 * STORAGE_UNIT);
+    ContainerBalancerConfiguration config = new 
ContainerBalancerConfigBuilder(mockedSCM.getNodeCount()).build();
     config.setThreshold(99.99);
 
     ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
@@ -302,14 +256,8 @@ public class TestContainerBalancerDatanodeNodeLimit {
   public void testMetrics(@Nonnull MockedSCM mockedSCM) throws IOException, 
NodeNotFoundException {
     OzoneConfiguration ozoneConfig = new OzoneConfiguration();
     ozoneConfig.set("hdds.datanode.du.refresh.period", "1ms");
-    ContainerBalancerConfiguration config = 
balancerConfigByOzoneConfig(ozoneConfig);
-    int nodeCount = mockedSCM.getCluster().getNodeCount();
-    if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) {
-      config.setMaxDatanodesPercentageToInvolvePerIteration(100);
-    }
+    ContainerBalancerConfiguration config = new 
ContainerBalancerConfigBuilder(mockedSCM.getNodeCount()).build();
     config.setBalancingInterval(Duration.ofMillis(2));
-    config.setThreshold(10);
-    config.setIterations(1);
     config.setMaxSizeEnteringTarget(6 * STORAGE_UNIT);
     // deliberately set max size per iteration to a low value, 6 GB
     config.setMaxSizeToMovePerIteration(6 * STORAGE_UNIT);
@@ -338,15 +286,7 @@ public class TestContainerBalancerDatanodeNodeLimit {
   @ParameterizedTest(name = "MockedSCM #{index}: {0}")
   @MethodSource("createMockedSCMs")
   public void containerBalancerShouldSelectOnlyClosedContainers(@Nonnull 
MockedSCM mockedSCM) {
-    ContainerBalancerConfiguration config = balancerConfigByOzoneConfig(new 
OzoneConfiguration());
-    int nodeCount = mockedSCM.getCluster().getNodeCount();
-    if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) {
-      config.setMaxDatanodesPercentageToInvolvePerIteration(100);
-    }
-    config.setIterations(1);
-    config.setThreshold(10);
-    config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
-    config.setMaxSizeEnteringTarget(50 * STORAGE_UNIT);
+    ContainerBalancerConfiguration config = new 
ContainerBalancerConfigBuilder(mockedSCM.getNodeCount()).build();
 
     Map<ContainerID, ContainerInfo> cidToInfoMap = 
mockedSCM.getCluster().getCidToInfoMap();
     // Make all containers open, balancer should not select any of them
@@ -380,15 +320,7 @@ public class TestContainerBalancerDatanodeNodeLimit {
   @MethodSource("createMockedSCMs")
   public void balancerShouldNotSelectNonClosedContainerReplicas(@Nonnull 
MockedSCM mockedSCM)
       throws ContainerNotFoundException {
-    ContainerBalancerConfiguration config = balancerConfigByOzoneConfig(new 
OzoneConfiguration());
-    int nodeCount = mockedSCM.getCluster().getNodeCount();
-    if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) {
-      config.setMaxDatanodesPercentageToInvolvePerIteration(100);
-    }
-    config.setIterations(1);
-    config.setThreshold(10);
-    config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
-    config.setMaxSizeEnteringTarget(50 * STORAGE_UNIT);
+    ContainerBalancerConfiguration config = new 
ContainerBalancerConfigBuilder(mockedSCM.getNodeCount()).build();
 
     // Let's mock such that all replicas have CLOSING state
     Map<ContainerID, Set<ContainerReplica>> cidToReplicasMap = 
mockedSCM.getCluster().getCidToReplicasMap();
@@ -418,12 +350,7 @@ public class TestContainerBalancerDatanodeNodeLimit {
   @ParameterizedTest(name = "MockedSCM #{index}: {0}")
   @MethodSource("createMockedSCMs")
   public void containerBalancerShouldObeyMaxSizeToMoveLimit(@Nonnull MockedSCM 
mockedSCM) {
-    ContainerBalancerConfiguration config = balancerConfigByOzoneConfig(new 
OzoneConfiguration());
-    int nodeCount = mockedSCM.getCluster().getNodeCount();
-    if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) {
-      config.setMaxDatanodesPercentageToInvolvePerIteration(100);
-    }
-    config.setIterations(1);
+    ContainerBalancerConfiguration config = new 
ContainerBalancerConfigBuilder(mockedSCM.getNodeCount()).build();
     config.setThreshold(1);
     config.setMaxSizeToMovePerIteration(10 * STORAGE_UNIT);
     config.setMaxSizeEnteringTarget(10 * STORAGE_UNIT);
@@ -441,15 +368,7 @@ public class TestContainerBalancerDatanodeNodeLimit {
   @ParameterizedTest(name = "MockedSCM #{index}: {0}")
   @MethodSource("createMockedSCMs")
   public void targetDatanodeShouldNotAlreadyContainSelectedContainer(@Nonnull 
MockedSCM mockedSCM) {
-    ContainerBalancerConfiguration config = balancerConfigByOzoneConfig(new 
OzoneConfiguration());
-    int nodeCount = mockedSCM.getCluster().getNodeCount();
-    if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) {
-      config.setMaxDatanodesPercentageToInvolvePerIteration(100);
-    }
-    config.setIterations(1);
-    config.setThreshold(10);
-    config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
-    config.setMaxSizeEnteringTarget(50 * STORAGE_UNIT);
+    ContainerBalancerConfiguration config = new 
ContainerBalancerConfigBuilder(mockedSCM.getNodeCount()).build();
 
     ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
 
@@ -468,15 +387,7 @@ public class TestContainerBalancerDatanodeNodeLimit {
   @ParameterizedTest(name = "MockedSCM #{index}: {0}")
   @MethodSource("createMockedSCMs")
   public void containerMoveSelectionShouldFollowPlacementPolicy(@Nonnull 
MockedSCM mockedSCM) {
-    ContainerBalancerConfiguration config = balancerConfigByOzoneConfig(new 
OzoneConfiguration());
-    int nodeCount = mockedSCM.getCluster().getNodeCount();
-    if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) {
-      config.setMaxDatanodesPercentageToInvolvePerIteration(100);
-    }
-    config.setIterations(1);
-    config.setThreshold(10);
-    config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
-    config.setMaxSizeEnteringTarget(50 * STORAGE_UNIT);
+    ContainerBalancerConfiguration config = new 
ContainerBalancerConfigBuilder(mockedSCM.getNodeCount()).build();
 
     ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
 
@@ -511,15 +422,7 @@ public class TestContainerBalancerDatanodeNodeLimit {
   @ParameterizedTest(name = "MockedSCM #{index}: {0}")
   @MethodSource("createMockedSCMs")
   public void targetDatanodeShouldBeInServiceHealthy(@Nonnull MockedSCM 
mockedSCM) throws NodeNotFoundException {
-    ContainerBalancerConfiguration config = balancerConfigByOzoneConfig(new 
OzoneConfiguration());
-    int nodeCount = mockedSCM.getCluster().getNodeCount();
-    if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) {
-      config.setMaxDatanodesPercentageToInvolvePerIteration(100);
-    }
-    config.setIterations(1);
-    config.setThreshold(10);
-    config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
-    config.setMaxSizeEnteringTarget(50 * STORAGE_UNIT);
+    ContainerBalancerConfiguration config = new 
ContainerBalancerConfigBuilder(mockedSCM.getNodeCount()).build();
 
     ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
 
@@ -535,15 +438,7 @@ public class TestContainerBalancerDatanodeNodeLimit {
   @MethodSource("createMockedSCMs")
   public void selectedContainerShouldNotAlreadyHaveBeenSelected(@Nonnull 
MockedSCM mockedSCM)
       throws NodeNotFoundException, ContainerNotFoundException, 
TimeoutException, ContainerReplicaNotFoundException {
-    ContainerBalancerConfiguration config = balancerConfigByOzoneConfig(new 
OzoneConfiguration());
-    int nodeCount = mockedSCM.getCluster().getNodeCount();
-    if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) {
-      config.setMaxDatanodesPercentageToInvolvePerIteration(100);
-    }
-    config.setIterations(1);
-    config.setThreshold(10);
-    config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
-    config.setMaxSizeEnteringTarget(50 * STORAGE_UNIT);
+    ContainerBalancerConfiguration config = new 
ContainerBalancerConfigBuilder(mockedSCM.getNodeCount()).build();
 
     mockedSCM.enableLegacyReplicationManager();
 
@@ -570,15 +465,7 @@ public class TestContainerBalancerDatanodeNodeLimit {
   @ParameterizedTest(name = "MockedSCM #{index}: {0}")
   @MethodSource("createMockedSCMs")
   public void balancerShouldNotSelectConfiguredExcludeContainers(@Nonnull 
MockedSCM mockedSCM) {
-    ContainerBalancerConfiguration config = balancerConfigByOzoneConfig(new 
OzoneConfiguration());
-    int nodeCount = mockedSCM.getCluster().getNodeCount();
-    if (nodeCount < DATANODE_COUNT_LIMIT_FOR_SMALL_CLUSTER) {
-      config.setMaxDatanodesPercentageToInvolvePerIteration(100);
-    }
-    config.setIterations(1);
-    config.setThreshold(10);
-    config.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT);
-    config.setMaxSizeEnteringTarget(50 * STORAGE_UNIT);
+    ContainerBalancerConfiguration config = new 
ContainerBalancerConfigBuilder(mockedSCM.getNodeCount()).build();
     config.setExcludeContainers("1, 4, 5");
 
     ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
@@ -589,6 +476,154 @@ public class TestContainerBalancerDatanodeNodeLimit {
     }
   }
 
+  @ParameterizedTest(name = "MockedSCM #{index}: {0}")
+  @MethodSource("createMockedSCMs")
+  public void checkIterationResult(@Nonnull MockedSCM mockedSCM)
+      throws NodeNotFoundException, ContainerNotFoundException, 
TimeoutException {
+    ContainerBalancerConfiguration config = new 
ContainerBalancerConfigBuilder(mockedSCM.getNodeCount()).build();
+    config.setMaxSizeEnteringTarget(10 * STORAGE_UNIT);
+    config.setMaxSizeToMovePerIteration(100 * STORAGE_UNIT);
+
+    mockedSCM.enableLegacyReplicationManager();
+
+    ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
+    // According to the setup and configurations, this iteration's result 
should be ITERATION_COMPLETED.
+    assertEquals(ContainerBalancerTask.IterationResult.ITERATION_COMPLETED, 
task.getIterationResult());
+
+    // Now, limit maxSizeToMovePerIteration but fail all container moves.
+    // The result should still be ITERATION_COMPLETED.
+    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 * STORAGE_UNIT);
+
+    task = mockedSCM.startBalancerTask(config);
+    assertEquals(ContainerBalancerTask.IterationResult.ITERATION_COMPLETED, 
task.getIterationResult());
+
+    //Try the same but use MoveManager for container move instead of legacy RM.
+    mockedSCM.disableLegacyReplicationManager();
+    task = mockedSCM.startBalancerTask(config);
+    assertEquals(ContainerBalancerTask.IterationResult.ITERATION_COMPLETED, 
task.getIterationResult());
+  }
+
+  /**
+   * Tests the situation where some container moves time out because they take 
longer than "move.timeout".
+   */
+  @ParameterizedTest(name = "MockedSCM #{index}: {0}")
+  @MethodSource("createMockedSCMs")
+  public void checkIterationResultTimeout(@Nonnull MockedSCM mockedSCM)
+      throws NodeNotFoundException, ContainerNotFoundException, 
TimeoutException, ContainerReplicaNotFoundException {
+    ContainerBalancerConfiguration config = new 
ContainerBalancerConfigBuilder(mockedSCM.getNodeCount()).build();
+    config.setMaxSizeEnteringTarget(50 * STORAGE_UNIT);
+    config.setMaxSizeToMovePerIteration(100 * STORAGE_UNIT);
+    config.setMaxDatanodesPercentageToInvolvePerIteration(100);
+    config.setMoveTimeout(Duration.ofMillis(50));
+
+    CompletableFuture<MoveManager.MoveResult> completedFuture =
+        CompletableFuture.completedFuture(MoveManager.MoveResult.COMPLETED);
+    when(mockedSCM.getReplicationManager()
+        .move(any(ContainerID.class), any(DatanodeDetails.class), 
any(DatanodeDetails.class)))
+        .thenReturn(completedFuture)
+        .thenAnswer(invocation -> genCompletableFuture(150));
+
+    mockedSCM.enableLegacyReplicationManager();
+    ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
+
+    // According to the setup and configurations, this iteration's result 
should be ITERATION_COMPLETED.
+    assertEquals(ContainerBalancerTask.IterationResult.ITERATION_COMPLETED, 
task.getIterationResult());
+    assertEquals(1, 
task.getMetrics().getNumContainerMovesCompletedInLatestIteration());
+    
assertThat(task.getMetrics().getNumContainerMovesTimeoutInLatestIteration()).isGreaterThanOrEqualTo(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();
+    when(mockedSCM.getMoveManager()
+        .move(any(ContainerID.class), any(DatanodeDetails.class), 
any(DatanodeDetails.class)))
+        .thenReturn(completedFuture)
+        .thenAnswer(invocation -> genCompletableFuture(150));
+
+    task = mockedSCM.startBalancerTask(config);
+    assertEquals(ContainerBalancerTask.IterationResult.ITERATION_COMPLETED, 
task.getIterationResult());
+    assertEquals(1, 
task.getMetrics().getNumContainerMovesCompletedInLatestIteration());
+    
assertThat(task.getMetrics().getNumContainerMovesTimeoutInLatestIteration()).isGreaterThanOrEqualTo(1);
+  }
+
+  @ParameterizedTest(name = "MockedSCM #{index}: {0}")
+  @MethodSource("createMockedSCMs")
+  public void checkIterationResultTimeoutFromReplicationManager(@Nonnull 
MockedSCM mockedSCM)
+      throws NodeNotFoundException, ContainerNotFoundException, 
TimeoutException, ContainerReplicaNotFoundException {
+    ContainerBalancerConfiguration config = new 
ContainerBalancerConfigBuilder(mockedSCM.getNodeCount()).build();
+    config.setMaxSizeEnteringTarget(10 * STORAGE_UNIT);
+    config.setMaxSizeToMovePerIteration(100 * STORAGE_UNIT);
+    config.setMoveTimeout(Duration.ofMillis(500));
+
+    CompletableFuture<MoveManager.MoveResult> future =
+        CompletableFuture.supplyAsync(() -> 
MoveManager.MoveResult.REPLICATION_FAIL_TIME_OUT);
+    CompletableFuture<MoveManager.MoveResult> future2 =
+        CompletableFuture.supplyAsync(() -> 
MoveManager.MoveResult.DELETION_FAIL_TIME_OUT);
+    when(mockedSCM.getReplicationManager()
+        .move(any(ContainerID.class), any(DatanodeDetails.class), 
any(DatanodeDetails.class)))
+        .thenReturn(future, future2);
+
+    mockedSCM.enableLegacyReplicationManager();
+
+    ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
+    
assertThat(task.getMetrics().getNumContainerMovesTimeoutInLatestIteration()).isGreaterThan(0);
+    assertEquals(0, 
task.getMetrics().getNumContainerMovesCompletedInLatestIteration());
+
+    // Try the same test with MoveManager instead of LegacyReplicationManager.
+    when(mockedSCM.getMoveManager()
+        .move(any(ContainerID.class), any(DatanodeDetails.class), 
any(DatanodeDetails.class)))
+        .thenReturn(future).thenAnswer(invocation -> future2);
+
+    mockedSCM.disableLegacyReplicationManager();
+
+    task = mockedSCM.startBalancerTask(config);
+    
assertThat(task.getMetrics().getNumContainerMovesTimeoutInLatestIteration()).isGreaterThan(0);
+    assertEquals(0, 
task.getMetrics().getNumContainerMovesCompletedInLatestIteration());
+  }
+
+  @ParameterizedTest(name = "MockedSCM #{index}: {0}")
+  @MethodSource("createMockedSCMs")
+  public void checkIterationResultException(@Nonnull MockedSCM mockedSCM)
+      throws NodeNotFoundException, ContainerNotFoundException, 
TimeoutException, ContainerReplicaNotFoundException {
+    int nodeCount = mockedSCM.getNodeCount();
+    ContainerBalancerConfiguration config = new 
ContainerBalancerConfigBuilder(mockedSCM.getNodeCount()).build();
+    config.setMaxSizeEnteringTarget(10 * STORAGE_UNIT);
+    config.setMaxSizeToMovePerIteration(100 * STORAGE_UNIT);
+    config.setMoveTimeout(Duration.ofMillis(500));
+
+    CompletableFuture<MoveManager.MoveResult> future = new 
CompletableFuture<>();
+    future.completeExceptionally(new RuntimeException("Runtime Exception"));
+    when(mockedSCM.getReplicationManager()
+        .move(any(ContainerID.class), any(DatanodeDetails.class), 
any(DatanodeDetails.class)))
+        .thenReturn(genCompletableFutureWithException(1))
+        .thenThrow(new ContainerNotFoundException("Test Container not found"))
+        .thenReturn(future);
+
+    mockedSCM.enableLegacyReplicationManager();
+
+    ContainerBalancerTask task = mockedSCM.startBalancerTask(config);
+    assertEquals(ContainerBalancerTask.IterationResult.ITERATION_COMPLETED, 
task.getIterationResult());
+
+    int expectedMovesFailed = (nodeCount > 6) ? 3 : 1;
+    
assertThat(task.getMetrics().getNumContainerMovesFailed()).isGreaterThanOrEqualTo(expectedMovesFailed);
+
+    // Try the same test but with MoveManager instead of ReplicationManager.
+    when(mockedSCM.getMoveManager()
+        .move(any(ContainerID.class), any(DatanodeDetails.class), 
any(DatanodeDetails.class)))
+        .thenReturn(genCompletableFutureWithException(1))
+        .thenThrow(new ContainerNotFoundException("Test Container not found"))
+        .thenReturn(future);
+
+    mockedSCM.disableLegacyReplicationManager();
+    task = mockedSCM.startBalancerTask(config);
+    assertEquals(ContainerBalancerTask.IterationResult.ITERATION_COMPLETED, 
task.getIterationResult());
+    
assertThat(task.getMetrics().getNumContainerMovesFailed()).isGreaterThanOrEqualTo(expectedMovesFailed);
+  }
+
   public static List<DatanodeUsageInfo> getUnBalancedNodes(@Nonnull 
ContainerBalancerTask task) {
     ArrayList<DatanodeUsageInfo> result = new ArrayList<>();
     result.addAll(task.getOverUtilizedNodes());
@@ -604,9 +639,24 @@ public class TestContainerBalancerDatanodeNodeLimit {
     return new MockedSCM(new TestableCluster(datanodeCount, STORAGE_UNIT));
   }
 
-  private static @Nonnull ContainerBalancerConfiguration 
balancerConfigByOzoneConfig(
-      @Nonnull OzoneConfiguration ozoneConfiguration
-  ) {
-    return ozoneConfiguration.getObject(ContainerBalancerConfiguration.class);
+  private static CompletableFuture<MoveManager.MoveResult> 
genCompletableFuture(int sleepMilSec) {
+    return CompletableFuture.supplyAsync(() -> {
+      try {
+        Thread.sleep(sleepMilSec);
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
+      return MoveManager.MoveResult.COMPLETED;
+    });
+  }
+
+  private static CompletableFuture<MoveManager.MoveResult> 
genCompletableFutureWithException(int sleepMilSec) {
+    return CompletableFuture.supplyAsync(() -> {
+      try {
+        Thread.sleep(sleepMilSec);
+      } catch (Exception ignored) {
+      }
+      throw new RuntimeException("Runtime Exception after doing work");
+    });
   }
 }
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 d0e9cd53fe..e689e8d114 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
@@ -32,7 +32,6 @@ 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;
@@ -60,7 +59,6 @@ import org.slf4j.event.Level;
 
 import java.io.IOException;
 import java.time.Clock;
-import java.time.Duration;
 import java.time.ZoneId;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -73,16 +71,16 @@ import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
+import static 
org.apache.hadoop.hdds.scm.container.replication.ReplicationManager.ReplicationManagerConfiguration;
+import static org.assertj.core.api.Assertions.assertThat;
 import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotSame;
 import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static 
org.apache.hadoop.hdds.scm.container.replication.ReplicationManager.ReplicationManagerConfiguration;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.mockito.Mockito.doNothing;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.anyString;
 import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.anyString;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -118,7 +116,7 @@ public class TestContainerBalancerTask {
   private static final ThreadLocalRandom RANDOM = ThreadLocalRandom.current();
 
   private StatefulServiceStateManager serviceStateManager;
-  private static final long STORAGE_UNIT = OzoneConsts.GB;
+  static final long STORAGE_UNIT = OzoneConsts.GB;
 
   /**
    * Sets up configuration values and creates a mock cluster.
@@ -337,229 +335,6 @@ public class TestContainerBalancerTask {
         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.
-     */
-    assertEquals(
-        ContainerBalancerTask.IterationResult.ITERATION_COMPLETED,
-        containerBalancerTask.getIterationResult());
-    stopBalancer();
-
-    /*
-    Now, limit maxSizeToMovePerIteration but fail all container moves. The
-    result should still be ITERATION_COMPLETED.
-     */
-    when(replicationManager.move(any(ContainerID.class),
-            any(DatanodeDetails.class),
-            any(DatanodeDetails.class)))
-        .thenReturn(CompletableFuture.completedFuture(
-            MoveManager.MoveResult.REPLICATION_FAIL_NODE_UNHEALTHY));
-    balancerConfiguration.setMaxSizeToMovePerIteration(10 * STORAGE_UNIT);
-
-    startBalancer(balancerConfiguration);
-
-    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);
-    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<MoveManager.MoveResult> completedFuture =
-        CompletableFuture.completedFuture(MoveManager.MoveResult.COMPLETED);
-    when(replicationManager.move(any(ContainerID.class),
-            any(DatanodeDetails.class),
-            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.
-     */
-    assertEquals(ContainerBalancerTask.IterationResult.ITERATION_COMPLETED,
-        containerBalancerTask.getIterationResult());
-    assertEquals(1, 
containerBalancerTask.getMetrics().getNumContainerMovesCompletedInLatestIteration());
-    assertThat(containerBalancerTask.getMetrics()
-            .getNumContainerMovesTimeoutInLatestIteration()).isGreaterThan(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);
-    when(moveManager.move(any(ContainerID.class),
-            any(DatanodeDetails.class),
-            any(DatanodeDetails.class)))
-        .thenReturn(completedFuture)
-        .thenAnswer(invocation -> genCompletableFuture(2000));
-
-    startBalancer(balancerConfiguration);
-    assertEquals(ContainerBalancerTask.IterationResult.ITERATION_COMPLETED,
-        containerBalancerTask.getIterationResult());
-    assertEquals(1, 
containerBalancerTask.getMetrics().getNumContainerMovesCompletedInLatestIteration());
-    assertThat(containerBalancerTask.getMetrics()
-        .getNumContainerMovesTimeoutInLatestIteration()).isGreaterThan(1);
-    stopBalancer();
-  }
-
-  @Test
-  public void checkIterationResultTimeoutFromReplicationManager()
-      throws NodeNotFoundException, IOException,
-      IllegalContainerBalancerStateException,
-      InvalidContainerBalancerConfigurationException, TimeoutException {
-    CompletableFuture<MoveManager.MoveResult> future
-        = CompletableFuture.supplyAsync(() ->
-        MoveManager.MoveResult.REPLICATION_FAIL_TIME_OUT);
-    CompletableFuture<MoveManager.MoveResult> future2
-        = CompletableFuture.supplyAsync(() ->
-        MoveManager.MoveResult.DELETION_FAIL_TIME_OUT);
-    when(replicationManager.move(any(ContainerID.class),
-            any(DatanodeDetails.class),
-            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);
-
-    assertThat(containerBalancerTask.getMetrics()
-        .getNumContainerMovesTimeoutInLatestIteration()).isGreaterThan(0);
-    assertEquals(0, 
containerBalancerTask.getMetrics().getNumContainerMovesCompletedInLatestIteration());
-    stopBalancer();
-
-    /*
-    Try the same test with MoveManager instead of LegacyReplicationManager.
-     */
-    when(moveManager.move(any(ContainerID.class),
-            any(DatanodeDetails.class),
-            any(DatanodeDetails.class)))
-        .thenReturn(future).thenAnswer(invocation -> future2);
-
-    rmConf.setEnableLegacy(false);
-    startBalancer(balancerConfiguration);
-    assertThat(containerBalancerTask.getMetrics()
-        .getNumContainerMovesTimeoutInLatestIteration()).isGreaterThan(0);
-    assertEquals(0, 
containerBalancerTask.getMetrics().getNumContainerMovesCompletedInLatestIteration());
-    stopBalancer();
-  }
-
-  @Test
-  public void checkIterationResultException()
-      throws NodeNotFoundException, IOException,
-      IllegalContainerBalancerStateException,
-      InvalidContainerBalancerConfigurationException,
-      TimeoutException {
-
-    CompletableFuture<MoveManager.MoveResult> future =
-        new CompletableFuture<>();
-    future.completeExceptionally(new RuntimeException("Runtime Exception"));
-    when(replicationManager.move(any(ContainerID.class),
-            any(DatanodeDetails.class),
-            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);
-
-    assertEquals(ContainerBalancerTask.IterationResult.ITERATION_COMPLETED,
-        containerBalancerTask.getIterationResult());
-    assertThat(containerBalancerTask.getMetrics().getNumContainerMovesFailed())
-        .isGreaterThanOrEqualTo(3);
-    stopBalancer();
-
-    /*
-    Try the same test but with MoveManager instead of ReplicationManager.
-     */
-    when(moveManager.move(any(ContainerID.class),
-            any(DatanodeDetails.class),
-            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);
-    assertEquals(ContainerBalancerTask.IterationResult.ITERATION_COMPLETED,
-        containerBalancerTask.getIterationResult());
-    assertThat(containerBalancerTask.getMetrics().getNumContainerMovesFailed())
-        .isGreaterThanOrEqualTo(3);
-    stopBalancer();
-  }
-
   @Test
   public void testDelayedStart() throws InterruptedException, TimeoutException 
{
     conf.setTimeDuration("hdds.scm.wait.time.after.safemode.exit", 10,


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


Reply via email to