You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2019/06/19 21:04:14 UTC

[hbase] branch branch-1 updated: HBASE-22426 Disable region split/merge switch doen't work when 'hbase.assignment.usezk' is set true

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

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


The following commit(s) were added to refs/heads/branch-1 by this push:
     new 4895fad  HBASE-22426 Disable region split/merge switch doen't work when 'hbase.assignment.usezk' is set true
4895fad is described below

commit 4895fad5b0b49f08b2ae6f042fb98beaa6215f2b
Author: Pankaj <pa...@huawei.com>
AuthorDate: Thu May 16 00:55:02 2019 +0530

    HBASE-22426 Disable region split/merge switch doen't work when 'hbase.assignment.usezk' is set true
    
    Signed-off-by: Andrew Purtell <ap...@apache.org>
---
 .../hadoop/hbase/master/AssignmentManager.java     | 35 ++++++++++++++++++----
 .../hbase/client/TestSplitOrMergeStatus.java       |  2 +-
 ...estSplitOrMergeStatusWithZKBasedAssignment.java | 35 ++++++++++++++++++++++
 3 files changed, 66 insertions(+), 6 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
index a039e6a..ef49f99 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
@@ -67,6 +67,7 @@ import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.TableStateManager;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Admin.MasterSwitchType;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
@@ -1082,8 +1083,11 @@ public class AssignmentManager extends ZooKeeperListener {
       case RS_ZK_REQUEST_REGION_SPLIT:
       case RS_ZK_REGION_SPLITTING:
       case RS_ZK_REGION_SPLIT:
-        if (!handleRegionSplitting(
-            rt, encodedName, prettyPrintedRegionName, sn)) {
+        // If region split not enabled then skip only if event type is RS_ZK_REQUEST_REGION_SPLIT,
+        // allow on-going split operations
+        if ((!isRegionSplitOrMergeEnabled(rt, prettyPrintedRegionName, MasterSwitchType.SPLIT)
+            && rt.getEventType() == EventType.RS_ZK_REQUEST_REGION_SPLIT)
+            || !handleRegionSplitting(rt, encodedName, prettyPrintedRegionName, sn)) {
           deleteSplittingNode(encodedName, sn);
         }
         break;
@@ -1093,8 +1097,11 @@ public class AssignmentManager extends ZooKeeperListener {
       case RS_ZK_REGION_MERGED:
         // Merged region is a new region, we can't find it in the region states now.
         // However, the two merging regions are not new. They should be in state for merging.
-        if (!handleRegionMerging(
-            rt, encodedName, prettyPrintedRegionName, sn)) {
+        // If region merge not enabled then skip only if event type is RS_ZK_REQUEST_REGION_MERGE,
+        // allow on-going merge operations
+        if ((!isRegionSplitOrMergeEnabled(rt, prettyPrintedRegionName, MasterSwitchType.MERGE)
+            && rt.getEventType() == EventType.RS_ZK_REQUEST_REGION_MERGE)
+            || !handleRegionMerging(rt, encodedName, prettyPrintedRegionName, sn)) {
           deleteMergingNode(encodedName, sn);
         }
         break;
@@ -1222,7 +1229,25 @@ public class AssignmentManager extends ZooKeeperListener {
     }
   }
 
-  //For unit tests only
+  /**
+   * Check whether region split or merge enabled.
+   * @param rt Region transition info
+   * @param prettyPrintedRegionName Region name
+   * @param switchType Region operation type
+   * @param eventType Event type
+   * @return true if region split/merge enabled
+   */
+  private boolean isRegionSplitOrMergeEnabled(RegionTransition rt, String prettyPrintedRegionName,
+      MasterSwitchType switchType) {
+    if (!((HMaster) server).getSplitOrMergeTracker().isSplitOrMergeEnabled(switchType)) {
+      LOG.warn("Region " + switchType + " not enabled, skipping " + rt.getEventType()
+          + " of reigon " + prettyPrintedRegionName);
+      return false;
+    }
+    return true;
+  }
+
+  // For unit tests only
   boolean wasClosedHandlerCalled(HRegionInfo hri) {
     AtomicBoolean b = closedRegionHandlerCalled.get(hri);
     //compareAndSet to be sure that unit tests don't see stale values. Means,
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
index 477be1e..10214de 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatus.java
@@ -46,7 +46,7 @@ import static org.junit.Assert.assertTrue;
 public class TestSplitOrMergeStatus {
 
   private static final Log LOG = LogFactory.getLog(TestSplitOrMergeStatus.class);
-  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static byte [] FAMILY = Bytes.toBytes("testFamily");
 
   /**
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatusWithZKBasedAssignment.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatusWithZKBasedAssignment.java
new file mode 100644
index 0000000..8fb3d34
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitOrMergeStatusWithZKBasedAssignment.java
@@ -0,0 +1,35 @@
+/**
+ * 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 org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+@Category({ MediumTests.class, ClientTests.class })
+public class TestSplitOrMergeStatusWithZKBasedAssignment extends TestSplitOrMergeStatus {
+  /**
+   * @throws java.lang.Exception
+   */
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.getConfiguration().setBoolean("hbase.assignment.usezk", true);
+    TEST_UTIL.startMiniCluster(2);
+  }
+}
\ No newline at end of file