You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2019/02/23 13:08:48 UTC

[hbase] branch branch-2.0 updated: HBASE-21929 The checks at the end of TestRpcClientLeaks are not executed

This is an automated email from the ASF dual-hosted git repository.

zhangduo pushed a commit to branch branch-2.0
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.0 by this push:
     new cd033d9  HBASE-21929 The checks at the end of TestRpcClientLeaks are not executed
cd033d9 is described below

commit cd033d9ed22ffd3b89407da637ee881fe85ca082
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Tue Feb 19 13:28:39 2019 +0800

    HBASE-21929 The checks at the end of TestRpcClientLeaks are not executed
    
    Signed-off-by: Xu Cang <xu...@apache.org>
---
 .../hadoop/hbase/ipc/TestRpcClientLeaks.java       | 41 +++++++++++-----------
 1 file changed, 20 insertions(+), 21 deletions(-)

diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java
index 4922fd7..7cce8e8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java
@@ -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());
     }
   }
 }