|
@@ -223,7 +223,7 @@ public class LeafQueue implements CSQueue {
|
|
|
{
|
|
|
// Sanity check
|
|
|
CSQueueUtils.checkMaxCapacity(getQueueName(), capacity, maximumCapacity);
|
|
|
- float absCapacity = parent.getAbsoluteCapacity() * capacity;
|
|
|
+ float absCapacity = getParent().getAbsoluteCapacity() * capacity;
|
|
|
CSQueueUtils.checkAbsoluteCapacities(getQueueName(), absCapacity, absoluteMaxCapacity);
|
|
|
|
|
|
this.capacity = capacity;
|
|
@@ -256,7 +256,7 @@ public class LeafQueue implements CSQueue {
|
|
|
|
|
|
// Update metrics
|
|
|
CSQueueUtils.updateQueueStatistics(
|
|
|
- this, parent, clusterResource, minimumAllocation);
|
|
|
+ this, getParent(), clusterResource, minimumAllocation);
|
|
|
|
|
|
LOG.info("Initializing " + queueName + "\n" +
|
|
|
"capacity = " + capacity +
|
|
@@ -339,10 +339,15 @@ public class LeafQueue implements CSQueue {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public CSQueue getParent() {
|
|
|
+ public synchronized CSQueue getParent() {
|
|
|
return parent;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public synchronized void setParent(CSQueue newParentQueue) {
|
|
|
+ this.parent = (ParentQueue)newParentQueue;
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public String getQueueName() {
|
|
|
return queueName;
|
|
@@ -350,7 +355,7 @@ public class LeafQueue implements CSQueue {
|
|
|
|
|
|
@Override
|
|
|
public String getQueuePath() {
|
|
|
- return parent.getQueuePath() + "." + getQueueName();
|
|
|
+ return getParent().getQueuePath() + "." + getQueueName();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -430,7 +435,9 @@ public class LeafQueue implements CSQueue {
|
|
|
synchronized void setMaxCapacity(float maximumCapacity) {
|
|
|
// Sanity check
|
|
|
CSQueueUtils.checkMaxCapacity(getQueueName(), capacity, maximumCapacity);
|
|
|
- float absMaxCapacity = CSQueueUtils.computeAbsoluteMaximumCapacity(maximumCapacity, parent);
|
|
|
+ float absMaxCapacity =
|
|
|
+ CSQueueUtils.computeAbsoluteMaximumCapacity(
|
|
|
+ maximumCapacity, getParent());
|
|
|
CSQueueUtils.checkAbsoluteCapacities(getQueueName(), absoluteCapacity, absMaxCapacity);
|
|
|
|
|
|
this.maximumCapacity = maximumCapacity;
|
|
@@ -453,10 +460,6 @@ public class LeafQueue implements CSQueue {
|
|
|
this.userLimitFactor = userLimitFactor;
|
|
|
}
|
|
|
|
|
|
- synchronized void setParentQueue(CSQueue parent) {
|
|
|
- this.parent = parent;
|
|
|
- }
|
|
|
-
|
|
|
@Override
|
|
|
public synchronized int getNumApplications() {
|
|
|
return getNumPendingApplications() + getNumActiveApplications();
|
|
@@ -559,26 +562,28 @@ public class LeafQueue implements CSQueue {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized void reinitialize(CSQueue queue, Resource clusterResource)
|
|
|
+ public synchronized void reinitialize(
|
|
|
+ CSQueue newlyParsedQueue, Resource clusterResource)
|
|
|
throws IOException {
|
|
|
// Sanity check
|
|
|
- if (!(queue instanceof LeafQueue) ||
|
|
|
- !queue.getQueuePath().equals(getQueuePath())) {
|
|
|
+ if (!(newlyParsedQueue instanceof LeafQueue) ||
|
|
|
+ !newlyParsedQueue.getQueuePath().equals(getQueuePath())) {
|
|
|
throw new IOException("Trying to reinitialize " + getQueuePath() +
|
|
|
- " from " + queue.getQueuePath());
|
|
|
+ " from " + newlyParsedQueue.getQueuePath());
|
|
|
}
|
|
|
|
|
|
- LeafQueue leafQueue = (LeafQueue)queue;
|
|
|
+ LeafQueue newlyParsedLeafQueue = (LeafQueue)newlyParsedQueue;
|
|
|
setupQueueConfigs(
|
|
|
clusterResource,
|
|
|
- leafQueue.capacity, leafQueue.absoluteCapacity,
|
|
|
- leafQueue.maximumCapacity, leafQueue.absoluteMaxCapacity,
|
|
|
- leafQueue.userLimit, leafQueue.userLimitFactor,
|
|
|
- leafQueue.maxApplications,
|
|
|
- leafQueue.getMaxApplicationsPerUser(),
|
|
|
- leafQueue.getMaximumActiveApplications(),
|
|
|
- leafQueue.getMaximumActiveApplicationsPerUser(),
|
|
|
- leafQueue.state, leafQueue.acls);
|
|
|
+ newlyParsedLeafQueue.capacity, newlyParsedLeafQueue.absoluteCapacity,
|
|
|
+ newlyParsedLeafQueue.maximumCapacity,
|
|
|
+ newlyParsedLeafQueue.absoluteMaxCapacity,
|
|
|
+ newlyParsedLeafQueue.userLimit, newlyParsedLeafQueue.userLimitFactor,
|
|
|
+ newlyParsedLeafQueue.maxApplications,
|
|
|
+ newlyParsedLeafQueue.getMaxApplicationsPerUser(),
|
|
|
+ newlyParsedLeafQueue.getMaximumActiveApplications(),
|
|
|
+ newlyParsedLeafQueue.getMaximumActiveApplicationsPerUser(),
|
|
|
+ newlyParsedLeafQueue.state, newlyParsedLeafQueue.acls);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -591,7 +596,7 @@ public class LeafQueue implements CSQueue {
|
|
|
}
|
|
|
|
|
|
// Check if parent-queue allows access
|
|
|
- return parent.hasAccess(acl, user);
|
|
|
+ return getParent().hasAccess(acl, user);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -649,10 +654,10 @@ public class LeafQueue implements CSQueue {
|
|
|
|
|
|
// Inform the parent queue
|
|
|
try {
|
|
|
- parent.submitApplication(application, userName, queue);
|
|
|
+ getParent().submitApplication(application, userName, queue);
|
|
|
} catch (AccessControlException ace) {
|
|
|
LOG.info("Failed to submit application to parent-queue: " +
|
|
|
- parent.getQueuePath(), ace);
|
|
|
+ getParent().getQueuePath(), ace);
|
|
|
removeApplication(application, user);
|
|
|
throw ace;
|
|
|
}
|
|
@@ -708,7 +713,7 @@ public class LeafQueue implements CSQueue {
|
|
|
}
|
|
|
|
|
|
// Inform the parent queue
|
|
|
- parent.finishApplication(application, queue);
|
|
|
+ getParent().finishApplication(application, queue);
|
|
|
}
|
|
|
|
|
|
public synchronized void removeApplication(FiCaSchedulerApp application, User user) {
|
|
@@ -1183,34 +1188,32 @@ public class LeafQueue implements CSQueue {
|
|
|
return (rmContainer != null) ? rmContainer.getContainer() :
|
|
|
createContainer(application, node, capability, priority);
|
|
|
}
|
|
|
-
|
|
|
|
|
|
- public Container createContainer(FiCaSchedulerApp application, FiCaSchedulerNode node,
|
|
|
+ Container createContainer(FiCaSchedulerApp application, FiCaSchedulerNode node,
|
|
|
Resource capability, Priority priority) {
|
|
|
-
|
|
|
+
|
|
|
NodeId nodeId = node.getRMNode().getNodeID();
|
|
|
ContainerId containerId = BuilderUtils.newContainerId(application
|
|
|
.getApplicationAttemptId(), application.getNewContainerId());
|
|
|
- ContainerToken containerToken = null;
|
|
|
-
|
|
|
- // If security is enabled, send the container-tokens too.
|
|
|
- if (UserGroupInformation.isSecurityEnabled()) {
|
|
|
- containerToken =
|
|
|
- containerTokenSecretManager.createContainerToken(containerId, nodeId,
|
|
|
- application.getUser(), capability);
|
|
|
- if (containerToken == null) {
|
|
|
- return null; // Try again later.
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
+
|
|
|
// Create the container
|
|
|
Container container = BuilderUtils.newContainer(containerId, nodeId,
|
|
|
node.getRMNode().getHttpAddress(), capability, priority,
|
|
|
- containerToken);
|
|
|
-
|
|
|
+ null);
|
|
|
+
|
|
|
return container;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Create <code>ContainerToken</code>, only in secure-mode
|
|
|
+ */
|
|
|
+ ContainerToken createContainerToken(
|
|
|
+ FiCaSchedulerApp application, Container container) {
|
|
|
+ return containerTokenSecretManager.createContainerToken(
|
|
|
+ container.getId(), container.getNodeId(),
|
|
|
+ application.getUser(), container.getResource());
|
|
|
+ }
|
|
|
+
|
|
|
private Resource assignContainer(Resource clusterResource, FiCaSchedulerNode node,
|
|
|
FiCaSchedulerApp application, Priority priority,
|
|
|
ResourceRequest request, NodeType type, RMContainer rmContainer) {
|
|
@@ -1246,6 +1249,17 @@ public class LeafQueue implements CSQueue {
|
|
|
unreserve(application, priority, node, rmContainer);
|
|
|
}
|
|
|
|
|
|
+ // Create container tokens in secure-mode
|
|
|
+ if (UserGroupInformation.isSecurityEnabled()) {
|
|
|
+ ContainerToken containerToken =
|
|
|
+ createContainerToken(application, container);
|
|
|
+ if (containerToken == null) {
|
|
|
+ // Something went wrong...
|
|
|
+ return Resources.none();
|
|
|
+ }
|
|
|
+ container.setContainerToken(containerToken);
|
|
|
+ }
|
|
|
+
|
|
|
// Inform the application
|
|
|
RMContainer allocatedContainer =
|
|
|
application.allocate(type, node, priority, request, container);
|
|
@@ -1351,7 +1365,7 @@ public class LeafQueue implements CSQueue {
|
|
|
}
|
|
|
|
|
|
// Inform the parent queue
|
|
|
- parent.completedContainer(clusterResource, application,
|
|
|
+ getParent().completedContainer(clusterResource, application,
|
|
|
node, rmContainer, null, event);
|
|
|
}
|
|
|
}
|
|
@@ -1361,7 +1375,7 @@ public class LeafQueue implements CSQueue {
|
|
|
// Update queue metrics
|
|
|
Resources.addTo(usedResources, resource);
|
|
|
CSQueueUtils.updateQueueStatistics(
|
|
|
- this, parent, clusterResource, minimumAllocation);
|
|
|
+ this, getParent(), clusterResource, minimumAllocation);
|
|
|
++numContainers;
|
|
|
|
|
|
// Update user metrics
|
|
@@ -1386,7 +1400,7 @@ public class LeafQueue implements CSQueue {
|
|
|
// Update queue metrics
|
|
|
Resources.subtractFrom(usedResources, resource);
|
|
|
CSQueueUtils.updateQueueStatistics(
|
|
|
- this, parent, clusterResource, minimumAllocation);
|
|
|
+ this, getParent(), clusterResource, minimumAllocation);
|
|
|
--numContainers;
|
|
|
|
|
|
// Update user metrics
|
|
@@ -1417,7 +1431,7 @@ public class LeafQueue implements CSQueue {
|
|
|
|
|
|
// Update metrics
|
|
|
CSQueueUtils.updateQueueStatistics(
|
|
|
- this, parent, clusterResource, minimumAllocation);
|
|
|
+ this, getParent(), clusterResource, minimumAllocation);
|
|
|
|
|
|
// Update application properties
|
|
|
for (FiCaSchedulerApp application : activeApplications) {
|
|
@@ -1488,7 +1502,7 @@ public class LeafQueue implements CSQueue {
|
|
|
synchronized (this) {
|
|
|
allocateResource(clusterResource, application, container.getResource());
|
|
|
}
|
|
|
- parent.recoverContainer(clusterResource, application, container);
|
|
|
+ getParent().recoverContainer(clusterResource, application, container);
|
|
|
|
|
|
}
|
|
|
|