|
@@ -21,8 +21,12 @@ package org.apache.hadoop.yarn.server.resourcemanager;
|
|
|
import static org.apache.hadoop.metrics2.lib.Interns.info;
|
|
|
|
|
|
import java.util.Map;
|
|
|
+import java.util.concurrent.ScheduledThreadPoolExecutor;
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
+import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
|
|
+import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.metrics2.MetricsInfo;
|
|
|
import org.apache.hadoop.metrics2.MetricsSystem;
|
|
@@ -66,6 +70,8 @@ public class ClusterMetrics {
|
|
|
rmEventProcCPUAvg;
|
|
|
@Metric("RM Event Processor CPU Usage 60 second Max") MutableGaugeLong
|
|
|
rmEventProcCPUMax;
|
|
|
+ @Metric("# of Containers assigned in the last second") MutableGaugeInt
|
|
|
+ containerAssignedPerSecond;
|
|
|
|
|
|
private boolean rmEventProcMonitorEnable = false;
|
|
|
|
|
@@ -85,6 +91,22 @@ public class ClusterMetrics {
|
|
|
private static volatile ClusterMetrics INSTANCE = null;
|
|
|
private static MetricsRegistry registry;
|
|
|
|
|
|
+ private AtomicInteger numContainersAssigned = new AtomicInteger(0);
|
|
|
+ private ScheduledThreadPoolExecutor assignCounterExecutor;
|
|
|
+
|
|
|
+ ClusterMetrics() {
|
|
|
+ assignCounterExecutor = new ScheduledThreadPoolExecutor(1,
|
|
|
+ new ThreadFactoryBuilder().
|
|
|
+ setDaemon(true).setNameFormat("ContainerAssignmentCounterThread").
|
|
|
+ build());
|
|
|
+ assignCounterExecutor.scheduleAtFixedRate(new Runnable() {
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ containerAssignedPerSecond.set(numContainersAssigned.getAndSet(0));
|
|
|
+ }
|
|
|
+ }, 1, 1, TimeUnit.SECONDS);
|
|
|
+ }
|
|
|
+
|
|
|
public static ClusterMetrics getMetrics() {
|
|
|
if(!isInitialized.get()){
|
|
|
synchronized (ClusterMetrics.class) {
|
|
@@ -120,6 +142,9 @@ public class ClusterMetrics {
|
|
|
|
|
|
@VisibleForTesting
|
|
|
public synchronized static void destroy() {
|
|
|
+ if (INSTANCE != null && INSTANCE.getAssignCounterExecutor() != null) {
|
|
|
+ INSTANCE.getAssignCounterExecutor().shutdownNow();
|
|
|
+ }
|
|
|
isInitialized.set(false);
|
|
|
INSTANCE = null;
|
|
|
}
|
|
@@ -319,4 +344,16 @@ public class ClusterMetrics {
|
|
|
public void incrUtilizedVirtualCores(long delta) {
|
|
|
utilizedVirtualCores.incr(delta);
|
|
|
}
|
|
|
+
|
|
|
+ public int getContainerAssignedPerSecond() {
|
|
|
+ return containerAssignedPerSecond.value();
|
|
|
+ }
|
|
|
+
|
|
|
+ public void incrNumContainerAssigned() {
|
|
|
+ numContainersAssigned.incrementAndGet();
|
|
|
+ }
|
|
|
+
|
|
|
+ private ScheduledThreadPoolExecutor getAssignCounterExecutor(){
|
|
|
+ return assignCounterExecutor;
|
|
|
+ }
|
|
|
}
|