|
@@ -36,9 +36,8 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
|
import org.apache.hadoop.security.AccessControlException;
|
|
import org.apache.hadoop.security.AccessControlException;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.security.authorize.AccessControlList;
|
|
import org.apache.hadoop.security.authorize.AccessControlList;
|
|
-import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
|
|
|
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
import org.apache.hadoop.yarn.api.records.Container;
|
|
import org.apache.hadoop.yarn.api.records.Container;
|
|
-import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerToken;
|
|
import org.apache.hadoop.yarn.api.records.ContainerToken;
|
|
import org.apache.hadoop.yarn.api.records.Priority;
|
|
import org.apache.hadoop.yarn.api.records.Priority;
|
|
import org.apache.hadoop.yarn.api.records.QueueACL;
|
|
import org.apache.hadoop.yarn.api.records.QueueACL;
|
|
@@ -47,20 +46,18 @@ import org.apache.hadoop.yarn.api.records.QueueState;
|
|
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
|
|
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
-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.security.ContainerTokenIdentifier;
|
|
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent;
|
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
|
|
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo;
|
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApp;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
|
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
|
|
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
|
|
import org.apache.hadoop.yarn.util.BuilderUtils;
|
|
import org.apache.hadoop.yarn.util.BuilderUtils;
|
|
|
|
|
|
@@ -85,8 +82,10 @@ public class LeafQueue implements Queue {
|
|
private float usedCapacity = 0.0f;
|
|
private float usedCapacity = 0.0f;
|
|
private volatile int numContainers;
|
|
private volatile int numContainers;
|
|
|
|
|
|
- Set<CSApp> applications;
|
|
|
|
-
|
|
|
|
|
|
+ Set<SchedulerApp> applications;
|
|
|
|
+ Map<ApplicationAttemptId, SchedulerApp> applicationsMap =
|
|
|
|
+ new HashMap<ApplicationAttemptId, SchedulerApp>();
|
|
|
|
+
|
|
public final Resource minimumAllocation;
|
|
public final Resource minimumAllocation;
|
|
|
|
|
|
private ContainerTokenSecretManager containerTokenSecretManager;
|
|
private ContainerTokenSecretManager containerTokenSecretManager;
|
|
@@ -109,7 +108,7 @@ public class LeafQueue implements Queue {
|
|
|
|
|
|
public LeafQueue(CapacitySchedulerContext cs,
|
|
public LeafQueue(CapacitySchedulerContext cs,
|
|
String queueName, Queue parent,
|
|
String queueName, Queue parent,
|
|
- Comparator<CSApp> applicationComparator, Queue old) {
|
|
|
|
|
|
+ Comparator<SchedulerApp> applicationComparator, Queue old) {
|
|
this.scheduler = cs;
|
|
this.scheduler = cs;
|
|
this.queueName = queueName;
|
|
this.queueName = queueName;
|
|
this.parent = parent;
|
|
this.parent = parent;
|
|
@@ -158,7 +157,7 @@ public class LeafQueue implements Queue {
|
|
" name=" + queueName +
|
|
" name=" + queueName +
|
|
", fullname=" + getQueuePath());
|
|
", fullname=" + getQueuePath());
|
|
|
|
|
|
- this.applications = new TreeSet<CSApp>(applicationComparator);
|
|
|
|
|
|
+ this.applications = new TreeSet<SchedulerApp>(applicationComparator);
|
|
}
|
|
}
|
|
|
|
|
|
private synchronized void setupQueueConfigs(
|
|
private synchronized void setupQueueConfigs(
|
|
@@ -362,7 +361,7 @@ public class LeafQueue implements Queue {
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
- public void submitApplication(CSApp application, String userName,
|
|
|
|
|
|
+ public void submitApplication(SchedulerApp application, String userName,
|
|
String queue) throws AccessControlException {
|
|
String queue) throws AccessControlException {
|
|
// Careful! Locking order is important!
|
|
// Careful! Locking order is important!
|
|
|
|
|
|
@@ -423,10 +422,11 @@ public class LeafQueue implements Queue {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- private synchronized void addApplication(CSApp application, User user) {
|
|
|
|
|
|
+ private synchronized void addApplication(SchedulerApp application, User user) {
|
|
// Accept
|
|
// Accept
|
|
user.submitApplication();
|
|
user.submitApplication();
|
|
applications.add(application);
|
|
applications.add(application);
|
|
|
|
+ applicationsMap.put(application.getApplicationAttemptId(), application);
|
|
|
|
|
|
LOG.info("Application added -" +
|
|
LOG.info("Application added -" +
|
|
" appId: " + application.getApplicationId() +
|
|
" appId: " + application.getApplicationId() +
|
|
@@ -436,7 +436,7 @@ public class LeafQueue implements Queue {
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
- public void finishApplication(CSApp application, String queue) {
|
|
|
|
|
|
+ public void finishApplication(SchedulerApp application, String queue) {
|
|
// Careful! Locking order is important!
|
|
// Careful! Locking order is important!
|
|
synchronized (this) {
|
|
synchronized (this) {
|
|
removeApplication(application, getUser(application.getUser()));
|
|
removeApplication(application, getUser(application.getUser()));
|
|
@@ -446,8 +446,9 @@ public class LeafQueue implements Queue {
|
|
parent.finishApplication(application, queue);
|
|
parent.finishApplication(application, queue);
|
|
}
|
|
}
|
|
|
|
|
|
- public synchronized void removeApplication(CSApp application, User user) {
|
|
|
|
|
|
+ public synchronized void removeApplication(SchedulerApp application, User user) {
|
|
applications.remove(application);
|
|
applications.remove(application);
|
|
|
|
+ applicationsMap.remove(application.getApplicationAttemptId());
|
|
|
|
|
|
user.finishApplication();
|
|
user.finishApplication();
|
|
if (user.getApplications() == 0) {
|
|
if (user.getApplications() == 0) {
|
|
@@ -461,24 +462,31 @@ public class LeafQueue implements Queue {
|
|
" #user-applications: " + user.getApplications() +
|
|
" #user-applications: " + user.getApplications() +
|
|
" #queue-applications: " + getNumApplications());
|
|
" #queue-applications: " + getNumApplications());
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ private synchronized SchedulerApp getApplication(
|
|
|
|
+ ApplicationAttemptId applicationAttemptId) {
|
|
|
|
+ return applicationsMap.get(applicationAttemptId);
|
|
|
|
+ }
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public synchronized Resource
|
|
public synchronized Resource
|
|
- assignContainers(Resource clusterResource, CSNode node) {
|
|
|
|
|
|
+ assignContainers(Resource clusterResource, SchedulerNode node) {
|
|
|
|
|
|
LOG.info("DEBUG --- assignContainers:" +
|
|
LOG.info("DEBUG --- assignContainers:" +
|
|
" node=" + node.getNodeAddress() +
|
|
" node=" + node.getNodeAddress() +
|
|
" #applications=" + applications.size());
|
|
" #applications=" + applications.size());
|
|
|
|
|
|
// Check for reserved resources
|
|
// Check for reserved resources
|
|
- CSApp reservedApplication = node.getReservedApplication();
|
|
|
|
- if (reservedApplication != null) {
|
|
|
|
- return assignReservedContainers(reservedApplication, node,
|
|
|
|
|
|
+ RMContainer reservedContainer = node.getReservedContainer();
|
|
|
|
+ if (reservedContainer != null) {
|
|
|
|
+ SchedulerApp application =
|
|
|
|
+ getApplication(reservedContainer.getApplicationAttemptId());
|
|
|
|
+ return assignReservedContainer(application, node, reservedContainer,
|
|
clusterResource);
|
|
clusterResource);
|
|
}
|
|
}
|
|
-
|
|
|
|
- // Try to assign containers to applications in fifo order
|
|
|
|
- for (CSApp application : applications) {
|
|
|
|
|
|
+
|
|
|
|
+ // Try to assign containers to applications in order
|
|
|
|
+ for (SchedulerApp application : applications) {
|
|
|
|
|
|
LOG.info("DEBUG --- pre-assignContainers for application "
|
|
LOG.info("DEBUG --- pre-assignContainers for application "
|
|
+ application.getApplicationId());
|
|
+ application.getApplicationId());
|
|
@@ -497,6 +505,7 @@ public class LeafQueue implements Queue {
|
|
}
|
|
}
|
|
|
|
|
|
// Are we going over limits by allocating to this application?
|
|
// Are we going over limits by allocating to this application?
|
|
|
|
+
|
|
ResourceRequest required =
|
|
ResourceRequest required =
|
|
application.getResourceRequest(priority, RMNode.ANY);
|
|
application.getResourceRequest(priority, RMNode.ANY);
|
|
|
|
|
|
@@ -520,7 +529,7 @@ public class LeafQueue implements Queue {
|
|
// Try to schedule
|
|
// Try to schedule
|
|
Resource assigned =
|
|
Resource assigned =
|
|
assignContainersOnNode(clusterResource, node, application, priority,
|
|
assignContainersOnNode(clusterResource, node, application, priority,
|
|
- false);
|
|
|
|
|
|
+ null);
|
|
|
|
|
|
// Did we schedule or reserve a container?
|
|
// Did we schedule or reserve a container?
|
|
if (Resources.greaterThan(assigned, Resources.none())) {
|
|
if (Resources.greaterThan(assigned, Resources.none())) {
|
|
@@ -552,30 +561,21 @@ public class LeafQueue implements Queue {
|
|
|
|
|
|
}
|
|
}
|
|
|
|
|
|
- private synchronized Resource assignReservedContainers(CSApp application,
|
|
|
|
- CSNode node, Resource clusterResource) {
|
|
|
|
- synchronized (application) {
|
|
|
|
- for (Priority priority : application.getPriorities()) {
|
|
|
|
-
|
|
|
|
- // Do we reserve containers at this 'priority'?
|
|
|
|
- if (application.isReserved(node, priority)) {
|
|
|
|
-
|
|
|
|
- // Do we really need this reservation still?
|
|
|
|
- ResourceRequest offSwitchRequest =
|
|
|
|
- application.getResourceRequest(priority, RMNode.ANY);
|
|
|
|
- if (offSwitchRequest.getNumContainers() == 0) {
|
|
|
|
- // Release
|
|
|
|
- unreserve(application, priority, node);
|
|
|
|
- return offSwitchRequest.getCapability();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // Try to assign if we have sufficient resources
|
|
|
|
- assignContainersOnNode(clusterResource, node, application, priority,
|
|
|
|
- true);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
|
|
+ private synchronized Resource assignReservedContainer(SchedulerApp application,
|
|
|
|
+ SchedulerNode node, RMContainer rmContainer, Resource clusterResource) {
|
|
|
|
+ // Do we still need this reservation?
|
|
|
|
+ Priority priority = rmContainer.getReservedPriority();
|
|
|
|
+ if (application.getTotalRequiredResources(priority) == 0) {
|
|
|
|
+ // Release
|
|
|
|
+ Container container = rmContainer.getContainer();
|
|
|
|
+ completedContainer(clusterResource, application, node,
|
|
|
|
+ rmContainer, RMContainerEventType.RELEASED);
|
|
|
|
+ return container.getResource();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ // Try to assign if we have sufficient resources
|
|
|
|
+ assignContainersOnNode(clusterResource, node, application, priority, rmContainer);
|
|
|
|
+
|
|
// Doesn't matter... since it's already charged for at time of reservation
|
|
// Doesn't matter... since it's already charged for at time of reservation
|
|
// "re-reservation" is *free*
|
|
// "re-reservation" is *free*
|
|
return org.apache.hadoop.yarn.server.resourcemanager.resource.Resource.NONE;
|
|
return org.apache.hadoop.yarn.server.resourcemanager.resource.Resource.NONE;
|
|
@@ -599,12 +599,12 @@ public class LeafQueue implements Queue {
|
|
return true;
|
|
return true;
|
|
}
|
|
}
|
|
|
|
|
|
- private void setUserResourceLimit(CSApp application, Resource resourceLimit) {
|
|
|
|
|
|
+ private void setUserResourceLimit(SchedulerApp application, Resource resourceLimit) {
|
|
application.setAvailableResourceLimit(resourceLimit);
|
|
application.setAvailableResourceLimit(resourceLimit);
|
|
metrics.setAvailableResourcesToUser(application.getUser(), resourceLimit);
|
|
metrics.setAvailableResourcesToUser(application.getUser(), resourceLimit);
|
|
}
|
|
}
|
|
|
|
|
|
- private Resource computeUserLimit(CSApp application,
|
|
|
|
|
|
+ private Resource computeUserLimit(SchedulerApp application,
|
|
Resource clusterResource, Resource required) {
|
|
Resource clusterResource, Resource required) {
|
|
// What is our current capacity?
|
|
// What is our current capacity?
|
|
// * It is equal to the max(required, queue-capacity) if
|
|
// * It is equal to the max(required, queue-capacity) if
|
|
@@ -688,80 +688,87 @@ public class LeafQueue implements Queue {
|
|
return (a + (b - 1)) / b;
|
|
return (a + (b - 1)) / b;
|
|
}
|
|
}
|
|
|
|
|
|
- boolean needContainers(CSApp application, Priority priority) {
|
|
|
|
- ResourceRequest offSwitchRequest =
|
|
|
|
- application.getResourceRequest(priority, RMNode.ANY);
|
|
|
|
-
|
|
|
|
- int requiredContainers = offSwitchRequest.getNumContainers();
|
|
|
|
- int reservedContainers = application.getReservedContainers(priority);
|
|
|
|
|
|
+ boolean needContainers(SchedulerApp application, Priority priority) {
|
|
|
|
+ int requiredContainers = application.getTotalRequiredResources(priority);
|
|
|
|
+ int reservedContainers = application.getNumReservedContainers(priority);
|
|
return ((requiredContainers - reservedContainers) > 0);
|
|
return ((requiredContainers - reservedContainers) > 0);
|
|
}
|
|
}
|
|
|
|
|
|
- Resource assignContainersOnNode(Resource clusterResource, CSNode node,
|
|
|
|
- CSApp application, Priority priority, boolean reserved) {
|
|
|
|
|
|
+ Resource assignContainersOnNode(Resource clusterResource, SchedulerNode node,
|
|
|
|
+ SchedulerApp application, Priority priority, RMContainer reservedContainer) {
|
|
|
|
|
|
Resource assigned = Resources.none();
|
|
Resource assigned = Resources.none();
|
|
|
|
|
|
// Data-local
|
|
// Data-local
|
|
- assigned = assignNodeLocalContainers(clusterResource, node, application, priority);
|
|
|
|
|
|
+ assigned =
|
|
|
|
+ assignNodeLocalContainers(clusterResource, node, application, priority,
|
|
|
|
+ reservedContainer);
|
|
if (Resources.greaterThan(assigned, Resources.none())) {
|
|
if (Resources.greaterThan(assigned, Resources.none())) {
|
|
return assigned;
|
|
return assigned;
|
|
}
|
|
}
|
|
|
|
|
|
// Rack-local
|
|
// Rack-local
|
|
- assigned = assignRackLocalContainers(clusterResource, node, application, priority);
|
|
|
|
|
|
+ assigned =
|
|
|
|
+ assignRackLocalContainers(clusterResource, node, application, priority,
|
|
|
|
+ reservedContainer);
|
|
if (Resources.greaterThan(assigned, Resources.none())) {
|
|
if (Resources.greaterThan(assigned, Resources.none())) {
|
|
- return assigned;
|
|
|
|
|
|
+ return assigned;
|
|
}
|
|
}
|
|
|
|
|
|
// Off-switch
|
|
// Off-switch
|
|
return assignOffSwitchContainers(clusterResource, node, application,
|
|
return assignOffSwitchContainers(clusterResource, node, application,
|
|
- priority, reserved);
|
|
|
|
|
|
+ priority, reservedContainer);
|
|
}
|
|
}
|
|
|
|
|
|
- Resource assignNodeLocalContainers(Resource clusterResource, CSNode node,
|
|
|
|
- CSApp application, Priority priority) {
|
|
|
|
|
|
+ Resource assignNodeLocalContainers(Resource clusterResource, SchedulerNode node,
|
|
|
|
+ SchedulerApp application, Priority priority,
|
|
|
|
+ RMContainer reservedContainer) {
|
|
ResourceRequest request = application.getResourceRequest(priority, node
|
|
ResourceRequest request = application.getResourceRequest(priority, node
|
|
.getNodeAddress());
|
|
.getNodeAddress());
|
|
if (request != null) {
|
|
if (request != null) {
|
|
- if (canAssign(application, priority, node, NodeType.DATA_LOCAL, false)) {
|
|
|
|
|
|
+ if (canAssign(application, priority, node, NodeType.DATA_LOCAL,
|
|
|
|
+ reservedContainer)) {
|
|
return assignContainer(clusterResource, node, application, priority, request,
|
|
return assignContainer(clusterResource, node, application, priority, request,
|
|
- NodeType.DATA_LOCAL);
|
|
|
|
|
|
+ NodeType.DATA_LOCAL, reservedContainer);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
return Resources.none();
|
|
return Resources.none();
|
|
}
|
|
}
|
|
|
|
|
|
- Resource assignRackLocalContainers(Resource clusterResource, CSNode node,
|
|
|
|
- CSApp application, Priority priority) {
|
|
|
|
|
|
+ Resource assignRackLocalContainers(Resource clusterResource,
|
|
|
|
+ SchedulerNode node, SchedulerApp application, Priority priority,
|
|
|
|
+ RMContainer reservedContainer) {
|
|
ResourceRequest request =
|
|
ResourceRequest request =
|
|
application.getResourceRequest(priority, node.getRackName());
|
|
application.getResourceRequest(priority, node.getRackName());
|
|
if (request != null) {
|
|
if (request != null) {
|
|
- if (canAssign(application, priority, node, NodeType.RACK_LOCAL, false)) {
|
|
|
|
|
|
+ if (canAssign(application, priority, node, NodeType.RACK_LOCAL,
|
|
|
|
+ reservedContainer)) {
|
|
return assignContainer(clusterResource, node, application, priority, request,
|
|
return assignContainer(clusterResource, node, application, priority, request,
|
|
- NodeType.RACK_LOCAL);
|
|
|
|
|
|
+ NodeType.RACK_LOCAL, reservedContainer);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return Resources.none();
|
|
return Resources.none();
|
|
}
|
|
}
|
|
|
|
|
|
- Resource assignOffSwitchContainers(Resource clusterResource, CSNode node,
|
|
|
|
- CSApp application, Priority priority, boolean reserved) {
|
|
|
|
|
|
+ Resource assignOffSwitchContainers(Resource clusterResource, SchedulerNode node,
|
|
|
|
+ SchedulerApp application, Priority priority,
|
|
|
|
+ RMContainer reservedContainer) {
|
|
ResourceRequest request =
|
|
ResourceRequest request =
|
|
application.getResourceRequest(priority, RMNode.ANY);
|
|
application.getResourceRequest(priority, RMNode.ANY);
|
|
if (request != null) {
|
|
if (request != null) {
|
|
- if (canAssign(application, priority, node, NodeType.OFF_SWITCH, reserved)) {
|
|
|
|
|
|
+ if (canAssign(application, priority, node, NodeType.OFF_SWITCH,
|
|
|
|
+ reservedContainer)) {
|
|
return assignContainer(clusterResource, node, application, priority, request,
|
|
return assignContainer(clusterResource, node, application, priority, request,
|
|
- NodeType.OFF_SWITCH);
|
|
|
|
|
|
+ NodeType.OFF_SWITCH, reservedContainer);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
return Resources.none();
|
|
return Resources.none();
|
|
}
|
|
}
|
|
|
|
|
|
- boolean canAssign(CSApp application, Priority priority,
|
|
|
|
- CSNode node, NodeType type, boolean reserved) {
|
|
|
|
|
|
+ boolean canAssign(SchedulerApp application, Priority priority,
|
|
|
|
+ SchedulerNode node, NodeType type, RMContainer reservedContainer) {
|
|
|
|
|
|
ResourceRequest offSwitchRequest =
|
|
ResourceRequest offSwitchRequest =
|
|
application.getResourceRequest(priority, RMNode.ANY);
|
|
application.getResourceRequest(priority, RMNode.ANY);
|
|
@@ -781,18 +788,18 @@ public class LeafQueue implements Queue {
|
|
|
|
|
|
if (requiredContainers > 0) {
|
|
if (requiredContainers > 0) {
|
|
// No 'delay' for reserved containers
|
|
// No 'delay' for reserved containers
|
|
- if (reserved) {
|
|
|
|
|
|
+ if (reservedContainer != null) {
|
|
return true;
|
|
return true;
|
|
}
|
|
}
|
|
|
|
|
|
-// // Check if we have waited long enough
|
|
|
|
-// if (missedNodes < (requiredContainers * localityWaitFactor)) {
|
|
|
|
-// LOG.info("Application " + application.getApplicationId() +
|
|
|
|
-// " has missed " + missedNodes + " opportunities," +
|
|
|
|
-// " waitFactor= " + localityWaitFactor +
|
|
|
|
-// " for cluster of size " + scheduler.getNumClusterNodes());
|
|
|
|
-// return false;
|
|
|
|
-// }
|
|
|
|
|
|
+ // Check if we have waited long enough
|
|
|
|
+ if (missedNodes < (requiredContainers * localityWaitFactor)) {
|
|
|
|
+ LOG.info("Application " + application.getApplicationId() +
|
|
|
|
+ " has missed " + missedNodes + " opportunities," +
|
|
|
|
+ " waitFactor= " + localityWaitFactor +
|
|
|
|
+ " for cluster of size " + scheduler.getNumClusterNodes());
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
return true;
|
|
return true;
|
|
}
|
|
}
|
|
return false;
|
|
return false;
|
|
@@ -830,157 +837,162 @@ public class LeafQueue implements Queue {
|
|
|
|
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
- private Resource assignContainer(Resource clusterResource, CSNode node,
|
|
|
|
- CSApp application,
|
|
|
|
- Priority priority, ResourceRequest request, NodeType type) {
|
|
|
|
|
|
+
|
|
|
|
+ private Container getContainer(RMContainer rmContainer,
|
|
|
|
+ SchedulerApp application, SchedulerNode node, Resource capability) {
|
|
|
|
+ if (rmContainer != null) {
|
|
|
|
+ return rmContainer.getContainer();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ Container container =
|
|
|
|
+ BuilderUtils.newContainer(this.recordFactory,
|
|
|
|
+ application.getApplicationAttemptId(),
|
|
|
|
+ application.getNewContainerId(),
|
|
|
|
+ node.getNodeID(),
|
|
|
|
+ node.getHttpAddress(), capability);
|
|
|
|
+
|
|
|
|
+ // If security is enabled, send the container-tokens too.
|
|
|
|
+ if (UserGroupInformation.isSecurityEnabled()) {
|
|
|
|
+ ContainerToken containerToken =
|
|
|
|
+ this.recordFactory.newRecordInstance(ContainerToken.class);
|
|
|
|
+ ContainerTokenIdentifier tokenidentifier =
|
|
|
|
+ new ContainerTokenIdentifier(container.getId(),
|
|
|
|
+ container.getNodeId().toString(), container.getResource());
|
|
|
|
+ containerToken.setIdentifier(
|
|
|
|
+ ByteBuffer.wrap(tokenidentifier.getBytes()));
|
|
|
|
+ containerToken.setKind(ContainerTokenIdentifier.KIND.toString());
|
|
|
|
+ containerToken.setPassword(
|
|
|
|
+ ByteBuffer.wrap(
|
|
|
|
+ containerTokenSecretManager.createPassword(tokenidentifier))
|
|
|
|
+ );
|
|
|
|
+ containerToken.setService(container.getNodeId().toString());
|
|
|
|
+ container.setContainerToken(containerToken);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ return container;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private Resource assignContainer(Resource clusterResource, SchedulerNode node,
|
|
|
|
+ SchedulerApp application, Priority priority,
|
|
|
|
+ ResourceRequest request, NodeType type, RMContainer rmContainer) {
|
|
LOG.info("DEBUG --- assignContainers:" +
|
|
LOG.info("DEBUG --- assignContainers:" +
|
|
" node=" + node.getNodeAddress() +
|
|
" node=" + node.getNodeAddress() +
|
|
" application=" + application.getApplicationId().getId() +
|
|
" application=" + application.getApplicationId().getId() +
|
|
" priority=" + priority.getPriority() +
|
|
" priority=" + priority.getPriority() +
|
|
" request=" + request + " type=" + type);
|
|
" request=" + request + " type=" + type);
|
|
Resource capability = request.getCapability();
|
|
Resource capability = request.getCapability();
|
|
-
|
|
|
|
- Resource available = node.getAvailableResource();
|
|
|
|
-
|
|
|
|
- if (available.getMemory() > 0) {
|
|
|
|
-
|
|
|
|
- int availableContainers =
|
|
|
|
- available.getMemory() / capability.getMemory(); // TODO: A buggy
|
|
|
|
- // application
|
|
|
|
- // with this
|
|
|
|
- // zero would
|
|
|
|
- // crash the
|
|
|
|
- // scheduler.
|
|
|
|
-
|
|
|
|
- if (availableContainers > 0) {
|
|
|
|
- List<Container> containers =
|
|
|
|
- new ArrayList<Container>();
|
|
|
|
- Container container =
|
|
|
|
- BuilderUtils.newContainer(this.recordFactory,
|
|
|
|
- application.getApplicationAttemptId(),
|
|
|
|
- application.getNewContainerId(),
|
|
|
|
- node.getNodeID(),
|
|
|
|
- node.getHttpAddress(), capability);
|
|
|
|
-
|
|
|
|
- // If security is enabled, send the container-tokens too.
|
|
|
|
- if (UserGroupInformation.isSecurityEnabled()) {
|
|
|
|
- ContainerToken containerToken = this.recordFactory.newRecordInstance(ContainerToken.class);
|
|
|
|
- ContainerTokenIdentifier tokenidentifier =
|
|
|
|
- new ContainerTokenIdentifier(container.getId(),
|
|
|
|
- container.getNodeId().toString(), container.getResource());
|
|
|
|
- containerToken.setIdentifier(ByteBuffer.wrap(tokenidentifier.getBytes()));
|
|
|
|
- containerToken.setKind(ContainerTokenIdentifier.KIND.toString());
|
|
|
|
- containerToken.setPassword(ByteBuffer.wrap(containerTokenSecretManager
|
|
|
|
- .createPassword(tokenidentifier)));
|
|
|
|
- containerToken.setService(container.getNodeId().toString());
|
|
|
|
- container.setContainerToken(containerToken);
|
|
|
|
- }
|
|
|
|
|
|
|
|
- containers.add(container);
|
|
|
|
|
|
+ Resource available = node.getAvailableResource();
|
|
|
|
|
|
- // Allocate
|
|
|
|
- allocate(application, type, priority, request, node, containers);
|
|
|
|
|
|
+ assert (available.getMemory() > 0);
|
|
|
|
|
|
- // Did we previously reserve containers at this 'priority'?
|
|
|
|
- if (application.isReserved(node, priority)){
|
|
|
|
- unreserve(application, priority, node);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- LOG.info("assignedContainer" +
|
|
|
|
- " application=" + application.getApplicationId() +
|
|
|
|
- " container=" + container +
|
|
|
|
- " queue=" + this.toString() +
|
|
|
|
- " util=" + getUtilization() +
|
|
|
|
- " used=" + usedResources +
|
|
|
|
- " cluster=" + clusterResource);
|
|
|
|
-
|
|
|
|
- return container.getResource();
|
|
|
|
- } else {
|
|
|
|
- // Reserve by 'charging' in advance...
|
|
|
|
- reserve(application, priority, node, request.getCapability());
|
|
|
|
-
|
|
|
|
- LOG.info("Reserved container " +
|
|
|
|
- " application=" + application.getApplicationId() +
|
|
|
|
- " resource=" + request.getCapability() +
|
|
|
|
- " queue=" + this.toString() +
|
|
|
|
- " util=" + getUtilization() +
|
|
|
|
- " used=" + usedResources +
|
|
|
|
- " cluster=" + clusterResource);
|
|
|
|
|
|
+ // Create the container if necessary
|
|
|
|
+ Container container =
|
|
|
|
+ getContainer(rmContainer, application, node, capability);
|
|
|
|
|
|
- return request.getCapability();
|
|
|
|
|
|
+ // Can we allocate a container on this node?
|
|
|
|
+ int availableContainers =
|
|
|
|
+ available.getMemory() / capability.getMemory();
|
|
|
|
+ if (availableContainers > 0) {
|
|
|
|
+ // Allocate...
|
|
|
|
|
|
|
|
+ // Did we previously reserve containers at this 'priority'?
|
|
|
|
+ if (rmContainer != null){
|
|
|
|
+ unreserve(application, priority, node, rmContainer);
|
|
}
|
|
}
|
|
- }
|
|
|
|
|
|
|
|
- return Resources.none();
|
|
|
|
- }
|
|
|
|
|
|
+ // Inform the application
|
|
|
|
+ RMContainer allocatedContainer =
|
|
|
|
+ application.allocate(type, node, priority, request, container);
|
|
|
|
+ if (allocatedContainer == null) {
|
|
|
|
+ // Did the application need this resource?
|
|
|
|
+ return Resources.none();
|
|
|
|
+ }
|
|
|
|
|
|
- private void allocate(CSApp application, NodeType type,
|
|
|
|
- Priority priority, ResourceRequest request,
|
|
|
|
- CSNode node, List<Container> containers) {
|
|
|
|
- // Allocate container to the application
|
|
|
|
- // TODO: acm: refactor2 FIXME
|
|
|
|
- application.allocate(type, node, priority, request, null);
|
|
|
|
-
|
|
|
|
- for (Container container : containers) {
|
|
|
|
- // Create the container and 'start' it.
|
|
|
|
- ContainerId containerId = container.getId();
|
|
|
|
- RMContext rmContext = this.scheduler.getRMContext();
|
|
|
|
- EventHandler eventHandler = rmContext.getDispatcher().getEventHandler();
|
|
|
|
- RMContainer rmContainer = new RMContainerImpl(container, application
|
|
|
|
- .getApplicationAttemptId(), node.getNodeID(),
|
|
|
|
- eventHandler, rmContext.getContainerAllocationExpirer());
|
|
|
|
- // TODO: FIX
|
|
|
|
-// if (rmContext.getRMContainers().putIfAbsent(containerId, rmContainer) != null) {
|
|
|
|
-// LOG.error("Duplicate container addition! ContainerID : "
|
|
|
|
-// + containerId);
|
|
|
|
-// } else {
|
|
|
|
-// eventHandler.handle(new RMContainerEvent(containerId,
|
|
|
|
-// RMContainerEventType.START));
|
|
|
|
-// }
|
|
|
|
|
|
+ // Inform the node
|
|
|
|
+ node.allocateContainer(application.getApplicationId(),
|
|
|
|
+ allocatedContainer);
|
|
|
|
+
|
|
|
|
+ LOG.info("assignedContainer" +
|
|
|
|
+ " application=" + application.getApplicationId() +
|
|
|
|
+ " container=" + container +
|
|
|
|
+ " containerId=" + container.getId() +
|
|
|
|
+ " queue=" + this +
|
|
|
|
+ " util=" + getUtilization() +
|
|
|
|
+ " used=" + usedResources +
|
|
|
|
+ " cluster=" + clusterResource);
|
|
|
|
+
|
|
|
|
+ return container.getResource();
|
|
|
|
+ } else {
|
|
|
|
+ // Reserve by 'charging' in advance...
|
|
|
|
+ reserve(application, priority, node, rmContainer, container);
|
|
|
|
+
|
|
|
|
+ LOG.info("Reserved container " +
|
|
|
|
+ " application=" + application.getApplicationId() +
|
|
|
|
+ " resource=" + request.getCapability() +
|
|
|
|
+ " queue=" + this.toString() +
|
|
|
|
+ " util=" + getUtilization() +
|
|
|
|
+ " used=" + usedResources +
|
|
|
|
+ " cluster=" + clusterResource);
|
|
|
|
+
|
|
|
|
+ return request.getCapability();
|
|
}
|
|
}
|
|
-
|
|
|
|
- // Inform the NodeManager about the allocation
|
|
|
|
- // TODO: acm: refactor2 FIXME
|
|
|
|
-// node.allocateContainer(application.getApplicationId(),
|
|
|
|
-// containers);
|
|
|
|
}
|
|
}
|
|
|
|
|
|
- private void reserve(CSApp application, Priority priority,
|
|
|
|
- CSNode node, Resource resource) {
|
|
|
|
- application.reserveResource(node, priority, resource);
|
|
|
|
- node.reserveResource(application, priority, resource);
|
|
|
|
|
|
+ private void reserve(SchedulerApp application, Priority priority,
|
|
|
|
+ SchedulerNode node, RMContainer rmContainer, Container container) {
|
|
|
|
+ rmContainer = application.reserve(node, priority, rmContainer, container);
|
|
|
|
+ node.reserveResource(application, priority, rmContainer);
|
|
|
|
+
|
|
|
|
+ // Update reserved metrics if this is the first reservation
|
|
|
|
+ if (rmContainer == null) {
|
|
|
|
+ getMetrics().reserveResource(
|
|
|
|
+ application.getUser(), container.getResource());
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
- private void unreserve(CSApp application, Priority priority,
|
|
|
|
- CSNode node) {
|
|
|
|
|
|
+ private void unreserve(SchedulerApp application, Priority priority,
|
|
|
|
+ SchedulerNode node, RMContainer rmContainer) {
|
|
// Done with the reservation?
|
|
// Done with the reservation?
|
|
- if (application.isReserved(node, priority)) {
|
|
|
|
- application.unreserveResource(node, priority);
|
|
|
|
- node.unreserveResource(application, priority);
|
|
|
|
- }
|
|
|
|
|
|
+ application.unreserve(node, priority);
|
|
|
|
+ node.unreserveResource(application);
|
|
|
|
+
|
|
|
|
+ // Update reserved metrics
|
|
|
|
+ getMetrics().unreserveResource(
|
|
|
|
+ application.getUser(), rmContainer.getContainer().getResource());
|
|
}
|
|
}
|
|
|
|
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public void completedContainer(Resource clusterResource,
|
|
public void completedContainer(Resource clusterResource,
|
|
- Container container, Resource containerResource, CSApp application) {
|
|
|
|
|
|
+ SchedulerApp application, SchedulerNode node, RMContainer rmContainer,
|
|
|
|
+ RMContainerEventType event) {
|
|
if (application != null) {
|
|
if (application != null) {
|
|
// Careful! Locking order is important!
|
|
// Careful! Locking order is important!
|
|
synchronized (this) {
|
|
synchronized (this) {
|
|
|
|
+
|
|
|
|
+ Container container = rmContainer.getContainer();
|
|
|
|
|
|
- // Inform the application - this might be an allocated container or
|
|
|
|
- // an unfulfilled reservation
|
|
|
|
- // TODO: acm: refactor2 FIXME
|
|
|
|
- //application.completedContainer(container, containerResource);
|
|
|
|
-
|
|
|
|
|
|
+ // Inform the application & the node
|
|
|
|
+ // Note: It's safe to assume that all state changes to RMContainer
|
|
|
|
+ // happen under scheduler's lock...
|
|
|
|
+ // So, this is, in effect, a transaction across application & node
|
|
|
|
+ if (rmContainer.getState() == RMContainerState.RESERVED) {
|
|
|
|
+ application.unreserve(node, rmContainer.getReservedPriority());
|
|
|
|
+ node.unreserveResource(application);
|
|
|
|
+ } else {
|
|
|
|
+ application.containerCompleted(rmContainer, event);
|
|
|
|
+ node.releaseContainer(container);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+
|
|
// Book-keeping
|
|
// Book-keeping
|
|
releaseResource(clusterResource,
|
|
releaseResource(clusterResource,
|
|
- application.getUser(), containerResource);
|
|
|
|
|
|
+ application.getUser(), container.getResource());
|
|
|
|
|
|
LOG.info("completedContainer" +
|
|
LOG.info("completedContainer" +
|
|
" container=" + container +
|
|
" container=" + container +
|
|
- " resource=" + containerResource +
|
|
|
|
|
|
+ " resource=" + container.getResource() +
|
|
" queue=" + this +
|
|
" queue=" + this +
|
|
" util=" + getUtilization() +
|
|
" util=" + getUtilization() +
|
|
" used=" + usedResources +
|
|
" used=" + usedResources +
|
|
@@ -988,29 +1000,41 @@ public class LeafQueue implements Queue {
|
|
}
|
|
}
|
|
|
|
|
|
// Inform the parent queue
|
|
// Inform the parent queue
|
|
- parent.completedContainer(clusterResource, container,
|
|
|
|
- containerResource, application);
|
|
|
|
|
|
+ parent.completedContainer(clusterResource, application,
|
|
|
|
+ node, rmContainer, event);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
private synchronized void allocateResource(Resource clusterResource,
|
|
private synchronized void allocateResource(Resource clusterResource,
|
|
String userName, Resource resource) {
|
|
String userName, Resource resource) {
|
|
|
|
+ // Update queue metrics
|
|
Resources.addTo(usedResources, resource);
|
|
Resources.addTo(usedResources, resource);
|
|
updateResource(clusterResource);
|
|
updateResource(clusterResource);
|
|
++numContainers;
|
|
++numContainers;
|
|
|
|
|
|
|
|
+ // Update user metrics
|
|
User user = getUser(userName);
|
|
User user = getUser(userName);
|
|
user.assignContainer(resource);
|
|
user.assignContainer(resource);
|
|
|
|
+
|
|
|
|
+ LOG.info(getQueueName() +
|
|
|
|
+ " used=" + usedResources + " numContainers=" + numContainers +
|
|
|
|
+ " user=" + userName + " resources=" + user.getConsumedResources());
|
|
}
|
|
}
|
|
|
|
|
|
private synchronized void releaseResource(Resource clusterResource,
|
|
private synchronized void releaseResource(Resource clusterResource,
|
|
String userName, Resource resource) {
|
|
String userName, Resource resource) {
|
|
|
|
+ // Update queue metrics
|
|
Resources.subtractFrom(usedResources, resource);
|
|
Resources.subtractFrom(usedResources, resource);
|
|
updateResource(clusterResource);
|
|
updateResource(clusterResource);
|
|
--numContainers;
|
|
--numContainers;
|
|
|
|
|
|
|
|
+ // Update user metrics
|
|
User user = getUser(userName);
|
|
User user = getUser(userName);
|
|
user.releaseContainer(resource);
|
|
user.releaseContainer(resource);
|
|
|
|
+
|
|
|
|
+ LOG.info(getQueueName() +
|
|
|
|
+ " used=" + usedResources + " numContainers=" + numContainers +
|
|
|
|
+ " user=" + userName + " resources=" + user.getConsumedResources());
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -1062,7 +1086,7 @@ public class LeafQueue implements Queue {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public void recoverContainer(Resource clusterResource,
|
|
public void recoverContainer(Resource clusterResource,
|
|
- CSApp application, Container container) {
|
|
|
|
|
|
+ SchedulerApp application, Container container) {
|
|
// Careful! Locking order is important!
|
|
// Careful! Locking order is important!
|
|
synchronized (this) {
|
|
synchronized (this) {
|
|
allocateResource(clusterResource, application.getUser(), container.getResource());
|
|
allocateResource(clusterResource, application.getUser(), container.getResource());
|