You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ch...@apache.org on 2018/02/27 20:48:09 UTC

hbase git commit: HBASE-19989 - READY_TO_MERGE and READY_TO_SPLIT do not update region state correctly

Repository: hbase
Updated Branches:
  refs/heads/branch-1 f8494ee4e -> 40e0ee331


HBASE-19989 - READY_TO_MERGE and READY_TO_SPLIT do not update region state correctly

Author: Ben Lau <be...@oath.com>


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

Branch: refs/heads/branch-1
Commit: 40e0ee331cc949f72cc44483275319673cb070b9
Parents: f8494ee
Author: Rahul Gidwani <ch...@apache.org>
Authored: Tue Feb 27 12:41:48 2018 -0800
Committer: Rahul Gidwani <ch...@apache.org>
Committed: Tue Feb 27 12:41:48 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/master/AssignmentManager.java  |  7 ++-
 .../regionserver/TestZKLessMergeOnCluster.java  | 45 ++++++++++++++++++++
 .../regionserver/TestZKLessSplitOnCluster.java  | 45 ++++++++++++++++++++
 3 files changed, 95 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/40e0ee33/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
----------------------------------------------------------------------
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 bc1e02f..e2acc09 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
@@ -4638,11 +4638,13 @@ public class AssignmentManager extends ZooKeeperListener {
         if (!((HMaster)server).getSplitOrMergeTracker().isSplitOrMergeEnabled(
                 Admin.MasterSwitchType.SPLIT)) {
           errorMsg = "split switch is off!";
+          break;
         }
       } catch (IOException exp) {
         errorMsg = StringUtils.stringifyException(exp);
+        break;
       }
-      break;
+      // Break out only for errors, otherwise fall through
     case SPLIT_PONR:
     case SPLIT:
       errorMsg =
@@ -4667,8 +4669,9 @@ public class AssignmentManager extends ZooKeeperListener {
       if (!((HMaster)server).getSplitOrMergeTracker().isSplitOrMergeEnabled(
               Admin.MasterSwitchType.MERGE)) {
         errorMsg = "merge switch is off!";
+        break;
       }
-      break;
+      // Break out only for errors, otherwise fall through
     case MERGE_PONR:
     case MERGED:
       errorMsg = onRegionMerge(serverName, code, hri,

http://git-wip-us.apache.org/repos/asf/hbase/blob/40e0ee33/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestZKLessMergeOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestZKLessMergeOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestZKLessMergeOnCluster.java
new file mode 100644
index 0000000..4900af8
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestZKLessMergeOnCluster.java
@@ -0,0 +1,45 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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.regionserver;
+
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Like {@link TestRegionMergeTransaction} in that we're testing
+ * {@link RegionMergeTransaction} only the below tests are against a running
+ * cluster where {@link TestRegionMergeTransaction} is tests against bare
+ * {@link HRegion}.
+ */
+@Category(LargeTests.class)
+public class TestZKLessMergeOnCluster extends TestRegionMergeTransactionOnCluster {
+  @BeforeClass
+  public static void beforeAllTests() throws Exception {
+    // Don't use ZK for region assignment
+    TEST_UTIL.getConfiguration().setBoolean("hbase.assignment.usezk", false);
+    setupOnce();
+  }
+
+  @AfterClass
+  public static void afterAllTests() throws Exception {
+    TestRegionMergeTransactionOnCluster.afterAllTests();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/40e0ee33/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestZKLessSplitOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestZKLessSplitOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestZKLessSplitOnCluster.java
new file mode 100644
index 0000000..1201c01
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestZKLessSplitOnCluster.java
@@ -0,0 +1,45 @@
+/**
+ *
+ * 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.regionserver;
+
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Like {@link TestSplitTransaction} in that we're testing {@link SplitTransaction}
+ * only the below tests are against a running cluster where {@link TestSplitTransaction}
+ * is tests against a bare {@link HRegion}.
+ */
+@Category(LargeTests.class)
+public class TestZKLessSplitOnCluster extends TestSplitTransactionOnCluster {
+  @BeforeClass
+  public static void before() throws Exception {
+    // Don't use ZK for region assignment
+    TESTING_UTIL.getConfiguration().setBoolean("hbase.assignment.usezk", false);
+    setupOnce();
+  }
+
+  @AfterClass
+  public static void after() throws Exception {
+    TestSplitTransactionOnCluster.after();
+  }
+}
+