HDDS-1146. Adding container related metrics in SCM. (#1541)
This commit is contained in:
parent
13b427fc05
commit
14b4fbc019
|
@ -233,4 +233,12 @@ public class AbstractContainerReportHandler {
|
||||||
return replicaState.get() == ContainerReplicaProto.State.UNHEALTHY;
|
return replicaState.get() == ContainerReplicaProto.State.UNHEALTHY;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return ContainerManager.
|
||||||
|
* @return {@link ContainerManager}
|
||||||
|
*/
|
||||||
|
protected ContainerManager getContainerManager() {
|
||||||
|
return containerManager;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
|
@ -179,4 +179,12 @@ public interface ContainerManager extends Closeable {
|
||||||
*/
|
*/
|
||||||
ContainerInfo getMatchingContainer(long size, String owner,
|
ContainerInfo getMatchingContainer(long size, String owner,
|
||||||
Pipeline pipeline, List<ContainerID> excludedContainerIDS);
|
Pipeline pipeline, List<ContainerID> excludedContainerIDS);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Once after report processor handler completes, call this to notify
|
||||||
|
* container manager to increment metrics.
|
||||||
|
* @param isFullReport
|
||||||
|
* @param success
|
||||||
|
*/
|
||||||
|
void notifyContainerReportProcessing(boolean isFullReport, boolean success);
|
||||||
}
|
}
|
||||||
|
|
|
@ -104,7 +104,9 @@ public class ContainerReportHandler extends AbstractContainerReportHandler
|
||||||
*/
|
*/
|
||||||
nodeManager.setContainers(datanodeDetails, containersInDn);
|
nodeManager.setContainers(datanodeDetails, containersInDn);
|
||||||
|
|
||||||
|
containerManager.notifyContainerReportProcessing(true, true);
|
||||||
} catch (NodeNotFoundException ex) {
|
} catch (NodeNotFoundException ex) {
|
||||||
|
containerManager.notifyContainerReportProcessing(true, false);
|
||||||
LOG.error("Received container report from unknown datanode {} {}",
|
LOG.error("Received container report from unknown datanode {} {}",
|
||||||
datanodeDetails, ex);
|
datanodeDetails, ex);
|
||||||
}
|
}
|
||||||
|
|
|
@ -57,6 +57,7 @@ public class IncrementalContainerReportHandler extends
|
||||||
LOG.debug("Processing incremental container report from data node {}",
|
LOG.debug("Processing incremental container report from data node {}",
|
||||||
report.getDatanodeDetails().getUuid());
|
report.getDatanodeDetails().getUuid());
|
||||||
|
|
||||||
|
boolean success = true;
|
||||||
for (ContainerReplicaProto replicaProto :
|
for (ContainerReplicaProto replicaProto :
|
||||||
report.getReport().getReportList()) {
|
report.getReport().getReportList()) {
|
||||||
try {
|
try {
|
||||||
|
@ -66,16 +67,25 @@ public class IncrementalContainerReportHandler extends
|
||||||
nodeManager.addContainer(dd, id);
|
nodeManager.addContainer(dd, id);
|
||||||
processContainerReplica(dd, replicaProto);
|
processContainerReplica(dd, replicaProto);
|
||||||
} catch (ContainerNotFoundException e) {
|
} catch (ContainerNotFoundException e) {
|
||||||
|
success = false;
|
||||||
LOG.warn("Container {} not found!", replicaProto.getContainerID());
|
LOG.warn("Container {} not found!", replicaProto.getContainerID());
|
||||||
} catch (NodeNotFoundException ex) {
|
} catch (NodeNotFoundException ex) {
|
||||||
|
success = false;
|
||||||
LOG.error("Received ICR from unknown datanode {} {}",
|
LOG.error("Received ICR from unknown datanode {} {}",
|
||||||
report.getDatanodeDetails(), ex);
|
report.getDatanodeDetails(), ex);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
|
success = false;
|
||||||
LOG.error("Exception while processing ICR for container {}",
|
LOG.error("Exception while processing ICR for container {}",
|
||||||
replicaProto.getContainerID());
|
replicaProto.getContainerID());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (success) {
|
||||||
|
getContainerManager().notifyContainerReportProcessing(false, true);
|
||||||
|
} else {
|
||||||
|
getContainerManager().notifyContainerReportProcessing(false, false);
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -572,4 +572,21 @@ public class SCMContainerManager implements ContainerManager {
|
||||||
this.scmContainerManagerMetrics.unRegister();
|
this.scmContainerManagerMetrics.unRegister();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void notifyContainerReportProcessing(boolean isFullReport,
|
||||||
|
boolean success) {
|
||||||
|
if (isFullReport) {
|
||||||
|
if (success) {
|
||||||
|
scmContainerManagerMetrics.incNumContainerReportsProcessedSuccessful();
|
||||||
|
} else {
|
||||||
|
scmContainerManagerMetrics.incNumContainerReportsProcessedFailed();
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
if (success) {
|
||||||
|
scmContainerManagerMetrics.incNumICRReportsProcessedSuccessful();
|
||||||
|
} else {
|
||||||
|
scmContainerManagerMetrics.incNumICRReportsProcessedFailed();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,3 +1,19 @@
|
||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p/>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p/>
|
||||||
|
* 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.container.metrics;
|
package org.apache.hadoop.hdds.scm.container.metrics;
|
||||||
|
|
||||||
|
|
||||||
|
@ -26,6 +42,12 @@ public final class SCMContainerManagerMetrics {
|
||||||
private @Metric MutableCounterLong numFailureDeleteContainers;
|
private @Metric MutableCounterLong numFailureDeleteContainers;
|
||||||
private @Metric MutableCounterLong numListContainerOps;
|
private @Metric MutableCounterLong numListContainerOps;
|
||||||
|
|
||||||
|
|
||||||
|
private @Metric MutableCounterLong numContainerReportsProcessedSuccessful;
|
||||||
|
private @Metric MutableCounterLong numContainerReportsProcessedFailed;
|
||||||
|
private @Metric MutableCounterLong numICRReportsProcessedSuccessful;
|
||||||
|
private @Metric MutableCounterLong numICRReportsProcessedFailed;
|
||||||
|
|
||||||
private SCMContainerManagerMetrics() {
|
private SCMContainerManagerMetrics() {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -67,6 +89,38 @@ public final class SCMContainerManagerMetrics {
|
||||||
this.numListContainerOps.incr();
|
this.numListContainerOps.incr();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void incNumContainerReportsProcessedSuccessful() {
|
||||||
|
this.numContainerReportsProcessedSuccessful.incr();
|
||||||
|
}
|
||||||
|
|
||||||
|
public void incNumContainerReportsProcessedFailed() {
|
||||||
|
this.numContainerReportsProcessedFailed.incr();
|
||||||
|
}
|
||||||
|
|
||||||
|
public void incNumICRReportsProcessedSuccessful() {
|
||||||
|
this.numICRReportsProcessedSuccessful.incr();
|
||||||
|
}
|
||||||
|
|
||||||
|
public void incNumICRReportsProcessedFailed() {
|
||||||
|
this.numICRReportsProcessedFailed.incr();
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getNumContainerReportsProcessedSuccessful() {
|
||||||
|
return numContainerReportsProcessedSuccessful.value();
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getNumContainerReportsProcessedFailed() {
|
||||||
|
return numContainerReportsProcessedFailed.value();
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getNumICRReportsProcessedSuccessful() {
|
||||||
|
return numICRReportsProcessedSuccessful.value();
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getNumICRReportsProcessedFailed() {
|
||||||
|
return numICRReportsProcessedFailed.value();
|
||||||
|
}
|
||||||
|
|
||||||
public long getNumSuccessfulCreateContainers() {
|
public long getNumSuccessfulCreateContainers() {
|
||||||
return numSuccessfulCreateContainers.value();
|
return numSuccessfulCreateContainers.value();
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,22 +1,44 @@
|
||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p/>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p/>
|
||||||
|
* 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.container.metrics;
|
package org.apache.hadoop.hdds.scm.container.metrics;
|
||||||
|
|
||||||
import org.apache.commons.lang3.RandomUtils;
|
import org.apache.commons.lang3.RandomUtils;
|
||||||
|
import org.apache.hadoop.hdds.client.ReplicationFactor;
|
||||||
|
import org.apache.hadoop.hdds.client.ReplicationType;
|
||||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||||
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||||
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
||||||
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
||||||
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
||||||
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
||||||
|
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
|
||||||
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.HashMap;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL;
|
||||||
import static org.apache.hadoop.test.MetricsAsserts.getLongCounter;
|
import static org.apache.hadoop.test.MetricsAsserts.getLongCounter;
|
||||||
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
|
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
|
||||||
import static org.junit.Assert.fail;
|
import static org.junit.Assert.fail;
|
||||||
|
@ -28,16 +50,15 @@ public class TestSCMContainerManagerMetrics {
|
||||||
|
|
||||||
private MiniOzoneCluster cluster;
|
private MiniOzoneCluster cluster;
|
||||||
private StorageContainerManager scm;
|
private StorageContainerManager scm;
|
||||||
private XceiverClientManager xceiverClientManager;
|
|
||||||
private String containerOwner = "OZONE";
|
private String containerOwner = "OZONE";
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setup() throws Exception {
|
public void setup() throws Exception {
|
||||||
OzoneConfiguration conf = new OzoneConfiguration();
|
OzoneConfiguration conf = new OzoneConfiguration();
|
||||||
|
conf.set(HDDS_CONTAINER_REPORT_INTERVAL, "3000s");
|
||||||
cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
|
cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
|
||||||
cluster.waitForClusterToBeReady();
|
cluster.waitForClusterToBeReady();
|
||||||
scm = cluster.getStorageContainerManager();
|
scm = cluster.getStorageContainerManager();
|
||||||
xceiverClientManager = new XceiverClientManager(conf);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -51,7 +72,6 @@ public class TestSCMContainerManagerMetrics {
|
||||||
MetricsRecordBuilder metrics;
|
MetricsRecordBuilder metrics;
|
||||||
ContainerManager containerManager = scm.getContainerManager();
|
ContainerManager containerManager = scm.getContainerManager();
|
||||||
metrics = getMetrics(SCMContainerManagerMetrics.class.getSimpleName());
|
metrics = getMetrics(SCMContainerManagerMetrics.class.getSimpleName());
|
||||||
|
|
||||||
long numSuccessfulCreateContainers = getLongCounter(
|
long numSuccessfulCreateContainers = getLongCounter(
|
||||||
"NumSuccessfulCreateContainers", metrics);
|
"NumSuccessfulCreateContainers", metrics);
|
||||||
|
|
||||||
|
@ -108,5 +128,40 @@ public class TestSCMContainerManagerMetrics {
|
||||||
metrics = getMetrics(SCMContainerManagerMetrics.class.getSimpleName());
|
metrics = getMetrics(SCMContainerManagerMetrics.class.getSimpleName());
|
||||||
Assert.assertEquals(getLongCounter("NumListContainerOps",
|
Assert.assertEquals(getLongCounter("NumListContainerOps",
|
||||||
metrics), 1);
|
metrics), 1);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testReportProcessingMetrics() throws Exception {
|
||||||
|
String volumeName = "vol1";
|
||||||
|
String bucketName = "bucket1";
|
||||||
|
String key = "key1";
|
||||||
|
|
||||||
|
MetricsRecordBuilder metrics =
|
||||||
|
getMetrics(SCMContainerManagerMetrics.class.getSimpleName());
|
||||||
|
Assert.assertEquals(getLongCounter("NumContainerReportsProcessedSuccessful",
|
||||||
|
metrics), 1);
|
||||||
|
|
||||||
|
// Create key should create container on DN.
|
||||||
|
cluster.getRpcClient().getObjectStore().getClientProxy()
|
||||||
|
.createVolume(volumeName);
|
||||||
|
cluster.getRpcClient().getObjectStore().getClientProxy()
|
||||||
|
.createBucket(volumeName, bucketName);
|
||||||
|
OzoneOutputStream ozoneOutputStream = cluster.getRpcClient()
|
||||||
|
.getObjectStore().getClientProxy().createKey(volumeName, bucketName,
|
||||||
|
key, 0, ReplicationType.RATIS, ReplicationFactor.ONE,
|
||||||
|
new HashMap<>());
|
||||||
|
|
||||||
|
String data = "file data";
|
||||||
|
ozoneOutputStream.write(data.getBytes(), 0, data.length());
|
||||||
|
ozoneOutputStream.close();
|
||||||
|
|
||||||
|
|
||||||
|
GenericTestUtils.waitFor(() -> {
|
||||||
|
final MetricsRecordBuilder scmMetrics =
|
||||||
|
getMetrics(SCMContainerManagerMetrics.class.getSimpleName());
|
||||||
|
return getLongCounter("NumICRReportsProcessedSuccessful",
|
||||||
|
scmMetrics) == 1;
|
||||||
|
}, 1000, 500000);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue