You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by en...@apache.org on 2015/06/21 02:20:59 UTC

hbase git commit: HBASE-13935 Orphaned namespace table ZK node should not prevent master to start (Stephen Yuan Jiang)

Repository: hbase
Updated Branches:
  refs/heads/branch-1 34cd3377f -> eb9b234ce


HBASE-13935 Orphaned namespace table ZK node should not prevent master to start (Stephen Yuan Jiang)


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

Branch: refs/heads/branch-1
Commit: eb9b234ce2582010c4bc011a58b3e49626963040
Parents: 34cd337
Author: Enis Soztutar <en...@apache.org>
Authored: Sat Jun 20 17:20:38 2015 -0700
Committer: Enis Soztutar <en...@apache.org>
Committed: Sat Jun 20 17:20:38 2015 -0700

----------------------------------------------------------------------
 .../master/handler/CreateTableHandler.java      | 16 ++++-
 .../master/procedure/CreateTableProcedure.java  | 17 +++--
 .../procedure/TestCreateTableProcedure2.java    | 68 ++++++++++++++++++++
 3 files changed, 93 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/eb9b234c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java
index 46dbe21..a639407 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java
@@ -121,7 +121,12 @@ public class CreateTableHandler extends EventHandler {
         throw new TableExistsException(tableName);
       }
 
-      checkAndSetEnablingTable(assignmentManager, tableName);
+      // During master initialization, the ZK state could be inconsistent from failed DDL
+      // in the past. If we fail here, it would prevent master to start.  We should force
+      // setting the system table state regardless the table state.
+      boolean skipTableStateCheck =
+          !((HMaster) this.server).isInitialized() && tableName.isSystemTable();
+      checkAndSetEnablingTable(assignmentManager, tableName, skipTableStateCheck);
       success = true;
     } finally {
       if (!success) {
@@ -132,7 +137,7 @@ public class CreateTableHandler extends EventHandler {
   }
 
   static void checkAndSetEnablingTable(final AssignmentManager assignmentManager,
-      final TableName tableName) throws IOException {
+      final TableName tableName, boolean skipTableStateCheck) throws IOException {
     // If we have multiple client threads trying to create the table at the
     // same time, given the async nature of the operation, the table
     // could be in a state where hbase:meta table hasn't been updated yet in
@@ -144,7 +149,12 @@ public class CreateTableHandler extends EventHandler {
     // We could have cleared the hbase.rootdir and not zk.  How can we detect this case?
     // Having to clean zk AND hdfs is awkward.
     try {
-      if (!assignmentManager.getTableStateManager().setTableStateIfNotInStates(tableName,
+      if (skipTableStateCheck) {
+        assignmentManager.getTableStateManager().setTableState(
+          tableName,
+          ZooKeeperProtos.Table.State.ENABLING);
+      } else if (!assignmentManager.getTableStateManager().setTableStateIfNotInStates(
+        tableName,
         ZooKeeperProtos.Table.State.ENABLING,
         ZooKeeperProtos.Table.State.ENABLING,
         ZooKeeperProtos.Table.State.ENABLED)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb9b234c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
index a791b70..150f01b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
@@ -287,13 +287,20 @@ public class CreateTableProcedure
       setFailure("master-create-table", new TableExistsException(getTableName()));
       return false;
     }
-    TableStateManager tsm = env.getMasterServices().getAssignmentManager().getTableStateManager();
-    if (tsm.isTableState(tableName, true, ZooKeeperProtos.Table.State.ENABLING,
+    // During master initialization, the ZK state could be inconsistent from failed DDL
+    // in the past. If we fail here, it would prevent master to start.  We should force
+    // setting the system table state regardless the table state.
+    boolean skipTableStateCheck =
+        !(env.getMasterServices().isInitialized()) && tableName.isSystemTable();
+    if (!skipTableStateCheck) {
+      TableStateManager tsm = env.getMasterServices().getAssignmentManager().getTableStateManager();
+      if (tsm.isTableState(tableName, true, ZooKeeperProtos.Table.State.ENABLING,
           ZooKeeperProtos.Table.State.ENABLED)) {
-      LOG.warn("The table " + tableName + " does not exist in meta but has a znode. " +
+        LOG.warn("The table " + tableName + " does not exist in meta but has a znode. " +
                "run hbck to fix inconsistencies.");
-      setFailure("master-create-table", new TableExistsException(getTableName()));
-      return false;
+        setFailure("master-create-table", new TableExistsException(getTableName()));
+        return false;
+      }
     }
     return true;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/eb9b234c/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure2.java
new file mode 100644
index 0000000..3a8795a
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestCreateTableProcedure2.java
@@ -0,0 +1,68 @@
+/**
+ *
+ * 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.procedure;
+
+import static org.junit.Assert.assertTrue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
+import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.junit.After;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(MediumTests.class)
+public class TestCreateTableProcedure2 {
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final Log LOG = LogFactory.getLog(TestCreateTableProcedure2.class);
+
+  @After
+  public void tearDown() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+    TEST_UTIL.shutdownMiniZKCluster();
+  }
+
+  @Test
+  public void testMasterRestartAfterNameSpaceEnablingNodeIsCreated() throws Exception {
+    // Step 1: start mini zk cluster.
+    MiniZooKeeperCluster zkCluster;
+    zkCluster = TEST_UTIL.startMiniZKCluster();
+    // Step 2: add an orphaned system table ZNODE
+    TableName tableName = TableName.valueOf("hbase:namespace");
+    ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
+    String znode = ZKUtil.joinZNode(zkw.tableZNode, tableName.getNameAsString());
+    ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder();
+    builder.setState(ZooKeeperProtos.Table.State.ENABLED);
+    byte [] data = ProtobufUtil.prependPBMagic(builder.build().toByteArray());
+    ZKUtil.createSetData(zkw, znode, data);
+    LOG.info("Create an orphaned Znode " + znode + " with data " + data);
+    // Step 3: link the zk cluster to hbase cluster
+    TEST_UTIL.setZkCluster(zkCluster);
+    // Step 4: start hbase cluster and expect master to start successfully.
+    TEST_UTIL.startMiniCluster();
+    assertTrue(TEST_UTIL.getHBaseCluster().getLiveMasterThreads().size() == 1);
+  }
+}