You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by hu...@apache.org on 2020/12/18 17:25:33 UTC

[hbase] branch branch-2 updated: HBASE-25293 Followup jira to address the client handling issue when chaning from meta replica to non-meta-replica at the server side. (#2768) (#2786)

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

huaxiangsun 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 8159fc8  HBASE-25293 Followup jira to address the client handling issue when chaning from meta replica to non-meta-replica at the server side. (#2768) (#2786)
8159fc8 is described below

commit 8159fc8dbea68dfebdfdc256179832502e79e821
Author: huaxiangsun <hu...@apache.org>
AuthorDate: Fri Dec 18 09:25:04 2020 -0800

    HBASE-25293 Followup jira to address the client handling issue when chaning from meta replica to non-meta-replica at the server side. (#2768) (#2786)
    
    Signed-off-by: stack <st...@apache.org>
---
 .../hbase/client/AsyncNonMetaRegionLocator.java    |   2 +-
 .../client/CatalogReplicaLoadBalanceSelector.java  |   2 +
 .../CatalogReplicaLoadBalanceSimpleSelector.java   |  19 ++-
 ...estCatalogReplicaLoadBalanceSimpleSelector.java | 132 +++++++++++++++++++++
 4 files changed, 144 insertions(+), 11 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
index 12e735b..5798ee5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
@@ -210,7 +210,7 @@ class AsyncNonMetaRegionLocator {
 
         this.metaReplicaSelector = CatalogReplicaLoadBalanceSelectorFactory.createSelector(
           replicaSelectorClass, META_TABLE_NAME, conn.getChoreService(), () -> {
-            int numOfReplicas = 1;
+            int numOfReplicas = CatalogReplicaLoadBalanceSelector.UNINITIALIZED_NUM_OF_REPLICAS;
             try {
               RegionLocations metaLocations = conn.registry.getMetaRegionLocations().get(
                 conn.connConf.getReadRpcTimeoutNs(), TimeUnit.NANOSECONDS);
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceSelector.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceSelector.java
index c3ce868..27be88a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceSelector.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceSelector.java
@@ -28,6 +28,8 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 interface CatalogReplicaLoadBalanceSelector {
 
+  int UNINITIALIZED_NUM_OF_REPLICAS = -1;
+
   /**
    * This method is called when input location is stale, i.e, when clients run into
    * org.apache.hadoop.hbase.NotServingRegionException.
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceSimpleSelector.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceSimpleSelector.java
index f0f0672..f78dfb1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceSimpleSelector.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaLoadBalanceSimpleSelector.java
@@ -110,7 +110,6 @@ class CatalogReplicaLoadBalanceSimpleSelector implements
   private final TableName tableName;
   private final IntSupplier getNumOfReplicas;
   private volatile boolean isStopped = false;
-  private final static int UNINITIALIZED_NUM_OF_REPLICAS = -1;
 
   CatalogReplicaLoadBalanceSimpleSelector(TableName tableName, ChoreService choreService,
     IntSupplier getNumOfReplicas) {
@@ -119,7 +118,7 @@ class CatalogReplicaLoadBalanceSimpleSelector implements
     this.getNumOfReplicas = getNumOfReplicas;
 
     // This numOfReplicas is going to be lazy initialized.
-    this.numOfReplicas = UNINITIALIZED_NUM_OF_REPLICAS;
+    this.numOfReplicas = CatalogReplicaLoadBalanceSelector.UNINITIALIZED_NUM_OF_REPLICAS;
     // Start chores
     this.choreService.scheduleChore(getCacheCleanupChore(this));
     this.choreService.scheduleChore(getRefreshReplicaCountChore(this));
@@ -148,7 +147,7 @@ class CatalogReplicaLoadBalanceSimpleSelector implements
    */
   private int getRandomReplicaId() {
     int cachedNumOfReplicas = this.numOfReplicas;
-    if (cachedNumOfReplicas == UNINITIALIZED_NUM_OF_REPLICAS) {
+    if (cachedNumOfReplicas == CatalogReplicaLoadBalanceSelector.UNINITIALIZED_NUM_OF_REPLICAS) {
       cachedNumOfReplicas = refreshCatalogReplicaCount();
       this.numOfReplicas = cachedNumOfReplicas;
     }
@@ -264,16 +263,16 @@ class CatalogReplicaLoadBalanceSimpleSelector implements
   private int refreshCatalogReplicaCount() {
     int newNumOfReplicas = this.getNumOfReplicas.getAsInt();
     LOG.debug("Refreshed replica count {}", newNumOfReplicas);
-    if (newNumOfReplicas == 1) {
-      LOG.warn("Table {}'s region replica count is 1, maybe a misconfiguration or failure to "
-        + "fetch the replica count", tableName);
+    // If the returned number of replicas is -1, it is caused by failure to fetch the
+    // replica count. Do not update the numOfReplicas in this case.
+    if (newNumOfReplicas == CatalogReplicaLoadBalanceSelector.UNINITIALIZED_NUM_OF_REPLICAS) {
+      LOG.error("Failed to fetch Table {}'s region replica count", tableName);
+      return this.numOfReplicas;
     }
-    int cachedNumOfReplicas = this.numOfReplicas;
 
-    // If the returned number of replicas is 1, it is mostly caused by failure to fetch the
-    // replica count. Do not update the numOfReplicas in this case.
+    int cachedNumOfReplicas = this.numOfReplicas;
     if ((cachedNumOfReplicas == UNINITIALIZED_NUM_OF_REPLICAS) ||
-      ((cachedNumOfReplicas != newNumOfReplicas) && (newNumOfReplicas != 1))) {
+      (cachedNumOfReplicas != newNumOfReplicas)) {
       this.numOfReplicas = newNumOfReplicas;
     }
     return newNumOfReplicas;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java
new file mode 100644
index 0000000..ee50bf9
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java
@@ -0,0 +1,132 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW;
+import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category({ MediumTests.class, ClientTests.class })
+public class TestCatalogReplicaLoadBalanceSimpleSelector {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestCatalogReplicaLoadBalanceSimpleSelector.class);
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+    TestCatalogReplicaLoadBalanceSimpleSelector.class);
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  private static final int NB_SERVERS = 4;
+  private static int numOfMetaReplica = NB_SERVERS - 1;
+
+  private static AsyncConnectionImpl CONN;
+
+  private static ConnectionRegistry registry;
+  private static Admin admin;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+
+    TEST_UTIL.startMiniCluster(NB_SERVERS);
+    admin = TEST_UTIL.getAdmin();
+    admin.balancerSwitch(false, true);
+
+    // Enable hbase:meta replication.
+    HBaseTestingUtility.setReplicas(admin, TableName.META_TABLE_NAME, numOfMetaReplica);
+    TEST_UTIL.waitFor(30000, () -> TEST_UTIL.getMiniHBaseCluster().getRegions(
+      TableName.META_TABLE_NAME).size() >= numOfMetaReplica);
+
+    registry = ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
+    CONN = new AsyncConnectionImpl(conf, registry,
+      registry.getClusterId().get(), User.getCurrent());
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    IOUtils.closeQuietly(CONN);
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testMetaChangeFromReplicaNoReplica() throws IOException, InterruptedException {
+    String replicaSelectorClass = CONN.getConfiguration().
+      get(RegionLocator.LOCATOR_META_REPLICAS_MODE_LOADBALANCE_SELECTOR,
+        CatalogReplicaLoadBalanceSimpleSelector.class.getName());
+
+    CatalogReplicaLoadBalanceSelector metaSelector = CatalogReplicaLoadBalanceSelectorFactory
+      .createSelector(replicaSelectorClass, META_TABLE_NAME, CONN.getChoreService(), () -> {
+        int numOfReplicas = CatalogReplicaLoadBalanceSelector.UNINITIALIZED_NUM_OF_REPLICAS;
+        try {
+          RegionLocations metaLocations = CONN.registry.getMetaRegionLocations().get
+            (CONN.connConf.getReadRpcTimeoutNs(), TimeUnit.NANOSECONDS);
+          numOfReplicas = metaLocations.size();
+        } catch (Exception e) {
+          LOG.error("Failed to get table {}'s region replication, ", META_TABLE_NAME, e);
+        }
+        return numOfReplicas;
+      });
+
+    assertNotEquals(
+      metaSelector.select(TableName.valueOf("test"), EMPTY_START_ROW, RegionLocateType.CURRENT),
+      RegionReplicaUtil.DEFAULT_REPLICA_ID);
+
+    // Change to No meta replica
+    HBaseTestingUtility.setReplicas(admin, TableName.META_TABLE_NAME, 1);
+    TEST_UTIL.waitFor(30000, () -> TEST_UTIL.getMiniHBaseCluster().getRegions(
+      TableName.META_TABLE_NAME).size() == 1);
+
+    CatalogReplicaLoadBalanceSelector metaSelectorWithNoReplica =
+      CatalogReplicaLoadBalanceSelectorFactory.createSelector(
+        replicaSelectorClass, META_TABLE_NAME, CONN.getChoreService(), () -> {
+        int numOfReplicas = CatalogReplicaLoadBalanceSelector.UNINITIALIZED_NUM_OF_REPLICAS;
+        try {
+          RegionLocations metaLocations = CONN.registry.getMetaRegionLocations().get(
+            CONN.connConf.getReadRpcTimeoutNs(), TimeUnit.NANOSECONDS);
+          numOfReplicas = metaLocations.size();
+        } catch (Exception e) {
+          LOG.error("Failed to get table {}'s region replication, ", META_TABLE_NAME, e);
+        }
+        return numOfReplicas;
+      });
+    assertEquals(
+      metaSelectorWithNoReplica.select(TableName.valueOf("test"), EMPTY_START_ROW,
+        RegionLocateType.CURRENT), RegionReplicaUtil.DEFAULT_REPLICA_ID);
+  }
+}