diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ContainerSafeModeRule.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ContainerSafeModeRule.java index 14091b23222..496d481b611 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ContainerSafeModeRule.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ContainerSafeModeRule.java @@ -76,6 +76,8 @@ public class ContainerSafeModeRule extends maxContainer = containerMap.size(); } + long cutOff = (long) Math.ceil(maxContainer * safeModeCutoff); + getSafeModeMetrics().setNumContainerWithOneReplicaReportedThreshold(cutOff); } @@ -105,6 +107,8 @@ public class ContainerSafeModeRule extends if (containerMap.containsKey(c.getContainerID())) { if(containerMap.remove(c.getContainerID()) != null) { containerWithMinReplicas.getAndAdd(1); + getSafeModeMetrics() + .incCurrentContainersWithOneReplicaReportedCount(); } } }); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/HealthyPipelineSafeModeRule.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/HealthyPipelineSafeModeRule.java index aee17b1ce0e..7a00d760fa4 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/HealthyPipelineSafeModeRule.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/HealthyPipelineSafeModeRule.java @@ -89,6 +89,9 @@ public class HealthyPipelineSafeModeRule LOG.info(" Total pipeline count is {}, healthy pipeline " + "threshold count is {}", pipelineCount, healthyPipelineThresholdCount); + + getSafeModeMetrics().setNumHealthyPipelinesThreshold( + healthyPipelineThresholdCount); } @Override @@ -135,9 +138,9 @@ public class HealthyPipelineSafeModeRule // If the pipeline is open state mean, all 3 datanodes are reported // for this pipeline. currentHealthyPipelineCount++; + getSafeModeMetrics().incCurrentHealthyPipelinesCount(); } } - if (scmInSafeMode()) { SCMSafeModeManager.getLogger().info( "SCM in safe mode. Healthy pipelines reported count is {}, " + diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/OneReplicaPipelineSafeModeRule.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/OneReplicaPipelineSafeModeRule.java index f8f41b7d362..841d8ff6654 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/OneReplicaPipelineSafeModeRule.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/OneReplicaPipelineSafeModeRule.java @@ -85,6 +85,9 @@ public class OneReplicaPipelineSafeModeRule extends "datanode reported threshold count is {}", totalPipelineCount, thresholdCount); + getSafeModeMetrics().setNumPipelinesWithAtleastOneReplicaReportedThreshold( + thresholdCount); + } @Override @@ -120,6 +123,8 @@ public class OneReplicaPipelineSafeModeRule extends if (pipeline.getFactor() == HddsProtos.ReplicationFactor.THREE && !reportedPipelineIDSet.contains(pipelineID)) { reportedPipelineIDSet.add(pipelineID); + getSafeModeMetrics() + .incCurrentHealthyPipelinesWithAtleastOneReplicaReportedCount(); } } 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 365259ae1cb..a22d1623fdc 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 @@ -96,6 +96,8 @@ public class SCMSafeModeManager { private final EventQueue eventPublisher; private final PipelineManager pipelineManager; + private final SafeModeMetrics safeModeMetrics; + public SCMSafeModeManager(Configuration conf, List allContainers, PipelineManager pipelineManager, EventQueue eventQueue) { @@ -106,7 +108,9 @@ public class SCMSafeModeManager { HddsConfigKeys.HDDS_SCM_SAFEMODE_ENABLED, HddsConfigKeys.HDDS_SCM_SAFEMODE_ENABLED_DEFAULT); + if (isSafeModeEnabled) { + this.safeModeMetrics = SafeModeMetrics.create(); ContainerSafeModeRule containerSafeModeRule = new ContainerSafeModeRule(CONT_EXIT_RULE, eventQueue, config, allContainers, this); @@ -132,10 +136,21 @@ public class SCMSafeModeManager { } emitSafeModeStatus(); } else { + this.safeModeMetrics = null; exitSafeMode(eventQueue); } } + public void stop() { + if (isSafeModeEnabled) { + this.safeModeMetrics.unRegister(); + } + } + + public SafeModeMetrics getSafeModeMetrics() { + return safeModeMetrics; + } + /** * Emit Safe mode status. */ diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeExitRule.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeExitRule.java index eceb8a3c005..05e84dbbb3d 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeExitRule.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeExitRule.java @@ -107,4 +107,8 @@ public abstract class SafeModeExitRule implements EventHandler { return safeModeManager.getInSafeMode(); } + protected SafeModeMetrics getSafeModeMetrics() { + return safeModeManager.getSafeModeMetrics(); + } + } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeMetrics.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeMetrics.java new file mode 100644 index 00000000000..80b8257c40b --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeMetrics.java @@ -0,0 +1,111 @@ +/* + * 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.safemode; + +import org.apache.hadoop.metrics2.MetricsSystem; +import org.apache.hadoop.metrics2.annotation.Metric; +import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; +import org.apache.hadoop.metrics2.lib.MutableCounterLong; + +/** + * This class is used for maintaining SafeMode metric information, which can + * be used for monitoring during SCM startup when SCM is still in SafeMode. + */ +public class SafeModeMetrics { + private static final String SOURCE_NAME = + SafeModeMetrics.class.getSimpleName(); + + + // These all values will be set to some values when safemode is enabled. + private @Metric MutableCounterLong + numContainerWithOneReplicaReportedThreshold; + private @Metric MutableCounterLong + currentContainersWithOneReplicaReportedCount; + + // When hdds.scm.safemode.pipeline-availability.check is set then only + // below metrics will have some values, otherwise they will be zero. + private @Metric MutableCounterLong numHealthyPipelinesThreshold; + private @Metric MutableCounterLong currentHealthyPipelinesCount; + private @Metric MutableCounterLong + numPipelinesWithAtleastOneReplicaReportedThreshold; + private @Metric MutableCounterLong + currentPipelinesWithAtleastOneReplicaReportedCount; + + public static SafeModeMetrics create() { + MetricsSystem ms = DefaultMetricsSystem.instance(); + return ms.register(SOURCE_NAME, + "SCM Safemode Metrics", + new SafeModeMetrics()); + } + + public void setNumHealthyPipelinesThreshold(long val) { + this.numHealthyPipelinesThreshold.incr(val); + } + + public void incCurrentHealthyPipelinesCount() { + this.currentHealthyPipelinesCount.incr(); + } + + public void setNumPipelinesWithAtleastOneReplicaReportedThreshold(long val) { + this.numPipelinesWithAtleastOneReplicaReportedThreshold.incr(val); + } + + public void incCurrentHealthyPipelinesWithAtleastOneReplicaReportedCount() { + this.currentPipelinesWithAtleastOneReplicaReportedCount.incr(); + } + + public void setNumContainerWithOneReplicaReportedThreshold(long val) { + this.numContainerWithOneReplicaReportedThreshold.incr(val); + } + + public void incCurrentContainersWithOneReplicaReportedCount() { + this.currentContainersWithOneReplicaReportedCount.incr(); + } + + public MutableCounterLong getNumHealthyPipelinesThreshold() { + return numHealthyPipelinesThreshold; + } + + public MutableCounterLong getCurrentHealthyPipelinesCount() { + return currentHealthyPipelinesCount; + } + + public MutableCounterLong + getNumPipelinesWithAtleastOneReplicaReportedThreshold() { + return numPipelinesWithAtleastOneReplicaReportedThreshold; + } + + public MutableCounterLong getCurrentPipelinesWithAtleastOneReplicaCount() { + return currentPipelinesWithAtleastOneReplicaReportedCount; + } + + public MutableCounterLong getNumContainerWithOneReplicaReportedThreshold() { + return numContainerWithOneReplicaReportedThreshold; + } + + public MutableCounterLong getCurrentContainersWithOneReplicaReportedCount() { + return currentContainersWithOneReplicaReportedCount; + } + + + public void unRegister() { + MetricsSystem ms = DefaultMetricsSystem.instance(); + ms.unregisterSource(SOURCE_NAME); + } +} 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 8c4a514649c..b13f2cb7cfd 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 @@ -1017,6 +1017,8 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl } catch (Exception ex) { LOG.error("SCM Metadata store stop failed", ex); } + + scmSafeModeManager.stop(); } /** 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 fa582e39c37..7ddf84e7769 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 @@ -45,6 +45,7 @@ import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager; import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode; import org.apache.hadoop.hdds.server.events.EventQueue; import org.apache.hadoop.test.GenericTestUtils; +import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; @@ -90,10 +91,10 @@ public class TestSCMSafeModeManager { private void testSafeMode(int numContainers) throws Exception { containers = new ArrayList<>(); containers.addAll(HddsTestUtils.getContainerInfo(numContainers)); - // Assign open state to containers to be included in the safe mode - // container list + + // Currently only considered containers which are not in open state. for (ContainerInfo container : containers) { - container.setState(HddsProtos.LifeCycleState.OPEN); + container.setState(HddsProtos.LifeCycleState.CLOSED); } scmSafeModeManager = new SCMSafeModeManager( config, containers, null, queue); @@ -101,15 +102,28 @@ public class TestSCMSafeModeManager { assertTrue(scmSafeModeManager.getInSafeMode()); queue.fireEvent(SCMEvents.NODE_REGISTRATION_CONT_REPORT, HddsTestUtils.createNodeRegistrationContainerReport(containers)); + + long cutOff = (long) Math.ceil(numContainers * config.getDouble( + HddsConfigKeys.HDDS_SCM_SAFEMODE_THRESHOLD_PCT, + HddsConfigKeys.HDDS_SCM_SAFEMODE_THRESHOLD_PCT_DEFAULT)); + + Assert.assertEquals(cutOff, scmSafeModeManager.getSafeModeMetrics() + .getNumContainerWithOneReplicaReportedThreshold().value()); + GenericTestUtils.waitFor(() -> { return !scmSafeModeManager.getInSafeMode(); }, 100, 1000 * 5); + + Assert.assertEquals(cutOff, scmSafeModeManager.getSafeModeMetrics() + .getCurrentContainersWithOneReplicaReportedCount().value()); + } @Test public void testSafeModeExitRule() throws Exception { containers = new ArrayList<>(); - containers.addAll(HddsTestUtils.getContainerInfo(25 * 4)); + int numContainers = 100; + containers.addAll(HddsTestUtils.getContainerInfo(numContainers)); // Assign open state to containers to be included in the safe mode // container list for (ContainerInfo container : containers) { @@ -118,15 +132,30 @@ public class TestSCMSafeModeManager { scmSafeModeManager = new SCMSafeModeManager( config, containers, null, queue); + long cutOff = (long) Math.ceil(numContainers * config.getDouble( + HddsConfigKeys.HDDS_SCM_SAFEMODE_THRESHOLD_PCT, + HddsConfigKeys.HDDS_SCM_SAFEMODE_THRESHOLD_PCT_DEFAULT)); + + Assert.assertEquals(cutOff, scmSafeModeManager.getSafeModeMetrics() + .getNumContainerWithOneReplicaReportedThreshold().value()); + assertTrue(scmSafeModeManager.getInSafeMode()); testContainerThreshold(containers.subList(0, 25), 0.25); + Assert.assertEquals(25, scmSafeModeManager.getSafeModeMetrics() + .getCurrentContainersWithOneReplicaReportedCount().value()); assertTrue(scmSafeModeManager.getInSafeMode()); testContainerThreshold(containers.subList(25, 50), 0.50); + Assert.assertEquals(50, scmSafeModeManager.getSafeModeMetrics() + .getCurrentContainersWithOneReplicaReportedCount().value()); assertTrue(scmSafeModeManager.getInSafeMode()); testContainerThreshold(containers.subList(50, 75), 0.75); + Assert.assertEquals(75, scmSafeModeManager.getSafeModeMetrics() + .getCurrentContainersWithOneReplicaReportedCount().value()); assertTrue(scmSafeModeManager.getInSafeMode()); testContainerThreshold(containers.subList(75, 100), 1.0); + Assert.assertEquals(100, scmSafeModeManager.getSafeModeMetrics() + .getCurrentContainersWithOneReplicaReportedCount().value()); GenericTestUtils.waitFor(() -> { return !scmSafeModeManager.getInSafeMode(); @@ -248,7 +277,6 @@ public class TestSCMSafeModeManager { pipelineManager, queue); assertTrue(scmSafeModeManager.getInSafeMode()); - testContainerThreshold(containers, 1.0); List pipelines = pipelineManager.getPipelines(); @@ -260,6 +288,14 @@ public class TestSCMSafeModeManager { scmSafeModeManager.getOneReplicaPipelineSafeModeRule() .getThresholdCount(); + Assert.assertEquals(healthyPipelineThresholdCount, + scmSafeModeManager.getSafeModeMetrics() + .getNumHealthyPipelinesThreshold().value()); + + Assert.assertEquals(oneReplicaThresholdCount, + scmSafeModeManager.getSafeModeMetrics() + .getNumPipelinesWithAtleastOneReplicaReportedThreshold().value()); + // Because even if no pipelines are there, and threshold we set to zero, // we shall a get an event when datanode is registered. In that case, // validate will return true, and add this to validatedRules. @@ -273,13 +309,27 @@ public class TestSCMSafeModeManager { if (i < healthyPipelineThresholdCount) { checkHealthy(i + 1); + Assert.assertEquals(i + 1, + scmSafeModeManager.getSafeModeMetrics() + .getCurrentHealthyPipelinesCount().value()); } if (i < oneReplicaThresholdCount) { checkOpen(i + 1); + Assert.assertEquals(i + 1, + scmSafeModeManager.getSafeModeMetrics() + .getCurrentPipelinesWithAtleastOneReplicaCount().value()); } } + Assert.assertEquals(healthyPipelineThresholdCount, + scmSafeModeManager.getSafeModeMetrics() + .getCurrentHealthyPipelinesCount().value()); + + Assert.assertEquals(oneReplicaThresholdCount, + scmSafeModeManager.getSafeModeMetrics() + .getCurrentPipelinesWithAtleastOneReplicaCount().value()); + GenericTestUtils.waitFor(() -> { return !scmSafeModeManager.getInSafeMode();