HDDS-1353 : Metrics scm_pipeline_metrics_num_pipeline_creation_failed keeps increasing because of BackgroundPipelineCreator. (#681)
This commit is contained in:
parent
d2637cb176
commit
993f36ee33
@ -0,0 +1,36 @@
|
||||
/**
|
||||
* 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.pipeline;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Exception thrown when there are not enough Datanodes to create a pipeline.
|
||||
*/
|
||||
public class InsufficientDatanodesException extends IOException {
|
||||
|
||||
|
||||
public InsufficientDatanodesException() {
|
||||
super();
|
||||
}
|
||||
|
||||
public InsufficientDatanodesException(String message) {
|
||||
super(message);
|
||||
}
|
||||
}
|
@ -105,7 +105,7 @@ public Pipeline create(ReplicationFactor factor) throws IOException {
|
||||
String e = String
|
||||
.format("Cannot create pipeline of factor %d using %d nodes.",
|
||||
factor.getNumber(), dns.size());
|
||||
throw new IOException(e);
|
||||
throw new InsufficientDatanodesException(e);
|
||||
}
|
||||
|
||||
Pipeline pipeline = Pipeline.newBuilder()
|
||||
|
@ -152,6 +152,8 @@ public synchronized Pipeline createPipeline(
|
||||
nodeManager.addPipeline(pipeline);
|
||||
metrics.incNumPipelineCreated();
|
||||
return pipeline;
|
||||
} catch (InsufficientDatanodesException idEx) {
|
||||
throw idEx;
|
||||
} catch (IOException ex) {
|
||||
metrics.incNumPipelineCreationFailed();
|
||||
throw ex;
|
||||
|
@ -18,6 +18,9 @@
|
||||
|
||||
package org.apache.hadoop.hdds.scm.pipeline;
|
||||
|
||||
import static org.apache.hadoop.test.MetricsAsserts.getLongCounter;
|
||||
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.hdds.HddsConfigKeys;
|
||||
@ -30,6 +33,7 @@
|
||||
import org.apache.hadoop.hdds.scm.container.MockNodeManager;
|
||||
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode;
|
||||
import org.apache.hadoop.hdds.server.events.EventQueue;
|
||||
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
@ -208,4 +212,61 @@ public void testPipelineReport() throws IOException {
|
||||
// clean up
|
||||
pipelineManager.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPipelineCreationFailedMetric() throws Exception {
|
||||
MockNodeManager nodeManagerMock = new MockNodeManager(true,
|
||||
20);
|
||||
SCMPipelineManager pipelineManager =
|
||||
new SCMPipelineManager(conf, nodeManagerMock, new EventQueue());
|
||||
PipelineProvider mockRatisProvider =
|
||||
new MockRatisPipelineProvider(nodeManagerMock,
|
||||
pipelineManager.getStateManager(), conf);
|
||||
pipelineManager.setPipelineProvider(HddsProtos.ReplicationType.RATIS,
|
||||
mockRatisProvider);
|
||||
|
||||
MetricsRecordBuilder metrics = getMetrics(
|
||||
SCMPipelineMetrics.class.getSimpleName());
|
||||
long numPipelineCreated = getLongCounter("NumPipelineCreated",
|
||||
metrics);
|
||||
Assert.assertTrue(numPipelineCreated == 0);
|
||||
|
||||
// 3 DNs are unhealthy.
|
||||
// Create 5 pipelines (Use up 15 Datanodes)
|
||||
for (int i = 0; i < 5; i++) {
|
||||
Pipeline pipeline = pipelineManager
|
||||
.createPipeline(HddsProtos.ReplicationType.RATIS,
|
||||
HddsProtos.ReplicationFactor.THREE);
|
||||
Assert.assertNotNull(pipeline);
|
||||
}
|
||||
|
||||
metrics = getMetrics(
|
||||
SCMPipelineMetrics.class.getSimpleName());
|
||||
numPipelineCreated = getLongCounter("NumPipelineCreated", metrics);
|
||||
Assert.assertTrue(numPipelineCreated == 5);
|
||||
|
||||
long numPipelineCreateFailed = getLongCounter(
|
||||
"NumPipelineCreationFailed", metrics);
|
||||
Assert.assertTrue(numPipelineCreateFailed == 0);
|
||||
|
||||
//This should fail...
|
||||
try {
|
||||
pipelineManager.createPipeline(HddsProtos.ReplicationType.RATIS,
|
||||
HddsProtos.ReplicationFactor.THREE);
|
||||
Assert.fail();
|
||||
} catch (InsufficientDatanodesException idEx) {
|
||||
Assert.assertEquals(
|
||||
"Cannot create pipeline of factor 3 using 1 nodes.",
|
||||
idEx.getMessage());
|
||||
}
|
||||
|
||||
metrics = getMetrics(
|
||||
SCMPipelineMetrics.class.getSimpleName());
|
||||
numPipelineCreated = getLongCounter("NumPipelineCreated", metrics);
|
||||
Assert.assertTrue(numPipelineCreated == 5);
|
||||
|
||||
numPipelineCreateFailed = getLongCounter(
|
||||
"NumPipelineCreationFailed", metrics);
|
||||
Assert.assertTrue(numPipelineCreateFailed == 0);
|
||||
}
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user