|
@@ -18,6 +18,7 @@
|
|
|
|
|
|
package org.apache.hadoop.ipc;
|
|
package org.apache.hadoop.ipc;
|
|
|
|
|
|
|
|
+import java.io.IOException;
|
|
import java.net.InetSocketAddress;
|
|
import java.net.InetSocketAddress;
|
|
import java.util.concurrent.Callable;
|
|
import java.util.concurrent.Callable;
|
|
import java.util.concurrent.CompletionService;
|
|
import java.util.concurrent.CompletionService;
|
|
@@ -26,6 +27,7 @@ import java.util.concurrent.ExecutorService;
|
|
import java.util.concurrent.Executors;
|
|
import java.util.concurrent.Executors;
|
|
import java.util.concurrent.Future;
|
|
import java.util.concurrent.Future;
|
|
|
|
|
|
|
|
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
|
import org.apache.hadoop.thirdparty.protobuf.BlockingService;
|
|
import org.apache.hadoop.thirdparty.protobuf.BlockingService;
|
|
import org.apache.hadoop.thirdparty.protobuf.RpcController;
|
|
import org.apache.hadoop.thirdparty.protobuf.RpcController;
|
|
import org.apache.hadoop.thirdparty.protobuf.ServiceException;
|
|
import org.apache.hadoop.thirdparty.protobuf.ServiceException;
|
|
@@ -33,18 +35,27 @@ import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.ipc.protobuf.TestProtos;
|
|
import org.apache.hadoop.ipc.protobuf.TestProtos;
|
|
import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.TestProtobufRpcHandoffProto;
|
|
import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.TestProtobufRpcHandoffProto;
|
|
import org.junit.Assert;
|
|
import org.junit.Assert;
|
|
|
|
+import org.junit.Before;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
|
|
+import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
|
|
|
|
+import static org.apache.hadoop.test.MetricsAsserts.assertCounterGt;
|
|
|
|
+import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
|
|
|
|
+
|
|
public class TestProtoBufRpcServerHandoff {
|
|
public class TestProtoBufRpcServerHandoff {
|
|
|
|
|
|
public static final Logger LOG =
|
|
public static final Logger LOG =
|
|
LoggerFactory.getLogger(TestProtoBufRpcServerHandoff.class);
|
|
LoggerFactory.getLogger(TestProtoBufRpcServerHandoff.class);
|
|
|
|
|
|
- @Test(timeout = 20000)
|
|
|
|
- public void test() throws Exception {
|
|
|
|
- Configuration conf = new Configuration();
|
|
|
|
|
|
+ private static Configuration conf = null;
|
|
|
|
+ private static RPC.Server server = null;
|
|
|
|
+ private static InetSocketAddress address = null;
|
|
|
|
+
|
|
|
|
+ @Before
|
|
|
|
+ public void setUp() throws IOException {
|
|
|
|
+ conf = new Configuration();
|
|
|
|
|
|
TestProtoBufRpcServerHandoffServer serverImpl =
|
|
TestProtoBufRpcServerHandoffServer serverImpl =
|
|
new TestProtoBufRpcServerHandoffServer();
|
|
new TestProtoBufRpcServerHandoffServer();
|
|
@@ -53,7 +64,7 @@ public class TestProtoBufRpcServerHandoff {
|
|
|
|
|
|
RPC.setProtocolEngine(conf, TestProtoBufRpcServerHandoffProtocol.class,
|
|
RPC.setProtocolEngine(conf, TestProtoBufRpcServerHandoffProtocol.class,
|
|
ProtobufRpcEngine2.class);
|
|
ProtobufRpcEngine2.class);
|
|
- RPC.Server server = new RPC.Builder(conf)
|
|
|
|
|
|
+ server = new RPC.Builder(conf)
|
|
.setProtocol(TestProtoBufRpcServerHandoffProtocol.class)
|
|
.setProtocol(TestProtoBufRpcServerHandoffProtocol.class)
|
|
.setInstance(blockingService)
|
|
.setInstance(blockingService)
|
|
.setVerbose(true)
|
|
.setVerbose(true)
|
|
@@ -61,10 +72,13 @@ public class TestProtoBufRpcServerHandoff {
|
|
.build();
|
|
.build();
|
|
server.start();
|
|
server.start();
|
|
|
|
|
|
- InetSocketAddress address = server.getListenerAddress();
|
|
|
|
|
|
+ address = server.getListenerAddress();
|
|
long serverStartTime = System.currentTimeMillis();
|
|
long serverStartTime = System.currentTimeMillis();
|
|
LOG.info("Server started at: " + address + " at time: " + serverStartTime);
|
|
LOG.info("Server started at: " + address + " at time: " + serverStartTime);
|
|
|
|
+ }
|
|
|
|
|
|
|
|
+ @Test(timeout = 20000)
|
|
|
|
+ public void test() throws Exception {
|
|
final TestProtoBufRpcServerHandoffProtocol client = RPC.getProxy(
|
|
final TestProtoBufRpcServerHandoffProtocol client = RPC.getProxy(
|
|
TestProtoBufRpcServerHandoffProtocol.class, 1, address, conf);
|
|
TestProtoBufRpcServerHandoffProtocol.class, 1, address, conf);
|
|
|
|
|
|
@@ -93,6 +107,40 @@ public class TestProtoBufRpcServerHandoff {
|
|
|
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Test(timeout = 20000)
|
|
|
|
+ public void testHandoffMetrics() throws Exception {
|
|
|
|
+ final TestProtoBufRpcServerHandoffProtocol client = RPC.getProxy(
|
|
|
|
+ TestProtoBufRpcServerHandoffProtocol.class, 1, address, conf);
|
|
|
|
+
|
|
|
|
+ ExecutorService executorService = Executors.newFixedThreadPool(2);
|
|
|
|
+ CompletionService<ClientInvocationCallable> completionService =
|
|
|
|
+ new ExecutorCompletionService<ClientInvocationCallable>(
|
|
|
|
+ executorService);
|
|
|
|
+
|
|
|
|
+ completionService.submit(new ClientInvocationCallable(client, 5000L));
|
|
|
|
+ completionService.submit(new ClientInvocationCallable(client, 5000L));
|
|
|
|
+
|
|
|
|
+ long submitTime = System.currentTimeMillis();
|
|
|
|
+ Future<ClientInvocationCallable> future1 = completionService.take();
|
|
|
|
+ Future<ClientInvocationCallable> future2 = completionService.take();
|
|
|
|
+
|
|
|
|
+ ClientInvocationCallable callable1 = future1.get();
|
|
|
|
+ ClientInvocationCallable callable2 = future2.get();
|
|
|
|
+
|
|
|
|
+ LOG.info(callable1.toString());
|
|
|
|
+ LOG.info(callable2.toString());
|
|
|
|
+
|
|
|
|
+ // Ensure the 5 second sleep responses are within a reasonable time of each
|
|
|
|
+ // other.
|
|
|
|
+ Assert.assertTrue(Math.abs(callable1.endTime - callable2.endTime) < 2000L);
|
|
|
|
+ Assert.assertTrue(System.currentTimeMillis() - submitTime < 7000L);
|
|
|
|
+
|
|
|
|
+ // Check rpcMetrics
|
|
|
|
+ MetricsRecordBuilder rb = getMetrics(server.rpcMetrics.name());
|
|
|
|
+ assertCounterGt("DeferredRpcProcessingTimeNumOps", 1L, rb);
|
|
|
|
+ assertCounter("RpcProcessingTimeNumOps", 2L, rb);
|
|
|
|
+ }
|
|
|
|
+
|
|
private static class ClientInvocationCallable
|
|
private static class ClientInvocationCallable
|
|
implements Callable<ClientInvocationCallable> {
|
|
implements Callable<ClientInvocationCallable> {
|
|
final TestProtoBufRpcServerHandoffProtocol client;
|
|
final TestProtoBufRpcServerHandoffProtocol client;
|