|
@@ -39,6 +39,7 @@ import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.Rpc
|
|
import org.apache.hadoop.ipc.protobuf.TestProtos;
|
|
import org.apache.hadoop.ipc.protobuf.TestProtos;
|
|
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
|
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
|
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
|
|
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
|
|
|
|
+import org.apache.hadoop.metrics2.lib.MutableRatesWithAggregation;
|
|
import org.apache.hadoop.net.NetUtils;
|
|
import org.apache.hadoop.net.NetUtils;
|
|
import org.apache.hadoop.security.AccessControlException;
|
|
import org.apache.hadoop.security.AccessControlException;
|
|
import org.apache.hadoop.security.SecurityUtil;
|
|
import org.apache.hadoop.security.SecurityUtil;
|
|
@@ -95,6 +96,9 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|
import java.util.concurrent.atomic.AtomicReference;
|
|
import java.util.concurrent.atomic.AtomicReference;
|
|
import java.util.concurrent.locks.ReentrantLock;
|
|
import java.util.concurrent.locks.ReentrantLock;
|
|
|
|
|
|
|
|
+import static org.apache.hadoop.test.MetricsAsserts.assertGaugeGt;
|
|
|
|
+import static org.apache.hadoop.test.MetricsAsserts.assertGaugeGte;
|
|
|
|
+import static org.apache.hadoop.test.MetricsAsserts.mockMetricsRecordBuilder;
|
|
import static org.assertj.core.api.Assertions.assertThat;
|
|
import static org.assertj.core.api.Assertions.assertThat;
|
|
import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
|
|
import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
|
|
import static org.apache.hadoop.test.MetricsAsserts.assertCounterGt;
|
|
import static org.apache.hadoop.test.MetricsAsserts.assertCounterGt;
|
|
@@ -1397,6 +1401,82 @@ public class TestRPC extends TestRpcBase {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Test the rpcCallSucesses metric in RpcMetrics.
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testRpcCallSuccessesMetric() throws Exception {
|
|
|
|
+ final Server server;
|
|
|
|
+ TestRpcService proxy = null;
|
|
|
|
+
|
|
|
|
+ server = setupTestServer(conf, 5);
|
|
|
|
+ try {
|
|
|
|
+ proxy = getClient(addr, conf);
|
|
|
|
+
|
|
|
|
+ // 10 successful responses
|
|
|
|
+ for (int i = 0; i < 10; i++) {
|
|
|
|
+ proxy.ping(null, newEmptyRequest());
|
|
|
|
+ }
|
|
|
|
+ MetricsRecordBuilder rpcMetrics =
|
|
|
|
+ getMetrics(server.getRpcMetrics().name());
|
|
|
|
+ assertCounter("RpcCallSuccesses", 10L, rpcMetrics);
|
|
|
|
+ // rpcQueueTimeNumOps equals total number of RPC calls.
|
|
|
|
+ assertCounter("RpcQueueTimeNumOps", 10L, rpcMetrics);
|
|
|
|
+
|
|
|
|
+ // 2 failed responses with ERROR status and 1 more successful response.
|
|
|
|
+ for (int i = 0; i < 2; i++) {
|
|
|
|
+ try {
|
|
|
|
+ proxy.error(null, newEmptyRequest());
|
|
|
|
+ } catch (ServiceException ignored) {
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ proxy.ping(null, newEmptyRequest());
|
|
|
|
+
|
|
|
|
+ rpcMetrics = getMetrics(server.getRpcMetrics().name());
|
|
|
|
+ assertCounter("RpcCallSuccesses", 11L, rpcMetrics);
|
|
|
|
+ assertCounter("RpcQueueTimeNumOps", 13L, rpcMetrics);
|
|
|
|
+ } finally {
|
|
|
|
+ stop(server, proxy);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Test per-type overall RPC processing time metric.
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testOverallRpcProcessingTimeMetric() throws Exception {
|
|
|
|
+ final Server server;
|
|
|
|
+ TestRpcService proxy = null;
|
|
|
|
+
|
|
|
|
+ server = setupTestServer(conf, 5);
|
|
|
|
+ try {
|
|
|
|
+ proxy = getClient(addr, conf);
|
|
|
|
+
|
|
|
|
+ // Sent 1 ping request and 2 lockAndSleep requests
|
|
|
|
+ proxy.ping(null, newEmptyRequest());
|
|
|
|
+ proxy.lockAndSleep(null, newSleepRequest(10));
|
|
|
|
+ proxy.lockAndSleep(null, newSleepRequest(12));
|
|
|
|
+
|
|
|
|
+ MetricsRecordBuilder rb = mockMetricsRecordBuilder();
|
|
|
|
+ MutableRatesWithAggregation rates =
|
|
|
|
+ server.rpcDetailedMetrics.getOverallRpcProcessingRates();
|
|
|
|
+ rates.snapshot(rb, true);
|
|
|
|
+
|
|
|
|
+ // Verify the ping request.
|
|
|
|
+ // Overall processing time for ping is zero when this test is run together with
|
|
|
|
+ // the rest of tests. Thus, we use assertGaugeGte() for OverallPingAvgTime.
|
|
|
|
+ assertCounter("OverallPingNumOps", 1L, rb);
|
|
|
|
+ assertGaugeGte("OverallPingAvgTime", 0.0, rb);
|
|
|
|
+
|
|
|
|
+ // Verify lockAndSleep requests. AvgTime should be greater than 10 ms,
|
|
|
|
+ // since we sleep for 10 and 12 ms respectively.
|
|
|
|
+ assertCounter("OverallLockAndSleepNumOps", 2L, rb);
|
|
|
|
+ assertGaugeGt("OverallLockAndSleepAvgTime", 10.0, rb);
|
|
|
|
+
|
|
|
|
+ } finally {
|
|
|
|
+ stop(server, proxy);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
/**
|
|
/**
|
|
* Test RPC backoff by queue full.
|
|
* Test RPC backoff by queue full.
|
|
*/
|
|
*/
|