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/01/16 13:06:21 UTC

[hbase] branch branch-2 updated: HBASE-21731 Do not need to use ClusterConnection in IntegrationTestBigLinkedListWithVisibility

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

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


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 4e79241  HBASE-21731 Do not need to use ClusterConnection in IntegrationTestBigLinkedListWithVisibility
4e79241 is described below

commit 4e792414f65c8a810cf9bfc00ea82e24407b832e
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Wed Jan 16 15:14:43 2019 +0800

    HBASE-21731 Do not need to use ClusterConnection in IntegrationTestBigLinkedListWithVisibility
    
    Signed-off-by: Peter Somogyi <ps...@apache.org>
---
 ...IntegrationTestBigLinkedListWithVisibility.java | 39 +++++++++++-----------
 1 file changed, 19 insertions(+), 20 deletions(-)

diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java
index a8c595a..6b39776 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java
@@ -22,7 +22,6 @@ import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
 import java.util.Iterator;
 import java.util.UUID;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
@@ -38,7 +37,6 @@ import org.apache.hadoop.hbase.chaos.factories.MonkeyFactory;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.BufferedMutator;
 import org.apache.hadoop.hbase.client.BufferedMutatorParams;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Delete;
@@ -71,6 +69,7 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
 
 /**
@@ -450,24 +449,24 @@ public class IntegrationTestBigLinkedListWithVisibility extends IntegrationTestB
 
     @Override
     protected void handleFailure(Counters counters) throws IOException {
-      Configuration conf = job.getConfiguration();
-      ClusterConnection conn = (ClusterConnection) ConnectionFactory.createConnection(conf);
-      TableName tableName = TableName.valueOf(COMMON_TABLE_NAME);
-      CounterGroup g = counters.getGroup("undef");
-      Iterator<Counter> it = g.iterator();
-      while (it.hasNext()) {
-        String keyString = it.next().getName();
-        byte[] key = Bytes.toBytes(keyString);
-        HRegionLocation loc = conn.relocateRegion(tableName, key);
-        LOG.error("undefined row " + keyString + ", " + loc);
-      }
-      g = counters.getGroup("unref");
-      it = g.iterator();
-      while (it.hasNext()) {
-        String keyString = it.next().getName();
-        byte[] key = Bytes.toBytes(keyString);
-        HRegionLocation loc = conn.relocateRegion(tableName, key);
-        LOG.error("unreferred row " + keyString + ", " + loc);
+      try (Connection conn = ConnectionFactory.createConnection(job.getConfiguration())) {
+        TableName tableName = TableName.valueOf(COMMON_TABLE_NAME);
+        CounterGroup g = counters.getGroup("undef");
+        Iterator<Counter> it = g.iterator();
+        while (it.hasNext()) {
+          String keyString = it.next().getName();
+          byte[] key = Bytes.toBytes(keyString);
+          HRegionLocation loc = conn.getRegionLocator(tableName).getRegionLocation(key, true);
+          LOG.error("undefined row " + keyString + ", " + loc);
+        }
+        g = counters.getGroup("unref");
+        it = g.iterator();
+        while (it.hasNext()) {
+          String keyString = it.next().getName();
+          byte[] key = Bytes.toBytes(keyString);
+          HRegionLocation loc = conn.getRegionLocator(tableName).getRegionLocation(key, true);
+          LOG.error("unreferred row " + keyString + ", " + loc);
+        }
       }
     }
   }