|
@@ -106,6 +106,9 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|
// the AM finishes it informs the RM to stop the may-be-already-completed
|
|
// the AM finishes it informs the RM to stop the may-be-already-completed
|
|
// containers.
|
|
// containers.
|
|
private final Map<ContainerId, Long> recentlyStoppedContainers;
|
|
private final Map<ContainerId, Long> recentlyStoppedContainers;
|
|
|
|
+ // Save the reported completed containers in case of lost heartbeat responses.
|
|
|
|
+ // These completed containers will be sent again till a successful response.
|
|
|
|
+ private final Map<ContainerId, ContainerStatus> pendingCompletedContainers;
|
|
// Duration for which to track recently stopped container.
|
|
// Duration for which to track recently stopped container.
|
|
private long durationToTrackStoppedContainers;
|
|
private long durationToTrackStoppedContainers;
|
|
|
|
|
|
@@ -126,6 +129,8 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|
this.metrics = metrics;
|
|
this.metrics = metrics;
|
|
this.recentlyStoppedContainers =
|
|
this.recentlyStoppedContainers =
|
|
new LinkedHashMap<ContainerId, Long>();
|
|
new LinkedHashMap<ContainerId, Long>();
|
|
|
|
+ this.pendingCompletedContainers =
|
|
|
|
+ new HashMap<ContainerId, ContainerStatus>();
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -358,11 +363,10 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|
List<ContainerStatus> containerStatuses = new ArrayList<ContainerStatus>();
|
|
List<ContainerStatus> containerStatuses = new ArrayList<ContainerStatus>();
|
|
for (Container container : this.context.getContainers().values()) {
|
|
for (Container container : this.context.getContainers().values()) {
|
|
ContainerId containerId = container.getContainerId();
|
|
ContainerId containerId = container.getContainerId();
|
|
- ApplicationId applicationId = container.getContainerId()
|
|
|
|
- .getApplicationAttemptId().getApplicationId();
|
|
|
|
|
|
+ ApplicationId applicationId = containerId.getApplicationAttemptId()
|
|
|
|
+ .getApplicationId();
|
|
org.apache.hadoop.yarn.api.records.ContainerStatus containerStatus =
|
|
org.apache.hadoop.yarn.api.records.ContainerStatus containerStatus =
|
|
container.cloneAndGetContainerStatus();
|
|
container.cloneAndGetContainerStatus();
|
|
- containerStatuses.add(containerStatus);
|
|
|
|
if (containerStatus.getState() == ContainerState.COMPLETE) {
|
|
if (containerStatus.getState() == ContainerState.COMPLETE) {
|
|
if (isApplicationStopped(applicationId)) {
|
|
if (isApplicationStopped(applicationId)) {
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
@@ -370,14 +374,21 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|
+ containerId + " from NM context.");
|
|
+ containerId + " from NM context.");
|
|
}
|
|
}
|
|
context.getContainers().remove(containerId);
|
|
context.getContainers().remove(containerId);
|
|
|
|
+ pendingCompletedContainers.put(containerId, containerStatus);
|
|
} else {
|
|
} else {
|
|
- // Adding to finished containers cache. Cache will keep it around at
|
|
|
|
- // least for #durationToTrackStoppedContainers duration. In the
|
|
|
|
- // subsequent call to stop container it will get removed from cache.
|
|
|
|
- addCompletedContainer(container.getContainerId());
|
|
|
|
|
|
+ if (!isContainerRecentlyStopped(containerId)) {
|
|
|
|
+ pendingCompletedContainers.put(containerId, containerStatus);
|
|
|
|
+ // Adding to finished containers cache. Cache will keep it around at
|
|
|
|
+ // least for #durationToTrackStoppedContainers duration. In the
|
|
|
|
+ // subsequent call to stop container it will get removed from cache.
|
|
|
|
+ addCompletedContainer(containerId);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
+ } else {
|
|
|
|
+ containerStatuses.add(containerStatus);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+ containerStatuses.addAll(pendingCompletedContainers.values());
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
LOG.debug("Sending out " + containerStatuses.size()
|
|
LOG.debug("Sending out " + containerStatuses.size()
|
|
+ " container statuses: " + containerStatuses);
|
|
+ " container statuses: " + containerStatuses);
|
|
@@ -397,8 +408,8 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|
new ArrayList<NMContainerStatus>();
|
|
new ArrayList<NMContainerStatus>();
|
|
for (Container container : this.context.getContainers().values()) {
|
|
for (Container container : this.context.getContainers().values()) {
|
|
ContainerId containerId = container.getContainerId();
|
|
ContainerId containerId = container.getContainerId();
|
|
- ApplicationId applicationId = container.getContainerId()
|
|
|
|
- .getApplicationAttemptId().getApplicationId();
|
|
|
|
|
|
+ ApplicationId applicationId = containerId.getApplicationAttemptId()
|
|
|
|
+ .getApplicationId();
|
|
if (!this.context.getApplications().containsKey(applicationId)) {
|
|
if (!this.context.getApplications().containsKey(applicationId)) {
|
|
context.getContainers().remove(containerId);
|
|
context.getContainers().remove(containerId);
|
|
continue;
|
|
continue;
|
|
@@ -410,7 +421,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|
// Adding to finished containers cache. Cache will keep it around at
|
|
// Adding to finished containers cache. Cache will keep it around at
|
|
// least for #durationToTrackStoppedContainers duration. In the
|
|
// least for #durationToTrackStoppedContainers duration. In the
|
|
// subsequent call to stop container it will get removed from cache.
|
|
// subsequent call to stop container it will get removed from cache.
|
|
- addCompletedContainer(container.getContainerId());
|
|
|
|
|
|
+ addCompletedContainer(containerId);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
LOG.info("Sending out " + containerStatuses.size()
|
|
LOG.info("Sending out " + containerStatuses.size()
|
|
@@ -457,7 +468,9 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|
ContainerId containerId = iter.next();
|
|
ContainerId containerId = iter.next();
|
|
// remove the container only if the container is at DONE state
|
|
// remove the container only if the container is at DONE state
|
|
Container nmContainer = context.getContainers().get(containerId);
|
|
Container nmContainer = context.getContainers().get(containerId);
|
|
- if (nmContainer != null && nmContainer.getContainerState().equals(
|
|
|
|
|
|
+ if (nmContainer == null) {
|
|
|
|
+ iter.remove();
|
|
|
|
+ } else if (nmContainer.getContainerState().equals(
|
|
org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState.DONE)) {
|
|
org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState.DONE)) {
|
|
context.getContainers().remove(containerId);
|
|
context.getContainers().remove(containerId);
|
|
removedContainers.add(containerId);
|
|
removedContainers.add(containerId);
|
|
@@ -469,6 +482,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|
LOG.info("Removed completed containers from NM context: "
|
|
LOG.info("Removed completed containers from NM context: "
|
|
+ removedContainers);
|
|
+ removedContainers);
|
|
}
|
|
}
|
|
|
|
+ pendingCompletedContainers.clear();
|
|
}
|
|
}
|
|
|
|
|
|
private void trackAppsForKeepAlive(List<ApplicationId> appIds) {
|
|
private void trackAppsForKeepAlive(List<ApplicationId> appIds) {
|
|
@@ -507,7 +521,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|
recentlyStoppedContainers.clear();
|
|
recentlyStoppedContainers.clear();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@Private
|
|
@Private
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
public void removeVeryOldStoppedContainersFromCache() {
|
|
public void removeVeryOldStoppedContainersFromCache() {
|
|
@@ -605,6 +619,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
|
|
ResourceManagerConstants.RM_INVALID_IDENTIFIER;
|
|
ResourceManagerConstants.RM_INVALID_IDENTIFIER;
|
|
dispatcher.getEventHandler().handle(
|
|
dispatcher.getEventHandler().handle(
|
|
new NodeManagerEvent(NodeManagerEventType.RESYNC));
|
|
new NodeManagerEvent(NodeManagerEventType.RESYNC));
|
|
|
|
+ pendingCompletedContainers.clear();
|
|
break;
|
|
break;
|
|
}
|
|
}
|
|
|
|
|