You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-commits@hadoop.apache.org by ac...@apache.org on 2011/05/13 03:56:13 UTC
svn commit: r1102536 - in
/hadoop/mapreduce/branches/MR-279/yarn/yarn-server/yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler:
Application.java capacity/LeafQueue.java
Author: acmurthy
Date: Fri May 13 01:56:13 2011
New Revision: 1102536
URL: http://svn.apache.org/viewvc?rev=1102536&view=rev
Log:
Better fix for reserved slots v/s delay scheduling.
Modified:
hadoop/mapreduce/branches/MR-279/yarn/yarn-server/yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Application.java
hadoop/mapreduce/branches/MR-279/yarn/yarn-server/yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
Modified: hadoop/mapreduce/branches/MR-279/yarn/yarn-server/yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Application.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/branches/MR-279/yarn/yarn-server/yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Application.java?rev=1102536&r1=1102535&r2=1102536&view=diff
==============================================================================
--- hadoop/mapreduce/branches/MR-279/yarn/yarn-server/yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Application.java (original)
+++ hadoop/mapreduce/branches/MR-279/yarn/yarn-server/yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Application.java Fri May 13 01:56:13 2011
@@ -403,11 +403,6 @@ public class Application {
return schedulingOpportunities;
}
- private static final int OVERRIDE = 1000000;
- synchronized public void overrideSchedulingOpportunities(Priority priority) {
- this.schedulingOpportunities.put(priority, OVERRIDE);
- }
-
synchronized public void showRequests() {
for (Priority priority : getPriorities()) {
Map<String, ResourceRequest> requests = getResourceRequests(priority);
Modified: hadoop/mapreduce/branches/MR-279/yarn/yarn-server/yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/branches/MR-279/yarn/yarn-server/yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java?rev=1102536&r1=1102535&r2=1102536&view=diff
==============================================================================
--- hadoop/mapreduce/branches/MR-279/yarn/yarn-server/yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java (original)
+++ hadoop/mapreduce/branches/MR-279/yarn/yarn-server/yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java Fri May 13 01:56:13 2011
@@ -538,7 +538,8 @@ public class LeafQueue implements Queue
// Try to schedule
Resource assigned =
- assignContainersOnNode(clusterResource, node, application, priority);
+ assignContainersOnNode(clusterResource, node, application, priority,
+ false);
// Did we schedule or reserve a container?
if (Resources.greaterThan(assigned, Resources.none())) {
@@ -577,9 +578,6 @@ public class LeafQueue implements Queue
// Do we reserve containers at this 'priority'?
if (application.isReserved(node, priority)) {
- // Do not care about locality
- application.overrideSchedulingOpportunities(priority);
-
// Do we really need this reservation still?
ResourceRequest offSwitchRequest =
application.getResourceRequest(priority, NodeManager.ANY);
@@ -590,7 +588,8 @@ public class LeafQueue implements Queue
}
// Try to assign if we have sufficient resources
- assignContainersOnNode(clusterResource, node, application, priority);
+ assignContainersOnNode(clusterResource, node, application, priority,
+ true);
}
}
}
@@ -691,7 +690,7 @@ public class LeafQueue implements Queue
}
Resource assignContainersOnNode(Resource clusterResource, NodeInfo node,
- Application application, Priority priority) {
+ Application application, Priority priority, boolean reserved) {
Resource assigned = Resources.none();
@@ -708,7 +707,8 @@ public class LeafQueue implements Queue
}
// Off-switch
- return assignOffSwitchContainers(clusterResource, node, application, priority);
+ return assignOffSwitchContainers(clusterResource, node, application,
+ priority, reserved);
}
Resource assignNodeLocalContainers(Resource clusterResource, NodeInfo node,
@@ -716,7 +716,7 @@ public class LeafQueue implements Queue
ResourceRequest request =
application.getResourceRequest(priority, node.getNodeAddress());
if (request != null) {
- if (canAssign(application, priority, node, NodeType.DATA_LOCAL)) {
+ if (canAssign(application, priority, node, NodeType.DATA_LOCAL, false)) {
return assignContainer(clusterResource, node, application, priority, request,
NodeType.DATA_LOCAL);
}
@@ -730,7 +730,7 @@ public class LeafQueue implements Queue
ResourceRequest request =
application.getResourceRequest(priority, node.getRackName());
if (request != null) {
- if (canAssign(application, priority, node, NodeType.RACK_LOCAL)) {
+ if (canAssign(application, priority, node, NodeType.RACK_LOCAL, false)) {
return assignContainer(clusterResource, node, application, priority, request,
NodeType.RACK_LOCAL);
}
@@ -739,11 +739,11 @@ public class LeafQueue implements Queue
}
Resource assignOffSwitchContainers(Resource clusterResource, NodeInfo node,
- Application application, Priority priority) {
+ Application application, Priority priority, boolean reserved) {
ResourceRequest request =
application.getResourceRequest(priority, NodeManager.ANY);
if (request != null) {
- if (canAssign(application, priority, node, NodeType.OFF_SWITCH)) {
+ if (canAssign(application, priority, node, NodeType.OFF_SWITCH, reserved)) {
return assignContainer(clusterResource, node, application, priority, request,
NodeType.OFF_SWITCH);
}
@@ -753,7 +753,7 @@ public class LeafQueue implements Queue
}
boolean canAssign(Application application, Priority priority,
- NodeInfo node, NodeType type) {
+ NodeInfo node, NodeType type, boolean reserved) {
ResourceRequest offSwitchRequest =
application.getResourceRequest(priority, NodeManager.ANY);
@@ -772,6 +772,12 @@ public class LeafQueue implements Queue
scheduler.getNumClusterNodes());
if (requiredContainers > 0) {
+ // No 'delay' for reserved containers
+ if (reserved) {
+ return true;
+ }
+
+ // Check if we have waited long enough
if (missedNodes < (requiredContainers * localityWaitFactor)) {
LOG.info("Application " + application.getApplicationId() +
" has missed " + missedNodes + " opportunities," +