|
@@ -19,6 +19,7 @@ package org.apache.hadoop.ipc.metrics;
|
|
|
|
|
|
import java.util.concurrent.TimeUnit;
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
|
|
+import org.apache.commons.lang3.StringUtils;
|
|
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
|
|
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
|
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
import org.apache.hadoop.ipc.Server;
|
|
import org.apache.hadoop.ipc.Server;
|
|
@@ -48,9 +49,12 @@ public class RpcMetrics {
|
|
final MetricsRegistry registry;
|
|
final MetricsRegistry registry;
|
|
final String name;
|
|
final String name;
|
|
final boolean rpcQuantileEnable;
|
|
final boolean rpcQuantileEnable;
|
|
|
|
+
|
|
|
|
+ public static final TimeUnit DEFAULT_METRIC_TIME_UNIT =
|
|
|
|
+ TimeUnit.MILLISECONDS;
|
|
/** The time unit used when storing/accessing time durations. */
|
|
/** The time unit used when storing/accessing time durations. */
|
|
- public final static TimeUnit TIMEUNIT = TimeUnit.MILLISECONDS;
|
|
|
|
-
|
|
|
|
|
|
+ private final TimeUnit metricsTimeUnit;
|
|
|
|
+
|
|
RpcMetrics(Server server, Configuration conf) {
|
|
RpcMetrics(Server server, Configuration conf) {
|
|
String port = String.valueOf(server.getListenerAddress().getPort());
|
|
String port = String.valueOf(server.getListenerAddress().getPort());
|
|
name = "RpcActivityForPort" + port;
|
|
name = "RpcActivityForPort" + port;
|
|
@@ -63,6 +67,7 @@ public class RpcMetrics {
|
|
rpcQuantileEnable = (intervals.length > 0) && conf.getBoolean(
|
|
rpcQuantileEnable = (intervals.length > 0) && conf.getBoolean(
|
|
CommonConfigurationKeys.RPC_METRICS_QUANTILE_ENABLE,
|
|
CommonConfigurationKeys.RPC_METRICS_QUANTILE_ENABLE,
|
|
CommonConfigurationKeys.RPC_METRICS_QUANTILE_ENABLE_DEFAULT);
|
|
CommonConfigurationKeys.RPC_METRICS_QUANTILE_ENABLE_DEFAULT);
|
|
|
|
+ metricsTimeUnit = getMetricsTimeUnit(conf);
|
|
if (rpcQuantileEnable) {
|
|
if (rpcQuantileEnable) {
|
|
rpcQueueTimeQuantiles =
|
|
rpcQueueTimeQuantiles =
|
|
new MutableQuantiles[intervals.length];
|
|
new MutableQuantiles[intervals.length];
|
|
@@ -75,19 +80,19 @@ public class RpcMetrics {
|
|
for (int i = 0; i < intervals.length; i++) {
|
|
for (int i = 0; i < intervals.length; i++) {
|
|
int interval = intervals[i];
|
|
int interval = intervals[i];
|
|
rpcQueueTimeQuantiles[i] = registry.newQuantiles("rpcQueueTime"
|
|
rpcQueueTimeQuantiles[i] = registry.newQuantiles("rpcQueueTime"
|
|
- + interval + "s", "rpc queue time in " + TIMEUNIT, "ops",
|
|
|
|
|
|
+ + interval + "s", "rpc queue time in " + metricsTimeUnit, "ops",
|
|
"latency", interval);
|
|
"latency", interval);
|
|
rpcLockWaitTimeQuantiles[i] = registry.newQuantiles(
|
|
rpcLockWaitTimeQuantiles[i] = registry.newQuantiles(
|
|
"rpcLockWaitTime" + interval + "s",
|
|
"rpcLockWaitTime" + interval + "s",
|
|
- "rpc lock wait time in " + TIMEUNIT, "ops",
|
|
|
|
|
|
+ "rpc lock wait time in " + metricsTimeUnit, "ops",
|
|
"latency", interval);
|
|
"latency", interval);
|
|
rpcProcessingTimeQuantiles[i] = registry.newQuantiles(
|
|
rpcProcessingTimeQuantiles[i] = registry.newQuantiles(
|
|
"rpcProcessingTime" + interval + "s",
|
|
"rpcProcessingTime" + interval + "s",
|
|
- "rpc processing time in " + TIMEUNIT, "ops",
|
|
|
|
|
|
+ "rpc processing time in " + metricsTimeUnit, "ops",
|
|
"latency", interval);
|
|
"latency", interval);
|
|
deferredRpcProcessingTimeQuantiles[i] = registry.newQuantiles(
|
|
deferredRpcProcessingTimeQuantiles[i] = registry.newQuantiles(
|
|
"deferredRpcProcessingTime" + interval + "s",
|
|
"deferredRpcProcessingTime" + interval + "s",
|
|
- "deferred rpc processing time in " + TIMEUNIT, "ops",
|
|
|
|
|
|
+ "deferred rpc processing time in " + metricsTimeUnit, "ops",
|
|
"latency", interval);
|
|
"latency", interval);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -141,6 +146,27 @@ public class RpcMetrics {
|
|
return server.getNumDroppedConnections();
|
|
return server.getNumDroppedConnections();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public TimeUnit getMetricsTimeUnit() {
|
|
|
|
+ return metricsTimeUnit;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public static TimeUnit getMetricsTimeUnit(Configuration conf) {
|
|
|
|
+ TimeUnit metricsTimeUnit = RpcMetrics.DEFAULT_METRIC_TIME_UNIT;
|
|
|
|
+ String timeunit = conf.get(CommonConfigurationKeys.RPC_METRICS_TIME_UNIT);
|
|
|
|
+ if (StringUtils.isNotEmpty(timeunit)) {
|
|
|
|
+ try {
|
|
|
|
+ metricsTimeUnit = TimeUnit.valueOf(timeunit);
|
|
|
|
+ } catch (IllegalArgumentException e) {
|
|
|
|
+ LOG.info("Config key {} 's value {} does not correspond to enum values"
|
|
|
|
+ + " of java.util.concurrent.TimeUnit. Hence default unit"
|
|
|
|
+ + " {} will be used",
|
|
|
|
+ CommonConfigurationKeys.RPC_METRICS_TIME_UNIT, timeunit,
|
|
|
|
+ RpcMetrics.DEFAULT_METRIC_TIME_UNIT);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ return metricsTimeUnit;
|
|
|
|
+ }
|
|
|
|
+
|
|
// Public instrumentation methods that could be extracted to an
|
|
// Public instrumentation methods that could be extracted to an
|
|
// abstract class if we decide to do custom instrumentation classes a la
|
|
// abstract class if we decide to do custom instrumentation classes a la
|
|
// JobTrackerInstrumentation. The methods with //@Override comment are
|
|
// JobTrackerInstrumentation. The methods with //@Override comment are
|