|
@@ -56,12 +56,10 @@ import org.slf4j.LoggerFactory;
|
|
|
import javax.management.ObjectName;
|
|
|
import java.io.IOException;
|
|
|
import java.net.InetAddress;
|
|
|
-import java.util.Collections;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.UUID;
|
|
|
-import java.util.concurrent.ConcurrentHashMap;
|
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
|
|
|
/**
|
|
@@ -85,11 +83,7 @@ public class SCMNodeManager
|
|
|
static final Logger LOG =
|
|
|
LoggerFactory.getLogger(SCMNodeManager.class);
|
|
|
|
|
|
-
|
|
|
private final NodeStateManager nodeStateManager;
|
|
|
- // Individual live node stats
|
|
|
- // TODO: NodeStat should be moved to NodeStatemanager (NodeStateMap)
|
|
|
- private final ConcurrentHashMap<UUID, SCMNodeStat> nodeStats;
|
|
|
// Should we maintain aggregated stats? If this is not frequently used, we
|
|
|
// can always calculate it from nodeStats whenever required.
|
|
|
// Aggregated node stats
|
|
@@ -124,7 +118,6 @@ public class SCMNodeManager
|
|
|
StorageContainerManager scmManager, EventPublisher eventPublisher)
|
|
|
throws IOException {
|
|
|
this.nodeStateManager = new NodeStateManager(conf, eventPublisher);
|
|
|
- this.nodeStats = new ConcurrentHashMap<>();
|
|
|
this.scmStat = new SCMNodeStat();
|
|
|
this.clusterID = clusterID;
|
|
|
this.version = VersionInfo.getLatestVersion();
|
|
@@ -297,8 +290,10 @@ public class SCMNodeManager
|
|
|
|
|
|
|
|
|
private void updateNodeStat(UUID dnId, NodeReportProto nodeReport) {
|
|
|
- SCMNodeStat stat = nodeStats.get(dnId);
|
|
|
- if (stat == null) {
|
|
|
+ SCMNodeStat stat;
|
|
|
+ try {
|
|
|
+ stat = nodeStateManager.getNodeStat(dnId);
|
|
|
+ } catch (NodeNotFoundException e) {
|
|
|
LOG.debug("SCM updateNodeStat based on heartbeat from previous" +
|
|
|
"dead datanode {}", dnId);
|
|
|
stat = new SCMNodeStat();
|
|
@@ -317,9 +312,9 @@ public class SCMNodeManager
|
|
|
}
|
|
|
scmStat.subtract(stat);
|
|
|
stat.set(totalCapacity, totalScmUsed, totalRemaining);
|
|
|
- nodeStats.put(dnId, stat);
|
|
|
scmStat.add(stat);
|
|
|
}
|
|
|
+ nodeStateManager.setNodeStat(dnId, stat);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -377,7 +372,7 @@ public class SCMNodeManager
|
|
|
UUID dnId = datanodeDetails.getUuid();
|
|
|
try {
|
|
|
nodeStateManager.addNode(datanodeDetails);
|
|
|
- nodeStats.put(dnId, new SCMNodeStat());
|
|
|
+ nodeStateManager.setNodeStat(dnId, new SCMNodeStat());
|
|
|
if(inStartupChillMode.get() &&
|
|
|
nodeStateManager.getTotalNodeCount() >= getMinimumChillModeNodes()) {
|
|
|
inStartupChillMode.getAndSet(false);
|
|
@@ -446,17 +441,25 @@ public class SCMNodeManager
|
|
|
*/
|
|
|
@Override
|
|
|
public Map<UUID, SCMNodeStat> getNodeStats() {
|
|
|
- return Collections.unmodifiableMap(nodeStats);
|
|
|
+ return nodeStateManager.getNodeStatsMap();
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Return the node stat of the specified datanode.
|
|
|
* @param datanodeDetails - datanode ID.
|
|
|
- * @return node stat if it is live/stale, null if it is dead or does't exist.
|
|
|
+ * @return node stat if it is live/stale, null if it is decommissioned or
|
|
|
+ * doesn't exist.
|
|
|
*/
|
|
|
@Override
|
|
|
public SCMNodeMetric getNodeStat(DatanodeDetails datanodeDetails) {
|
|
|
- return new SCMNodeMetric(nodeStats.get(datanodeDetails.getUuid()));
|
|
|
+ try {
|
|
|
+ return new SCMNodeMetric(
|
|
|
+ nodeStateManager.getNodeStat(datanodeDetails.getUuid()));
|
|
|
+ } catch (NodeNotFoundException e) {
|
|
|
+ LOG.info("SCM getNodeStat from a decommissioned or removed datanode {}",
|
|
|
+ datanodeDetails.getUuid());
|
|
|
+ return null;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -492,18 +495,22 @@ public class SCMNodeManager
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Remove the node stats and update the storage stats
|
|
|
- * in this SCM Node Manager.
|
|
|
+ * Update the node stats and cluster storage stats in this SCM Node Manager.
|
|
|
*
|
|
|
* @param dnUuid datanode uuid.
|
|
|
*/
|
|
|
@Override
|
|
|
public void processDeadNode(UUID dnUuid) {
|
|
|
- SCMNodeStat stat = nodeStats.get(dnUuid);
|
|
|
- LOG.trace("Update stat values as Datanode {} is dead.", dnUuid);
|
|
|
- if (stat != null) {
|
|
|
- scmStat.subtract(stat);
|
|
|
- stat.set(0, 0, 0);
|
|
|
+ try {
|
|
|
+ SCMNodeStat stat = nodeStateManager.getNodeStat(dnUuid);
|
|
|
+ if (stat != null) {
|
|
|
+ LOG.trace("Update stat values as Datanode {} is dead.", dnUuid);
|
|
|
+ scmStat.subtract(stat);
|
|
|
+ stat.set(0, 0, 0);
|
|
|
+ }
|
|
|
+ } catch (NodeNotFoundException e) {
|
|
|
+ LOG.warn("Can't update stats based on message of dead Datanode {}, it"
|
|
|
+ + " doesn't exist or decommissioned already.", dnUuid);
|
|
|
}
|
|
|
}
|
|
|
}
|