From dbf027ec5ed8cab1cff77eb6cc40ea0aa3afc9e3 Mon Sep 17 00:00:00 2001 From: chenheng Date: Wed, 31 Aug 2016 21:47:53 +0800 Subject: [PATCH] HBASE-15278 AsyncRPCClient hangs if Connection closes before RPC call response --- .../org/apache/hadoop/hbase/ipc/RpcServer.java | 2 +- .../apache/hadoop/hbase/ipc/AbstractTestIPC.java | 55 ++++++++++++++++++++++ 2 files changed, 56 insertions(+), 1 deletion(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java index 759da82..c787c98 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java @@ -2677,7 +2677,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { } Connection register(SocketChannel channel) { - Connection connection = new Connection(channel, System.currentTimeMillis()); + Connection connection = getConnection(channel, System.currentTimeMillis()); add(connection); if (LOG.isDebugEnabled()) { LOG.debug("Server connection from " + connection + diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java index 771ef93..2e26e13 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java @@ -35,6 +35,8 @@ import com.google.protobuf.ServiceException; import java.io.IOException; import java.net.InetSocketAddress; +import java.nio.ByteBuffer; +import java.nio.channels.SocketChannel; import java.util.ArrayList; import java.util.List; @@ -287,4 +289,57 @@ public abstract class AbstractTestIPC { rpcServer.stop(); } } + + static class TestFailingRpcServer extends TestRpcServer { + + TestFailingRpcServer() throws IOException { + this(new FifoRpcScheduler(CONF, 1), CONF); + } + + TestFailingRpcServer(Configuration conf) throws IOException { + this(new FifoRpcScheduler(conf, 1), conf); + } + + TestFailingRpcServer(RpcScheduler scheduler, Configuration conf) throws IOException { + super(scheduler, conf); + } + + class FailingConnection extends Connection { + public FailingConnection(SocketChannel channel, long lastContact) { + super(channel, lastContact); + } + + @Override + protected void processRequest(ByteBuffer buf) throws IOException, InterruptedException { + // this will throw exception after the connection header is read, and an RPC is sent + // from client + throw new DoNotRetryIOException("Failing for test"); + } + } + + @Override + protected Connection getConnection(SocketChannel channel, long time) { + return new FailingConnection(channel, time); + } + } + + /** Tests that the connection closing is handled by the client with outstanding RPC calls */ + @Test (timeout = 30000) + public void testConnectionCloseWithOutstandingRPCs() throws InterruptedException, IOException { + Configuration conf = new Configuration(CONF); + RpcServer rpcServer = new TestFailingRpcServer(conf); + try (AbstractRpcClient client = createRpcClient(conf)) { + rpcServer.start(); + BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress()); + EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello").build(); + stub.echo( + new PayloadCarryingRpcController(CellUtil.createCellScanner(ImmutableList. of(CELL))), + param); + fail("RPC should have failed because connection closed"); + } catch (ServiceException e) { + LOG.info("Caught expected exception: " + e.toString()); + } finally { + rpcServer.stop(); + } + } } -- 2.5.4 (Apple Git-61)