|
@@ -23,10 +23,14 @@ import java.util.ArrayList;
|
|
|
import java.util.Collection;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
+import java.util.Set;
|
|
|
+import java.util.Timer;
|
|
|
+import java.util.TimerTask;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.service.AbstractService;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
@@ -34,18 +38,25 @@ import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
|
|
|
import org.apache.hadoop.yarn.api.records.Container;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerState;
|
|
|
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
|
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMoveEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerFinishedEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerRecoverEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
|
@@ -54,6 +65,7 @@ import org.apache.hadoop.yarn.util.resource.Resources;
|
|
|
|
|
|
import com.google.common.util.concurrent.SettableFuture;
|
|
|
|
|
|
+
|
|
|
@SuppressWarnings("unchecked")
|
|
|
public abstract class AbstractYarnScheduler
|
|
|
<T extends SchedulerApplicationAttempt, N extends SchedulerNode>
|
|
@@ -72,6 +84,7 @@ public abstract class AbstractYarnScheduler
|
|
|
|
|
|
protected RMContext rmContext;
|
|
|
protected Map<ApplicationId, SchedulerApplication<T>> applications;
|
|
|
+ protected int nmExpireInterval;
|
|
|
|
|
|
protected final static List<Container> EMPTY_CONTAINER_LIST =
|
|
|
new ArrayList<Container>();
|
|
@@ -87,6 +100,15 @@ public abstract class AbstractYarnScheduler
|
|
|
super(name);
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ public void serviceInit(Configuration conf) throws Exception {
|
|
|
+ nmExpireInterval =
|
|
|
+ conf.getInt(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS,
|
|
|
+ YarnConfiguration.DEFAULT_RM_NM_EXPIRY_INTERVAL_MS);
|
|
|
+ createReleaseCache();
|
|
|
+ super.serviceInit(conf);
|
|
|
+ }
|
|
|
+
|
|
|
public synchronized List<Container> getTransferredContainers(
|
|
|
ApplicationAttemptId currentAttempt) {
|
|
|
ApplicationId appId = currentAttempt.getApplicationId();
|
|
@@ -281,6 +303,19 @@ public abstract class AbstractYarnScheduler
|
|
|
((RMContainerImpl)rmContainer).setAMContainer(true);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ synchronized (schedulerAttempt) {
|
|
|
+ Set<ContainerId> releases = schedulerAttempt.getPendingRelease();
|
|
|
+ if (releases.contains(container.getContainerId())) {
|
|
|
+ // release the container
|
|
|
+ rmContainer.handle(new RMContainerFinishedEvent(container
|
|
|
+ .getContainerId(), SchedulerUtils.createAbnormalContainerStatus(
|
|
|
+ container.getContainerId(), SchedulerUtils.RELEASED_CONTAINER),
|
|
|
+ RMContainerEventType.RELEASED));
|
|
|
+ releases.remove(container.getContainerId());
|
|
|
+ LOG.info(container.getContainerId() + " is released by application.");
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -320,6 +355,62 @@ public abstract class AbstractYarnScheduler
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ protected void createReleaseCache() {
|
|
|
+ // Cleanup the cache after nm expire interval.
|
|
|
+ new Timer().schedule(new TimerTask() {
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ for (SchedulerApplication<T> app : applications.values()) {
|
|
|
+
|
|
|
+ T attempt = app.getCurrentAppAttempt();
|
|
|
+ synchronized (attempt) {
|
|
|
+ for (ContainerId containerId : attempt.getPendingRelease()) {
|
|
|
+ RMAuditLogger.logFailure(
|
|
|
+ app.getUser(),
|
|
|
+ AuditConstants.RELEASE_CONTAINER,
|
|
|
+ "Unauthorized access or invalid container",
|
|
|
+ "Scheduler",
|
|
|
+ "Trying to release container not owned by app or with invalid id.",
|
|
|
+ attempt.getApplicationId(), containerId);
|
|
|
+ }
|
|
|
+ attempt.getPendingRelease().clear();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ LOG.info("Release request cache is cleaned up");
|
|
|
+ }
|
|
|
+ }, nmExpireInterval);
|
|
|
+ }
|
|
|
+
|
|
|
+ // clean up a completed container
|
|
|
+ protected abstract void completedContainer(RMContainer rmContainer,
|
|
|
+ ContainerStatus containerStatus, RMContainerEventType event);
|
|
|
+
|
|
|
+ protected void releaseContainers(List<ContainerId> containers,
|
|
|
+ SchedulerApplicationAttempt attempt) {
|
|
|
+ for (ContainerId containerId : containers) {
|
|
|
+ RMContainer rmContainer = getRMContainer(containerId);
|
|
|
+ if (rmContainer == null) {
|
|
|
+ if (System.currentTimeMillis() - ResourceManager.getClusterTimeStamp()
|
|
|
+ < nmExpireInterval) {
|
|
|
+ LOG.info(containerId + " doesn't exist. Add the container"
|
|
|
+ + " to the release request cache as it maybe on recovery.");
|
|
|
+ synchronized (attempt) {
|
|
|
+ attempt.getPendingRelease().add(containerId);
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ RMAuditLogger.logFailure(attempt.getUser(),
|
|
|
+ AuditConstants.RELEASE_CONTAINER,
|
|
|
+ "Unauthorized access or invalid container", "Scheduler",
|
|
|
+ "Trying to release container not owned by app or with invalid id.",
|
|
|
+ attempt.getApplicationId(), containerId);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ completedContainer(rmContainer,
|
|
|
+ SchedulerUtils.createAbnormalContainerStatus(containerId,
|
|
|
+ SchedulerUtils.RELEASED_CONTAINER), RMContainerEventType.RELEASED);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
public SchedulerNode getSchedulerNode(NodeId nodeId) {
|
|
|
return nodes.get(nodeId);
|
|
|
}
|