HDDS-848. Create SCM metrics related to container state. Contributed by Bharat Viswanadham.

This commit is contained in:
Yiqun Lin 2018-12-03 17:16:34 +08:00
parent 042c8ef593
commit 3044b78bd0
3 changed files with 91 additions and 6 deletions

View File

@ -59,4 +59,9 @@ public interface SCMMXBean extends ServiceRuntimeInfo {
* @return String
*/
double getChillModeCurrentContainerThreshold();
/**
* Returns the container count in all states.
*/
Map<String, Integer> getContainerStateCount();
}

View File

@ -30,6 +30,7 @@ import com.google.protobuf.BlockingService;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdds.HddsUtils;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.block.BlockManager;
@ -925,6 +926,16 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
return scmChillModeManager.getCurrentContainerThreshold();
}
@Override
public Map<String, Integer> getContainerStateCount() {
Map<String, Integer> nodeStateCount = new HashMap<>();
for (HddsProtos.LifeCycleState state: HddsProtos.LifeCycleState.values()) {
nodeStateCount.put(state.toString(), containerManager.getContainers(
state).size());
}
return nodeStateCount;
}
/**
* Startup options.
*/

View File

@ -20,6 +20,10 @@ package org.apache.hadoop.ozone.scm;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@ -31,8 +35,11 @@ import javax.management.MBeanServer;
import javax.management.ObjectName;
import java.io.IOException;
import java.lang.management.ManagementFactory;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Iterator;
import java.util.UUID;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeoutException;
@ -118,16 +125,77 @@ public class TestSCMMXBean {
assertEquals(scm.getCurrentContainerThreshold(), containerThreshold, 0);
}
@Test
public void testSCMContainerStateCount() throws Exception {
ObjectName bean = new ObjectName(
"Hadoop:service=StorageContainerManager,"
+ "name=StorageContainerManagerInfo,"
+ "component=ServerRuntime");
TabularData data = (TabularData) mbs.getAttribute(
bean, "ContainerStateCount");
Map<String, Integer> containerStateCount = scm.getContainerStateCount();
verifyEquals(data, containerStateCount);
// Do some changes like allocate containers and change the container states
ContainerManager scmContainerManager = scm.getContainerManager();
List<ContainerInfo> containerInfoList = new ArrayList<>();
for (int i=0; i < 10; i++) {
containerInfoList.add(scmContainerManager.allocateContainer(HddsProtos
.ReplicationType.STAND_ALONE, HddsProtos.ReplicationFactor.ONE,
UUID.randomUUID().toString()));
}
long containerID;
for (int i=0; i < 10; i++) {
if (i % 2 == 0) {
containerID = containerInfoList.get(i).getContainerID();
scmContainerManager.updateContainerState(
new ContainerID(containerID), HddsProtos.LifeCycleEvent.FINALIZE);
assertEquals(scmContainerManager.getContainer(new ContainerID(
containerID)).getState(), HddsProtos.LifeCycleState.CLOSING);
} else {
containerID = containerInfoList.get(i).getContainerID();
scmContainerManager.updateContainerState(
new ContainerID(containerID), HddsProtos.LifeCycleEvent.FINALIZE);
scmContainerManager.updateContainerState(
new ContainerID(containerID), HddsProtos.LifeCycleEvent.CLOSE);
assertEquals(scmContainerManager.getContainer(new ContainerID(
containerID)).getState(), HddsProtos.LifeCycleState.CLOSED);
}
}
data = (TabularData) mbs.getAttribute(
bean, "ContainerStateCount");
containerStateCount = scm.getContainerStateCount();
containerStateCount.forEach((k, v) -> {
if(k == HddsProtos.LifeCycleState.CLOSING.toString()) {
assertEquals((int)v, 5);
} else if (k == HddsProtos.LifeCycleState.CLOSED.toString()) {
assertEquals((int)v, 5);
} else {
// Remaining all container state count should be zero.
assertEquals((int)v, 0);
}
});
verifyEquals(data, containerStateCount);
}
/**
* An internal function used to compare a TabularData returned
* by JMX with the expected data in a Map.
*/
private void verifyEquals(TabularData data1,
Map<String, Integer> data2) {
if (data1 == null || data2 == null) {
private void verifyEquals(TabularData actualData,
Map<String, Integer> expectedData) {
if (actualData == null || expectedData == null) {
fail("Data should not be null.");
}
for (Object obj : data1.values()) {
for (Object obj : actualData.values()) {
// Each TabularData is a set of CompositeData
assertTrue(obj instanceof CompositeData);
CompositeData cds = (CompositeData) obj;
@ -136,8 +204,9 @@ public class TestSCMMXBean {
String key = it.next().toString();
String value = it.next().toString();
int num = Integer.parseInt(value);
assertTrue(data2.containsKey(key));
assertEquals(data2.get(key).intValue(), num);
assertTrue(expectedData.containsKey(key));
assertEquals(expectedData.remove(key).intValue(), num);
}
assertTrue(expectedData.isEmpty());
}
}