HDDS-1231. Add ChillMode metrics. Contributed by Bharat Viswanadham.

This commit is contained in:
Bharat Viswanadham 2019-05-29 11:55:44 -07:00 committed by Anu Engineer
parent abf76ac371
commit 751f0df710
8 changed files with 200 additions and 6 deletions

View File

@ -76,6 +76,8 @@ public ContainerSafeModeRule(String ruleName, EventQueue eventQueue,
maxContainer = containerMap.size();
}
long cutOff = (long) Math.ceil(maxContainer * safeModeCutoff);
getSafeModeMetrics().setNumContainerWithOneReplicaReportedThreshold(cutOff);
}
@ -105,6 +107,8 @@ protected void process(NodeRegistrationContainerReport reportsProto) {
if (containerMap.containsKey(c.getContainerID())) {
if(containerMap.remove(c.getContainerID()) != null) {
containerWithMinReplicas.getAndAdd(1);
getSafeModeMetrics()
.incCurrentContainersWithOneReplicaReportedCount();
}
}
});

View File

@ -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 @@ protected void process(PipelineReportFromDatanode
// 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 {}, " +

View File

@ -85,6 +85,9 @@ public OneReplicaPipelineSafeModeRule(String ruleName, EventQueue eventQueue,
"datanode reported threshold count is {}", totalPipelineCount,
thresholdCount);
getSafeModeMetrics().setNumPipelinesWithAtleastOneReplicaReportedThreshold(
thresholdCount);
}
@Override
@ -120,6 +123,8 @@ protected void process(PipelineReportFromDatanode
if (pipeline.getFactor() == HddsProtos.ReplicationFactor.THREE &&
!reportedPipelineIDSet.contains(pipelineID)) {
reportedPipelineIDSet.add(pipelineID);
getSafeModeMetrics()
.incCurrentHealthyPipelinesWithAtleastOneReplicaReportedCount();
}
}

View File

@ -96,6 +96,8 @@ public class SCMSafeModeManager {
private final EventQueue eventPublisher;
private final PipelineManager pipelineManager;
private final SafeModeMetrics safeModeMetrics;
public SCMSafeModeManager(Configuration conf,
List<ContainerInfo> allContainers, PipelineManager pipelineManager,
EventQueue eventQueue) {
@ -106,7 +108,9 @@ public SCMSafeModeManager(Configuration conf,
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 SCMSafeModeManager(Configuration conf,
}
emitSafeModeStatus();
} else {
this.safeModeMetrics = null;
exitSafeMode(eventQueue);
}
}
public void stop() {
if (isSafeModeEnabled) {
this.safeModeMetrics.unRegister();
}
}
public SafeModeMetrics getSafeModeMetrics() {
return safeModeMetrics;
}
/**
* Emit Safe mode status.
*/

View File

@ -107,4 +107,8 @@ protected boolean scmInSafeMode() {
return safeModeManager.getInSafeMode();
}
protected SafeModeMetrics getSafeModeMetrics() {
return safeModeManager.getSafeModeMetrics();
}
}

View File

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

View File

@ -1017,6 +1017,8 @@ public void stop() {
} catch (Exception ex) {
LOG.error("SCM Metadata store stop failed", ex);
}
scmSafeModeManager.stop();
}
/**

View File

@ -45,6 +45,7 @@
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 void testSafeModeStateWithNullContainers() {
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 @@ private void testSafeMode(int numContainers) throws Exception {
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 void testSafeModeExitRule() throws Exception {
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 void testSafeModeExitRuleWithPipelineAvailabilityCheck(
pipelineManager, queue);
assertTrue(scmSafeModeManager.getInSafeMode());
testContainerThreshold(containers, 1.0);
List<Pipeline> pipelines = pipelineManager.getPipelines();
@ -260,6 +288,14 @@ public void testSafeModeExitRuleWithPipelineAvailabilityCheck(
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 void testSafeModeExitRuleWithPipelineAvailabilityCheck(
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();