You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bi...@apache.org on 2020/01/06 07:19:19 UTC

[hbase] branch master updated: HBASE-23636 Disable table may hang when regionserver stop or abort. (#982)

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

binlijin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/master by this push:
     new 33f45d4  HBASE-23636 Disable table may hang when regionserver stop or abort. (#982)
33f45d4 is described below

commit 33f45d441385c37d5cb00a40a29d0aaa0caac7db
Author: binlijin <bi...@gmail.com>
AuthorDate: Mon Jan 6 15:19:10 2020 +0800

    HBASE-23636 Disable table may hang when regionserver stop or abort. (#982)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
    Signed-off-by: virajjasani <34...@users.noreply.github.com>
---
 .../hbase/master/assignment/AssignmentManager.java |   2 +-
 .../master/procedure/ServerCrashProcedure.java     |  16 ++-
 .../assignment/TestRaceBetweenSCPAndDTP.java       | 139 +++++++++++++++++++++
 3 files changed, 154 insertions(+), 3 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
index 4118681..efd144d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -1764,7 +1764,7 @@ public class AssignmentManager {
 
   // should be called under the RegionStateNode lock
   // for SCP
-  void regionClosedAbnormally(RegionStateNode regionNode) throws IOException {
+  public void regionClosedAbnormally(RegionStateNode regionNode) throws IOException {
     RegionState.State state = regionNode.getState();
     ServerName regionLocation = regionNode.getRegionLocation();
     regionNode.transitionState(State.ABNORMALLY_CLOSED);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
index a6ef5ca..178343f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
@@ -482,8 +482,20 @@ public class ServerCrashProcedure
           regionNode.getProcedure().serverCrashed(env, regionNode, getServerName());
           continue;
         }
-        if (env.getMasterServices().getTableStateManager().isTableState(regionNode.getTable(),
-          TableState.State.DISABLING, TableState.State.DISABLED)) {
+        if (env.getMasterServices().getTableStateManager()
+          .isTableState(regionNode.getTable(), TableState.State.DISABLING)) {
+          // We need to change the state here otherwise the TRSP scheduled by DTP will try to
+          // close the region from a dead server and will never succeed. Please see HBASE-23636
+          // for more details.
+          env.getAssignmentManager().regionClosedAbnormally(regionNode);
+          LOG.info("{} found table disabling for region {}, set it state to ABNORMALLY_CLOSED.",
+            this, regionNode);
+          continue;
+        }
+        if (env.getMasterServices().getTableStateManager()
+          .isTableState(regionNode.getTable(), TableState.State.DISABLED)) {
+          // This should not happen, table disabled but has regions on server.
+          LOG.warn("Found table disabled for region {}, procDetails: {}", regionNode, this);
           continue;
         }
         // force to assign to a new candidate server, see HBASE-23035 for more details.
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRaceBetweenSCPAndDTP.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRaceBetweenSCPAndDTP.java
new file mode 100644
index 0000000..9bfc252
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRaceBetweenSCPAndDTP.java
@@ -0,0 +1,139 @@
+/**
+ * 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.master.assignment;
+
+import java.io.IOException;
+import java.util.concurrent.CountDownLatch;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
+import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.zookeeper.KeeperException;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Testcase for HBASE-23636.
+ */
+@Category({ MasterTests.class, LargeTests.class })
+public class TestRaceBetweenSCPAndDTP {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestRaceBetweenSCPAndDTP.class);
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static TableName NAME = TableName.valueOf("Race");
+
+  private static byte[] CF = Bytes.toBytes("cf");
+
+  private static CountDownLatch ARRIVE_GET_REGIONS_ON_TABLE;
+
+  private static CountDownLatch RESUME_GET_REGIONS_ON_SERVER;
+
+  private static final class AssignmentManagerForTest extends AssignmentManager {
+
+    public AssignmentManagerForTest(MasterServices master) {
+      super(master);
+    }
+
+    @Override
+    public TransitRegionStateProcedure[] createUnassignProceduresForDisabling(TableName tableName) {
+      if (ARRIVE_GET_REGIONS_ON_TABLE != null) {
+        ARRIVE_GET_REGIONS_ON_TABLE.countDown();
+        ARRIVE_GET_REGIONS_ON_TABLE = null;
+        try {
+          RESUME_GET_REGIONS_ON_SERVER.await();
+        } catch (InterruptedException e) {
+        }
+      }
+      TransitRegionStateProcedure[] procs = super.createUnassignProceduresForDisabling(tableName);
+      return procs;
+    }
+  }
+
+  public static final class HMasterForTest extends HMaster {
+
+    public HMasterForTest(Configuration conf) throws IOException, KeeperException {
+      super(conf);
+    }
+
+    @Override
+    protected AssignmentManager createAssignmentManager(MasterServices master) {
+      return new AssignmentManagerForTest(master);
+    }
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.getConfiguration().setClass(HConstants.MASTER_IMPL, HMasterForTest.class, HMaster.class);
+    UTIL.startMiniCluster(2);
+    UTIL.createTable(NAME, CF);
+    UTIL.waitTableAvailable(NAME);
+    UTIL.getAdmin().balancerSwitch(false, true);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void test() throws Exception {
+    RegionInfo region = UTIL.getMiniHBaseCluster().getRegions(NAME).get(0).getRegionInfo();
+    AssignmentManager am = UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager();
+    ServerName sn = am.getRegionStates().getRegionState(region).getServerName();
+
+    ARRIVE_GET_REGIONS_ON_TABLE = new CountDownLatch(1);
+    RESUME_GET_REGIONS_ON_SERVER = new CountDownLatch(1);
+
+    UTIL.getAdmin().disableTableAsync(NAME);
+    ARRIVE_GET_REGIONS_ON_TABLE.await();
+
+    UTIL.getMiniHBaseCluster().stopRegionServer(sn);
+    // Wait ServerCrashProcedure to init.
+    Thread.sleep(1000);
+    ProcedureExecutor<?> procExec =
+        UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
+    long scpProcId =
+        procExec.getProcedures().stream().filter(p -> p instanceof ServerCrashProcedure)
+            .map(p -> (ServerCrashProcedure) p).findAny().get().getProcId();
+    UTIL.waitFor(60000, () -> procExec.isFinished(scpProcId));
+    RESUME_GET_REGIONS_ON_SERVER.countDown();
+
+    long dtpProcId =
+        procExec.getProcedures().stream().filter(p -> p instanceof DisableTableProcedure)
+            .map(p -> (DisableTableProcedure) p).findAny().get().getProcId();
+    UTIL.waitFor(60000, () -> procExec.isFinished(dtpProcId));
+  }
+}