http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/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 8e51f25..4472b65 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
@@ -63,7 +63,6 @@ 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;
@@ -79,6 +78,7 @@ import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.coprocessor.BypassCoprocessorException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.executor.ExecutorType;
@@ -97,17 +97,14 @@ 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;
@@ -185,7 +182,7 @@ import com.google.protobuf.Service;
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
 @SuppressWarnings("deprecation")
-public class HMaster extends HRegionServer implements MasterServices, Server {
+public class HMaster extends HRegionServer implements MasterServices {
   private static final Log LOG = LogFactory.getLog(HMaster.class.getName());
 
   /**
@@ -256,8 +253,7 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
   // Tracker for region normalizer state
   private RegionNormalizerTracker regionNormalizerTracker;
 
-  /** Namespace stuff */
-  private TableNamespaceManager tableNamespaceManager;
+  private ClusterSchemaService clusterSchemaService;
 
   // Metrics for the HMaster
   final MetricsMaster metricsMaster;
@@ -368,9 +364,6 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
    * 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 {
@@ -570,10 +563,6 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
 
   /**
    * Initialize all ZK based system trackers.
-   * @throws IOException
-   * @throws InterruptedException
-   * @throws KeeperException
-   * @throws CoordinatedStateException
    */
   void initializeZKBasedSystemTrackers() throws IOException,
       InterruptedException, KeeperException, CoordinatedStateException {
@@ -588,12 +577,10 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
       this.balancer, this.service, this.metricsMaster,
       this.tableLockManager, tableStateManager);
 
-    this.regionServerTracker = new RegionServerTracker(zooKeeper, this,
-        this.serverManager);
+    this.regionServerTracker = new RegionServerTracker(zooKeeper, this, 
this.serverManager);
     this.regionServerTracker.start();
 
-    this.drainingServerTracker = new DrainingServerTracker(zooKeeper, this,
-      this.serverManager);
+    this.drainingServerTracker = new DrainingServerTracker(zooKeeper, this, 
this.serverManager);
     this.drainingServerTracker.start();
 
     // Set the cluster as up.  If new RSs, they'll be waiting on this before
@@ -630,11 +617,6 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
    * <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 {
@@ -781,8 +763,8 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
     this.catalogJanitorChore = new CatalogJanitor(this, this);
     getChoreService().scheduleChore(catalogJanitorChore);
 
-    status.setStatus("Starting namespace manager");
-    initNamespace();
+    status.setStatus("Starting cluster schema service");
+    initClusterSchemaService();
 
     if (this.cpHost != null) {
       try {
@@ -848,11 +830,6 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
 
   /**
    * 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)
@@ -874,7 +851,7 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
           RegionState r = MetaTableLocator.getMetaRegionState(zkw, replicaId);
           LOG.info("Closing excess replica of meta region " + r.getRegion());
           // send a close and wait for a max of 30 seconds
-          ServerManager.closeRegionSilentlyAndWait(getConnection(), 
r.getServerName(),
+          ServerManager.closeRegionSilentlyAndWait(getClusterConnection(), 
r.getServerName(),
               r.getRegion(), 30000);
           ZKUtil.deleteNode(zkw, zkw.getZNodeForReplica(replicaId));
         }
@@ -888,12 +865,6 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
 
   /**
    * 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 {
@@ -915,7 +886,7 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
         metaState.getServerName(), null);
 
     if (!metaState.isOpened() || !metaTableLocator.verifyMetaRegionLocation(
-        this.getConnection(), this.getZooKeeper(), timeout, replicaId)) {
+        this.getClusterConnection(), this.getZooKeeper(), timeout, replicaId)) 
{
       ServerName currentMetaServer = metaState.getServerName();
       if (serverManager.isServerOnline(currentMetaServer)) {
         if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
@@ -965,10 +936,10 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
     status.setStatus("META assigned.");
   }
 
-  void initNamespace() throws IOException {
-    //create namespace manager
-    tableNamespaceManager = new TableNamespaceManager(this);
-    tableNamespaceManager.start();
+  void initClusterSchemaService() throws IOException, InterruptedException {
+    this.clusterSchemaService = new ClusterSchemaServiceImpl(this);
+    this.clusterSchemaService.startAndWait();
+    if (!this.clusterSchemaService.isRunning()) throw new 
HBaseIOException("Failed start");
   }
 
   void initQuotaManager() throws IOException {
@@ -1014,7 +985,6 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
   /**
    * 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>();
@@ -1050,11 +1020,6 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
     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
@@ -1201,7 +1166,6 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
 
   /**
    * @return Get remote side's InetAddress
-   * @throws UnknownHostException
    */
   InetAddress getRemoteInetAddress(final int port,
       final long serverStartCode) throws UnknownHostException {
@@ -1336,9 +1300,8 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
    * Perform normalization of cluster (invoked by {@link 
RegionNormalizerChore}).
    *
    * @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
+   *    (specifically, if HMaster hasn't been initialized properly or 
normalization
+   *    is globally disabled)
    */
   public boolean normalizeRegions() throws IOException {
     if (!this.initialized) {
@@ -1478,9 +1441,9 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
     if (isStopped()) {
       throw new MasterNotRunningException();
     }
-
+    checkInitialized();
     String namespace = hTableDescriptor.getTableName().getNamespaceAsString();
-    ensureNamespaceExists(namespace);
+    this.clusterSchemaService.getNamespace(namespace);
 
     HRegionInfo[] newRegions = 
ModifyRegionUtils.createHRegionInfos(hTableDescriptor, splitKeys);
     checkInitialized();
@@ -2167,8 +2130,7 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
    * The set of loaded coprocessors is stored in a static set. Since it's
    * statically allocated, it does not require that HMaster's cpHost be
    * initialized prior to accessing it.
-   * @return a String representation of the set of names of the loaded
-   * coprocessors.
+   * @return a String representation of the set of names of the loaded 
coprocessors.
    */
   public static String getLoadedCoprocessors() {
     return CoprocessorHost.getLoadedCoprocessors().toString();
@@ -2305,18 +2267,9 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
 
   void checkInitialized() throws PleaseHoldException, 
ServerNotRunningYetException {
     checkServiceStarted();
-    if (!this.initialized) {
-      throw new PleaseHoldException("Master is initializing");
-    }
+    if (!isInitialized()) 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.
@@ -2411,7 +2364,6 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
   /**
    * 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,
@@ -2452,138 +2404,116 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
   }
 
   @Override
-  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);
-    }
+  public ClusterSchema getClusterSchema() {
+    return this.clusterSchemaService;
   }
 
-  @Override
-  public void createNamespaceSync(
-      final NamespaceDescriptor descriptor,
-      final long nonceGroup,
-      final long nonce) throws IOException {
-    LOG.info(getClientIdAuditPrefix() + " creating " + 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).
+   * @return procedure id
+   */
+  long 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)) {
+      throw new BypassCoprocessorException();
+    }
+    LOG.info(getClientIdAuditPrefix() + " creating " + namespaceDescriptor);
     // Execute the operation synchronously - wait for the operation to 
complete before continuing.
-    long procId = this.procedureExecutor.submitProcedure(
-      new CreateNamespaceProcedure(procedureExecutor.getEnvironment(), 
descriptor),
-      nonceGroup,
-      nonce);
-    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
+    long procId = getClusterSchema().createNamespace(namespaceDescriptor, 
nonceGroup, nonce);
+    if (this.cpHost != null) 
this.cpHost.postCreateNamespace(namespaceDescriptor);
+    return procId;
   }
 
-  @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;
-      }
+  /**
+   * 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 procedure id
+   */
+  long 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)) {
+      throw new BypassCoprocessorException();
     }
-    LOG.info(getClientIdAuditPrefix() + " modify " + descriptor);
+    LOG.info(getClientIdAuditPrefix() + " modify " + namespaceDescriptor);
     // Execute the operation synchronously - wait for the operation to 
complete before continuing.
-    long procId = this.procedureExecutor.submitProcedure(
-      new ModifyNamespaceProcedure(procedureExecutor.getEnvironment(), 
descriptor),
-      nonceGroup,
-      nonce);
-    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
-    if (cpHost != null) {
-      cpHost.postModifyNamespace(descriptor);
-    }
+    long procId = getClusterSchema().modifyNamespace(namespaceDescriptor, 
nonceGroup, nonce);
+    if (this.cpHost != null) 
this.cpHost.postModifyNamespace(namespaceDescriptor);
+    return procId;
   }
 
-  @Override
-  public void deleteNamespace(
-      final String name,
-      final long nonceGroup,
-      final long nonce) throws IOException {
-    checkNamespaceManagerReady();
-    if (cpHost != null) {
-      if (cpHost.preDeleteNamespace(name)) {
-        return;
-      }
+  /**
+   * 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 procedure id
+   */
+  long deleteNamespace(final String name, final long nonceGroup, final long 
nonce)
+  throws IOException {
+    checkInitialized();
+    if (this.cpHost != null && this.cpHost.preDeleteNamespace(name)) {
+      throw new BypassCoprocessorException();
     }
     LOG.info(getClientIdAuditPrefix() + " delete " + name);
     // Execute the operation synchronously - wait for the operation to 
complete before continuing.
-    long procId = this.procedureExecutor.submitProcedure(
-      new DeleteNamespaceProcedure(procedureExecutor.getEnvironment(), name),
-      nonceGroup,
-      nonce);
-    ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
-    if (cpHost != null) {
-      cpHost.postDeleteNamespace(name);
-    }
+    long procId = getClusterSchema().deleteNamespace(name, nonceGroup, nonce);
+    if (this.cpHost != null) this.cpHost.postDeleteNamespace(name);
+    return procId;
   }
 
   /**
-   * 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
+   * Get a Namespace
+   * @param name Name of the Namespace
+   * @return Namespace descriptor for <code>name</code>
    */
-  private void ensureNamespaceExists(final String name)
-      throws IOException, NamespaceNotFoundException {
-    checkNamespaceManagerReady();
-    NamespaceDescriptor nsd = tableNamespaceManager.get(name);
-    if (nsd == null) {
-      throw new NamespaceNotFoundException(name);
-    }
+  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;
   }
 
-  @Override
-  public NamespaceDescriptor getNamespaceDescriptor(String name) throws 
IOException {
-    checkNamespaceManagerReady();
-
+  /**
+   * Get all Namespaces
+   * @return All Namespace descriptors
+   */
+  List<NamespaceDescriptor> getNamespaces() throws IOException {
+    checkInitialized();
+    final List<NamespaceDescriptor> nsds = new 
ArrayList<NamespaceDescriptor>();
+    boolean bypass = false;
     if (cpHost != null) {
-      cpHost.preGetNamespaceDescriptor(name);
+      bypass = cpHost.preListNamespaceDescriptors(nsds);
     }
-
-    NamespaceDescriptor nsd = tableNamespaceManager.get(name);
-    if (nsd == null) {
-      throw new NamespaceNotFoundException(name);
-    }
-
-    if (cpHost != null) {
-      cpHost.postGetNamespaceDescriptor(nsd);
+    if (!bypass) {
+      nsds.addAll(this.clusterSchemaService.getNamespaces());
+      if (this.cpHost != null) this.cpHost.postListNamespaceDescriptors(nsds);
     }
-
-    return nsd;
+    return nsds;
   }
 
   @Override
-  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());
+  public List<TableName> listTableNamesByNamespace(String name) throws 
IOException {
+    checkInitialized();
+    return listTableNames(name, null, true);
+  }
 
-      if (cpHost != null) {
-        cpHost.postListNamespaceDescriptors(descriptors);
-      }
-    }
-    return descriptors;
+  @Override
+  public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) 
throws IOException {
+    checkInitialized();
+    return listTableDescriptors(name, null, null, true);
   }
 
   @Override
@@ -2617,21 +2547,8 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
     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
-   *
    * @param namespace the namespace to query, or null if querying for all
    * @param regex The regular expression to match against, or null if querying 
for all
    * @param tableNameList the list of table names, or null if querying for all
@@ -2640,51 +2557,17 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
    */
   public List<HTableDescriptor> listTableDescriptors(final String namespace, 
final String regex,
       final List<TableName> tableNameList, final boolean includeSysTables)
-      throws IOException {
-    final List<HTableDescriptor> descriptors = new 
ArrayList<HTableDescriptor>();
-
-    boolean bypass = false;
-    if (cpHost != null) {
-      bypass = cpHost.preGetTableDescriptors(tableNameList, descriptors, 
regex);
-    }
-
+  throws IOException {
+    List<HTableDescriptor> htds = new ArrayList<HTableDescriptor>();
+    boolean bypass = cpHost != null?
+        cpHost.preGetTableDescriptors(tableNameList, htds, regex): false;
     if (!bypass) {
-      if (tableNameList == null || tableNameList.size() == 0) {
-        // request for all TableDescriptors
-        Collection<HTableDescriptor> htds;
-        if (namespace != null && namespace.length() > 0) {
-          htds = tableDescriptors.getByNamespace(namespace).values();
-        } else {
-          htds = tableDescriptors.getAll().values();
-        }
-
-        for (HTableDescriptor desc: htds) {
-          if (tableStateManager.isTablePresent(desc.getTableName())
-              && (includeSysTables || !desc.getTableName().isSystemTable())) {
-            descriptors.add(desc);
-          }
-        }
-      } else {
-        for (TableName s: tableNameList) {
-          if (tableStateManager.isTablePresent(s)) {
-            HTableDescriptor desc = tableDescriptors.get(s);
-            if (desc != null) {
-              descriptors.add(desc);
-            }
-          }
-        }
-      }
-
-      // Retains only those matched by regular expression.
-      if (regex != null) {
-        filterTablesByRegex(descriptors, Pattern.compile(regex));
-      }
-
+      htds = getTableDescriptors(htds, namespace, regex, tableNameList, 
includeSysTables);
       if (cpHost != null) {
-        cpHost.postGetTableDescriptors(tableNameList, descriptors, regex);
+        cpHost.postGetTableDescriptors(tableNameList, htds, regex);
       }
     }
-    return descriptors;
+    return htds;
   }
 
   /**
@@ -2696,46 +2579,58 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
    */
   public List<TableName> listTableNames(final String namespace, final String 
regex,
       final boolean includeSysTables) throws IOException {
-    final List<HTableDescriptor> descriptors = new 
ArrayList<HTableDescriptor>();
-
-    boolean bypass = false;
-    if (cpHost != null) {
-      bypass = cpHost.preGetTableNames(descriptors, regex);
+    List<HTableDescriptor> htds = new ArrayList<HTableDescriptor>();
+    boolean bypass = cpHost != null? cpHost.preGetTableNames(htds, regex): 
false;
+    if (!bypass) {
+      htds = getTableDescriptors(htds, namespace, regex, null, 
includeSysTables);
+      if (cpHost != null) cpHost.postGetTableNames(htds, regex);
     }
+    List<TableName> result = new ArrayList<TableName>(htds.size());
+    for (HTableDescriptor htd: htds) result.add(htd.getTableName());
+    return result;
+  }
 
-    if (!bypass) {
-      // get all descriptors
-      Collection<HTableDescriptor> htds;
+  /**
+   * @return list of table table descriptors after filtering by regex and 
whether to include system
+   *    tables, etc.
+   * @throws IOException
+   */
+  private List<HTableDescriptor> getTableDescriptors(final 
List<HTableDescriptor> htds,
+      final String namespace, final String regex, final List<TableName> 
tableNameList,
+      final boolean includeSysTables)
+  throws IOException {
+    if (tableNameList == null || tableNameList.size() == 0) {
+      // request for all TableDescriptors
+      Collection<HTableDescriptor> allHtds;
       if (namespace != null && namespace.length() > 0) {
-        htds = tableDescriptors.getByNamespace(namespace).values();
+        // Do a check on the namespace existence. Will fail if does not exist.
+        this.clusterSchemaService.getNamespace(namespace);
+        allHtds = tableDescriptors.getByNamespace(namespace).values();
       } else {
-        htds = tableDescriptors.getAll().values();
+        allHtds = tableDescriptors.getAll().values();
       }
-
-      for (HTableDescriptor htd: htds) {
-        if (includeSysTables || !htd.getTableName().isSystemTable()) {
-          descriptors.add(htd);
+      for (HTableDescriptor desc: allHtds) {
+        if (tableStateManager.isTablePresent(desc.getTableName())
+            && (includeSysTables || !desc.getTableName().isSystemTable())) {
+          htds.add(desc);
         }
       }
-
-      // Retains only those matched by regular expression.
-      if (regex != null) {
-        filterTablesByRegex(descriptors, Pattern.compile(regex));
-      }
-
-      if (cpHost != null) {
-        cpHost.postGetTableNames(descriptors, regex);
+    } else {
+      for (TableName s: tableNameList) {
+        if (tableStateManager.isTablePresent(s)) {
+          HTableDescriptor desc = tableDescriptors.get(s);
+          if (desc != null) {
+            htds.add(desc);
+          }
+        }
       }
     }
 
-    List<TableName> result = new ArrayList<TableName>(descriptors.size());
-    for (HTableDescriptor htd: descriptors) {
-      result.add(htd.getTableName());
-    }
-    return result;
+    // Retains only those matched by regular expression.
+    if (regex != null) filterTablesByRegex(htds, Pattern.compile(regex));
+    return htds;
   }
 
-
   /**
    * Removes the table descriptors that don't match the pattern.
    * @param descriptors list of table descriptors to filter
@@ -2848,11 +2743,8 @@ public class HMaster extends HRegionServer implements 
MasterServices, Server {
    * Queries the state of the {@link RegionNormalizerTracker}. If it's not 
initialized,
    * false is returned.
    */
-   public boolean isNormalizerOn() {
-    if (null == regionNormalizerTracker) {
-      return false;
-    }
-    return regionNormalizerTracker.isNormalizerOn();
+  public boolean isNormalizerOn() {
+    return null == regionNormalizerTracker? false: 
regionNormalizerTracker.isNormalizerOn();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/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 b269c3d..141fa88 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
@@ -55,121 +55,21 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
-import org.apache.hadoop.hbase.protobuf.generated.*;
 import 
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
 import 
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionResponse;
 import 
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
 import 
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
 import 
org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
 import 
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
 import 
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
 import 
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AbortProcedureRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AbortProcedureResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListProceduresRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListProceduresResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.*;
 import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetQuotaRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetQuotaResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
-import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionResponse;
 import 
org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
 import 
org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
 import 
org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
@@ -188,8 +88,8 @@ import 
org.apache.hadoop.hbase.security.access.AccessController;
 import org.apache.hadoop.hbase.security.visibility.VisibilityController;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ByteStringer;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.zookeeper.KeeperException;
 
@@ -458,11 +358,11 @@ public class MasterRpcServices extends RSRpcServices
   public CreateNamespaceResponse createNamespace(RpcController controller,
      CreateNamespaceRequest request) throws ServiceException {
     try {
-      master.createNamespace(
+      long procId = master.createNamespace(
         ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()),
         request.getNonceGroup(),
         request.getNonce());
-      return CreateNamespaceResponse.getDefaultInstance();
+      return CreateNamespaceResponse.newBuilder().setProcId(procId).build();
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -506,11 +406,11 @@ public class MasterRpcServices extends RSRpcServices
   public DeleteNamespaceResponse deleteNamespace(RpcController controller,
       DeleteNamespaceRequest request) throws ServiceException {
     try {
-      master.deleteNamespace(
+      long procId = master.deleteNamespace(
         request.getNamespaceName(),
         request.getNonceGroup(),
         request.getNonce());
-      return DeleteNamespaceResponse.getDefaultInstance();
+      return DeleteNamespaceResponse.newBuilder().setProcId(procId).build();
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -832,7 +732,7 @@ public class MasterRpcServices extends RSRpcServices
     try {
       return GetNamespaceDescriptorResponse.newBuilder()
         .setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(
-            master.getNamespaceDescriptor(request.getNamespaceName())))
+            master.getNamespace(request.getNamespaceName())))
         .build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -977,10 +877,8 @@ public class MasterRpcServices extends RSRpcServices
 
   /**
    * Checks if the specified procedure is done.
-   * @return true if the procedure is done,
-   *   false if the procedure is in the process of completing
-   * @throws ServiceException if invalid procedure, or
-   *  a failed procedure with progress failure reason.
+   * @return true if the procedure is done, false if the procedure is in the 
process of completing
+   * @throws ServiceException if invalid procedure or failed procedure with 
progress failure reason.
    */
   @Override
   public IsProcedureDoneResponse isProcedureDone(RpcController controller,
@@ -1120,7 +1018,7 @@ public class MasterRpcServices extends RSRpcServices
     try {
       ListNamespaceDescriptorsResponse.Builder response =
         ListNamespaceDescriptorsResponse.newBuilder();
-      for(NamespaceDescriptor ns: master.listNamespaceDescriptors()) {
+      for(NamespaceDescriptor ns: master.getNamespaces()) {
         
response.addNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(ns));
       }
       return response.build();
@@ -1200,11 +1098,11 @@ public class MasterRpcServices extends RSRpcServices
   public ModifyNamespaceResponse modifyNamespace(RpcController controller,
       ModifyNamespaceRequest request) throws ServiceException {
     try {
-      master.modifyNamespace(
+      long procId = master.modifyNamespace(
         ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()),
         request.getNonceGroup(),
         request.getNonce());
-      return ModifyNamespaceResponse.getDefaultInstance();
+      return ModifyNamespaceResponse.newBuilder().setProcId(procId).build();
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -1305,10 +1203,9 @@ public class MasterRpcServices extends RSRpcServices
       master.checkInitialized();
       master.snapshotManager.checkSnapshotSupport();
 
-    // ensure namespace exists
+      // Ensure namespace exists. Will throw exception if non-known NS.
       TableName dstTable = TableName.valueOf(request.getSnapshot().getTable());
-      master.getNamespaceDescriptor(dstTable.getNamespaceAsString());
-
+      master.getNamespace(dstTable.getNamespaceAsString());
       SnapshotDescription reqSnapshot = request.getSnapshot();
       master.snapshotManager.restoreSnapshot(reqSnapshot);
       return RestoreSnapshotResponse.newBuilder().build();

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/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 af0e490..ec7db0c 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,21 +21,20 @@ 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;
@@ -46,6 +45,11 @@ 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();
@@ -81,11 +85,6 @@ 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();
@@ -280,54 +279,6 @@ 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?
@@ -338,21 +289,6 @@ 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/46303dfd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index 50f07c1..a95279c 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -215,7 +215,7 @@ public class ServerManager {
     Configuration c = master.getConfiguration();
     maxSkew = c.getLong("hbase.master.maxclockskew", 30000);
     warningSkew = c.getLong("hbase.master.warningclockskew", 10000);
-    this.connection = connect ? master.getConnection() : null;
+    this.connection = connect ? master.getClusterConnection() : null;
     int pingMaxAttempts = Math.max(1, master.getConfiguration().getInt(
       "hbase.master.maximum.ping.server.attempts", 10));
     int pingSleepInterval = Math.max(1, master.getConfiguration().getInt(

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/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 bbeaf76..69d1280 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.MetaTableAccessor;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
@@ -46,20 +46,25 @@ 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.exceptions.TimeoutIOException;
 import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Threads;
 
 import com.google.common.collect.Sets;
 
 /**
- * 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}
+ * 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.
  */
 @InterfaceAudience.Private
 public class TableNamespaceManager {
@@ -90,7 +95,7 @@ public class TableNamespaceManager {
   private long exclusiveLockTimeoutMs;
   private long sharedLockTimeoutMs;
 
-  public TableNamespaceManager(MasterServices masterServices) {
+  TableNamespaceManager(MasterServices masterServices) {
     this.masterServices = masterServices;
     this.conf = masterServices.getConfiguration();
 
@@ -104,7 +109,7 @@ public class TableNamespaceManager {
 
   public void start() throws IOException {
     if (!MetaTableAccessor.tableExists(masterServices.getConnection(),
-      TableName.NAMESPACE_TABLE_NAME)) {
+        TableName.NAMESPACE_TABLE_NAME)) {
       LOG.info("Namespace table not found. Creating...");
       createNamespaceTable(masterServices);
     }
@@ -113,7 +118,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(false)) {
+      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 "
@@ -269,16 +274,48 @@ public class TableNamespaceManager {
   }
 
   /**
+   * 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, HConstants.NO_NONCE, 
HConstants.NO_NONCE);
+    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 " + procId + " is still running");
+  }
+
+  /**
    * This method checks if the namespace table is assigned and then
-   * tries to create its HTable. If it was already created before, it also 
makes
+   * 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
-   * @throws IOException
+   * @return true if the namespace table manager is ready to serve, false 
otherwise
    */
   @SuppressWarnings("deprecation")
-  public synchronized boolean isTableAvailableAndInitialized(
-      final boolean createNamespaceAync) throws IOException {
+  public synchronized boolean isTableAvailableAndInitialized()
+  throws IOException {
     // Did we already get a table? If so, still make sure it's available
     if (isTableNamespaceManagerInitialized()) {
       return true;
@@ -293,34 +330,10 @@ public class TableNamespaceManager {
         zkNamespaceManager.start();
 
         if (get(nsTable, NamespaceDescriptor.DEFAULT_NAMESPACE.getName()) == 
null) {
-          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);
-          }
+          blockingCreateNamespace(NamespaceDescriptor.DEFAULT_NAMESPACE);
         }
         if (get(nsTable, NamespaceDescriptor.SYSTEM_NAMESPACE.getName()) == 
null) {
-          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);
-          }
+          blockingCreateNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE);
         }
 
         if (!initGoodSofar) {
@@ -410,4 +423,4 @@ public class TableNamespaceManager {
     }
     return maxRegions;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/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 657bbfb..f934737 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().getTableNamespaceManager();
+    return 
env.getMasterServices().getClusterSchema().getTableNamespaceManager();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/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 5a42614..2f99167 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().getTableNamespaceManager();
+    return 
env.getMasterServices().getClusterSchema().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/46303dfd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
index 1e86254..baef112 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
@@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.backup.HFileArchiver;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -374,7 +374,7 @@ public class DeleteTableProcedure
    */
   private static void cleanAnyRemainingRows(final MasterProcedureEnv env,
       final TableName tableName) throws IOException {
-    ClusterConnection connection = env.getMasterServices().getConnection();
+    Connection connection = env.getMasterServices().getConnection();
     Scan tableScan = MetaTableAccessor.getScanForTableName(connection, 
tableName);
     try (Table metaTable =
         connection.getTable(TableName.META_TABLE_NAME)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/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 30de252..0f8c172 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,8 +266,9 @@ public class ModifyNamespaceProcedure
   }
 
   private TableNamespaceManager getTableNamespaceManager(final 
MasterProcedureEnv env) {
-    return env.getMasterServices().getTableNamespaceManager();
+    return 
env.getMasterServices().getClusterSchema().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/46303dfd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
index 5c9f6f4..bdcd89c 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
@@ -730,7 +730,7 @@ implements ServerProcedureInterface {
     boolean metaAssigned = false;
     // Is hbase:meta location available yet?
     if (mtl.isLocationAvailable(zkw)) {
-      ClusterConnection connection = env.getMasterServices().getConnection();
+      ClusterConnection connection = 
env.getMasterServices().getClusterConnection();
       // Is hbase:meta location good yet?
       long timeout =
         env.getMasterConfiguration().getLong(KEY_SHORT_WAIT_ON_META, 
DEFAULT_SHORT_WAIT_ON_META);

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/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 f24f8c0..8035d32 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 The name of the namespace
+   * @param name 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.getNamespaceDescriptor(namespaceAsString);
+      return this.master.getClusterSchema().getNamespace(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.listNamespaceDescriptors();
+    List<NamespaceDescriptor> namespaces = 
this.master.getClusterSchema().getNamespaces();
     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/46303dfd/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 211fed5..00046ba 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
@@ -79,6 +79,7 @@ import org.apache.hadoop.hbase.YouAreDeadException;
 import org.apache.hadoop.hbase.ZNodeClearer;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionUtils;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
 import org.apache.hadoop.hbase.conf.ConfigurationManager;
@@ -196,8 +197,7 @@ 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);
 
@@ -1867,7 +1867,12 @@ public class HRegionServer extends HasThread implements
   }
 
   @Override
-  public ClusterConnection getConnection() {
+  public Connection getConnection() {
+    return getClusterConnection();
+  }
+
+  @Override
+  public ClusterConnection getClusterConnection() {
     return this.clusterConnection;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
index 8d38b09..b86de12 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
@@ -187,5 +187,11 @@ public class ReplicationSyncUp extends Configured 
implements Tool {
     public ChoreService getChoreService() {
       return null;
     }
+
+    @Override
+    public ClusterConnection getClusterConnection() {
+      // TODO Auto-generated method stub
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp 
b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
index f132b2b..878c5bf 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
@@ -124,7 +124,7 @@
 </div>
 <%
 if ( fqtn != null ) {
-  table = (HTable) master.getConnection().getTable(fqtn);
+  table = (HTable) master.getConnection().getTable(TableName.valueOf(fqtn));
   if (table.getTableDescriptor().getRegionReplication() > 1) {
     tableHeader = "<h2>Table Regions</h2><table class=\"table table-striped\" 
style=\"table-layout: fixed; word-wrap: break-word;\"><tr><th 
style=\"width:22%\">Name</th><th>Region Server</th><th 
style=\"width:22%\">Start Key</th><th style=\"width:22%\">End 
Key</th><th>Locality</th><th>Requests</th><th>ReplicaID</th></tr>";
     withReplica = true;
@@ -199,7 +199,7 @@ if ( fqtn != null ) {
 </table>
 <%} else {
   Admin admin = master.getConnection().getAdmin();
-  RegionLocator r = master.getConnection().getRegionLocator(table.getName());
+  RegionLocator r = 
master.getClusterConnection().getRegionLocator(table.getName());
   try { %>
 <h2>Table Attributes</h2>
 <table class="table table-striped">

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index c126b19..a7fc75b 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -306,4 +306,10 @@ public class MockRegionServerServices implements 
RegionServerServices {
   public double getCompactionPressure() {
     return 0;
   }
+
+  @Override
+  public ClusterConnection getClusterConnection() {
+    // TODO Auto-generated method stub
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/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 c24d8a3..f9e2a16 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_existing_namespace");
+        admin.listTableDescriptorsByNamespace("non_existant_namespace");
         return null;
       }
     }, NamespaceNotFoundException.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java
index e84d34c..618717b 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java
@@ -65,7 +65,7 @@ public class TestShortCircuitConnection {
     htd.addFamily(hcd);
     UTIL.createTable(htd, null);
     HRegionServer regionServer = UTIL.getRSForFirstRegionInTable(tn);
-    ClusterConnection connection = regionServer.getConnection();
+    ClusterConnection connection = regionServer.getClusterConnection();
     HTableInterface tableIf = connection.getTable(tn);
     assertTrue(tableIf instanceof HTable);
     HTable table = (HTable) tableIf;

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
index ef4a579..638811a 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
@@ -1586,7 +1586,13 @@ public class TestMasterObserver {
     cp.enableBypass(true);
     cp.resetStates();
 
-    admin.modifyNamespace(NamespaceDescriptor.create(testNamespace).build());
+    boolean expected = false;
+    try {
+      admin.modifyNamespace(NamespaceDescriptor.create(testNamespace).build());
+    } catch (BypassCoprocessorException ce) {
+      expected = true;
+    }
+    assertTrue(expected);
     assertTrue("Test namespace should not have been modified",
         cp.preModifyNamespaceCalledOnly());
 
@@ -1594,7 +1600,13 @@ public class TestMasterObserver {
     assertTrue("Test namespace descriptor should have been called",
         cp.wasGetNamespaceDescriptorCalled());
 
-    admin.deleteNamespace(testNamespace);
+    expected = false;
+    try {
+      admin.deleteNamespace(testNamespace);
+    } catch (BypassCoprocessorException ce) {
+      expected = true;
+    }
+    assertTrue(expected);
     assertTrue("Test namespace should not have been deleted", 
cp.preDeleteNamespaceCalledOnly());
 
     assertNotNull(admin.getNamespaceDescriptor(testNamespace));
@@ -1614,7 +1626,13 @@ public class TestMasterObserver {
     cp.enableBypass(true);
     cp.resetStates();
 
-    admin.createNamespace(NamespaceDescriptor.create(testNamespace).build());
+    expected = false;
+    try {
+      admin.createNamespace(NamespaceDescriptor.create(testNamespace).build());
+    } catch (BypassCoprocessorException ce) {
+      expected = true;
+    }
+    assertTrue(expected);
     assertTrue("Test namespace should not be created", 
cp.preCreateNamespaceCalledOnly());
 
     // turn on bypass, run the test

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index eb8f803..234ad20 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -651,4 +651,10 @@ ClientProtos.ClientService.BlockingInterface, 
RegionServerServices {
   public double getCompactionPressure() {
     return 0;
   }
+
+  @Override
+  public ClusterConnection getClusterConnection() {
+    // TODO Auto-generated method stub
+    return null;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
index e3283e9..e10ab2a 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java
@@ -326,5 +326,11 @@ public class TestActiveMasterManager {
     public ChoreService getChoreService() {
       return null;
     }
+
+    @Override
+    public ClusterConnection getClusterConnection() {
+      // TODO Auto-generated method stub
+      return null;
+    }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/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 8e35bbf..e26bd82 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,7 +46,6 @@ 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;
@@ -143,11 +142,10 @@ public class TestCatalogJanitor {
             ServerName.valueOf("example.org,12345,6789"),
           HRegionInfo.FIRST_META_REGIONINFO);
       // Set hbase.rootdir into test dir.
-      FileSystem fs = FileSystem.get(this.c);
+      FileSystem.get(this.c);
       Path rootdir = FSUtils.getRootDir(this.c);
       FSUtils.setRootDir(this.c, rootdir);
-      AdminProtos.AdminService.BlockingInterface hri =
-        Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
+      Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
     }
 
     @Override
@@ -208,6 +206,12 @@ public class TestCatalogJanitor {
     public ChoreService getChoreService() {
       return null;
     }
+
+    @Override
+    public ClusterConnection getClusterConnection() {
+      // TODO Auto-generated method stub
+      return null;
+    }
   }
 
   /**
@@ -402,48 +406,6 @@ 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.
@@ -536,32 +498,35 @@ 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;
+    }
+
+    @Override
+    public ClusterConnection getClusterConnection() {
+      // TODO Auto-generated method stub
+      return null;
+    }
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
index a19d5d8..142437c 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
@@ -100,6 +100,12 @@ public class TestClockSkewDetection {
       public ChoreService getChoreService() {
         return null;
       }
+
+      @Override
+      public ClusterConnection getClusterConnection() {
+        // TODO Auto-generated method stub
+        return null;
+      }
     }, null, false);
 
     LOG.debug("regionServerStartup 1");

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/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 972834a..398a898 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
@@ -32,6 +32,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.CoordinatedStateException;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
@@ -62,8 +63,10 @@ import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestRule;
 import org.mockito.Mockito;
 
 import com.google.protobuf.ServiceException;
@@ -80,6 +83,8 @@ import com.google.protobuf.ServiceException;
 public class TestMasterNoCluster {
   private static final Log LOG = LogFactory.getLog(TestMasterNoCluster.class);
   private static final HBaseTestingUtility TESTUTIL = new 
HBaseTestingUtility();
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().
+      withTimeout(this.getClass()).withLookingForStuckThread(true).build();
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
@@ -122,7 +127,7 @@ public class TestMasterNoCluster {
    * @throws KeeperException
    * @throws InterruptedException
    */
-  @Test (timeout=30000)
+  @Test
   public void testStopDuringStart()
   throws IOException, KeeperException, InterruptedException {
     CoordinatedStateManager cp = 
CoordinatedStateManagerFactory.getCoordinatedStateManager(
@@ -141,7 +146,7 @@ public class TestMasterNoCluster {
    * @throws KeeperException
    * @throws InterruptedException
    */
-  @Test (timeout=30000)
+  @Test
   public void testFailover()
   throws IOException, KeeperException, InterruptedException, ServiceException {
     final long now = System.currentTimeMillis();
@@ -193,6 +198,9 @@ public class TestMasterNoCluster {
       }
 
       @Override
+      void initClusterSchemaService() throws IOException, InterruptedException 
{}
+
+      @Override
       ServerManager createServerManager(Server master, MasterServices services)
       throws IOException {
         ServerManager sm = super.createServerManager(master, services);
@@ -218,10 +226,6 @@ public class TestMasterNoCluster {
           return null;
         }
       }
-
-      @Override
-      void initNamespace() {
-      }
     };
     master.start();
 
@@ -266,6 +270,9 @@ public class TestMasterNoCluster {
       { }
 
       @Override
+      void initClusterSchemaService() throws IOException, InterruptedException 
{}
+
+      @Override
       void initializeZKBasedSystemTrackers() throws IOException,
       InterruptedException, KeeperException, CoordinatedStateException {
         super.initializeZKBasedSystemTrackers();
@@ -294,10 +301,6 @@ public class TestMasterNoCluster {
           return null;
         }
       }
-
-      @Override
-      void initNamespace() {
-      }
     };
     master.start();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
index 65c8649..c7707b7 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java
@@ -168,6 +168,12 @@ public class TestSplitLogManager {
     public ChoreService getChoreService() {
       return null;
     }
+
+    @Override
+    public ClusterConnection getClusterConnection() {
+      // TODO Auto-generated method stub
+      return null;
+    }
   }
 
   static Stoppable stopper = new Stoppable() {

Reply via email to