|
@@ -29,7 +29,6 @@ import org.apache.hadoop.io.retry.RetryProxy;
|
|
|
import org.apache.hadoop.ipc.Client.ConnectionId;
|
|
|
import org.apache.hadoop.ipc.Server.Call;
|
|
|
import org.apache.hadoop.ipc.Server.Connection;
|
|
|
-import org.apache.hadoop.ipc.metrics.RpcMetrics;
|
|
|
import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcErrorCodeProto;
|
|
|
import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcStatusProto;
|
|
|
import org.apache.hadoop.ipc.protobuf.TestProtos;
|
|
@@ -1098,8 +1097,8 @@ public class TestRPC extends TestRpcBase {
|
|
|
proxy.lockAndSleep(null, newSleepRequest(5));
|
|
|
rpcMetrics = getMetrics(server.getRpcMetrics().name());
|
|
|
assertGauge("RpcLockWaitTimeAvgTime",
|
|
|
- (double)(RpcMetrics.TIMEUNIT.convert(10L, TimeUnit.SECONDS)),
|
|
|
- rpcMetrics);
|
|
|
+ (double)(server.getRpcMetrics().getMetricsTimeUnit().convert(10L,
|
|
|
+ TimeUnit.SECONDS)), rpcMetrics);
|
|
|
} finally {
|
|
|
if (proxy2 != null) {
|
|
|
RPC.stopProxy(proxy2);
|
|
@@ -1603,6 +1602,70 @@ public class TestRPC extends TestRpcBase {
|
|
|
assertTrue(rpcEngine instanceof StoppedRpcEngine);
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testRpcMetricsInNanos() throws Exception {
|
|
|
+ final Server server;
|
|
|
+ TestRpcService proxy = null;
|
|
|
+
|
|
|
+ final int interval = 1;
|
|
|
+ conf.setBoolean(CommonConfigurationKeys.
|
|
|
+ RPC_METRICS_QUANTILE_ENABLE, true);
|
|
|
+ conf.set(CommonConfigurationKeys.
|
|
|
+ RPC_METRICS_PERCENTILES_INTERVALS_KEY, "" + interval);
|
|
|
+ conf.set(CommonConfigurationKeys.RPC_METRICS_TIME_UNIT, "NANOSECONDS");
|
|
|
+
|
|
|
+ server = setupTestServer(conf, 5);
|
|
|
+ String testUser = "testUserInNanos";
|
|
|
+ UserGroupInformation anotherUser =
|
|
|
+ UserGroupInformation.createRemoteUser(testUser);
|
|
|
+ TestRpcService proxy2 =
|
|
|
+ anotherUser.doAs((PrivilegedAction<TestRpcService>) () -> {
|
|
|
+ try {
|
|
|
+ return RPC.getProxy(TestRpcService.class, 0,
|
|
|
+ server.getListenerAddress(), conf);
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.error("Something went wrong.", e);
|
|
|
+ }
|
|
|
+ return null;
|
|
|
+ });
|
|
|
+ try {
|
|
|
+ proxy = getClient(addr, conf);
|
|
|
+ for (int i = 0; i < 100; i++) {
|
|
|
+ proxy.ping(null, newEmptyRequest());
|
|
|
+ proxy.echo(null, newEchoRequest("" + i));
|
|
|
+ proxy2.echo(null, newEchoRequest("" + i));
|
|
|
+ }
|
|
|
+ MetricsRecordBuilder rpcMetrics =
|
|
|
+ getMetrics(server.getRpcMetrics().name());
|
|
|
+ assertEquals("Expected zero rpc lock wait time",
|
|
|
+ 0, getDoubleGauge("RpcLockWaitTimeAvgTime", rpcMetrics), 0.001);
|
|
|
+ MetricsAsserts.assertQuantileGauges("RpcQueueTime" + interval + "s",
|
|
|
+ rpcMetrics);
|
|
|
+ MetricsAsserts.assertQuantileGauges("RpcProcessingTime" + interval + "s",
|
|
|
+ rpcMetrics);
|
|
|
+
|
|
|
+ proxy.lockAndSleep(null, newSleepRequest(5));
|
|
|
+ rpcMetrics = getMetrics(server.getRpcMetrics().name());
|
|
|
+ assertGauge("RpcLockWaitTimeAvgTime",
|
|
|
+ (double)(server.getRpcMetrics().getMetricsTimeUnit().convert(10L,
|
|
|
+ TimeUnit.SECONDS)), rpcMetrics);
|
|
|
+ LOG.info("RpcProcessingTimeAvgTime: {} , RpcQueueTimeAvgTime: {}",
|
|
|
+ getDoubleGauge("RpcProcessingTimeAvgTime", rpcMetrics),
|
|
|
+ getDoubleGauge("RpcQueueTimeAvgTime", rpcMetrics));
|
|
|
+
|
|
|
+ assertTrue(getDoubleGauge("RpcProcessingTimeAvgTime", rpcMetrics)
|
|
|
+ > 4000000D);
|
|
|
+ assertTrue(getDoubleGauge("RpcQueueTimeAvgTime", rpcMetrics)
|
|
|
+ > 4000D);
|
|
|
+ } finally {
|
|
|
+ if (proxy2 != null) {
|
|
|
+ RPC.stopProxy(proxy2);
|
|
|
+ }
|
|
|
+ stop(server, proxy);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
public static void main(String[] args) throws Exception {
|
|
|
new TestRPC().testCallsInternal(conf);
|
|
|
}
|