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/12/20 05:52:29 UTC

[hbase] branch branch-2.2 updated: HBASE-23594 Procedure stuck due to region happen to recorded on two servers. (#953)

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

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


The following commit(s) were added to refs/heads/branch-2.2 by this push:
     new 4499092  HBASE-23594 Procedure stuck due to region happen to recorded on two servers. (#953)
4499092 is described below

commit 44990920b18662b2d72dffac8ae658846d3d2c79
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Fri Dec 20 13:40:39 2019 +0800

    HBASE-23594 Procedure stuck due to region happen to recorded on two servers. (#953)
    
    Signed-off-by: stack <st...@apache.org>
---
 .../master/procedure/ServerCrashProcedure.java     |  31 ++--
 .../assignment/TestRaceBetweenSCPAndTRSP.java      | 157 +++++++++++++++++++++
 2 files changed, 178 insertions(+), 10 deletions(-)

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 84c3980..12c699bc 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
@@ -466,20 +466,31 @@ public class ServerCrashProcedure
       RegionStateNode regionNode = am.getRegionStates().getOrCreateRegionStateNode(region);
       regionNode.lock();
       try {
+        // This is possible, as when a server is dead, TRSP will fail to schedule a RemoteProcedure
+        // to us and then try to assign the region to a new RS. And before it has updated the region
+        // location to the new RS, we may have already called the am.getRegionsOnServer so we will
+        // consider the region is still on us. And then before we arrive here, the TRSP could have
+        // updated the region location, or even finished itself, so the region is no longer on us
+        // any more, we should not try to assign it again. Please see HBASE-23594 for more details.
+        if (!serverName.equals(regionNode.getRegionLocation())) {
+          LOG.info("{} found a region {} which is no longer on us {}, give up assigning...", this,
+            regionNode, serverName);
+          continue;
+        }
         if (regionNode.getProcedure() != null) {
           LOG.info("{} found RIT {}; {}", this, regionNode.getProcedure(), regionNode);
           regionNode.getProcedure().serverCrashed(env, regionNode, getServerName());
-        } else {
-          if (env.getMasterServices().getTableStateManager().isTableState(regionNode.getTable(),
-            TableState.State.DISABLING, TableState.State.DISABLED)) {
-            continue;
-          }
-          // force to assign to a new candidate server, see HBASE-23035 for more details.
-          TransitRegionStateProcedure proc =
-              TransitRegionStateProcedure.assign(env, region, true, null);
-          regionNode.setProcedure(proc);
-          addChildProcedure(proc);
+          continue;
+        }
+        if (env.getMasterServices().getTableStateManager().isTableState(regionNode.getTable(),
+          TableState.State.DISABLING, TableState.State.DISABLED)) {
+          continue;
         }
+        // force to assign to a new candidate server, see HBASE-23035 for more details.
+        TransitRegionStateProcedure proc =
+          TransitRegionStateProcedure.assign(env, region, true, null);
+        regionNode.setProcedure(proc);
+        addChildProcedure(proc);
       } finally {
         regionNode.unlock();
       }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRaceBetweenSCPAndTRSP.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRaceBetweenSCPAndTRSP.java
new file mode 100644
index 0000000..6286104
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRaceBetweenSCPAndTRSP.java
@@ -0,0 +1,157 @@
+/**
+ * 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.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+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.RegionPlan;
+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-23594.
+ */
+@Category({ MasterTests.class, LargeTests.class })
+public class TestRaceBetweenSCPAndTRSP {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestRaceBetweenSCPAndTRSP.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_REGION_OPENING;
+
+  private static CountDownLatch RESUME_REGION_OPENING;
+
+  private static CountDownLatch ARRIVE_GET_REGIONS_ON_SERVER;
+
+  private static CountDownLatch RESUME_GET_REGIONS_ON_SERVER;
+
+  private static final class AssignmentManagerForTest extends AssignmentManager {
+
+    public AssignmentManagerForTest(MasterServices master) {
+      super(master);
+    }
+
+    @Override
+    void regionOpening(RegionStateNode regionNode) throws IOException {
+      super.regionOpening(regionNode);
+      if (regionNode.getRegionInfo().getTable().equals(NAME) && ARRIVE_REGION_OPENING != null) {
+        ARRIVE_REGION_OPENING.countDown();
+        ARRIVE_REGION_OPENING = null;
+        try {
+          RESUME_REGION_OPENING.await();
+        } catch (InterruptedException e) {
+        }
+      }
+    }
+
+    @Override
+    public List<RegionInfo> getRegionsOnServer(ServerName serverName) {
+      List<RegionInfo> regions = super.getRegionsOnServer(serverName);
+      if (ARRIVE_GET_REGIONS_ON_SERVER != null) {
+        ARRIVE_GET_REGIONS_ON_SERVER.countDown();
+        ARRIVE_GET_REGIONS_ON_SERVER = null;
+        try {
+          RESUME_GET_REGIONS_ON_SERVER.await();
+        } catch (InterruptedException e) {
+        }
+      }
+      return regions;
+    }
+  }
+
+  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_REGION_OPENING = new CountDownLatch(1);
+    RESUME_REGION_OPENING = new CountDownLatch(1);
+    ARRIVE_GET_REGIONS_ON_SERVER = new CountDownLatch(1);
+    RESUME_GET_REGIONS_ON_SERVER = new CountDownLatch(1);
+
+    Future<byte[]> moveFuture = am.moveAsync(new RegionPlan(region, sn, sn));
+    ARRIVE_REGION_OPENING.await();
+
+    UTIL.getMiniHBaseCluster().killRegionServer(sn);
+    ARRIVE_GET_REGIONS_ON_SERVER.await();
+    RESUME_REGION_OPENING.countDown();
+
+    moveFuture.get();
+    ProcedureExecutor<?> procExec =
+      UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
+    long scpProcId =
+      procExec.getProcedures().stream().filter(p -> p instanceof ServerCrashProcedure)
+        .map(p -> (ServerCrashProcedure) p).findAny().get().getProcId();
+    RESUME_GET_REGIONS_ON_SERVER.countDown();
+    UTIL.waitFor(60000, () -> procExec.isFinished(scpProcId));
+  }
+}