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

[21/51] [abbrv] hbase git commit: HBASE-21154 Remove hbase:namespace table; fold it into hbase:meta

HBASE-21154 Remove hbase:namespace table; fold it into hbase:meta


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

Branch: refs/heads/HBASE-20952
Commit: 1acbd36c903b048141866b143507bfce124a5c5f
Parents: a0e3cb6
Author: zhangduo <zh...@apache.org>
Authored: Fri Nov 23 19:56:41 2018 +0800
Committer: Duo Zhang <zh...@apache.org>
Committed: Mon Nov 26 14:51:56 2018 +0800

----------------------------------------------------------------------
 .../hbase/client/TableDescriptorBuilder.java    |  38 +-
 .../org/apache/hadoop/hbase/HConstants.java     |   8 +-
 .../hadoop/hbase/NamespaceDescriptor.java       |   2 +-
 .../java/org/apache/hadoop/hbase/TableName.java |   9 +-
 .../src/main/protobuf/MasterProcedure.proto     |   9 +-
 .../hbase/rsgroup/RSGroupInfoManagerImpl.java   |  31 +-
 .../hbase/rsgroup/TestRSGroupsBasics.java       |   4 +-
 .../hbase/tmpl/master/MasterStatusTmpl.jamon    |   2 -
 .../apache/hadoop/hbase/ZKNamespaceManager.java | 215 -----------
 .../hbase/master/ClusterSchemaServiceImpl.java  |  30 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |  29 +-
 .../hbase/master/TableNamespaceManager.java     | 372 +++++--------------
 .../hbase/master/locking/LockProcedure.java     |   2 +-
 .../AbstractStateMachineNamespaceProcedure.java |  46 ++-
 .../procedure/CreateNamespaceProcedure.java     | 156 ++------
 .../procedure/DeleteNamespaceProcedure.java     | 217 +++--------
 .../master/procedure/InitMetaProcedure.java     |  42 +++
 .../procedure/MasterProcedureScheduler.java     |   9 +-
 .../procedure/ModifyNamespaceProcedure.java     | 109 ++----
 .../procedure/TableProcedureInterface.java      |   7 +
 .../hbase/master/procedure/TableQueue.java      |   4 +-
 .../hadoop/hbase/regionserver/HRegion.java      |   6 +-
 .../hbase/regionserver/HRegionServer.java       |  21 +-
 .../hbase/security/access/AccessController.java |   2 +-
 .../hadoop/hbase/util/FSTableDescriptors.java   |  11 +
 .../hadoop/hbase/HBaseTestingUtility.java       |   1 -
 .../hadoop/hbase/TestGlobalMemStoreSize.java    |   4 +-
 .../org/apache/hadoop/hbase/TestNamespace.java  |  19 +-
 .../client/TestAsyncNamespaceAdminApi.java      |  20 -
 .../hadoop/hbase/client/TestFromClientSide.java |   2 +-
 .../hadoop/hbase/master/AbstractTestDLS.java    |   8 +-
 .../hadoop/hbase/master/TestMasterMetrics.java  |  71 ++--
 .../TestMasterOperationsForRegionReplicas.java  |  10 +-
 .../TestMasterRestartAfterDisablingTable.java   |  15 +-
 .../hbase/master/TestMigrateNamespaceTable.java | 108 ++++++
 .../hadoop/hbase/master/TestRestartCluster.java |   7 +-
 .../hadoop/hbase/master/TestRollingRestart.java |  10 +-
 .../assignment/TestRegionMoveAndAbandon.java    |  13 +-
 .../TestFavoredStochasticLoadBalancer.java      |   1 -
 .../procedure/TestMasterProcedureScheduler.java |   4 +-
 .../procedure/TestModifyNamespaceProcedure.java |   6 +-
 .../master/procedure/TestProcedurePriority.java |   2 -
 .../hbase/regionserver/TestLogRoller.java       |   5 +
 .../regionserver/TestPerColumnFamilyFlush.java  |   7 -
 .../TestRegionReplicasWithRestartScenarios.java |   2 +-
 .../regionserver/TestRegionServerMetrics.java   |   2 +-
 .../wal/AbstractTestLogRolling.java             |   3 -
 .../TestReplicationWALEntryFilters.java         |   7 -
 .../hadoop/hbase/util/TestHBaseFsckMOB.java     |   1 -
 49 files changed, 602 insertions(+), 1107 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1acbd36c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
index d1c3f78..22c71c1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
@@ -220,11 +220,28 @@ public class TableDescriptorBuilder {
     RESERVED_KEYWORDS.add(IS_META_KEY);
   }
 
+  /**
+   * @deprecated namespace table has been folded into the ns family in meta table, do not use this
+   *             any more.
+   */
   @InterfaceAudience.Private
+  @Deprecated
   public final static String NAMESPACE_FAMILY_INFO = "info";
+
+  /**
+   * @deprecated namespace table has been folded into the ns family in meta table, do not use this
+   *             any more.
+   */
   @InterfaceAudience.Private
+  @Deprecated
   public final static byte[] NAMESPACE_FAMILY_INFO_BYTES = Bytes.toBytes(NAMESPACE_FAMILY_INFO);
+
+  /**
+   * @deprecated namespace table has been folded into the ns family in meta table, do not use this
+   *             any more.
+   */
   @InterfaceAudience.Private
+  @Deprecated
   public final static byte[] NAMESPACE_COL_DESC_BYTES = Bytes.toBytes("d");
 
   /**
@@ -245,22 +262,21 @@ public class TableDescriptorBuilder {
       CP_HTD_ATTR_VALUE_PARAM_VALUE_PATTERN + "),?");
   private static final Pattern CP_HTD_ATTR_KEY_PATTERN =
     Pattern.compile("^coprocessor\\$([0-9]+)$", Pattern.CASE_INSENSITIVE);
+
   /**
    * Table descriptor for namespace table
+   * @deprecated We have folded the data in namespace table into meta table, so do not use it any
+   *             more.
    */
-  // TODO We used to set CacheDataInL1 for NS table. When we have BucketCache in file mode, now the
-  // NS data goes to File mode BC only. Test how that affect the system. If too much, we have to
-  // rethink about adding back the setCacheDataInL1 for NS table.
-  public static final TableDescriptor NAMESPACE_TABLEDESC
-    = TableDescriptorBuilder.newBuilder(TableName.NAMESPACE_TABLE_NAME)
+  @Deprecated
+  public static final TableDescriptor NAMESPACE_TABLEDESC =
+    TableDescriptorBuilder.newBuilder(TableName.NAMESPACE_TABLE_NAME)
       .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(NAMESPACE_FAMILY_INFO_BYTES)
-        // Ten is arbitrary number.  Keep versions to help debugging.
-        .setMaxVersions(10)
-        .setInMemory(true)
-        .setBlocksize(8 * 1024)
-        .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-        .build())
+        // Ten is arbitrary number. Keep versions to help debugging.
+        .setMaxVersions(10).setInMemory(true).setBlocksize(8 * 1024)
+        .setScope(HConstants.REPLICATION_SCOPE_LOCAL).build())
       .build();
+
   private final ModifyableTableDescriptor desc;
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/1acbd36c/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index fbfab4b..fdc3d82 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -26,7 +26,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.UUID;
 import java.util.regex.Pattern;
-
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -508,6 +507,13 @@ public final class HConstants {
   public static final byte[] REPLICATION_BARRIER_FAMILY =
       Bytes.toBytes(REPLICATION_BARRIER_FAMILY_STR);
 
+  /** The namespace family as a string */
+  public static final String NAMESPACE_FAMILY_STR = "ns";
+
+  /** The namespace family */
+  public static final byte[] NAMESPACE_FAMILY = Bytes.toBytes(NAMESPACE_FAMILY_STR);
+
+  public static final byte[] NAMESPACE_COL_DESC_QUALIFIER = Bytes.toBytes("d");
   /**
    * The meta table version column qualifier.
    * We keep current version of the meta table in this column in <code>-ROOT-</code>

http://git-wip-us.apache.org/repos/asf/hbase/blob/1acbd36c/hbase-common/src/main/java/org/apache/hadoop/hbase/NamespaceDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/NamespaceDescriptor.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/NamespaceDescriptor.java
index d1b69ba..07386b5 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/NamespaceDescriptor.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/NamespaceDescriptor.java
@@ -166,7 +166,7 @@ public class NamespaceDescriptor {
 
     private Builder(NamespaceDescriptor ns) {
       this.bName = ns.name;
-      this.bConfiguration = ns.configuration;
+      this.bConfiguration.putAll(ns.configuration);
     }
 
     private Builder(String name) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1acbd36c/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
index e6cabbc..093bf20 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
@@ -80,9 +80,14 @@ public final class TableName implements Comparable<TableName> {
   public static final TableName META_TABLE_NAME =
       valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta");
 
-  /** The Namespace table's name. */
+  /**
+   * The Namespace table's name.
+   * @deprecated We have folded the data in namespace table into meta table, so do not use it any
+   *             more.
+   */
+  @Deprecated
   public static final TableName NAMESPACE_TABLE_NAME =
-      valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "namespace");
+    valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "namespace");
 
   public static final String OLD_META_STR = ".META.";
   public static final String OLD_ROOT_STR = "-ROOT-";

