1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.ipc;
19
20 import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
21 import static org.junit.Assert.assertTrue;
22
23 import com.google.common.collect.Lists;
24
25 import java.io.IOException;
26 import java.net.Socket;
27 import java.net.SocketAddress;
28 import java.util.List;
29
30 import org.apache.commons.logging.Log;
31 import org.apache.commons.logging.LogFactory;
32 import org.apache.hadoop.conf.Configuration;
33 import org.apache.hadoop.hbase.CategoryBasedTimeout;
34 import org.apache.hadoop.hbase.HBaseTestingUtility;
35 import org.apache.hadoop.hbase.HConstants;
36 import org.apache.hadoop.hbase.TableName;
37 import org.apache.hadoop.hbase.client.Connection;
38 import org.apache.hadoop.hbase.client.ConnectionFactory;
39 import org.apache.hadoop.hbase.client.Get;
40 import org.apache.hadoop.hbase.client.MetricsConnection;
41 import org.apache.hadoop.hbase.client.RetriesExhaustedException;
42 import org.apache.hadoop.hbase.client.Table;
43 import org.apache.hadoop.hbase.testclassification.MediumTests;
44 import org.junit.AfterClass;
45 import org.junit.BeforeClass;
46 import org.junit.Rule;
47 import org.junit.Test;
48 import org.junit.experimental.categories.Category;
49 import org.junit.rules.ExpectedException;
50 import org.junit.rules.TestRule;
51
52 @Category(MediumTests.class)
53 public class TestRpcClientLeaks {
54 @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
55 withLookingForStuckThread(true).build();
56
57 public static class MyRpcClientImpl extends BlockingRpcClient {
58 public static List<Socket> savedSockets = Lists.newArrayList();
59 @Rule public ExpectedException thrown = ExpectedException.none();
60
61 public MyRpcClientImpl(Configuration conf) {
62 super(conf);
63 }
64
65 public MyRpcClientImpl(Configuration conf, String clusterId, SocketAddress address,
66 MetricsConnection metrics) {
67 super(conf, clusterId, address, metrics);
68 }
69
70 @Override
71 protected BlockingRpcConnection createConnection(ConnectionId remoteId) throws IOException {
72 return new BlockingRpcConnection(this, remoteId) {
73 @Override
74 protected synchronized void setupConnection() throws IOException {
75 super.setupConnection();
76 synchronized (savedSockets) {
77 savedSockets.add(socket);
78 }
79 throw new IOException("Sample exception for " +
80 "verifying socket closure in case of exceptions.");
81 }
82 };
83 }
84 }
85
86 private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
87
88 @BeforeClass
89 public static void setup() throws Exception {
90 UTIL.startMiniCluster();
91 }
92
93 @AfterClass
94 public static void teardown() throws Exception {
95 UTIL.shutdownMiniCluster();
96 }
97
98 public static final Log LOG = LogFactory.getLog(TestRpcClientLeaks.class);
99
100 @Test(expected=RetriesExhaustedException.class)
101 public void testSocketClosed() throws IOException, InterruptedException {
102 String tableName = "testSocketClosed";
103 TableName name = TableName.valueOf(tableName);
104 UTIL.createTable(name, fam1).close();
105
106 Configuration conf = new Configuration(UTIL.getConfiguration());
107 conf.set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY,
108 MyRpcClientImpl.class.getName());
109 conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
110 Connection connection = ConnectionFactory.createConnection(conf);
111 Table table = connection.getTable(TableName.valueOf(tableName));
112 table.get(new Get("asd".getBytes()));
113 connection.close();
114 for (Socket socket : MyRpcClientImpl.savedSockets) {
115 assertTrue("Socket + " + socket + " is not closed", socket.isClosed());
116 }
117 }
118 }