|
@@ -71,13 +71,14 @@ 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.RegisterNodeManagerResponse;
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerRequest;
|
|
|
+
|
|
|
+import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
|
|
|
import org.apache.hadoop.yarn.server.api.records.QueuedContainersStatus;
|
|
|
import org.apache.hadoop.yarn.server.api.records.MasterKey;
|
|
|
import org.apache.hadoop.yarn.server.api.records.NodeAction;
|
|
|
import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
|
|
|
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext;
|
|
|
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
|
|
@@ -401,8 +402,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|
|
|
|
|
LOG.info(successfullRegistrationMsg);
|
|
|
LOG.info("Notifying ContainerManager to unblock new container-requests");
|
|
|
- ((ContainerManagerImpl) this.context.getContainerManager())
|
|
|
- .setBlockNewContainerRequests(false);
|
|
|
+ this.context.getContainerManager().setBlockNewContainerRequests(false);
|
|
|
}
|
|
|
|
|
|
private List<ApplicationId> createKeepAliveApplicationList() {
|
|
@@ -465,10 +465,8 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|
|
* @return Resource utilization of all the containers.
|
|
|
*/
|
|
|
private ResourceUtilization getContainersUtilization() {
|
|
|
- ContainerManagerImpl containerManager =
|
|
|
- (ContainerManagerImpl) this.context.getContainerManager();
|
|
|
ContainersMonitor containersMonitor =
|
|
|
- containerManager.getContainersMonitor();
|
|
|
+ this.context.getContainerManager().getContainersMonitor();
|
|
|
return containersMonitor.getContainersUtilization();
|
|
|
}
|
|
|
|
|
@@ -735,7 +733,6 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|
|
Set<NodeLabel> nodeLabelsForHeartbeat =
|
|
|
nodeLabelsHandler.getNodeLabelsForHeartbeat();
|
|
|
NodeStatus nodeStatus = getNodeStatus(lastHeartbeatID);
|
|
|
-
|
|
|
NodeHeartbeatRequest request =
|
|
|
NodeHeartbeatRequest.newInstance(nodeStatus,
|
|
|
NodeStatusUpdaterImpl.this.context
|
|
@@ -760,82 +757,70 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|
|
nextHeartBeatInterval = response.getNextHeartBeatInterval();
|
|
|
updateMasterKeys(response);
|
|
|
|
|
|
- if (response.getNodeAction() == NodeAction.SHUTDOWN) {
|
|
|
- LOG.warn("Recieved SHUTDOWN signal from Resourcemanager as part of"
|
|
|
- + " heartbeat, hence shutting down.");
|
|
|
- LOG.warn("Message from ResourceManager: "
|
|
|
- + response.getDiagnosticsMessage());
|
|
|
- context.setDecommissioned(true);
|
|
|
- dispatcher.getEventHandler().handle(
|
|
|
- new NodeManagerEvent(NodeManagerEventType.SHUTDOWN));
|
|
|
- break;
|
|
|
- }
|
|
|
- if (response.getNodeAction() == NodeAction.RESYNC) {
|
|
|
- LOG.warn("Node is out of sync with ResourceManager,"
|
|
|
- + " hence resyncing.");
|
|
|
- LOG.warn("Message from ResourceManager: "
|
|
|
- + response.getDiagnosticsMessage());
|
|
|
- // Invalidate the RMIdentifier while resync
|
|
|
- NodeStatusUpdaterImpl.this.rmIdentifier =
|
|
|
- ResourceManagerConstants.RM_INVALID_IDENTIFIER;
|
|
|
- dispatcher.getEventHandler().handle(
|
|
|
- new NodeManagerEvent(NodeManagerEventType.RESYNC));
|
|
|
- pendingCompletedContainers.clear();
|
|
|
- break;
|
|
|
- }
|
|
|
-
|
|
|
- nodeLabelsHandler.verifyRMHeartbeatResponseForNodeLabels(response);
|
|
|
-
|
|
|
- // Explicitly put this method after checking the resync response. We
|
|
|
- // don't want to remove the completed containers before resync
|
|
|
- // because these completed containers will be reported back to RM
|
|
|
- // when NM re-registers with RM.
|
|
|
- // Only remove the cleanedup containers that are acked
|
|
|
- removeOrTrackCompletedContainersFromContext(response
|
|
|
+ if (!handleShutdownOrResyncCommand(response)) {
|
|
|
+ nodeLabelsHandler.verifyRMHeartbeatResponseForNodeLabels(
|
|
|
+ response);
|
|
|
+
|
|
|
+ // Explicitly put this method after checking the resync
|
|
|
+ // response. We
|
|
|
+ // don't want to remove the completed containers before resync
|
|
|
+ // because these completed containers will be reported back to RM
|
|
|
+ // when NM re-registers with RM.
|
|
|
+ // Only remove the cleanedup containers that are acked
|
|
|
+ removeOrTrackCompletedContainersFromContext(response
|
|
|
.getContainersToBeRemovedFromNM());
|
|
|
|
|
|
- logAggregationReportForAppsTempList.clear();
|
|
|
- lastHeartbeatID = response.getResponseId();
|
|
|
- List<ContainerId> containersToCleanup = response
|
|
|
- .getContainersToCleanup();
|
|
|
- if (!containersToCleanup.isEmpty()) {
|
|
|
- dispatcher.getEventHandler().handle(
|
|
|
- new CMgrCompletedContainersEvent(containersToCleanup,
|
|
|
- CMgrCompletedContainersEvent.Reason.BY_RESOURCEMANAGER));
|
|
|
- }
|
|
|
- List<ApplicationId> appsToCleanup =
|
|
|
- response.getApplicationsToCleanup();
|
|
|
- //Only start tracking for keepAlive on FINISH_APP
|
|
|
- trackAppsForKeepAlive(appsToCleanup);
|
|
|
- if (!appsToCleanup.isEmpty()) {
|
|
|
- dispatcher.getEventHandler().handle(
|
|
|
- new CMgrCompletedAppsEvent(appsToCleanup,
|
|
|
- CMgrCompletedAppsEvent.Reason.BY_RESOURCEMANAGER));
|
|
|
- }
|
|
|
-
|
|
|
- Map<ApplicationId, ByteBuffer> systemCredentials =
|
|
|
- response.getSystemCredentialsForApps();
|
|
|
- if (systemCredentials != null && !systemCredentials.isEmpty()) {
|
|
|
- ((NMContext) context)
|
|
|
- .setSystemCrendentialsForApps(parseCredentials(systemCredentials));
|
|
|
- }
|
|
|
+ logAggregationReportForAppsTempList.clear();
|
|
|
+ lastHeartbeatID = response.getResponseId();
|
|
|
+ List<ContainerId> containersToCleanup = response
|
|
|
+ .getContainersToCleanup();
|
|
|
+ if (!containersToCleanup.isEmpty()) {
|
|
|
+ dispatcher.getEventHandler().handle(
|
|
|
+ new CMgrCompletedContainersEvent(containersToCleanup,
|
|
|
+ CMgrCompletedContainersEvent.Reason
|
|
|
+ .BY_RESOURCEMANAGER));
|
|
|
+ }
|
|
|
+ List<ApplicationId> appsToCleanup =
|
|
|
+ response.getApplicationsToCleanup();
|
|
|
+ //Only start tracking for keepAlive on FINISH_APP
|
|
|
+ trackAppsForKeepAlive(appsToCleanup);
|
|
|
+ if (!appsToCleanup.isEmpty()) {
|
|
|
+ dispatcher.getEventHandler().handle(
|
|
|
+ new CMgrCompletedAppsEvent(appsToCleanup,
|
|
|
+ CMgrCompletedAppsEvent.Reason.BY_RESOURCEMANAGER));
|
|
|
+ }
|
|
|
+ Map<ApplicationId, ByteBuffer> systemCredentials =
|
|
|
+ response.getSystemCredentialsForApps();
|
|
|
+ if (systemCredentials != null && !systemCredentials.isEmpty()) {
|
|
|
+ ((NMContext) context).setSystemCrendentialsForApps(
|
|
|
+ parseCredentials(systemCredentials));
|
|
|
+ }
|
|
|
+ List<org.apache.hadoop.yarn.api.records.Container>
|
|
|
+ containersToDecrease = response.getContainersToDecrease();
|
|
|
+ if (!containersToDecrease.isEmpty()) {
|
|
|
+ dispatcher.getEventHandler().handle(
|
|
|
+ new CMgrDecreaseContainersResourceEvent(
|
|
|
+ containersToDecrease)
|
|
|
+ );
|
|
|
+ }
|
|
|
|
|
|
- List<org.apache.hadoop.yarn.api.records.Container>
|
|
|
- containersToDecrease = response.getContainersToDecrease();
|
|
|
- if (!containersToDecrease.isEmpty()) {
|
|
|
- dispatcher.getEventHandler().handle(
|
|
|
- new CMgrDecreaseContainersResourceEvent(containersToDecrease)
|
|
|
- );
|
|
|
- }
|
|
|
+ // SignalContainer request originally comes from end users via
|
|
|
+ // ClientRMProtocol's SignalContainer. Forward the request to
|
|
|
+ // ContainerManager which will dispatch the event to
|
|
|
+ // ContainerLauncher.
|
|
|
+ List<SignalContainerRequest> containersToSignal = response
|
|
|
+ .getContainersToSignalList();
|
|
|
+ if (containersToSignal.size() != 0) {
|
|
|
+ dispatcher.getEventHandler().handle(
|
|
|
+ new CMgrSignalContainersEvent(containersToSignal));
|
|
|
+ }
|
|
|
|
|
|
- // SignalContainer request originally comes from end users via
|
|
|
- // ClientRMProtocol's SignalContainer. Forward the request to
|
|
|
- // ContainerManager which will dispatch the event to ContainerLauncher.
|
|
|
- List<SignalContainerRequest> containersToSignal = response
|
|
|
- .getContainersToSignalList();
|
|
|
- if (containersToSignal.size() != 0) {
|
|
|
- dispatcher.getEventHandler().handle(
|
|
|
- new CMgrSignalContainersEvent(containersToSignal));
|
|
|
+ // Update QueuingLimits if ContainerManager supports queuing
|
|
|
+ ContainerQueuingLimit queuingLimit =
|
|
|
+ response.getContainerQueuingLimit();
|
|
|
+ if (queuingLimit != null) {
|
|
|
+ context.getContainerManager().updateQueuingLimit(queuingLimit);
|
|
|
+ }
|
|
|
}
|
|
|
} catch (ConnectException e) {
|
|
|
//catch and throw the exception if tried MAX wait time to connect RM
|
|
@@ -883,6 +868,34 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|
|
statusUpdater.start();
|
|
|
}
|
|
|
|
|
|
+ private boolean handleShutdownOrResyncCommand(
|
|
|
+ NodeHeartbeatResponse response) {
|
|
|
+ if (response.getNodeAction() == NodeAction.SHUTDOWN) {
|
|
|
+ LOG.warn("Recieved SHUTDOWN signal from Resourcemanager as part of"
|
|
|
+ + " heartbeat, hence shutting down.");
|
|
|
+ LOG.warn("Message from ResourceManager: "
|
|
|
+ + response.getDiagnosticsMessage());
|
|
|
+ context.setDecommissioned(true);
|
|
|
+ dispatcher.getEventHandler().handle(
|
|
|
+ new NodeManagerEvent(NodeManagerEventType.SHUTDOWN));
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ if (response.getNodeAction() == NodeAction.RESYNC) {
|
|
|
+ LOG.warn("Node is out of sync with ResourceManager,"
|
|
|
+ + " hence resyncing.");
|
|
|
+ LOG.warn("Message from ResourceManager: "
|
|
|
+ + response.getDiagnosticsMessage());
|
|
|
+ // Invalidate the RMIdentifier while resync
|
|
|
+ NodeStatusUpdaterImpl.this.rmIdentifier =
|
|
|
+ ResourceManagerConstants.RM_INVALID_IDENTIFIER;
|
|
|
+ dispatcher.getEventHandler().handle(
|
|
|
+ new NodeManagerEvent(NodeManagerEventType.RESYNC));
|
|
|
+ pendingCompletedContainers.clear();
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
private List<LogAggregationReport> getLogAggregationReportsForApps(
|
|
|
ConcurrentLinkedQueue<LogAggregationReport> lastestLogAggregationStatus) {
|
|
|
LogAggregationReport status;
|