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 2018/01/24 09:50:56 UTC

[14/34] hbase git commit: HBASE-19774 incorrect behavior of locateRegionInMeta

HBASE-19774 incorrect behavior of locateRegionInMeta


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/be496ba8
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/be496ba8
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/be496ba8

Branch: refs/heads/HBASE-19064
Commit: be496ba8ea6a60702387f653a4fb7f0d8d11ec34
Parents: 3a1e433
Author: Sergey Soldatov <ss...@apache.org>
Authored: Thu Jan 11 13:40:38 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Mon Jan 22 11:48:55 2018 -0800

----------------------------------------------------------------------
 .../hbase/client/ConnectionImplementation.java  | 23 ++++-----
 .../client/TestScannersFromClientSide.java      | 50 +++++++++++++++-----
 2 files changed, 49 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/be496ba8/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 562630f..8807884 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -786,11 +786,14 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
     // build the key of the meta region we should be looking for.
     // the extra 9's on the end are necessary to allow "exact" matches
     // without knowing the precise region names.
-    byte[] metaKey = RegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
+    byte[] metaStartKey = RegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
+    byte[] metaStopKey =
+      RegionInfo.createRegionName(tableName, HConstants.EMPTY_START_ROW, "", false);
 
     Scan s = new Scan();
     s.setReversed(true);
-    s.withStartRow(metaKey);
+    s.withStartRow(metaStartKey);
+    s.withStopRow(metaStopKey, true);
     s.addFamily(HConstants.CATALOG_FAMILY);
 
     if (this.useMetaReplicas) {
@@ -840,12 +843,11 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
         // convert the row result into the HRegionLocation we need!
         RegionLocations locations = MetaTableAccessor.getRegionLocations(regionInfoRow);
         if (locations == null || locations.getRegionLocation(replicaId) == null) {
-          throw new IOException("HRegionInfo was null in " +
-            tableName + ", row=" + regionInfoRow);
+          throw new IOException("RegionInfo null in " + tableName + ", row=" + regionInfoRow);
         }
         RegionInfo regionInfo = locations.getRegionLocation(replicaId).getRegion();
         if (regionInfo == null) {
-          throw new IOException("HRegionInfo was null or empty in " +
+          throw new IOException("RegionInfo null or empty in " +
             TableName.META_TABLE_NAME + ", row=" + regionInfoRow);
         }
 
@@ -857,13 +859,12 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
             "hbase:meta might be damaged.");
         }
         if (regionInfo.isSplit()) {
-          throw new RegionOfflineException(
-              "the only available region for the required row is a split parent,"
-                  + " the daughters should be online soon: " + regionInfo.getRegionNameAsString());
+          throw new RegionOfflineException ("Region for row is a split parent, daughters not online: " +
+              regionInfo.getRegionNameAsString());
         }
         if (regionInfo.isOffline()) {
-          throw new RegionOfflineException("the region is offline, could"
-              + " be caused by a disable table call: " + regionInfo.getRegionNameAsString());
+          throw new RegionOfflineException("Region offline; disable table call? " +
+              regionInfo.getRegionNameAsString());
         }
 
         ServerName serverName = locations.getRegionLocation(replicaId).getServerName();
@@ -908,7 +909,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
         // Only relocate the parent region if necessary
         if(!(e instanceof RegionOfflineException ||
             e instanceof NoServerForRegionException)) {
-          relocateRegion(TableName.META_TABLE_NAME, metaKey, replicaId);
+          relocateRegion(TableName.META_TABLE_NAME, metaStartKey, replicaId);
         }
       } finally {
         userRegionLock.unlock();

http://git-wip-us.apache.org/repos/asf/hbase/blob/be496ba8/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
index 5441e2b..9173726 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
@@ -16,19 +16,6 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import java.util.function.Consumer;
-import java.util.stream.IntStream;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CompareOperator;
@@ -41,6 +28,7 @@ import org.apache.hadoop.hbase.HTestConst;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.ColumnPrefixFilter;
 import org.apache.hadoop.hbase.filter.ColumnRangeFilter;
@@ -61,6 +49,21 @@ import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.stream.IntStream;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+
 /**
  * A client-side test, mostly testing scanners with various parameters.
  */
@@ -236,6 +239,27 @@ public class TestScannersFromClientSide {
       clientScanner.getCacheSize() <= 1);
   }
 
+  /**
+   * Scan on not existing table should throw the exception with correct message
+   */
+  @Test
+  public void testScannerForNotExistingTable() {
+    String[] tableNames = {"A", "Z", "A:A", "Z:Z"};
+    for(String tableName : tableNames) {
+      try {
+        Table table = TEST_UTIL.getConnection().getTable(TableName.valueOf(tableName));
+        testSmallScan(table, true, 1, 5);
+        fail("TableNotFoundException was not thrown");
+      } catch (TableNotFoundException e) {
+        // We expect that the message for TableNotFoundException would have only the table name only
+        // Otherwise that would mean that localeRegionInMeta doesn't work properly
+        assertEquals(e.getMessage(), tableName);
+      } catch (Exception e) {
+        fail("Unexpected exception " + e.getMessage());
+      }
+    }
+  }
+
   @Test
   public void testSmallScan() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());