Browse Source

MAPREDUCE-3238. Small cleanup in SchedulerApp. (Todd Lipcon via mahadev)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1206921 13f79535-47bb-0310-9956-ffa450edef68
Mahadev Konar 13 years ago
parent
commit
7f0f8cb7b0

+ 2 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -114,6 +114,8 @@ Release 0.23.1 - Unreleased
     MAPREDUCE-3371. Review and improve the yarn-api javadocs. (Ravi Prakash
     via mahadev)
 
+    MAPREDUCE-3238. Small cleanup in SchedulerApp. (Todd Lipcon via mahadev)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 29 - 32
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApp.java

@@ -53,6 +53,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerReservedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Multiset;
+
+/**
+ * Represents an Application from the viewpoint of the scheduler.
+ * Each running Application in the RM corresponds to one instance
+ * of this class.
+ */
 public class SchedulerApp {
 
   private static final Log LOG = LogFactory.getLog(SchedulerApp.class);
@@ -76,11 +84,16 @@ public class SchedulerApp {
   final Map<Priority, Map<NodeId, RMContainer>> reservedContainers = 
       new HashMap<Priority, Map<NodeId, RMContainer>>();
   
-  Map<Priority, Integer> schedulingOpportunities = 
-      new HashMap<Priority, Integer>();
+  /**
+   * Count how many times the application has been given an opportunity
+   * to schedule a task at each priority. Each time the scheduler
+   * asks the application for a task at this priority, it is incremented,
+   * and each time the application successfully schedules a task, it
+   * is reset to 0.
+   */
+  Multiset<Priority> schedulingOpportunities = HashMultiset.create();
   
-  Map<Priority, Integer> reReservations =
-      new HashMap<Priority, Integer>();
+  Multiset<Priority> reReservations = HashMultiset.create();
 
   Resource currentReservation = recordFactory
       .newRecordInstance(Resource.class);
@@ -282,49 +295,33 @@ public class SchedulerApp {
   }
 
   synchronized public void resetSchedulingOpportunities(Priority priority) {
-    this.schedulingOpportunities.put(priority, Integer.valueOf(0));
+    this.schedulingOpportunities.setCount(priority, 0);
   }
 
   synchronized public void addSchedulingOpportunity(Priority priority) {
-    Integer schedulingOpportunities = 
-        this.schedulingOpportunities.get(priority);
-    if (schedulingOpportunities == null) {
-      schedulingOpportunities = 0;
-    }
-    ++schedulingOpportunities;
-    this.schedulingOpportunities.put(priority, schedulingOpportunities);
+    this.schedulingOpportunities.setCount(priority,
+        schedulingOpportunities.count(priority) + 1);
   }
 
+  /**
+   * Return the number of times the application has been given an opportunity
+   * to schedule a task at the given priority since the last time it
+   * successfully did so.
+   */
   synchronized public int getSchedulingOpportunities(Priority priority) {
-    Integer schedulingOpportunities = 
-        this.schedulingOpportunities.get(priority);
-    if (schedulingOpportunities == null) {
-      schedulingOpportunities = 0;
-      this.schedulingOpportunities.put(priority, schedulingOpportunities);
-    }
-    return schedulingOpportunities;
+    return this.schedulingOpportunities.count(priority);
   }
 
   synchronized void resetReReservations(Priority priority) {
-    this.reReservations.put(priority, Integer.valueOf(0));
+    this.reReservations.setCount(priority, 0);
   }
 
   synchronized void addReReservation(Priority priority) {
-    Integer reReservations = this.reReservations.get(priority);
-    if (reReservations == null) {
-      reReservations = 0;
-    }
-    ++reReservations;
-    this.reReservations.put(priority, reReservations);
+    this.reReservations.add(priority);
   }
 
   synchronized public int getReReservations(Priority priority) {
-    Integer reReservations = this.reReservations.get(priority);
-    if (reReservations == null) {
-      reReservations = 0;
-      this.reReservations.put(priority, reReservations);
-    }
-    return reReservations;
+    return this.reReservations.count(priority);
   }
 
   public synchronized int getNumReservedContainers(Priority priority) {