diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManager.java index 74e70bbe058a..a903be614f1f 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManager.java @@ -83,6 +83,13 @@ default List getContainers() { */ List getContainers(LifeCycleState state); + /** + * Returns the size of containers which are in the specified state. + * + * @return size of containers. + */ + int getContainerStateCount(LifeCycleState state); + /** * Returns true if the container exist, false otherwise. * @param id Container ID diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java index 5e220dff59bb..cc1ee326b7a7 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerManagerImpl.java @@ -157,19 +157,38 @@ public List getContainers(final ContainerID startID, final List containersIds = new ArrayList<>(containerStateManager.getContainerIDs()); Collections.sort(containersIds); - return containersIds.stream() - .filter(id -> id.getId() >= start).limit(count) - .map(ContainerID::getProtobuf) - .map(containerStateManager::getContainer) - .collect(Collectors.toList()); + List containers; + lock.lock(); + try { + containers = containersIds.stream() + .filter(id -> id.getId() >= start).limit(count) + .map(ContainerID::getProtobuf) + .map(containerStateManager::getContainer) + .collect(Collectors.toList()); + } finally { + lock.unlock(); + } + return containers; } @Override public List getContainers(final LifeCycleState state) { - return containerStateManager.getContainerIDs(state).stream() - .map(ContainerID::getProtobuf) - .map(containerStateManager::getContainer) - .filter(Objects::nonNull).collect(Collectors.toList()); + List containers; + lock.lock(); + try { + containers = containerStateManager.getContainerIDs(state).stream() + .map(ContainerID::getProtobuf) + .map(containerStateManager::getContainer) + .filter(Objects::nonNull).collect(Collectors.toList()); + } finally { + lock.unlock(); + } + return containers; + } + + @Override + public int getContainerStateCount(final LifeCycleState state) { + return containerStateManager.getContainerIDs(state).size(); } @Override diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java index 2fb788d5d7cf..6c9783c5d0a2 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java @@ -1761,7 +1761,7 @@ public Map getContainerStateCount() { Map nodeStateCount = new HashMap<>(); for (HddsProtos.LifeCycleState state : HddsProtos.LifeCycleState.values()) { nodeStateCount.put(state.toString(), - containerManager.getContainers(state).size()); + containerManager.getContainerStateCount(state)); } return nodeStateCount; } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java index 528fa7946ce5..ca5df9070157 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerManagerImpl.java @@ -154,5 +154,19 @@ public void testGetContainers() throws Exception { containerManager.getContainers(HddsProtos.LifeCycleState.OPEN).size()); Assert.assertEquals(2, containerManager .getContainers(HddsProtos.LifeCycleState.CLOSING).size()); + containerManager.updateContainerState(cidArray[1], + HddsProtos.LifeCycleEvent.QUASI_CLOSE); + containerManager.updateContainerState(cidArray[2], + HddsProtos.LifeCycleEvent.FINALIZE); + containerManager.updateContainerState(cidArray[2], + HddsProtos.LifeCycleEvent.CLOSE); + Assert.assertEquals(7, containerManager. + getContainerStateCount(HddsProtos.LifeCycleState.OPEN)); + Assert.assertEquals(1, containerManager + .getContainerStateCount(HddsProtos.LifeCycleState.CLOSING)); + Assert.assertEquals(1, containerManager + .getContainerStateCount(HddsProtos.LifeCycleState.QUASI_CLOSED)); + Assert.assertEquals(1, containerManager + .getContainerStateCount(HddsProtos.LifeCycleState.CLOSED)); } }