|
@@ -19,8 +19,8 @@ package org.apache.hadoop.hdds.scm.node;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
|
-import org.apache.hadoop.hdds.scm.HddsServerUtil;
|
|
|
+import org.apache.hadoop.hdds.scm.node.states.NodeAlreadyExistsException;
|
|
|
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
|
|
|
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
|
|
import org.apache.hadoop.hdds.scm.VersionInfo;
|
|
|
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
|
|
@@ -29,7 +29,6 @@ import org.apache.hadoop.hdds.server.events.Event;
|
|
|
import org.apache.hadoop.hdds.server.events.EventHandler;
|
|
|
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
|
|
import org.apache.hadoop.hdds.server.events.TypedEvent;
|
|
|
-import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
|
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
|
|
@@ -50,8 +49,6 @@ import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode;
|
|
|
import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
|
|
|
import org.apache.hadoop.ozone.protocol.commands.ReregisterCommand;
|
|
|
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
|
|
|
-import org.apache.hadoop.util.Time;
|
|
|
-import org.apache.hadoop.util.concurrent.HadoopExecutors;
|
|
|
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
@@ -63,39 +60,15 @@ import java.util.Collections;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
-import java.util.Queue;
|
|
|
import java.util.UUID;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
-import java.util.concurrent.ConcurrentLinkedQueue;
|
|
|
-import java.util.concurrent.ScheduledExecutorService;
|
|
|
-import java.util.concurrent.TimeUnit;
|
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
-import java.util.concurrent.atomic.AtomicInteger;
|
|
|
-import java.util.stream.Collectors;
|
|
|
-
|
|
|
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.DEAD;
|
|
|
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState
|
|
|
- .HEALTHY;
|
|
|
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState
|
|
|
- .INVALID;
|
|
|
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.STALE;
|
|
|
-import static org.apache.hadoop.util.Time.monotonicNow;
|
|
|
|
|
|
/**
|
|
|
* Maintains information about the Datanodes on SCM side.
|
|
|
* <p>
|
|
|
* Heartbeats under SCM is very simple compared to HDFS heartbeatManager.
|
|
|
* <p>
|
|
|
- * Here we maintain 3 maps, and we propagate a node from healthyNodesMap to
|
|
|
- * staleNodesMap to deadNodesMap. This moving of a node from one map to another
|
|
|
- * is controlled by 4 configuration variables. These variables define how many
|
|
|
- * heartbeats must go missing for the node to move from one map to another.
|
|
|
- * <p>
|
|
|
- * Each heartbeat that SCMNodeManager receives is put into heartbeatQueue. The
|
|
|
- * worker thread wakes up and grabs that heartbeat from the queue. The worker
|
|
|
- * thread will lookup the healthynodes map and set the timestamp if the entry
|
|
|
- * is there. if not it will look up stale and deadnodes map.
|
|
|
- * <p>
|
|
|
* The getNode(byState) functions make copy of node maps and then creates a list
|
|
|
* based on that. It should be assumed that these get functions always report
|
|
|
* *stale* information. For example, getting the deadNodeCount followed by
|
|
@@ -113,33 +86,18 @@ public class SCMNodeManager
|
|
|
static final Logger LOG =
|
|
|
LoggerFactory.getLogger(SCMNodeManager.class);
|
|
|
|
|
|
- /**
|
|
|
- * Key = NodeID, value = timestamp.
|
|
|
- */
|
|
|
- private final ConcurrentHashMap<UUID, Long> healthyNodes;
|
|
|
- private final ConcurrentHashMap<UUID, Long> staleNodes;
|
|
|
- private final ConcurrentHashMap<UUID, Long> deadNodes;
|
|
|
- private final Queue<HeartbeatQueueItem> heartbeatQueue;
|
|
|
- private final ConcurrentHashMap<UUID, DatanodeDetails> nodes;
|
|
|
+
|
|
|
+ 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
|
|
|
private SCMNodeStat scmStat;
|
|
|
- // TODO: expose nodeStats and scmStat as metrics
|
|
|
- private final AtomicInteger healthyNodeCount;
|
|
|
- private final AtomicInteger staleNodeCount;
|
|
|
- private final AtomicInteger deadNodeCount;
|
|
|
- private final AtomicInteger totalNodes;
|
|
|
- private long staleNodeIntervalMs;
|
|
|
- private final long deadNodeIntervalMs;
|
|
|
- private final long heartbeatCheckerIntervalMs;
|
|
|
- private final long datanodeHBIntervalSeconds;
|
|
|
- private final ScheduledExecutorService executorService;
|
|
|
- private long lastHBcheckStart;
|
|
|
- private long lastHBcheckFinished = 0;
|
|
|
- private long lastHBProcessedCount;
|
|
|
+ // Should we create ChillModeManager and extract all the chill mode logic
|
|
|
+ // to a new class?
|
|
|
private int chillModeNodeCount;
|
|
|
- private final int maxHBToProcessPerLoop;
|
|
|
private final String clusterID;
|
|
|
private final VersionInfo version;
|
|
|
/**
|
|
@@ -168,47 +126,19 @@ public class SCMNodeManager
|
|
|
*/
|
|
|
public SCMNodeManager(OzoneConfiguration conf, String clusterID,
|
|
|
StorageContainerManager scmManager) throws IOException {
|
|
|
- heartbeatQueue = new ConcurrentLinkedQueue<>();
|
|
|
- healthyNodes = new ConcurrentHashMap<>();
|
|
|
- deadNodes = new ConcurrentHashMap<>();
|
|
|
- staleNodes = new ConcurrentHashMap<>();
|
|
|
- nodes = new ConcurrentHashMap<>();
|
|
|
- nodeStats = new ConcurrentHashMap<>();
|
|
|
- scmStat = new SCMNodeStat();
|
|
|
-
|
|
|
- healthyNodeCount = new AtomicInteger(0);
|
|
|
- staleNodeCount = new AtomicInteger(0);
|
|
|
- deadNodeCount = new AtomicInteger(0);
|
|
|
- totalNodes = new AtomicInteger(0);
|
|
|
+ this.nodeStateManager = new NodeStateManager(conf);
|
|
|
+ this.nodeStats = new ConcurrentHashMap<>();
|
|
|
+ this.scmStat = new SCMNodeStat();
|
|
|
this.clusterID = clusterID;
|
|
|
this.version = VersionInfo.getLatestVersion();
|
|
|
- commandQueue = new CommandQueue();
|
|
|
-
|
|
|
+ this.commandQueue = new CommandQueue();
|
|
|
// TODO: Support this value as a Percentage of known machines.
|
|
|
- chillModeNodeCount = 1;
|
|
|
-
|
|
|
- staleNodeIntervalMs = HddsServerUtil.getStaleNodeInterval(conf);
|
|
|
- deadNodeIntervalMs = HddsServerUtil.getDeadNodeInterval(conf);
|
|
|
- heartbeatCheckerIntervalMs =
|
|
|
- HddsServerUtil.getScmheartbeatCheckerInterval(conf);
|
|
|
- datanodeHBIntervalSeconds = HddsServerUtil.getScmHeartbeatInterval(conf);
|
|
|
- maxHBToProcessPerLoop = HddsServerUtil.getMaxHBToProcessPerLoop(conf);
|
|
|
-
|
|
|
- executorService = HadoopExecutors.newScheduledThreadPool(1,
|
|
|
- new ThreadFactoryBuilder().setDaemon(true)
|
|
|
- .setNameFormat("SCM Heartbeat Processing Thread - %d").build());
|
|
|
-
|
|
|
- LOG.info("Entering startup chill mode.");
|
|
|
+ this.chillModeNodeCount = 1;
|
|
|
this.inStartupChillMode = new AtomicBoolean(true);
|
|
|
this.inManualChillMode = new AtomicBoolean(false);
|
|
|
-
|
|
|
- Preconditions.checkState(heartbeatCheckerIntervalMs > 0);
|
|
|
- executorService.schedule(this, heartbeatCheckerIntervalMs,
|
|
|
- TimeUnit.MILLISECONDS);
|
|
|
-
|
|
|
- registerMXBean();
|
|
|
-
|
|
|
this.scmManager = scmManager;
|
|
|
+ LOG.info("Entering startup chill mode.");
|
|
|
+ registerMXBean();
|
|
|
}
|
|
|
|
|
|
private void registerMXBean() {
|
|
@@ -227,12 +157,11 @@ public class SCMNodeManager
|
|
|
* Removes a data node from the management of this Node Manager.
|
|
|
*
|
|
|
* @param node - DataNode.
|
|
|
- * @throws UnregisteredNodeException
|
|
|
+ * @throws NodeNotFoundException
|
|
|
*/
|
|
|
@Override
|
|
|
- public void removeNode(DatanodeDetails node) {
|
|
|
- // TODO : Fix me when adding the SCM CLI.
|
|
|
-
|
|
|
+ public void removeNode(DatanodeDetails node) throws NodeNotFoundException {
|
|
|
+ nodeStateManager.removeNode(node);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -244,31 +173,8 @@ public class SCMNodeManager
|
|
|
* @return List of Datanodes that are known to SCM in the requested state.
|
|
|
*/
|
|
|
@Override
|
|
|
- public List<DatanodeDetails> getNodes(NodeState nodestate)
|
|
|
- throws IllegalArgumentException {
|
|
|
- Map<UUID, Long> set;
|
|
|
- switch (nodestate) {
|
|
|
- case HEALTHY:
|
|
|
- synchronized (this) {
|
|
|
- set = Collections.unmodifiableMap(new HashMap<>(healthyNodes));
|
|
|
- }
|
|
|
- break;
|
|
|
- case STALE:
|
|
|
- synchronized (this) {
|
|
|
- set = Collections.unmodifiableMap(new HashMap<>(staleNodes));
|
|
|
- }
|
|
|
- break;
|
|
|
- case DEAD:
|
|
|
- synchronized (this) {
|
|
|
- set = Collections.unmodifiableMap(new HashMap<>(deadNodes));
|
|
|
- }
|
|
|
- break;
|
|
|
- default:
|
|
|
- throw new IllegalArgumentException("Unknown node state requested.");
|
|
|
- }
|
|
|
-
|
|
|
- return set.entrySet().stream().map(entry -> nodes.get(entry.getKey()))
|
|
|
- .collect(Collectors.toList());
|
|
|
+ public List<DatanodeDetails> getNodes(NodeState nodestate) {
|
|
|
+ return nodeStateManager.getNodes(nodestate);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -278,12 +184,7 @@ public class SCMNodeManager
|
|
|
*/
|
|
|
@Override
|
|
|
public List<DatanodeDetails> getAllNodes() {
|
|
|
- Map<UUID, DatanodeDetails> set;
|
|
|
- synchronized (this) {
|
|
|
- set = Collections.unmodifiableMap(new HashMap<>(nodes));
|
|
|
- }
|
|
|
- return set.entrySet().stream().map(entry -> nodes.get(entry.getKey()))
|
|
|
- .collect(Collectors.toList());
|
|
|
+ return nodeStateManager.getAllNodes();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -315,14 +216,16 @@ public class SCMNodeManager
|
|
|
if (inStartupChillMode.get()) {
|
|
|
return "Still in chill mode, waiting on nodes to report in." +
|
|
|
String.format(" %d nodes reported, minimal %d nodes required.",
|
|
|
- totalNodes.get(), getMinimumChillModeNodes());
|
|
|
+ nodeStateManager.getTotalNodeCount(), getMinimumChillModeNodes());
|
|
|
}
|
|
|
if (inManualChillMode.get()) {
|
|
|
return "Out of startup chill mode, but in manual chill mode." +
|
|
|
- String.format(" %d nodes have reported in.", totalNodes.get());
|
|
|
+ String.format(" %d nodes have reported in.",
|
|
|
+ nodeStateManager.getTotalNodeCount());
|
|
|
}
|
|
|
return "Out of chill mode." +
|
|
|
- String.format(" %d nodes have reported in.", totalNodes.get());
|
|
|
+ String.format(" %d nodes have reported in.",
|
|
|
+ nodeStateManager.getTotalNodeCount());
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -376,33 +279,7 @@ public class SCMNodeManager
|
|
|
*/
|
|
|
@Override
|
|
|
public int getNodeCount(NodeState nodestate) {
|
|
|
- switch (nodestate) {
|
|
|
- case HEALTHY:
|
|
|
- return healthyNodeCount.get();
|
|
|
- case STALE:
|
|
|
- return staleNodeCount.get();
|
|
|
- case DEAD:
|
|
|
- return deadNodeCount.get();
|
|
|
- case INVALID:
|
|
|
- // This is unknown due to the fact that some nodes can be in
|
|
|
- // transit between the other states. Returning a count for that is not
|
|
|
- // possible. The fact that we have such state is to deal with the fact
|
|
|
- // that this information might not be consistent always.
|
|
|
- return 0;
|
|
|
- default:
|
|
|
- return 0;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Used for testing.
|
|
|
- *
|
|
|
- * @return true if the HB check is done.
|
|
|
- */
|
|
|
- @VisibleForTesting
|
|
|
- @Override
|
|
|
- public boolean waitForHeartbeatProcessed() {
|
|
|
- return lastHBcheckFinished != 0;
|
|
|
+ return nodeStateManager.getNodeCount(nodestate);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -413,236 +290,14 @@ public class SCMNodeManager
|
|
|
*/
|
|
|
@Override
|
|
|
public NodeState getNodeState(DatanodeDetails datanodeDetails) {
|
|
|
- // There is a subtle race condition here, hence we also support
|
|
|
- // the NODEState.UNKNOWN. It is possible that just before we check the
|
|
|
- // healthyNodes, we have removed the node from the healthy list but stil
|
|
|
- // not added it to Stale Nodes list.
|
|
|
- // We can fix that by adding the node to stale list before we remove, but
|
|
|
- // then the node is in 2 states to avoid this race condition. Instead we
|
|
|
- // just deal with the possibilty of getting a state called unknown.
|
|
|
-
|
|
|
- UUID id = datanodeDetails.getUuid();
|
|
|
- if(healthyNodes.containsKey(id)) {
|
|
|
- return HEALTHY;
|
|
|
- }
|
|
|
-
|
|
|
- if(staleNodes.containsKey(id)) {
|
|
|
- return STALE;
|
|
|
- }
|
|
|
-
|
|
|
- if(deadNodes.containsKey(id)) {
|
|
|
- return DEAD;
|
|
|
- }
|
|
|
-
|
|
|
- return INVALID;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * This is the real worker thread that processes the HB queue. We do the
|
|
|
- * following things in this thread.
|
|
|
- * <p>
|
|
|
- * Process the Heartbeats that are in the HB Queue. Move Stale or Dead node to
|
|
|
- * healthy if we got a heartbeat from them. Move Stales Node to dead node
|
|
|
- * table if it is needed. Move healthy nodes to stale nodes if it is needed.
|
|
|
- * <p>
|
|
|
- * if it is a new node, we call register node and add it to the list of nodes.
|
|
|
- * This will be replaced when we support registration of a node in SCM.
|
|
|
- *
|
|
|
- * @see Thread#run()
|
|
|
- */
|
|
|
- @Override
|
|
|
- public void run() {
|
|
|
- lastHBcheckStart = monotonicNow();
|
|
|
- lastHBProcessedCount = 0;
|
|
|
-
|
|
|
- // Process the whole queue.
|
|
|
- while (!heartbeatQueue.isEmpty() &&
|
|
|
- (lastHBProcessedCount < maxHBToProcessPerLoop)) {
|
|
|
- HeartbeatQueueItem hbItem = heartbeatQueue.poll();
|
|
|
- synchronized (this) {
|
|
|
- handleHeartbeat(hbItem);
|
|
|
- }
|
|
|
- // we are shutting down or something give up processing the rest of
|
|
|
- // HBs. This will terminate the HB processing thread.
|
|
|
- if (Thread.currentThread().isInterrupted()) {
|
|
|
- LOG.info("Current Thread is isInterrupted, shutting down HB " +
|
|
|
- "processing thread for Node Manager.");
|
|
|
- return;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- if (lastHBProcessedCount >= maxHBToProcessPerLoop) {
|
|
|
- LOG.error("SCM is being flooded by heartbeats. Not able to keep up with" +
|
|
|
- " the heartbeat counts. Processed {} heartbeats. Breaking out of" +
|
|
|
- " loop. Leaving rest to be processed later. ", lastHBProcessedCount);
|
|
|
- }
|
|
|
-
|
|
|
- // Iterate over the Stale nodes and decide if we need to move any node to
|
|
|
- // dead State.
|
|
|
- long currentTime = monotonicNow();
|
|
|
- for (Map.Entry<UUID, Long> entry : staleNodes.entrySet()) {
|
|
|
- if (currentTime - entry.getValue() > deadNodeIntervalMs) {
|
|
|
- synchronized (this) {
|
|
|
- moveStaleNodeToDead(entry);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // Iterate over the healthy nodes and decide if we need to move any node to
|
|
|
- // Stale State.
|
|
|
- currentTime = monotonicNow();
|
|
|
- for (Map.Entry<UUID, Long> entry : healthyNodes.entrySet()) {
|
|
|
- if (currentTime - entry.getValue() > staleNodeIntervalMs) {
|
|
|
- synchronized (this) {
|
|
|
- moveHealthyNodeToStale(entry);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- lastHBcheckFinished = monotonicNow();
|
|
|
-
|
|
|
- monitorHBProcessingTime();
|
|
|
-
|
|
|
- // we purposefully make this non-deterministic. Instead of using a
|
|
|
- // scheduleAtFixedFrequency we will just go to sleep
|
|
|
- // and wake up at the next rendezvous point, which is currentTime +
|
|
|
- // heartbeatCheckerIntervalMs. This leads to the issue that we are now
|
|
|
- // heart beating not at a fixed cadence, but clock tick + time taken to
|
|
|
- // work.
|
|
|
- //
|
|
|
- // This time taken to work can skew the heartbeat processor thread.
|
|
|
- // The reason why we don't care is because of the following reasons.
|
|
|
- //
|
|
|
- // 1. checkerInterval is general many magnitudes faster than datanode HB
|
|
|
- // frequency.
|
|
|
- //
|
|
|
- // 2. if we have too much nodes, the SCM would be doing only HB
|
|
|
- // processing, this could lead to SCM's CPU starvation. With this
|
|
|
- // approach we always guarantee that HB thread sleeps for a little while.
|
|
|
- //
|
|
|
- // 3. It is possible that we will never finish processing the HB's in the
|
|
|
- // thread. But that means we have a mis-configured system. We will warn
|
|
|
- // the users by logging that information.
|
|
|
- //
|
|
|
- // 4. And the most important reason, heartbeats are not blocked even if
|
|
|
- // this thread does not run, they will go into the processing queue.
|
|
|
-
|
|
|
- if (!Thread.currentThread().isInterrupted() &&
|
|
|
- !executorService.isShutdown()) {
|
|
|
- executorService.schedule(this, heartbeatCheckerIntervalMs, TimeUnit
|
|
|
- .MILLISECONDS);
|
|
|
- } else {
|
|
|
- LOG.info("Current Thread is interrupted, shutting down HB processing " +
|
|
|
- "thread for Node Manager.");
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * If we have taken too much time for HB processing, log that information.
|
|
|
- */
|
|
|
- private void monitorHBProcessingTime() {
|
|
|
- if (TimeUnit.MILLISECONDS.toSeconds(lastHBcheckFinished -
|
|
|
- lastHBcheckStart) > datanodeHBIntervalSeconds) {
|
|
|
- LOG.error("Total time spend processing datanode HB's is greater than " +
|
|
|
- "configured values for datanode heartbeats. Please adjust the" +
|
|
|
- " heartbeat configs. Time Spend on HB processing: {} seconds " +
|
|
|
- "Datanode heartbeat Interval: {} seconds , heartbeats " +
|
|
|
- "processed: {}",
|
|
|
- TimeUnit.MILLISECONDS
|
|
|
- .toSeconds(lastHBcheckFinished - lastHBcheckStart),
|
|
|
- datanodeHBIntervalSeconds, lastHBProcessedCount);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Moves a Healthy node to a Stale node state.
|
|
|
- *
|
|
|
- * @param entry - Map Entry
|
|
|
- */
|
|
|
- private void moveHealthyNodeToStale(Map.Entry<UUID, Long> entry) {
|
|
|
- LOG.trace("Moving healthy node to stale: {}", entry.getKey());
|
|
|
- healthyNodes.remove(entry.getKey());
|
|
|
- healthyNodeCount.decrementAndGet();
|
|
|
- staleNodes.put(entry.getKey(), entry.getValue());
|
|
|
- staleNodeCount.incrementAndGet();
|
|
|
-
|
|
|
- if (scmManager != null) {
|
|
|
- // remove stale node's container report
|
|
|
- scmManager.removeContainerReport(entry.getKey().toString());
|
|
|
+ try {
|
|
|
+ return nodeStateManager.getNodeState(datanodeDetails);
|
|
|
+ } catch (NodeNotFoundException e) {
|
|
|
+ // TODO: should we throw NodeNotFoundException?
|
|
|
+ return null;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Moves a Stale node to a dead node state.
|
|
|
- *
|
|
|
- * @param entry - Map Entry
|
|
|
- */
|
|
|
- private void moveStaleNodeToDead(Map.Entry<UUID, Long> entry) {
|
|
|
- LOG.trace("Moving stale node to dead: {}", entry.getKey());
|
|
|
- staleNodes.remove(entry.getKey());
|
|
|
- staleNodeCount.decrementAndGet();
|
|
|
- deadNodes.put(entry.getKey(), entry.getValue());
|
|
|
- deadNodeCount.incrementAndGet();
|
|
|
-
|
|
|
- // Update SCM node stats
|
|
|
- SCMNodeStat deadNodeStat = nodeStats.get(entry.getKey());
|
|
|
- scmStat.subtract(deadNodeStat);
|
|
|
- nodeStats.remove(entry.getKey());
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Handles a single heartbeat from a datanode.
|
|
|
- *
|
|
|
- * @param hbItem - heartbeat item from a datanode.
|
|
|
- */
|
|
|
- private void handleHeartbeat(HeartbeatQueueItem hbItem) {
|
|
|
- lastHBProcessedCount++;
|
|
|
-
|
|
|
- DatanodeDetails datanodeDetails = hbItem.getDatanodeDetails();
|
|
|
- UUID datanodeUuid = datanodeDetails.getUuid();
|
|
|
- NodeReportProto nodeReport = hbItem.getNodeReport();
|
|
|
- long recvTimestamp = hbItem.getRecvTimestamp();
|
|
|
- long processTimestamp = Time.monotonicNow();
|
|
|
- if (LOG.isTraceEnabled()) {
|
|
|
- //TODO: add average queue time of heartbeat request as metrics
|
|
|
- LOG.trace("Processing Heartbeat from datanode {}: queueing time {}",
|
|
|
- datanodeUuid, processTimestamp - recvTimestamp);
|
|
|
- }
|
|
|
-
|
|
|
- // If this node is already in the list of known and healthy nodes
|
|
|
- // just set the last timestamp and return.
|
|
|
- if (healthyNodes.containsKey(datanodeUuid)) {
|
|
|
- healthyNodes.put(datanodeUuid, processTimestamp);
|
|
|
- updateNodeStat(datanodeUuid, nodeReport);
|
|
|
- return;
|
|
|
- }
|
|
|
-
|
|
|
- // A stale node has heartbeat us we need to remove the node from stale
|
|
|
- // list and move to healthy list.
|
|
|
- if (staleNodes.containsKey(datanodeUuid)) {
|
|
|
- staleNodes.remove(datanodeUuid);
|
|
|
- healthyNodes.put(datanodeUuid, processTimestamp);
|
|
|
- healthyNodeCount.incrementAndGet();
|
|
|
- staleNodeCount.decrementAndGet();
|
|
|
- updateNodeStat(datanodeUuid, nodeReport);
|
|
|
- return;
|
|
|
- }
|
|
|
-
|
|
|
- // A dead node has heartbeat us, we need to remove that node from dead
|
|
|
- // node list and move it to the healthy list.
|
|
|
- if (deadNodes.containsKey(datanodeUuid)) {
|
|
|
- deadNodes.remove(datanodeUuid);
|
|
|
- healthyNodes.put(datanodeUuid, processTimestamp);
|
|
|
- deadNodeCount.decrementAndGet();
|
|
|
- healthyNodeCount.incrementAndGet();
|
|
|
- updateNodeStat(datanodeUuid, nodeReport);
|
|
|
- return;
|
|
|
- }
|
|
|
-
|
|
|
- LOG.warn("SCM receive heartbeat from unregistered datanode {}",
|
|
|
- datanodeUuid);
|
|
|
- this.commandQueue.addCommand(datanodeUuid,
|
|
|
- new ReregisterCommand());
|
|
|
- }
|
|
|
|
|
|
private void updateNodeStat(UUID dnId, NodeReportProto nodeReport) {
|
|
|
SCMNodeStat stat = nodeStats.get(dnId);
|
|
@@ -679,24 +334,6 @@ public class SCMNodeManager
|
|
|
@Override
|
|
|
public void close() throws IOException {
|
|
|
unregisterMXBean();
|
|
|
- executorService.shutdown();
|
|
|
- try {
|
|
|
- if (!executorService.awaitTermination(5, TimeUnit.SECONDS)) {
|
|
|
- executorService.shutdownNow();
|
|
|
- }
|
|
|
-
|
|
|
- if (!executorService.awaitTermination(5, TimeUnit.SECONDS)) {
|
|
|
- LOG.error("Unable to shutdown NodeManager properly.");
|
|
|
- }
|
|
|
- } catch (InterruptedException e) {
|
|
|
- executorService.shutdownNow();
|
|
|
- Thread.currentThread().interrupt();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- @VisibleForTesting
|
|
|
- long getLastHBProcessedCount() {
|
|
|
- return lastHBProcessedCount;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -739,27 +376,22 @@ public class SCMNodeManager
|
|
|
datanodeDetails.setHostName(hostname);
|
|
|
datanodeDetails.setIpAddress(ip);
|
|
|
}
|
|
|
- RegisteredCommand responseCommand = verifyDatanodeUUID(datanodeDetails);
|
|
|
- if (responseCommand != null) {
|
|
|
- return responseCommand;
|
|
|
- }
|
|
|
UUID dnId = datanodeDetails.getUuid();
|
|
|
- nodes.put(dnId, datanodeDetails);
|
|
|
- totalNodes.incrementAndGet();
|
|
|
- healthyNodes.put(dnId, monotonicNow());
|
|
|
- healthyNodeCount.incrementAndGet();
|
|
|
- nodeStats.put(dnId, new SCMNodeStat());
|
|
|
-
|
|
|
- if(inStartupChillMode.get() &&
|
|
|
- totalNodes.get() >= getMinimumChillModeNodes()) {
|
|
|
- inStartupChillMode.getAndSet(false);
|
|
|
- LOG.info("Leaving startup chill mode.");
|
|
|
+ try {
|
|
|
+ nodeStateManager.addNode(datanodeDetails);
|
|
|
+ nodeStats.put(dnId, new SCMNodeStat());
|
|
|
+ if(inStartupChillMode.get() &&
|
|
|
+ nodeStateManager.getTotalNodeCount() >= getMinimumChillModeNodes()) {
|
|
|
+ inStartupChillMode.getAndSet(false);
|
|
|
+ LOG.info("Leaving startup chill mode.");
|
|
|
+ }
|
|
|
+ // Updating Node Report, as registration is successful
|
|
|
+ updateNodeStat(datanodeDetails.getUuid(), nodeReport);
|
|
|
+ LOG.info("Data node with ID: {} Registered.", datanodeDetails.getUuid());
|
|
|
+ } catch (NodeAlreadyExistsException e) {
|
|
|
+ LOG.trace("Datanode is already registered. Datanode: {}",
|
|
|
+ datanodeDetails.toString());
|
|
|
}
|
|
|
-
|
|
|
- // Updating Node Report, as registration is successful
|
|
|
- updateNodeStat(datanodeDetails.getUuid(), nodeReport);
|
|
|
- LOG.info("Data node with ID: {} Registered.",
|
|
|
- datanodeDetails.getUuid());
|
|
|
RegisteredCommand.Builder builder =
|
|
|
RegisteredCommand.newBuilder().setErrorCode(ErrorCode.success)
|
|
|
.setDatanodeUUID(datanodeDetails.getUuidString())
|
|
@@ -770,46 +402,25 @@ public class SCMNodeManager
|
|
|
return builder.build();
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Verifies the datanode does not have a valid UUID already.
|
|
|
- *
|
|
|
- * @param datanodeDetails - Datanode Details.
|
|
|
- * @return SCMCommand
|
|
|
- */
|
|
|
- private RegisteredCommand verifyDatanodeUUID(
|
|
|
- DatanodeDetails datanodeDetails) {
|
|
|
- if (datanodeDetails.getUuid() != null &&
|
|
|
- nodes.containsKey(datanodeDetails.getUuid())) {
|
|
|
- LOG.trace("Datanode is already registered. Datanode: {}",
|
|
|
- datanodeDetails.toString());
|
|
|
- return RegisteredCommand.newBuilder()
|
|
|
- .setErrorCode(ErrorCode.success)
|
|
|
- .setClusterID(this.clusterID)
|
|
|
- .setDatanodeUUID(datanodeDetails.getUuidString())
|
|
|
- .build();
|
|
|
- }
|
|
|
- return null;
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Send heartbeat to indicate the datanode is alive and doing well.
|
|
|
*
|
|
|
* @param datanodeDetails - DatanodeDetailsProto.
|
|
|
- * @param nodeReport - node report.
|
|
|
* @return SCMheartbeat response.
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
@Override
|
|
|
- public List<SCMCommand> sendHeartbeat(
|
|
|
- DatanodeDetails datanodeDetails, NodeReportProto nodeReport) {
|
|
|
-
|
|
|
+ public List<SCMCommand> processHeartbeat(DatanodeDetails datanodeDetails) {
|
|
|
Preconditions.checkNotNull(datanodeDetails, "Heartbeat is missing " +
|
|
|
"DatanodeDetails.");
|
|
|
- heartbeatQueue.add(
|
|
|
- new HeartbeatQueueItem.Builder()
|
|
|
- .setDatanodeDetails(datanodeDetails)
|
|
|
- .setNodeReport(nodeReport)
|
|
|
- .build());
|
|
|
+ try {
|
|
|
+ nodeStateManager.updateLastHeartbeatTime(datanodeDetails);
|
|
|
+ } catch (NodeNotFoundException e) {
|
|
|
+ LOG.warn("SCM receive heartbeat from unregistered datanode {}",
|
|
|
+ datanodeDetails);
|
|
|
+ commandQueue.addCommand(datanodeDetails.getUuid(),
|
|
|
+ new ReregisterCommand());
|
|
|
+ }
|
|
|
return commandQueue.getCommand(datanodeDetails.getUuid());
|
|
|
}
|
|
|
|
|
@@ -855,11 +466,6 @@ public class SCMNodeManager
|
|
|
this.commandQueue.addCommand(dnId, command);
|
|
|
}
|
|
|
|
|
|
- @VisibleForTesting
|
|
|
- public void setStaleNodeIntervalMs(long interval) {
|
|
|
- this.staleNodeIntervalMs = interval;
|
|
|
- }
|
|
|
-
|
|
|
@Override
|
|
|
public void onMessage(CommandForDatanode commandForDatanode,
|
|
|
EventPublisher publisher) {
|