HDDS-1231. Add ChillMode metrics. Contributed by Bharat Viswanadham.
This commit is contained in:
parent
abf76ac371
commit
751f0df710
|
@ -76,6 +76,8 @@ public class ContainerSafeModeRule extends
|
||||||
maxContainer = containerMap.size();
|
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.containsKey(c.getContainerID())) {
|
||||||
if(containerMap.remove(c.getContainerID()) != null) {
|
if(containerMap.remove(c.getContainerID()) != null) {
|
||||||
containerWithMinReplicas.getAndAdd(1);
|
containerWithMinReplicas.getAndAdd(1);
|
||||||
|
getSafeModeMetrics()
|
||||||
|
.incCurrentContainersWithOneReplicaReportedCount();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
|
@ -89,6 +89,9 @@ public class HealthyPipelineSafeModeRule
|
||||||
|
|
||||||
LOG.info(" Total pipeline count is {}, healthy pipeline " +
|
LOG.info(" Total pipeline count is {}, healthy pipeline " +
|
||||||
"threshold count is {}", pipelineCount, healthyPipelineThresholdCount);
|
"threshold count is {}", pipelineCount, healthyPipelineThresholdCount);
|
||||||
|
|
||||||
|
getSafeModeMetrics().setNumHealthyPipelinesThreshold(
|
||||||
|
healthyPipelineThresholdCount);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -135,9 +138,9 @@ public class HealthyPipelineSafeModeRule
|
||||||
// If the pipeline is open state mean, all 3 datanodes are reported
|
// If the pipeline is open state mean, all 3 datanodes are reported
|
||||||
// for this pipeline.
|
// for this pipeline.
|
||||||
currentHealthyPipelineCount++;
|
currentHealthyPipelineCount++;
|
||||||
|
getSafeModeMetrics().incCurrentHealthyPipelinesCount();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (scmInSafeMode()) {
|
if (scmInSafeMode()) {
|
||||||
SCMSafeModeManager.getLogger().info(
|
SCMSafeModeManager.getLogger().info(
|
||||||
"SCM in safe mode. Healthy pipelines reported count is {}, " +
|
"SCM in safe mode. Healthy pipelines reported count is {}, " +
|
||||||
|
|
|
@ -85,6 +85,9 @@ public class OneReplicaPipelineSafeModeRule extends
|
||||||
"datanode reported threshold count is {}", totalPipelineCount,
|
"datanode reported threshold count is {}", totalPipelineCount,
|
||||||
thresholdCount);
|
thresholdCount);
|
||||||
|
|
||||||
|
getSafeModeMetrics().setNumPipelinesWithAtleastOneReplicaReportedThreshold(
|
||||||
|
thresholdCount);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -120,6 +123,8 @@ public class OneReplicaPipelineSafeModeRule extends
|
||||||
if (pipeline.getFactor() == HddsProtos.ReplicationFactor.THREE &&
|
if (pipeline.getFactor() == HddsProtos.ReplicationFactor.THREE &&
|
||||||
!reportedPipelineIDSet.contains(pipelineID)) {
|
!reportedPipelineIDSet.contains(pipelineID)) {
|
||||||
reportedPipelineIDSet.add(pipelineID);
|
reportedPipelineIDSet.add(pipelineID);
|
||||||
|
getSafeModeMetrics()
|
||||||
|
.incCurrentHealthyPipelinesWithAtleastOneReplicaReportedCount();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -96,6 +96,8 @@ public class SCMSafeModeManager {
|
||||||
private final EventQueue eventPublisher;
|
private final EventQueue eventPublisher;
|
||||||
private final PipelineManager pipelineManager;
|
private final PipelineManager pipelineManager;
|
||||||
|
|
||||||
|
private final SafeModeMetrics safeModeMetrics;
|
||||||
|
|
||||||
public SCMSafeModeManager(Configuration conf,
|
public SCMSafeModeManager(Configuration conf,
|
||||||
List<ContainerInfo> allContainers, PipelineManager pipelineManager,
|
List<ContainerInfo> allContainers, PipelineManager pipelineManager,
|
||||||
EventQueue eventQueue) {
|
EventQueue eventQueue) {
|
||||||
|
@ -106,7 +108,9 @@ public class SCMSafeModeManager {
|
||||||
HddsConfigKeys.HDDS_SCM_SAFEMODE_ENABLED,
|
HddsConfigKeys.HDDS_SCM_SAFEMODE_ENABLED,
|
||||||
HddsConfigKeys.HDDS_SCM_SAFEMODE_ENABLED_DEFAULT);
|
HddsConfigKeys.HDDS_SCM_SAFEMODE_ENABLED_DEFAULT);
|
||||||
|
|
||||||
|
|
||||||
if (isSafeModeEnabled) {
|
if (isSafeModeEnabled) {
|
||||||
|
this.safeModeMetrics = SafeModeMetrics.create();
|
||||||
ContainerSafeModeRule containerSafeModeRule =
|
ContainerSafeModeRule containerSafeModeRule =
|
||||||
new ContainerSafeModeRule(CONT_EXIT_RULE, eventQueue, config,
|
new ContainerSafeModeRule(CONT_EXIT_RULE, eventQueue, config,
|
||||||
allContainers, this);
|
allContainers, this);
|
||||||
|
@ -132,10 +136,21 @@ public class SCMSafeModeManager {
|
||||||
}
|
}
|
||||||
emitSafeModeStatus();
|
emitSafeModeStatus();
|
||||||
} else {
|
} else {
|
||||||
|
this.safeModeMetrics = null;
|
||||||
exitSafeMode(eventQueue);
|
exitSafeMode(eventQueue);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void stop() {
|
||||||
|
if (isSafeModeEnabled) {
|
||||||
|
this.safeModeMetrics.unRegister();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public SafeModeMetrics getSafeModeMetrics() {
|
||||||
|
return safeModeMetrics;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Emit Safe mode status.
|
* Emit Safe mode status.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -107,4 +107,8 @@ public abstract class SafeModeExitRule<T> implements EventHandler<T> {
|
||||||
return safeModeManager.getInSafeMode();
|
return safeModeManager.getInSafeMode();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected SafeModeMetrics getSafeModeMetrics() {
|
||||||
|
return safeModeManager.getSafeModeMetrics();
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
|
@ -1017,6 +1017,8 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
||||||
} catch (Exception ex) {
|
} catch (Exception ex) {
|
||||||
LOG.error("SCM Metadata store stop failed", ex);
|
LOG.error("SCM Metadata store stop failed", ex);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
scmSafeModeManager.stop();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -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.scm.server.SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode;
|
||||||
import org.apache.hadoop.hdds.server.events.EventQueue;
|
import org.apache.hadoop.hdds.server.events.EventQueue;
|
||||||
import org.apache.hadoop.test.GenericTestUtils;
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
|
import org.junit.Assert;
|
||||||
import org.junit.BeforeClass;
|
import org.junit.BeforeClass;
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
@ -90,10 +91,10 @@ public class TestSCMSafeModeManager {
|
||||||
private void testSafeMode(int numContainers) throws Exception {
|
private void testSafeMode(int numContainers) throws Exception {
|
||||||
containers = new ArrayList<>();
|
containers = new ArrayList<>();
|
||||||
containers.addAll(HddsTestUtils.getContainerInfo(numContainers));
|
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) {
|
for (ContainerInfo container : containers) {
|
||||||
container.setState(HddsProtos.LifeCycleState.OPEN);
|
container.setState(HddsProtos.LifeCycleState.CLOSED);
|
||||||
}
|
}
|
||||||
scmSafeModeManager = new SCMSafeModeManager(
|
scmSafeModeManager = new SCMSafeModeManager(
|
||||||
config, containers, null, queue);
|
config, containers, null, queue);
|
||||||
|
@ -101,15 +102,28 @@ public class TestSCMSafeModeManager {
|
||||||
assertTrue(scmSafeModeManager.getInSafeMode());
|
assertTrue(scmSafeModeManager.getInSafeMode());
|
||||||
queue.fireEvent(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
|
queue.fireEvent(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
|
||||||
HddsTestUtils.createNodeRegistrationContainerReport(containers));
|
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(() -> {
|
GenericTestUtils.waitFor(() -> {
|
||||||
return !scmSafeModeManager.getInSafeMode();
|
return !scmSafeModeManager.getInSafeMode();
|
||||||
}, 100, 1000 * 5);
|
}, 100, 1000 * 5);
|
||||||
|
|
||||||
|
Assert.assertEquals(cutOff, scmSafeModeManager.getSafeModeMetrics()
|
||||||
|
.getCurrentContainersWithOneReplicaReportedCount().value());
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSafeModeExitRule() throws Exception {
|
public void testSafeModeExitRule() throws Exception {
|
||||||
containers = new ArrayList<>();
|
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
|
// Assign open state to containers to be included in the safe mode
|
||||||
// container list
|
// container list
|
||||||
for (ContainerInfo container : containers) {
|
for (ContainerInfo container : containers) {
|
||||||
|
@ -118,15 +132,30 @@ public class TestSCMSafeModeManager {
|
||||||
scmSafeModeManager = new SCMSafeModeManager(
|
scmSafeModeManager = new SCMSafeModeManager(
|
||||||
config, containers, null, queue);
|
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());
|
assertTrue(scmSafeModeManager.getInSafeMode());
|
||||||
|
|
||||||
testContainerThreshold(containers.subList(0, 25), 0.25);
|
testContainerThreshold(containers.subList(0, 25), 0.25);
|
||||||
|
Assert.assertEquals(25, scmSafeModeManager.getSafeModeMetrics()
|
||||||
|
.getCurrentContainersWithOneReplicaReportedCount().value());
|
||||||
assertTrue(scmSafeModeManager.getInSafeMode());
|
assertTrue(scmSafeModeManager.getInSafeMode());
|
||||||
testContainerThreshold(containers.subList(25, 50), 0.50);
|
testContainerThreshold(containers.subList(25, 50), 0.50);
|
||||||
|
Assert.assertEquals(50, scmSafeModeManager.getSafeModeMetrics()
|
||||||
|
.getCurrentContainersWithOneReplicaReportedCount().value());
|
||||||
assertTrue(scmSafeModeManager.getInSafeMode());
|
assertTrue(scmSafeModeManager.getInSafeMode());
|
||||||
testContainerThreshold(containers.subList(50, 75), 0.75);
|
testContainerThreshold(containers.subList(50, 75), 0.75);
|
||||||
|
Assert.assertEquals(75, scmSafeModeManager.getSafeModeMetrics()
|
||||||
|
.getCurrentContainersWithOneReplicaReportedCount().value());
|
||||||
assertTrue(scmSafeModeManager.getInSafeMode());
|
assertTrue(scmSafeModeManager.getInSafeMode());
|
||||||
testContainerThreshold(containers.subList(75, 100), 1.0);
|
testContainerThreshold(containers.subList(75, 100), 1.0);
|
||||||
|
Assert.assertEquals(100, scmSafeModeManager.getSafeModeMetrics()
|
||||||
|
.getCurrentContainersWithOneReplicaReportedCount().value());
|
||||||
|
|
||||||
GenericTestUtils.waitFor(() -> {
|
GenericTestUtils.waitFor(() -> {
|
||||||
return !scmSafeModeManager.getInSafeMode();
|
return !scmSafeModeManager.getInSafeMode();
|
||||||
|
@ -248,7 +277,6 @@ public class TestSCMSafeModeManager {
|
||||||
pipelineManager, queue);
|
pipelineManager, queue);
|
||||||
|
|
||||||
assertTrue(scmSafeModeManager.getInSafeMode());
|
assertTrue(scmSafeModeManager.getInSafeMode());
|
||||||
|
|
||||||
testContainerThreshold(containers, 1.0);
|
testContainerThreshold(containers, 1.0);
|
||||||
|
|
||||||
List<Pipeline> pipelines = pipelineManager.getPipelines();
|
List<Pipeline> pipelines = pipelineManager.getPipelines();
|
||||||
|
@ -260,6 +288,14 @@ public class TestSCMSafeModeManager {
|
||||||
scmSafeModeManager.getOneReplicaPipelineSafeModeRule()
|
scmSafeModeManager.getOneReplicaPipelineSafeModeRule()
|
||||||
.getThresholdCount();
|
.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,
|
// 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,
|
// we shall a get an event when datanode is registered. In that case,
|
||||||
// validate will return true, and add this to validatedRules.
|
// validate will return true, and add this to validatedRules.
|
||||||
|
@ -273,13 +309,27 @@ public class TestSCMSafeModeManager {
|
||||||
|
|
||||||
if (i < healthyPipelineThresholdCount) {
|
if (i < healthyPipelineThresholdCount) {
|
||||||
checkHealthy(i + 1);
|
checkHealthy(i + 1);
|
||||||
|
Assert.assertEquals(i + 1,
|
||||||
|
scmSafeModeManager.getSafeModeMetrics()
|
||||||
|
.getCurrentHealthyPipelinesCount().value());
|
||||||
}
|
}
|
||||||
|
|
||||||
if (i < oneReplicaThresholdCount) {
|
if (i < oneReplicaThresholdCount) {
|
||||||
checkOpen(i + 1);
|
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(() -> {
|
GenericTestUtils.waitFor(() -> {
|
||||||
return !scmSafeModeManager.getInSafeMode();
|
return !scmSafeModeManager.getInSafeMode();
|
||||||
|
|
Loading…
Reference in New Issue