소스 검색

Disabling locality-wait in CapacityScheduler for now to prevent uber-slow scheduling for apps with no data-locality constraints (sleep-job like). Contributed by Vinod Kumar Vavilapalli.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/MR-279@1137508 13f79535-47bb-0310-9956-ffa450edef68
Vinod Kumar Vavilapalli 14 년 전
부모
커밋
53f316818f

+ 3 - 0
mapreduce/CHANGES.txt

@@ -5,6 +5,9 @@ Trunk (unreleased changes)
 
 
     MAPREDUCE-279
     MAPREDUCE-279
 
 
+    Disabling locality-wait in CapacityScheduler for now to prevent uber-slow
+    scheduling for apps with no data-locality constraints (sleep-job like). (vinodkv)
+
     Fix to schedule reduces irrespective of the headroom when all maps are
     Fix to schedule reduces irrespective of the headroom when all maps are
     done so as to avoid stall in reduce-scheduling when slow-start is
     done so as to avoid stall in reduce-scheduling when slow-start is
     disabled. (Sharad Agarwal via vinodkv).
     disabled. (Sharad Agarwal via vinodkv).

+ 3 - 4
mapreduce/mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java

@@ -41,10 +41,11 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MapReduceChildJVM;
 import org.apache.hadoop.mapred.MapReduceChildJVM;
 import org.apache.hadoop.mapred.ShuffleHandler;
 import org.apache.hadoop.mapred.ShuffleHandler;
 import org.apache.hadoop.mapred.Task;
 import org.apache.hadoop.mapred.Task;
-import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TaskAttemptContextImpl;
 import org.apache.hadoop.mapred.WrappedJvmID;
 import org.apache.hadoop.mapred.WrappedJvmID;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.OutputCommitter;
 import org.apache.hadoop.mapreduce.OutputCommitter;
@@ -58,7 +59,6 @@ import org.apache.hadoop.mapreduce.jobhistory.TaskAttemptStartedEvent;
 import org.apache.hadoop.mapreduce.jobhistory.TaskAttemptUnsuccessfulCompletionEvent;
 import org.apache.hadoop.mapreduce.jobhistory.TaskAttemptUnsuccessfulCompletionEvent;
 import org.apache.hadoop.mapreduce.security.TokenCache;
 import org.apache.hadoop.mapreduce.security.TokenCache;
 import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
 import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
-import org.apache.hadoop.mapred.TaskAttemptContextImpl;
 import org.apache.hadoop.mapreduce.v2.MRConstants;
 import org.apache.hadoop.mapreduce.v2.MRConstants;
 import org.apache.hadoop.mapreduce.v2.api.records.Counters;
 import org.apache.hadoop.mapreduce.v2.api.records.Counters;
 import org.apache.hadoop.mapreduce.v2.api.records.Phase;
 import org.apache.hadoop.mapreduce.v2.api.records.Phase;
@@ -90,7 +90,6 @@ import org.apache.hadoop.mapreduce.v2.app.speculate.SpeculatorEvent;
 import org.apache.hadoop.mapreduce.v2.app.taskclean.TaskCleanupEvent;
 import org.apache.hadoop.mapreduce.v2.app.taskclean.TaskCleanupEvent;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
@@ -908,7 +907,7 @@ public abstract class TaskAttemptImpl implements
 
 
   private static class RequestContainerTransition implements
   private static class RequestContainerTransition implements
       SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
       SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
-    boolean rescheduled = false;
+    private final boolean rescheduled;
     public RequestContainerTransition(boolean rescheduled) {
     public RequestContainerTransition(boolean rescheduled) {
       this.rescheduled = rescheduled;
       this.rescheduled = rescheduled;
     }
     }

+ 3 - 3
mapreduce/mr-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/ContainerRequestEvent.java

@@ -24,9 +24,9 @@ import org.apache.hadoop.yarn.api.records.Resource;
 
 
 public class ContainerRequestEvent extends ContainerAllocatorEvent {
 public class ContainerRequestEvent extends ContainerAllocatorEvent {
   
   
-  private Resource capability;
-  private String[] hosts;
-  private String[] racks;
+  private final Resource capability;
+  private final String[] hosts;
+  private final String[] racks;
   private boolean earlierAttemptFailed = false;
   private boolean earlierAttemptFailed = false;
 
 
   public ContainerRequestEvent(TaskAttemptId attemptID, 
   public ContainerRequestEvent(TaskAttemptId attemptID, 

+ 8 - 8
mapreduce/yarn/yarn-server/yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java

@@ -810,14 +810,14 @@ public class LeafQueue implements Queue {
           return true;
           return true;
         }
         }
         
         
-        // Check if we have waited long enough
-        if (missedNodes < (requiredContainers * localityWaitFactor)) {
-          LOG.info("Application " + application.getApplicationId() + 
-              " has missed " + missedNodes + " opportunities," +
-              " waitFactor= " + localityWaitFactor + 
-              " for cluster of size " + scheduler.getNumClusterNodes());
-          return false;
-        }
+//        // Check if we have waited long enough
+//        if (missedNodes < (requiredContainers * localityWaitFactor)) {
+//          LOG.info("Application " + application.getApplicationId() + 
+//              " has missed " + missedNodes + " opportunities," +
+//              " waitFactor= " + localityWaitFactor + 
+//              " for cluster of size " + scheduler.getNumClusterNodes());
+//          return false;
+//        }
         return true;
         return true;
       }
       }
       return false;
       return false;