HBASE-21929 The checks at the end of TestRpcClientLeaks are not executed

Signed-off-by: Xu Cang <xucang@apache.org>
This commit is contained in:
Duo Zhang 2019-02-19 13:28:39 +08:00 committed by zhangduo
parent f38223739f
commit 07d84eb809
1 changed files with 20 additions and 21 deletions

View File

@ -19,11 +19,13 @@ package org.apache.hadoop.hbase.ipc;
import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.net.Socket;
import java.net.SocketAddress;
import java.util.List;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
@ -43,26 +45,23 @@ import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.ExpectedException;
import org.junit.rules.TestName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
@Category(MediumTests.class)
public class TestRpcClientLeaks {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestRpcClientLeaks.class);
HBaseClassTestRule.forClass(TestRpcClientLeaks.class);
@Rule
public TestName name = new TestName();
private static BlockingQueue<Socket> SAVED_SOCKETS = new LinkedBlockingQueue<>();
public static class MyRpcClientImpl extends BlockingRpcClient {
public static List<Socket> savedSockets = Lists.newArrayList();
@Rule public ExpectedException thrown = ExpectedException.none();
public MyRpcClientImpl(Configuration conf) {
super(conf);
@ -79,11 +78,9 @@ public class TestRpcClientLeaks {
@Override
protected synchronized void setupConnection() throws IOException {
super.setupConnection();
synchronized (savedSockets) {
savedSockets.add(socket);
}
throw new IOException("Sample exception for " +
"verifying socket closure in case of exceptions.");
SAVED_SOCKETS.add(socket);
throw new IOException(
"Sample exception for verifying socket closure in case of exceptions.");
}
};
}
@ -103,21 +100,23 @@ public class TestRpcClientLeaks {
public static final Logger LOG = LoggerFactory.getLogger(TestRpcClientLeaks.class);
@Test(expected=RetriesExhaustedException.class)
@Test
public void testSocketClosed() throws IOException, InterruptedException {
TableName tableName = TableName.valueOf(name.getMethodName());
UTIL.createTable(tableName, fam1).close();
Configuration conf = new Configuration(UTIL.getConfiguration());
conf.set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY,
MyRpcClientImpl.class.getName());
conf.set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, MyRpcClientImpl.class.getName());
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
Connection connection = ConnectionFactory.createConnection(conf);
Table table = connection.getTable(TableName.valueOf(name.getMethodName()));
table.get(new Get(Bytes.toBytes("asd")));
connection.close();
for (Socket socket : MyRpcClientImpl.savedSockets) {
assertTrue("Socket + " + socket + " is not closed", socket.isClosed());
try (Connection connection = ConnectionFactory.createConnection(conf);
Table table = connection.getTable(TableName.valueOf(name.getMethodName()))) {
table.get(new Get(Bytes.toBytes("asd")));
fail("Should fail because the injected error");
} catch (RetriesExhaustedException e) {
// expected
}
for (Socket socket : SAVED_SOCKETS) {
assertTrue("Socket " + socket + " is not closed", socket.isClosed());
}
}
}