diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt index ed9c7e46c1c..292a42b2de3 100644 --- a/hadoop-mapreduce-project/CHANGES.txt +++ b/hadoop-mapreduce-project/CHANGES.txt @@ -134,6 +134,8 @@ Branch-2 ( Unreleased changes ) MAPREDUCE-4355. Add RunningJob.getJobStatus() (kkambatl via tucu) + MAPREDUCE-3451. Port Fair Scheduler to MR2 (pwendell via tucu) + IMPROVEMENTS MAPREDUCE-4146. Support limits on task status string length and number of diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Resources.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Resources.java index 4f41bfde8a0..668d1da8914 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Resources.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/Resources.java @@ -91,6 +91,16 @@ public class Resources { public static Resource multiply(Resource lhs, int by) { return multiplyTo(clone(lhs), by); } + + /** + * Mutliply a resource by a {@code double}. Note that integral + * resource quantites are subject to rounding during cast. + */ + public static Resource multiply(Resource lhs, double by) { + Resource out = clone(lhs); + out.setMemory((int) (lhs.getMemory() * by)); + return out; + } public static boolean equals(Resource lhs, Resource rhs) { return lhs.getMemory() == rhs.getMemory(); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java index 8054d0bf3ce..0dfdf20b52d 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java @@ -26,6 +26,14 @@ import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.event.EventHandler; +/** + * Represents the ResourceManager's view of an application container. See + * {@link RMContainerImpl} for an implementation. Containers may be in one + * of several states, given in {@link RMContainerState}. An RMContainer + * instance may exist even if there is no actual running container, such as + * when resources are being reserved to fill space for a future container + * allocation. + */ public interface RMContainer extends EventHandler { ContainerId getContainerId(); diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerEventType.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerEventType.java index 0cc37d0380a..ace443508cc 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerEventType.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerEventType.java @@ -20,10 +20,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmcontainer; public enum RMContainerEventType { - // Source: scheduler - START, - // Source: SchedulerApp + START, ACQUIRED, KILL, // Also from Node on NodeRemoval RESERVED, diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java index 6c3e7952e3d..f776f5efd57 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java @@ -45,6 +45,9 @@ public class SchedulerUtils { public static final String LOST_CONTAINER = "Container released on a *lost* node"; + public static final String PREEMPTED_CONTAINER = + "Container preempted by scheduler"; + public static final String COMPLETED_APPLICATION = "Container of a completed application"; diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfigurationException.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfigurationException.java new file mode 100644 index 00000000000..38672b5e5f9 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfigurationException.java @@ -0,0 +1,35 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; + +/** + * Thrown when the allocation file for {@link QueueManager} is malformed. + */ +@Private +@Unstable +public class AllocationConfigurationException extends Exception { + private static final long serialVersionUID = 4046517047810854249L; + + public AllocationConfigurationException(String message) { + super(message); + } +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AppSchedulable.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AppSchedulable.java new file mode 100644 index 00000000000..7f253abd214 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AppSchedulable.java @@ -0,0 +1,337 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.security.UserGroupInformation; +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.NodeId; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.factories.RecordFactory; +import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; +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.rmnode.RMNode; +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.SchedulerApp; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; +import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager; +import org.apache.hadoop.yarn.util.BuilderUtils; + +@Private +@Unstable +public class AppSchedulable extends Schedulable { + private FairScheduler scheduler; + private FSSchedulerApp app; + private Resource demand = Resources.createResource(0); + private boolean runnable = false; // everyone starts as not runnable + private long startTime; + private static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); + private static final Log LOG = LogFactory.getLog(AppSchedulable.class); + private FSQueue queue; + private ContainerTokenSecretManager containerTokenSecretManager; + + public AppSchedulable(FairScheduler scheduler, FSSchedulerApp app, FSQueue queue) { + this.scheduler = scheduler; + this.app = app; + this.startTime = System.currentTimeMillis(); + this.queue = queue; + this.containerTokenSecretManager = scheduler. + getContainerTokenSecretManager(); + } + + @Override + public String getName() { + return app.getApplicationId().toString(); + } + + public SchedulerApp getApp() { + return app; + } + + @Override + public void updateDemand() { + demand = Resources.createResource(0); + // Demand is current consumption plus outstanding requests + Resources.addTo(demand, app.getCurrentConsumption()); + + // Add up outstanding resource requests + for (Priority p : app.getPriorities()) { + for (ResourceRequest r : app.getResourceRequests(p).values()) { + Resource total = Resources.multiply(r.getCapability(), r.getNumContainers()); + Resources.addTo(demand, total); + } + } + } + + @Override + public Resource getDemand() { + return demand; + } + + @Override + public long getStartTime() { + return startTime; + } + + @Override + public void redistributeShare() {} + + @Override + public Resource getResourceUsage() { + return this.app.getCurrentConsumption(); + } + + + @Override + public Resource getMinShare() { + return Resources.createResource(0); + } + + /** + * Get metrics reference from containing queue. + */ + public QueueMetrics getMetrics() { + return this.queue.getQueueSchedulable().getMetrics(); + } + + @Override + public double getWeight() { + return scheduler.getAppWeight(this); + } + + @Override + public Priority getPriority() { + // Right now per-app priorities are not passed to scheduler, + // so everyone has the same priority. + Priority p = recordFactory.newRecordInstance(Priority.class); + p.setPriority(1); + return p; + } + + /** + * Is this application runnable? Runnable means that the user and queue + * application counts are within configured quotas. + */ + public boolean getRunnable() { + return runnable; + } + + public void setRunnable(boolean runnable) { + this.runnable = runnable; + } + + /** + * Create and return a container object reflecting an allocation for the + * given appliction on the given node with the given capability and + * priority. + */ + public Container createContainer(SchedulerApp application, SchedulerNode 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, + capability); + if (containerToken == null) { + return null; // Try again later. + } + } + + // Create the container + Container container = BuilderUtils.newContainer(containerId, nodeId, + node.getRMNode().getHttpAddress(), capability, priority, + containerToken); + + return container; + } + + /** + * Reserve a spot for {@code container} on this {@code node}. If + * the container is {@code alreadyReserved} on the node, simply + * update relevant bookeeping. This dispatches ro relevant handlers + * in the {@link SchedulerNode} and {@link SchedulerApp} classes. + */ + private void reserve(SchedulerApp application, Priority priority, + SchedulerNode node, Container container, boolean alreadyReserved) { + LOG.info("Making reservation: node=" + node.getHostName() + + " app_id=" + app.getApplicationId()); + if (!alreadyReserved) { + getMetrics().reserveResource(application.getUser(), container.getResource()); + RMContainer rmContainer = application.reserve(node, priority, null, + container); + node.reserveResource(application, priority, rmContainer); + getMetrics().reserveResource(this.app.getUser(), + container.getResource()); + scheduler.getRootQueueMetrics().reserveResource(this.app.getUser(), + container.getResource()); + } + + else { + RMContainer rmContainer = node.getReservedContainer(); + application.reserve(node, priority, rmContainer, container); + node.reserveResource(application, priority, rmContainer); + } + } + + /** + * Remove the reservation on {@code node} for {@ application} at the given + * {@link Priority}. This dispatches to the SchedulerApp and SchedulerNode + * handlers for an unreservation. + */ + private void unreserve(SchedulerApp application, Priority priority, + SchedulerNode node) { + RMContainer rmContainer = node.getReservedContainer(); + application.unreserve(node, priority); + node.unreserveResource(application); + getMetrics().unreserveResource( + application.getUser(), rmContainer.getContainer().getResource()); + scheduler.getRootQueueMetrics().unreserveResource( + application.getUser(), rmContainer.getContainer().getResource()); + } + + /** + * Assign a container to this node to facilitate {@code request}. If node does + * not have enough memory, create a reservation. This is called once we are + * sure the particular request should be facilitated by this node. + */ + private Resource assignContainer(SchedulerNode node, + SchedulerApp application, Priority priority, + ResourceRequest request, NodeType type, boolean reserved) { + + // How much does this request need? + Resource capability = request.getCapability(); + + // How much does the node have? + Resource available = node.getAvailableResource(); + + Container container = null; + if (reserved) { + container = node.getReservedContainer().getContainer(); + } else { + container = createContainer(application, node, capability, priority); + } + + // Can we allocate a container on this node? + int availableContainers = + available.getMemory() / capability.getMemory(); + + if (availableContainers > 0) { + // Inform the application of the new container for this request + RMContainer allocatedContainer = + application.allocate(type, node, priority, request, container); + if (allocatedContainer == null) { + // Did the application need this resource? + return Resources.none(); + } + else { + // TODO this should subtract resource just assigned + // TEMPROARY + getMetrics().setAvailableResourcesToQueue( + this.scheduler.getClusterCapacity()); + } + + + // If we had previously made a reservation, delete it + if (reserved) { + this.unreserve(application, priority, node); + } + + // Inform the node + node.allocateContainer(application.getApplicationId(), + allocatedContainer); + + return container.getResource(); + } else { + // The desired container won't fit here, so reserve + reserve(application, priority, node, container, reserved); + + return Resources.none(); + } + } + + + @Override + public Resource assignContainer(SchedulerNode node, boolean reserved) { + LOG.info("Node offered to app: " + getName() + " reserved: " + reserved); + + if (reserved) { + RMContainer rmContainer = node.getReservedContainer(); + Priority priority = rmContainer.getReservedPriority(); + + // Make sure the application still needs requests at this priority + if (app.getTotalRequiredResources(priority) == 0) { + this.unreserve(app, priority, node); + return Resources.none(); + } + } else { + // If this app is over quota, don't schedule anything + if (!(getRunnable())) { return Resources.none(); } + + } + // For each priority, see if we can schedule a node local, rack local + // or off-switch request. Rack of off-switch requests may be delayed + // (not scheduled) in order to promote better locality. + for (Priority priority : app.getPriorities()) { + app.addSchedulingOpportunity(priority); + NodeType allowedLocality = app.getAllowedLocalityLevel(priority, + scheduler.getNumClusterNodes(), scheduler.getNodeLocalityThreshold(), + scheduler.getRackLocalityThreshold()); + + ResourceRequest localRequest = app.getResourceRequest(priority, + node.getHostName()); + if (localRequest != null && localRequest.getNumContainers() != 0) { + return assignContainer(node, app, priority, + localRequest, NodeType.NODE_LOCAL, reserved); + } + + ResourceRequest rackLocalRequest = app.getResourceRequest(priority, + node.getRackName()); + if (rackLocalRequest != null && rackLocalRequest.getNumContainers() != 0 + && (allowedLocality.equals(NodeType.RACK_LOCAL) || + allowedLocality.equals(NodeType.OFF_SWITCH))) { + return assignContainer(node, app, priority, rackLocalRequest, + NodeType.RACK_LOCAL, reserved); + } + + ResourceRequest offSwitchRequest = app.getResourceRequest(priority, + RMNode.ANY); + if (offSwitchRequest != null && offSwitchRequest.getNumContainers() != 0 + && allowedLocality.equals(NodeType.OFF_SWITCH)) { + return assignContainer(node, app, priority, offSwitchRequest, + NodeType.OFF_SWITCH, reserved); + } + } + return Resources.none(); + } +} \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java new file mode 100644 index 00000000000..d7e6e7032b2 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java @@ -0,0 +1,82 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; + +import java.util.ArrayList; +import java.util.Collection; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApp; + +/** + * A queue containing several applications. + */ +@Private +@Unstable +public class FSQueue { + /** Queue name. */ + private String name; + + /** Applications in this specific queue; does not include children queues' jobs. */ + private Collection applications = new ArrayList(); + + /** Scheduling mode for jobs inside the queue (fair or FIFO) */ + private SchedulingMode schedulingMode; + + private FairScheduler scheduler; + + private FSQueueSchedulable queueSchedulable; + + public FSQueue(FairScheduler scheduler, String name) { + this.name = name; + this.queueSchedulable = new FSQueueSchedulable(scheduler, this); + this.scheduler = scheduler; + } + + public Collection getApplications() { + return applications; + } + + public void addApp(FSSchedulerApp app) { + applications.add(app); + queueSchedulable.addApp(new AppSchedulable(scheduler, app, this)); + } + + public void removeJob(SchedulerApp app) { + applications.remove(app); + queueSchedulable.removeApp(app); + } + + public String getName() { + return name; + } + + public SchedulingMode getSchedulingMode() { + return schedulingMode; + } + + public void setSchedulingMode(SchedulingMode schedulingMode) { + this.schedulingMode = schedulingMode; + } + + public FSQueueSchedulable getQueueSchedulable() { + return queueSchedulable; + } +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueueSchedulable.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueueSchedulable.java new file mode 100644 index 00000000000..62d39ef47a0 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueueSchedulable.java @@ -0,0 +1,285 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.authorize.AccessControlList; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.QueueACL; +import org.apache.hadoop.yarn.api.records.QueueInfo; +import org.apache.hadoop.yarn.api.records.QueueState; +import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.factories.RecordFactory; +import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; +import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue; +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; + +@Private +@Unstable +public class FSQueueSchedulable extends Schedulable implements Queue { + public static final Log LOG = LogFactory.getLog( + FSQueueSchedulable.class.getName()); + + private FairScheduler scheduler; + private FSQueue queue; + private QueueManager queueMgr; + private List appScheds = new LinkedList(); + private Resource demand = Resources.createResource(0); + private QueueMetrics metrics; + private RecordFactory recordFactory = + RecordFactoryProvider.getRecordFactory(null); + + // Variables used for preemption + long lastTimeAtMinShare; + long lastTimeAtHalfFairShare; + + public FSQueueSchedulable(FairScheduler scheduler, FSQueue queue) { + this.scheduler = scheduler; + this.queue = queue; + this.queueMgr = scheduler.getQueueManager(); + this.metrics = QueueMetrics.forQueue(this.getName(), null, true, scheduler.getConf()); + this.lastTimeAtMinShare = scheduler.getClock().getTime(); + this.lastTimeAtHalfFairShare = scheduler.getClock().getTime(); + } + + public void addApp(AppSchedulable app) { + appScheds.add(app); + } + + public void removeApp(SchedulerApp app) { + for (Iterator it = appScheds.iterator(); it.hasNext();) { + AppSchedulable appSched = it.next(); + if (appSched.getApp() == app) { + it.remove(); + break; + } + } + } + + /** + * Update demand by asking apps in the queue to update + */ + @Override + public void updateDemand() { + demand = Resources.createResource(0); + for (AppSchedulable sched: appScheds) { + sched.updateDemand(); + Resource toAdd = sched.getDemand(); + LOG.debug("Counting resource from " + sched.getName() + " " + toAdd.toString()); + LOG.debug("Total resource consumption for " + this.getName() + " now " + demand.toString()); + demand = Resources.add(demand, toAdd); + + } + // if demand exceeds the cap for this queue, limit to the max + Resource maxRes = queueMgr.getMaxResources(queue.getName()); + if(Resources.greaterThan(demand, maxRes)) { + demand = maxRes; + } + } + + /** + * Distribute the queue's fair share among its jobs + */ + @Override + public void redistributeShare() { + if (queue.getSchedulingMode() == SchedulingMode.FAIR) { + SchedulingAlgorithms.computeFairShares(appScheds, getFairShare()); + } else { + for (AppSchedulable sched: appScheds) { + sched.setFairShare(Resources.createResource(0)); + } + } + } + + @Override + public Resource getDemand() { + return demand; + } + + @Override + public Resource getMinShare() { + return queueMgr.getMinResources(queue.getName()); + } + + @Override + public double getWeight() { + return queueMgr.getQueueWeight(queue.getName()); + } + + @Override + public long getStartTime() { + return 0; + } + + @Override + public Resource assignContainer(SchedulerNode node, boolean reserved) { + LOG.debug("Node offered to queue: " + this.getName() + " reserved: " + reserved); + // If this queue is over its limit, reject + if (Resources.greaterThan(this.getResourceUsage(), + queueMgr.getMaxResources(queue.getName()))) { + return Resources.none(); + } + + // If this node already has reserved resources for an app, first try to + // finish allocating resources for that app. + if (reserved) { + for (AppSchedulable sched : appScheds) { + if (sched.getApp().getApplicationAttemptId() == + node.getReservedContainer().getApplicationAttemptId()) { + return sched.assignContainer(node, reserved); + } + } + return Resources.none(); // We should never get here + } + + // Otherwise, chose app to schedule based on given policy (fair vs fifo). + else { + SchedulingMode mode = queue.getSchedulingMode(); + + Comparator comparator; + if (mode == SchedulingMode.FIFO) { + comparator = new SchedulingAlgorithms.FifoComparator(); + } else if (mode == SchedulingMode.FAIR) { + comparator = new SchedulingAlgorithms.FairShareComparator(); + } else { + throw new RuntimeException("Unsupported queue scheduling mode " + mode); + } + + Collections.sort(appScheds, comparator); + for (AppSchedulable sched: appScheds) { + return sched.assignContainer(node, reserved); + } + + return Resources.none(); + } + + } + + @Override + public String getName() { + return queue.getName(); + } + + FSQueue getQueue() { + return queue; + } + + public Collection getAppSchedulables() { + return appScheds; + } + + public long getLastTimeAtMinShare() { + return lastTimeAtMinShare; + } + + public void setLastTimeAtMinShare(long lastTimeAtMinShare) { + this.lastTimeAtMinShare = lastTimeAtMinShare; + } + + public long getLastTimeAtHalfFairShare() { + return lastTimeAtHalfFairShare; + } + + public void setLastTimeAtHalfFairShare(long lastTimeAtHalfFairShare) { + this.lastTimeAtHalfFairShare = lastTimeAtHalfFairShare; + } + + @Override + public QueueMetrics getMetrics() { + return metrics; + } + + @Override + public Resource getResourceUsage() { + Resource usage = Resources.createResource(0); + for (AppSchedulable app : appScheds) { + Resources.addTo(usage, app.getResourceUsage()); + } + return usage; + } + + @Override + public Priority getPriority() { + Priority p = recordFactory.newRecordInstance(Priority.class); + p.setPriority(1); + return p; + } + + @Override + public Map getQueueAcls() { + Map acls = this.queueMgr.getQueueAcls(this.getName()); + return new HashMap(acls); + } + + @Override + public QueueInfo getQueueInfo(boolean includeChildQueues, boolean recursive) { + QueueInfo queueInfo = recordFactory.newRecordInstance(QueueInfo.class); + queueInfo.setQueueName(getQueueName()); + // TODO: we might change these queue metrics around a little bit + // to match the semantics of the fair scheduler. + queueInfo.setCapacity((float) getFairShare().getMemory() / + scheduler.getClusterCapacity().getMemory()); + queueInfo.setCapacity((float) getResourceUsage().getMemory() / + scheduler.getClusterCapacity().getMemory()); + + queueInfo.setChildQueues(new ArrayList()); + queueInfo.setQueueState(QueueState.RUNNING); + return queueInfo; + } + + @Override + public List getQueueUserAclInfo(UserGroupInformation user) { + QueueUserACLInfo userAclInfo = + recordFactory.newRecordInstance(QueueUserACLInfo.class); + List operations = new ArrayList(); + for (QueueACL operation : QueueACL.values()) { + Map acls = this.queueMgr.getQueueAcls(this.getName()); + if (acls.get(operation).isUserAllowed(user)) { + operations.add(operation); + } + } + + userAclInfo.setQueueName(getQueueName()); + userAclInfo.setUserAcls(operations); + return Collections.singletonList(userAclInfo); + } + + @Override + public String getQueueName() { + return getName(); + } +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerApp.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerApp.java new file mode 100644 index 00000000000..de18e3abc56 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerApp.java @@ -0,0 +1,161 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.server.resourcemanager.RMContext; +import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApp; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; + +/** + * This class extends the application lifecycle management contained with + * the {@link SchedulerApp} class and adds delay-scheduling information + * specific to the FairScheduler. + */ +public class FSSchedulerApp extends SchedulerApp { + private static final Log LOG = LogFactory.getLog(SchedulerApp.class); + + /** + * Delay scheduling: We often want to prioritize scheduling of node-local + * containers over rack-local or off-switch containers. To acheive this + * we first only allow node-local assigments for a given prioirty level, + * then relax the locality threshold once we've had a long enough period + * without succesfully scheduling. We measure both the number of "missed" + * scheduling opportunities since the last container was scheduled + * at the current allowed level and the time since the last container + * was scheduled. Currently we use only the former. + */ + + // Current locality threshold + final Map allowedLocalityLevel = new HashMap< + Priority, NodeType>(); + + // Time of the last container scheduled at the current allowed level + Map lastScheduledContainer = new HashMap(); + + public FSSchedulerApp(ApplicationAttemptId applicationAttemptId, + String user, Queue queue, ActiveUsersManager activeUsersManager, + RMContext rmContext, ApplicationStore store) { + super(applicationAttemptId, user, queue, activeUsersManager, + rmContext, store); + } + + /** + * Should be called when an application has successfully scheduled a container, + * or when the scheduling locality threshold is relaxed. + * Reset various internal counters which affect delay scheduling + * + * @param priority The priority of the container scheduled. + */ + synchronized public void resetSchedulingOpportunities(Priority priority) { + this.lastScheduledContainer.put(priority, System.currentTimeMillis()); + super.resetSchedulingOpportunities(priority); + } + + /** + * Return the level at which we are allowed to schedule containers, given the + * current size of the cluster and thresholds indicating how many nodes to + * fail at (as a fraction of cluster size) before relaxing scheduling + * constraints. + */ + public synchronized NodeType getAllowedLocalityLevel(Priority priority, + int numNodes, double nodeLocalityThreshold, double rackLocalityThreshold) { + // upper limit on threshold + if (nodeLocalityThreshold > 1.0) { nodeLocalityThreshold = 1.0; } + if (rackLocalityThreshold > 1.0) { rackLocalityThreshold = 1.0; } + + // If delay scheduling is not being used, can schedule anywhere + if (nodeLocalityThreshold < 0.0 || rackLocalityThreshold < 0.0) { + return NodeType.OFF_SWITCH; + } + + // Default level is NODE_LOCAL + if (!allowedLocalityLevel.containsKey(priority)) { + allowedLocalityLevel.put(priority, NodeType.NODE_LOCAL); + return NodeType.NODE_LOCAL; + } + + NodeType allowed = allowedLocalityLevel.get(priority); + + // If level is already most liberal, we're done + if (allowed.equals(NodeType.OFF_SWITCH)) return NodeType.OFF_SWITCH; + + double threshold = allowed.equals(NodeType.NODE_LOCAL) ? nodeLocalityThreshold : + rackLocalityThreshold; + + // Relax locality constraints once we've surpassed threshold. + if (this.getSchedulingOpportunities(priority) > (numNodes * threshold)) { + if (allowed.equals(NodeType.NODE_LOCAL)) { + allowedLocalityLevel.put(priority, NodeType.RACK_LOCAL); + this.resetSchedulingOpportunities(priority); + } + else if (allowed.equals(NodeType.RACK_LOCAL)) { + allowedLocalityLevel.put(priority, NodeType.OFF_SWITCH); + this.resetSchedulingOpportunities(priority); + } + } + return allowedLocalityLevel.get(priority); + } + + + synchronized public RMContainer allocate(NodeType type, SchedulerNode node, + Priority priority, ResourceRequest request, + Container container) { + // Update allowed locality level + NodeType allowed = this.allowedLocalityLevel.get(priority); + if (allowed != null) { + if (allowed.equals(NodeType.OFF_SWITCH) && + (type.equals(NodeType.NODE_LOCAL) || + type.equals(NodeType.RACK_LOCAL))) { + this.resetAllowedLocalityLevel(priority, type); + } + else if (allowed.equals(NodeType.RACK_LOCAL) && + type.equals(NodeType.NODE_LOCAL)) { + this.resetAllowedLocalityLevel(priority, type); + } + } + return super.allocate(type, node, priority, request, container); + } + + /** + * Should be called when the scheduler assigns a container at a higher + * degree of locality than the current threshold. Reset the allowed locality + * level to a higher degree of locality. + */ + public synchronized void resetAllowedLocalityLevel(Priority priority, + NodeType level) { + NodeType old = allowedLocalityLevel.get(priority); + LOG.info("Raising locality level from " + old + " to " + level + " at " + + " priority " + priority); + allowedLocalityLevel.put(priority, level); + } +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java new file mode 100644 index 00000000000..2b311f45e2a --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java @@ -0,0 +1,981 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.Clock; +import org.apache.hadoop.yarn.SystemClock; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.QueueACL; +import org.apache.hadoop.yarn.api.records.QueueInfo; +import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; +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.factory.providers.RecordFactoryProvider; +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.recovery.Store.RMState; +import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; +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.RMContainerState; +import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApp; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; +import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager; + +@LimitedPrivate("yarn") +@Unstable +@SuppressWarnings("unchecked") +public class FairScheduler implements ResourceScheduler { + + private boolean initialized; + private FairSchedulerConfiguration conf; + private ContainerTokenSecretManager containerTokenSecretManager; + private RMContext rmContext; + private Resource minimumAllocation; + private Resource maximumAllocation; + private QueueManager queueMgr; + private Clock clock; + + private static final Log LOG = LogFactory.getLog(FairScheduler.class); + + // How often fair shares are re-calculated (ms) + protected long UPDATE_INTERVAL = 500; + + // Whether to use username in place of "default" queue name + private boolean userAsDefaultQueue = false; + + private final static List EMPTY_CONTAINER_LIST = + new ArrayList(); + + private static final Allocation EMPTY_ALLOCATION = + new Allocation(EMPTY_CONTAINER_LIST, Resources.createResource(0)); + + // Aggregate metrics + QueueMetrics rootMetrics; + + //Time when we last updated preemption vars + protected long lastPreemptionUpdateTime; + //Time we last ran preemptTasksIfNecessary + private long lastPreemptCheckTime; + + + // This stores per-application scheduling information, indexed by + // attempt ID's for fast lookup. + protected Map applications + = new HashMap(); + + // Nodes in the cluster, indexed by NodeId + private Map nodes = + new ConcurrentHashMap(); + + // Aggregate capacity of the cluster + private Resource clusterCapacity = + RecordFactoryProvider.getRecordFactory(null).newRecordInstance(Resource.class); + + // How often tasks are preempted (must be longer than a couple + // of heartbeats to give task-kill commands a chance to act). + protected long preemptionInterval = 15000; + + protected boolean preemptionEnabled; + protected boolean sizeBasedWeight; // Give larger weights to larger jobs + protected WeightAdjuster weightAdjuster; // Can be null for no weight adjuster + protected double nodeLocalityThreshold; // Cluster threshold for node locality + protected double rackLocalityThreshold; // Cluster threshold for rack locality + private FairSchedulerEventLog eventLog; // Machine-readable event log + protected boolean assignMultiple; // Allocate multiple containers per heartbeat + protected int maxAssign; // Max containers to assign per heartbeat + + + public FairSchedulerConfiguration getConf() { + return this.conf; + } + + public QueueManager getQueueManager() { + return this.queueMgr; + } + + public List getQueueSchedulables() { + List scheds = new ArrayList(); + for (FSQueue queue: queueMgr.getQueues()) { + scheds.add(queue.getQueueSchedulable()); + } + return scheds; + } + + private RMContainer getRMContainer(ContainerId containerId) { + SchedulerApp application = + applications.get(containerId.getApplicationAttemptId()); + return (application == null) ? null : application.getRMContainer(containerId); + } + + /** + * A runnable which calls {@link FairScheduler#update()} every + * UPDATE_INTERVAL milliseconds. + */ + private class UpdateThread implements Runnable { + public void run() { + while (initialized) { + try { + Thread.sleep(UPDATE_INTERVAL); + update(); + preemptTasksIfNecessary(); + } catch (Exception e) { + LOG.error("Exception in fair scheduler UpdateThread", e); + } + } + } + } + + /** + * Recompute the internal variables used by the scheduler - per-job weights, + * fair shares, deficits, minimum slot allocations, and amount of used and + * required resources per job. + */ + protected void update() { + synchronized (this) { + queueMgr.reloadAllocsIfNecessary(); // Relaod alloc file + updateRunnability(); // Set job runnability based on user/queue limits + updatePreemptionVariables(); // Determine if any queues merit preemption + + // Update demands of apps and queues + for (FSQueue queue: queueMgr.getQueues()) { + queue.getQueueSchedulable().updateDemand(); + } + + // Compute fair shares based on updated demands + List queueScheds = this.getQueueSchedulables(); + SchedulingAlgorithms.computeFairShares( + queueScheds, clusterCapacity); + + // Update queue metrics for this queue + for (FSQueueSchedulable sched : queueScheds) { + sched.getMetrics().setAvailableResourcesToQueue(sched.getFairShare()); + } + + // Use the computed shares to assign shares within each queue + for (FSQueue queue: queueMgr.getQueues()) { + queue.getQueueSchedulable().redistributeShare(); + } + + // Update recorded capacity of root queue (child queues are updated + // when fair share is calculated). + rootMetrics.setAvailableResourcesToQueue(clusterCapacity); + } + } + + /** + * Update the preemption fields for all QueueScheduables, i.e. the times since + * each queue last was at its guaranteed share and at > 1/2 of its fair share + * for each type of task. + */ + private void updatePreemptionVariables() { + long now = clock.getTime(); + lastPreemptionUpdateTime = now; + for (FSQueueSchedulable sched: getQueueSchedulables()) { + if (!isStarvedForMinShare(sched)) { + sched.setLastTimeAtMinShare(now); + } + if (!isStarvedForFairShare(sched)) { + sched.setLastTimeAtHalfFairShare(now); + } + } + } + + /** + * Is a queue below its min share for the given task type? + */ + boolean isStarvedForMinShare(FSQueueSchedulable sched) { + Resource desiredShare = Resources.min(sched.getMinShare(), sched.getDemand()); + return Resources.lessThan(sched.getResourceUsage(), desiredShare); + } + + /** + * Is a queue being starved for fair share for the given task type? + * This is defined as being below half its fair share. + */ + boolean isStarvedForFairShare(FSQueueSchedulable sched) { + Resource desiredFairShare = Resources.max( + Resources.multiply(sched.getFairShare(), .5), sched.getDemand()); + return Resources.lessThan(sched.getResourceUsage(), desiredFairShare); + } + + /** + * Check for queues that need tasks preempted, either because they have been + * below their guaranteed share for minSharePreemptionTimeout or they + * have been below half their fair share for the fairSharePreemptionTimeout. + * If such queues exist, compute how many tasks of each type need to be + * preempted and then select the right ones using preemptTasks. + * + * This method computes and logs the number of tasks we want to preempt even + * if preemption is disabled, for debugging purposes. + */ + protected void preemptTasksIfNecessary() { + if (!preemptionEnabled) + return; + + long curTime = clock.getTime(); + if (curTime - lastPreemptCheckTime < preemptionInterval) + return; + lastPreemptCheckTime = curTime; + + Resource resToPreempt = Resources.none(); + + for (FSQueueSchedulable sched: getQueueSchedulables()) { + resToPreempt = Resources.add(resToPreempt, resToPreempt(sched, curTime)); + } + if (Resources.greaterThan(resToPreempt, Resources.none())) { + preemptResources(getQueueSchedulables(), resToPreempt); + } + } + + /** + * Preempt a quantity of resources from a list of QueueSchedulables. + * The policy for this is to pick apps from queues that are over their fair + * share, but make sure that no queue is placed below its fair share in the + * process. We further prioritize preemption by choosing containers with + * lowest priority to preempt. + */ + protected void preemptResources(List scheds, Resource toPreempt) { + if (scheds.isEmpty() || Resources.equals(toPreempt, Resources.none())) + return; + + Map apps = new HashMap(); + Map queues = new HashMap(); + + // Collect running containers from over-scheduled queues + List runningContainers = new ArrayList(); + for (FSQueueSchedulable sched: scheds) { + if (Resources.greaterThan(sched.getResourceUsage(), sched.getFairShare())) { + for (AppSchedulable as: sched.getAppSchedulables()) { + for (RMContainer c : as.getApp().getLiveContainers()) { + runningContainers.add(c); + apps.put(c, as.getApp()); + queues.put(c, sched); + } + } + } + } + + // Sort containers into reverse order of priority + Collections.sort(runningContainers, new Comparator() { + public int compare(RMContainer c1, RMContainer c2) { + return c2.getContainer().getPriority().compareTo( + c1.getContainer().getPriority()); + } + }); + + // Scan down the sorted list of task statuses until we've killed enough + // tasks, making sure we don't kill too many from any queue + for (RMContainer container: runningContainers) { + FSQueueSchedulable sched = queues.get(container); + if (Resources.greaterThan(sched.getResourceUsage(), sched.getFairShare())) { + LOG.info("Preempting container (prio=" + container.getContainer().getPriority() + + "res=" + container.getContainer().getResource() + + ") from queue " + sched.getQueue().getName()); + ContainerStatus status = SchedulerUtils.createAbnormalContainerStatus( + container.getContainerId(), SchedulerUtils.PREEMPTED_CONTAINER); + + // TODO: Not sure if this ever actually adds this to the list of cleanup + // containers on the RMNode (see SchedulerNode.releaseContainer()). + this.completedContainer(container, status, RMContainerEventType.KILL); + + toPreempt = Resources.subtract(toPreempt, + container.getContainer().getResource()); + if (Resources.equals(toPreempt, Resources.none())) { + break; + } + } + } + } + + /** + * Return the resource amount that this queue is allowed to preempt, if any. + * If the queue has been below its min share for at least its preemption + * timeout, it should preempt the difference between its current share and + * this min share. If it has been below half its fair share for at least the + * fairSharePreemptionTimeout, it should preempt enough tasks to get up to + * its full fair share. If both conditions hold, we preempt the max of the + * two amounts (this shouldn't happen unless someone sets the timeouts to + * be identical for some reason). + */ + protected Resource resToPreempt(FSQueueSchedulable sched, long curTime) { + String queue = sched.getName(); + long minShareTimeout = queueMgr.getMinSharePreemptionTimeout(queue); + long fairShareTimeout = queueMgr.getFairSharePreemptionTimeout(); + Resource resDueToMinShare = Resources.none(); + Resource resDueToFairShare = Resources.none(); + if (curTime - sched.getLastTimeAtMinShare() > minShareTimeout) { + Resource target = Resources.min(sched.getMinShare(), sched.getDemand()); + resDueToMinShare = Resources.max(Resources.none(), + Resources.subtract(target, sched.getResourceUsage())); + } + if (curTime - sched.getLastTimeAtHalfFairShare() > fairShareTimeout) { + Resource target = Resources.min(sched.getFairShare(), sched.getDemand()); + resDueToFairShare = Resources.max(Resources.none(), + Resources.subtract(target, sched.getResourceUsage())); + } + Resource resToPreempt = Resources.max(resDueToMinShare, resDueToFairShare); + if (Resources.greaterThan(resToPreempt, Resources.none())) { + String message = "Should preempt " + resToPreempt + " res for queue " + + sched.getName() + ": resDueToMinShare = " + resDueToMinShare + + ", resDueToFairShare = " + resDueToFairShare; + LOG.info(message); + } + return resToPreempt; + } + + /** + * This updates the runnability of all apps based on whether or not + * any users/queues have exceeded their capacity. + */ + private void updateRunnability() { + List apps = new ArrayList(); + + // Start by marking everything as not runnable + for (FSQueue p: queueMgr.getQueues()) { + for (AppSchedulable a: p.getQueueSchedulable().getAppSchedulables()) { + a.setRunnable(false); + apps.add(a); + } + } + // Create a list of sorted jobs in order of start time and priority + Collections.sort(apps, new FifoAppComparator()); + // Mark jobs as runnable in order of start time and priority, until + // user or queue limits have been reached. + Map userApps = new HashMap(); + Map queueApps = new HashMap(); + + for (AppSchedulable app: apps) { + String user = app.getApp().getUser(); + String queue = app.getApp().getQueueName(); + int userCount = userApps.containsKey(user) ? userApps.get(user) : 0; + int queueCount = queueApps.containsKey(queue) ? queueApps.get(queue) : 0; + if (userCount < queueMgr.getUserMaxApps(user) && + queueCount < queueMgr.getQueueMaxApps(queue)) { + userApps.put(user, userCount + 1); + queueApps.put(queue, queueCount + 1); + app.setRunnable(true); + } + } + } + + public ContainerTokenSecretManager getContainerTokenSecretManager() { + return this.containerTokenSecretManager; + } + + public double getAppWeight(AppSchedulable app) { + if (!app.getRunnable()) { + // Job won't launch tasks, but don't return 0 to avoid division errors + return 1.0; + } else { + double weight = 1.0; + if (sizeBasedWeight) { + // Set weight based on current demand + weight = Math.log1p(app.getDemand().getMemory()) / Math.log(2); + } + weight *= app.getPriority().getPriority(); + if (weightAdjuster != null) { + // Run weight through the user-supplied weightAdjuster + weight = weightAdjuster.adjustWeight(app, weight); + } + return weight; + } + } + + @Override + public Resource getMinimumResourceCapability() { + return this.minimumAllocation; + } + + @Override + public Resource getMaximumResourceCapability() { + return this.maximumAllocation; + } + + public double getNodeLocalityThreshold() { + return this.nodeLocalityThreshold; + } + + public double getRackLocalityThreshold() { + return this.rackLocalityThreshold; + } + + public Resource getClusterCapacity() { + return this.clusterCapacity; + } + + public Clock getClock() { + return this.clock; + } + + protected void setClock(Clock clock) { + this.clock = clock; + } + + public FairSchedulerEventLog getEventLog() { + return eventLog; + } + + /** + * Add a new application to the scheduler, with a given id, queue name, + * and user. This will accept a new app even if the user or queue is above + * configured limits, but the app will not be marked as runnable. + */ + protected synchronized void + addApplication(ApplicationAttemptId applicationAttemptId, + String queueName, String user) { + + FSQueue queue = this.queueMgr.getQueue(queueName); + + FSSchedulerApp schedulerApp = + new FSSchedulerApp(applicationAttemptId, user, + queue.getQueueSchedulable(), new ActiveUsersManager(this.getRootQueueMetrics()), + rmContext, null); + + // Inforce ACLs + UserGroupInformation userUgi; + try { + userUgi = UserGroupInformation.getCurrentUser(); + } catch (IOException ioe) { + LOG.info("Failed to get current user information"); + return; + } + + List info = queue.getQueueSchedulable().getQueueUserAclInfo( + userUgi); // Always a signleton list + if (!info.get(0).getUserAcls().contains(QueueACL.SUBMIT_APPLICATIONS)) { + LOG.info("User " + userUgi.getUserName() + + " cannot submit" + " applications to queue " + queue.getName()); + return; + } + + queue.addApp(schedulerApp); + queue.getQueueSchedulable().getMetrics().submitApp(user, + applicationAttemptId.getAttemptId()); + rootMetrics.submitApp(user, applicationAttemptId.getAttemptId()); + + applications.put(applicationAttemptId, schedulerApp); + + LOG.info("Application Submission: " + applicationAttemptId + + ", user: " + user + + ", currently active: " + applications.size()); + + rmContext.getDispatcher().getEventHandler().handle( + new RMAppAttemptEvent(applicationAttemptId, + RMAppAttemptEventType.APP_ACCEPTED)); + } + + private synchronized void removeApplication( + ApplicationAttemptId applicationAttemptId, + RMAppAttemptState rmAppAttemptFinalState) { + LOG.info("Application " + applicationAttemptId + " is done." + + " finalState=" + rmAppAttemptFinalState); + + SchedulerApp application = applications.get(applicationAttemptId); + + if (application == null) { + LOG.info("Unknown application " + applicationAttemptId + " has completed!"); + return; + } + + // Release all the running containers + for (RMContainer rmContainer : application.getLiveContainers()) { + completedContainer(rmContainer, + SchedulerUtils.createAbnormalContainerStatus( + rmContainer.getContainerId(), + SchedulerUtils.COMPLETED_APPLICATION), + RMContainerEventType.KILL); + } + + // Release all reserved containers + for (RMContainer rmContainer : application.getReservedContainers()) { + completedContainer(rmContainer, + SchedulerUtils.createAbnormalContainerStatus( + rmContainer.getContainerId(), + "Application Complete"), + RMContainerEventType.KILL); + } + + // Clean up pending requests, metrics etc. + application.stop(rmAppAttemptFinalState); + + // Inform the queue + FSQueue queue = this.queueMgr.getQueue(application.getQueue().getQueueName()); + queue.removeJob(application); + + // Remove from our data-structure + applications.remove(applicationAttemptId); + } + + /** + * Clean up a completed container. + */ + private synchronized void completedContainer(RMContainer rmContainer, + ContainerStatus containerStatus, RMContainerEventType event) { + if (rmContainer == null) { + LOG.info("Null container completed..."); + return; + } + + Container container = rmContainer.getContainer(); + + // Get the application for the finished container + ApplicationAttemptId applicationAttemptId = container.getId().getApplicationAttemptId(); + SchedulerApp application = applications.get(applicationAttemptId); + if (application == null) { + LOG.info("Container " + container + " of" + + " unknown application " + applicationAttemptId + + " completed with event " + event); + return; + } + + // Get the node on which the container was allocated + SchedulerNode node = nodes.get(container.getNodeId()); + + if (rmContainer.getState() == RMContainerState.RESERVED) { + application.unreserve(node, rmContainer.getReservedPriority()); + node.unreserveResource(application); + } else { + application.containerCompleted(rmContainer, containerStatus, event); + node.releaseContainer(container); + } + + LOG.info("Application " + applicationAttemptId + + " released container " + container.getId() + + " on node: " + node + + " with event: " + event); + } + + private synchronized void addNode(RMNode node) { + this.nodes.put(node.getNodeID(), new SchedulerNode(node)); + Resources.addTo(clusterCapacity, node.getTotalCapability()); + + LOG.info("Added node " + node.getNodeAddress() + + " cluster capacity: " + clusterCapacity); + } + + private synchronized void removeNode(RMNode rmNode) { + SchedulerNode node = this.nodes.get(rmNode.getNodeID()); + Resources.subtractFrom(clusterCapacity, rmNode.getTotalCapability()); + + // Remove running containers + List runningContainers = node.getRunningContainers(); + for (RMContainer container : runningContainers) { + completedContainer(container, + SchedulerUtils.createAbnormalContainerStatus( + container.getContainerId(), + SchedulerUtils.LOST_CONTAINER), + RMContainerEventType.KILL); + } + + // Remove reservations, if any + RMContainer reservedContainer = node.getReservedContainer(); + if (reservedContainer != null) { + completedContainer(reservedContainer, + SchedulerUtils.createAbnormalContainerStatus( + reservedContainer.getContainerId(), + SchedulerUtils.LOST_CONTAINER), + RMContainerEventType.KILL); + } + + this.nodes.remove(rmNode.getNodeID()); + LOG.info("Removed node " + rmNode.getNodeAddress() + + " cluster capacity: " + clusterCapacity); + } + + @Override + public Allocation allocate(ApplicationAttemptId appAttemptId, + List ask, List release) { + + // Make sure this application exists + SchedulerApp application = applications.get(appAttemptId); + if (application == null) { + LOG.info("Calling allocate on removed " + + "or non existant application " + appAttemptId); + return EMPTY_ALLOCATION; + } + + // Sanity check + SchedulerUtils.normalizeRequests(ask, minimumAllocation.getMemory()); + + // Release containers + for (ContainerId releasedContainerId : release) { + RMContainer rmContainer = getRMContainer(releasedContainerId); + if (rmContainer == null) { + RMAuditLogger.logFailure(application.getUser(), + AuditConstants.RELEASE_CONTAINER, + "Unauthorized access or invalid container", "FairScheduler", + "Trying to release container not owned by app or with invalid id", + application.getApplicationId(), releasedContainerId); + } + completedContainer(rmContainer, + SchedulerUtils.createAbnormalContainerStatus( + releasedContainerId, + SchedulerUtils.RELEASED_CONTAINER), + RMContainerEventType.RELEASED); + } + + synchronized (application) { + + if (!ask.isEmpty()) { + + if(LOG.isDebugEnabled()) { + LOG.debug("allocate: pre-update" + + " applicationAttemptId=" + appAttemptId + + " application=" + application.getApplicationId()); + } + application.showRequests(); + + // Update application requests + application.updateResourceRequests(ask); + + LOG.debug("allocate: post-update"); + application.showRequests(); + } + + if(LOG.isDebugEnabled()) { + LOG.debug("allocate:" + + " applicationAttemptId=" + appAttemptId + + " #ask=" + ask.size()); + } + + return new Allocation( + application.pullNewlyAllocatedContainers(), + application.getHeadroom()); + } + } + + /** + * Process a container which has launched on a node, as reported by the + * node. + */ + private void containerLaunchedOnNode(ContainerId containerId, SchedulerNode node) { + // Get the application for the finished container + ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId(); + SchedulerApp application = applications.get(applicationAttemptId); + if (application == null) { + LOG.info("Unknown application: " + applicationAttemptId + + " launched container " + containerId + + " on node: " + node); + return; + } + + application.containerLaunchedOnNode(containerId, node.getNodeID()); + } + + /** + * Process a heartbeat update from a node. + */ + private synchronized void nodeUpdate(RMNode nm, + List newlyLaunchedContainers, + List completedContainers) { + LOG.info("nodeUpdate: " + nm + " cluster capacity: " + clusterCapacity); + eventLog.log("HEARTBEAT", nm.getHostName()); + SchedulerNode node = nodes.get(nm.getNodeID()); + + // Processing the newly launched containers + for (ContainerStatus launchedContainer : newlyLaunchedContainers) { + containerLaunchedOnNode(launchedContainer.getContainerId(), node); + } + + // Process completed containers + for (ContainerStatus completedContainer : completedContainers) { + ContainerId containerId = completedContainer.getContainerId(); + LOG.debug("Container FINISHED: " + containerId); + completedContainer(getRMContainer(containerId), + completedContainer, RMContainerEventType.FINISHED); + } + + // Assign new containers... + // 1. Check for reserved applications + // 2. Schedule if there are no reservations + + // If we have have an application that has reserved a resource on this node + // already, we try to complete the reservation. + RMContainer reservedContainer = node.getReservedContainer(); + if (reservedContainer != null) { + SchedulerApp reservedApplication = + applications.get(reservedContainer.getApplicationAttemptId()); + + // Try to fulfill the reservation + LOG.info("Trying to fulfill reservation for application " + + reservedApplication.getApplicationId() + " on node: " + nm); + + FSQueue queue = queueMgr.getQueue(reservedApplication.getQueueName()); + queue.getQueueSchedulable().assignContainer(node, true); + } + + + // Otherwise, schedule at queue which is furthest below fair share + else { + int assignedContainers = 0; + while (true) { + // At most one task is scheduled each iteration of this loop + List scheds = this.getQueueSchedulables(); + Collections.sort(scheds, new SchedulingAlgorithms.FairShareComparator()); + boolean assignedContainer = false; + for (FSQueueSchedulable sched : scheds) { + Resource assigned = sched.assignContainer(node, false); + if (Resources.greaterThan(assigned, Resources.none())) { + eventLog.log("ASSIGN", nm.getHostName(), assigned); + assignedContainers++; + assignedContainer = true; + break; + } + } + if (!assignedContainer) { break; } + if (!assignMultiple) { break; } + if ((assignedContainers >= maxAssign) && (maxAssign > 0)) { break; } + } + } + } + + @Override + public SchedulerNodeReport getNodeReport(NodeId nodeId) { + SchedulerNode node = nodes.get(nodeId); + return node == null ? null : new SchedulerNodeReport(node); + } + + @Override + public SchedulerAppReport getSchedulerAppInfo( + ApplicationAttemptId appAttemptId) { + if (!this.applications.containsKey(appAttemptId)) { + LOG.error("Request for appInfo of unknown attempt" + appAttemptId); + return null; + } + return new SchedulerAppReport(this.applications.get(appAttemptId)); + } + + @Override + public QueueMetrics getRootQueueMetrics() { + return rootMetrics; + } + + @Override + public void handle(SchedulerEvent event) { + switch(event.getType()) { + case NODE_ADDED: + { + if (!(event instanceof NodeAddedSchedulerEvent)) { + throw new RuntimeException("Unexpected event type: " + event); + } + NodeAddedSchedulerEvent nodeAddedEvent = (NodeAddedSchedulerEvent)event; + addNode(nodeAddedEvent.getAddedRMNode()); + } + break; + case NODE_REMOVED: + { + if (!(event instanceof NodeRemovedSchedulerEvent)) { + throw new RuntimeException("Unexpected event type: " + event); + } + NodeRemovedSchedulerEvent nodeRemovedEvent = (NodeRemovedSchedulerEvent)event; + removeNode(nodeRemovedEvent.getRemovedRMNode()); + } + break; + case NODE_UPDATE: + { + if (!(event instanceof NodeUpdateSchedulerEvent)) { + throw new RuntimeException("Unexpected event type: " + event); + } + NodeUpdateSchedulerEvent nodeUpdatedEvent = + (NodeUpdateSchedulerEvent)event; + this.nodeUpdate(nodeUpdatedEvent.getRMNode(), + nodeUpdatedEvent.getNewlyLaunchedContainers(), + nodeUpdatedEvent.getCompletedContainers()); + } + break; + case APP_ADDED: + { + if (!(event instanceof AppAddedSchedulerEvent)) { + throw new RuntimeException("Unexpected event type: " + event); + } + AppAddedSchedulerEvent appAddedEvent = (AppAddedSchedulerEvent) event; + String queue = appAddedEvent.getQueue(); + + // Potentially set queue to username if configured to do so + String def = YarnConfiguration.DEFAULT_QUEUE_NAME; + if (queue.equals(def) && userAsDefaultQueue) { + queue = appAddedEvent.getUser(); + } + + addApplication(appAddedEvent.getApplicationAttemptId(), queue, + appAddedEvent.getUser()); + } + break; + case APP_REMOVED: + { + if (!(event instanceof AppRemovedSchedulerEvent)) { + throw new RuntimeException("Unexpected event type: " + event); + } + AppRemovedSchedulerEvent appRemovedEvent = (AppRemovedSchedulerEvent)event; + this.removeApplication(appRemovedEvent.getApplicationAttemptID(), + appRemovedEvent.getFinalAttemptState()); + } + break; + case CONTAINER_EXPIRED: + { + if (!(event instanceof ContainerExpiredSchedulerEvent)) { + throw new RuntimeException("Unexpected event type: " + event); + } + ContainerExpiredSchedulerEvent containerExpiredEvent = + (ContainerExpiredSchedulerEvent) event; + ContainerId containerId = containerExpiredEvent.getContainerId(); + completedContainer(getRMContainer(containerId), + SchedulerUtils.createAbnormalContainerStatus( + containerId, + SchedulerUtils.EXPIRED_CONTAINER), + RMContainerEventType.EXPIRE); + } + break; + default: + LOG.error("Unknown event arrived at FairScheduler: " + event.toString()); + } + } + + @Override + public void recover(RMState state) throws Exception { + // NOT IMPLEMENTED + } + + @Override + public synchronized void reinitialize(Configuration conf, + ContainerTokenSecretManager containerTokenSecretManager, + RMContext rmContext) + throws IOException + { + if (!this.initialized) { + this.conf = new FairSchedulerConfiguration(conf); + this.rootMetrics = QueueMetrics.forQueue("root", null, true, conf); + this.containerTokenSecretManager = containerTokenSecretManager; + this.rmContext = rmContext; + this.clock = new SystemClock(); + this.eventLog = new FairSchedulerEventLog(); + eventLog.init(this.conf); + minimumAllocation = this.conf.getMinimumMemoryAllocation(); + maximumAllocation = this.conf.getMaximumMemoryAllocation(); + userAsDefaultQueue = this.conf.getUserAsDefaultQueue(); + nodeLocalityThreshold = this.conf.getLocalityThresholdNode(); + rackLocalityThreshold = this.conf.getLocalityThresholdRack(); + preemptionEnabled = this.conf.getPreemptionEnabled(); + assignMultiple = this.conf.getAssignMultiple(); + maxAssign = this.conf.getMaxAssign(); + + Thread updateThread = new Thread(new UpdateThread()); + updateThread.start(); + + initialized = true; + + sizeBasedWeight = this.conf.getSizeBasedWeight(); + + queueMgr = new QueueManager(this); + + try { + queueMgr.initialize(); + } + catch (Exception e) { + throw new IOException("Failed to start FairScheduler", e); + } + } else { + this.conf = new FairSchedulerConfiguration(conf); + userAsDefaultQueue = this.conf.getUserAsDefaultQueue(); + nodeLocalityThreshold = this.conf.getLocalityThresholdNode(); + rackLocalityThreshold = this.conf.getLocalityThresholdRack(); + preemptionEnabled = this.conf.getPreemptionEnabled(); + try { + queueMgr.reloadAllocs(); + + } + catch (Exception e) { + throw new IOException("Failed to initialize FairScheduler", e); + } + } + } + + @Override + public QueueInfo getQueueInfo(String queueName, boolean includeChildQueues, + boolean recursive) throws IOException { + if (!queueMgr.exists(queueName)) { + return null; + } + return queueMgr.getQueue(queueName).getQueueSchedulable().getQueueInfo( + includeChildQueues, recursive); + } + + @Override + public List getQueueUserAclInfo() { + UserGroupInformation user = null; + try { + user = UserGroupInformation.getCurrentUser(); + } catch (IOException ioe) { + return new ArrayList(); + } + + List userAcls = new ArrayList(); + + for (FSQueue queue : queueMgr.getQueues()) { + userAcls.addAll(queue.getQueueSchedulable().getQueueUserAclInfo(user)); + } + return userAcls; + } + + @Override + public int getNumClusterNodes() { + return this.nodes.size(); + } + +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java new file mode 100644 index 00000000000..8a7f7fc45ed --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerConfiguration.java @@ -0,0 +1,111 @@ +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; + +import java.io.File; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources; + +public class FairSchedulerConfiguration extends Configuration { + public static final String FS_CONFIGURATION_FILE = "fair-scheduler.xml"; + + private static final String CONF_PREFIX = "yarn.scheduler.fair."; + + protected static final String ALLOCATION_FILE = CONF_PREFIX + "allocation.file"; + protected static final String EVENT_LOG_DIR = "eventlog.dir"; + + /** Whether to use the user name as the queue name (instead of "default") if + * the request does not specify a queue. */ + protected static final String USER_AS_DEFAULT_QUEUE = CONF_PREFIX + "user-as-default-queue"; + protected static final boolean DEFAULT_USER_AS_DEFAULT_QUEUE = false; + + protected static final String LOCALITY_THRESHOLD = CONF_PREFIX + "locality.threshold"; + protected static final float DEFAULT_LOCALITY_THRESHOLD = -1.0f; + + /** Cluster threshold for node locality. */ + protected static final String LOCALITY_THRESHOLD_NODE = CONF_PREFIX + "locality.threshold.node"; + protected static final float DEFAULT_LOCALITY_THRESHOLD_NODE = + DEFAULT_LOCALITY_THRESHOLD; + + /** Cluster threshold for rack locality. */ + protected static final String LOCALITY_THRESHOLD_RACK = CONF_PREFIX + "locality.threshold.rack"; + protected static final float DEFAULT_LOCALITY_THRESHOLD_RACK = + DEFAULT_LOCALITY_THRESHOLD; + + /** Whether preemption is enabled. */ + protected static final String PREEMPTION = CONF_PREFIX + "preemption"; + protected static final boolean DEFAULT_PREEMPTION = false; + + /** Whether to assign multiple containers in one check-in. */ + protected static final String ASSIGN_MULTIPLE = CONF_PREFIX + "assignmultiple"; + protected static final boolean DEFAULT_ASSIGN_MULTIPLE = true; + + /** Whether to give more weight to apps requiring many resources. */ + protected static final String SIZE_BASED_WEIGHT = CONF_PREFIX + "sizebasedweight"; + protected static final boolean DEFAULT_SIZE_BASED_WEIGHT = false; + + /** Maximum number of containers to assign on each check-in. */ + protected static final String MAX_ASSIGN = CONF_PREFIX + "max.assign"; + protected static final int DEFAULT_MAX_ASSIGN = -1; + + public FairSchedulerConfiguration(Configuration conf) { + super(conf); + addResource(FS_CONFIGURATION_FILE); + } + + public Resource getMinimumMemoryAllocation() { + int mem = getInt( + YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, + YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB); + return Resources.createResource(mem); + } + + public Resource getMaximumMemoryAllocation() { + int mem = getInt( + YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, + YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB); + return Resources.createResource(mem); + } + + public boolean getUserAsDefaultQueue() { + return getBoolean(USER_AS_DEFAULT_QUEUE, DEFAULT_USER_AS_DEFAULT_QUEUE); + } + + public float getLocalityThreshold() { + return getFloat(LOCALITY_THRESHOLD, DEFAULT_LOCALITY_THRESHOLD); + } + + public float getLocalityThresholdNode() { + return getFloat(LOCALITY_THRESHOLD_NODE, DEFAULT_LOCALITY_THRESHOLD_NODE); + } + + public float getLocalityThresholdRack() { + return getFloat(LOCALITY_THRESHOLD_RACK, DEFAULT_LOCALITY_THRESHOLD_RACK); + } + + public boolean getPreemptionEnabled() { + return getBoolean(PREEMPTION, DEFAULT_PREEMPTION); + } + + public boolean getAssignMultiple() { + return getBoolean(ASSIGN_MULTIPLE, DEFAULT_ASSIGN_MULTIPLE); + } + + public int getMaxAssign() { + return getInt(MAX_ASSIGN, DEFAULT_MAX_ASSIGN); + } + + public boolean getSizeBasedWeight() { + return getBoolean(SIZE_BASED_WEIGHT, DEFAULT_SIZE_BASED_WEIGHT); + } + + public String getAllocationFile() { + return get(ALLOCATION_FILE); + } + + public String getEventlogDir() { + return get(EVENT_LOG_DIR, new File(System.getProperty("hadoop.log.dir", + "/tmp/")).getAbsolutePath() + File.separator + "fairscheduler"); + } +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerEventLog.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerEventLog.java new file mode 100644 index 00000000000..1f2205b77b5 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerEventLog.java @@ -0,0 +1,143 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; + +import java.io.File; +import java.io.IOException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.DailyRollingFileAppender; +import org.apache.log4j.Level; +import org.apache.log4j.Logger; +import org.apache.log4j.PatternLayout; +import org.apache.log4j.spi.LoggingEvent; + +/** + * Event log used by the fair scheduler for machine-readable debug info. + * This class uses a log4j rolling file appender to write the log, but uses + * a custom tab-separated event format of the form: + *
+ * DATE    EVENT_TYPE   PARAM_1   PARAM_2   ...
+ * 
+ * Various event types are used by the fair scheduler. The purpose of logging + * in this format is to enable tools to parse the history log easily and read + * internal scheduler variables, rather than trying to make the log human + * readable. The fair scheduler also logs human readable messages in the + * JobTracker's main log. + * + * Constructing this class creates a disabled log. It must be initialized + * using {@link FairSchedulerEventLog#init(Configuration, String)} to begin + * writing to the file. + */ +@Private +@Unstable +class FairSchedulerEventLog { + private static final Log LOG = LogFactory.getLog(FairSchedulerEventLog.class.getName()); + + /** Set to true if logging is disabled due to an error. */ + private boolean logDisabled = true; + + /** + * Log directory, set by mapred.fairscheduler.eventlog.location in conf file; + * defaults to {hadoop.log.dir}/fairscheduler. + */ + private String logDir; + + /** + * Active log file, which is {LOG_DIR}/hadoop-{user}-fairscheduler.log. + * Older files are also stored as {LOG_FILE}.date (date format YYYY-MM-DD). + */ + private String logFile; + + /** Log4j appender used to write to the log file */ + private DailyRollingFileAppender appender; + + boolean init(FairSchedulerConfiguration conf) { + try { + logDir = conf.getEventlogDir(); + Path logDirPath = new Path(logDir); + FileSystem fs = logDirPath.getFileSystem(conf); + if (!fs.exists(logDirPath)) { + if (!fs.mkdirs(logDirPath)) { + throw new IOException( + "Mkdirs failed to create " + logDirPath.toString()); + } + } + String username = System.getProperty("user.name"); + logFile = String.format("%s%shadoop-%s-fairscheduler.log", + logDir, File.separator, username); + logDisabled = false; + PatternLayout layout = new PatternLayout("%d{ISO8601}\t%m%n"); + appender = new DailyRollingFileAppender(layout, logFile, "'.'yyyy-MM-dd"); + appender.activateOptions(); + LOG.info("Initialized fair scheduler event log, logging to " + logFile); + } catch (IOException e) { + LOG.error( + "Failed to initialize fair scheduler event log. Disabling it.", e); + logDisabled = true; + } + return !(logDisabled); + } + + /** + * Log an event, writing a line in the log file of the form + *
+   * DATE    EVENT_TYPE   PARAM_1   PARAM_2   ...
+   * 
+ */ + synchronized void log(String eventType, Object... params) { + try { + if (logDisabled) + return; + StringBuffer buffer = new StringBuffer(); + buffer.append(eventType); + for (Object param: params) { + buffer.append("\t"); + buffer.append(param); + } + String message = buffer.toString(); + Logger logger = Logger.getLogger(getClass()); + appender.append(new LoggingEvent("", logger, Level.INFO, message, null)); + } catch (Exception e) { + LOG.error("Failed to append to fair scheduler event log", e); + logDisabled = true; + } + } + + /** + * Flush and close the log. + */ + void shutdown() { + try { + if (appender != null) + appender.close(); + } catch (Exception e) {} + logDisabled = true; + } + + boolean isEnabled() { + return !logDisabled; + } +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FifoAppComparator.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FifoAppComparator.java new file mode 100644 index 00000000000..c87f4ca6b8c --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FifoAppComparator.java @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; + +import java.io.Serializable; +import java.util.Comparator; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; + +/** + * Order {@link AppSchedulable} objects by priority and then by submit time, as + * in the default scheduler in Hadoop. + */ +@Private +@Unstable +public class FifoAppComparator implements Comparator, Serializable { + private static final long serialVersionUID = 3428835083489547918L; + + public int compare(AppSchedulable a1, AppSchedulable a2) { + int res = a1.getPriority().compareTo(a2.getPriority()); + if (res == 0) { + if (a1.getStartTime() < a2.getStartTime()) { + res = -1; + } else { + res = (a1.getStartTime() == a2.getStartTime() ? 0 : 1); + } + } + if (res == 0) { + // If there is a tie, break it by app ID to get a deterministic order + res = a1.getApp().getApplicationId().compareTo(a2.getApp().getApplicationId()); + } + return res; + } +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/NewJobWeightBooster.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/NewJobWeightBooster.java new file mode 100644 index 00000000000..c643027904a --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/NewJobWeightBooster.java @@ -0,0 +1,60 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; + +/** + * A {@link WeightAdjuster} implementation that gives a weight boost to new jobs + * for a certain amount of time -- by default, a 3x weight boost for 60 seconds. + * This can be used to make shorter jobs finish faster, emulating Shortest Job + * First scheduling while not starving long jobs. + */ +@Private +@Unstable +public class NewJobWeightBooster extends Configured implements WeightAdjuster { + private static final float DEFAULT_FACTOR = 3; + private static final long DEFAULT_DURATION = 5 * 60 * 1000; + + private float factor; + private long duration; + + public void setConf(Configuration conf) { + if (conf != null) { + factor = conf.getFloat("mapred.newjobweightbooster.factor", + DEFAULT_FACTOR); + duration = conf.getLong("mapred.newjobweightbooster.duration", + DEFAULT_DURATION); + } + super.setConf(conf); + } + + public double adjustWeight(AppSchedulable app, double curWeight) { + long start = app.getStartTime(); + long now = System.currentTimeMillis(); + if (now - start < duration) { + return curWeight * factor; + } else { + return curWeight; + } + } +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java new file mode 100644 index 00000000000..08c15a3d6e0 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java @@ -0,0 +1,513 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.net.URLConnection; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import javax.xml.parsers.DocumentBuilder; +import javax.xml.parsers.DocumentBuilderFactory; +import javax.xml.parsers.ParserConfigurationException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.authorize.AccessControlList; +import org.apache.hadoop.yarn.api.records.QueueACL; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApp; +import org.w3c.dom.Document; +import org.w3c.dom.Element; +import org.w3c.dom.Node; +import org.w3c.dom.NodeList; +import org.w3c.dom.Text; +import org.xml.sax.SAXException; + +/** + * Maintains a list of queues as well as scheduling parameters for each queue, + * such as guaranteed share allocations, from the fair scheduler config file. + */ +@Private +@Unstable +public class QueueManager { + public static final Log LOG = LogFactory.getLog( + QueueManager.class.getName()); + + /** Time to wait between checks of the allocation file */ + public static final long ALLOC_RELOAD_INTERVAL = 10 * 1000; + + /** + * Time to wait after the allocation has been modified before reloading it + * (this is done to prevent loading a file that hasn't been fully written). + */ + public static final long ALLOC_RELOAD_WAIT = 5 * 1000; + + private final FairScheduler scheduler; + + // Minimum resource allocation for each queue + private Map minQueueResources = new HashMap(); + // Maximum amount of resources per queue + private Map maxQueueResources = new HashMap(); + // Sharing weights for each queue + private Map queueWeights = new HashMap(); + + // Max concurrent running applications for each queue and for each user; in addition, + // for users that have no max specified, we use the userMaxJobsDefault. + private Map queueMaxApps = new HashMap(); + private Map userMaxApps = new HashMap(); + private int userMaxAppsDefault = Integer.MAX_VALUE; + private int queueMaxAppsDefault = Integer.MAX_VALUE; + + // ACL's for each queue. Only specifies non-default ACL's from configuration. + private Map> queueAcls = + new HashMap>(); + + // Min share preemption timeout for each queue in seconds. If a job in the queue + // waits this long without receiving its guaranteed share, it is allowed to + // preempt other jobs' tasks. + private Map minSharePreemptionTimeouts = + new HashMap(); + + // Default min share preemption timeout for queues where it is not set + // explicitly. + private long defaultMinSharePreemptionTimeout = Long.MAX_VALUE; + + // Preemption timeout for jobs below fair share in seconds. If a job remains + // below half its fair share for this long, it is allowed to preempt tasks. + private long fairSharePreemptionTimeout = Long.MAX_VALUE; + + SchedulingMode defaultSchedulingMode = SchedulingMode.FAIR; + + private Object allocFile; // Path to XML file containing allocations. This + // is either a URL to specify a classpath resource + // (if the fair-scheduler.xml on the classpath is + // used) or a String to specify an absolute path (if + // mapred.fairscheduler.allocation.file is used). + + private Map queues = new HashMap(); + + private long lastReloadAttempt; // Last time we tried to reload the queues file + private long lastSuccessfulReload; // Last time we successfully reloaded queues + private boolean lastReloadAttemptFailed = false; + + public QueueManager(FairScheduler scheduler) { + this.scheduler = scheduler; + } + + public void initialize() throws IOException, SAXException, + AllocationConfigurationException, ParserConfigurationException { + FairSchedulerConfiguration conf = scheduler.getConf(); + this.allocFile = conf.getAllocationFile(); + if (allocFile == null) { + // No allocation file specified in jobconf. Use the default allocation + // file, fair-scheduler.xml, looking for it on the classpath. + allocFile = new Configuration().getResource("fair-scheduler.xml"); + if (allocFile == null) { + LOG.error("The fair scheduler allocation file fair-scheduler.xml was " + + "not found on the classpath, and no other config file is given " + + "through mapred.fairscheduler.allocation.file."); + } + } + reloadAllocs(); + lastSuccessfulReload = scheduler.getClock().getTime(); + lastReloadAttempt = scheduler.getClock().getTime(); + // Create the default queue + getQueue(YarnConfiguration.DEFAULT_QUEUE_NAME); + } + + /** + * Get a queue by name, creating it if necessary + */ + public synchronized FSQueue getQueue(String name) { + FSQueue queue = queues.get(name); + if (queue == null) { + queue = new FSQueue(scheduler, name); + queue.setSchedulingMode(defaultSchedulingMode); + queues.put(name, queue); + } + return queue; + } + + /** + * Return whether a queue exists already. + */ + public synchronized boolean exists(String name) { + return queues.containsKey(name); + } + + /** + * Get the queue for a given AppSchedulable. + */ + public FSQueue getQueueForApp(AppSchedulable app) { + return this.getQueue(app.getApp().getQueueName()); + } + + /** + * Reload allocations file if it hasn't been loaded in a while + */ + public void reloadAllocsIfNecessary() { + long time = scheduler.getClock().getTime(); + if (time > lastReloadAttempt + ALLOC_RELOAD_INTERVAL) { + lastReloadAttempt = time; + if (null == allocFile) { + return; + } + try { + // Get last modified time of alloc file depending whether it's a String + // (for a path name) or an URL (for a classloader resource) + long lastModified; + if (allocFile instanceof String) { + File file = new File((String) allocFile); + lastModified = file.lastModified(); + } else { // allocFile is an URL + URLConnection conn = ((URL) allocFile).openConnection(); + lastModified = conn.getLastModified(); + } + if (lastModified > lastSuccessfulReload && + time > lastModified + ALLOC_RELOAD_WAIT) { + reloadAllocs(); + lastSuccessfulReload = time; + lastReloadAttemptFailed = false; + } + } catch (Exception e) { + // Throwing the error further out here won't help - the RPC thread + // will catch it and report it in a loop. Instead, just log it and + // hope somebody will notice from the log. + // We log the error only on the first failure so we don't fill up the + // JobTracker's log with these messages. + if (!lastReloadAttemptFailed) { + LOG.error("Failed to reload fair scheduler config file - " + + "will use existing allocations.", e); + } + lastReloadAttemptFailed = true; + } + } + } + + /** + * Updates the allocation list from the allocation config file. This file is + * expected to be in the XML format specified in the design doc. + * + * @throws IOException if the config file cannot be read. + * @throws AllocationConfigurationException if allocations are invalid. + * @throws ParserConfigurationException if XML parser is misconfigured. + * @throws SAXException if config file is malformed. + */ + public void reloadAllocs() throws IOException, ParserConfigurationException, + SAXException, AllocationConfigurationException { + if (allocFile == null) return; + // Create some temporary hashmaps to hold the new allocs, and we only save + // them in our fields if we have parsed the entire allocs file successfully. + Map minQueueResources = new HashMap(); + Map maxQueueResources = new HashMap(); + Map queueMaxApps = new HashMap(); + Map userMaxApps = new HashMap(); + Map queueWeights = new HashMap(); + Map queueModes = new HashMap(); + Map minSharePreemptionTimeouts = new HashMap(); + Map> queueAcls = + new HashMap>(); + int userMaxAppsDefault = Integer.MAX_VALUE; + int queueMaxAppsDefault = Integer.MAX_VALUE; + SchedulingMode defaultSchedulingMode = SchedulingMode.FAIR; + + // Remember all queue names so we can display them on web UI, etc. + List queueNamesInAllocFile = new ArrayList(); + + // Read and parse the allocations file. + DocumentBuilderFactory docBuilderFactory = + DocumentBuilderFactory.newInstance(); + docBuilderFactory.setIgnoringComments(true); + DocumentBuilder builder = docBuilderFactory.newDocumentBuilder(); + Document doc; + if (allocFile instanceof String) { + doc = builder.parse(new File((String) allocFile)); + } else { + doc = builder.parse(allocFile.toString()); + } + Element root = doc.getDocumentElement(); + if (!"allocations".equals(root.getTagName())) + throw new AllocationConfigurationException("Bad fair scheduler config " + + "file: top-level element not "); + NodeList elements = root.getChildNodes(); + for (int i = 0; i < elements.getLength(); i++) { + Node node = elements.item(i); + if (!(node instanceof Element)) + continue; + Element element = (Element)node; + if ("queue".equals(element.getTagName()) || + "pool".equals(element.getTagName())) { + String queueName = element.getAttribute("name"); + Map acls = + new HashMap(); + queueNamesInAllocFile.add(queueName); + NodeList fields = element.getChildNodes(); + for (int j = 0; j < fields.getLength(); j++) { + Node fieldNode = fields.item(j); + if (!(fieldNode instanceof Element)) + continue; + Element field = (Element) fieldNode; + if ("minResources".equals(field.getTagName())) { + String text = ((Text)field.getFirstChild()).getData().trim(); + int val = Integer.parseInt(text); + minQueueResources.put(queueName, Resources.createResource(val)); + } else if ("maxResources".equals(field.getTagName())) { + String text = ((Text)field.getFirstChild()).getData().trim(); + int val = Integer.parseInt(text); + maxQueueResources.put(queueName, Resources.createResource(val)); + } else if ("maxRunningApps".equals(field.getTagName())) { + String text = ((Text)field.getFirstChild()).getData().trim(); + int val = Integer.parseInt(text); + queueMaxApps.put(queueName, val); + } else if ("weight".equals(field.getTagName())) { + String text = ((Text)field.getFirstChild()).getData().trim(); + double val = Double.parseDouble(text); + queueWeights.put(queueName, val); + } else if ("minSharePreemptionTimeout".equals(field.getTagName())) { + String text = ((Text)field.getFirstChild()).getData().trim(); + long val = Long.parseLong(text) * 1000L; + minSharePreemptionTimeouts.put(queueName, val); + } else if ("schedulingMode".equals(field.getTagName())) { + String text = ((Text)field.getFirstChild()).getData().trim(); + queueModes.put(queueName, parseSchedulingMode(text)); + } else if ("aclSubmitApps".equals(field.getTagName())) { + String text = ((Text)field.getFirstChild()).getData().trim(); + acls.put(QueueACL.SUBMIT_APPLICATIONS, new AccessControlList(text)); + } else if ("aclAdministerApps".equals(field.getTagName())) { + String text = ((Text)field.getFirstChild()).getData().trim(); + acls.put(QueueACL.ADMINISTER_QUEUE, new AccessControlList(text)); + } + } + queueAcls.put(queueName, acls); + if (maxQueueResources.containsKey(queueName) && minQueueResources.containsKey(queueName) + && Resources.lessThan(maxQueueResources.get(queueName), + minQueueResources.get(queueName))) { + LOG.warn(String.format("Queue %s has max resources %d less than min resources %d", + queueName, maxQueueResources.get(queueName), minQueueResources.get(queueName))); + } + } else if ("user".equals(element.getTagName())) { + String userName = element.getAttribute("name"); + NodeList fields = element.getChildNodes(); + for (int j = 0; j < fields.getLength(); j++) { + Node fieldNode = fields.item(j); + if (!(fieldNode instanceof Element)) + continue; + Element field = (Element) fieldNode; + if ("maxRunningApps".equals(field.getTagName())) { + String text = ((Text)field.getFirstChild()).getData().trim(); + int val = Integer.parseInt(text); + userMaxApps.put(userName, val); + } + } + } else if ("userMaxAppsDefault".equals(element.getTagName())) { + String text = ((Text)element.getFirstChild()).getData().trim(); + int val = Integer.parseInt(text); + userMaxAppsDefault = val; + } else if ("fairSharePreemptionTimeout".equals(element.getTagName())) { + String text = ((Text)element.getFirstChild()).getData().trim(); + long val = Long.parseLong(text) * 1000L; + fairSharePreemptionTimeout = val; + } else if ("defaultMinSharePreemptionTimeout".equals(element.getTagName())) { + String text = ((Text)element.getFirstChild()).getData().trim(); + long val = Long.parseLong(text) * 1000L; + defaultMinSharePreemptionTimeout = val; + } else if ("queueMaxAppsDefault".equals(element.getTagName())) { + String text = ((Text)element.getFirstChild()).getData().trim(); + int val = Integer.parseInt(text); + queueMaxAppsDefault = val;} + else if ("defaultQueueSchedulingMode".equals(element.getTagName())) { + String text = ((Text)element.getFirstChild()).getData().trim(); + defaultSchedulingMode = parseSchedulingMode(text); + } else { + LOG.warn("Bad element in allocations file: " + element.getTagName()); + } + } + + // Commit the reload; also create any queue defined in the alloc file + // if it does not already exist, so it can be displayed on the web UI. + synchronized(this) { + this.minQueueResources = minQueueResources; + this.maxQueueResources = maxQueueResources; + this.queueMaxApps = queueMaxApps; + this.userMaxApps = userMaxApps; + this.queueWeights = queueWeights; + this.userMaxAppsDefault = userMaxAppsDefault; + this.queueMaxAppsDefault = queueMaxAppsDefault; + this.defaultSchedulingMode = defaultSchedulingMode; + this.minSharePreemptionTimeouts = minSharePreemptionTimeouts; + this.queueAcls = queueAcls; + for (String name: queueNamesInAllocFile) { + FSQueue queue = getQueue(name); + if (queueModes.containsKey(name)) { + queue.setSchedulingMode(queueModes.get(name)); + } else { + queue.setSchedulingMode(defaultSchedulingMode); + } + } + } + } + + private SchedulingMode parseSchedulingMode(String text) + throws AllocationConfigurationException { + text = text.toLowerCase(); + if (text.equals("fair")) { + return SchedulingMode.FAIR; + } else if (text.equals("fifo")) { + return SchedulingMode.FIFO; + } else { + throw new AllocationConfigurationException( + "Unknown scheduling mode : " + text + "; expected 'fifo' or 'fair'"); + } + } + + /** + * Get the minimum resource allocation for the given queue. + * @return the cap set on this queue, or 0 if not set. + */ + public Resource getMinResources(String queue) { + if (minQueueResources.containsKey(queue)) { + return minQueueResources.get(queue); + } else{ + return Resources.createResource(0); + } + } + + /** + * Get the maximum resource allocation for the given queue. + * @return the cap set on this queue, or Integer.MAX_VALUE if not set. + */ + Resource getMaxResources(String queueName) { + if (maxQueueResources.containsKey(queueName)) { + return maxQueueResources.get(queueName); + } else { + return Resources.createResource(Integer.MAX_VALUE); + } + } + + /** + * Add an app in the appropriate queue + */ + public synchronized void addApp(FSSchedulerApp app) { + getQueue(app.getQueueName()).addApp(app); + } + + /** + * Remove an app + */ + public synchronized void removeJob(SchedulerApp app) { + getQueue(app.getQueueName()).removeJob(app); + } + + /** + * Get a collection of all queues + */ + public synchronized Collection getQueues() { + return queues.values(); + } + + + /** + * Get all queue names that have been seen either in the allocation file or in + * a submitted app. + */ + public synchronized Collection getQueueNames() { + List list = new ArrayList(); + for (FSQueue queue: getQueues()) { + list.add(queue.getName()); + } + Collections.sort(list); + return list; + } + + public int getUserMaxApps(String user) { + if (userMaxApps.containsKey(user)) { + return userMaxApps.get(user); + } else { + return userMaxAppsDefault; + } + } + + public int getQueueMaxApps(String queue) { + if (queueMaxApps.containsKey(queue)) { + return queueMaxApps.get(queue); + } else { + return queueMaxAppsDefault; + } + } + + public double getQueueWeight(String queue) { + if (queueWeights.containsKey(queue)) { + return queueWeights.get(queue); + } else { + return 1.0; + } + } + + /** + * Get a queue's min share preemption timeout, in milliseconds. This is the + * time after which jobs in the queue may kill other queues' tasks if they + * are below their min share. + */ + public long getMinSharePreemptionTimeout(String queueName) { + if (minSharePreemptionTimeouts.containsKey(queueName)) { + return minSharePreemptionTimeouts.get(queueName); + } + return defaultMinSharePreemptionTimeout; + } + + /** + * Get the fair share preemption, in milliseconds. This is the time + * after which any job may kill other jobs' tasks if it is below half + * its fair share. + */ + public long getFairSharePreemptionTimeout() { + return fairSharePreemptionTimeout; + } + + /** + * Get the ACLs associated with this queue. If a given ACL is not explicitly + * configured, include the default value for that ACL. + */ + public Map getQueueAcls(String queue) { + HashMap out = new HashMap(); + + if (queueAcls.containsKey(queue)) { + out.putAll(queueAcls.get(queue)); + } + if (!out.containsKey(QueueACL.ADMINISTER_QUEUE)) { + out.put(QueueACL.ADMINISTER_QUEUE, new AccessControlList("*")); + } + if (!out.containsKey(QueueACL.SUBMIT_APPLICATIONS)) { + out.put(QueueACL.SUBMIT_APPLICATIONS, new AccessControlList("*")); + } + return out; + } +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.java new file mode 100644 index 00000000000..628da699fea --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.java @@ -0,0 +1,125 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; + +/** + * A Schedulable represents an entity that can launch tasks, such as a job + * or a queue. It provides a common interface so that algorithms such as fair + * sharing can be applied both within a queue and across queues. There are + * currently two types of Schedulables: JobSchedulables, which represent a + * single job, and QueueSchedulables, which allocate among jobs in their queue. + * + * Separate sets of Schedulables are used for maps and reduces. Each queue has + * both a mapSchedulable and a reduceSchedulable, and so does each job. + * + * A Schedulable is responsible for three roles: + * 1) It can launch tasks through assignTask(). + * 2) It provides information about the job/queue to the scheduler, including: + * - Demand (maximum number of tasks required) + * - Number of currently running tasks + * - Minimum share (for queues) + * - Job/queue weight (for fair sharing) + * - Start time and priority (for FIFO) + * 3) It can be assigned a fair share, for use with fair scheduling. + * + * Schedulable also contains two methods for performing scheduling computations: + * - updateDemand() is called periodically to compute the demand of the various + * jobs and queues, which may be expensive (e.g. jobs must iterate through all + * their tasks to count failed tasks, tasks that can be speculated, etc). + * - redistributeShare() is called after demands are updated and a Schedulable's + * fair share has been set by its parent to let it distribute its share among + * the other Schedulables within it (e.g. for queues that want to perform fair + * sharing among their jobs). + */ +@Private +@Unstable +abstract class Schedulable { + /** Fair share assigned to this Schedulable */ + private Resource fairShare = Resources.createResource(0); + + /** + * Name of job/queue, used for debugging as well as for breaking ties in + * scheduling order deterministically. + */ + public abstract String getName(); + + /** + * Maximum number of resources required by this Schedulable. This is defined as + * number of currently utilized resources + number of unlaunched resources (that + * are either not yet launched or need to be speculated). + */ + public abstract Resource getDemand(); + + /** Get the aggregate amount of resources consumed by the schedulable. */ + public abstract Resource getResourceUsage(); + + /** Minimum Resource share assigned to the schedulable. */ + public abstract Resource getMinShare(); + + + /** Job/queue weight in fair sharing. */ + public abstract double getWeight(); + + /** Start time for jobs in FIFO queues; meaningless for QueueSchedulables.*/ + public abstract long getStartTime(); + + /** Job priority for jobs in FIFO queues; meaningless for QueueSchedulables. */ + public abstract Priority getPriority(); + + /** Refresh the Schedulable's demand and those of its children if any. */ + public abstract void updateDemand(); + + /** + * Distribute the fair share assigned to this Schedulable among its + * children (used in queues where the internal scheduler is fair sharing). + */ + public abstract void redistributeShare(); + + /** + * Assign a container on this node if possible, and return the amount of + * resources assigned. If {@code reserved} is true, it means a reservation + * already exists on this node, and the schedulable should fulfill that + * reservation if possible. + */ + public abstract Resource assignContainer(SchedulerNode node, boolean reserved); + + /** Assign a fair share to this Schedulable. */ + public void setFairShare(Resource fairShare) { + this.fairShare = fairShare; + } + + /** Get the fair share assigned to this Schedulable. */ + public Resource getFairShare() { + return fairShare; + } + + /** Convenient toString implementation for debugging. */ + @Override + public String toString() { + return String.format("[%s, demand=%s, running=%s, share=%s,], w=%.1f]", + getName(), getDemand(), getResourceUsage(), fairShare, getWeight()); + } +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingAlgorithms.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingAlgorithms.java new file mode 100644 index 00000000000..64545923756 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingAlgorithms.java @@ -0,0 +1,223 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; + +import java.io.Serializable; +import java.util.Collection; +import java.util.Comparator; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources; + +/** + * Utility class containing scheduling algorithms used in the fair scheduler. + */ +@Private +@Unstable +class SchedulingAlgorithms { + public static final Log LOG = LogFactory.getLog( + SchedulingAlgorithms.class.getName()); + + /** + * Compare Schedulables in order of priority and then submission time, as in + * the default FIFO scheduler in Hadoop. + */ + public static class FifoComparator implements Comparator, Serializable { + private static final long serialVersionUID = -5905036205491177060L; + + @Override + public int compare(Schedulable s1, Schedulable s2) { + int res = s1.getPriority().compareTo(s2.getPriority()); + if (res == 0) { + res = (int) Math.signum(s1.getStartTime() - s2.getStartTime()); + } + if (res == 0) { + // In the rare case where jobs were submitted at the exact same time, + // compare them by name (which will be the JobID) to get a deterministic + // ordering, so we don't alternately launch tasks from different jobs. + res = s1.getName().compareTo(s2.getName()); + } + return res; + } + } + + /** + * Compare Schedulables via weighted fair sharing. In addition, Schedulables + * below their min share get priority over those whose min share is met. + * + * Schedulables below their min share are compared by how far below it they + * are as a ratio. For example, if job A has 8 out of a min share of 10 tasks + * and job B has 50 out of a min share of 100, then job B is scheduled next, + * because B is at 50% of its min share and A is at 80% of its min share. + * + * Schedulables above their min share are compared by (runningTasks / weight). + * If all weights are equal, slots are given to the job with the fewest tasks; + * otherwise, jobs with more weight get proportionally more slots. + */ + public static class FairShareComparator implements Comparator, Serializable { + private static final long serialVersionUID = 5564969375856699313L; + + @Override + public int compare(Schedulable s1, Schedulable s2) { + double minShareRatio1, minShareRatio2; + double useToWeightRatio1, useToWeightRatio2; + Resource minShare1 = Resources.min(s1.getMinShare(), s1.getDemand()); + Resource minShare2 = Resources.min(s2.getMinShare(), s2.getDemand()); + boolean s1Needy = Resources.lessThan(s1.getResourceUsage(), minShare1); + boolean s2Needy = Resources.lessThan(s2.getResourceUsage(), minShare2); + Resource one = Resources.createResource(1); + minShareRatio1 = (double) s1.getResourceUsage().getMemory() / + Resources.max(minShare1, one).getMemory(); + minShareRatio2 = (double) s2.getResourceUsage().getMemory() / + Resources.max(minShare2, one).getMemory(); + useToWeightRatio1 = s1.getResourceUsage().getMemory() / s1.getWeight(); + useToWeightRatio2 = s2.getResourceUsage().getMemory() / s2.getWeight(); + int res = 0; + if (s1Needy && !s2Needy) + res = -1; + else if (s2Needy && !s1Needy) + res = 1; + else if (s1Needy && s2Needy) + res = (int) Math.signum(minShareRatio1 - minShareRatio2); + else // Neither schedulable is needy + res = (int) Math.signum(useToWeightRatio1 - useToWeightRatio2); + if (res == 0) { + // Apps are tied in fairness ratio. Break the tie by submit time and job + // name to get a deterministic ordering, which is useful for unit tests. + res = (int) Math.signum(s1.getStartTime() - s2.getStartTime()); + if (res == 0) + res = s1.getName().compareTo(s2.getName()); + } + return res; + } + } + + /** + * Number of iterations for the binary search in computeFairShares. This is + * equivalent to the number of bits of precision in the output. 25 iterations + * gives precision better than 0.1 slots in clusters with one million slots. + */ + private static final int COMPUTE_FAIR_SHARES_ITERATIONS = 25; + + /** + * Given a set of Schedulables and a number of slots, compute their weighted + * fair shares. The min shares and demands of the Schedulables are assumed to + * be set beforehand. We compute the fairest possible allocation of shares + * to the Schedulables that respects their min shares and demands. + * + * To understand what this method does, we must first define what weighted + * fair sharing means in the presence of minimum shares and demands. If there + * were no minimum shares and every Schedulable had an infinite demand (i.e. + * could launch infinitely many tasks), then weighted fair sharing would be + * achieved if the ratio of slotsAssigned / weight was equal for each + * Schedulable and all slots were assigned. Minimum shares and demands add + * two further twists: + * - Some Schedulables may not have enough tasks to fill all their share. + * - Some Schedulables may have a min share higher than their assigned share. + * + * To deal with these possibilities, we define an assignment of slots as + * being fair if there exists a ratio R such that: + * - Schedulables S where S.demand < R * S.weight are assigned share S.demand + * - Schedulables S where S.minShare > R * S.weight are given share S.minShare + * - All other Schedulables S are assigned share R * S.weight + * - The sum of all the shares is totalSlots. + * + * We call R the weight-to-slots ratio because it converts a Schedulable's + * weight to the number of slots it is assigned. + * + * We compute a fair allocation by finding a suitable weight-to-slot ratio R. + * To do this, we use binary search. Given a ratio R, we compute the number + * of slots that would be used in total with this ratio (the sum of the shares + * computed using the conditions above). If this number of slots is less than + * totalSlots, then R is too small and more slots could be assigned. If the + * number of slots is more than totalSlots, then R is too large. + * + * We begin the binary search with a lower bound on R of 0 (which means that + * all Schedulables are only given their minShare) and an upper bound computed + * to be large enough that too many slots are given (by doubling R until we + * either use more than totalSlots slots or we fulfill all jobs' demands). + * The helper method slotsUsedWithWeightToSlotRatio computes the total number + * of slots used with a given value of R. + * + * The running time of this algorithm is linear in the number of Schedulables, + * because slotsUsedWithWeightToSlotRatio is linear-time and the number of + * iterations of binary search is a constant (dependent on desired precision). + */ + public static void computeFairShares( + Collection schedulables, Resource totalResources) { + // Find an upper bound on R that we can use in our binary search. We start + // at R = 1 and double it until we have either used totalSlots slots or we + // have met all Schedulables' demands (if total demand < totalSlots). + Resource totalDemand = Resources.createResource(0); + for (Schedulable sched: schedulables) { + Resources.addTo(totalDemand, sched.getDemand()); + } + Resource cap = Resources.min(totalDemand, totalResources); + double rMax = 1.0; + while (Resources.lessThan(resUsedWithWeightToResRatio(rMax, schedulables), cap)) { + rMax *= 2.0; + } + // Perform the binary search for up to COMPUTE_FAIR_SHARES_ITERATIONS steps + double left = 0; + double right = rMax; + for (int i = 0; i < COMPUTE_FAIR_SHARES_ITERATIONS; i++) { + double mid = (left + right) / 2.0; + if (Resources.lessThan(resUsedWithWeightToResRatio(mid, schedulables), cap)) { + left = mid; + } else { + right = mid; + } + } + // Set the fair shares based on the value of R we've converged to + for (Schedulable sched: schedulables) { + sched.setFairShare(computeShare(sched, right)); + } + } + + /** + * Compute the number of slots that would be used given a weight-to-slot + * ratio w2sRatio, for use in the computeFairShares algorithm as described + * in #{@link SchedulingAlgorithms#computeFairShares(Collection, double)}. + */ + private static Resource resUsedWithWeightToResRatio(double w2sRatio, + Collection schedulables) { + Resource slotsTaken = Resources.createResource(0); + for (Schedulable sched: schedulables) { + Resource share = computeShare(sched, w2sRatio); + Resources.addTo(slotsTaken, share); + } + return slotsTaken; + } + + /** + * Compute the resources assigned to a Schedulable given a particular + * res-to-slot ratio r2sRatio, for use in computeFairShares as described + * in #{@link SchedulingAlgorithms#computeFairShares(Collection, double)}. + */ + private static Resource computeShare(Schedulable sched, double r2sRatio) { + double share = sched.getWeight() * r2sRatio; + share = Math.max(share, sched.getMinShare().getMemory()); + share = Math.min(share, sched.getDemand().getMemory()); + return Resources.createResource((int) share); + } +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingMode.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingMode.java new file mode 100644 index 00000000000..b84d824f6cb --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingMode.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; + +/** + * Internal scheduling modes for queues. + */ +@Private +@Unstable +public enum SchedulingMode { + FAIR, FIFO +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/WeightAdjuster.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/WeightAdjuster.java new file mode 100644 index 00000000000..2fa71badbc7 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/WeightAdjuster.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configurable; + +/** + * A pluggable object for altering the weights of apps in the fair scheduler, + * which is used for example by {@link NewJobWeightBooster} to give higher + * weight to new jobs so that short jobs finish faster. + * + * May implement {@link Configurable} to access configuration parameters. + */ +@Private +@Unstable +public interface WeightAdjuster { + public double adjustWeight(AppSchedulable app, double curWeight); +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java index 957fe676d34..0c1d6f26c7b 100644 --- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java @@ -18,14 +18,17 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp; -import static org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebApp.QUEUE_NAME; import static org.apache.hadoop.yarn.util.StringHelper.join; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.QUEUE_NAME; + import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; import org.apache.hadoop.yarn.util.StringHelper; import org.apache.hadoop.yarn.webapp.Controller; +import org.apache.hadoop.yarn.webapp.WebAppException; import org.apache.hadoop.yarn.webapp.YarnWebParams; import com.google.inject.Inject; @@ -71,6 +74,12 @@ public class RmController extends Controller { render(CapacitySchedulerPage.class); return; } + + if (rs instanceof FairScheduler) { + context().setStatus(404); + throw new WebAppException("Fair Scheduler UI not yet supported"); + } + setTitle("Default Scheduler"); render(DefaultSchedulerPage.class); } diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSSchedulerApp.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSSchedulerApp.java new file mode 100644 index 00000000000..a8da65ce965 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSSchedulerApp.java @@ -0,0 +1,116 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler; + +import static org.junit.Assert.assertEquals; + +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.factories.RecordFactory; +import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSSchedulerApp; +import org.junit.Test; +import org.mockito.Mockito; + +public class TestFSSchedulerApp { + private static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); + + private ApplicationAttemptId createAppAttemptId(int appId, int attemptId) { + ApplicationAttemptId attId = recordFactory.newRecordInstance(ApplicationAttemptId.class); + ApplicationId appIdImpl = recordFactory.newRecordInstance(ApplicationId.class); + appIdImpl.setId(appId); + attId.setAttemptId(attemptId); + attId.setApplicationId(appIdImpl); + return attId; + } + + @Test + public void testDelayScheduling() { + Queue queue = Mockito.mock(Queue.class); + Priority prio = Mockito.mock(Priority.class); + Mockito.when(prio.getPriority()).thenReturn(1); + double nodeLocalityThreshold = .5; + double rackLocalityThreshold = .6; + + ApplicationAttemptId applicationAttemptId = createAppAttemptId(1, 1); + FSSchedulerApp schedulerApp = + new FSSchedulerApp(applicationAttemptId, "user1", queue , null, null, null); + + // Default level should be node-local + assertEquals(NodeType.NODE_LOCAL, schedulerApp.getAllowedLocalityLevel( + prio, 10, nodeLocalityThreshold, rackLocalityThreshold)); + + // First five scheduling opportunities should remain node local + for (int i = 0; i < 5; i++) { + schedulerApp.addSchedulingOpportunity(prio); + assertEquals(NodeType.NODE_LOCAL, schedulerApp.getAllowedLocalityLevel( + prio, 10, nodeLocalityThreshold, rackLocalityThreshold)); + } + + // After five it should switch to rack local + schedulerApp.addSchedulingOpportunity(prio); + assertEquals(NodeType.RACK_LOCAL, schedulerApp.getAllowedLocalityLevel( + prio, 10, nodeLocalityThreshold, rackLocalityThreshold)); + + // Manually set back to node local + schedulerApp.resetAllowedLocalityLevel(prio, NodeType.NODE_LOCAL); + schedulerApp.resetSchedulingOpportunities(prio); + assertEquals(NodeType.NODE_LOCAL, schedulerApp.getAllowedLocalityLevel( + prio, 10, nodeLocalityThreshold, rackLocalityThreshold)); + + // Now escalate again to rack-local, then to off-switch + for (int i = 0; i < 5; i++) { + schedulerApp.addSchedulingOpportunity(prio); + assertEquals(NodeType.NODE_LOCAL, schedulerApp.getAllowedLocalityLevel( + prio, 10, nodeLocalityThreshold, rackLocalityThreshold)); + } + + schedulerApp.addSchedulingOpportunity(prio); + assertEquals(NodeType.RACK_LOCAL, schedulerApp.getAllowedLocalityLevel( + prio, 10, nodeLocalityThreshold, rackLocalityThreshold)); + + for (int i = 0; i < 6; i++) { + schedulerApp.addSchedulingOpportunity(prio); + assertEquals(NodeType.RACK_LOCAL, schedulerApp.getAllowedLocalityLevel( + prio, 10, nodeLocalityThreshold, rackLocalityThreshold)); + } + + schedulerApp.addSchedulingOpportunity(prio); + assertEquals(NodeType.OFF_SWITCH, schedulerApp.getAllowedLocalityLevel( + prio, 10, nodeLocalityThreshold, rackLocalityThreshold)); + } + + @Test + /** + * Ensure that when negative paramaters are given (signaling delay scheduling + * no tin use), the least restrictive locality level is returned. + */ + public void testLocalityLevelWithoutDelays() { + Queue queue = Mockito.mock(Queue.class); + Priority prio = Mockito.mock(Priority.class); + Mockito.when(prio.getPriority()).thenReturn(1); + + ApplicationAttemptId applicationAttemptId = createAppAttemptId(1, 1); + FSSchedulerApp schedulerApp = + new FSSchedulerApp(applicationAttemptId, "user1", queue , null, null, null); + assertEquals(NodeType.OFF_SWITCH, schedulerApp.getAllowedLocalityLevel( + prio, 10, -1.0, -1.0)); + } +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java new file mode 100644 index 00000000000..2b8d6cc0295 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java @@ -0,0 +1,992 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.io.PrintWriter; +import java.util.ArrayList; +import java.util.Collection; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.authorize.AccessControlList; +import org.apache.hadoop.yarn.Clock; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.QueueACL; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.AsyncDispatcher; +import org.apache.hadoop.yarn.factories.RecordFactory; +import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; +import org.apache.hadoop.yarn.server.resourcemanager.MockNodes; +import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store; +import org.apache.hadoop.yarn.server.resourcemanager.recovery.StoreFactory; +import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; +import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class TestFairScheduler { + + private class MockClock implements Clock { + private long time = 0; + @Override + public long getTime() { + return time; + } + + public void tick(int seconds) { + time = time + seconds * 1000; + } + + } + + final static String TEST_DIR = new File(System.getProperty("test.build.data", + "/tmp")).getAbsolutePath(); + + final static String ALLOC_FILE = new File(TEST_DIR, + "test-queues").getAbsolutePath(); + + private FairScheduler scheduler; + private ResourceManager resourceManager; + private static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); + + private int APP_ID = 1; // Incrementing counter for schedling apps + private int ATTEMPT_ID = 1; // Incrementing counter for scheduling attempts + + // HELPER METHODS + @Before + public void setUp() throws IOException { + scheduler = new FairScheduler(); + Configuration conf = new Configuration(); + // All tests assume only one assignment per node update + conf.set(FairSchedulerConfiguration.ASSIGN_MULTIPLE, "false"); + Store store = StoreFactory.getStore(conf); + resourceManager = new ResourceManager(store); + resourceManager.init(conf); + ((AsyncDispatcher)resourceManager.getRMContext().getDispatcher()).start(); + scheduler.reinitialize(conf, null, resourceManager.getRMContext()); + } + + @After + public void tearDown() { + scheduler = null; + resourceManager = null; + } + + private ApplicationAttemptId createAppAttemptId(int appId, int attemptId) { + ApplicationAttemptId attId = recordFactory.newRecordInstance(ApplicationAttemptId.class); + ApplicationId appIdImpl = recordFactory.newRecordInstance(ApplicationId.class); + appIdImpl.setId(appId); + attId.setAttemptId(attemptId); + attId.setApplicationId(appIdImpl); + return attId; + } + + + private ResourceRequest createResourceRequest(int memory, String host, int priority, int numContainers) { + ResourceRequest request = recordFactory.newRecordInstance(ResourceRequest.class); + request.setCapability(Resources.createResource(memory)); + request.setHostName(host); + request.setNumContainers(numContainers); + Priority prio = recordFactory.newRecordInstance(Priority.class); + prio.setPriority(priority); + request.setPriority(prio); + return request; + } + + /** + * Creates a single container priority-1 request and submits to + * scheduler. + */ + private ApplicationAttemptId createSchedulingRequest(int memory, String queueId, String userId) { + return createSchedulingRequest(memory, queueId, userId, 1); + } + + private ApplicationAttemptId createSchedulingRequest(int memory, String queueId, String userId, int numContainers) { + return createSchedulingRequest(memory, queueId, userId, numContainers, 1); + } + + private ApplicationAttemptId createSchedulingRequest(int memory, String queueId, String userId, int numContainers, int priority) { + ApplicationAttemptId id = createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++); + scheduler.addApplication(id, queueId, userId); + List ask = new ArrayList(); + ResourceRequest request = createResourceRequest(memory, "*", priority, numContainers); + ask.add(request); + scheduler.allocate(id, ask, new ArrayList()); + return id; + } + + // TESTS + + @Test + public void testAggregateCapacityTracking() throws Exception { + // Add a node + RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024)); + NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); + scheduler.handle(nodeEvent1); + assertEquals(1024, scheduler.getClusterCapacity().getMemory()); + + // Add another node + RMNode node2 = MockNodes.newNodeInfo(1, Resources.createResource(512)); + NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2); + scheduler.handle(nodeEvent2); + assertEquals(1536, scheduler.getClusterCapacity().getMemory()); + + // Remove the first node + NodeRemovedSchedulerEvent nodeEvent3 = new NodeRemovedSchedulerEvent(node1); + scheduler.handle(nodeEvent3); + assertEquals(512, scheduler.getClusterCapacity().getMemory()); + } + + @Test + public void testSimpleFairShareCalculation() { + // Add one big node (only care about aggregate capacity) + RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(10 * 1024)); + NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); + scheduler.handle(nodeEvent1); + + // Have two queues which want entire cluster capacity + createSchedulingRequest(10 * 1024, "queue1", "user1"); + createSchedulingRequest(10 * 1024, "queue2", "user1"); + + scheduler.update(); + + Collection queues = scheduler.getQueueManager().getQueues(); + assertEquals(3, queues.size()); + + for (FSQueue p : queues) { + if (p.getName() != "default") { + assertEquals(5120, p.getQueueSchedulable().getFairShare().getMemory()); + } + } + } + + @Test + public void testSimpleContainerAllocation() { + // Add a node + RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024)); + NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); + scheduler.handle(nodeEvent1); + + // Add another node + RMNode node2 = MockNodes.newNodeInfo(1, Resources.createResource(512)); + NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2); + scheduler.handle(nodeEvent2); + + createSchedulingRequest(512, "queue1", "user1", 2); + + scheduler.update(); + + NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1, + new ArrayList(), new ArrayList()); + scheduler.handle(updateEvent); + + assertEquals(512, scheduler.getQueueManager().getQueue("queue1"). + getQueueSchedulable().getResourceUsage().getMemory()); + + NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2, + new ArrayList(), new ArrayList()); + scheduler.handle(updateEvent2); + + assertEquals(1024, scheduler.getQueueManager().getQueue("queue1"). + getQueueSchedulable().getResourceUsage().getMemory()); + } + + @Test + public void testSimpleContainerReservation() throws InterruptedException { + // Add a node + RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024)); + NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); + scheduler.handle(nodeEvent1); + + // Queue 1 requests full capacity of node + createSchedulingRequest(1024, "queue1", "user1", 1); + scheduler.update(); + NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1, + new ArrayList(), new ArrayList()); + scheduler.handle(updateEvent); + + // Make sure queue 1 is allocated app capacity + assertEquals(1024, scheduler.getQueueManager().getQueue("queue1"). + getQueueSchedulable().getResourceUsage().getMemory()); + + // Now queue 2 requests likewise + ApplicationAttemptId attId = createSchedulingRequest(1024, "queue2", "user1", 1); + scheduler.update(); + scheduler.handle(updateEvent); + + // Make sure queue 2 is waiting with a reservation + assertEquals(0, scheduler.getQueueManager().getQueue("queue2"). + getQueueSchedulable().getResourceUsage().getMemory()); + assertEquals(1024, scheduler.applications.get(attId).getCurrentReservation().getMemory()); + + // Now another node checks in with capacity + RMNode node2 = MockNodes.newNodeInfo(1, Resources.createResource(1024)); + NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2); + NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2, + new ArrayList(), new ArrayList()); + scheduler.handle(nodeEvent2); + scheduler.handle(updateEvent2); + + // Make sure this goes to queue 2 + assertEquals(1024, scheduler.getQueueManager().getQueue("queue2"). + getQueueSchedulable().getResourceUsage().getMemory()); + + // The old reservation should still be there... + assertEquals(1024, scheduler.applications.get(attId).getCurrentReservation().getMemory()); + // ... but it should disappear when we update the first node. + scheduler.handle(updateEvent); + assertEquals(0, scheduler.applications.get(attId).getCurrentReservation().getMemory()); + + } + + @Test + public void testUserAsDefaultQueue() throws Exception { + Configuration conf = new Configuration(); + conf.set(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, "true"); + scheduler.reinitialize(conf, null, resourceManager.getRMContext()); + AppAddedSchedulerEvent appAddedEvent = new AppAddedSchedulerEvent( + createAppAttemptId(1, 1), "default", "user1"); + scheduler.handle(appAddedEvent); + assertEquals(1, scheduler.getQueueManager().getQueue("user1").getApplications().size()); + assertEquals(0, scheduler.getQueueManager().getQueue("default").getApplications().size()); + + conf.set(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, "false"); + scheduler.reinitialize(conf, null, resourceManager.getRMContext()); + AppAddedSchedulerEvent appAddedEvent2 = new AppAddedSchedulerEvent( + createAppAttemptId(2, 1), "default", "user2"); + scheduler.handle(appAddedEvent2); + assertEquals(1, scheduler.getQueueManager().getQueue("user1").getApplications().size()); + assertEquals(1, scheduler.getQueueManager().getQueue("default").getApplications().size()); + assertEquals(0, scheduler.getQueueManager().getQueue("user2").getApplications().size()); + } + + @Test + public void testFairShareWithMinAlloc() throws Exception { + Configuration conf = new Configuration(); + conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); + scheduler.reinitialize(conf, null, resourceManager.getRMContext()); + + PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE)); + out.println(""); + out.println(""); + out.println(""); + out.println("1024"); + out.println(""); + out.println(""); + out.println("2048"); + out.println(""); + out.println(""); + out.close(); + + QueueManager queueManager = scheduler.getQueueManager(); + queueManager.initialize(); + + // Add one big node (only care about aggregate capacity) + RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(3 * 1024)); + NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); + scheduler.handle(nodeEvent1); + + createSchedulingRequest(2 * 1024, "queueA", "user1"); + createSchedulingRequest(2 * 1024, "queueB", "user1"); + + scheduler.update(); + + Collection queues = scheduler.getQueueManager().getQueues(); + assertEquals(3, queues.size()); + + for (FSQueue p : queues) { + if (p.getName().equals("queueA")) { + assertEquals(1024, p.getQueueSchedulable().getFairShare().getMemory()); + } + else if (p.getName().equals("queueB")) { + assertEquals(2048, p.getQueueSchedulable().getFairShare().getMemory()); + } + } + + } + + /** + * Make allocation requests and ensure they are reflected in queue demand. + */ + @Test + public void testQueueDemandCalculation() throws Exception { + ApplicationAttemptId id11 = createAppAttemptId(1, 1); + scheduler.addApplication(id11, "queue1", "user1"); + ApplicationAttemptId id21 = createAppAttemptId(2, 1); + scheduler.addApplication(id21, "queue2", "user1"); + ApplicationAttemptId id22 = createAppAttemptId(2, 2); + scheduler.addApplication(id22, "queue2", "user1"); + + // First ask, queue1 requests 1024 + List ask1 = new ArrayList(); + ResourceRequest request1 = createResourceRequest(1024, "*", 1, 1); + ask1.add(request1); + scheduler.allocate(id11, ask1, new ArrayList()); + + // Second ask, queue2 requests 1024 + (2 * 512) + List ask2 = new ArrayList(); + ResourceRequest request2 = createResourceRequest(1024, "foo", 1, 1); + ResourceRequest request3 = createResourceRequest(512, "bar", 1, 2); + ask2.add(request2); + ask2.add(request3); + scheduler.allocate(id21, ask2, new ArrayList()); + + // Third ask, queue2 requests 1024 + List ask3 = new ArrayList(); + ResourceRequest request4 = createResourceRequest(1024, "*", 1, 1); + ask3.add(request4); + scheduler.allocate(id22, ask3, new ArrayList()); + + scheduler.update(); + + assertEquals(1024, scheduler.getQueueManager().getQueue("queue1").getQueueSchedulable().getDemand().getMemory()); + assertEquals(1024 + 1024 + (2 * 512), scheduler.getQueueManager().getQueue("queue2").getQueueSchedulable().getDemand().getMemory()); + + } + + @Test + public void testAppAdditionAndRemoval() throws Exception { + AppAddedSchedulerEvent appAddedEvent1 = new AppAddedSchedulerEvent( + createAppAttemptId(1, 1), "default", "user1"); + scheduler.handle(appAddedEvent1); + + // Scheduler should have one queue (the default) + assertEquals(1, scheduler.getQueueManager().getQueues().size()); + + // That queue should have one app + assertEquals(1, scheduler.getQueueManager().getQueue("default").getApplications().size()); + + AppRemovedSchedulerEvent appRemovedEvent1 = new AppRemovedSchedulerEvent( + createAppAttemptId(1, 1), RMAppAttemptState.FINISHED); + + // Now remove app + scheduler.handle(appRemovedEvent1); + + // Default queue should have no apps + assertEquals(0, scheduler.getQueueManager().getQueue("default").getApplications().size()); + } + + @Test + public void testAllocationFileParsing() throws Exception { + Configuration conf = new Configuration(); + conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); + scheduler.reinitialize(conf, null, resourceManager.getRMContext()); + + PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE)); + out.println(""); + out.println(""); + // Give queue A a minimum of 1024 M + out.println(""); + out.println("1024"); + out.println(""); + // Give queue B a minimum of 2048 M + out.println(""); + out.println("2048"); + out.println("alice,bob admins"); + out.println(""); + // Give queue C no minimum + out.println(""); + out.println("alice,bob admins"); + out.println(""); + // Give queue D a limit of 3 running apps + out.println(""); + out.println("3"); + out.println(""); + // Give queue E a preemption timeout of one minute + out.println(""); + out.println("60"); + out.println(""); + // Set default limit of apps per queue to 15 + out.println("15"); + // Set default limit of apps per user to 5 + out.println("5"); + // Give user1 a limit of 10 jobs + out.println(""); + out.println("10"); + out.println(""); + // Set default min share preemption timeout to 2 minutes + out.println("120" + + ""); + // Set fair share preemption timeout to 5 minutes + out.println("300"); + out.println(""); + out.close(); + + QueueManager queueManager = scheduler.getQueueManager(); + queueManager.initialize(); + + assertEquals(6, queueManager.getQueues().size()); // 5 in file + default queue + assertEquals(Resources.createResource(0), + queueManager.getMinResources(YarnConfiguration.DEFAULT_QUEUE_NAME)); + assertEquals(Resources.createResource(0), + queueManager.getMinResources(YarnConfiguration.DEFAULT_QUEUE_NAME)); + + assertEquals(Resources.createResource(1024), + queueManager.getMinResources("queueA")); + assertEquals(Resources.createResource(2048), + queueManager.getMinResources("queueB")); + assertEquals(Resources.createResource(0), + queueManager.getMinResources("queueC")); + assertEquals(Resources.createResource(0), + queueManager.getMinResources("queueD")); + assertEquals(Resources.createResource(0), + queueManager.getMinResources("queueE")); + + assertEquals(15, queueManager.getQueueMaxApps(YarnConfiguration.DEFAULT_QUEUE_NAME)); + assertEquals(15, queueManager.getQueueMaxApps("queueA")); + assertEquals(15, queueManager.getQueueMaxApps("queueB")); + assertEquals(15, queueManager.getQueueMaxApps("queueC")); + assertEquals(3, queueManager.getQueueMaxApps("queueD")); + assertEquals(15, queueManager.getQueueMaxApps("queueE")); + assertEquals(10, queueManager.getUserMaxApps("user1")); + assertEquals(5, queueManager.getUserMaxApps("user2")); + + // Unspecified queues should get default ACL + Map aclsA = queueManager.getQueueAcls("queueA"); + assertTrue(aclsA.containsKey(QueueACL.ADMINISTER_QUEUE)); + assertEquals("*", aclsA.get(QueueACL.ADMINISTER_QUEUE).getAclString()); + assertTrue(aclsA.containsKey(QueueACL.SUBMIT_APPLICATIONS)); + assertEquals("*", aclsA.get(QueueACL.SUBMIT_APPLICATIONS).getAclString()); + + // Queue B ACL + Map aclsB = queueManager.getQueueAcls("queueB"); + assertTrue(aclsB.containsKey(QueueACL.ADMINISTER_QUEUE)); + assertEquals("alice,bob admins", aclsB.get(QueueACL.ADMINISTER_QUEUE).getAclString()); + + // Queue c ACL + Map aclsC = queueManager.getQueueAcls("queueC"); + assertTrue(aclsC.containsKey(QueueACL.SUBMIT_APPLICATIONS)); + assertEquals("alice,bob admins", aclsC.get(QueueACL.SUBMIT_APPLICATIONS).getAclString()); + + assertEquals(120000, queueManager.getMinSharePreemptionTimeout( + YarnConfiguration.DEFAULT_QUEUE_NAME)); + assertEquals(120000, queueManager.getMinSharePreemptionTimeout("queueA")); + assertEquals(120000, queueManager.getMinSharePreemptionTimeout("queueB")); + assertEquals(120000, queueManager.getMinSharePreemptionTimeout("queueC")); + assertEquals(120000, queueManager.getMinSharePreemptionTimeout("queueD")); + assertEquals(120000, queueManager.getMinSharePreemptionTimeout("queueA")); + assertEquals(60000, queueManager.getMinSharePreemptionTimeout("queueE")); + assertEquals(300000, queueManager.getFairSharePreemptionTimeout()); + } + + @Test + public void testBackwardsCompatibleAllocationFileParsing() throws Exception { + Configuration conf = new Configuration(); + conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); + scheduler.reinitialize(conf, null, resourceManager.getRMContext()); + + PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE)); + out.println(""); + out.println(""); + // Give queue A a minimum of 1024 M + out.println(""); + out.println("1024"); + out.println(""); + // Give queue B a minimum of 2048 M + out.println(""); + out.println("2048"); + out.println("alice,bob admins"); + out.println(""); + // Give queue C no minimum + out.println(""); + out.println("alice,bob admins"); + out.println(""); + // Give queue D a limit of 3 running apps + out.println(""); + out.println("3"); + out.println(""); + // Give queue E a preemption timeout of one minute + out.println(""); + out.println("60"); + out.println(""); + // Set default limit of apps per queue to 15 + out.println("15"); + // Set default limit of apps per user to 5 + out.println("5"); + // Give user1 a limit of 10 jobs + out.println(""); + out.println("10"); + out.println(""); + // Set default min share preemption timeout to 2 minutes + out.println("120" + + ""); + // Set fair share preemption timeout to 5 minutes + out.println("300"); + out.println(""); + out.close(); + + QueueManager queueManager = scheduler.getQueueManager(); + queueManager.initialize(); + + assertEquals(6, queueManager.getQueues().size()); // 5 in file + default queue + assertEquals(Resources.createResource(0), + queueManager.getMinResources(YarnConfiguration.DEFAULT_QUEUE_NAME)); + assertEquals(Resources.createResource(0), + queueManager.getMinResources(YarnConfiguration.DEFAULT_QUEUE_NAME)); + + assertEquals(Resources.createResource(1024), + queueManager.getMinResources("queueA")); + assertEquals(Resources.createResource(2048), + queueManager.getMinResources("queueB")); + assertEquals(Resources.createResource(0), + queueManager.getMinResources("queueC")); + assertEquals(Resources.createResource(0), + queueManager.getMinResources("queueD")); + assertEquals(Resources.createResource(0), + queueManager.getMinResources("queueE")); + + assertEquals(15, queueManager.getQueueMaxApps(YarnConfiguration.DEFAULT_QUEUE_NAME)); + assertEquals(15, queueManager.getQueueMaxApps("queueA")); + assertEquals(15, queueManager.getQueueMaxApps("queueB")); + assertEquals(15, queueManager.getQueueMaxApps("queueC")); + assertEquals(3, queueManager.getQueueMaxApps("queueD")); + assertEquals(15, queueManager.getQueueMaxApps("queueE")); + assertEquals(10, queueManager.getUserMaxApps("user1")); + assertEquals(5, queueManager.getUserMaxApps("user2")); + + // Unspecified queues should get default ACL + Map aclsA = queueManager.getQueueAcls("queueA"); + assertTrue(aclsA.containsKey(QueueACL.ADMINISTER_QUEUE)); + assertEquals("*", aclsA.get(QueueACL.ADMINISTER_QUEUE).getAclString()); + assertTrue(aclsA.containsKey(QueueACL.SUBMIT_APPLICATIONS)); + assertEquals("*", aclsA.get(QueueACL.SUBMIT_APPLICATIONS).getAclString()); + + // Queue B ACL + Map aclsB = queueManager.getQueueAcls("queueB"); + assertTrue(aclsB.containsKey(QueueACL.ADMINISTER_QUEUE)); + assertEquals("alice,bob admins", aclsB.get(QueueACL.ADMINISTER_QUEUE).getAclString()); + + // Queue c ACL + Map aclsC = queueManager.getQueueAcls("queueC"); + assertTrue(aclsC.containsKey(QueueACL.SUBMIT_APPLICATIONS)); + assertEquals("alice,bob admins", aclsC.get(QueueACL.SUBMIT_APPLICATIONS).getAclString()); + + assertEquals(120000, queueManager.getMinSharePreemptionTimeout( + YarnConfiguration.DEFAULT_QUEUE_NAME)); + assertEquals(120000, queueManager.getMinSharePreemptionTimeout("queueA")); + assertEquals(120000, queueManager.getMinSharePreemptionTimeout("queueB")); + assertEquals(120000, queueManager.getMinSharePreemptionTimeout("queueC")); + assertEquals(120000, queueManager.getMinSharePreemptionTimeout("queueD")); + assertEquals(120000, queueManager.getMinSharePreemptionTimeout("queueA")); + assertEquals(60000, queueManager.getMinSharePreemptionTimeout("queueE")); + assertEquals(300000, queueManager.getFairSharePreemptionTimeout()); + } + + @Test + public void testIsStarvedForMinShare() throws Exception { + Configuration conf = new Configuration(); + conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); + scheduler.reinitialize(conf, null, resourceManager.getRMContext()); + + PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE)); + out.println(""); + out.println(""); + out.println(""); + out.println("2048"); + out.println(""); + out.println(""); + out.println("2048"); + out.println(""); + out.println(""); + out.close(); + + QueueManager queueManager = scheduler.getQueueManager(); + queueManager.initialize(); + + // Add one big node (only care about aggregate capacity) + RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(4 * 1024)); + NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); + scheduler.handle(nodeEvent1); + + // Queue A wants 3 * 1024. Node update gives this all to A + createSchedulingRequest(3 * 1024, "queueA", "user1"); + scheduler.update(); + NodeUpdateSchedulerEvent nodeEvent2 = new NodeUpdateSchedulerEvent(node1, + new LinkedList(), new LinkedList()); + scheduler.handle(nodeEvent2); + + // Queue B arrives and wants 1 * 1024 + createSchedulingRequest(1 * 1024, "queueB", "user1"); + scheduler.update(); + Collection queues = scheduler.getQueueManager().getQueues(); + assertEquals(3, queues.size()); + + // Queue A should be above min share, B below. + for (FSQueue p : queues) { + if (p.getName().equals("queueA")) { + assertEquals(false, scheduler.isStarvedForMinShare(p.getQueueSchedulable())); + } + else if (p.getName().equals("queueB")) { + assertEquals(true, scheduler.isStarvedForMinShare(p.getQueueSchedulable())); + } + } + + // Node checks in again, should allocate for B + scheduler.handle(nodeEvent2); + // Now B should have min share ( = demand here) + for (FSQueue p : queues) { + if (p.getName().equals("queueB")) { + assertEquals(false, scheduler.isStarvedForMinShare(p.getQueueSchedulable())); + } + } + } + + @Test + public void testIsStarvedForFairShare() throws Exception { + Configuration conf = new Configuration(); + conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); + scheduler.reinitialize(conf, null, resourceManager.getRMContext()); + + PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE)); + out.println(""); + out.println(""); + out.println(""); + out.println(".25"); + out.println(""); + out.println(""); + out.println(".75"); + out.println(""); + out.println(""); + out.close(); + + QueueManager queueManager = scheduler.getQueueManager(); + queueManager.initialize(); + + // Add one big node (only care about aggregate capacity) + RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(4 * 1024)); + NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); + scheduler.handle(nodeEvent1); + + // Queue A wants 3 * 1024. Node update gives this all to A + createSchedulingRequest(3 * 1024, "queueA", "user1"); + scheduler.update(); + NodeUpdateSchedulerEvent nodeEvent2 = new NodeUpdateSchedulerEvent(node1, + new LinkedList(), new LinkedList()); + scheduler.handle(nodeEvent2); + + // Queue B arrives and wants 1 * 1024 + createSchedulingRequest(1 * 1024, "queueB", "user1"); + scheduler.update(); + Collection queues = scheduler.getQueueManager().getQueues(); + assertEquals(3, queues.size()); + + // Queue A should be above fair share, B below. + for (FSQueue p : queues) { + if (p.getName().equals("queueA")) { + assertEquals(false, scheduler.isStarvedForFairShare(p.getQueueSchedulable())); + } + else if (p.getName().equals("queueB")) { + assertEquals(true, scheduler.isStarvedForFairShare(p.getQueueSchedulable())); + } + } + + // Node checks in again, should allocate for B + scheduler.handle(nodeEvent2); + // B should not be starved for fair share, since entire demand is + // satisfied. + for (FSQueue p : queues) { + if (p.getName().equals("queueB")) { + assertEquals(false, scheduler.isStarvedForFairShare(p.getQueueSchedulable())); + } + } + } + + @Test + /** + * Make sure containers are chosen to be preempted in the correct order. Right + * now this means decreasing order of priority. + */ + public void testChoiceOfPreemptedContainers() throws Exception { + Configuration conf = new Configuration(); + conf.set(FairSchedulerConfiguration.ALLOCATION_FILE + ".allocation.file", ALLOC_FILE); + scheduler.reinitialize(conf, null, resourceManager.getRMContext()); + + PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE)); + out.println(""); + out.println(""); + out.println(""); + out.println(".25"); + out.println(""); + out.println(""); + out.println(".25"); + out.println(""); + out.println(""); + out.println(".25"); + out.println(""); + out.println(""); + out.println(".25"); + out.println(""); + out.println(""); + out.close(); + + QueueManager queueManager = scheduler.getQueueManager(); + queueManager.initialize(); + + // Create four nodes + RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(2 * 1024)); + NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); + scheduler.handle(nodeEvent1); + + RMNode node2 = MockNodes.newNodeInfo(1, Resources.createResource(2 * 1024)); + NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2); + scheduler.handle(nodeEvent2); + + RMNode node3 = MockNodes.newNodeInfo(1, Resources.createResource(2 * 1024)); + NodeAddedSchedulerEvent nodeEvent3 = new NodeAddedSchedulerEvent(node3); + scheduler.handle(nodeEvent3); + + + // Queue A and B each request three containers + ApplicationAttemptId app1 = + createSchedulingRequest(1 * 1024, "queueA", "user1", 1, 1); + ApplicationAttemptId app2 = + createSchedulingRequest(1 * 1024, "queueA", "user1", 1, 2); + ApplicationAttemptId app3 = + createSchedulingRequest(1 * 1024, "queueA", "user1", 1, 3); + + ApplicationAttemptId app4 = + createSchedulingRequest(1 * 1024, "queueB", "user1", 1, 1); + ApplicationAttemptId app5 = + createSchedulingRequest(1 * 1024, "queueB", "user1", 1, 2); + ApplicationAttemptId app6 = + createSchedulingRequest(1 * 1024, "queueB", "user1", 1, 3); + + scheduler.update(); + + // Sufficient node check-ins to fully schedule containers + for (int i = 0; i < 2; i++) { + NodeUpdateSchedulerEvent nodeUpdate1 = new NodeUpdateSchedulerEvent(node1, + new LinkedList(), new LinkedList()); + scheduler.handle(nodeUpdate1); + + NodeUpdateSchedulerEvent nodeUpdate2 = new NodeUpdateSchedulerEvent(node2, + new LinkedList(), new LinkedList()); + scheduler.handle(nodeUpdate2); + + NodeUpdateSchedulerEvent nodeUpdate3 = new NodeUpdateSchedulerEvent(node3, + new LinkedList(), new LinkedList()); + scheduler.handle(nodeUpdate3); + } + + assertEquals(1, scheduler.applications.get(app1).getLiveContainers().size()); + assertEquals(1, scheduler.applications.get(app2).getLiveContainers().size()); + assertEquals(1, scheduler.applications.get(app3).getLiveContainers().size()); + assertEquals(1, scheduler.applications.get(app4).getLiveContainers().size()); + assertEquals(1, scheduler.applications.get(app5).getLiveContainers().size()); + assertEquals(1, scheduler.applications.get(app6).getLiveContainers().size()); + + // Now new requests arrive from queues C and D + ApplicationAttemptId app7 = + createSchedulingRequest(1 * 1024, "queueC", "user1", 1, 1); + ApplicationAttemptId app8 = + createSchedulingRequest(1 * 1024, "queueC", "user1", 1, 2); + ApplicationAttemptId app9 = + createSchedulingRequest(1 * 1024, "queueC", "user1", 1, 3); + + ApplicationAttemptId app10 = + createSchedulingRequest(1 * 1024, "queueD", "user1", 1, 1); + ApplicationAttemptId app11 = + createSchedulingRequest(1 * 1024, "queueD", "user1", 1, 2); + ApplicationAttemptId app12 = + createSchedulingRequest(1 * 1024, "queueD", "user1", 1, 3); + + scheduler.update(); + + // We should be able to claw back one container from A and B each. + // Make sure it is lowest priority container. + scheduler.preemptResources(scheduler.getQueueSchedulables(), + Resources.createResource(2 * 1024)); + assertEquals(1, scheduler.applications.get(app1).getLiveContainers().size()); + assertEquals(1, scheduler.applications.get(app2).getLiveContainers().size()); + assertEquals(0, scheduler.applications.get(app3).getLiveContainers().size()); + assertEquals(1, scheduler.applications.get(app4).getLiveContainers().size()); + assertEquals(1, scheduler.applications.get(app5).getLiveContainers().size()); + assertEquals(0, scheduler.applications.get(app6).getLiveContainers().size()); + + // We should be able to claw back another container from A and B each. + // Make sure it is lowest priority container. + scheduler.preemptResources(scheduler.getQueueSchedulables(), + Resources.createResource(2 * 1024)); + assertEquals(1, scheduler.applications.get(app1).getLiveContainers().size()); + assertEquals(0, scheduler.applications.get(app2).getLiveContainers().size()); + assertEquals(0, scheduler.applications.get(app3).getLiveContainers().size()); + assertEquals(1, scheduler.applications.get(app4).getLiveContainers().size()); + assertEquals(0, scheduler.applications.get(app5).getLiveContainers().size()); + assertEquals(0, scheduler.applications.get(app6).getLiveContainers().size()); + + // Now A and B are below fair share, so preemption shouldn't do anything + scheduler.preemptResources(scheduler.getQueueSchedulables(), + Resources.createResource(2 * 1024)); + assertEquals(1, scheduler.applications.get(app1).getLiveContainers().size()); + assertEquals(0, scheduler.applications.get(app2).getLiveContainers().size()); + assertEquals(0, scheduler.applications.get(app3).getLiveContainers().size()); + assertEquals(1, scheduler.applications.get(app4).getLiveContainers().size()); + assertEquals(0, scheduler.applications.get(app5).getLiveContainers().size()); + assertEquals(0, scheduler.applications.get(app6).getLiveContainers().size()); + } + + @Test + /** + * Tests the timing of decision to preempt tasks. + */ + public void testPreemptionDecision() throws Exception { + Configuration conf = new Configuration(); + conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); + MockClock clock = new MockClock(); + scheduler.setClock(clock); + scheduler.reinitialize(conf, null, resourceManager.getRMContext()); + + PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE)); + out.println(""); + out.println(""); + out.println(""); + out.println(".25"); + out.println("1024"); + out.println(""); + out.println(""); + out.println(".25"); + out.println("1024"); + out.println(""); + out.println(""); + out.println(".25"); + out.println("1024"); + out.println(""); + out.println(""); + out.println(".25"); + out.println("1024"); + out.println(""); + out.print("5"); + out.print("10"); + out.println(""); + out.close(); + + QueueManager queueManager = scheduler.getQueueManager(); + queueManager.initialize(); + + // Create four nodes + RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(2 * 1024)); + NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); + scheduler.handle(nodeEvent1); + + RMNode node2 = MockNodes.newNodeInfo(1, Resources.createResource(2 * 1024)); + NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2); + scheduler.handle(nodeEvent2); + + RMNode node3 = MockNodes.newNodeInfo(1, Resources.createResource(2 * 1024)); + NodeAddedSchedulerEvent nodeEvent3 = new NodeAddedSchedulerEvent(node3); + scheduler.handle(nodeEvent3); + + + // Queue A and B each request three containers + ApplicationAttemptId app1 = + createSchedulingRequest(1 * 1024, "queueA", "user1", 1, 1); + ApplicationAttemptId app2 = + createSchedulingRequest(1 * 1024, "queueA", "user1", 1, 2); + ApplicationAttemptId app3 = + createSchedulingRequest(1 * 1024, "queueA", "user1", 1, 3); + + ApplicationAttemptId app4 = + createSchedulingRequest(1 * 1024, "queueB", "user1", 1, 1); + ApplicationAttemptId app5 = + createSchedulingRequest(1 * 1024, "queueB", "user1", 1, 2); + ApplicationAttemptId app6 = + createSchedulingRequest(1 * 1024, "queueB", "user1", 1, 3); + + scheduler.update(); + + // Sufficient node check-ins to fully schedule containers + for (int i = 0; i < 2; i++) { + NodeUpdateSchedulerEvent nodeUpdate1 = new NodeUpdateSchedulerEvent(node1, + new LinkedList(), new LinkedList()); + scheduler.handle(nodeUpdate1); + + NodeUpdateSchedulerEvent nodeUpdate2 = new NodeUpdateSchedulerEvent(node2, + new LinkedList(), new LinkedList()); + scheduler.handle(nodeUpdate2); + + NodeUpdateSchedulerEvent nodeUpdate3 = new NodeUpdateSchedulerEvent(node3, + new LinkedList(), new LinkedList()); + scheduler.handle(nodeUpdate3); + } + + // Now new requests arrive from queues C and D + ApplicationAttemptId app7 = + createSchedulingRequest(1 * 1024, "queueC", "user1", 1, 1); + ApplicationAttemptId app8 = + createSchedulingRequest(1 * 1024, "queueC", "user1", 1, 2); + ApplicationAttemptId app9 = + createSchedulingRequest(1 * 1024, "queueC", "user1", 1, 3); + + ApplicationAttemptId app10 = + createSchedulingRequest(1 * 1024, "queueD", "user1", 1, 1); + ApplicationAttemptId app11 = + createSchedulingRequest(1 * 1024, "queueD", "user1", 1, 2); + ApplicationAttemptId app12 = + createSchedulingRequest(1 * 1024, "queueD", "user1", 1, 3); + + scheduler.update(); + + FSQueueSchedulable schedC = + scheduler.getQueueManager().getQueue("queueC").getQueueSchedulable(); + FSQueueSchedulable schedD = + scheduler.getQueueManager().getQueue("queueD").getQueueSchedulable(); + + assertTrue(Resources.equals( + Resources.none(), scheduler.resToPreempt(schedC, clock.getTime()))); + assertTrue(Resources.equals( + Resources.none(), scheduler.resToPreempt(schedD, clock.getTime()))); + // After minSharePreemptionTime has passed, they should want to preempt min + // share. + clock.tick(6); + assertTrue(Resources.equals( + Resources.createResource(1024), scheduler.resToPreempt(schedC, clock.getTime()))); + assertTrue(Resources.equals( + Resources.createResource(1024), scheduler.resToPreempt(schedD, clock.getTime()))); + + // After fairSharePreemptionTime has passed, they should want to preempt + // fair share. + scheduler.update(); + clock.tick(6); + assertTrue(Resources.equals( + Resources.createResource(1536), scheduler.resToPreempt(schedC, clock.getTime()))); + assertTrue(Resources.equals( + Resources.createResource(1536), scheduler.resToPreempt(schedD, clock.getTime()))); + } +} diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm new file mode 100644 index 00000000000..b5da6bbd343 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/FairScheduler.apt.vm @@ -0,0 +1,179 @@ +~~ Licensed under the Apache License, Version 2.0 (the "License"); +~~ you may not use this file except in compliance with the License. +~~ You may obtain a copy of the License at +~~ +~~ http://www.apache.org/licenses/LICENSE-2.0 +~~ +~~ Unless required by applicable law or agreed to in writing, software +~~ distributed under the License is distributed on an "AS IS" BASIS, +~~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +~~ See the License for the specific language governing permissions and +~~ limitations under the License. See accompanying LICENSE file. + + --- + Hadoop Map Reduce Next Generation-${project.version} - Fair Scheduler + --- + --- + ${maven.build.timestamp} + +Hadoop MapReduce Next Generation - Fair Scheduler + + \[ {{{./index.html}Go Back}} \] + +%{toc|section=1|fromDepth=0} + +* {Purpose} + + This document describes the <<>>, a pluggable scheduler for Hadoop + which provides a way to share large clusters. <> The Fair Scheduler + implementation is currently under development and should be considered experimental. + +* {Introduction} + + Fair scheduling is a method of assigning resources to applications such that + all apps get, on average, an equal share of resources over time. + Hadoop NextGen is capable of scheduling multiple resource types, such as + Memory and CPU. Currently only memory is supported, so a "cluster share" is + a proportion of aggregate memory in the cluster. When there is a single app + running, that app uses the entire cluster. When other apps are submitted, + resources that free up are assigned to the new apps, so that each app gets + roughly the same amount of resources. Unlike the default Hadoop scheduler, + which forms a queue of apps, this lets short apps finish in reasonable time + while not starving long-lived apps. It is also a reasonable way to share a + cluster between a number of users. Finally, fair sharing can also work with + app priorities - the priorities are used as weights to determine the + fraction of total resources that each app should get. + + The scheduler organizes apps further into "queues", and shares resources + fairly between these queues. By default, all users share a single queue, + called “default”. If an app specifically lists a queue in a container + resource request, the request is submitted to that queue. It is also + possible to assign queues based on the user name included with the request + through configuration. Within each queue, fair sharing is used to share + capacity between the running apps. queues can also be given weights to share + the cluster non-proportionally in the config file. + + In addition to providing fair sharing, the Fair Scheduler allows assigning + guaranteed minimum shares to queues, which is useful for ensuring that + certain users, groups or production applications always get sufficient + resources. When a queue contains apps, it gets at least its minimum share, + but when the queue does not need its full guaranteed share, the excess is + split between other running apps. This lets the scheduler guarantee capacity + for queues while utilizing resources efficiently when these queues don't + contain applications. + + The Fair Scheduler lets all apps run by default, but it is also possible to + limit the number of running apps per user and per queue through the config + file. This can be useful when a user must submit hundreds of apps at once, + or in general to improve performance if running too many apps at once would + cause too much intermediate data to be created or too much context-switching. + Limiting the apps does not cause any subsequently submitted apps to fail, + only to wait in the scheduler's queue until some of the user's earlier apps + finish. apps to run from each user/queue are chosen in order of priority and + then submit time, as in the default FIFO scheduler in Hadoop. + + Certain add-ons are not yet supported which existed in the original (MR1) + Fair Scheduler. Among them, is the use of a custom policies governing + priority “boosting” over certain apps. + +* {Installation} + + To use the Fair Scheduler first assign the appropriate scheduler class in + yarn-site.xml: + +------ + + yarn.resourcemanager.scheduler.class + org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler + +------ + +* {Configuration} + + Customizing the Fair Scheduler typically involves altering two files. First, + scheduler-wide options can be set by adding configuration properties in the + fair-scheduler.xml file in your existing configuration directory. Second, in + most cases users will want to create a manifest file listing which queues + exist and their respective weights and capacities. The location of this file + is flexible - but it must be declared in fair-scheduler.xml. + + * <<>> + + * Path to allocation file. An allocation file is an XML manifest describing + queues and their properties, in addition to certain policy defaults. This file + must be in XML format as described in the next section. + + * <<>> + + * The smallest container size the scheduler can allocate, in MB of memory. + + * <<>> + + * The largest container the scheduler can allocate, in MB of memory. + + * <<>> + + * Whether to use the username associated with the allocation as the default + queue name, in the event that a queue name is not specified. If this is set + to "false" or unset, all jobs have a shared default queue, called "default". + + * <<>> + + * Whether to use preemption. Note that preemption is experimental in the current + version. + + * <<>> + + * Whether to assign shares to individual apps based on their size, rather than + providing an equal share to all apps regardless of size. + + * <<>> + + * Whether to allow multiple container assignments in one heartbeat. + +Allocation file format + + The allocation file must be in XML format. The format contains three types of + elements: + + * <>, which represent queues. Each may contain the following + properties: + + * minResources: minimum amount of aggregate memory + + * maxResources: maximum amount of aggregate memory + + * maxRunningApps: limit the number of apps from the queue to run at once + + * weight: to share the cluster non-proportionally with other queues + + * schedulingMode: either "fifo" or "fair" depending on the in-queue scheduling + policy desired + + * <>, which represent settings governing the behavior of individual + users. They can contain a single property: maxRunningApps, a limit on the + number of running apps for a particular user. + + * <>, which sets the default running app limit + for any users whose limit is not otherwise specified. + + An example allocation file is given here: + +--- + + + + 100000 + 900000 + 50 + 2.0 + fair + + + 30 + + 5 + +--- + + Note that for backwards compatibility with the original FairScheduler, "queue" elements can instead be named as "pool" elements.