|
@@ -25,9 +25,11 @@ import junit.framework.Assert;
|
|
|
import org.apache.avro.ipc.Server;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
+import org.apache.hadoop.yarn.api.ClientRMProtocol;
|
|
|
import org.apache.hadoop.yarn.api.ContainerManager;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
|
|
|
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationIdRequest;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
|
|
@@ -47,6 +49,7 @@ import org.apache.hadoop.yarn.factory.providers.YarnRemoteExceptionFactoryProvid
|
|
|
import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
|
|
|
import org.apache.hadoop.yarn.ipc.RPCUtil;
|
|
|
import org.apache.hadoop.yarn.ipc.YarnRPC;
|
|
|
+import org.apache.hadoop.yarn.util.Records;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
public class TestRPC {
|
|
@@ -65,6 +68,35 @@ public class TestRPC {
|
|
|
// test(HadoopYarnRPC.class.getName());
|
|
|
// }
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testUnknownCall() {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.set(YarnConfiguration.IPC_RPC_IMPL, HadoopYarnProtoRPC.class
|
|
|
+ .getName());
|
|
|
+ YarnRPC rpc = YarnRPC.create(conf);
|
|
|
+ String bindAddr = "localhost:0";
|
|
|
+ InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);
|
|
|
+ Server server = rpc.getServer(ContainerManager.class,
|
|
|
+ new DummyContainerManager(), addr, conf, null, 1);
|
|
|
+ server.start();
|
|
|
+
|
|
|
+ // Any unrelated protocol would do
|
|
|
+ ClientRMProtocol proxy = (ClientRMProtocol) rpc.getProxy(
|
|
|
+ ClientRMProtocol.class, NetUtils.createSocketAddr("localhost:"
|
|
|
+ + server.getPort()), conf);
|
|
|
+
|
|
|
+ try {
|
|
|
+ proxy.getNewApplicationId(Records
|
|
|
+ .newRecord(GetNewApplicationIdRequest.class));
|
|
|
+ Assert.fail("Excepted RPC call to fail with unknown method.");
|
|
|
+ } catch (YarnRemoteException e) {
|
|
|
+ Assert.assertEquals("Unknown method getNewApplicationId called on "
|
|
|
+ + "org.apache.hadoop.yarn.proto.ClientRMProtocol"
|
|
|
+ + "$ClientRMProtocolService$BlockingInterface protocol.", e
|
|
|
+ .getMessage());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testHadoopProtoRPC() throws Exception {
|
|
|
test(HadoopYarnProtoRPC.class.getName());
|