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

erose 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 47923be824f HDDS-14012. SCM needs to log safemode exit rules at 
regular intervals (#9376)
47923be824f is described below

commit 47923be824f6fa6aee455fe20badf1eacef3b8ba
Author: sreejasahithi <[email protected]>
AuthorDate: Tue Jan 13 21:47:09 2026 +0530

    HDDS-14012. SCM needs to log safemode exit rules at regular intervals 
(#9376)
---
 .../org/apache/hadoop/hdds/HddsConfigKeys.java     |   4 +
 .../common/src/main/resources/ozone-default.xml    |  10 ++
 .../safemode/AbstractContainerSafeModeRule.java    |   4 +-
 .../hdds/scm/safemode/SCMSafeModeManager.java      | 119 +++++++++++++++++
 .../hdds/scm/server/StorageContainerManager.java   |  17 ++-
 .../hdds/scm/safemode/TestSCMSafeModeManager.java  | 143 +++++++++++++++++++++
 .../ozone/reconfig/TestScmReconfiguration.java     |  15 +++
 7 files changed, 309 insertions(+), 3 deletions(-)

diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
index 98a26e7212b..0473d6da36a 100644
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
+++ 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
@@ -112,6 +112,10 @@ public final class HddsConfigKeys {
   public static final double
       HDDS_SCM_SAFEMODE_ONE_NODE_REPORTED_PIPELINE_PCT_DEFAULT = 0.90;
 
+  public static final String HDDS_SCM_SAFEMODE_LOG_INTERVAL =
+      "hdds.scm.safemode.log.interval";
+  public static final String HDDS_SCM_SAFEMODE_LOG_INTERVAL_DEFAULT = "1m";
+
   // This configuration setting is used as a fallback location by all
   // Ozone/HDDS services for their metadata. It is useful as a single
   // config point for test/PoC clusters.
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml 
b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index 3b169c331fe..b143b8786f4 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -1749,6 +1749,16 @@
     </description>
   </property>
 
+  <property>
+    <name>hdds.scm.safemode.log.interval</name>
+    <value>1m</value>
+    <tag>HDDS,SCM,OPERATION</tag>
+    <description>
+      Interval at which SCM logs safemode status while SCM is in safemode.
+      Default is 1 minute.
+    </description>
+  </property>
+
   <property>
     <name>hdds.container.action.max.limit</name>
     <value>20</value>
diff --git 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/AbstractContainerSafeModeRule.java
 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/AbstractContainerSafeModeRule.java
index d81816a4af6..09480009455 100644
--- 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/AbstractContainerSafeModeRule.java
+++ 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/AbstractContainerSafeModeRule.java
@@ -175,7 +175,7 @@ protected int getMinReplica(ContainerID id) {
   public String getStatusText() {
     String status = String.format("%1.2f%% of [" + getContainerType() + "] " +
             "Containers(%s / %s) with at least N reported replica (=%1.2f) >= 
" +
-            "safeModeCutoff (=%1.2f);",
+            "safeModeCutoff (=%1.2f)",
         getCurrentContainerThreshold() * 100,
         getNumberOfContainersWithMinReplica(), getTotalNumberOfContainers(),
         getCurrentContainerThreshold(), getSafeModeCutoff());
@@ -186,7 +186,7 @@ public String getStatusText() {
 
     if (!sampleContainers.isEmpty()) {
       String sampleECContainerText = "Sample  " + getContainerType() + " 
Containers not satisfying the criteria : "
-          + sampleContainers + ";";
+          + sampleContainers;
       status = status.concat("\n").concat(sampleECContainerText);
     }
 
diff --git 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SCMSafeModeManager.java
 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SCMSafeModeManager.java
index 67d47d101df..2c9173b2bf0 100644
--- 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SCMSafeModeManager.java
+++ 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SCMSafeModeManager.java
@@ -20,12 +20,18 @@
 import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_SCM_SAFEMODE_ENABLED;
 import static 
org.apache.hadoop.hdds.HddsConfigKeys.HDDS_SCM_SAFEMODE_ENABLED_DEFAULT;
 
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.scm.container.ContainerManager;
 import org.apache.hadoop.hdds.scm.ha.SCMContext;
@@ -79,6 +85,10 @@ public class SCMSafeModeManager implements SafeModeManager {
   private final SCMContext scmContext;
   private final SafeModeMetrics safeModeMetrics;
 
+  private long safeModeLogIntervalMs;
+  private ScheduledExecutorService safeModeLogExecutor;
+  private ScheduledFuture<?> safeModeLogTask;
+
   public SCMSafeModeManager(final ConfigurationSource conf,
                             final NodeManager nodeManager,
                             final PipelineManager pipelineManager,
@@ -89,6 +99,10 @@ public SCMSafeModeManager(final ConfigurationSource conf,
     this.serviceManager = serviceManager;
     this.scmContext = scmContext;
     this.safeModeMetrics = SafeModeMetrics.create();
+    this.safeModeLogIntervalMs = conf.getTimeDuration(
+        HddsConfigKeys.HDDS_SCM_SAFEMODE_LOG_INTERVAL,
+        HddsConfigKeys.HDDS_SCM_SAFEMODE_LOG_INTERVAL_DEFAULT,
+        TimeUnit.MILLISECONDS);
 
     SafeModeRuleFactory.initialize(conf, scmContext, eventQueue,
         pipelineManager, containerManager, nodeManager);
@@ -107,9 +121,11 @@ public SCMSafeModeManager(final ConfigurationSource conf,
 
   public void start() {
     emitSafeModeStatus();
+    startSafeModePeriodicLogger();
   }
 
   public void stop() {
+    stopSafeModePeriodicLogger();
     safeModeMetrics.unRegister();
   }
 
@@ -121,9 +137,11 @@ private void emitSafeModeStatus() {
     final SafeModeStatus safeModeStatus = status.get();
     safeModeMetrics.setScmInSafeMode(safeModeStatus.isInSafeMode());
     scmContext.updateSafeModeStatus(safeModeStatus);
+    logSafeModeStatus();
 
     // notify SCMServiceManager
     if (!safeModeStatus.isInSafeMode()) {
+      stopSafeModePeriodicLogger();
       // If safemode is off, then notify the delayed listeners with a delay.
       serviceManager.notifyStatusChanged();
     } else if (safeModeStatus.isPreCheckComplete()) {
@@ -226,6 +244,107 @@ public double getCurrentContainerThreshold() {
         .getCurrentContainerThreshold();
   }
 
+  /**
+   * Starts periodic logging of safe mode status.
+   * Logs are generated at the interval specified by
+   * {@link HddsConfigKeys#HDDS_SCM_SAFEMODE_LOG_INTERVAL}.
+   */
+  private synchronized void startSafeModePeriodicLogger() {
+    if (!getInSafeMode()) {
+      return;
+    }
+    if (safeModeLogExecutor == null) {
+      safeModeLogExecutor = Executors.newScheduledThreadPool(1,
+          new ThreadFactoryBuilder()
+              .setNameFormat(scmContext.threadNamePrefix() + 
"SCM-SafeMode-Log-%d")
+              .setDaemon(true)
+              .build());
+    }
+
+    if (safeModeLogTask != null && !safeModeLogTask.isDone()) {
+      safeModeLogTask.cancel(false);
+    }
+    safeModeLogTask = safeModeLogExecutor.scheduleAtFixedRate(() -> {
+      try {
+        logSafeModeStatus();
+      } catch (Throwable t) {
+        LOG.warn("Safe mode periodic logger encountered an error", t);
+      }
+    }, safeModeLogIntervalMs, safeModeLogIntervalMs, TimeUnit.MILLISECONDS);
+    LOG.info("Started periodic Safe Mode logging with interval {} ms", 
safeModeLogIntervalMs);
+  }
+
+  /**
+   * Logs the safe mode status.
+   * Includes state, preCheckComplete, validatedPreCheckRules count,
+   * validatedRules count, and individual rule statuses.
+   */
+  private synchronized void logSafeModeStatus() {
+    SafeModeStatus safeModeStatus = status.get();
+    int validatedCount = validatedRules.size();
+    int preCheckValidatedCount = validatedPreCheckRules.size();
+    StringBuilder statusLog = new StringBuilder();
+    
+    statusLog.append("\nSCM SafeMode Status | 
state=").append(safeModeStatus.name())
+        .append(" 
preCheckComplete=").append(safeModeStatus.isPreCheckComplete())
+        .append(" validatedPreCheckRules=").append(preCheckValidatedCount)
+        .append('/').append(preCheckRules.size())
+        .append(" validatedRules=").append(validatedCount)
+        .append('/').append(exitRules.size());
+    
+    for (SafeModeExitRule<?> rule : exitRules.values()) {
+      String name = rule.getRuleName();
+      boolean isValidated = validatedRules.contains(name);
+      String statusText = rule.getStatusText();
+      statusLog.append(String.format("%nSCM SafeMode Status | %s (%s) %s",
+          name,
+          isValidated ? "validated" : "waiting",
+          statusText));
+    }
+
+    LOG.info(statusLog.toString());
+    if (!getInSafeMode()) {
+      stopSafeModePeriodicLogger();
+    }
+  }
+
+  /**
+   * Stops the periodic safe mode logger.
+   * Called when safe mode exits.
+   */
+  private synchronized void stopSafeModePeriodicLogger() {
+    if (safeModeLogExecutor != null) {
+      safeModeLogExecutor.shutdownNow();
+      safeModeLogExecutor = null;
+      LOG.info("Stopped periodic Safe Mode logging");
+    }
+  }
+
+  /**
+   * Updates the Safe Mode logging interval dynamically.
+   * This method cancels the existing periodic logging task (if any) and
+   * schedules a new one with the updated interval, without recreating the
+   * executor thread pool.
+   *
+   * @param newInterval The new interval duration
+   * @param unit The time unit of the new interval
+   */
+  public synchronized void reconfigureLogInterval(long newInterval, TimeUnit 
unit) {
+    long newIntervalMs = unit.toMillis(newInterval);
+    if (this.safeModeLogIntervalMs == newIntervalMs) {
+      return;
+    }
+
+    LOG.info("Reconfiguring Safe Mode Log Interval from {} ms to {} ms",
+        this.safeModeLogIntervalMs, newIntervalMs);
+
+    this.safeModeLogIntervalMs = newIntervalMs;
+    
+    if (getInSafeMode()) {
+      startSafeModePeriodicLogger();
+    }
+  }
+  
   /**
    * Possible states of SCM SafeMode.
    */
diff --git 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
index 58507ac2171..bb6d59e0d05 100644
--- 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
+++ 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
@@ -402,7 +402,9 @@ private StorageContainerManager(OzoneConfiguration conf,
         new ReconfigurationHandler("SCM", conf, this::checkAdminAccess)
             .register(OZONE_ADMINISTRATORS, this::reconfOzoneAdmins)
             .register(OZONE_READONLY_ADMINISTRATORS,
-                this::reconfOzoneReadOnlyAdmins);
+                this::reconfOzoneReadOnlyAdmins)
+            .register(HddsConfigKeys.HDDS_SCM_SAFEMODE_LOG_INTERVAL,
+                this::reconfigureSafeModeLogInterval);
 
     
reconfigurationHandler.setReconfigurationCompleteCallback(reconfigurationHandler.defaultLoggingCallback());
 
@@ -2207,6 +2209,19 @@ private String reconfOzoneReadOnlyAdmins(String newVal) {
     return String.valueOf(newVal);
   }
 
+  private String reconfigureSafeModeLogInterval(String newLogInterval) {
+    getConfiguration().set(HddsConfigKeys.HDDS_SCM_SAFEMODE_LOG_INTERVAL, 
newLogInterval);
+    long newIntervalMs = getConfiguration().getTimeDuration(
+        HddsConfigKeys.HDDS_SCM_SAFEMODE_LOG_INTERVAL,
+        HddsConfigKeys.HDDS_SCM_SAFEMODE_LOG_INTERVAL_DEFAULT,
+        TimeUnit.MILLISECONDS);
+    
+    scmSafeModeManager.reconfigureLogInterval(newIntervalMs, 
TimeUnit.MILLISECONDS);
+
+    LOG.info("Reconfigured {} to {}", 
HddsConfigKeys.HDDS_SCM_SAFEMODE_LOG_INTERVAL, newLogInterval);
+    return newLogInterval;
+  }
+  
   /**
    * This will remove the given SCM node from HA Ring by removing it from
    * Ratis Ring.
diff --git 
a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java
 
b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java
index 1ef531f8bf8..c05c66b3cc1 100644
--- 
a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java
+++ 
b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java
@@ -814,4 +814,147 @@ public void testPipelinesNotCreatedUntilPreCheckPasses() 
throws Exception {
             scmSafeModeManager.getSafeModeMetrics().getScmInSafeMode().value() 
== 0,
         100, 1000 * 5);
   }
+
+  /**
+   * Test that each safemode rule's getStatusText is being logged periodically
+   * while SCM is in safe mode, and that the logger stops with a final summary
+   * when safe mode is force-exited.
+   */
+  @Test
+  public void testSafeModePeriodicLoggingStopsOnForceExit() throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration(config);
+    conf.set(HddsConfigKeys.HDDS_SCM_SAFEMODE_LOG_INTERVAL, "500ms");
+    conf.setInt(HddsConfigKeys.HDDS_SCM_SAFEMODE_MIN_DATANODE, 3);
+
+    containers = new ArrayList<>(HddsTestUtils.getContainerInfo(50));
+    for (ContainerInfo container : containers) {
+      container.setState(HddsProtos.LifeCycleState.CLOSED);
+      container.setNumberOfKeys(10);
+    }
+
+    MockNodeManager mockNodeManager = new MockNodeManager(true, 5);
+    PipelineManager pipelineManager = mock(PipelineManager.class);
+    ContainerManager containerManager = mock(ContainerManager.class);
+    
when(containerManager.getContainers(ReplicationType.RATIS)).thenReturn(containers);
+
+    GenericTestUtils.LogCapturer logCapturer =
+        
GenericTestUtils.LogCapturer.captureLogs(SCMSafeModeManager.getLogger());
+
+    scmSafeModeManager = new SCMSafeModeManager(conf, mockNodeManager, 
pipelineManager,
+        containerManager, serviceManager, queue, scmContext);
+    scmSafeModeManager.start();
+    assertTrue(scmSafeModeManager.getInSafeMode());
+
+    try {
+      // Verify periodic logging is active while in safe mode
+      verifyPeriodicLoggingActive(logCapturer);
+
+      logCapturer.clearOutput();
+      scmSafeModeManager.forceExitSafeMode();
+      assertFalse(scmSafeModeManager.getInSafeMode());
+
+      // Verify final summary was logged with OUT_OF_SAFE_MODE state
+      String exitLog = logCapturer.getOutput();
+      assertThat(exitLog).contains("SCM SafeMode Status | 
state=OUT_OF_SAFE_MODE");
+      assertThat(exitLog).contains("Stopped periodic Safe Mode logging");
+      assertThat(exitLog).contains("SCM force-exiting safe mode");
+
+    } finally {
+      logCapturer.stopCapturing();
+    }
+  }
+
+  /**
+   * Test that each safemode rule's getStatusText is being logged periodically
+   * while SCM is in safe mode, and that the logger stops with a final summary
+   * when safe mode is exited normally.
+   */
+  @Test
+  public void testSafeModePeriodicLoggingStopsOnNormalExit() throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration(config);
+    conf.set(HddsConfigKeys.HDDS_SCM_SAFEMODE_LOG_INTERVAL, "500ms");
+    conf.setInt(HddsConfigKeys.HDDS_SCM_SAFEMODE_MIN_DATANODE, 1);
+
+    MockNodeManager mockNodeManager = new MockNodeManager(true, 3);
+    ContainerManager containerManager = mock(ContainerManager.class);
+    when(containerManager.getContainers(ReplicationType.RATIS)).thenReturn(new 
ArrayList<>());
+    
+    PipelineManagerImpl pipelineManager = 
PipelineManagerImpl.newPipelineManager(
+        conf,
+        SCMHAManagerStub.getInstance(true),
+        mockNodeManager,
+        scmMetadataStore.getPipelineTable(),
+        queue,
+        scmContext,
+        serviceManager,
+        Clock.system(ZoneOffset.UTC));
+
+    PipelineProvider<RatisReplicationConfig> mockRatisProvider =
+        new MockRatisPipelineProvider(mockNodeManager,
+            pipelineManager.getStateManager(), conf);
+    pipelineManager.setPipelineProvider(HddsProtos.ReplicationType.RATIS,
+        mockRatisProvider);
+    pipelineManager.getBackgroundPipelineCreator().stop();
+
+    GenericTestUtils.LogCapturer logCapturer =
+        
GenericTestUtils.LogCapturer.captureLogs(SCMSafeModeManager.getLogger());
+
+    scmSafeModeManager = new SCMSafeModeManager(conf, mockNodeManager, 
pipelineManager,
+        containerManager, serviceManager, queue, scmContext);
+    scmSafeModeManager.start();
+    assertTrue(scmSafeModeManager.getInSafeMode());
+
+    try {
+      // Verify periodic logging is active while in safe mode
+      verifyPeriodicLoggingActive(logCapturer);
+      logCapturer.clearOutput();
+      
+      SCMDatanodeProtocolServer.NodeRegistrationContainerReport nodeReport =
+          HddsTestUtils.createNodeRegistrationContainerReport(new 
ArrayList<>());
+      queue.fireEvent(SCMEvents.NODE_REGISTRATION_CONT_REPORT, nodeReport);
+      queue.fireEvent(SCMEvents.CONTAINER_REGISTRATION_REPORT, nodeReport);
+      queue.processAll(5000);
+      GenericTestUtils.waitFor(() -> scmSafeModeManager.getPreCheckComplete(), 
100, 5000);
+      assertThat(logCapturer.getOutput()).contains("SCM SafeMode Status | 
state=PRE_CHECKS_PASSED");
+      
+      Pipeline pipeline = pipelineManager.createPipeline(
+          RatisReplicationConfig.getInstance(ReplicationFactor.THREE));
+      pipeline = pipelineManager.getPipeline(pipeline.getId());
+      MockRatisPipelineProvider.markPipelineHealthy(pipeline);
+      firePipelineEvent(pipelineManager, pipeline);
+      queue.processAll(5000);
+      
+      GenericTestUtils.waitFor(() -> !scmSafeModeManager.getInSafeMode(), 100, 
5000);
+
+      // Verify final summary was logged with OUT_OF_SAFE_MODE state
+      String exitLog = logCapturer.getOutput();
+      assertThat(exitLog).contains("SCM SafeMode Status | 
state=OUT_OF_SAFE_MODE");
+      assertThat(exitLog).contains("Stopped periodic Safe Mode logging");
+      assertThat(exitLog).contains("SCM exiting safe mode");
+      assertFalse(scmSafeModeManager.getInSafeMode());
+      
+    } finally {
+      logCapturer.stopCapturing();
+    }
+  }
+  
+  /**
+   * Verifies that periodic safe mode logging is working.
+   * Checks that status messages are being logged at the configured interval.
+   */
+  private void verifyPeriodicLoggingActive(GenericTestUtils.LogCapturer 
logCapturer)
+      throws InterruptedException {
+    Map<String, Pair<Boolean, String>> ruleStatuses = 
scmSafeModeManager.getRuleStatus();
+    for (int i = 0; i < 2; i++) {
+      logCapturer.clearOutput();
+      // Wait for configured interval (500ms + small buffer) for next log 
message
+      Thread.sleep(600);
+      String logOutput = logCapturer.getOutput();
+
+      assertThat(logOutput).contains("SCM SafeMode Status | state=");
+      for (String ruleName : ruleStatuses.keySet()) {
+        assertThat(logOutput).contains("SCM SafeMode Status | " + ruleName);
+      }
+    }
+  }
 }
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/reconfig/TestScmReconfiguration.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/reconfig/TestScmReconfiguration.java
index 8bbd5d4db18..751e602f167 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/reconfig/TestScmReconfiguration.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/reconfig/TestScmReconfiguration.java
@@ -26,6 +26,7 @@
 import java.util.Set;
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.hadoop.conf.ReconfigurationException;
+import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.conf.ReconfigurationHandler;
 import org.apache.hadoop.hdds.scm.ScmConfig;
 import org.apache.hadoop.hdds.scm.block.SCMBlockDeletingService;
@@ -49,6 +50,7 @@ void reconfigurableProperties() {
     Set<String> expected = ImmutableSet.<String>builder()
         .add(OZONE_ADMINISTRATORS)
         .add(OZONE_READONLY_ADMINISTRATORS)
+        .add(HddsConfigKeys.HDDS_SCM_SAFEMODE_LOG_INTERVAL)
         .addAll(new ReplicationManagerConfiguration()
             .reconfigurableProperties())
         .addAll(new WritableECContainerProviderConfig()
@@ -125,4 +127,17 @@ void blockDeletionPerInterval() throws 
ReconfigurationException {
     assertEquals(newValue, blockDeletingService.getBlockDeleteTXNum());
   }
 
+  @Test
+  void safeModeLogInterval() throws ReconfigurationException {
+
+    getSubject().reconfigurePropertyImpl(
+        HddsConfigKeys.HDDS_SCM_SAFEMODE_LOG_INTERVAL, "30s");
+
+    assertEquals(
+        "30s",
+        cluster().getStorageContainerManager()
+            .getConfiguration()
+            .get(HddsConfigKeys.HDDS_SCM_SAFEMODE_LOG_INTERVAL));
+  }
+
 }


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

Reply via email to