http://git-wip-us.apache.org/repos/asf/hbase/blob/1acbd36c/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
index 8bb69cf..44ac952 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -117,7 +117,7 @@ enum CreateNamespaceState {
   CREATE_NAMESPACE_PREPARE = 1;
   CREATE_NAMESPACE_CREATE_DIRECTORY = 2;
   CREATE_NAMESPACE_INSERT_INTO_NS_TABLE = 3;
-  CREATE_NAMESPACE_UPDATE_ZK = 4;
+  CREATE_NAMESPACE_UPDATE_ZK = 4[deprecated=true];
   CREATE_NAMESPACE_SET_NAMESPACE_QUOTA = 5;
 }
 
@@ -128,7 +128,7 @@ message CreateNamespaceStateData {
 enum ModifyNamespaceState {
   MODIFY_NAMESPACE_PREPARE = 1;
   MODIFY_NAMESPACE_UPDATE_NS_TABLE = 2;
-  MODIFY_NAMESPACE_UPDATE_ZK = 3;
+  MODIFY_NAMESPACE_UPDATE_ZK = 3[deprecated=true];
 }
 
 message ModifyNamespaceStateData {
@@ -139,7 +139,7 @@ message ModifyNamespaceStateData {
 enum DeleteNamespaceState {
   DELETE_NAMESPACE_PREPARE = 1;
   DELETE_NAMESPACE_DELETE_FROM_NS_TABLE = 2;
-  DELETE_NAMESPACE_REMOVE_FROM_ZK = 3;
+  DELETE_NAMESPACE_REMOVE_FROM_ZK = 3[deprecated=true];
   DELETE_NAMESPACE_DELETE_DIRECTORIES = 4;
   DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA = 5;
 }
@@ -475,7 +475,8 @@ message ReopenTableRegionsStateData {
 }
 
 enum InitMetaState {
-   INIT_META_ASSIGN_META = 1;
+  INIT_META_ASSIGN_META = 1;
+  INIT_META_CREATE_NAMESPACES = 2;
 }
 
 message InitMetaStateData {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1acbd36c/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
index 4a881d3..cdd68d1 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.rsgroup;
 
 import com.google.protobuf.ServiceException;
-
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.util.ArrayList;
@@ -34,7 +33,6 @@ import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
 import java.util.concurrent.atomic.AtomicBoolean;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
@@ -85,6 +83,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
+
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
 
@@ -757,12 +756,9 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
         assignedRegions.clear();
         found.set(true);
         try {
-          conn.getTable(TableName.NAMESPACE_TABLE_NAME);
-          conn.getTable(RSGROUP_TABLE_NAME);
           boolean rootMetaFound =
               masterServices.getMetaTableLocator().verifyMetaRegionLocation(
                   conn, masterServices.getZooKeeper(), 1);
-          final AtomicBoolean nsFound = new AtomicBoolean(false);
           if (rootMetaFound) {
             MetaTableAccessor.Visitor visitor = new DefaultVisitorBase() {
               @Override
@@ -791,36 +787,13 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
                     }
                     foundRegions.add(info);
                   }
-                  if (TableName.NAMESPACE_TABLE_NAME.equals(info.getTable())) {
-                    Cell cell = row.getColumnLatestCell(HConstants.CATALOG_FAMILY,
-                        HConstants.SERVER_QUALIFIER);
-                    ServerName sn = null;
-                    if(cell != null) {
-                      sn = ServerName.parseVersionedServerName(CellUtil.cloneValue(cell));
-                    }
-                    if (sn == null) {
-                      nsFound.set(false);
-                    } else if (tsm.isTableState(TableName.NAMESPACE_TABLE_NAME,
-                        TableState.State.ENABLED)) {
-                      try {
-                        ClientProtos.ClientService.BlockingInterface rs = conn.getClient(sn);
-                        ClientProtos.GetRequest request =
-                            RequestConverter.buildGetRequest(info.getRegionName(),
-                                new Get(ROW_KEY));
-                        rs.get(null, request);
-                        nsFound.set(true);
-                      } catch(Exception ex) {
-                        LOG.debug("Caught exception while verifying group region", ex);
-                      }
-                    }
-                  }
                 }
                 return true;
               }
             };
             MetaTableAccessor.fullScanRegions(conn, visitor);
             // if no regions in meta then we have to create the table
-            if (foundRegions.size() < 1 && rootMetaFound && !createSent && nsFound.get()) {
+            if (foundRegions.size() < 1 && rootMetaFound && !createSent) {
               createRSGroupTable();
               createSent = true;
             }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1acbd36c/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBasics.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBasics.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBasics.java
index 7415ab5..3860c59 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBasics.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBasics.java
@@ -93,8 +93,8 @@ public class TestRSGroupsBasics extends TestRSGroupsBase {
     assertEquals(4, defaultInfo.getServers().size());
     // Assignment of root and meta regions.
     int count = master.getAssignmentManager().getRegionStates().getRegionAssignments().size();
-    //3 meta,namespace, group
-    assertEquals(3, count);
+    // 2 meta, group
+    assertEquals(2, count);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/1acbd36c/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
index 03056e1..7bb6c40 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
@@ -447,8 +447,6 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
             description = "The hbase:acl table holds information about acl.";
         } else if (tableName.equals(VisibilityConstants.LABELS_TABLE_NAME)){
             description = "The hbase:labels table holds information about visibility labels.";
-        } else if (tableName.equals(TableName.NAMESPACE_TABLE_NAME)){
-            description = "The hbase:namespace table holds information about namespaces.";
         } else if (tableName.equals(QuotaUtil.QUOTA_TABLE_NAME)){
             description = "The hbase:quota table holds quota information about number" +
             " or size of requests in a given time frame.";

http://git-wip-us.apache.org/repos/asf/hbase/blob/1acbd36c/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java
deleted file mode 100644
index 101c40e..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ZKNamespaceManager.java
+++ /dev/null
@@ -1,215 +0,0 @@
-/**
- * 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;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.NavigableMap;
-import java.util.NavigableSet;
-import java.util.concurrent.ConcurrentSkipListMap;
-
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZKListener;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-
-/**
- * Class servers two purposes:
- *
- * 1. Broadcast NamespaceDescriptor information via ZK
- * (Done by the Master)
- * 2. Consume broadcasted NamespaceDescriptor changes
- * (Done by the RegionServers)
- *
- */
-@InterfaceAudience.Private
-public class ZKNamespaceManager extends ZKListener {
-  private static final Logger LOG = LoggerFactory.getLogger(ZKNamespaceManager.class);
-  private final String nsZNode;
-  private final NavigableMap<String,NamespaceDescriptor> cache;
-
-  public ZKNamespaceManager(ZKWatcher zkw) throws IOException {
-    super(zkw);
-    nsZNode = zkw.getZNodePaths().namespaceZNode;
-    cache = new ConcurrentSkipListMap<>();
-  }
-
-  public void start() throws IOException {
-    watcher.registerListener(this);
-    try {
-      if (ZKUtil.watchAndCheckExists(watcher, nsZNode)) {
-        List<ZKUtil.NodeAndData> existing =
-            ZKUtil.getChildDataAndWatchForNewChildren(watcher, nsZNode);
-        if (existing != null) {
-          refreshNodes(existing);
-        }
-      } else {
-        ZKUtil.createWithParents(watcher, nsZNode);
-      }
-    } catch (KeeperException e) {
-      throw new IOException("Failed to initialize ZKNamespaceManager", e);
-    }
-  }
-
-  public void stop() throws IOException {
-    this.watcher.unregisterListener(this);
-  }
-
-  public NamespaceDescriptor get(String name) {
-    return cache.get(name);
-  }
-
-  public void update(NamespaceDescriptor ns) throws IOException {
-    writeNamespace(ns);
-    cache.put(ns.getName(), ns);
-  }
-
-  public void remove(String name) throws IOException {
-    deleteNamespace(name);
-    cache.remove(name);
-  }
-
-  public NavigableSet<NamespaceDescriptor> list() throws IOException {
-    NavigableSet<NamespaceDescriptor> ret =
-        Sets.newTreeSet(NamespaceDescriptor.NAMESPACE_DESCRIPTOR_COMPARATOR);
-    for(NamespaceDescriptor ns: cache.values()) {
-      ret.add(ns);
-    }
-    return ret;
-  }
-
-  @Override
-  public void nodeCreated(String path) {
-    if (nsZNode.equals(path)) {
-      try {
-        List<ZKUtil.NodeAndData> nodes =
-            ZKUtil.getChildDataAndWatchForNewChildren(watcher, nsZNode);
-        refreshNodes(nodes);
-      } catch (KeeperException ke) {
-        String msg = "Error reading data from zookeeper";
-        LOG.error(msg, ke);
-        watcher.abort(msg, ke);
-      } catch (IOException e) {
-        String msg = "Error parsing data from zookeeper";
-        LOG.error(msg, e);
-        watcher.abort(msg, e);
-      }
-    }
-  }
-
-  @Override
-  public void nodeDeleted(String path) {
-    if (nsZNode.equals(ZKUtil.getParent(path))) {
-      String nsName = ZKUtil.getNodeName(path);
-      cache.remove(nsName);
-    }
-  }
-
-  @Override
-  public void nodeDataChanged(String path) {
-    if (nsZNode.equals(ZKUtil.getParent(path))) {
-      try {
-        byte[] data = ZKUtil.getDataAndWatch(watcher, path);
-        NamespaceDescriptor ns =
-            ProtobufUtil.toNamespaceDescriptor(
-                HBaseProtos.NamespaceDescriptor.parseFrom(data));
-        cache.put(ns.getName(), ns);
-      } catch (KeeperException ke) {
-        String msg = "Error reading data from zookeeper for node "+path;
-        LOG.error(msg, ke);
-        // only option is to abort
-        watcher.abort(msg, ke);
-      } catch (IOException ioe) {
-        String msg = "Error deserializing namespace: "+path;
-        LOG.error(msg, ioe);
-        watcher.abort(msg, ioe);
-      }
-    }
-  }
-
-  @Override
-  public void nodeChildrenChanged(String path) {
-    if (nsZNode.equals(path)) {
-      try {
-        List<ZKUtil.NodeAndData> nodes =
-            ZKUtil.getChildDataAndWatchForNewChildren(watcher, nsZNode);
-        refreshNodes(nodes);
-      } catch (KeeperException ke) {
-        LOG.error("Error reading data from zookeeper for path "+path, ke);
-        watcher.abort("ZooKeeper error get node children for path "+path, ke);
-      } catch (IOException e) {
-        LOG.error("Error deserializing namespace child from: "+path, e);
-        watcher.abort("Error deserializing namespace child from: " + path, e);
-      }
-    }
-  }
-
-  private void deleteNamespace(String name) throws IOException {
-    String zNode = ZNodePaths.joinZNode(nsZNode, name);
-    try {
-      ZKUtil.deleteNode(watcher, zNode);
-    } catch (KeeperException e) {
-      if (e instanceof KeeperException.NoNodeException) {
-        // If the node does not exist, it could be already deleted. Continue without fail.
-        LOG.warn("The ZNode " + zNode + " for namespace " + name + " does not exist.");
-      } else {
-        LOG.error("Failed updating permissions for namespace " + name, e);
-        throw new IOException("Failed updating permissions for namespace " + name, e);
-      }
-    }
-  }
-
-  private void writeNamespace(NamespaceDescriptor ns) throws IOException {
-    String zNode = ZNodePaths.joinZNode(nsZNode, ns.getName());
-    try {
-      ZKUtil.createWithParents(watcher, zNode);
-      ZKUtil.updateExistingNodeData(watcher, zNode,
-          ProtobufUtil.toProtoNamespaceDescriptor(ns).toByteArray(), -1);
-    } catch (KeeperException e) {
-      LOG.error("Failed updating permissions for namespace "+ns.getName(), e);
-      throw new IOException("Failed updating permissions for namespace "+ns.getName(), e);
-    }
-  }
-
-  private void refreshNodes(List<ZKUtil.NodeAndData> nodes) throws IOException {
-    for (ZKUtil.NodeAndData n : nodes) {
-      if (n.isEmpty()) continue;
-      String path = n.getNode();
-      String namespace = ZKUtil.getNodeName(path);
-      byte[] nodeData = n.getData();
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Updating namespace cache from node " + namespace + " with data: " +
-            Bytes.toStringBinary(nodeData));
-      }
-      NamespaceDescriptor ns =
-          ProtobufUtil.toNamespaceDescriptor(
-              HBaseProtos.NamespaceDescriptor.parseFrom(nodeData));
-      cache.put(ns.getName(), ns);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/1acbd36c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
index 5af7614..1e631b0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
@@ -18,15 +18,10 @@
 package org.apache.hadoop.hbase.master;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
-import java.util.Set;
-
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NamespaceNotFoundException;
 import org.apache.hadoop.hbase.ServiceNotRunningException;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
@@ -34,15 +29,17 @@ import org.apache.hadoop.hbase.master.procedure.ModifyNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.AbstractService;
 import org.apache.hadoop.hbase.util.NonceKey;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.AbstractService;
 
 @InterfaceAudience.Private
 class ClusterSchemaServiceImpl extends AbstractService implements ClusterSchemaService {
+
   private final TableNamespaceManager tableNamespaceManager;
   private final MasterServices masterServices;
-  private final static List<NamespaceDescriptor> EMPTY_NAMESPACE_LIST =
-    Collections.unmodifiableList(new ArrayList<NamespaceDescriptor>(0));
 
   ClusterSchemaServiceImpl(final MasterServices masterServices) {
     this.masterServices = masterServices;
@@ -51,9 +48,10 @@ class ClusterSchemaServiceImpl extends AbstractService implements ClusterSchemaS
 
   // All below are synchronized so consistent view on whether running or not.
 
-
   private synchronized void checkIsRunning() throws ServiceNotRunningException {
-    if (!isRunning()) throw new ServiceNotRunningException();
+    if (!isRunning()) {
+      throw new ServiceNotRunningException();
+    }
   }
 
   @Override
@@ -70,10 +68,6 @@ class ClusterSchemaServiceImpl extends AbstractService implements ClusterSchemaS
   protected void doStop() {
     // This is no stop for the table manager.
     notifyStopped();
-    TableNamespaceManager tnsm = getTableNamespaceManager();
-    if (tnsm != null) {
-      tnsm.stop("Stopping");
-    }
   }
 
   @Override
@@ -123,10 +117,8 @@ class ClusterSchemaServiceImpl extends AbstractService implements ClusterSchemaS
   @Override
   public List<NamespaceDescriptor> getNamespaces() throws IOException {
     checkIsRunning();
-    Set<NamespaceDescriptor> set = getTableNamespaceManager().list();
-    if (set == null || set.isEmpty()) return EMPTY_NAMESPACE_LIST;
-    List<NamespaceDescriptor> list = new ArrayList<>(set.size());
-    list.addAll(set);
-    return Collections.unmodifiableList(list);
+    return getTableNamespaceManager().list().stream()
+      .sorted(NamespaceDescriptor.NAMESPACE_DESCRIPTOR_COMPARATOR)
+      .collect(ImmutableList.toImmutableList());
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1acbd36c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index ae04283..428030d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -1112,10 +1112,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     getChoreService().scheduleChore(catalogJanitorChore);
     this.serverManager.startChore();
 
-    // NAMESPACE READ!!!!
-    // Here we expect hbase:namespace to be online. See inside initClusterSchemaService.
-    // TODO: Fix this. Namespace is a pain being a sort-of system table. Fold it in to hbase:meta.
-    // isNamespace does like isMeta and waits until namespace is onlined before allowing progress.
+    // Only for rolling upgrade, where we need to migrate the data in namespace table to meta table.
     if (!waitForNamespaceOnline()) {
       return;
     }
@@ -1243,20 +1240,28 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   /**
    * Check hbase:namespace table is assigned. If not, startup will hang looking for the ns table
-   * (TODO: Fix this! NS should not hold-up startup).
+   * <p/>
+   * This is for rolling upgrading, later we will migrate the data in ns table to the ns family of
+   * meta table. And if this is a new clsuter, this method will return immediately as there will be
+   * no namespace table/region.
    * @return True if namespace table is up/online.
    */
-  @VisibleForTesting
-  public boolean waitForNamespaceOnline() throws InterruptedException {
-    List<RegionInfo> ris = this.assignmentManager.getRegionStates().
-        getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME);
+  private boolean waitForNamespaceOnline() throws InterruptedException, IOException {
+    TableState nsTableState =
+      MetaTableAccessor.getTableState(getClusterConnection(), TableName.NAMESPACE_TABLE_NAME);
+    if (nsTableState == null || nsTableState.isDisabled()) {
+      // this means we have already migrated the data and disabled or deleted the namespace table,
+      // or this is a new depliy which does not have a namespace table from the beginning.
+      return true;
+    }
+    List<RegionInfo> ris =
+      this.assignmentManager.getRegionStates().getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME);
     if (ris.isEmpty()) {
-      // If empty, means we've not assigned the namespace table yet... Just return true so startup
-      // continues and the namespace table gets created.
+      // maybe this will not happen any more, but anyway, no harm to add a check here...
       return true;
     }
     // Else there are namespace regions up in meta. Ensure they are assigned before we go on.
-    for (RegionInfo ri: ris) {
+    for (RegionInfo ri : ris) {
       isRegionOnline(ri);
     }
     return true;

http://git-wip-us.apache.org/repos/asf/hbase/blob/1acbd36c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
index aefeebe..21178e5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java
@@ -19,324 +19,159 @@
 package org.apache.hadoop.hbase.master;
 
 import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.NavigableSet;
-
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.stream.Collectors;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellBuilderFactory;
-import org.apache.hadoop.hbase.CellBuilderType;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.ZKNamespaceManager;
+import org.apache.hadoop.hbase.client.BufferedMutator;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
-import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
-import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
-import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.Threads;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.protobuf.CodedInputStream;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 
 /**
- * This is a helper class used internally to manage the namespace metadata that is stored in
- * TableName.NAMESPACE_TABLE_NAME. It also mirrors updates to the ZK store by forwarding updates to
- * {@link org.apache.hadoop.hbase.ZKNamespaceManager}.
- *
- * WARNING: Do not use. Go via the higher-level {@link ClusterSchema} API instead. This manager
- * is likely to go aways anyways.
+ * This is a helper class used internally to manage the namespace metadata that is stored in the ns
+ * family in meta table.
  */
 @InterfaceAudience.Private
-@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IS2_INCONSISTENT_SYNC",
-  justification="TODO: synchronize access on nsTable but it is done in tiers above and this " +
-    "class is going away/shrinking")
-public class TableNamespaceManager implements Stoppable {
-  private static final Logger LOG = LoggerFactory.getLogger(TableNamespaceManager.class);
-  private volatile boolean stopped = false;
-
-  private Configuration conf;
-  private MasterServices masterServices;
-  private Table nsTable = null; // FindBugs: IS2_INCONSISTENT_SYNC TODO: Access is not synchronized
-  private ZKNamespaceManager zkNamespaceManager;
-  private boolean initialized;
+public class TableNamespaceManager {
 
   public static final String KEY_MAX_REGIONS = "hbase.namespace.quota.maxregions";
   public static final String KEY_MAX_TABLES = "hbase.namespace.quota.maxtables";
   static final String NS_INIT_TIMEOUT = "hbase.master.namespace.init.timeout";
   static final int DEFAULT_NS_INIT_TIMEOUT = 300000;
 
+  private final ConcurrentMap<String, NamespaceDescriptor> cache = new ConcurrentHashMap<>();
+
+  private final MasterServices masterServices;
+
   TableNamespaceManager(MasterServices masterServices) {
     this.masterServices = masterServices;
-    this.conf = masterServices.getConfiguration();
   }
 
-  public void start() throws IOException {
-    if (!MetaTableAccessor.tableExists(masterServices.getConnection(),
-        TableName.NAMESPACE_TABLE_NAME)) {
-      LOG.info("Namespace table not found. Creating...");
-      createNamespaceTable(masterServices);
+  private void migrateNamespaceTable() throws IOException {
+    try (Table nsTable = masterServices.getConnection().getTable(TableName.NAMESPACE_TABLE_NAME);
+      ResultScanner scanner = nsTable.getScanner(
+        new Scan().addFamily(TableDescriptorBuilder.NAMESPACE_FAMILY_INFO_BYTES).readAllVersions());
+      BufferedMutator mutator =
+        masterServices.getConnection().getBufferedMutator(TableName.META_TABLE_NAME)) {
+      for (Result result;;) {
+        result = scanner.next();
+        if (result == null) {
+          break;
+        }
+        Put put = new Put(result.getRow());
+        result
+          .getColumnCells(TableDescriptorBuilder.NAMESPACE_FAMILY_INFO_BYTES,
+            TableDescriptorBuilder.NAMESPACE_COL_DESC_BYTES)
+          .forEach(c -> put.addColumn(HConstants.NAMESPACE_FAMILY,
+            HConstants.NAMESPACE_COL_DESC_QUALIFIER, c.getTimestamp(), CellUtil.cloneValue(c)));
+        mutator.mutate(put);
+      }
     }
+    // schedule a disable procedure instead of block waiting here, as when disabling a table we will
+    // wait until master is initialized, but we are part of the initialization...
+    masterServices.getMasterProcedureExecutor().submitProcedure(
+      new DisableTableProcedure(masterServices.getMasterProcedureExecutor().getEnvironment(),
+        TableName.NAMESPACE_TABLE_NAME, false));
+  }
 
-    try {
-      // Wait for the namespace table to be initialized.
-      long startTime = EnvironmentEdgeManager.currentTime();
-      int timeout = conf.getInt(NS_INIT_TIMEOUT, DEFAULT_NS_INIT_TIMEOUT);
-      while (!isTableAvailableAndInitialized()) {
-        if (EnvironmentEdgeManager.currentTime() - startTime + 100 > timeout) {
-          // We can't do anything if ns is not online.
-          throw new IOException("Timedout " + timeout + "ms waiting for namespace table to "
-              + "be assigned and enabled: " + getTableState());
+  private void loadNamespaceIntoCache() throws IOException {
+    try (Table table = masterServices.getConnection().getTable(TableName.META_TABLE_NAME);
+      ResultScanner scanner = table.getScanner(HConstants.NAMESPACE_FAMILY)) {
+      for (Result result;;) {
+        result = scanner.next();
+        if (result == null) {
+          break;
         }
-        Thread.sleep(100);
+        Cell cell = result.getColumnLatestCell(HConstants.NAMESPACE_FAMILY,
+          HConstants.NAMESPACE_COL_DESC_QUALIFIER);
+        NamespaceDescriptor ns = ProtobufUtil
+          .toNamespaceDescriptor(HBaseProtos.NamespaceDescriptor.parseFrom(CodedInputStream
+            .newInstance(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength())));
+        cache.put(ns.getName(), ns);
       }
-    } catch (InterruptedException e) {
-      throw (InterruptedIOException) new InterruptedIOException().initCause(e);
     }
   }
 
-  private synchronized Table getNamespaceTable() throws IOException {
-    if (!isTableNamespaceManagerInitialized()) {
-      throw new IOException(this.getClass().getName() + " isn't ready to serve");
+  public void start() throws IOException {
+    TableState nsTableState = MetaTableAccessor.getTableState(masterServices.getConnection(),
+      TableName.NAMESPACE_TABLE_NAME);
+    if (nsTableState != null && nsTableState.isEnabled()) {
+      migrateNamespaceTable();
     }
-    return nsTable;
+    loadNamespaceIntoCache();
   }
 
-  /*
+  /**
    * check whether a namespace has already existed.
    */
-  public boolean doesNamespaceExist(final String namespaceName) throws IOException {
-    if (nsTable == null) {
-      throw new IOException(this.getClass().getName() + " isn't ready to serve");
-    }
-    return (get(nsTable, namespaceName) != null);
+  public boolean doesNamespaceExist(String namespaceName) throws IOException {
+    return cache.containsKey(namespaceName);
   }
 
-  public synchronized NamespaceDescriptor get(String name) throws IOException {
-    if (!isTableNamespaceManagerInitialized()) {
-      return null;
-    }
-    return zkNamespaceManager.get(name);
+  public NamespaceDescriptor get(String name) throws IOException {
+    return cache.get(name);
   }
 
-  private NamespaceDescriptor get(Table table, String name) throws IOException {
-    Result res = table.get(new Get(Bytes.toBytes(name)));
-    if (res.isEmpty()) {
-      return null;
-    }
-    byte[] val = CellUtil.cloneValue(res.getColumnLatestCell(
-        HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES, HTableDescriptor.NAMESPACE_COL_DESC_BYTES));
-    return
-        ProtobufUtil.toNamespaceDescriptor(
-            HBaseProtos.NamespaceDescriptor.parseFrom(val));
+  public void addOrUpdateNamespace(NamespaceDescriptor ns) throws IOException {
+    insertNamespaceToMeta(masterServices.getConnection(), ns);
+    cache.put(ns.getName(), ns);
   }
 
-  public void insertIntoNSTable(final NamespaceDescriptor ns) throws IOException {
-    if (nsTable == null) {
-      throw new IOException(this.getClass().getName() + " isn't ready to serve");
-    }
+  public static void insertNamespaceToMeta(Connection conn, NamespaceDescriptor ns)
+      throws IOException {
     byte[] row = Bytes.toBytes(ns.getName());
-    Put p = new Put(row, true);
-    p.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
-          .setRow(row)
-          .setFamily(TableDescriptorBuilder.NAMESPACE_FAMILY_INFO_BYTES)
-          .setQualifier(TableDescriptorBuilder.NAMESPACE_COL_DESC_BYTES)
-          .setTimestamp(p.getTimestamp())
-          .setType(Cell.Type.Put)
-          .setValue(ProtobufUtil.toProtoNamespaceDescriptor(ns).toByteArray())
-          .build());
-    nsTable.put(p);
-  }
-
-  public void updateZKNamespaceManager(final NamespaceDescriptor ns) throws IOException {
-    try {
-      zkNamespaceManager.update(ns);
-    } catch (IOException ex) {
-      String msg = "Failed to update namespace information in ZK.";
-      LOG.error(msg, ex);
-      throw new IOException(msg, ex);
+    Put put = new Put(row, true).addColumn(HConstants.NAMESPACE_FAMILY,
+      HConstants.NAMESPACE_COL_DESC_QUALIFIER,
+      ProtobufUtil.toProtoNamespaceDescriptor(ns).toByteArray());
+    try (Table table = conn.getTable(TableName.META_TABLE_NAME)) {
+      table.put(put);
     }
   }
 
-  public void removeFromNSTable(final String namespaceName) throws IOException {
-    if (nsTable == null) {
-      throw new IOException(this.getClass().getName() + " isn't ready to serve");
-    }
+  public void deleteNamespace(String namespaceName) throws IOException {
     Delete d = new Delete(Bytes.toBytes(namespaceName));
-    nsTable.delete(d);
-  }
-
-  public void removeFromZKNamespaceManager(final String namespaceName) throws IOException {
-    zkNamespaceManager.remove(namespaceName);
-  }
-
-  public synchronized NavigableSet<NamespaceDescriptor> list() throws IOException {
-    NavigableSet<NamespaceDescriptor> ret =
-        Sets.newTreeSet(NamespaceDescriptor.NAMESPACE_DESCRIPTOR_COMPARATOR);
-    ResultScanner scanner =
-        getNamespaceTable().getScanner(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES);
-    try {
-      for(Result r : scanner) {
-        byte[] val = CellUtil.cloneValue(r.getColumnLatestCell(
-          HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
-          HTableDescriptor.NAMESPACE_COL_DESC_BYTES));
-        ret.add(ProtobufUtil.toNamespaceDescriptor(
-            HBaseProtos.NamespaceDescriptor.parseFrom(val)));
-      }
-    } finally {
-      scanner.close();
-    }
-    return ret;
-  }
-
-  private void createNamespaceTable(MasterServices masterServices) throws IOException {
-    masterServices.createSystemTable(HTableDescriptor.NAMESPACE_TABLEDESC);
-  }
-
-  @SuppressWarnings("deprecation")
-  private boolean isTableNamespaceManagerInitialized() throws IOException {
-    if (initialized) {
-      this.nsTable = this.masterServices.getConnection().getTable(TableName.NAMESPACE_TABLE_NAME);
-      return true;
-    }
-    return false;
-  }
-
-  /**
-   * Create Namespace in a blocking manner. Keeps trying until
-   * {@link ClusterSchema#HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY} expires.
-   * Note, by-passes notifying coprocessors and name checks. Use for system namespaces only.
-   */
-  private void blockingCreateNamespace(final NamespaceDescriptor namespaceDescriptor)
-      throws IOException {
-    ClusterSchema clusterSchema = this.masterServices.getClusterSchema();
-    long procId = clusterSchema.createNamespace(namespaceDescriptor, null, ProcedurePrepareLatch.getNoopLatch());
-    block(this.masterServices, procId);
-  }
-
-
-  /**
-   * An ugly utility to be removed when refactor TableNamespaceManager.
-   * @throws TimeoutIOException
-   */
-  private static void block(final MasterServices services, final long procId)
-  throws TimeoutIOException {
-    int timeoutInMillis = services.getConfiguration().
-        getInt(ClusterSchema.HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY,
-            ClusterSchema.DEFAULT_HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT);
-    long deadlineTs = EnvironmentEdgeManager.currentTime() + timeoutInMillis;
-    ProcedureExecutor<MasterProcedureEnv> procedureExecutor =
-        services.getMasterProcedureExecutor();
-    while(EnvironmentEdgeManager.currentTime() < deadlineTs) {
-      if (procedureExecutor.isFinished(procId)) return;
-      // Sleep some
-      Threads.sleep(10);
-    }
-    throw new TimeoutIOException("Procedure pid=" + procId + " is still running");
-  }
-
-  /**
-   * This method checks if the namespace table is assigned and then
-   * tries to create its Table reference. If it was already created before, it also makes
-   * sure that the connection isn't closed.
-   * @return true if the namespace table manager is ready to serve, false otherwise
-   */
-  @SuppressWarnings("deprecation")
-  public synchronized boolean isTableAvailableAndInitialized()
-  throws IOException {
-    // Did we already get a table? If so, still make sure it's available
-    if (isTableNamespaceManagerInitialized()) {
-      return true;
-    }
-
-    // Now check if the table is assigned, if not then fail fast
-    if (isTableAssigned() && isTableEnabled()) {
-      try {
-        boolean initGoodSofar = true;
-        nsTable = this.masterServices.getConnection().getTable(TableName.NAMESPACE_TABLE_NAME);
-        zkNamespaceManager = new ZKNamespaceManager(masterServices.getZooKeeper());
-        zkNamespaceManager.start();
-
-        if (get(nsTable, NamespaceDescriptor.DEFAULT_NAMESPACE.getName()) == null) {
-          blockingCreateNamespace(NamespaceDescriptor.DEFAULT_NAMESPACE);
-        }
-        if (get(nsTable, NamespaceDescriptor.SYSTEM_NAMESPACE.getName()) == null) {
-          blockingCreateNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE);
-        }
-
-        if (!initGoodSofar) {
-          // some required namespace is created asynchronized. We should complete init later.
-          return false;
-        }
-
-        ResultScanner scanner = nsTable.getScanner(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES);
-        try {
-          for (Result result : scanner) {
-            byte[] val =  CellUtil.cloneValue(result.getColumnLatestCell(
-                HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
-                HTableDescriptor.NAMESPACE_COL_DESC_BYTES));
-            NamespaceDescriptor ns =
-                ProtobufUtil.toNamespaceDescriptor(
-                    HBaseProtos.NamespaceDescriptor.parseFrom(val));
-            zkNamespaceManager.update(ns);
-          }
-        } finally {
-          scanner.close();
-        }
-        initialized = true;
-        return true;
-      } catch (IOException ie) {
-        LOG.warn("Caught exception in initializing namespace table manager", ie);
-        if (nsTable != null) {
-          nsTable.close();
-        }
-        throw ie;
-      }
+    try (Table table = masterServices.getConnection().getTable(TableName.META_TABLE_NAME)) {
+      table.delete(d);
     }
-    return false;
-  }
-
-  private TableState getTableState() throws IOException {
-    return masterServices.getTableStateManager().getTableState(TableName.NAMESPACE_TABLE_NAME);
-  }
-
-  private boolean isTableEnabled() throws IOException {
-    return getTableState().isEnabled();
+    cache.remove(namespaceName);
   }
 
-  private boolean isTableAssigned() {
-    // TODO: we have a better way now (wait on event)
-    return masterServices.getAssignmentManager()
-        .getRegionStates().hasTableRegionStates(TableName.NAMESPACE_TABLE_NAME);
+  public List<NamespaceDescriptor> list() throws IOException {
+    return cache.values().stream().collect(Collectors.toList());
   }
 
   public void validateTableAndRegionCount(NamespaceDescriptor desc) throws IOException {
     if (getMaxRegions(desc) <= 0) {
-      throw new ConstraintException("The max region quota for " + desc.getName()
-          + " is less than or equal to zero.");
+      throw new ConstraintException(
+        "The max region quota for " + desc.getName() + " is less than or equal to zero.");
     }
     if (getMaxTables(desc) <= 0) {
-      throw new ConstraintException("The max tables quota for " + desc.getName()
-          + " is less than or equal to zero.");
+      throw new ConstraintException(
+        "The max tables quota for " + desc.getName() + " is less than or equal to zero.");
     }
   }
 
@@ -371,31 +206,4 @@ public class TableNamespaceManager implements Stoppable {
     }
     return maxRegions;
   }
-
-  @Override
-  public boolean isStopped() {
-    return this.stopped;
-  }
-
-  @Override
-  public void stop(String why) {
-    if (this.stopped) {
-      return;
-    }
-    try {
-      if (this.zkNamespaceManager != null) {
-        this.zkNamespaceManager.stop();
-      }
-    } catch (IOException ioe) {
-      LOG.warn("Failed NamespaceManager close", ioe);
-    }
-    try {
-      if (this.nsTable != null) {
-        this.nsTable.close();
-      }
-    } catch (IOException ioe) {
-      LOG.warn("Failed Namespace Table close", ioe);
-    }
-    this.stopped = true;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1acbd36c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
index 3a87bbc..4fba2dd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/locking/LockProcedure.java
@@ -366,7 +366,7 @@ public final class LockProcedure extends Procedure<MasterProcedureEnv>
   }
 
   private LockInterface setupNamespaceLock() throws IllegalArgumentException {
-    this.tableName = TableName.NAMESPACE_TABLE_NAME;
+    this.tableName = TableProcedureInterface.DUMMY_NAMESPACE_TABLE_NAME;
     switch (type) {
       case EXCLUSIVE:
         this.opType = TableOperationType.EDIT;

http://git-wip-us.apache.org/repos/asf/hbase/blob/1acbd36c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
index 341d116..e751034 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineNamespaceProcedure.java
@@ -18,18 +18,25 @@
 
 package org.apache.hadoop.hbase.master.procedure;
 
+import java.io.IOException;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.master.TableNamespaceManager;
 import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
 import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.util.FSUtils;
 
 /**
- * Base class for all the Namespace procedures that want to use a StateMachineProcedure.
- * It provide some basic helpers like basic locking and basic toStringClassDetails().
+ * Base class for all the Namespace procedures that want to use a StateMachineProcedure. It provide
+ * some basic helpers like basic locking and basic toStringClassDetails().
  */
 @InterfaceAudience.Private
 public abstract class AbstractStateMachineNamespaceProcedure<TState>
-    extends StateMachineProcedure<MasterProcedureEnv, TState>
-    implements TableProcedureInterface {
+    extends StateMachineProcedure<MasterProcedureEnv, TState> implements TableProcedureInterface {
 
   private final ProcedurePrepareLatch syncLatch;
 
@@ -52,7 +59,7 @@ public abstract class AbstractStateMachineNamespaceProcedure<TState>
 
   @Override
   public TableName getTableName() {
-    return TableName.NAMESPACE_TABLE_NAME;
+    return DUMMY_NAMESPACE_TABLE_NAME;
   }
 
   @Override
@@ -83,6 +90,35 @@ public abstract class AbstractStateMachineNamespaceProcedure<TState>
     env.getProcedureScheduler().wakeNamespaceExclusiveLock(this, getNamespaceName());
   }
 
+  /**
+   * Insert/update the row into the ns family of meta table.
+   * @param env MasterProcedureEnv
+   */
+  protected static void addOrUpdateNamespace(MasterProcedureEnv env, NamespaceDescriptor ns)
+      throws IOException {
+    getTableNamespaceManager(env).addOrUpdateNamespace(ns);
+  }
+
+  protected static TableNamespaceManager getTableNamespaceManager(MasterProcedureEnv env) {
+    return env.getMasterServices().getClusterSchema().getTableNamespaceManager();
+  }
+
+  /**
+   * Create the namespace directory
+   * @param env MasterProcedureEnv
+   * @param nsDescriptor NamespaceDescriptor
+   */
+  protected static void createDirectory(MasterProcedureEnv env, NamespaceDescriptor nsDescriptor)
+      throws IOException {
+    createDirectory(env.getMasterServices().getMasterFileSystem(), nsDescriptor);
+  }
+
+  @VisibleForTesting
+  public static void createDirectory(MasterFileSystem mfs, NamespaceDescriptor nsDescriptor)
+      throws IOException {
+    mfs.getFileSystem().mkdirs(FSUtils.getNamespaceDir(mfs.getRootDir(), nsDescriptor.getName()));
+  }
+
   protected void releaseSyncLatch() {
     ProcedurePrepareLatch.releaseLatch(syncLatch, this);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1acbd36c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
index 2f56e83..28f7585 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
@@ -15,23 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.master.procedure;
 
 import java.io.IOException;
-
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NamespaceExistException;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
-import org.apache.hadoop.hbase.master.TableNamespaceManager;
-import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateNamespaceState;
-import org.apache.hadoop.hbase.util.FSUtils;
 
 /**
  * The procedure to create a new namespace.
@@ -42,10 +38,8 @@ public class CreateNamespaceProcedure
   private static final Logger LOG = LoggerFactory.getLogger(CreateNamespaceProcedure.class);
 
   private NamespaceDescriptor nsDescriptor;
-  private Boolean traceEnabled;
 
   public CreateNamespaceProcedure() {
-    this.traceEnabled = null;
   }
 
   public CreateNamespaceProcedure(final MasterProcedureEnv env,
@@ -57,43 +51,40 @@ public class CreateNamespaceProcedure
       final NamespaceDescriptor nsDescriptor, ProcedurePrepareLatch latch) {
     super(env, latch);
     this.nsDescriptor = nsDescriptor;
-    this.traceEnabled = null;
   }
 
   @Override
   protected Flow executeFromState(final MasterProcedureEnv env, final CreateNamespaceState state)
       throws InterruptedException {
-    if (isTraceEnabled()) {
-      LOG.trace(this + " execute state=" + state);
-    }
+    LOG.trace("{} execute state={}", this, state);
     try {
       switch (state) {
-      case CREATE_NAMESPACE_PREPARE:
-        boolean success = prepareCreate(env);
-        releaseSyncLatch();
-        if (!success) {
-          assert isFailed() : "createNamespace should have an exception here";
+        case CREATE_NAMESPACE_PREPARE:
+          boolean success = prepareCreate(env);
+          releaseSyncLatch();
+          if (!success) {
+            assert isFailed() : "createNamespace should have an exception here";
+            return Flow.NO_MORE_STATE;
+          }
+          setNextState(CreateNamespaceState.CREATE_NAMESPACE_CREATE_DIRECTORY);
+          break;
+        case CREATE_NAMESPACE_CREATE_DIRECTORY:
+          createDirectory(env, nsDescriptor);
+          setNextState(CreateNamespaceState.CREATE_NAMESPACE_INSERT_INTO_NS_TABLE);
+          break;
+        case CREATE_NAMESPACE_INSERT_INTO_NS_TABLE:
+          addOrUpdateNamespace(env, nsDescriptor);
+          setNextState(CreateNamespaceState.CREATE_NAMESPACE_SET_NAMESPACE_QUOTA);
+          break;
+        case CREATE_NAMESPACE_UPDATE_ZK:
+          // not used any more
+          setNextState(CreateNamespaceState.CREATE_NAMESPACE_SET_NAMESPACE_QUOTA);
+          break;
+        case CREATE_NAMESPACE_SET_NAMESPACE_QUOTA:
+          setNamespaceQuota(env, nsDescriptor);
           return Flow.NO_MORE_STATE;
-        }
-        setNextState(CreateNamespaceState.CREATE_NAMESPACE_CREATE_DIRECTORY);
-        break;
-      case CREATE_NAMESPACE_CREATE_DIRECTORY:
-        createDirectory(env, nsDescriptor);
-        setNextState(CreateNamespaceState.CREATE_NAMESPACE_INSERT_INTO_NS_TABLE);
-        break;
-      case CREATE_NAMESPACE_INSERT_INTO_NS_TABLE:
-        insertIntoNSTable(env, nsDescriptor);
-        setNextState(CreateNamespaceState.CREATE_NAMESPACE_UPDATE_ZK);
-        break;
-      case CREATE_NAMESPACE_UPDATE_ZK:
-        updateZKNamespaceManager(env, nsDescriptor);
-        setNextState(CreateNamespaceState.CREATE_NAMESPACE_SET_NAMESPACE_QUOTA);
-        break;
-      case CREATE_NAMESPACE_SET_NAMESPACE_QUOTA:
-        setNamespaceQuota(env, nsDescriptor);
-        return Flow.NO_MORE_STATE;
-      default:
-        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+        default:
+          throw new UnsupportedOperationException(this + " unhandled state=" + state);
       }
     } catch (IOException e) {
       if (isRollbackSupported(state)) {
@@ -145,39 +136,26 @@ public class CreateNamespaceProcedure
   }
 
   @Override
-  protected void serializeStateData(ProcedureStateSerializer serializer)
-      throws IOException {
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
     super.serializeStateData(serializer);
 
     MasterProcedureProtos.CreateNamespaceStateData.Builder createNamespaceMsg =
-        MasterProcedureProtos.CreateNamespaceStateData.newBuilder().setNamespaceDescriptor(
-          ProtobufUtil.toProtoNamespaceDescriptor(this.nsDescriptor));
+      MasterProcedureProtos.CreateNamespaceStateData.newBuilder()
+        .setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(this.nsDescriptor));
     serializer.serialize(createNamespaceMsg.build());
   }
 
   @Override
-  protected void deserializeStateData(ProcedureStateSerializer serializer)
-      throws IOException {
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
     super.deserializeStateData(serializer);
 
     MasterProcedureProtos.CreateNamespaceStateData createNamespaceMsg =
-        serializer.deserialize(MasterProcedureProtos.CreateNamespaceStateData.class);
+      serializer.deserialize(MasterProcedureProtos.CreateNamespaceStateData.class);
     nsDescriptor = ProtobufUtil.toNamespaceDescriptor(createNamespaceMsg.getNamespaceDescriptor());
   }
 
-  private boolean isBootstrapNamespace() {
-    return nsDescriptor.equals(NamespaceDescriptor.DEFAULT_NAMESPACE) ||
-        nsDescriptor.equals(NamespaceDescriptor.SYSTEM_NAMESPACE);
-  }
-
   @Override
   protected boolean waitInitialized(MasterProcedureEnv env) {
-    // Namespace manager might not be ready if master is not fully initialized,
-    // return false to reject user namespace creation; return true for default
-    // and system namespace creation (this is part of master initialization).
-    if (isBootstrapNamespace()) {
-      return false;
-    }
     return env.waitInitialized(this);
   }
 
@@ -202,12 +180,11 @@ public class CreateNamespaceProcedure
   /**
    * Action before any real action of creating namespace.
    * @param env MasterProcedureEnv
-   * @throws IOException
    */
   private boolean prepareCreate(final MasterProcedureEnv env) throws IOException {
     if (getTableNamespaceManager(env).doesNamespaceExist(nsDescriptor.getName())) {
       setFailure("master-create-namespace",
-          new NamespaceExistException("Namespace " + nsDescriptor.getName() + " already exists"));
+        new NamespaceExistException("Namespace " + nsDescriptor.getName() + " already exists"));
       return false;
     }
     getTableNamespaceManager(env).validateTableAndRegionCount(nsDescriptor);
@@ -215,77 +192,14 @@ public class CreateNamespaceProcedure
   }
 
   /**
-   * Create the namespace directory
-   * @param env MasterProcedureEnv
-   * @param nsDescriptor NamespaceDescriptor
-   * @throws IOException
-   */
-  protected static void createDirectory(
-      final MasterProcedureEnv env,
-      final NamespaceDescriptor nsDescriptor) throws IOException {
-    MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
-    mfs.getFileSystem().mkdirs(
-      FSUtils.getNamespaceDir(mfs.getRootDir(), nsDescriptor.getName()));
-  }
-
-  /**
-   * Insert the row into ns table
-   * @param env MasterProcedureEnv
-   * @param nsDescriptor NamespaceDescriptor
-   * @throws IOException
-   */
-  protected static void insertIntoNSTable(
-      final MasterProcedureEnv env,
-      final NamespaceDescriptor nsDescriptor) throws IOException {
-    getTableNamespaceManager(env).insertIntoNSTable(nsDescriptor);
-  }
-
-  /**
-   * Update ZooKeeper.
-   * @param env MasterProcedureEnv
-   * @param nsDescriptor NamespaceDescriptor
-   * @throws IOException
-   */
-  protected static void updateZKNamespaceManager(
-      final MasterProcedureEnv env,
-      final NamespaceDescriptor nsDescriptor) throws IOException {
-    getTableNamespaceManager(env).updateZKNamespaceManager(nsDescriptor);
-  }
-
-  /**
    * Set quota for the namespace
    * @param env MasterProcedureEnv
    * @param nsDescriptor NamespaceDescriptor
-   * @throws IOException
    **/
-  protected static void setNamespaceQuota(
-      final MasterProcedureEnv env,
+  private static void setNamespaceQuota(final MasterProcedureEnv env,
       final NamespaceDescriptor nsDescriptor) throws IOException {
     if (env.getMasterServices().isInitialized()) {
       env.getMasterServices().getMasterQuotaManager().setNamespaceQuota(nsDescriptor);
     }
   }
-
-  private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
-    return env.getMasterServices().getClusterSchema().getTableNamespaceManager();
-  }
-
-  /**
-   * The procedure could be restarted from a different machine. If the variable is null, we need to
-   * retrieve it.
-   * @return traceEnabled
-   */
-  private Boolean isTraceEnabled() {
-    if (traceEnabled == null) {
-      traceEnabled = LOG.isTraceEnabled();
-    }
-    return traceEnabled;
-  }
-
-  @Override
-  protected boolean shouldWaitClientAck(MasterProcedureEnv env) {
-    // hbase and default namespaces are created on bootstrap internally by the system
-    // the client does not know about this procedures.
-    return !isBootstrapNamespace();
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1acbd36c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
index 8369a19..d3749a2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
@@ -15,29 +15,27 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.master.procedure;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NamespaceNotFoundException;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
-import org.apache.hadoop.hbase.master.TableNamespaceManager;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceState;
-import org.apache.hadoop.hbase.util.FSUtils;
 
 /**
  * The procedure to remove a namespace.
@@ -49,64 +47,59 @@ public class DeleteNamespaceProcedure
 
   private NamespaceDescriptor nsDescriptor;
   private String namespaceName;
-  private Boolean traceEnabled;
 
   public DeleteNamespaceProcedure() {
-    this.nsDescriptor = null;
-    this.traceEnabled = null;
   }
 
-  public DeleteNamespaceProcedure(final MasterProcedureEnv env, final String namespaceName) {
+  public DeleteNamespaceProcedure(MasterProcedureEnv env, String namespaceName) {
     this(env, namespaceName, null);
   }
 
-  public DeleteNamespaceProcedure(final MasterProcedureEnv env, final String namespaceName,
+  public DeleteNamespaceProcedure(MasterProcedureEnv env, String namespaceName,
       final ProcedurePrepareLatch latch) {
     super(env, latch);
     this.namespaceName = namespaceName;
-    this.nsDescriptor = null;
-    this.traceEnabled = null;
   }
 
   @Override
-  protected Flow executeFromState(final MasterProcedureEnv env, final DeleteNamespaceState state)
+  protected Flow executeFromState(MasterProcedureEnv env, DeleteNamespaceState state)
       throws InterruptedException {
     LOG.info(this.toString());
     try {
       switch (state) {
-      case DELETE_NAMESPACE_PREPARE:
-        boolean present = prepareDelete(env);
-        releaseSyncLatch();
-        if (!present) {
-          assert isFailed() : "Delete namespace should have an exception here";
+        case DELETE_NAMESPACE_PREPARE:
+          boolean present = prepareDelete(env);
+          releaseSyncLatch();
+          if (!present) {
+            assert isFailed() : "Delete namespace should have an exception here";
+            return Flow.NO_MORE_STATE;
+          }
+          setNextState(DeleteNamespaceState.DELETE_NAMESPACE_DELETE_FROM_NS_TABLE);
+          break;
+        case DELETE_NAMESPACE_DELETE_FROM_NS_TABLE:
+          deleteNamespace(env, namespaceName);
+          setNextState(DeleteNamespaceState.DELETE_NAMESPACE_DELETE_DIRECTORIES);
+          break;
+        case DELETE_NAMESPACE_REMOVE_FROM_ZK:
+          // not used any more
+          setNextState(DeleteNamespaceState.DELETE_NAMESPACE_DELETE_DIRECTORIES);
+          break;
+        case DELETE_NAMESPACE_DELETE_DIRECTORIES:
+          deleteDirectory(env, namespaceName);
+          setNextState(DeleteNamespaceState.DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA);
+          break;
+        case DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA:
+          removeNamespaceQuota(env, namespaceName);
           return Flow.NO_MORE_STATE;
-        }
-        setNextState(DeleteNamespaceState.DELETE_NAMESPACE_DELETE_FROM_NS_TABLE);
-        break;
-      case DELETE_NAMESPACE_DELETE_FROM_NS_TABLE:
-        deleteFromNSTable(env, namespaceName);
-        setNextState(DeleteNamespaceState.DELETE_NAMESPACE_REMOVE_FROM_ZK);
-        break;
-      case DELETE_NAMESPACE_REMOVE_FROM_ZK:
-        removeFromZKNamespaceManager(env, namespaceName);
-        setNextState(DeleteNamespaceState.DELETE_NAMESPACE_DELETE_DIRECTORIES);
-        break;
-      case DELETE_NAMESPACE_DELETE_DIRECTORIES:
-        deleteDirectory(env, namespaceName);
-        setNextState(DeleteNamespaceState.DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA);
-        break;
-      case DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA:
-        removeNamespaceQuota(env, namespaceName);
-        return Flow.NO_MORE_STATE;
-      default:
-        throw new UnsupportedOperationException(this + " unhandled state=" + state);
+        default:
+          throw new UnsupportedOperationException(this + " unhandled state=" + state);
       }
     } catch (IOException e) {
       if (isRollbackSupported(state)) {
         setFailure("master-delete-namespace", e);
       } else {
-        LOG.warn("Retriable error trying to delete namespace " + namespaceName +
-          " (in state=" + state + ")", e);
+        LOG.warn("Retriable error trying to delete namespace " + namespaceName + " (in state=" +
+          state + ")", e);
       }
     }
     return Flow.HAS_MORE_STATE;
@@ -139,7 +132,7 @@ public class DeleteNamespaceProcedure
 
   @Override
   protected DeleteNamespaceState getState(final int stateId) {
-    return DeleteNamespaceState.valueOf(stateId);
+    return DeleteNamespaceState.forNumber(stateId);
   }
 
   @Override
@@ -153,30 +146,28 @@ public class DeleteNamespaceProcedure
   }
 
   @Override
-  protected void serializeStateData(ProcedureStateSerializer serializer)
-      throws IOException {
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
     super.serializeStateData(serializer);
 
     MasterProcedureProtos.DeleteNamespaceStateData.Builder deleteNamespaceMsg =
-        MasterProcedureProtos.DeleteNamespaceStateData.newBuilder().setNamespaceName(namespaceName);
+      MasterProcedureProtos.DeleteNamespaceStateData.newBuilder().setNamespaceName(namespaceName);
     if (this.nsDescriptor != null) {
-      deleteNamespaceMsg.setNamespaceDescriptor(
-        ProtobufUtil.toProtoNamespaceDescriptor(this.nsDescriptor));
+      deleteNamespaceMsg
+        .setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(this.nsDescriptor));
     }
     serializer.serialize(deleteNamespaceMsg.build());
   }
 
   @Override
-  protected void deserializeStateData(ProcedureStateSerializer serializer)
-      throws IOException {
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
     super.deserializeStateData(serializer);
 
     MasterProcedureProtos.DeleteNamespaceStateData deleteNamespaceMsg =
-        serializer.deserialize(MasterProcedureProtos.DeleteNamespaceStateData.class);
+      serializer.deserialize(MasterProcedureProtos.DeleteNamespaceStateData.class);
     namespaceName = deleteNamespaceMsg.getNamespaceName();
     if (deleteNamespaceMsg.hasNamespaceDescriptor()) {
       nsDescriptor =
-          ProtobufUtil.toNamespaceDescriptor(deleteNamespaceMsg.getNamespaceDescriptor());
+        ProtobufUtil.toNamespaceDescriptor(deleteNamespaceMsg.getNamespaceDescriptor());
     }
   }
 
@@ -193,7 +184,6 @@ public class DeleteNamespaceProcedure
   /**
    * Action before any real action of deleting namespace.
    * @param env MasterProcedureEnv
-   * @throws IOException
    */
   private boolean prepareDelete(final MasterProcedureEnv env) throws IOException {
     if (getTableNamespaceManager(env).doesNamespaceExist(namespaceName) == false) {
@@ -201,8 +191,8 @@ public class DeleteNamespaceProcedure
       return false;
     }
     if (NamespaceDescriptor.RESERVED_NAMESPACES.contains(namespaceName)) {
-      setFailure("master-delete-namespace", new ConstraintException(
-          "Reserved namespace "+ namespaceName +" cannot be removed."));
+      setFailure("master-delete-namespace",
+        new ConstraintException("Reserved namespace " + namespaceName + " cannot be removed."));
       return false;
     }
 
@@ -214,9 +204,9 @@ public class DeleteNamespaceProcedure
       return false;
     }
     if (tableCount > 0) {
-      setFailure("master-delete-namespace", new ConstraintException(
-          "Only empty namespaces can be removed. Namespace "+ namespaceName + " has "
-          + tableCount +" tables"));
+      setFailure("master-delete-namespace",
+        new ConstraintException("Only empty namespaces can be removed. Namespace " + namespaceName +
+          " has " + tableCount + " tables"));
       return false;
     }
 
@@ -226,76 +216,28 @@ public class DeleteNamespaceProcedure
   }
 
   /**
-   * delete the row from namespace table
+   * delete the row from the ns family in meta table.
    * @param env MasterProcedureEnv
    * @param namespaceName name of the namespace in string format
-   * @throws IOException
    */
-  protected static void deleteFromNSTable(
-      final MasterProcedureEnv env,
-      final String namespaceName) throws IOException {
-    getTableNamespaceManager(env).removeFromNSTable(namespaceName);
-  }
-
-  /**
-   * undo the delete
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  private void undoDeleteFromNSTable(final MasterProcedureEnv env) {
-    try {
-      if (nsDescriptor != null) {
-        CreateNamespaceProcedure.insertIntoNSTable(env, nsDescriptor);
-      }
-    } catch (Exception e) {
-      // Ignore
-      LOG.debug("Rollback of deleteFromNSTable throws exception: " + e);
-    }
-  }
-
-  /**
-   * remove from ZooKeeper.
-   * @param env MasterProcedureEnv
-   * @param namespaceName name of the namespace in string format
-   * @throws IOException
-   */
-  protected static void removeFromZKNamespaceManager(
-      final MasterProcedureEnv env,
-      final String namespaceName) throws IOException {
-    getTableNamespaceManager(env).removeFromZKNamespaceManager(namespaceName);
-  }
-
-  /**
-   * undo the remove from ZooKeeper
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  private void undoRemoveFromZKNamespaceManager(final MasterProcedureEnv env) {
-    try {
-      if (nsDescriptor != null) {
-        CreateNamespaceProcedure.updateZKNamespaceManager(env, nsDescriptor);
-      }
-    } catch (Exception e) {
-      // Ignore
-      LOG.debug("Rollback of removeFromZKNamespaceManager throws exception: " + e);
-    }
+  private static void deleteNamespace(MasterProcedureEnv env, String namespaceName)
+      throws IOException {
+    getTableNamespaceManager(env).deleteNamespace(namespaceName);
   }
 
   /**
    * Delete the namespace directories from the file system
    * @param env MasterProcedureEnv
    * @param namespaceName name of the namespace in string format
-   * @throws IOException
    */
-  protected static void deleteDirectory(
-      final MasterProcedureEnv env,
-      final String namespaceName) throws IOException {
+  private static void deleteDirectory(MasterProcedureEnv env, String namespaceName)
+      throws IOException {
     MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
     FileSystem fs = mfs.getFileSystem();
     Path p = FSUtils.getNamespaceDir(mfs.getRootDir(), namespaceName);
 
     try {
-      for(FileStatus status : fs.listStatus(p)) {
+      for (FileStatus status : fs.listStatus(p)) {
         if (!HConstants.HBASE_NON_TABLE_DIRS.contains(status.getPath().getName())) {
           throw new IOException("Namespace directory contains table dir: " + status.getPath());
         }
@@ -310,57 +252,12 @@ public class DeleteNamespaceProcedure
   }
 
   /**
-   * undo delete directory
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   */
-  private void rollbackDeleteDirectory(final MasterProcedureEnv env) throws IOException {
-    try {
-      CreateNamespaceProcedure.createDirectory(env, nsDescriptor);
-    } catch (Exception e) {
-      // Ignore exception
-      LOG.debug("Rollback of deleteDirectory throws exception: " + e);
-    }
-  }
-
-  /**
    * remove quota for the namespace
    * @param env MasterProcedureEnv
    * @param namespaceName name of the namespace in string format
-   * @throws IOException
    **/
-  protected static void removeNamespaceQuota(
-      final MasterProcedureEnv env,
-      final String namespaceName) throws IOException {
+  private static void removeNamespaceQuota(final MasterProcedureEnv env, final String namespaceName)
+      throws IOException {
     env.getMasterServices().getMasterQuotaManager().removeNamespaceQuota(namespaceName);
   }
-
-  /**
-   * undo remove quota for the namespace
-   * @param env MasterProcedureEnv
-   * @throws IOException
-   **/
-  private void rollbacRemoveNamespaceQuota(final MasterProcedureEnv env) throws IOException {
-    try {
-      CreateNamespaceProcedure.setNamespaceQuota(env, nsDescriptor);
-    } catch (Exception e) {
-      // Ignore exception
-      LOG.debug("Rollback of removeNamespaceQuota throws exception: " + e);
-    }
-  }
-
-  private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
-    return env.getMasterServices().getClusterSchema().getTableNamespaceManager();
-  }
-  /**
-   * The procedure could be restarted from a different machine. If the variable is null, we need to
-   * retrieve it.
-   * @return traceEnabled
-   */
-  private Boolean isTraceEnabled() {
-    if (traceEnabled == null) {
-      traceEnabled = LOG.isTraceEnabled();
-    }
-    return traceEnabled;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1acbd36c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java
index 024f3ea..5ca7972 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java
@@ -17,6 +17,11 @@
  */
 package org.apache.hadoop.hbase.master.procedure;
 
+import static org.apache.hadoop.hbase.NamespaceDescriptor.DEFAULT_NAMESPACE;
+import static org.apache.hadoop.hbase.NamespaceDescriptor.SYSTEM_NAMESPACE;
+import static org.apache.hadoop.hbase.master.TableNamespaceManager.insertNamespaceToMeta;
+import static org.apache.hadoop.hbase.master.procedure.AbstractStateMachineNamespaceProcedure.createDirectory;
+
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.concurrent.CountDownLatch;
@@ -25,11 +30,15 @@ import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.InitMetaState;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.InitMetaStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
 
 /**
  * This procedure is used to initialize meta table for a new hbase deploy. It will just schedule an
@@ -38,8 +47,12 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.I
 @InterfaceAudience.Private
 public class InitMetaProcedure extends AbstractStateMachineTableProcedure<InitMetaState> {
 
+  private static final Logger LOG = LoggerFactory.getLogger(InitMetaProcedure.class);
+
   private CountDownLatch latch = new CountDownLatch(1);
 
+  private int attempts;
+
   @Override
   public TableName getTableName() {
     return TableName.META_TABLE_NAME;
@@ -53,10 +66,32 @@ public class InitMetaProcedure extends AbstractStateMachineTableProcedure<InitMe
   @Override
   protected Flow executeFromState(MasterProcedureEnv env, InitMetaState state)
       throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+    LOG.debug("Execute {}", this);
     switch (state) {
       case INIT_META_ASSIGN_META:
+        LOG.info("Going to assign meta");
         addChildProcedure(env.getAssignmentManager()
           .createAssignProcedures(Arrays.asList(RegionInfoBuilder.FIRST_META_REGIONINFO)));
+        setNextState(InitMetaState.INIT_META_CREATE_NAMESPACES);
+        return Flow.HAS_MORE_STATE;
+      case INIT_META_CREATE_NAMESPACES:
+        LOG.info("Going to create {} and {} namespaces", DEFAULT_NAMESPACE, SYSTEM_NAMESPACE);
+        try {
+          createDirectory(env, DEFAULT_NAMESPACE);
+          createDirectory(env, SYSTEM_NAMESPACE);
+          // here the TableNamespaceManager has not been initialized yet, so we have to insert the
+          // record directly into meta table, later the TableNamespaceManager will load these two
+          // namespaces when starting.
+          insertNamespaceToMeta(env.getMasterServices().getConnection(), DEFAULT_NAMESPACE);
+          insertNamespaceToMeta(env.getMasterServices().getConnection(), SYSTEM_NAMESPACE);
+        } catch (IOException e) {
+          long backoff = ProcedureUtil.getBackoffTimeMs(this.attempts++);
+          LOG.warn("Failed to init default and system namespaces, suspend {}secs", backoff, e);
+          setTimeout(Math.toIntExact(backoff));
+          setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT);
+          skipPersistence();
+          throw new ProcedureSuspendedException();
+        }
         return Flow.NO_MORE_STATE;
       default:
         throw new UnsupportedOperationException("unhandled state=" + state);
@@ -70,6 +105,13 @@ public class InitMetaProcedure extends AbstractStateMachineTableProcedure<InitMe
   }
 
   @Override
+  protected synchronized boolean setTimeoutFailure(MasterProcedureEnv env) {
+    setState(ProcedureProtos.ProcedureState.RUNNABLE);
+    env.getProcedureScheduler().addFront(this);
+    return false;
+  }
+
+  @Override
   protected LockState acquireLock(MasterProcedureEnv env) {
     if (env.getProcedureScheduler().waitTableExclusiveLock(this, getTableName())) {
       return LockState.LOCK_EVENT_WAIT;

http://git-wip-us.apache.org/repos/asf/hbase/blob/1acbd36c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
index edf015a..4bf16ec 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
@@ -808,11 +808,11 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     schedLock();
     try {
       final LockAndQueue systemNamespaceTableLock =
-        locking.getTableLock(TableName.NAMESPACE_TABLE_NAME);
+        locking.getTableLock(TableProcedureInterface.DUMMY_NAMESPACE_TABLE_NAME);
       if (!systemNamespaceTableLock.trySharedLock(procedure)) {
         waitProcedure(systemNamespaceTableLock, procedure);
         logLockedResource(LockedResourceType.TABLE,
-          TableName.NAMESPACE_TABLE_NAME.getNameAsString());
+          TableProcedureInterface.DUMMY_NAMESPACE_TABLE_NAME.getNameAsString());
         return true;
       }
 
@@ -840,13 +840,14 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     try {
       final LockAndQueue namespaceLock = locking.getNamespaceLock(namespace);
       final LockAndQueue systemNamespaceTableLock =
-          locking.getTableLock(TableName.NAMESPACE_TABLE_NAME);
+        locking.getTableLock(TableProcedureInterface.DUMMY_NAMESPACE_TABLE_NAME);
       int waitingCount = 0;
       if (namespaceLock.releaseExclusiveLock(procedure)) {
         waitingCount += wakeWaitingProcedures(namespaceLock);
       }
       if (systemNamespaceTableLock.releaseSharedLock()) {
-        addToRunQueue(tableRunQueue, getTableQueue(TableName.NAMESPACE_TABLE_NAME),
+        addToRunQueue(tableRunQueue,
+          getTableQueue(TableProcedureInterface.DUMMY_NAMESPACE_TABLE_NAME),
           () -> procedure + " released namespace exclusive lock");
         waitingCount += wakeWaitingProcedures(systemNamespaceTableLock);
       }