This is an automated email from the ASF dual-hosted git repository.
siddhant 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 34abace0b41 HDDS-13694. Container Balancer Stop Command Fails with
Error as Already Stopped (#9047)
34abace0b41 is described below
commit 34abace0b417835c0da56133c3942b373153a439
Author: Sarveksha Yeshavantha Raju
<[email protected]>
AuthorDate: Thu Sep 25 17:28:48 2025 +0530
HDDS-13694. Container Balancer Stop Command Fails with Error as Already
Stopped (#9047)
---
.../scm/container/balancer/ContainerBalancer.java | 38 +++++++++++++-------
.../container/balancer/TestContainerBalancer.java | 9 +++--
.../ozone/TestContainerBalancerOperations.java | 42 ++++++++++++++++++++++
3 files changed, 74 insertions(+), 15 deletions(-)
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 6a35b7245ca..07319471dcc 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
@@ -308,16 +308,13 @@ private void startBalancingThread(int nextIterationIndex,
}
/**
- * Validates balancer's state based on the specified expectedRunning.
+ * Validates balancer's eligibility based on SCM state.
* 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
+ * leader-ready or is in safe mode
*/
- private void validateState(boolean expectedRunning)
+ private void validateEligibility()
throws IllegalContainerBalancerStateException {
if (!scmContext.isLeaderReady()) {
LOG.warn("SCM is not leader ready");
@@ -328,6 +325,19 @@ private void validateState(boolean expectedRunning)
LOG.warn("SCM is in safe mode");
throw new IllegalContainerBalancerStateException("SCM is in safe mode");
}
+ }
+
+ /**
+ * Validates balancer's state based on the specified expectedRunning.
+ *
+ * @param expectedRunning true if ContainerBalancer is expected to be
+ * running, else false
+ * @throws IllegalContainerBalancerStateException if state does not
+ * match the specified expected state
+ */
+ private void validateState(boolean expectedRunning)
+ throws IllegalContainerBalancerStateException {
+ validateEligibility();
if (!expectedRunning && !canBalancerStart()) {
throw new IllegalContainerBalancerStateException(
"Expect ContainerBalancer as not running state" +
@@ -387,18 +397,22 @@ private static void blockTillTaskStop(Thread
balancingThread) {
*/
public void stopBalancer()
throws IOException, IllegalContainerBalancerStateException {
- Thread balancingThread;
+ Thread balancingThread = null;
lock.lock();
try {
- validateState(true);
+ validateEligibility();
saveConfiguration(config, false, 0);
- LOG.info("Trying to stop ContainerBalancer service.");
- task.stop();
- balancingThread = currentBalancingThread;
+ if (isBalancerRunning()) {
+ LOG.info("Trying to stop ContainerBalancer service.");
+ task.stop();
+ balancingThread = currentBalancingThread;
+ }
} finally {
lock.unlock();
}
- blockTillTaskStop(balancingThread);
+ if (balancingThread != null) {
+ blockTillTaskStop(balancingThread);
+ }
}
public void saveConfiguration(ContainerBalancerConfiguration configuration,
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 4bd4daeab99..33ed8f0e83c 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
@@ -20,6 +20,7 @@
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.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertSame;
@@ -128,6 +129,8 @@ public void testShouldRun() throws Exception {
@Test
public void testStartBalancerStop() throws Exception {
+ //stop should not throw an exception as it is idempotent
+ assertDoesNotThrow(() -> containerBalancer.stopBalancer());
startBalancer(balancerConfiguration);
assertThrows(IllegalContainerBalancerStateException.class,
() -> containerBalancer.startBalancer(balancerConfiguration),
@@ -142,9 +145,9 @@ public void testStartBalancerStop() throws Exception {
stopBalancer();
assertSame(ContainerBalancerTask.Status.STOPPED,
containerBalancer.getBalancerStatus());
- assertThrows(Exception.class,
- () -> containerBalancer.stopBalancer(),
- "Exception should be thrown when stop again");
+ // If the balancer is already stopped, the stop command should do nothing
+ // and return successfully as stopBalancer is idempotent
+ assertDoesNotThrow(() -> containerBalancer.stopBalancer());
}
@Test
diff --git
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerBalancerOperations.java
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerBalancerOperations.java
index 37a49040394..a2a534f839d 100644
---
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerBalancerOperations.java
+++
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerBalancerOperations.java
@@ -19,10 +19,12 @@
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_NODE_REPORT_INTERVAL;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
+import java.io.IOException;
import java.util.Optional;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.scm.PlacementPolicy;
@@ -178,4 +180,44 @@ public void testIfCBCLIOverridesConfigs() throws Exception
{
running = containerBalancerClient.getContainerBalancerStatus();
assertFalse(running);
}
+
+ /**
+ * Tests that stopBalancer is idempotent - once the balancer is in STOPPED
state,
+ * invoking stop again should be a no-op and return successfully with exit
code 0.
+ */
+ @Test
+ public void testStopBalancerIdempotent() throws IOException {
+ boolean running = containerBalancerClient.getContainerBalancerStatus();
+ assertFalse(running);
+ assertDoesNotThrow(() -> containerBalancerClient.stopContainerBalancer());
+
+ Optional<Double> threshold = Optional.of(0.1);
+ Optional<Integer> iterations = Optional.of(10000);
+ Optional<Integer> maxDatanodesPercentageToInvolvePerIteration =
+ Optional.of(100);
+ Optional<Long> maxSizeToMovePerIterationInGB = Optional.of(1L);
+ Optional<Long> maxSizeEnteringTargetInGB = Optional.of(6L);
+ Optional<Long> maxSizeLeavingSourceInGB = Optional.of(6L);
+ Optional<Integer> balancingInterval = Optional.of(70);
+ Optional<Integer> moveTimeout = Optional.of(65);
+ Optional<Integer> moveReplicationTimeout = Optional.of(50);
+ Optional<Boolean> networkTopologyEnable = Optional.of(false);
+ Optional<String> includeNodes = Optional.of("");
+ Optional<String> excludeNodes = Optional.of("");
+ containerBalancerClient.startContainerBalancer(threshold, iterations,
+ maxDatanodesPercentageToInvolvePerIteration,
+ maxSizeToMovePerIterationInGB, maxSizeEnteringTargetInGB,
+ maxSizeLeavingSourceInGB, balancingInterval, moveTimeout,
+ moveReplicationTimeout, networkTopologyEnable, includeNodes,
+ excludeNodes);
+ running = containerBalancerClient.getContainerBalancerStatus();
+ assertTrue(running);
+
+ containerBalancerClient.stopContainerBalancer();
+ running = containerBalancerClient.getContainerBalancerStatus();
+ assertFalse(running);
+
+ // Calling stop balancer again should not throw an exception
+ assertDoesNotThrow(() -> containerBalancerClient.stopContainerBalancer());
+ }
}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]