You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by sy...@apache.org on 2016/01/04 17:57:12 UTC

[27/29] hbase git commit: Revert "HBASE-14888 ClusterSchema: Add Namespace Operations"

Revert "HBASE-14888  ClusterSchema: Add Namespace Operations"

This reverts commit c5f3d17ae3a61cbf77cab89cddd8303e20e5e734.


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

Branch: refs/heads/hbase-12439
Commit: a82f7fc94a52e941b139367f1e67c03229213247
Parents: c5f3d17
Author: stack <st...@apache.org>
Authored: Sun Jan 3 07:15:18 2016 -0800
Committer: stack <st...@apache.org>
Committed: Sun Jan 3 07:15:18 2016 -0800

----------------------------------------------------------------------
 .../hbase/client/ConnectionImplementation.java  |   2 +-
 .../java/org/apache/hadoop/hbase/Service.java   |  50 ---
 .../hbase/ServiceNotRunningException.java       |  39 ---
 .../apache/hadoop/hbase/ResourceChecker.java    |   2 +-
 .../hadoop/hbase/master/ClusterSchema.java      | 153 ---------
 .../hbase/master/ClusterSchemaException.java    |  37 ---
 .../hbase/master/ClusterSchemaService.java      |  27 --
 .../hbase/master/ClusterSchemaServiceImpl.java  | 164 ----------
 .../org/apache/hadoop/hbase/master/HMaster.java | 314 +++++++++++++------
 .../hadoop/hbase/master/MasterRpcServices.java  |  10 +-
 .../hadoop/hbase/master/MasterServices.java     |  78 ++++-
 .../hadoop/hbase/master/ProcedureFuture.java    | 132 --------
 .../hbase/master/TableNamespaceManager.java     |  68 ++--
 .../procedure/CreateNamespaceProcedure.java     |   2 +-
 .../procedure/DeleteNamespaceProcedure.java     |   2 +-
 .../procedure/ModifyNamespaceProcedure.java     |   3 +-
 .../hbase/namespace/NamespaceStateManager.java  |   6 +-
 .../hbase/regionserver/HRegionServer.java       |   3 +-
 .../org/apache/hadoop/hbase/TestNamespace.java  |   2 +-
 .../hadoop/hbase/master/TestCatalogJanitor.java |  61 +++-
 .../hbase/master/TestMasterNoCluster.java       |   8 +
 21 files changed, 401 insertions(+), 762 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 3498590..0ef2a17 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -398,7 +398,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
       synchronized (this) {
         if (batchPool == null) {
           this.batchPool = getThreadPool(conf.getInt("hbase.hconnection.threads.max", 256),
-              conf.getInt("hbase.hconnection.threads.core", 256), "-shared", null);
+              conf.getInt("hbase.hconnection.threads.core", 256), "-shared-", null);
           this.cleanupPool = true;
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-common/src/main/java/org/apache/hadoop/hbase/Service.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/Service.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/Service.java
deleted file mode 100644
index 97d93cc..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/Service.java
+++ /dev/null
@@ -1,50 +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 org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * Simple Service.
- */
-// This is a WIP. We have Services throughout hbase. Either have all implement what is here or
-// just remove this as an experiment that did not work out.
-// TODO: Move on to guava Service after we update our guava version; later guava has nicer
-// Service implmentation.
-// TODO: Move all Services on to this one Interface.
-@InterfaceAudience.Private
-public interface Service {
-  /**
-   * Initiates service startup (if necessary), returning once the service has finished starting.
-   * @throws IOException Throws exception if already running and if we fail to start successfully.
-   */
-  void startAndWait() throws IOException;
-
-  /**
-   * @return True if this Service is running.
-   */
-  boolean isRunning();
-
-  /**
-   * Initiates service shutdown (if necessary), returning once the service has finished stopping.
-   * @throws IOException Throws exception if not running of if we fail to stop successfully.
-   */
-  void stopAndWait() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-common/src/main/java/org/apache/hadoop/hbase/ServiceNotRunningException.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ServiceNotRunningException.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ServiceNotRunningException.java
deleted file mode 100644
index f6325ea..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ServiceNotRunningException.java
+++ /dev/null
@@ -1,39 +0,0 @@
-package org.apache.hadoop.hbase;
-/**
- * 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.
- */
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-@SuppressWarnings("serial")
-@InterfaceAudience.Private
-public class ServiceNotRunningException extends HBaseIOException {
-  public ServiceNotRunningException() {
-  }
-
-  public ServiceNotRunningException(String message) {
-    super(message);
-  }
-
-  public ServiceNotRunningException(String message, Throwable cause) {
-    super(message, cause);
-  }
-
-  public ServiceNotRunningException(Throwable cause) {
-    super(cause);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java
index ee0380a..539aea3 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java
@@ -41,7 +41,7 @@ public class ResourceChecker {
 
   /**
    * Constructor
-   * @param tagLine The tagLine is added to the logs. Must not be null.
+   * @param tagLine - the tagLine is added to the logs. Must be be null.
    */
   public ResourceChecker(final String tagLine) {
     this.tagLine = tagLine;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java
deleted file mode 100644
index e632230..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java
+++ /dev/null
@@ -1,153 +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.master;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.List;
-import java.util.concurrent.Future;
-
-import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.ProcedureInfo;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * View and edit the current cluster schema. Use this API making any modification to
- * namespaces, tables, etc.
- *
- * <h2>Implementation Notes</h2>
- * Nonces are for when operation is non-idempotent to ensure once-only semantic, even
- * across process failures.
- */
-// ClusterSchema is introduced to encapsulate schema modification. Currently the different aspects
-// are spread about the code base. This effort is about cleanup, shutting down access, and
-// coalescing common code. In particular, we'd contain filesystem modification. Other
-// benefits are to make all schema modification work the same way (one way to do an operation only
-// rather than the current approach where how an operation is done varies with context) and to make
-// it so clusterschema modification can stand apart from Master to faciliate standalone
-// testing. It is part of the filesystem refactor project that undoes the dependency on a
-// layout in HDFS that mimics our model of tables have regions have column families have files.
-// With this Interface in place, with all modifications going via this route where no filesystem
-// particulars are exposed, redoing our internals will take less effort.
-//
-// Currently ClusterSchema Interface will include namespace and table manipulation. Ideally a
-// form of this Interface will go all the ways down to the file manipulation level but currently
-// TBD.
-//
-// ClusterSchema is private to the Master; only the Master knows current cluster state and has
-// means of editing/altering it.
-//
-// TODO: Remove Server argument when MasterServices are passed.
-// TODO: We return Future<ProcedureInfo> in the below from most methods. It may change to return
-// a ProcedureFuture subsequently.
-@InterfaceAudience.Private
-public interface ClusterSchema {
-  /**
-   * Timeout for cluster operations in milliseconds.
-   */
-  public static final String HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY =
-      "hbase.master.cluster.schema.operation.timeout";
-  /**
-   * Default operation timeout in milliseconds.
-   */
-  public static final int DEFAULT_HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT =
-      5 * 60 * 1000;
-
-  /**
-   * Utility method that will wait {@link #HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY}
-   * timeout and if exceptions, THROWs the exception doing conversion so palatable outside Master:
-   * i.e. {@link InterruptedException} becomes {@link InterruptedIOException} and so on.
-   *
-   * <<Utility>>
-   *
-   * @param future Future to wait on.
-   * @return On completion, info on the procedure that ran.
-   * @throws IOException
-   */
-  // TODO: Where to put this utility? It goes away?
-  ProcedureInfo get(final Future<ProcedureInfo> future) throws IOException;
-
-  /**
-   * For internals use only. Do not use! Provisionally part of this Interface.
-   * Prefer the high-level APIs available elsewhere in this API.
-   * @return Instance of {@link TableNamespaceManager}
-   */
-  // TODO: Remove from here. Keep internal. This Interface is too high-level to host this accessor.
-  TableNamespaceManager getTableNamespaceManager();
-
-  /**
-   * Create a new Namespace.
-   * @param namespaceDescriptor descriptor for new Namespace
-   * @param nonceGroup Identifier for the source of the request, a client or process.
-   * @param nonce A unique identifier for this operation from the client or process identified by
-   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
-   * @return Operation Future.
-   * Use {@link Future#get(long, java.util.concurrent.TimeUnit)} to wait on completion.
-   * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
-   * as well as {@link IOException}
-   */
-  Future<ProcedureInfo> createNamespace(NamespaceDescriptor namespaceDescriptor, long nonceGroup,
-      long nonce)
-  throws IOException;
-
-  /**
-   * Modify an existing Namespace.
-   * @param nonceGroup Identifier for the source of the request, a client or process.
-   * @param nonce A unique identifier for this operation from the client or process identified by
-   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
-   * @return Operation Future.
-   * Use {@link Future#get(long, java.util.concurrent.TimeUnit)} to wait on completion.
-   * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
-   * as well as {@link IOException}
-   */
-  Future<ProcedureInfo> modifyNamespace(NamespaceDescriptor descriptor, long nonceGroup,
-      long nonce)
-  throws IOException;
-
-  /**
-   * Delete an existing Namespace.
-   * Only empty Namespaces (no tables) can be removed.
-   * @param nonceGroup Identifier for the source of the request, a client or process.
-   * @param nonce A unique identifier for this operation from the client or process identified by
-   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
-   * @return Operation Future.
-   * Use {@link Future#get(long, java.util.concurrent.TimeUnit)} to wait on completion.
-   * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
-   * as well as {@link IOException}
-   */
-  Future<ProcedureInfo> deleteNamespace(String name, long nonceGroup, long nonce)
-  throws IOException;
-
-  /**
-   * Get a Namespace
-   * @param name Name of the Namespace
-   * @return Namespace descriptor for <code>name</code>
-   * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
-   * as well as {@link IOException}
-   */
-  // No Future here because presumption is that the request will go against cached metadata so
-  // return immediately -- no need of running a Procedure.
-  NamespaceDescriptor getNamespace(String name) throws IOException;
-
-  /**
-   * Get all Namespaces
-   * @return All Namespace descriptors
-   * @throws IOException
-   */
-  List<NamespaceDescriptor> getNamespaces() throws IOException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaException.java
deleted file mode 100644
index 62892b6..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaException.java
+++ /dev/null
@@ -1,37 +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.master;
-
-import org.apache.hadoop.hbase.HBaseIOException;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-@SuppressWarnings("serial")
-@InterfaceAudience.Private
-public class ClusterSchemaException extends HBaseIOException {
-  public ClusterSchemaException(String message) {
-    super(message);
-  }
-
-  public ClusterSchemaException(String message, Throwable cause) {
-    super(message, cause);
-  }
-
-  public ClusterSchemaException(Throwable cause) {
-    super(cause);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaService.java
deleted file mode 100644
index 43353ba..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaService.java
+++ /dev/null
@@ -1,27 +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.master;
-
-import org.apache.hadoop.hbase.Service;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * Mixes in ClusterSchema and Service
- */
-@InterfaceAudience.Private
-public interface ClusterSchemaService extends ClusterSchema, Service {}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/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
deleted file mode 100644
index c7f4692..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
+++ /dev/null
@@ -1,164 +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.master;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.NamespaceNotFoundException;
-import org.apache.hadoop.hbase.ProcedureInfo;
-import org.apache.hadoop.hbase.ServiceNotRunningException;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
-import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
-import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.master.procedure.ModifyNamespaceProcedure;
-import org.apache.hadoop.hbase.procedure2.Procedure;
-import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
-
-@InterfaceAudience.Private
-class ClusterSchemaServiceImpl implements ClusterSchemaService {
-  private boolean running = false;
-  private final TableNamespaceManager tableNamespaceManager;
-  private final MasterServices masterServices;
-  private final int clusterSchemaOperationTimeoutInMillis;
-  private final static List<NamespaceDescriptor> EMPTY_NAMESPACE_LIST =
-    Collections.unmodifiableList(new ArrayList<NamespaceDescriptor>(0));
-
-  ClusterSchemaServiceImpl(final MasterServices masterServices) {
-    this.masterServices = masterServices;
-    this.tableNamespaceManager = new TableNamespaceManager(masterServices);
-    this.clusterSchemaOperationTimeoutInMillis = this.masterServices.getConfiguration().
-      getInt(HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY,
-        DEFAULT_HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT);
-  }
-
-  // All below are synchronized so consistent view on whether running or not.
-
-  @Override
-  public synchronized boolean isRunning() {
-    return this.running;
-  }
-
-  private synchronized void checkIsRunning() throws ServiceNotRunningException {
-    if (!isRunning()) throw new ServiceNotRunningException();
-  }
-
-  @Override
-  public synchronized void startAndWait() throws IOException {
-    if (isRunning()) throw new IllegalStateException("Already running; cannot double-start.");
-    // Set to running FIRST because tableNamespaceManager start uses this class to do namespace ops
-    this.running = true;
-    this.tableNamespaceManager.start();
-  }
-
-  @Override
-  public synchronized void stopAndWait() throws IOException {
-    checkIsRunning();
-    // You can't stop tableNamespaceManager.
-    this.running = false;
-  }
-
-  @Override
-  public ProcedureInfo get(final Future<ProcedureInfo> future) throws IOException {
-    try {
-      ProcedureInfo pi =
-        future.get(this.clusterSchemaOperationTimeoutInMillis, TimeUnit.MILLISECONDS);
-      // If the procedure got an exception, throw it.
-      if (pi.getException() != null) throw pi.getException();
-      return pi;
-    } catch (ExecutionException ee) {
-      // No cleanup to do... just let the exception out.
-      if (ee.getCause() instanceof IOException) throw (IOException)ee.getCause();
-      else throw new ClusterSchemaException(ee.getCause());
-    } catch (InterruptedException e) {
-      IOException ioe = new InterruptedIOException();
-      ioe.initCause(e);
-      throw ioe;
-    } catch (TimeoutException e) {
-      throw new TimeoutIOException(future.toString());
-    }
-  }
-
-  @Override
-  public TableNamespaceManager getTableNamespaceManager() {
-    return this.tableNamespaceManager;
-  }
-
-  private Future<ProcedureInfo> submitProcedure(final Procedure<?> procedure, long nonceGroup,
-      long nonce)
-  throws ServiceNotRunningException {
-    checkIsRunning();
-    ProcedureExecutor<MasterProcedureEnv> pe = this.masterServices.getMasterProcedureExecutor();
-    long procId = pe.submitProcedure(procedure, nonceGroup, nonce);
-    return new ProcedureFuture(pe, procId);
-  }
-
-  @Override
-  public Future<ProcedureInfo> createNamespace(NamespaceDescriptor namespaceDescriptor,
-      long nonceGroup, long nonce)
-  throws IOException {
-    return submitProcedure(new CreateNamespaceProcedure(
-      this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor),
-        nonceGroup, nonce);
-  }
-
-  @Override
-  public Future<ProcedureInfo> modifyNamespace(NamespaceDescriptor namespaceDescriptor,
-      long nonceGroup, long nonce)
-  throws IOException {
-    return submitProcedure(new ModifyNamespaceProcedure(
-      this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor),
-        nonceGroup, nonce);
-  }
-
-  @Override
-  public Future<ProcedureInfo> deleteNamespace(String name, long nonceGroup, long nonce)
-  throws IOException {
-    return submitProcedure(new DeleteNamespaceProcedure(
-      this.masterServices.getMasterProcedureExecutor().getEnvironment(), name),
-        nonceGroup, nonce);
-  }
-
-  @Override
-  public NamespaceDescriptor getNamespace(String name) throws IOException {
-    NamespaceDescriptor nsd = getTableNamespaceManager().get(name);
-    if (nsd == null) throw new NamespaceNotFoundException(name);
-    return nsd;
-  }
-
-  @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<NamespaceDescriptor>(set.size());
-    list.addAll(set);
-    return Collections.unmodifiableList(list);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/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 f29cad6..8e51f25 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
@@ -36,7 +36,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
-import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
@@ -64,6 +63,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.NamespaceNotFoundException;
 import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.Server;
@@ -97,14 +97,17 @@ import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerChore;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerFactory;
 import org.apache.hadoop.hbase.master.procedure.AddColumnFamilyProcedure;
+import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.DeleteColumnFamilyProcedure;
+import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
+import org.apache.hadoop.hbase.master.procedure.ModifyNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
 import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
@@ -182,7 +185,7 @@ import com.google.protobuf.Service;
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
 @SuppressWarnings("deprecation")
-public class HMaster extends HRegionServer implements MasterServices {
+public class HMaster extends HRegionServer implements MasterServices, Server {
   private static final Log LOG = LogFactory.getLog(HMaster.class.getName());
 
   /**
@@ -253,7 +256,8 @@ public class HMaster extends HRegionServer implements MasterServices {
   // Tracker for region normalizer state
   private RegionNormalizerTracker regionNormalizerTracker;
 
-  private ClusterSchemaService clusterSchemaService;
+  /** Namespace stuff */
+  private TableNamespaceManager tableNamespaceManager;
 
   // Metrics for the HMaster
   final MetricsMaster metricsMaster;
@@ -364,6 +368,9 @@ public class HMaster extends HRegionServer implements MasterServices {
    * Remaining steps of initialization occur in
    * #finishActiveMasterInitialization(MonitoredTask) after
    * the master becomes the active one.
+   *
+   * @throws KeeperException
+   * @throws IOException
    */
   public HMaster(final Configuration conf, CoordinatedStateManager csm)
       throws IOException, KeeperException {
@@ -563,6 +570,10 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   /**
    * Initialize all ZK based system trackers.
+   * @throws IOException
+   * @throws InterruptedException
+   * @throws KeeperException
+   * @throws CoordinatedStateException
    */
   void initializeZKBasedSystemTrackers() throws IOException,
       InterruptedException, KeeperException, CoordinatedStateException {
@@ -619,6 +630,11 @@ public class HMaster extends HRegionServer implements MasterServices {
    * <li>Ensure assignment of meta/namespace regions<li>
    * <li>Handle either fresh cluster start or master failover</li>
    * </ol>
+   *
+   * @throws IOException
+   * @throws InterruptedException
+   * @throws KeeperException
+   * @throws CoordinatedStateException
    */
   private void finishActiveMasterInitialization(MonitoredTask status)
       throws IOException, InterruptedException, KeeperException, CoordinatedStateException {
@@ -765,8 +781,8 @@ public class HMaster extends HRegionServer implements MasterServices {
     this.catalogJanitorChore = new CatalogJanitor(this, this);
     getChoreService().scheduleChore(catalogJanitorChore);
 
-    status.setStatus("Starting cluster schema service");
-    initClusterSchemaService();
+    status.setStatus("Starting namespace manager");
+    initNamespace();
 
     if (this.cpHost != null) {
       try {
@@ -832,6 +848,11 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   /**
    * Create a {@link ServerManager} instance.
+   * @param master
+   * @param services
+   * @return An instance of {@link ServerManager}
+   * @throws org.apache.hadoop.hbase.ZooKeeperConnectionException
+   * @throws IOException
    */
   ServerManager createServerManager(final Server master,
       final MasterServices services)
@@ -867,6 +888,12 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   /**
    * Check <code>hbase:meta</code> is assigned. If not, assign it.
+   * @param status MonitoredTask
+   * @param previouslyFailedMetaRSs
+   * @param replicaId
+   * @throws InterruptedException
+   * @throws IOException
+   * @throws KeeperException
    */
   void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMetaRSs, int replicaId)
       throws InterruptedException, IOException, KeeperException {
@@ -938,10 +965,10 @@ public class HMaster extends HRegionServer implements MasterServices {
     status.setStatus("META assigned.");
   }
 
-  void initClusterSchemaService() throws IOException, InterruptedException {
-    this.clusterSchemaService = new ClusterSchemaServiceImpl(this);
-    this.clusterSchemaService.startAndWait();
-    if (!this.clusterSchemaService.isRunning()) throw new HBaseIOException("Failed start");
+  void initNamespace() throws IOException {
+    //create namespace manager
+    tableNamespaceManager = new TableNamespaceManager(this);
+    tableNamespaceManager.start();
   }
 
   void initQuotaManager() throws IOException {
@@ -987,6 +1014,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   /**
    * This function returns a set of region server names under hbase:meta recovering region ZK node
    * @return Set of meta server names which were recorded in ZK
+   * @throws KeeperException
    */
   private Set<ServerName> getPreviouselyFailedMetaServersFromZK() throws KeeperException {
     Set<ServerName> result = new HashSet<ServerName>();
@@ -1022,6 +1050,11 @@ public class HMaster extends HRegionServer implements MasterServices {
     return tableStateManager;
   }
 
+  @Override
+  public TableNamespaceManager getTableNamespaceManager() {
+    return tableNamespaceManager;
+  }
+
   /*
    * Start up all services. If any of these threads gets an unhandled exception
    * then they just die with a logged message.  This should be fine because
@@ -1168,6 +1201,7 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   /**
    * @return Get remote side's InetAddress
+   * @throws UnknownHostException
    */
   InetAddress getRemoteInetAddress(final int port,
       final long serverStartCode) throws UnknownHostException {
@@ -1304,6 +1338,7 @@ public class HMaster extends HRegionServer implements MasterServices {
    * @return true if normalization step was performed successfully, false otherwise
    *   (specifically, if HMaster hasn't been initialized properly or normalization
    *   is globally disabled)
+   * @throws IOException
    */
   public boolean normalizeRegions() throws IOException {
     if (!this.initialized) {
@@ -1443,9 +1478,9 @@ public class HMaster extends HRegionServer implements MasterServices {
     if (isStopped()) {
       throw new MasterNotRunningException();
     }
-    checkInitialized();
+
     String namespace = hTableDescriptor.getTableName().getNamespaceAsString();
-    this.clusterSchemaService.getNamespace(namespace);
+    ensureNamespaceExists(namespace);
 
     HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(hTableDescriptor, splitKeys);
     checkInitialized();
@@ -2270,9 +2305,18 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   void checkInitialized() throws PleaseHoldException, ServerNotRunningYetException {
     checkServiceStarted();
-    if (!isInitialized()) throw new PleaseHoldException("Master is initializing");
+    if (!this.initialized) {
+      throw new PleaseHoldException("Master is initializing");
+    }
   }
 
+  void checkNamespaceManagerReady() throws IOException {
+    checkInitialized();
+    if (tableNamespaceManager == null ||
+        !tableNamespaceManager.isTableAvailableAndInitialized(true)) {
+      throw new IOException("Table Namespace Manager not ready yet, try again later");
+    }
+  }
   /**
    * Report whether this master is currently the active master or not.
    * If not active master, we are parked on ZK waiting to become active.
@@ -2367,6 +2411,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   /**
    * Utility for constructing an instance of the passed HMaster class.
    * @param masterClass
+   * @param conf
    * @return HMaster instance.
    */
   public static HMaster constructMaster(Class<? extends HMaster> masterClass,
@@ -2407,110 +2452,138 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public ClusterSchema getClusterSchema() {
-    return this.clusterSchemaService;
+  public void createNamespace(
+      final NamespaceDescriptor descriptor,
+      final long nonceGroup,
+      final long nonce) throws IOException {
+    TableName.isLegalNamespaceName(Bytes.toBytes(descriptor.getName()));
+    checkNamespaceManagerReady();
+    if (cpHost != null) {
+      if (cpHost.preCreateNamespace(descriptor)) {
+        return;
+      }
+    }
+    createNamespaceSync(descriptor, nonceGroup, nonce);
+    if (cpHost != null) {
+      cpHost.postCreateNamespace(descriptor);
+    }
   }
 
-  /**
-   * Create a new Namespace.
-   * @param namespaceDescriptor descriptor for new Namespace
-   * @param nonceGroup Identifier for the source of the request, a client or process.
-   * @param nonce A unique identifier for this operation from the client or process identified by
-   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
-   */
-  void createNamespace(final NamespaceDescriptor namespaceDescriptor, final long nonceGroup,
-      final long nonce)
-  throws IOException {
-    checkInitialized();
-    TableName.isLegalNamespaceName(Bytes.toBytes(namespaceDescriptor.getName()));
-    if (this.cpHost != null && this.cpHost.preCreateNamespace(namespaceDescriptor)) return;
-    LOG.info(getClientIdAuditPrefix() + " creating " + namespaceDescriptor);
+  @Override
+  public void createNamespaceSync(
+      final NamespaceDescriptor descriptor,
+      final long nonceGroup,
+      final long nonce) throws IOException {
+    LOG.info(getClientIdAuditPrefix() + " creating " + descriptor);
     // Execute the operation synchronously - wait for the operation to complete before continuing.
-    Future<ProcedureInfo> future =
-      getClusterSchema().createNamespace(namespaceDescriptor, nonceGroup, nonce);
-    getClusterSchema().get(future);
-    if (this.cpHost != null) this.cpHost.postCreateNamespace(namespaceDescriptor);
+    long procId = this.procedureExecutor.submitProcedure(
+      new CreateNamespaceProcedure(procedureExecutor.getEnvironment(), descriptor),
+      nonceGroup,
+      nonce);
+    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
   }
 
-  /**
-   * Modify an existing Namespace.
-   * @param nonceGroup Identifier for the source of the request, a client or process.
-   * @param nonce A unique identifier for this operation from the client or process identified by
-   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).\
-   */
-  void modifyNamespace(final NamespaceDescriptor namespaceDescriptor, final long nonceGroup,
-      final long nonce)
-  throws IOException {
-    checkInitialized();
-    TableName.isLegalNamespaceName(Bytes.toBytes(namespaceDescriptor.getName()));
-    if (this.cpHost != null && this.cpHost.preModifyNamespace(namespaceDescriptor)) return;
-    LOG.info(getClientIdAuditPrefix() + " modify " + namespaceDescriptor);
+  @Override
+  public void modifyNamespace(
+      final NamespaceDescriptor descriptor,
+      final long nonceGroup,
+      final long nonce) throws IOException {
+    TableName.isLegalNamespaceName(Bytes.toBytes(descriptor.getName()));
+    checkNamespaceManagerReady();
+    if (cpHost != null) {
+      if (cpHost.preModifyNamespace(descriptor)) {
+        return;
+      }
+    }
+    LOG.info(getClientIdAuditPrefix() + " modify " + descriptor);
     // Execute the operation synchronously - wait for the operation to complete before continuing.
-    Future<ProcedureInfo> future =
-        getClusterSchema().modifyNamespace(namespaceDescriptor, nonceGroup, nonce);
-    getClusterSchema().get(future);
-    if (this.cpHost != null) this.cpHost.postModifyNamespace(namespaceDescriptor);
+    long procId = this.procedureExecutor.submitProcedure(
+      new ModifyNamespaceProcedure(procedureExecutor.getEnvironment(), descriptor),
+      nonceGroup,
+      nonce);
+    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
+    if (cpHost != null) {
+      cpHost.postModifyNamespace(descriptor);
+    }
   }
 
-  /**
-   * Delete an existing Namespace. Only empty Namespaces (no tables) can be removed.
-   * @param nonceGroup Identifier for the source of the request, a client or process.
-   * @param nonce A unique identifier for this operation from the client or process identified by
-   * <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
-   * @throws IOException
-   */
-  void deleteNamespace(final String name, final long nonceGroup, final long nonce)
-  throws IOException {
-    checkInitialized();
-    if (this.cpHost != null && this.cpHost.preDeleteNamespace(name)) return;
+  @Override
+  public void deleteNamespace(
+      final String name,
+      final long nonceGroup,
+      final long nonce) throws IOException {
+    checkNamespaceManagerReady();
+    if (cpHost != null) {
+      if (cpHost.preDeleteNamespace(name)) {
+        return;
+      }
+    }
     LOG.info(getClientIdAuditPrefix() + " delete " + name);
     // Execute the operation synchronously - wait for the operation to complete before continuing.
-    Future<ProcedureInfo> future = getClusterSchema().deleteNamespace(name, nonceGroup, nonce);
-    getClusterSchema().get(future);
-    if (this.cpHost != null) this.cpHost.postDeleteNamespace(name);
+    long procId = this.procedureExecutor.submitProcedure(
+      new DeleteNamespaceProcedure(procedureExecutor.getEnvironment(), name),
+      nonceGroup,
+      nonce);
+    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
+    if (cpHost != null) {
+      cpHost.postDeleteNamespace(name);
+    }
   }
 
   /**
-   * Get a Namespace
-   * @param name Name of the Namespace
-   * @return Namespace descriptor for <code>name</code>
+   * Ensure that the specified namespace exists, otherwise throws a NamespaceNotFoundException
+   *
+   * @param name the namespace to check
+   * @throws IOException if the namespace manager is not ready yet.
+   * @throws NamespaceNotFoundException if the namespace does not exists
    */
-  NamespaceDescriptor getNamespace(String name) throws IOException {
-    checkInitialized();
-    if (this.cpHost != null) this.cpHost.preGetNamespaceDescriptor(name);
-    NamespaceDescriptor nsd = this.clusterSchemaService.getNamespace(name);
-    if (this.cpHost != null) this.cpHost.postGetNamespaceDescriptor(nsd);
-    return nsd;
+  private void ensureNamespaceExists(final String name)
+      throws IOException, NamespaceNotFoundException {
+    checkNamespaceManagerReady();
+    NamespaceDescriptor nsd = tableNamespaceManager.get(name);
+    if (nsd == null) {
+      throw new NamespaceNotFoundException(name);
+    }
   }
 
-  /**
-   * Get all Namespaces
-   * @return All Namespace descriptors
-   */
-  List<NamespaceDescriptor> getNamespaces() throws IOException {
-    checkInitialized();
-    final List<NamespaceDescriptor> nsds = new ArrayList<NamespaceDescriptor>();
-    boolean bypass = false;
+  @Override
+  public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException {
+    checkNamespaceManagerReady();
+
     if (cpHost != null) {
-      bypass = cpHost.preListNamespaceDescriptors(nsds);
+      cpHost.preGetNamespaceDescriptor(name);
     }
-    if (!bypass) {
-      nsds.addAll(this.clusterSchemaService.getNamespaces());
-      if (this.cpHost != null) this.cpHost.postListNamespaceDescriptors(nsds);
+
+    NamespaceDescriptor nsd = tableNamespaceManager.get(name);
+    if (nsd == null) {
+      throw new NamespaceNotFoundException(name);
     }
-    return nsds;
-  }
 
-  @Override
-  public List<TableName> listTableNamesByNamespace(String name) throws IOException {
-    checkInitialized();
-    return listTableNames(name, null, true);
+    if (cpHost != null) {
+      cpHost.postGetNamespaceDescriptor(nsd);
+    }
+
+    return nsd;
   }
 
   @Override
-  public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
-    checkInitialized();
-    return listTableDescriptors(name, null, null, true);
+  public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException {
+    checkNamespaceManagerReady();
+
+    final List<NamespaceDescriptor> descriptors = new ArrayList<NamespaceDescriptor>();
+    boolean bypass = false;
+    if (cpHost != null) {
+      bypass = cpHost.preListNamespaceDescriptors(descriptors);
+    }
+
+    if (!bypass) {
+      descriptors.addAll(tableNamespaceManager.list());
+
+      if (cpHost != null) {
+        cpHost.postListNamespaceDescriptors(descriptors);
+      }
+    }
+    return descriptors;
   }
 
   @Override
@@ -2544,6 +2617,18 @@ public class HMaster extends HRegionServer implements MasterServices {
     return procInfoList;
   }
 
+  @Override
+  public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
+    ensureNamespaceExists(name);
+    return listTableDescriptors(name, null, null, true);
+  }
+
+  @Override
+  public List<TableName> listTableNamesByNamespace(String name) throws IOException {
+    ensureNamespaceExists(name);
+    return listTableNames(name, null, true);
+  }
+
   /**
    * Returns the list of table descriptors that match the specified request
    *
@@ -2568,8 +2653,6 @@ public class HMaster extends HRegionServer implements MasterServices {
         // request for all TableDescriptors
         Collection<HTableDescriptor> htds;
         if (namespace != null && namespace.length() > 0) {
-          // Do a check on the namespace existence. Will fail if does not exist.
-          this.clusterSchemaService.getNamespace(namespace);
           htds = tableDescriptors.getByNamespace(namespace).values();
         } else {
           htds = tableDescriptors.getAll().values();
@@ -2613,17 +2696,46 @@ public class HMaster extends HRegionServer implements MasterServices {
    */
   public List<TableName> listTableNames(final String namespace, final String regex,
       final boolean includeSysTables) throws IOException {
-    List<HTableDescriptor> htds = new ArrayList<HTableDescriptor>();
-    boolean bypass = cpHost != null? cpHost.preGetTableNames(htds, regex): false;
+    final List<HTableDescriptor> descriptors = new ArrayList<HTableDescriptor>();
+
+    boolean bypass = false;
+    if (cpHost != null) {
+      bypass = cpHost.preGetTableNames(descriptors, regex);
+    }
+
     if (!bypass) {
-      htds.addAll(listTableDescriptors(namespace, regex, null, includeSysTables));
-      if (cpHost != null) cpHost.postGetTableNames(htds, regex);
+      // get all descriptors
+      Collection<HTableDescriptor> htds;
+      if (namespace != null && namespace.length() > 0) {
+        htds = tableDescriptors.getByNamespace(namespace).values();
+      } else {
+        htds = tableDescriptors.getAll().values();
+      }
+
+      for (HTableDescriptor htd: htds) {
+        if (includeSysTables || !htd.getTableName().isSystemTable()) {
+          descriptors.add(htd);
+        }
+      }
+
+      // Retains only those matched by regular expression.
+      if (regex != null) {
+        filterTablesByRegex(descriptors, Pattern.compile(regex));
+      }
+
+      if (cpHost != null) {
+        cpHost.postGetTableNames(descriptors, regex);
+      }
+    }
+
+    List<TableName> result = new ArrayList<TableName>(descriptors.size());
+    for (HTableDescriptor htd: descriptors) {
+      result.add(htd.getTableName());
     }
-    List<TableName> result = new ArrayList<TableName>(htds.size());
-    for (HTableDescriptor htd: htds) result.add(htd.getTableName());
     return result;
   }
 
+
   /**
    * Removes the table descriptors that don't match the pattern.
    * @param descriptors list of table descriptors to filter

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 2528814..b269c3d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.NamespaceExistException;
 import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.ServerLoad;
@@ -833,7 +832,7 @@ public class MasterRpcServices extends RSRpcServices
     try {
       return GetNamespaceDescriptorResponse.newBuilder()
         .setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(
-            master.getNamespace(request.getNamespaceName())))
+            master.getNamespaceDescriptor(request.getNamespaceName())))
         .build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -1121,7 +1120,7 @@ public class MasterRpcServices extends RSRpcServices
     try {
       ListNamespaceDescriptorsResponse.Builder response =
         ListNamespaceDescriptorsResponse.newBuilder();
-      for(NamespaceDescriptor ns: master.getNamespaces()) {
+      for(NamespaceDescriptor ns: master.listNamespaceDescriptors()) {
         response.addNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(ns));
       }
       return response.build();
@@ -1306,9 +1305,10 @@ public class MasterRpcServices extends RSRpcServices
       master.checkInitialized();
       master.snapshotManager.checkSnapshotSupport();
 
-      // Ensure namespace exists. Will throw exception if non-known NS.
+    // ensure namespace exists
       TableName dstTable = TableName.valueOf(request.getSnapshot().getTable());
-      master.getNamespace(dstTable.getNamespaceAsString());
+      master.getNamespaceDescriptor(dstTable.getNamespaceAsString());
+
       SnapshotDescription reqSnapshot = request.getSnapshot();
       master.snapshotManager.restoreSnapshot(reqSnapshot);
       return RestoreSnapshotResponse.newBuilder().build();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index ec7db0c..af0e490 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -21,20 +21,21 @@ package org.apache.hadoop.hbase.master;
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 
 import com.google.protobuf.Service;
@@ -45,11 +46,6 @@ import com.google.protobuf.Service;
 @InterfaceAudience.Private
 public interface MasterServices extends Server {
   /**
-   * @return Master's instance of {@link ClusterSchema}
-   */
-  ClusterSchema getClusterSchema();
-
-  /**
    * @return Master's instance of the {@link AssignmentManager}
    */
   AssignmentManager getAssignmentManager();
@@ -85,6 +81,11 @@ public interface MasterServices extends Server {
   MasterCoprocessorHost getMasterCoprocessorHost();
 
   /**
+   * @return Master's instance of {@link TableNamespaceManager}
+   */
+  TableNamespaceManager getTableNamespaceManager();
+
+  /**
    * @return Master's instance of {@link MasterQuotaManager}
    */
   MasterQuotaManager getMasterQuotaManager();
@@ -279,6 +280,54 @@ public interface MasterServices extends Server {
   boolean isInitialized();
 
   /**
+   * Create a new namespace
+   * @param descriptor descriptor which describes the new namespace
+   * @param nonceGroup
+   * @param nonce
+   * @throws IOException
+   */
+  public void createNamespace(
+      final NamespaceDescriptor descriptor,
+      final long nonceGroup,
+      final long nonce) throws IOException;
+
+  /**
+   * Create a new namespace synchronously.
+   * @param descriptor descriptor which describes the new namespace
+   * @param nonceGroup
+   * @param nonce
+   * @throws IOException
+   */
+  public void createNamespaceSync(
+      final NamespaceDescriptor descriptor,
+      final long nonceGroup,
+      final long nonce) throws IOException;
+
+  /**
+   * Modify an existing namespace
+   * @param descriptor descriptor which updates the existing namespace
+   * @param nonceGroup
+   * @param nonce
+   * @throws IOException
+   */
+  public void modifyNamespace(
+      final NamespaceDescriptor descriptor,
+      final long nonceGroup,
+      final long nonce) throws IOException;
+
+  /**
+   * Delete an existing namespace. Only empty namespaces (no tables) can be removed.
+   * @param name namespace name
+   * @param nonceGroup
+   * @param nonce
+   * @throws IOException
+   */
+  public void deleteNamespace(
+      final String name,
+      final long nonceGroup,
+      final long nonce) throws IOException;
+
+  /**
    * Abort a procedure.
    * @param procId ID of the procedure
    * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted?
@@ -289,6 +338,21 @@ public interface MasterServices extends Server {
       throws IOException;
 
   /**
+   * Get a namespace descriptor by name
+   * @param name name of namespace descriptor
+   * @return A descriptor
+   * @throws IOException
+   */
+  public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException;
+
+  /**
+   * List available namespace descriptors
+   * @return A descriptor
+   * @throws IOException
+   */
+  public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException;
+
+  /**
    * List procedures
    * @return procedure list
    * @throws IOException

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ProcedureFuture.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ProcedureFuture.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ProcedureFuture.java
deleted file mode 100644
index 0a849fc..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ProcedureFuture.java
+++ /dev/null
@@ -1,132 +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.master;
-
-import java.io.IOException;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.apache.hadoop.hbase.ProcedureInfo;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.procedure2.Procedure;
-import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.Pair;
-
-/**
- * Map Future Interface on to Procedure result processing.
- */
-// Has no extra methods as of now beyond Future<ProcedureInfo>. Use #toString if you want to log
-// procId of procedure.
-// TODO: This should be in Procedure? Have it in master package for now. Lets out ProcedureInfo.
-// Implementation informed by HBaseAdmin#ProcedureFuture.
-@InterfaceAudience.Private
-class ProcedureFuture implements Future<ProcedureInfo> {
-  // Save exception so we can rethrow if called again. Same for result.
-  private ExecutionException exception = null;
-  private ProcedureInfo result = null;
-  private boolean done = false;
-  private boolean cancelled = false;
-  private final Long procId;
-  private final ProcedureExecutor<MasterProcedureEnv> procedureExecutor;
-
-  ProcedureFuture(final ProcedureExecutor<MasterProcedureEnv> procedureExecutor,
-      final long procId) {
-    this.procedureExecutor = procedureExecutor;
-    this.procId = procId;
-  }
-
-  @Override
-  public String toString() {
-    return "procId=" + this.procId;
-  }
-
-  @Override
-  public boolean cancel(boolean mayInterruptIfRunning) {
-    if (!this.cancelled) {
-      this.cancelled = this.procedureExecutor.abort(this.procId, mayInterruptIfRunning);
-    }
-    return this.cancelled;
-  }
-
-  @Override
-  public boolean isCancelled() {
-    return this.cancelled;
-  }
-
-  @Override
-  public boolean isDone() {
-    return done;
-  }
-
-  /**
-   * This method is unsupported. We will throw an UnsupportedOperationException. Only the lazy
-   * would call this method because they can avoid thinking through implication of a Procedure that
-   * might never return so this is disallowed. Use {@link #get(long, TimeUnit)}.
-   */
-  @Override
-  public ProcedureInfo get() throws InterruptedException, ExecutionException {
-    // TODO: should we ever spin forever?
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public ProcedureInfo get(long timeout, TimeUnit unit)
-  throws InterruptedException, ExecutionException, TimeoutException {
-    if (!this.done) {
-      // TODO: add this sort of facility to EnvironmentEdgeManager
-      long deadlineTs = EnvironmentEdgeManager.currentTime() + unit.toMillis(timeout);
-      try {
-        this.result = waitProcedureResult(procId, deadlineTs);
-      } catch (IOException e) {
-        this.exception = new ExecutionException(e);
-      }
-      this.done = true;
-    }
-    if (exception != null) {
-      throw exception;
-    }
-    return result;
-  }
-
-  /**
-   * @param procId
-   * @param deadlineTs
-   * @return A ProcedureInfo instance or null if procedure not found.
-   * @throws IOException
-   * @throws TimeoutException
-   * @throws InterruptedException
-   */
-  private ProcedureInfo waitProcedureResult(long procId, long deadlineTs)
-  throws IOException, TimeoutException, InterruptedException {
-    while (EnvironmentEdgeManager.currentTime() < deadlineTs) {
-      Pair<ProcedureInfo, Procedure> pair = this.procedureExecutor.getResultOrProcedure(procId);
-      if (pair.getFirst() != null) {
-        this.procedureExecutor.removeResult(procId);
-        return pair.getFirst();
-      } else {
-        if (pair.getSecond() == null) return null;
-      }
-      // TODO: Add a wait.
-    }
-    throw new TimeoutException("The procedure " + procId + " is still running");
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/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 07822fd..bbeaf76 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
@@ -27,17 +27,17 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZKNamespaceManager;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
@@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
+import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
 import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
@@ -55,12 +56,10 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import com.google.common.collect.Sets;
 
 /**
- * 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 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}
  */
 @InterfaceAudience.Private
 public class TableNamespaceManager {
@@ -91,7 +90,7 @@ public class TableNamespaceManager {
   private long exclusiveLockTimeoutMs;
   private long sharedLockTimeoutMs;
 
-  TableNamespaceManager(MasterServices masterServices) {
+  public TableNamespaceManager(MasterServices masterServices) {
     this.masterServices = masterServices;
     this.conf = masterServices.getConfiguration();
 
@@ -114,7 +113,7 @@ public class TableNamespaceManager {
       // 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()) {
+      while (!isTableAvailableAndInitialized(false)) {
         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 "
@@ -270,29 +269,16 @@ public class TableNamespaceManager {
   }
 
   /**
-   * Create Namespace in a blocking manner; don't return till success.
-   * Note, by-passes notifying coprocessors and name checks. Use for system namespaces only.
-   * @throws IOException 
-   * @throws InterruptedException 
-   */
-  private void createNamespace(final NamespaceDescriptor namespaceDescriptor)
-  throws IOException {
-    ClusterSchema clusterSchema = this.masterServices.getClusterSchema();
-    clusterSchema.get(clusterSchema.
-      createNamespace(namespaceDescriptor, HConstants.NO_NONCE, HConstants.NO_NONCE));
-  }
-
-  /**
    * 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
+   * tries to create its HTable. 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
    * @throws IOException
    */
   @SuppressWarnings("deprecation")
-  public synchronized boolean isTableAvailableAndInitialized()
-  throws IOException {
+  public synchronized boolean isTableAvailableAndInitialized(
+      final boolean createNamespaceAync) throws IOException {
     // Did we already get a table? If so, still make sure it's available
     if (isTableNamespaceManagerInitialized()) {
       return true;
@@ -307,10 +293,34 @@ public class TableNamespaceManager {
         zkNamespaceManager.start();
 
         if (get(nsTable, NamespaceDescriptor.DEFAULT_NAMESPACE.getName()) == null) {
-          createNamespace(NamespaceDescriptor.DEFAULT_NAMESPACE);
+          if (createNamespaceAync) {
+            masterServices.getMasterProcedureExecutor().submitProcedure(
+              new CreateNamespaceProcedure(
+                masterServices.getMasterProcedureExecutor().getEnvironment(),
+                NamespaceDescriptor.DEFAULT_NAMESPACE));
+            initGoodSofar = false;
+          }
+          else {
+            masterServices.createNamespaceSync(
+              NamespaceDescriptor.DEFAULT_NAMESPACE,
+              HConstants.NO_NONCE,
+              HConstants.NO_NONCE);
+          }
         }
         if (get(nsTable, NamespaceDescriptor.SYSTEM_NAMESPACE.getName()) == null) {
-          createNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE);
+          if (createNamespaceAync) {
+            masterServices.getMasterProcedureExecutor().submitProcedure(
+              new CreateNamespaceProcedure(
+                masterServices.getMasterProcedureExecutor().getEnvironment(),
+                NamespaceDescriptor.SYSTEM_NAMESPACE));
+            initGoodSofar = false;
+          }
+          else {
+            masterServices.createNamespaceSync(
+              NamespaceDescriptor.SYSTEM_NAMESPACE,
+              HConstants.NO_NONCE,
+              HConstants.NO_NONCE);
+          }
         }
 
         if (!initGoodSofar) {
@@ -400,4 +410,4 @@ public class TableNamespaceManager {
     }
     return maxRegions;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/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 f934737..657bbfb 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
@@ -348,7 +348,7 @@ public class CreateNamespaceProcedure
   }
 
   private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
-    return env.getMasterServices().getClusterSchema().getTableNamespaceManager();
+    return env.getMasterServices().getTableNamespaceManager();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/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 2f99167..5a42614 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
@@ -383,7 +383,7 @@ public class DeleteNamespaceProcedure
   }
 
   private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
-    return env.getMasterServices().getClusterSchema().getTableNamespaceManager();
+    return env.getMasterServices().getTableNamespaceManager();
   }
   /**
    * The procedure could be restarted from a different machine. If the variable is null, we need to

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
index 0f8c172..30de252 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java
@@ -266,9 +266,8 @@ public class ModifyNamespaceProcedure
   }
 
   private TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
-    return env.getMasterServices().getClusterSchema().getTableNamespaceManager();
+    return env.getMasterServices().getTableNamespaceManager();
   }
-
   /**
    * The procedure could be restarted from a different machine. If the variable is null, we need to
    * retrieve it.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
index 8035d32..f24f8c0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java
@@ -65,7 +65,7 @@ class NamespaceStateManager {
 
   /**
    * Gets an instance of NamespaceTableAndRegionInfo associated with namespace.
-   * @param name The name of the namespace
+   * @param The name of the namespace
    * @return An instance of NamespaceTableAndRegionInfo.
    */
   public NamespaceTableAndRegionInfo getState(String name) {
@@ -135,7 +135,7 @@ class NamespaceStateManager {
 
   private NamespaceDescriptor getNamespaceDescriptor(String namespaceAsString) {
     try {
-      return this.master.getClusterSchema().getNamespace(namespaceAsString);
+      return this.master.getNamespaceDescriptor(namespaceAsString);
     } catch (IOException e) {
       LOG.error("Error while fetching namespace descriptor for namespace : " + namespaceAsString);
       return null;
@@ -212,7 +212,7 @@ class NamespaceStateManager {
    * Initialize namespace state cache by scanning meta table.
    */
   private void initialize() throws IOException {
-    List<NamespaceDescriptor> namespaces = this.master.getClusterSchema().getNamespaces();
+    List<NamespaceDescriptor> namespaces = this.master.listNamespaceDescriptors();
     for (NamespaceDescriptor namespace : namespaces) {
       addNamespace(namespace.getName());
       List<TableName> tables = this.master.listTableNamesByNamespace(namespace.getName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 1255fa4..211fed5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -196,7 +196,8 @@ import sun.misc.SignalHandler;
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
 @SuppressWarnings("deprecation")
-public class HRegionServer extends HasThread implements RegionServerServices, LastSequenceId {
+public class HRegionServer extends HasThread implements
+    RegionServerServices, LastSequenceId {
 
   private static final Log LOG = LogFactory.getLog(HRegionServer.class);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
index f9e2a16..c24d8a3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
@@ -358,7 +358,7 @@ public class TestNamespace {
     runWithExpectedException(new Callable<Void>() {
       @Override
       public Void call() throws Exception {
-        admin.listTableDescriptorsByNamespace("non_existant_namespace");
+        admin.listTableDescriptorsByNamespace("non_existing_namespace");
         return null;
       }
     }, NamespaceNotFoundException.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index 785a006..8e35bbf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaMockingUtil;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.Server;
@@ -142,10 +143,11 @@ public class TestCatalogJanitor {
             ServerName.valueOf("example.org,12345,6789"),
           HRegionInfo.FIRST_META_REGIONINFO);
       // Set hbase.rootdir into test dir.
-      FileSystem.get(this.c);
+      FileSystem fs = FileSystem.get(this.c);
       Path rootdir = FSUtils.getRootDir(this.c);
       FSUtils.setRootDir(this.c, rootdir);
-      Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
+      AdminProtos.AdminService.BlockingInterface hri =
+        Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
     }
 
     @Override
@@ -400,6 +402,48 @@ public class TestCatalogJanitor {
     }
 
     @Override
+    public void createNamespace(
+        final NamespaceDescriptor descriptor,
+        final long nonceGroup,
+        final long nonce) throws IOException {
+      //To change body of implemented methods use File | Settings | File Templates.
+    }
+
+    @Override
+    public void createNamespaceSync(
+        final NamespaceDescriptor descriptor,
+        final long nonceGroup,
+        final long nonce) throws IOException {
+      //To change body of implemented methods use File | Settings | File Templates.
+    }
+
+    @Override
+    public void modifyNamespace(
+        final NamespaceDescriptor descriptor,
+        final long nonceGroup,
+        final long nonce) throws IOException {
+      //To change body of implemented methods use File | Settings | File Templates.
+    }
+
+    @Override
+    public void deleteNamespace(
+        final String name,
+        final long nonceGroup,
+        final long nonce) throws IOException {
+      //To change body of implemented methods use File | Settings | File Templates.
+    }
+
+    @Override
+    public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException {
+      return null;  //To change body of implemented methods use File | Settings | File Templates.
+    }
+
+    @Override
+    public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException {
+      return null;  //To change body of implemented methods use File | Settings | File Templates.
+    }
+
+    @Override
     public boolean abortProcedure(final long procId, final boolean mayInterruptIfRunning)
         throws IOException {
       return false;  //To change body of implemented methods use File | Settings | File Templates.
@@ -492,29 +536,32 @@ public class TestCatalogJanitor {
     }
 
     @Override
+    public TableNamespaceManager getTableNamespaceManager() {
+      return null;
+    }
+
+    @Override
     public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b,
         boolean forcible) throws IOException {
     }
 
     @Override
     public boolean isInitialized() {
+      // Auto-generated method stub
       return false;
     }
 
     @Override
     public long getLastMajorCompactionTimestamp(TableName table) throws IOException {
+      // Auto-generated method stub
       return 0;
     }
 
     @Override
     public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException {
+      // Auto-generated method stub
       return 0;
     }
-
-    @Override
-    public ClusterSchema getClusterSchema() {
-      return null;
-    }
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/a82f7fc9/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
index 20b492c..972834a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
@@ -218,6 +218,10 @@ public class TestMasterNoCluster {
           return null;
         }
       }
+
+      @Override
+      void initNamespace() {
+      }
     };
     master.start();
 
@@ -290,6 +294,10 @@ public class TestMasterNoCluster {
           return null;
         }
       }
+
+      @Override
+      void initNamespace() {
+      }
     };
     master.start();