瀏覽代碼

Revert "YARN-7728. Expose container preemptions related information in Capacity Scheduler queue metrics. Contributed by Eric Payne."

This reverts commit 3fa2e8658740feb310c75729c34bdf0c9942e2df.
Jason Lowe 7 年之前
父節點
當前提交
c328ab47fa

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

@@ -68,10 +68,6 @@ public class QueueMetrics implements MetricsSource {
     MutableCounterLong aggregateOffSwitchContainersAllocated;
   @Metric("Aggregate # of preempted containers") MutableCounterLong
       aggregateContainersPreempted;
-  @Metric("Aggregate # of preempted memory seconds") MutableCounterLong
-      aggregateMemoryMBSecondsPreempted;
-  @Metric("Aggregate # of preempted vcore seconds") MutableCounterLong
-      aggregateVcoreSecondsPreempted;
   @Metric("# of active users") MutableGaugeInt activeUsers;
   @Metric("# of active applications") MutableGaugeInt activeApplications;
   @Metric("App Attempt First Container Allocation Delay")
@@ -525,20 +521,6 @@ public class QueueMetrics implements MetricsSource {
     }
   }
 
-  public void updatePreemptedMemoryMBSeconds(long mbSeconds) {
-    aggregateMemoryMBSecondsPreempted.incr(mbSeconds);
-    if (parent != null) {
-      parent.updatePreemptedMemoryMBSeconds(mbSeconds);
-    }
-  }
-
-  public void updatePreemptedVcoreSeconds(long vcoreSeconds) {
-    aggregateVcoreSecondsPreempted.incr(vcoreSeconds);
-    if (parent != null) {
-      parent.updatePreemptedVcoreSeconds(vcoreSeconds);
-    }
-  }
-
   public void reserveResource(String partition, String user, Resource res) {
     if(partition == null || partition.equals(RMNodeLabelsManager.NO_LABEL)) {
       reserveResource(user, res);

+ 0 - 19
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java

@@ -21,7 +21,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.commons.lang.StringUtils;
-import org.apache.commons.lang.time.DateUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
@@ -36,7 +35,6 @@ import org.apache.hadoop.util.Time;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
@@ -1611,23 +1609,6 @@ public class CapacityScheduler extends
     LeafQueue queue = (LeafQueue)application.getQueue();
     queue.completedContainer(clusterResource, application, node, 
         rmContainer, containerStatus, event, null, true);
-    if (ContainerExitStatus.PREEMPTED == containerStatus.getExitStatus()) {
-      updateQueuePreemptionMetrics(queue, rmContainer);
-    }
-  }
-
-  private void updateQueuePreemptionMetrics(
-      CSQueue queue, RMContainer rmc) {
-    QueueMetrics qMetrics = queue.getMetrics();
-    long usedMillis = rmc.getFinishTime() - rmc.getCreationTime();
-    Resource containerResource = rmc.getAllocatedResource();
-    qMetrics.preemptContainer();
-    long mbSeconds = (containerResource.getMemorySize() * usedMillis)
-        / DateUtils.MILLIS_PER_SECOND;
-    long vcSeconds = (containerResource.getVirtualCores() * usedMillis)
-        / DateUtils.MILLIS_PER_SECOND;
-    qMetrics.updatePreemptedMemoryMBSeconds(mbSeconds);
-    qMetrics.updatePreemptedVcoreSeconds(vcSeconds);
   }
   
   @Override

+ 0 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerSurgicalPreemption.java

@@ -150,10 +150,6 @@ public class TestCapacitySchedulerSurgicalPreemption
     waitNumberOfLiveContainersOnNodeFromApp(cs.getNode(rmNode2.getNodeID()),
         am1.getApplicationAttemptId(), 16);
 
-    // Ensure preemption metrics were recored.
-    Assert.assertEquals("Number of preempted containers incorrectly recorded:",
-        4, cs.getQueue("root").getMetrics().getAggregatePreemptedContainers());
-
     rm1.close();
   }