You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by al...@apache.org on 2018/07/12 01:37:28 UTC

hbase git commit: HBASE-20860 Merged region's RIT state may not be cleaned after master restart

Repository: hbase
Updated Branches:
  refs/heads/branch-2.0 cd1ecae0d -> e3a67493c


HBASE-20860 Merged region's RIT state may not be cleaned after master restart


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

Branch: refs/heads/branch-2.0
Commit: e3a67493ca3e23125aae173e77c3db1880af3ad3
Parents: cd1ecae
Author: Allan Yang <al...@apache.org>
Authored: Thu Jul 12 09:36:33 2018 +0800
Committer: Allan Yang <al...@apache.org>
Committed: Thu Jul 12 09:36:33 2018 +0800

----------------------------------------------------------------------
 .../hbase/master/assignment/RegionStates.java   |   6 +
 .../TestMasterAbortWhileMergingTable.java       | 126 +++++++++++++++++++
 2 files changed, 132 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e3a67493/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
index 15a2fbc..af226b7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
@@ -500,6 +500,12 @@ public class RegionStates {
 
   public void deleteRegion(final RegionInfo regionInfo) {
     regionsMap.remove(regionInfo.getRegionName());
+    // See HBASE-20860
+    // After master restarts, merged regions' RIT state may not be cleaned,
+    // making sure they are cleaned here
+    if (regionInTransition.containsKey(regionInfo)) {
+      regionInTransition.remove(regionInfo);
+    }
     // Remove from the offline regions map too if there.
     if (this.regionOffline.containsKey(regionInfo)) {
       if (LOG.isTraceEnabled()) LOG.trace("Removing from regionOffline Map: " + regionInfo);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e3a67493/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMasterAbortWhileMergingTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMasterAbortWhileMergingTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMasterAbortWhileMergingTable.java
new file mode 100644
index 0000000..1af9bd0
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestMasterAbortWhileMergingTable.java
@@ -0,0 +1,126 @@
+/**
+ * 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.util.List;
+import java.util.Optional;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.Assert;
+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({MasterTests.class, MediumTests.class})
+public class TestMasterAbortWhileMergingTable {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMasterAbortWhileMergingTable.class);
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(TestMasterAbortWhileMergingTable.class);
+
+  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+  private static TableName TABLE_NAME = TableName.valueOf("test");
+  private static Admin admin;
+  private static byte[] CF = Bytes.toBytes("cf");
+  private static byte[] SPLITKEY = Bytes.toBytes("bbbbbbb");
+  private static CountDownLatch mergeCommitArrive = new CountDownLatch(1);
+
+
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    UTIL.getConfiguration().set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
+        MergeRegionObserver.class.getName());
+    UTIL.startMiniCluster(3);
+    admin = UTIL.getHBaseAdmin();
+    byte[][] splitKeys = new byte[1][];
+    splitKeys[0] = SPLITKEY;
+    UTIL.createTable(TABLE_NAME, CF, splitKeys);
+    UTIL.waitTableAvailable(TABLE_NAME);
+  }
+
+  @AfterClass
+  public static void cleanupTest() throws Exception {
+    try {
+      UTIL.shutdownMiniCluster();
+    } catch (Exception e) {
+      LOG.warn("failure shutting down cluster", e);
+    }
+  }
+
+  @Test
+  public void test() throws Exception {
+    List<RegionInfo> regionInfos = admin.getRegions(TABLE_NAME);
+    MergeTableRegionsProcedure mergeTableRegionsProcedure = new MergeTableRegionsProcedure(
+        UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor()
+            .getEnvironment(), regionInfos.get(0), regionInfos.get(1));
+    long procID = UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor()
+        .submitProcedure(mergeTableRegionsProcedure);
+    mergeCommitArrive.await();
+    UTIL.getMiniHBaseCluster().stopMaster(0);
+    UTIL.getMiniHBaseCluster().startMaster();
+    //wait until master initialized
+    UTIL.waitFor(30000,
+      () -> UTIL.getMiniHBaseCluster().getMaster() != null && UTIL
+        .getMiniHBaseCluster().getMaster().isInitialized());
+    UTIL.waitFor(30000, () -> UTIL.getMiniHBaseCluster().getMaster()
+      .getMasterProcedureExecutor().isFinished(procID));
+    Assert.assertTrue("Found region RIT, that's impossible!",
+      UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager()
+        .getRegionsInTransition().size() == 0);
+  }
+
+  public static class MergeRegionObserver implements MasterCoprocessor,
+      MasterObserver {
+
+    @Override
+    public Optional<MasterObserver> getMasterObserver() {
+      return Optional.of(this);
+    }
+
+    @Override
+    public void preMergeRegionsCommitAction(
+        ObserverContext<MasterCoprocessorEnvironment> ctx,
+        RegionInfo[] regionsToMerge, List<Mutation> metaEntries) {
+      mergeCommitArrive.countDown();
+      LOG.error("mergeCommitArrive countdown");
+    }
+  }
+
+}