|
@@ -17,41 +17,41 @@
|
|
|
|
|
|
package org.apache.hadoop.hdds.scm.container;
|
|
|
|
|
|
-import com.google.common.base.Preconditions;
|
|
|
+import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
|
|
|
+ .FAILED_TO_CHANGE_CONTAINER_STATE;
|
|
|
+
|
|
|
+import java.io.IOException;
|
|
|
+import java.util.HashSet;
|
|
|
+import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
+import java.util.NavigableSet;
|
|
|
+import java.util.Set;
|
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|
|
+import java.util.concurrent.atomic.AtomicLong;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.conf.StorageUnit;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
|
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
|
|
-import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
|
|
import org.apache.hadoop.hdds.scm.container.states.ContainerState;
|
|
|
import org.apache.hadoop.hdds.scm.container.states.ContainerStateMap;
|
|
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
|
|
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
|
|
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
|
|
|
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
|
|
|
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
|
|
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent;
|
|
|
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
|
|
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
|
|
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
|
|
import org.apache.hadoop.ozone.common.statemachine
|
|
|
.InvalidStateTransitionException;
|
|
|
import org.apache.hadoop.ozone.common.statemachine.StateMachine;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
-
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
-import java.io.IOException;
|
|
|
-import java.util.HashSet;
|
|
|
-import java.util.List;
|
|
|
-import java.util.Map;
|
|
|
-import java.util.NavigableSet;
|
|
|
-import java.util.Set;
|
|
|
-import java.util.concurrent.ConcurrentHashMap;
|
|
|
-import java.util.concurrent.atomic.AtomicLong;
|
|
|
-
|
|
|
-import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
|
|
|
- .FAILED_TO_CHANGE_CONTAINER_STATE;
|
|
|
+import com.google.common.base.Preconditions;
|
|
|
+import com.google.common.util.concurrent.AtomicLongMap;
|
|
|
|
|
|
/**
|
|
|
* A container state manager keeps track of container states and returns
|
|
@@ -121,6 +121,8 @@ public class ContainerStateManager {
|
|
|
private final ConcurrentHashMap<ContainerState, ContainerID> lastUsedMap;
|
|
|
private final ContainerStateMap containers;
|
|
|
private final AtomicLong containerCount;
|
|
|
+ private final AtomicLongMap<LifeCycleState> containerStateCount =
|
|
|
+ AtomicLongMap.create();
|
|
|
|
|
|
/**
|
|
|
* Constructs a Container State Manager that tracks all containers owned by
|
|
@@ -224,11 +226,12 @@ public class ContainerStateManager {
|
|
|
LifeCycleEvent.CLEANUP);
|
|
|
}
|
|
|
|
|
|
- void loadContainer(final ContainerInfo containerInfo)
|
|
|
- throws SCMException {
|
|
|
+
|
|
|
+ void loadContainer(final ContainerInfo containerInfo) throws SCMException {
|
|
|
containers.addContainer(containerInfo);
|
|
|
containerCount.set(Long.max(
|
|
|
containerInfo.getContainerID(), containerCount.get()));
|
|
|
+ containerStateCount.incrementAndGet(containerInfo.getState());
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -297,6 +300,7 @@ public class ContainerStateManager {
|
|
|
ContainerID.valueof(containerID));
|
|
|
Preconditions.checkNotNull(containerInfo);
|
|
|
containers.addContainer(containerInfo);
|
|
|
+ containerStateCount.incrementAndGet(containerInfo.getState());
|
|
|
LOG.trace("New container allocated: {}", containerInfo);
|
|
|
return containerInfo;
|
|
|
}
|
|
@@ -317,6 +321,8 @@ public class ContainerStateManager {
|
|
|
final LifeCycleState newState = stateMachine.getNextState(
|
|
|
info.getState(), event);
|
|
|
containers.updateState(containerID, info.getState(), newState);
|
|
|
+ containerStateCount.incrementAndGet(newState);
|
|
|
+ containerStateCount.decrementAndGet(info.getState());
|
|
|
return containers.getContainerInfo(containerID);
|
|
|
} catch (InvalidStateTransitionException ex) {
|
|
|
String error = String.format("Failed to update container state %s, " +
|
|
@@ -440,6 +446,16 @@ public class ContainerStateManager {
|
|
|
return containers.getContainerIDsByState(state);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Get count of containers in the current {@link LifeCycleState}.
|
|
|
+ *
|
|
|
+ * @param state {@link LifeCycleState}
|
|
|
+ * @return Count of containers
|
|
|
+ */
|
|
|
+ Integer getContainerCountByState(final LifeCycleState state) {
|
|
|
+ return Long.valueOf(containerStateCount.get(state)).intValue();
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Returns a set of ContainerIDs that match the Container.
|
|
|
*
|
|
@@ -467,8 +483,6 @@ public class ContainerStateManager {
|
|
|
return containers.getContainerInfo(containerID);
|
|
|
}
|
|
|
|
|
|
-
|
|
|
-
|
|
|
void close() throws IOException {
|
|
|
}
|
|
|
|