|
@@ -17,7 +17,6 @@
|
|
*/
|
|
*/
|
|
package org.apache.hadoop.yarn.server.resourcemanager;
|
|
package org.apache.hadoop.yarn.server.resourcemanager;
|
|
|
|
|
|
-import java.io.IOException;
|
|
|
|
import java.net.InetSocketAddress;
|
|
import java.net.InetSocketAddress;
|
|
import java.nio.ByteBuffer;
|
|
import java.nio.ByteBuffer;
|
|
|
|
|
|
@@ -37,7 +36,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
|
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
|
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.ipc.RPCUtil;
|
|
|
|
import org.apache.hadoop.yarn.ipc.YarnRPC;
|
|
import org.apache.hadoop.yarn.ipc.YarnRPC;
|
|
import org.apache.hadoop.yarn.server.api.ResourceTracker;
|
|
import org.apache.hadoop.yarn.server.api.ResourceTracker;
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
|
|
@@ -45,6 +43,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
|
|
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.NodeAction;
|
|
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
|
|
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
|
|
import org.apache.hadoop.yarn.server.api.records.RegistrationResponse;
|
|
import org.apache.hadoop.yarn.server.api.records.RegistrationResponse;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store.RMState;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store.RMState;
|
|
@@ -76,11 +75,19 @@ public class ResourceTrackerService extends AbstractService implements
|
|
|
|
|
|
private static final NodeHeartbeatResponse reboot = recordFactory
|
|
private static final NodeHeartbeatResponse reboot = recordFactory
|
|
.newRecordInstance(NodeHeartbeatResponse.class);
|
|
.newRecordInstance(NodeHeartbeatResponse.class);
|
|
|
|
+ private static final NodeHeartbeatResponse shutDown = recordFactory
|
|
|
|
+ .newRecordInstance(NodeHeartbeatResponse.class);
|
|
|
|
+
|
|
static {
|
|
static {
|
|
HeartbeatResponse rebootResp = recordFactory
|
|
HeartbeatResponse rebootResp = recordFactory
|
|
.newRecordInstance(HeartbeatResponse.class);
|
|
.newRecordInstance(HeartbeatResponse.class);
|
|
- rebootResp.setReboot(true);
|
|
|
|
|
|
+ rebootResp.setNodeAction(NodeAction.REBOOT);
|
|
reboot.setHeartbeatResponse(rebootResp);
|
|
reboot.setHeartbeatResponse(rebootResp);
|
|
|
|
+
|
|
|
|
+ HeartbeatResponse decommissionedResp = recordFactory
|
|
|
|
+ .newRecordInstance(HeartbeatResponse.class);
|
|
|
|
+ decommissionedResp.setNodeAction(NodeAction.SHUTDOWN);
|
|
|
|
+ shutDown.setHeartbeatResponse(decommissionedResp);
|
|
}
|
|
}
|
|
|
|
|
|
public ResourceTrackerService(RMContext rmContext,
|
|
public ResourceTrackerService(RMContext rmContext,
|
|
@@ -139,6 +146,7 @@ public class ResourceTrackerService extends AbstractService implements
|
|
super.stop();
|
|
super.stop();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @SuppressWarnings("unchecked")
|
|
@Override
|
|
@Override
|
|
public RegisterNodeManagerResponse registerNodeManager(
|
|
public RegisterNodeManagerResponse registerNodeManager(
|
|
RegisterNodeManagerRequest request) throws YarnRemoteException {
|
|
RegisterNodeManagerRequest request) throws YarnRemoteException {
|
|
@@ -149,121 +157,125 @@ public class ResourceTrackerService extends AbstractService implements
|
|
int httpPort = request.getHttpPort();
|
|
int httpPort = request.getHttpPort();
|
|
Resource capability = request.getResource();
|
|
Resource capability = request.getResource();
|
|
|
|
|
|
- try {
|
|
|
|
- // Check if this node is a 'valid' node
|
|
|
|
- if (!this.nodesListManager.isValidNode(host)) {
|
|
|
|
- LOG.info("Disallowed NodeManager from " + host);
|
|
|
|
- throw new IOException("Disallowed NodeManager from " + host);
|
|
|
|
- }
|
|
|
|
|
|
+ RegisterNodeManagerResponse response = recordFactory
|
|
|
|
+ .newRecordInstance(RegisterNodeManagerResponse.class);
|
|
|
|
+ RegistrationResponse regResponse = recordFactory
|
|
|
|
+ .newRecordInstance(RegistrationResponse.class);
|
|
|
|
+ SecretKey secretKey = this.containerTokenSecretManager
|
|
|
|
+ .createAndGetSecretKey(nodeId.toString());
|
|
|
|
+ regResponse.setSecretKey(ByteBuffer.wrap(secretKey.getEncoded()));
|
|
|
|
+
|
|
|
|
+ // Check if this node is a 'valid' node
|
|
|
|
+ if (!this.nodesListManager.isValidNode(host)) {
|
|
|
|
+ LOG.info("Disallowed NodeManager from " + host
|
|
|
|
+ + ", Sending SHUTDOWN signal to the NodeManager.");
|
|
|
|
+ regResponse.setNodeAction(NodeAction.SHUTDOWN);
|
|
|
|
+ response.setRegistrationResponse(regResponse);
|
|
|
|
+ return response;
|
|
|
|
+ }
|
|
|
|
|
|
- RMNode rmNode = new RMNodeImpl(nodeId, rmContext, host, cmPort,
|
|
|
|
- httpPort, resolve(host), capability);
|
|
|
|
|
|
+ RMNode rmNode = new RMNodeImpl(nodeId, rmContext, host, cmPort, httpPort,
|
|
|
|
+ resolve(host), capability);
|
|
|
|
|
|
- if (this.rmContext.getRMNodes().putIfAbsent(nodeId, rmNode) != null) {
|
|
|
|
- throw new IOException("Duplicate registration from the node!");
|
|
|
|
- }
|
|
|
|
|
|
+ if (this.rmContext.getRMNodes().putIfAbsent(nodeId, rmNode) != null) {
|
|
|
|
+ LOG.info("Duplicate registration from the node at: " + host
|
|
|
|
+ + ", Sending SHUTDOWN Signal to the NodeManager");
|
|
|
|
+ regResponse.setNodeAction(NodeAction.SHUTDOWN);
|
|
|
|
+ response.setRegistrationResponse(regResponse);
|
|
|
|
+ return response;
|
|
|
|
+ }
|
|
|
|
|
|
- this.nmLivelinessMonitor.register(nodeId);
|
|
|
|
|
|
+ this.rmContext.getDispatcher().getEventHandler().handle(
|
|
|
|
+ new RMNodeEvent(nodeId, RMNodeEventType.STARTED));
|
|
|
|
|
|
- LOG.info("NodeManager from node " + host +
|
|
|
|
- "(cmPort: " + cmPort + " httpPort: " + httpPort + ") "
|
|
|
|
- + "registered with capability: " + capability.getMemory()
|
|
|
|
- + ", assigned nodeId " + nodeId);
|
|
|
|
|
|
+ this.nmLivelinessMonitor.register(nodeId);
|
|
|
|
|
|
- RegistrationResponse regResponse = recordFactory.newRecordInstance(
|
|
|
|
- RegistrationResponse.class);
|
|
|
|
- SecretKey secretKey = this.containerTokenSecretManager
|
|
|
|
- .createAndGetSecretKey(nodeId.toString());
|
|
|
|
- regResponse.setSecretKey(ByteBuffer.wrap(secretKey.getEncoded()));
|
|
|
|
|
|
+ LOG.info("NodeManager from node " + host + "(cmPort: " + cmPort
|
|
|
|
+ + " httpPort: " + httpPort + ") " + "registered with capability: "
|
|
|
|
+ + capability.getMemory() + ", assigned nodeId " + nodeId);
|
|
|
|
|
|
- RegisterNodeManagerResponse response = recordFactory
|
|
|
|
- .newRecordInstance(RegisterNodeManagerResponse.class);
|
|
|
|
- response.setRegistrationResponse(regResponse);
|
|
|
|
- return response;
|
|
|
|
- } catch (IOException ioe) {
|
|
|
|
- LOG.info("Exception in node registration from " + nodeId.getHost(), ioe);
|
|
|
|
- throw RPCUtil.getRemoteException(ioe);
|
|
|
|
- }
|
|
|
|
|
|
+ regResponse.setNodeAction(NodeAction.NORMAL);
|
|
|
|
+ response.setRegistrationResponse(regResponse);
|
|
|
|
+ return response;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @SuppressWarnings("unchecked")
|
|
@Override
|
|
@Override
|
|
public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request)
|
|
public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request)
|
|
throws YarnRemoteException {
|
|
throws YarnRemoteException {
|
|
|
|
|
|
NodeStatus remoteNodeStatus = request.getNodeStatus();
|
|
NodeStatus remoteNodeStatus = request.getNodeStatus();
|
|
- try {
|
|
|
|
- /**
|
|
|
|
- * Here is the node heartbeat sequence...
|
|
|
|
- * 1. Check if it's a registered node
|
|
|
|
- * 2. Check if it's a valid (i.e. not excluded) node
|
|
|
|
- * 3. Check if it's a 'fresh' heartbeat i.e. not duplicate heartbeat
|
|
|
|
- * 4. Send healthStatus to RMNode
|
|
|
|
- */
|
|
|
|
-
|
|
|
|
- NodeId nodeId = remoteNodeStatus.getNodeId();
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Here is the node heartbeat sequence...
|
|
|
|
+ * 1. Check if it's a registered node
|
|
|
|
+ * 2. Check if it's a valid (i.e. not excluded) node
|
|
|
|
+ * 3. Check if it's a 'fresh' heartbeat i.e. not duplicate heartbeat
|
|
|
|
+ * 4. Send healthStatus to RMNode
|
|
|
|
+ */
|
|
|
|
+
|
|
|
|
+ NodeId nodeId = remoteNodeStatus.getNodeId();
|
|
|
|
+
|
|
|
|
+ // 1. Check if it's a registered node
|
|
|
|
+ RMNode rmNode = this.rmContext.getRMNodes().get(nodeId);
|
|
|
|
+ if (rmNode == null) {
|
|
|
|
+ /* node does not exist */
|
|
|
|
+ LOG.info("Node not found rebooting " + remoteNodeStatus.getNodeId());
|
|
|
|
|
|
- // 1. Check if it's a registered node
|
|
|
|
- RMNode rmNode = this.rmContext.getRMNodes().get(nodeId);
|
|
|
|
- if (rmNode == null) {
|
|
|
|
- /* node does not exist */
|
|
|
|
- LOG.info("Node not found rebooting " + remoteNodeStatus.getNodeId());
|
|
|
|
- return reboot;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // Send ping
|
|
|
|
- this.nmLivelinessMonitor.receivedPing(nodeId);
|
|
|
|
-
|
|
|
|
- // 2. Check if it's a valid (i.e. not excluded) node
|
|
|
|
- if (!this.nodesListManager.isValidNode(rmNode.getHostName())) {
|
|
|
|
- LOG.info("Disallowed NodeManager nodeId: " + nodeId +
|
|
|
|
- " hostname: " + rmNode.getNodeAddress());
|
|
|
|
- throw new IOException("Disallowed NodeManager nodeId: " +
|
|
|
|
- remoteNodeStatus.getNodeId());
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- NodeHeartbeatResponse nodeHeartBeatResponse = recordFactory
|
|
|
|
- .newRecordInstance(NodeHeartbeatResponse.class);
|
|
|
|
-
|
|
|
|
- // 3. Check if it's a 'fresh' heartbeat i.e. not duplicate heartbeat
|
|
|
|
- HeartbeatResponse lastHeartbeatResponse = rmNode
|
|
|
|
- .getLastHeartBeatResponse();
|
|
|
|
- if (remoteNodeStatus.getResponseId() + 1 == lastHeartbeatResponse
|
|
|
|
- .getResponseId()) {
|
|
|
|
- LOG.info("Received duplicate heartbeat from node " +
|
|
|
|
- rmNode.getNodeAddress());
|
|
|
|
- nodeHeartBeatResponse.setHeartbeatResponse(lastHeartbeatResponse);
|
|
|
|
- return nodeHeartBeatResponse;
|
|
|
|
- } else if (remoteNodeStatus.getResponseId() + 1 < lastHeartbeatResponse
|
|
|
|
- .getResponseId()) {
|
|
|
|
- LOG.info("Too far behind rm response id:" +
|
|
|
|
- lastHeartbeatResponse.getResponseId() + " nm response id:"
|
|
|
|
- + remoteNodeStatus.getResponseId());
|
|
|
|
- // TODO: Just sending reboot is not enough. Think more.
|
|
|
|
- this.rmContext.getDispatcher().getEventHandler().handle(
|
|
|
|
- new RMNodeEvent(nodeId, RMNodeEventType.REBOOTING));
|
|
|
|
- return reboot;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // Heartbeat response
|
|
|
|
- HeartbeatResponse latestResponse = recordFactory
|
|
|
|
- .newRecordInstance(HeartbeatResponse.class);
|
|
|
|
- latestResponse
|
|
|
|
- .setResponseId(lastHeartbeatResponse.getResponseId() + 1);
|
|
|
|
- latestResponse.addAllContainersToCleanup(rmNode.pullContainersToCleanUp());
|
|
|
|
- latestResponse.addAllApplicationsToCleanup(rmNode.pullAppsToCleanup());
|
|
|
|
-
|
|
|
|
- // 4. Send status to RMNode, saving the latest response.
|
|
|
|
|
|
+ // Updating the metrics directly as reboot event cannot be
|
|
|
|
+ // triggered on a null rmNode
|
|
|
|
+ ClusterMetrics.getMetrics().incrNumRebootedNMs();
|
|
|
|
+ return reboot;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // Send ping
|
|
|
|
+ this.nmLivelinessMonitor.receivedPing(nodeId);
|
|
|
|
+
|
|
|
|
+ // 2. Check if it's a valid (i.e. not excluded) node
|
|
|
|
+ if (!this.nodesListManager.isValidNode(rmNode.getHostName())) {
|
|
|
|
+ LOG.info("Disallowed NodeManager nodeId: " + nodeId + " hostname: "
|
|
|
|
+ + rmNode.getNodeAddress());
|
|
this.rmContext.getDispatcher().getEventHandler().handle(
|
|
this.rmContext.getDispatcher().getEventHandler().handle(
|
|
- new RMNodeStatusEvent(nodeId, remoteNodeStatus.getNodeHealthStatus(),
|
|
|
|
- remoteNodeStatus.getContainersStatuses(), latestResponse));
|
|
|
|
|
|
+ new RMNodeEvent(nodeId, RMNodeEventType.DECOMMISSION));
|
|
|
|
+ return shutDown;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ NodeHeartbeatResponse nodeHeartBeatResponse = recordFactory
|
|
|
|
+ .newRecordInstance(NodeHeartbeatResponse.class);
|
|
|
|
|
|
- nodeHeartBeatResponse.setHeartbeatResponse(latestResponse);
|
|
|
|
|
|
+ // 3. Check if it's a 'fresh' heartbeat i.e. not duplicate heartbeat
|
|
|
|
+ HeartbeatResponse lastHeartbeatResponse = rmNode.getLastHeartBeatResponse();
|
|
|
|
+ if (remoteNodeStatus.getResponseId() + 1 == lastHeartbeatResponse
|
|
|
|
+ .getResponseId()) {
|
|
|
|
+ LOG.info("Received duplicate heartbeat from node "
|
|
|
|
+ + rmNode.getNodeAddress());
|
|
|
|
+ nodeHeartBeatResponse.setHeartbeatResponse(lastHeartbeatResponse);
|
|
return nodeHeartBeatResponse;
|
|
return nodeHeartBeatResponse;
|
|
- } catch (IOException ioe) {
|
|
|
|
- LOG.info("Exception in heartbeat from node " +
|
|
|
|
- request.getNodeStatus().getNodeId(), ioe);
|
|
|
|
- throw RPCUtil.getRemoteException(ioe);
|
|
|
|
|
|
+ } else if (remoteNodeStatus.getResponseId() + 1 < lastHeartbeatResponse
|
|
|
|
+ .getResponseId()) {
|
|
|
|
+ LOG.info("Too far behind rm response id:"
|
|
|
|
+ + lastHeartbeatResponse.getResponseId() + " nm response id:"
|
|
|
|
+ + remoteNodeStatus.getResponseId());
|
|
|
|
+ // TODO: Just sending reboot is not enough. Think more.
|
|
|
|
+ this.rmContext.getDispatcher().getEventHandler().handle(
|
|
|
|
+ new RMNodeEvent(nodeId, RMNodeEventType.REBOOTING));
|
|
|
|
+ return reboot;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ // Heartbeat response
|
|
|
|
+ HeartbeatResponse latestResponse = recordFactory
|
|
|
|
+ .newRecordInstance(HeartbeatResponse.class);
|
|
|
|
+ latestResponse.setResponseId(lastHeartbeatResponse.getResponseId() + 1);
|
|
|
|
+ latestResponse.addAllContainersToCleanup(rmNode.pullContainersToCleanUp());
|
|
|
|
+ latestResponse.addAllApplicationsToCleanup(rmNode.pullAppsToCleanup());
|
|
|
|
+ latestResponse.setNodeAction(NodeAction.NORMAL);
|
|
|
|
+
|
|
|
|
+ // 4. Send status to RMNode, saving the latest response.
|
|
|
|
+ this.rmContext.getDispatcher().getEventHandler().handle(
|
|
|
|
+ new RMNodeStatusEvent(nodeId, remoteNodeStatus.getNodeHealthStatus(),
|
|
|
|
+ remoteNodeStatus.getContainersStatuses(), latestResponse));
|
|
|
|
+
|
|
|
|
+ nodeHeartBeatResponse.setHeartbeatResponse(latestResponse);
|
|
|
|
+ return nodeHeartBeatResponse;
|
|
}
|
|
}
|
|
|
|
|
|
public void recover(RMState state) {
|
|
public void recover(RMState state) {
|