|
@@ -55,7 +55,6 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
|
|
import javax.net.SocketFactory;
|
|
|
|
|
|
-import com.google.common.base.Supplier;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.commons.logging.impl.Log4JLogger;
|
|
@@ -63,7 +62,6 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
-import org.apache.hadoop.io.IntWritable;
|
|
|
import org.apache.hadoop.io.LongWritable;
|
|
|
import org.apache.hadoop.io.Writable;
|
|
|
import org.apache.hadoop.io.retry.DefaultFailoverProxyProvider;
|
|
@@ -78,9 +76,9 @@ import org.apache.hadoop.ipc.Server.Connection;
|
|
|
import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto;
|
|
|
import org.apache.hadoop.net.ConnectTimeoutException;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
-import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
|
|
import org.apache.log4j.Level;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Assume;
|
|
@@ -91,6 +89,7 @@ import org.mockito.internal.util.reflection.Whitebox;
|
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
|
import org.mockito.stubbing.Answer;
|
|
|
|
|
|
+import com.google.common.base.Supplier;
|
|
|
import com.google.common.primitives.Bytes;
|
|
|
import com.google.common.primitives.Ints;
|
|
|
|
|
@@ -122,6 +121,33 @@ public class TestIPC {
|
|
|
/** Directory where we can count open file descriptors on Linux */
|
|
|
private static final File FD_DIR = new File("/proc/self/fd");
|
|
|
|
|
|
+ static ConnectionId getConnectionId(InetSocketAddress addr, int rpcTimeout,
|
|
|
+ Configuration conf) throws IOException {
|
|
|
+ return ConnectionId.getConnectionId(addr, null, null, rpcTimeout, null,
|
|
|
+ conf);
|
|
|
+ }
|
|
|
+
|
|
|
+ static Writable call(Client client, InetSocketAddress addr,
|
|
|
+ int serviceClass, Configuration conf) throws IOException {
|
|
|
+ final LongWritable param = new LongWritable(RANDOM.nextLong());
|
|
|
+ final ConnectionId remoteId = getConnectionId(addr, MIN_SLEEP_TIME, conf);
|
|
|
+ return client.call(RPC.RpcKind.RPC_BUILTIN, param, remoteId, serviceClass,
|
|
|
+ null);
|
|
|
+ }
|
|
|
+
|
|
|
+ static LongWritable call(Client client, long param, InetSocketAddress addr,
|
|
|
+ Configuration conf) throws IOException {
|
|
|
+ return call(client, new LongWritable(param), addr, 0, conf);
|
|
|
+ }
|
|
|
+
|
|
|
+ static LongWritable call(Client client, LongWritable param,
|
|
|
+ InetSocketAddress addr, int rpcTimeout, Configuration conf)
|
|
|
+ throws IOException {
|
|
|
+ final ConnectionId remoteId = getConnectionId(addr, rpcTimeout, conf);
|
|
|
+ return (LongWritable)client.call(RPC.RpcKind.RPC_BUILTIN, param, remoteId,
|
|
|
+ RPC.RPC_SERVICE_CLASS_DEFAULT, null);
|
|
|
+ }
|
|
|
+
|
|
|
private static class TestServer extends Server {
|
|
|
// Tests can set callListener to run a piece of code each time the server
|
|
|
// receives a call. This code executes on the server thread, so it has
|
|
@@ -183,10 +209,9 @@ public class TestIPC {
|
|
|
public void run() {
|
|
|
for (int i = 0; i < count; i++) {
|
|
|
try {
|
|
|
- LongWritable param = new LongWritable(RANDOM.nextLong());
|
|
|
- LongWritable value =
|
|
|
- (LongWritable)client.call(param, server, null, null, 0, conf);
|
|
|
- if (!param.equals(value)) {
|
|
|
+ final long param = RANDOM.nextLong();
|
|
|
+ LongWritable value = call(client, param, server, conf);
|
|
|
+ if (value.get() != param) {
|
|
|
LOG.fatal("Call failed!");
|
|
|
failed = true;
|
|
|
break;
|
|
@@ -226,9 +251,8 @@ public class TestIPC {
|
|
|
@Override
|
|
|
public Object invoke(Object proxy, Method method, Object[] args)
|
|
|
throws Throwable {
|
|
|
- LongWritable param = new LongWritable(RANDOM.nextLong());
|
|
|
- LongWritable value = (LongWritable) client.call(param,
|
|
|
- NetUtils.getConnectAddress(server), null, null, 0, conf);
|
|
|
+ LongWritable value = call(client, RANDOM.nextLong(),
|
|
|
+ NetUtils.getConnectAddress(server), conf);
|
|
|
return returnValue(value);
|
|
|
}
|
|
|
|
|
@@ -298,8 +322,7 @@ public class TestIPC {
|
|
|
Client client = new Client(LongWritable.class, conf);
|
|
|
InetSocketAddress address = new InetSocketAddress("127.0.0.1", 10);
|
|
|
try {
|
|
|
- client.call(new LongWritable(RANDOM.nextLong()),
|
|
|
- address, null, null, 0, conf);
|
|
|
+ call(client, RANDOM.nextLong(), address, conf);
|
|
|
fail("Expected an exception to have been thrown");
|
|
|
} catch (IOException e) {
|
|
|
String message = e.getMessage();
|
|
@@ -412,7 +435,7 @@ public class TestIPC {
|
|
|
LongWritable param = clientParamClass.newInstance();
|
|
|
|
|
|
try {
|
|
|
- client.call(param, addr, null, null, 0, conf);
|
|
|
+ call(client, param, addr, 0, conf);
|
|
|
fail("Expected an exception to have been thrown");
|
|
|
} catch (Throwable t) {
|
|
|
assertExceptionContains(t, "Injected fault");
|
|
@@ -422,7 +445,7 @@ public class TestIPC {
|
|
|
// ie the internal state of the client or server should not be broken
|
|
|
// by the failed call
|
|
|
WRITABLE_FAULTS_ENABLED = false;
|
|
|
- client.call(param, addr, null, null, 0, conf);
|
|
|
+ call(client, param, addr, 0, conf);
|
|
|
|
|
|
} finally {
|
|
|
client.stop();
|
|
@@ -536,8 +559,7 @@ public class TestIPC {
|
|
|
|
|
|
InetSocketAddress address = new InetSocketAddress("127.0.0.1", 10);
|
|
|
try {
|
|
|
- client.call(new LongWritable(RANDOM.nextLong()),
|
|
|
- address, null, null, 0, conf);
|
|
|
+ call(client, RANDOM.nextLong(), address, conf);
|
|
|
fail("Expected an exception to have been thrown");
|
|
|
} catch (IOException e) {
|
|
|
assertTrue(e.getMessage().contains("Injected fault"));
|
|
@@ -574,8 +596,7 @@ public class TestIPC {
|
|
|
// Call should fail due to injected exception.
|
|
|
InetSocketAddress address = NetUtils.getConnectAddress(server);
|
|
|
try {
|
|
|
- client.call(new LongWritable(RANDOM.nextLong()),
|
|
|
- address, null, null, 0, conf);
|
|
|
+ call(client, RANDOM.nextLong(), address, conf);
|
|
|
fail("Expected an exception to have been thrown");
|
|
|
} catch (Exception e) {
|
|
|
LOG.info("caught expected exception", e);
|
|
@@ -586,8 +607,7 @@ public class TestIPC {
|
|
|
// (i.e. it should not have cached a half-constructed connection)
|
|
|
|
|
|
Mockito.reset(spyFactory);
|
|
|
- client.call(new LongWritable(RANDOM.nextLong()),
|
|
|
- address, null, null, 0, conf);
|
|
|
+ call(client, RANDOM.nextLong(), address, conf);
|
|
|
} finally {
|
|
|
client.stop();
|
|
|
server.stop();
|
|
@@ -605,15 +625,15 @@ public class TestIPC {
|
|
|
Client client = new Client(LongWritable.class, conf);
|
|
|
// set timeout to be less than MIN_SLEEP_TIME
|
|
|
try {
|
|
|
- client.call(new LongWritable(RANDOM.nextLong()),
|
|
|
- addr, null, null, MIN_SLEEP_TIME/2, conf);
|
|
|
+ call(client, new LongWritable(RANDOM.nextLong()), addr,
|
|
|
+ MIN_SLEEP_TIME / 2, conf);
|
|
|
fail("Expected an exception to have been thrown");
|
|
|
} catch (SocketTimeoutException e) {
|
|
|
LOG.info("Get a SocketTimeoutException ", e);
|
|
|
}
|
|
|
// set timeout to be bigger than 3*ping interval
|
|
|
- client.call(new LongWritable(RANDOM.nextLong()),
|
|
|
- addr, null, null, 3*PING_INTERVAL+MIN_SLEEP_TIME, conf);
|
|
|
+ call(client, new LongWritable(RANDOM.nextLong()), addr,
|
|
|
+ 3 * PING_INTERVAL + MIN_SLEEP_TIME, conf);
|
|
|
client.stop();
|
|
|
}
|
|
|
|
|
@@ -629,8 +649,8 @@ public class TestIPC {
|
|
|
Client client = new Client(LongWritable.class, conf);
|
|
|
// set the rpc timeout to twice the MIN_SLEEP_TIME
|
|
|
try {
|
|
|
- client.call(new LongWritable(RANDOM.nextLong()),
|
|
|
- addr, null, null, MIN_SLEEP_TIME*2, conf);
|
|
|
+ call(client, new LongWritable(RANDOM.nextLong()), addr,
|
|
|
+ MIN_SLEEP_TIME * 2, conf);
|
|
|
fail("Expected an exception to have been thrown");
|
|
|
} catch (SocketTimeoutException e) {
|
|
|
LOG.info("Get a SocketTimeoutException ", e);
|
|
@@ -743,8 +763,8 @@ public class TestIPC {
|
|
|
public void run() {
|
|
|
Client client = new Client(LongWritable.class, conf);
|
|
|
try {
|
|
|
- client.call(new LongWritable(Thread.currentThread().getId()),
|
|
|
- addr, null, null, 60000, conf);
|
|
|
+ call(client, new LongWritable(Thread.currentThread().getId()),
|
|
|
+ addr, 60000, conf);
|
|
|
} catch (Throwable e) {
|
|
|
LOG.error(e);
|
|
|
failures.incrementAndGet();
|
|
@@ -875,8 +895,7 @@ public class TestIPC {
|
|
|
public void run() {
|
|
|
Client client = new Client(LongWritable.class, clientConf);
|
|
|
try {
|
|
|
- client.call(new LongWritable(Thread.currentThread().getId()),
|
|
|
- addr, null, null, 0, clientConf);
|
|
|
+ call(client, Thread.currentThread().getId(), addr, clientConf);
|
|
|
callReturned.countDown();
|
|
|
Thread.sleep(10000);
|
|
|
} catch (IOException e) {
|
|
@@ -931,16 +950,15 @@ public class TestIPC {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Make a call from a client and verify if header info is changed in server side
|
|
|
*/
|
|
|
- private void callAndVerify(Server server, InetSocketAddress addr,
|
|
|
+ private static void callAndVerify(Server server, InetSocketAddress addr,
|
|
|
int serviceClass, boolean noChanged) throws IOException{
|
|
|
Client client = new Client(LongWritable.class, conf);
|
|
|
|
|
|
- client.call(new LongWritable(RANDOM.nextLong()),
|
|
|
- addr, null, null, MIN_SLEEP_TIME, serviceClass, conf);
|
|
|
+ call(client, addr, serviceClass, conf);
|
|
|
Connection connection = server.getConnections()[0];
|
|
|
int serviceClass2 = connection.getServiceClass();
|
|
|
assertFalse(noChanged ^ serviceClass == serviceClass2);
|
|
@@ -956,13 +974,11 @@ public class TestIPC {
|
|
|
|
|
|
// start client
|
|
|
Client client = new Client(LongWritable.class, conf);
|
|
|
- client.call(new LongWritable(RANDOM.nextLong()),
|
|
|
- addr, null, null, MIN_SLEEP_TIME, 0, conf);
|
|
|
+ call(client, addr, 0, conf);
|
|
|
client.stop();
|
|
|
|
|
|
// This call should throw IOException.
|
|
|
- client.call(new LongWritable(RANDOM.nextLong()),
|
|
|
- addr, null, null, MIN_SLEEP_TIME, 0, conf);
|
|
|
+ call(client, addr, 0, conf);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -992,7 +1008,7 @@ public class TestIPC {
|
|
|
@Test(timeout=30000)
|
|
|
public void testInterrupted() {
|
|
|
Client client = new Client(LongWritable.class, conf);
|
|
|
- client.getClientExecutor().submit(new Runnable() {
|
|
|
+ Client.getClientExecutor().submit(new Runnable() {
|
|
|
public void run() {
|
|
|
while(true);
|
|
|
}
|
|
@@ -1007,7 +1023,7 @@ public class TestIPC {
|
|
|
Assert.fail("The Client did not interrupt after handling an Interrupted Exception");
|
|
|
}
|
|
|
// Clear Thread interrupt
|
|
|
- Thread.currentThread().interrupted();
|
|
|
+ Thread.interrupted();
|
|
|
}
|
|
|
|
|
|
private long countOpenFileDescriptors() {
|
|
@@ -1363,11 +1379,10 @@ public class TestIPC {
|
|
|
int maxTimeoutRetries) throws IOException {
|
|
|
SocketFactory mockFactory = Mockito.mock(SocketFactory.class);
|
|
|
doThrow(new ConnectTimeoutException("fake")).when(mockFactory).createSocket();
|
|
|
- Client client = new Client(IntWritable.class, conf, mockFactory);
|
|
|
+ Client client = new Client(LongWritable.class, conf, mockFactory);
|
|
|
InetSocketAddress address = new InetSocketAddress("127.0.0.1", 9090);
|
|
|
try {
|
|
|
- client.call(new IntWritable(RANDOM.nextInt()), address, null, null, 0,
|
|
|
- conf);
|
|
|
+ call(client, RANDOM.nextLong(), address, conf);
|
|
|
fail("Not throwing the SocketTimeoutException");
|
|
|
} catch (SocketTimeoutException e) {
|
|
|
Mockito.verify(mockFactory, Mockito.times(maxTimeoutRetries))
|