HDFS-13854. RBF: The ProcessingAvgTime and ProxyAvgTime should display by JMX with ms unit. Contributed by yanghuafeng.

This commit is contained in:
Brahma Reddy Battula 2018-08-29 08:29:50 +05:30
parent 2651e2c43d
commit 64ad0298d4
2 changed files with 7 additions and 21 deletions

View File

@ -85,15 +85,6 @@ public class FederationRPCMetrics implements FederationRPCMBean {
new FederationRPCMetrics(conf, rpcServer)); new FederationRPCMetrics(conf, rpcServer));
} }
/**
* Convert nanoseconds to milliseconds.
* @param ns Time in nanoseconds.
* @return Time in milliseconds.
*/
private static double toMs(double ns) {
return ns / 1000000;
}
/** /**
* Reset the metrics system. * Reset the metrics system.
*/ */
@ -230,7 +221,7 @@ public class FederationRPCMetrics implements FederationRPCMBean {
@Override @Override
public double getProxyAvg() { public double getProxyAvg() {
return toMs(proxy.lastStat().mean()); return proxy.lastStat().mean();
} }
@Override @Override
@ -250,7 +241,7 @@ public class FederationRPCMetrics implements FederationRPCMBean {
@Override @Override
public double getProcessingAvg() { public double getProcessingAvg() {
return toMs(processing.lastStat().mean()); return processing.lastStat().mean();
} }
@Override @Override

View File

@ -35,6 +35,8 @@ import org.slf4j.LoggerFactory;
import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.common.util.concurrent.ThreadFactoryBuilder;
import static org.apache.hadoop.util.Time.monotonicNow;
/** /**
* Customizable RPC performance monitor. Receives events from the RPC server * Customizable RPC performance monitor. Receives events from the RPC server
* and aggregates them via JMX. * and aggregates them via JMX.
@ -120,12 +122,12 @@ public class FederationRPCPerformanceMonitor implements RouterRpcMonitor {
@Override @Override
public void startOp() { public void startOp() {
START_TIME.set(this.getNow()); START_TIME.set(monotonicNow());
} }
@Override @Override
public long proxyOp() { public long proxyOp() {
PROXY_TIME.set(this.getNow()); PROXY_TIME.set(monotonicNow());
long processingTime = getProcessingTime(); long processingTime = getProcessingTime();
if (processingTime >= 0) { if (processingTime >= 0) {
metrics.addProcessingTime(processingTime); metrics.addProcessingTime(processingTime);
@ -188,13 +190,6 @@ public class FederationRPCPerformanceMonitor implements RouterRpcMonitor {
metrics.incrRouterFailureLocked(); metrics.incrRouterFailureLocked();
} }
/**
* Get current time.
* @return Current time in nanoseconds.
*/
private long getNow() {
return System.nanoTime();
}
/** /**
* Get time between we receiving the operation and sending it to the Namenode. * Get time between we receiving the operation and sending it to the Namenode.
@ -214,7 +209,7 @@ public class FederationRPCPerformanceMonitor implements RouterRpcMonitor {
*/ */
private long getProxyTime() { private long getProxyTime() {
if (PROXY_TIME.get() != null && PROXY_TIME.get() > 0) { if (PROXY_TIME.get() != null && PROXY_TIME.get() > 0) {
return getNow() - PROXY_TIME.get(); return monotonicNow() - PROXY_TIME.get();
} }
return -1; return -1;
} }