Browse Source

YARN-2868. FairScheduler: Metric for latency to allocate first container for an application. (Ray Chiang via kasha)

Karthik Kambatla 10 years ago
parent
commit
972f1f1ab9

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -73,6 +73,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3350. YARN RackResolver spams logs with messages at info level. 
     (Wilfred Spiegelenburg via junping_du)
 
+    YARN-2868. FairScheduler: Metric for latency to allocate first container 
+    for an application. (Ray Chiang via kasha)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

+ 7 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.metrics2.lib.MetricsRegistry;
 import org.apache.hadoop.metrics2.lib.MutableCounterInt;
 import org.apache.hadoop.metrics2.lib.MutableCounterLong;
 import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
+import org.apache.hadoop.metrics2.lib.MutableRate;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -74,6 +75,7 @@ public class QueueMetrics implements MetricsSource {
   @Metric("# of reserved containers") MutableGaugeInt reservedContainers;
   @Metric("# of active users") MutableGaugeInt activeUsers;
   @Metric("# of active applications") MutableGaugeInt activeApplications;
+  @Metric("App Attempt First Container Allocation Delay") MutableRate appAttemptFirstContainerAllocationDelay;
   private final MutableGaugeInt[] runningTime;
   private TimeBucketMetrics<ApplicationId> runBuckets;
 
@@ -462,7 +464,11 @@ public class QueueMetrics implements MetricsSource {
       parent.deactivateApp(user);
     }
   }
-  
+
+  public void addAppAttemptFirstContainerAllocationDelay(long latency) {
+    appAttemptFirstContainerAllocationDelay.add(latency);
+  }
+
   public int getAppsSubmitted() {
     return appsSubmitted.value();
   }

+ 17 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java

@@ -25,6 +25,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.lang.time.DateUtils;
 import org.apache.commons.logging.Log;
@@ -93,6 +94,8 @@ public class SchedulerApplicationAttempt {
   private LogAggregationContext logAggregationContext;
   
   protected ResourceUsage attemptResourceUsage = new ResourceUsage();
+  private AtomicLong firstAllocationRequestSentTime = new AtomicLong(0);
+  private AtomicLong firstContainerAllocatedTime = new AtomicLong(0);
 
   protected List<RMContainer> newlyAllocatedContainers = 
       new ArrayList<RMContainer>();
@@ -648,4 +651,18 @@ public class SchedulerApplicationAttempt {
           Resources.clone(headroom));
     }
   }
+
+  public void recordContainerRequestTime(long value) {
+    firstAllocationRequestSentTime.compareAndSet(0, value);
+  }
+
+  public void recordContainerAllocationTime(long value) {
+    if (firstContainerAllocatedTime.compareAndSet(0, value)) {
+      long timediff = firstContainerAllocatedTime.longValue() -
+          firstAllocationRequestSentTime.longValue();
+      if (timediff > 0) {
+        queue.getMetrics().addAppAttemptFirstContainerAllocationDelay(timediff);
+      }
+    }
+  }
 }

+ 10 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java

@@ -898,6 +898,9 @@ public class FairScheduler extends
         clusterResource, minimumAllocation, getMaximumResourceCapability(),
         incrAllocation);
 
+    // Record container allocation start time
+    application.recordContainerRequestTime(getClock().getTime());
+
     // Set amResource for this app
     if (!application.getUnmanagedAM() && ask.size() == 1
         && application.getLiveContainers().isEmpty()) {
@@ -931,7 +934,7 @@ public class FairScheduler extends
         LOG.debug("Preempting " + application.getPreemptionContainers().size()
             + " container(s)");
       }
-      
+
       Set<ContainerId> preemptionContainerIds = new HashSet<ContainerId>();
       for (RMContainer container : application.getPreemptionContainers()) {
         preemptionContainerIds.add(container.getContainerId());
@@ -940,6 +943,12 @@ public class FairScheduler extends
       application.updateBlacklist(blacklistAdditions, blacklistRemovals);
       ContainersAndNMTokensAllocation allocation =
           application.pullNewlyAllocatedContainersAndNMTokens();
+
+      // Record container allocation time
+      if (!(allocation.getContainerList().isEmpty())) {
+        application.recordContainerAllocationTime(getClock().getTime());
+      }
+
       Resource headroom = application.getHeadroom();
       application.setApplicationHeadroomForMetrics(headroom);
       return new Allocation(allocation.getContainerList(), headroom,