|
@@ -32,11 +32,19 @@ import java.util.concurrent.atomic.AtomicLong;
|
|
|
import java.util.concurrent.atomic.AtomicLongArray;
|
|
|
import java.util.concurrent.atomic.AtomicReference;
|
|
|
|
|
|
+import com.google.common.base.Preconditions;
|
|
|
import com.google.common.util.concurrent.AtomicDoubleArray;
|
|
|
import org.apache.commons.lang.exception.ExceptionUtils;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
|
+import org.apache.hadoop.metrics2.MetricsCollector;
|
|
|
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
|
|
+import org.apache.hadoop.metrics2.MetricsSource;
|
|
|
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
|
|
+import org.apache.hadoop.metrics2.lib.Interns;
|
|
|
import org.apache.hadoop.metrics2.util.MBeans;
|
|
|
+import org.apache.hadoop.metrics2.util.Metrics2Util.NameValuePair;
|
|
|
+import org.apache.hadoop.metrics2.util.Metrics2Util.TopN;
|
|
|
|
|
|
import org.codehaus.jackson.map.ObjectMapper;
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
@@ -49,7 +57,8 @@ import org.slf4j.LoggerFactory;
|
|
|
* for large periods (on the order of seconds), as it offloads work to the
|
|
|
* decay sweep.
|
|
|
*/
|
|
|
-public class DecayRpcScheduler implements RpcScheduler, DecayRpcSchedulerMXBean {
|
|
|
+public class DecayRpcScheduler implements RpcScheduler,
|
|
|
+ DecayRpcSchedulerMXBean, MetricsSource {
|
|
|
/**
|
|
|
* Period controls how many milliseconds between each decay sweep.
|
|
|
*/
|
|
@@ -107,6 +116,12 @@ public class DecayRpcScheduler implements RpcScheduler, DecayRpcSchedulerMXBean
|
|
|
IPC_DECAYSCHEDULER_BACKOFF_RESPONSETIME_THRESHOLDS_KEY =
|
|
|
"decay-scheduler.backoff.responsetime.thresholds";
|
|
|
|
|
|
+ // Specifies the top N user's call count and scheduler decision
|
|
|
+ // Metrics2 Source
|
|
|
+ public static final String DECAYSCHEDULER_METRICS_TOP_USER_COUNT =
|
|
|
+ "decay-scheduler.metrics.top.user.count";
|
|
|
+ public static final int DECAYSCHEDULER_METRICS_TOP_USER_COUNT_DEFAULT = 10;
|
|
|
+
|
|
|
public static final Logger LOG =
|
|
|
LoggerFactory.getLogger(DecayRpcScheduler.class);
|
|
|
|
|
@@ -138,6 +153,8 @@ public class DecayRpcScheduler implements RpcScheduler, DecayRpcSchedulerMXBean
|
|
|
private final IdentityProvider identityProvider;
|
|
|
private final boolean backOffByResponseTimeEnabled;
|
|
|
private final long[] backOffResponseTimeThresholds;
|
|
|
+ private final String namespace;
|
|
|
+ private final int topUsersCount; // e.g., report top 10 users' metrics
|
|
|
|
|
|
/**
|
|
|
* This TimerTask will call decayCurrentCounts until
|
|
@@ -179,6 +196,7 @@ public class DecayRpcScheduler implements RpcScheduler, DecayRpcSchedulerMXBean
|
|
|
"at least 1");
|
|
|
}
|
|
|
this.numLevels = numLevels;
|
|
|
+ this.namespace = ns;
|
|
|
this.decayFactor = parseDecayFactor(ns, conf);
|
|
|
this.decayPeriodMillis = parseDecayPeriodMillis(ns, conf);
|
|
|
this.identityProvider = this.parseIdentityProvider(ns, conf);
|
|
@@ -199,8 +217,15 @@ public class DecayRpcScheduler implements RpcScheduler, DecayRpcSchedulerMXBean
|
|
|
responseTimeAvgInLastWindow = new AtomicDoubleArray(numLevels);
|
|
|
responseTimeCountInLastWindow = new AtomicLongArray(numLevels);
|
|
|
|
|
|
+ topUsersCount =
|
|
|
+ conf.getInt(DECAYSCHEDULER_METRICS_TOP_USER_COUNT,
|
|
|
+ DECAYSCHEDULER_METRICS_TOP_USER_COUNT_DEFAULT);
|
|
|
+ Preconditions.checkArgument(topUsersCount > 0,
|
|
|
+ "the number of top users for scheduler metrics must be at least 1");
|
|
|
+
|
|
|
MetricsProxy prox = MetricsProxy.getInstance(ns, numLevels);
|
|
|
prox.setDelegate(this);
|
|
|
+ prox.registerMetrics2Source(ns);
|
|
|
}
|
|
|
|
|
|
// Load configs
|
|
@@ -615,7 +640,8 @@ public class DecayRpcScheduler implements RpcScheduler, DecayRpcSchedulerMXBean
|
|
|
* MetricsProxy is a singleton because we may init multiple schedulers and we
|
|
|
* want to clean up resources when a new scheduler replaces the old one.
|
|
|
*/
|
|
|
- private static final class MetricsProxy implements DecayRpcSchedulerMXBean {
|
|
|
+ public static final class MetricsProxy implements DecayRpcSchedulerMXBean,
|
|
|
+ MetricsSource {
|
|
|
// One singleton per namespace
|
|
|
private static final HashMap<String, MetricsProxy> INSTANCES =
|
|
|
new HashMap<String, MetricsProxy>();
|
|
@@ -646,6 +672,11 @@ public class DecayRpcScheduler implements RpcScheduler, DecayRpcSchedulerMXBean
|
|
|
this.delegate = new WeakReference<DecayRpcScheduler>(obj);
|
|
|
}
|
|
|
|
|
|
+ void registerMetrics2Source(String namespace) {
|
|
|
+ final String name = "DecayRpcSchedulerMetrics2." + namespace;
|
|
|
+ DefaultMetricsSystem.instance().register(name, name, this);
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public String getSchedulingDecisionSummary() {
|
|
|
DecayRpcScheduler scheduler = delegate.get();
|
|
@@ -704,6 +735,14 @@ public class DecayRpcScheduler implements RpcScheduler, DecayRpcSchedulerMXBean
|
|
|
return scheduler.getResponseTimeCountInLastWindow();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void getMetrics(MetricsCollector collector, boolean all) {
|
|
|
+ DecayRpcScheduler scheduler = delegate.get();
|
|
|
+ if (scheduler != null) {
|
|
|
+ scheduler.getMetrics(collector, all);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
public int getUniqueIdentityCount() {
|
|
@@ -731,6 +770,89 @@ public class DecayRpcScheduler implements RpcScheduler, DecayRpcSchedulerMXBean
|
|
|
return ret;
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ public void getMetrics(MetricsCollector collector, boolean all) {
|
|
|
+ // Metrics2 interface to act as a Metric source
|
|
|
+ try {
|
|
|
+ MetricsRecordBuilder rb = collector.addRecord(getClass().getName())
|
|
|
+ .setContext(namespace);
|
|
|
+ addTotalCallVolume(rb);
|
|
|
+ addUniqueIdentityCount(rb);
|
|
|
+ addTopNCallerSummary(rb);
|
|
|
+ addAvgResponseTimePerPriority(rb);
|
|
|
+ addCallVolumePerPriority(rb);
|
|
|
+ } catch (Exception e) {
|
|
|
+ LOG.warn("Exception thrown while metric collection. Exception : "
|
|
|
+ + e.getMessage());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // Key: UniqueCallers
|
|
|
+ private void addUniqueIdentityCount(MetricsRecordBuilder rb) {
|
|
|
+ rb.addCounter(Interns.info("UniqueCallers", "Total unique callers"),
|
|
|
+ getUniqueIdentityCount());
|
|
|
+ }
|
|
|
+
|
|
|
+ // Key: CallVolume
|
|
|
+ private void addTotalCallVolume(MetricsRecordBuilder rb) {
|
|
|
+ rb.addCounter(Interns.info("CallVolume", "Total Call Volume"),
|
|
|
+ getTotalCallVolume());
|
|
|
+ }
|
|
|
+
|
|
|
+ // Key: Priority.0.CallVolume
|
|
|
+ private void addCallVolumePerPriority(MetricsRecordBuilder rb) {
|
|
|
+ for (int i = 0; i < responseTimeCountInLastWindow.length(); i++) {
|
|
|
+ rb.addGauge(Interns.info("Priority." + i + ".CallVolume", "Call volume " +
|
|
|
+ "of priority "+ i), responseTimeCountInLastWindow.get(i));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // Key: Priority.0.AvgResponseTime
|
|
|
+ private void addAvgResponseTimePerPriority(MetricsRecordBuilder rb) {
|
|
|
+ for (int i = 0; i < responseTimeAvgInLastWindow.length(); i++) {
|
|
|
+ rb.addGauge(Interns.info("Priority." + i + ".AvgResponseTime", "Average" +
|
|
|
+ " response time of priority " + i),
|
|
|
+ responseTimeAvgInLastWindow.get(i));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // Key: Top.0.Caller(xyz).Volume and Top.0.Caller(xyz).Priority
|
|
|
+ private void addTopNCallerSummary(MetricsRecordBuilder rb) {
|
|
|
+ final int topCallerCount = 10;
|
|
|
+ TopN topNCallers = getTopCallers(topCallerCount);
|
|
|
+ Map<Object, Integer> decisions = scheduleCacheRef.get();
|
|
|
+ for (int i=0; i < topNCallers.size(); i++) {
|
|
|
+ NameValuePair entry = topNCallers.poll();
|
|
|
+ String topCaller = "Top." + (topCallerCount - i) + "." +
|
|
|
+ "Caller(" + entry.getName() + ")";
|
|
|
+ String topCallerVolume = topCaller + ".Volume";
|
|
|
+ String topCallerPriority = topCaller + ".Priority";
|
|
|
+ rb.addCounter(Interns.info(topCallerVolume, topCallerVolume),
|
|
|
+ entry.getValue());
|
|
|
+ Integer priority = decisions.get(entry.getName());
|
|
|
+ if (priority != null) {
|
|
|
+ rb.addCounter(Interns.info(topCallerPriority, topCallerPriority),
|
|
|
+ priority);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // Get the top N callers' call count and scheduler decision
|
|
|
+ private TopN getTopCallers(int n) {
|
|
|
+ TopN topNCallers = new TopN(n);
|
|
|
+ Iterator<Map.Entry<Object, AtomicLong>> it =
|
|
|
+ callCounts.entrySet().iterator();
|
|
|
+ while (it.hasNext()) {
|
|
|
+ Map.Entry<Object, AtomicLong> entry = it.next();
|
|
|
+ String caller = entry.getKey().toString();
|
|
|
+ Long count = entry.getValue().get();
|
|
|
+ if (count > 0) {
|
|
|
+ topNCallers.offer(new NameValuePair(caller, count));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return topNCallers;
|
|
|
+ }
|
|
|
+
|
|
|
public String getSchedulingDecisionSummary() {
|
|
|
Map<Object, Integer> decisions = scheduleCacheRef.get();
|
|
|
if (decisions == null) {
|