|
@@ -46,7 +46,6 @@ import org.apache.hadoop.yarn.event.EventHandler;
|
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
import org.apache.hadoop.yarn.server.api.records.HeartbeatResponse;
|
|
import org.apache.hadoop.yarn.server.api.records.HeartbeatResponse;
|
|
-import org.apache.hadoop.yarn.server.api.records.MasterKey;
|
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.ClusterMetrics;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.ClusterMetrics;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.NodesListManagerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.NodesListManagerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.NodesListManagerEventType;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.NodesListManagerEventType;
|
|
@@ -105,8 +104,6 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
private HeartbeatResponse latestHeartBeatResponse = recordFactory
|
|
private HeartbeatResponse latestHeartBeatResponse = recordFactory
|
|
.newRecordInstance(HeartbeatResponse.class);
|
|
.newRecordInstance(HeartbeatResponse.class);
|
|
|
|
|
|
- private MasterKey currentMasterKey;
|
|
|
|
-
|
|
|
|
private static final StateMachineFactory<RMNodeImpl,
|
|
private static final StateMachineFactory<RMNodeImpl,
|
|
NodeState,
|
|
NodeState,
|
|
RMNodeEventType,
|
|
RMNodeEventType,
|
|
@@ -158,8 +155,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
RMNodeEvent> stateMachine;
|
|
RMNodeEvent> stateMachine;
|
|
|
|
|
|
public RMNodeImpl(NodeId nodeId, RMContext context, String hostName,
|
|
public RMNodeImpl(NodeId nodeId, RMContext context, String hostName,
|
|
- int cmPort, int httpPort, Node node, Resource capability,
|
|
|
|
- MasterKey masterKey) {
|
|
|
|
|
|
+ int cmPort, int httpPort, Node node, Resource capability) {
|
|
this.nodeId = nodeId;
|
|
this.nodeId = nodeId;
|
|
this.context = context;
|
|
this.context = context;
|
|
this.hostName = hostName;
|
|
this.hostName = hostName;
|
|
@@ -169,7 +165,6 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
this.nodeAddress = hostName + ":" + cmPort;
|
|
this.nodeAddress = hostName + ":" + cmPort;
|
|
this.httpAddress = hostName + ":" + httpPort;
|
|
this.httpAddress = hostName + ":" + httpPort;
|
|
this.node = node;
|
|
this.node = node;
|
|
- this.currentMasterKey = masterKey;
|
|
|
|
this.nodeHealthStatus.setIsNodeHealthy(true);
|
|
this.nodeHealthStatus.setIsNodeHealthy(true);
|
|
this.nodeHealthStatus.setHealthReport("Healthy");
|
|
this.nodeHealthStatus.setHealthReport("Healthy");
|
|
this.nodeHealthStatus.setLastHealthReportTime(System.currentTimeMillis());
|
|
this.nodeHealthStatus.setLastHealthReportTime(System.currentTimeMillis());
|
|
@@ -303,17 +298,6 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
this.readLock.unlock();
|
|
this.readLock.unlock();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
- @Override
|
|
|
|
- public MasterKey getCurrentMasterKey() {
|
|
|
|
- this.readLock.lock();
|
|
|
|
- try {
|
|
|
|
- return this.currentMasterKey;
|
|
|
|
- } finally {
|
|
|
|
- this.readLock.unlock();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
|
|
|
|
public void handle(RMNodeEvent event) {
|
|
public void handle(RMNodeEvent event) {
|
|
LOG.debug("Processing " + event.getNodeId() + " of type " + event.getType());
|
|
LOG.debug("Processing " + event.getNodeId() + " of type " + event.getType());
|
|
@@ -491,7 +475,6 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
|
|
|
|
// Switch the last heartbeatresponse.
|
|
// Switch the last heartbeatresponse.
|
|
rmNode.latestHeartBeatResponse = statusEvent.getLatestResponse();
|
|
rmNode.latestHeartBeatResponse = statusEvent.getLatestResponse();
|
|
- rmNode.currentMasterKey = statusEvent.getCurrentMasterKey();
|
|
|
|
|
|
|
|
NodeHealthStatus remoteNodeHealthStatus =
|
|
NodeHealthStatus remoteNodeHealthStatus =
|
|
statusEvent.getNodeHealthStatus();
|
|
statusEvent.getNodeHealthStatus();
|
|
@@ -573,7 +556,6 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
|
|
|
|
|
|
// Switch the last heartbeatresponse.
|
|
// Switch the last heartbeatresponse.
|
|
rmNode.latestHeartBeatResponse = statusEvent.getLatestResponse();
|
|
rmNode.latestHeartBeatResponse = statusEvent.getLatestResponse();
|
|
- rmNode.currentMasterKey = statusEvent.getCurrentMasterKey();
|
|
|
|
NodeHealthStatus remoteNodeHealthStatus = statusEvent.getNodeHealthStatus();
|
|
NodeHealthStatus remoteNodeHealthStatus = statusEvent.getNodeHealthStatus();
|
|
rmNode.setNodeHealthStatus(remoteNodeHealthStatus);
|
|
rmNode.setNodeHealthStatus(remoteNodeHealthStatus);
|
|
if (remoteNodeHealthStatus.getIsNodeHealthy()) {
|
|
if (remoteNodeHealthStatus.getIsNodeHealthy()) {
|