You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2013/12/05 00:59:07 UTC

[26/50] [abbrv] ACCUMULO-802 Renamed "TableNamespace" to "Namespace"

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
index 6f6304a..4b19edb 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
@@ -25,13 +25,13 @@ import java.util.TreeSet;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.security.SystemPermission;
-import org.apache.accumulo.core.security.TableNamespacePermission;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
@@ -49,7 +49,7 @@ import org.apache.zookeeper.KeeperException.Code;
 public class ZKPermHandler implements PermissionHandler {
   private static final Logger log = Logger.getLogger(ZKAuthorizor.class);
   private static PermissionHandler zkPermHandlerInstance = null;
-  
+
   private String ZKUserPath;
   private String ZKTablePath;
   private String ZKNamespacePath;
@@ -57,24 +57,24 @@ public class ZKPermHandler implements PermissionHandler {
   private final String ZKUserSysPerms = "/System";
   private final String ZKUserTablePerms = "/Tables";
   private final String ZKUserNamespacePerms = "/Namespaces";
-  
+
   public static synchronized PermissionHandler getInstance() {
     if (zkPermHandlerInstance == null)
       zkPermHandlerInstance = new ZKPermHandler();
     return zkPermHandlerInstance;
   }
-  
+
   @Override
   public void initialize(String instanceId, boolean initialize) {
     ZKUserPath = ZKSecurityTool.getInstancePath(instanceId) + "/users";
     ZKTablePath = ZKSecurityTool.getInstancePath(instanceId) + "/tables";
     ZKNamespacePath = ZKSecurityTool.getInstancePath(instanceId) + "/namespaces";
   }
-  
+
   public ZKPermHandler() {
     zooCache = new ZooCache();
   }
-  
+
   @Override
   public boolean hasTablePermission(String user, String table, TablePermission permission) throws TableNotFoundException {
     byte[] serializedPerms;
@@ -113,7 +113,7 @@ public class ZKPermHandler implements PermissionHandler {
     }
     return false;
   }
-  
+
   @Override
   public boolean hasCachedTablePermission(String user, String table, TablePermission permission) throws AccumuloSecurityException, TableNotFoundException {
     byte[] serializedPerms = zooCache.get(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table);
@@ -122,29 +122,29 @@ public class ZKPermHandler implements PermissionHandler {
     }
     return false;
   }
-  
+
   @Override
-  public boolean hasTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws TableNamespaceNotFoundException {
+  public boolean hasNamespacePermission(String user, String namespace, NamespacePermission permission) throws NamespaceNotFoundException {
     byte[] serializedPerms;
     try {
-      String path = ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + tableNamespace;
+      String path = ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace;
       ZooReaderWriter.getRetryingInstance().sync(path);
       serializedPerms = ZooReaderWriter.getRetryingInstance().getData(path, null);
     } catch (KeeperException e) {
       if (e.code() == Code.NONODE) {
-        // maybe the table namespace was just deleted?
+        // maybe the namespace was just deleted?
         try {
           // check for existence:
-          ZooReaderWriter.getRetryingInstance().getData(ZKNamespacePath + "/" + tableNamespace, null);
+          ZooReaderWriter.getRetryingInstance().getData(ZKNamespacePath + "/" + namespace, null);
           // it's there, you don't have permission
           return false;
         } catch (InterruptedException ex) {
-          log.warn("Unhandled InterruptedException, failing closed for table namespace permission check", e);
+          log.warn("Unhandled InterruptedException, failing closed for namespace permission check", e);
           return false;
         } catch (KeeperException ex) {
           // not there, throw an informative exception
           if (e.code() == Code.NONODE) {
-            throw new TableNamespaceNotFoundException(null, tableNamespace, "while checking permissions");
+            throw new NamespaceNotFoundException(null, namespace, "while checking permissions");
           }
           log.warn("Unhandled InterruptedException, failing closed for table permission check", e);
         }
@@ -157,20 +157,21 @@ public class ZKPermHandler implements PermissionHandler {
       return false;
     }
     if (serializedPerms != null) {
-      return ZKSecurityTool.convertTableNamespacePermissions(serializedPerms).contains(permission);
+      return ZKSecurityTool.convertNamespacePermissions(serializedPerms).contains(permission);
     }
     return false;
   }
-  
+
   @Override
-  public boolean hasCachedTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws AccumuloSecurityException, TableNamespaceNotFoundException {
-    byte[] serializedPerms = zooCache.get(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + tableNamespace);
+  public boolean hasCachedNamespacePermission(String user, String namespace, NamespacePermission permission) throws AccumuloSecurityException,
+      NamespaceNotFoundException {
+    byte[] serializedPerms = zooCache.get(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace);
     if (serializedPerms != null) {
-      return ZKSecurityTool.convertTableNamespacePermissions(serializedPerms).contains(permission);
+      return ZKSecurityTool.convertNamespacePermissions(serializedPerms).contains(permission);
     }
     return false;
   }
-  
+
   @Override
   public void grantSystemPermission(String user, SystemPermission permission) throws AccumuloSecurityException {
     try {
@@ -181,7 +182,7 @@ public class ZKPermHandler implements PermissionHandler {
       } else {
         perms = ZKSecurityTool.convertSystemPermissions(permBytes);
       }
-      
+
       if (perms.add(permission)) {
         synchronized (zooCache) {
           zooCache.clear();
@@ -197,7 +198,7 @@ public class ZKPermHandler implements PermissionHandler {
       throw new RuntimeException(e);
     }
   }
-  
+
   @Override
   public void grantTablePermission(String user, String table, TablePermission permission) throws AccumuloSecurityException {
     Set<TablePermission> tablePerms;
@@ -206,7 +207,7 @@ public class ZKPermHandler implements PermissionHandler {
       tablePerms = ZKSecurityTool.convertTablePermissions(serializedPerms);
     else
       tablePerms = new TreeSet<TablePermission>();
-    
+
     try {
       if (tablePerms.add(permission)) {
         synchronized (zooCache) {
@@ -224,22 +225,22 @@ public class ZKPermHandler implements PermissionHandler {
       throw new RuntimeException(e);
     }
   }
-  
+
   @Override
-  public void grantTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws AccumuloSecurityException {
-    Set<TableNamespacePermission> tableNamespacePerms;
-    byte[] serializedPerms = zooCache.get(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + tableNamespace);
+  public void grantNamespacePermission(String user, String namespace, NamespacePermission permission) throws AccumuloSecurityException {
+    Set<NamespacePermission> namespacePerms;
+    byte[] serializedPerms = zooCache.get(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace);
     if (serializedPerms != null)
-      tableNamespacePerms = ZKSecurityTool.convertTableNamespacePermissions(serializedPerms);
+      namespacePerms = ZKSecurityTool.convertNamespacePermissions(serializedPerms);
     else
-      tableNamespacePerms = new TreeSet<TableNamespacePermission>();
-    
+      namespacePerms = new TreeSet<NamespacePermission>();
+
     try {
-      if (tableNamespacePerms.add(permission)) {
+      if (namespacePerms.add(permission)) {
         synchronized (zooCache) {
-          zooCache.clear(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + tableNamespace);
+          zooCache.clear(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace);
           IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
-          zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + tableNamespace, ZKSecurityTool.convertTableNamespacePermissions(tableNamespacePerms),
+          zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace, ZKSecurityTool.convertNamespacePermissions(namespacePerms),
               NodeExistsPolicy.OVERWRITE);
         }
       }
@@ -251,17 +252,17 @@ public class ZKPermHandler implements PermissionHandler {
       throw new RuntimeException(e);
     }
   }
-  
+
   @Override
   public void revokeSystemPermission(String user, SystemPermission permission) throws AccumuloSecurityException {
     byte[] sysPermBytes = zooCache.get(ZKUserPath + "/" + user + ZKUserSysPerms);
-    
+
     // User had no system permission, nothing to revoke.
     if (sysPermBytes == null)
       return;
-    
+
     Set<SystemPermission> sysPerms = ZKSecurityTool.convertSystemPermissions(sysPermBytes);
-    
+
     try {
       if (sysPerms.remove(permission)) {
         synchronized (zooCache) {
@@ -278,15 +279,15 @@ public class ZKPermHandler implements PermissionHandler {
       throw new RuntimeException(e);
     }
   }
-  
+
   @Override
   public void revokeTablePermission(String user, String table, TablePermission permission) throws AccumuloSecurityException {
     byte[] serializedPerms = zooCache.get(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table);
-    
+
     // User had no table permission, nothing to revoke.
     if (serializedPerms == null)
       return;
-    
+
     Set<TablePermission> tablePerms = ZKSecurityTool.convertTablePermissions(serializedPerms);
     try {
       if (tablePerms.remove(permission)) {
@@ -306,24 +307,24 @@ public class ZKPermHandler implements PermissionHandler {
       throw new RuntimeException(e);
     }
   }
-  
+
   @Override
-  public void revokeTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws AccumuloSecurityException {
-    byte[] serializedPerms = zooCache.get(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + tableNamespace);
-    
-    // User had no table namespace permission, nothing to revoke.
+  public void revokeNamespacePermission(String user, String namespace, NamespacePermission permission) throws AccumuloSecurityException {
+    byte[] serializedPerms = zooCache.get(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace);
+
+    // User had no namespace permission, nothing to revoke.
     if (serializedPerms == null)
       return;
-    
-    Set<TableNamespacePermission> tableNamespacePerms = ZKSecurityTool.convertTableNamespacePermissions(serializedPerms);
+
+    Set<NamespacePermission> namespacePerms = ZKSecurityTool.convertNamespacePermissions(serializedPerms);
     try {
-      if (tableNamespacePerms.remove(permission)) {
+      if (namespacePerms.remove(permission)) {
         zooCache.clear();
         IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
-        if (tableNamespacePerms.size() == 0)
-          zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + tableNamespace, NodeMissingPolicy.SKIP);
+        if (namespacePerms.size() == 0)
+          zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace, NodeMissingPolicy.SKIP);
         else
-          zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + tableNamespace, ZKSecurityTool.convertTableNamespacePermissions(tableNamespacePerms),
+          zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace, ZKSecurityTool.convertNamespacePermissions(namespacePerms),
               NodeExistsPolicy.OVERWRITE);
       }
     } catch (KeeperException e) {
@@ -334,7 +335,7 @@ public class ZKPermHandler implements PermissionHandler {
       throw new RuntimeException(e);
     }
   }
-  
+
   @Override
   public void cleanTablePermissions(String table) throws AccumuloSecurityException {
     try {
@@ -352,15 +353,15 @@ public class ZKPermHandler implements PermissionHandler {
       throw new RuntimeException(e);
     }
   }
-  
+
   @Override
-  public void cleanTableNamespacePermissions(String tableNamespace) throws AccumuloSecurityException {
+  public void cleanNamespacePermissions(String namespace) throws AccumuloSecurityException {
     try {
       synchronized (zooCache) {
         zooCache.clear();
         IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
         for (String user : zooCache.getChildren(ZKUserPath))
-          zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + tableNamespace, NodeMissingPolicy.SKIP);
+          zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace, NodeMissingPolicy.SKIP);
       }
     } catch (KeeperException e) {
       log.error(e, e);
@@ -370,11 +371,11 @@ public class ZKPermHandler implements PermissionHandler {
       throw new RuntimeException(e);
     }
   }
-  
+
   @Override
   public void initializeSecurity(TCredentials itw, String rootuser) throws AccumuloSecurityException {
     IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
-    
+
     // create the root user with all system privileges, no table privileges, and no record-level authorizations
     Set<SystemPermission> rootPerms = new TreeSet<SystemPermission>();
     for (SystemPermission p : SystemPermission.values())
@@ -384,21 +385,21 @@ public class ZKPermHandler implements PermissionHandler {
     tablePerms.put(RootTable.ID, Collections.singleton(TablePermission.ALTER_TABLE));
     tablePerms.put(MetadataTable.ID, Collections.singleton(TablePermission.ALTER_TABLE));
     // essentially the same but on the system namespace, the ALTER_TABLE permission is now redundant
-    Map<String,Set<TableNamespacePermission>> tableNamespacePerms = new HashMap<String,Set<TableNamespacePermission>>();
-    tableNamespacePerms.put(Constants.SYSTEM_TABLE_NAMESPACE_ID, Collections.singleton(TableNamespacePermission.ALTER_NAMESPACE));
-    tableNamespacePerms.put(Constants.SYSTEM_TABLE_NAMESPACE_ID, Collections.singleton(TableNamespacePermission.ALTER_TABLE));
-    
+    Map<String,Set<NamespacePermission>> namespacePerms = new HashMap<String,Set<NamespacePermission>>();
+    namespacePerms.put(Constants.SYSTEM_NAMESPACE_ID, Collections.singleton(NamespacePermission.ALTER_NAMESPACE));
+    namespacePerms.put(Constants.SYSTEM_NAMESPACE_ID, Collections.singleton(NamespacePermission.ALTER_TABLE));
+
     try {
       // prep parent node of users with root username
       if (!zoo.exists(ZKUserPath))
         zoo.putPersistentData(ZKUserPath, rootuser.getBytes(), NodeExistsPolicy.FAIL);
-      
+
       initUser(rootuser);
       zoo.putPersistentData(ZKUserPath + "/" + rootuser + ZKUserSysPerms, ZKSecurityTool.convertSystemPermissions(rootPerms), NodeExistsPolicy.FAIL);
       for (Entry<String,Set<TablePermission>> entry : tablePerms.entrySet())
         createTablePerm(rootuser, entry.getKey(), entry.getValue());
-      for (Entry<String,Set<TableNamespacePermission>> entry : tableNamespacePerms.entrySet())
-        createTableNamespacePerm(rootuser, entry.getKey(), entry.getValue());
+      for (Entry<String,Set<NamespacePermission>> entry : namespacePerms.entrySet())
+        createNamespacePerm(rootuser, entry.getKey(), entry.getValue());
     } catch (KeeperException e) {
       log.error(e, e);
       throw new RuntimeException(e);
@@ -407,7 +408,7 @@ public class ZKPermHandler implements PermissionHandler {
       throw new RuntimeException(e);
     }
   }
-  
+
   /**
    * @param user
    * @throws AccumuloSecurityException
@@ -427,7 +428,7 @@ public class ZKPermHandler implements PermissionHandler {
       throw new RuntimeException(e);
     }
   }
-  
+
   /**
    * Sets up a new table configuration for the provided user/table. No checking for existence is done here, it should be done before calling.
    */
@@ -438,18 +439,18 @@ public class ZKPermHandler implements PermissionHandler {
           ZKSecurityTool.convertTablePermissions(perms), NodeExistsPolicy.FAIL);
     }
   }
-  
+
   /**
-   * Sets up a new table namespace configuration for the provided user/table. No checking for existence is done here, it should be done before calling.
+   * Sets up a new namespace configuration for the provided user/table. No checking for existence is done here, it should be done before calling.
    */
-  private void createTableNamespacePerm(String user, String namespace, Set<TableNamespacePermission> perms) throws KeeperException, InterruptedException {
+  private void createNamespacePerm(String user, String namespace, Set<NamespacePermission> perms) throws KeeperException, InterruptedException {
     synchronized (zooCache) {
       zooCache.clear();
       ZooReaderWriter.getRetryingInstance().putPersistentData(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace,
-          ZKSecurityTool.convertTableNamespacePermissions(perms), NodeExistsPolicy.FAIL);
+          ZKSecurityTool.convertNamespacePermissions(perms), NodeExistsPolicy.FAIL);
     }
   }
-  
+
   @Override
   public void cleanUser(String user) throws AccumuloSecurityException {
     try {
@@ -468,10 +469,10 @@ public class ZKPermHandler implements PermissionHandler {
       if (e.code().equals(KeeperException.Code.NONODE))
         throw new AccumuloSecurityException(user, SecurityErrorCode.USER_DOESNT_EXIST, e);
       throw new AccumuloSecurityException(user, SecurityErrorCode.CONNECTION_ERROR, e);
-      
+
     }
   }
-  
+
   @Override
   public boolean hasSystemPermission(String user, SystemPermission permission) throws AccumuloSecurityException {
     byte[] perms;
@@ -489,12 +490,12 @@ public class ZKPermHandler implements PermissionHandler {
       log.warn("Unhandled InterruptedException, failing closed for table permission check", e);
       return false;
     }
-    
+
     if (perms == null)
       return false;
     return ZKSecurityTool.convertSystemPermissions(perms).contains(permission);
   }
-  
+
   @Override
   public boolean hasCachedSystemPermission(String user, SystemPermission permission) throws AccumuloSecurityException {
     byte[] perms = zooCache.get(ZKUserPath + "/" + user + ZKUserSysPerms);
@@ -502,12 +503,12 @@ public class ZKPermHandler implements PermissionHandler {
       return false;
     return ZKSecurityTool.convertSystemPermissions(perms).contains(permission);
   }
-  
+
   @Override
   public boolean validSecurityHandlers(Authenticator authent, Authorizor author) {
     return true;
   }
-  
+
   @Override
   public void initTable(String table) throws AccumuloSecurityException {
     // All proper housekeeping is done on delete and permission granting, no work needs to be done here

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java
index ce62b7e..2ed430c 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java
@@ -31,7 +31,7 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.SystemPermission;
-import org.apache.accumulo.core.security.TableNamespacePermission;
+import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.log4j.Logger;
 
@@ -150,11 +150,11 @@ class ZKSecurityTool {
     return toReturn;
   }
   
-  public static byte[] convertTableNamespacePermissions(Set<TableNamespacePermission> namespacepermissions) {
+  public static byte[] convertNamespacePermissions(Set<NamespacePermission> namespacepermissions) {
     ByteArrayOutputStream bytes = new ByteArrayOutputStream(namespacepermissions.size());
     DataOutputStream out = new DataOutputStream(bytes);
     try {
-      for (TableNamespacePermission tnp : namespacepermissions)
+      for (NamespacePermission tnp : namespacepermissions)
         out.writeByte(tnp.getId());
     } catch (IOException e) {
       log.error(e, e);
@@ -163,10 +163,10 @@ class ZKSecurityTool {
     return bytes.toByteArray();
   }
   
-  public static Set<TableNamespacePermission> convertTableNamespacePermissions(byte[] namespacepermissions) {
-    Set<TableNamespacePermission> toReturn = new HashSet<TableNamespacePermission>();
+  public static Set<NamespacePermission> convertNamespacePermissions(byte[] namespacepermissions) {
+    Set<NamespacePermission> toReturn = new HashSet<NamespacePermission>();
     for (byte b : namespacepermissions)
-      toReturn.add(TableNamespacePermission.getPermissionById(b));
+      toReturn.add(NamespacePermission.getPermissionById(b));
     return toReturn;
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java b/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java
index 6d1cf08..bd3b6a9 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java
@@ -69,7 +69,7 @@ public class TableManager {
     zoo.putPersistentData(zTablePath + Constants.ZTABLE_FLUSH_ID, "0".getBytes(Constants.UTF8), existsPolicy);
     zoo.putPersistentData(zTablePath + Constants.ZTABLE_COMPACT_ID, "0".getBytes(Constants.UTF8), existsPolicy);
     zoo.putPersistentData(zTablePath + Constants.ZTABLE_COMPACT_CANCEL_ID, "0".getBytes(Constants.UTF8), existsPolicy);
-    zoo.putPersistentData(zTablePath + Constants.ZTABLE_NAMESPACE, Constants.DEFAULT_TABLE_NAMESPACE_ID.getBytes(Constants.UTF8), existsPolicy);
+    zoo.putPersistentData(zTablePath + Constants.ZTABLE_NAMESPACE, Constants.DEFAULT_NAMESPACE_ID.getBytes(Constants.UTF8), existsPolicy);
   }
   
   public synchronized static TableManager getInstance() {
@@ -317,7 +317,7 @@ public class TableManager {
   public void removeNamespaceFromTable(String tableId, String namespaceId) throws KeeperException, InterruptedException {
     // actually, revert it to the default namespace.
     String zPath = Constants.ZROOT + "/" + instance.getInstanceID() + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_NAMESPACE;
-    ZooReaderWriter.getRetryingInstance().putPersistentData(zPath, Constants.DEFAULT_TABLE_NAMESPACE_ID.getBytes(Constants.UTF8), NodeExistsPolicy.OVERWRITE);
+    ZooReaderWriter.getRetryingInstance().putPersistentData(zPath, Constants.DEFAULT_NAMESPACE_ID.getBytes(Constants.UTF8), NodeExistsPolicy.OVERWRITE);
   }
   
   public void cloneNamespace(String srcId, String newId, String namespaceName, Map<String,String> propertiesToSet, Set<String> propertiesToExclude,

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/master/src/main/java/org/apache/accumulo/master/Master.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/Master.java b/server/master/src/main/java/org/apache/accumulo/master/Master.java
index 64c6340..fcfdc8c 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/Master.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/Master.java
@@ -40,13 +40,13 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.RowIterator;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.TableOperationsImpl;
 import org.apache.accumulo.core.client.admin.TimeType;
-import org.apache.accumulo.core.client.impl.TableNamespaces;
+import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.ThriftTransportPool;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
@@ -78,8 +78,8 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.Credentials;
+import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.security.SecurityUtil;
-import org.apache.accumulo.core.security.TableNamespacePermission;
 import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.Daemon;
@@ -100,14 +100,14 @@ import org.apache.accumulo.master.tableOps.CancelCompactions;
 import org.apache.accumulo.master.tableOps.ChangeTableState;
 import org.apache.accumulo.master.tableOps.CloneTable;
 import org.apache.accumulo.master.tableOps.CompactRange;
+import org.apache.accumulo.master.tableOps.CreateNamespace;
 import org.apache.accumulo.master.tableOps.CreateTable;
-import org.apache.accumulo.master.tableOps.CreateTableNamespace;
+import org.apache.accumulo.master.tableOps.DeleteNamespace;
 import org.apache.accumulo.master.tableOps.DeleteTable;
-import org.apache.accumulo.master.tableOps.DeleteTableNamespace;
 import org.apache.accumulo.master.tableOps.ExportTable;
 import org.apache.accumulo.master.tableOps.ImportTable;
+import org.apache.accumulo.master.tableOps.RenameNamespace;
 import org.apache.accumulo.master.tableOps.RenameTable;
-import org.apache.accumulo.master.tableOps.RenameTableNamespace;
 import org.apache.accumulo.master.tableOps.TableRangeOp;
 import org.apache.accumulo.master.tableOps.TraceRepo;
 import org.apache.accumulo.master.tserverOps.ShutdownTServer;
@@ -331,35 +331,32 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
               NodeExistsPolicy.SKIP);
         }
 
-        // setup default and system table namespaces if not already there
-        String tableNamespaces = ZooUtil.getRoot(instance) + Constants.ZNAMESPACES;
-        String defaultTableNamespace = ZooUtil.getRoot(instance) + Constants.ZNAMESPACES + "/" + Constants.DEFAULT_TABLE_NAMESPACE_ID;
-        String systemTableNamespace = ZooUtil.getRoot(instance) + Constants.ZNAMESPACES + "/" + Constants.SYSTEM_TABLE_NAMESPACE_ID;
+        // setup default and system namespaces if not already there
+        String namespaces = ZooUtil.getRoot(instance) + Constants.ZNAMESPACES;
+        String defaultNamespace = ZooUtil.getRoot(instance) + Constants.ZNAMESPACES + "/" + Constants.DEFAULT_NAMESPACE_ID;
+        String systemNamespace = ZooUtil.getRoot(instance) + Constants.ZNAMESPACES + "/" + Constants.SYSTEM_NAMESPACE_ID;
         String tables = ZooUtil.getRoot(instance) + Constants.ZTABLES;
-        zoo.putPersistentData(tableNamespaces, new byte[0], NodeExistsPolicy.SKIP);
+        zoo.putPersistentData(namespaces, new byte[0], NodeExistsPolicy.SKIP);
 
-        zoo.putPersistentData(defaultTableNamespace, new byte[0], NodeExistsPolicy.SKIP);
-        zoo.putPersistentData(defaultTableNamespace + Constants.ZNAMESPACE_CONF, new byte[0], NodeExistsPolicy.SKIP);
-        zoo.putPersistentData(defaultTableNamespace + Constants.ZNAMESPACE_NAME, Constants.DEFAULT_TABLE_NAMESPACE.getBytes(Constants.UTF8),
-            NodeExistsPolicy.SKIP);
+        zoo.putPersistentData(defaultNamespace, new byte[0], NodeExistsPolicy.SKIP);
+        zoo.putPersistentData(defaultNamespace + Constants.ZNAMESPACE_CONF, new byte[0], NodeExistsPolicy.SKIP);
+        zoo.putPersistentData(defaultNamespace + Constants.ZNAMESPACE_NAME, Constants.DEFAULT_NAMESPACE.getBytes(Constants.UTF8), NodeExistsPolicy.SKIP);
 
-        zoo.putPersistentData(systemTableNamespace, new byte[0], NodeExistsPolicy.SKIP);
-        zoo.putPersistentData(systemTableNamespace + Constants.ZNAMESPACE_CONF, new byte[0], NodeExistsPolicy.SKIP);
-        zoo.putPersistentData(systemTableNamespace + Constants.ZNAMESPACE_NAME, Constants.SYSTEM_TABLE_NAMESPACE.getBytes(Constants.UTF8),
-            NodeExistsPolicy.SKIP);
+        zoo.putPersistentData(systemNamespace, new byte[0], NodeExistsPolicy.SKIP);
+        zoo.putPersistentData(systemNamespace + Constants.ZNAMESPACE_CONF, new byte[0], NodeExistsPolicy.SKIP);
+        zoo.putPersistentData(systemNamespace + Constants.ZNAMESPACE_NAME, Constants.SYSTEM_NAMESPACE.getBytes(Constants.UTF8), NodeExistsPolicy.SKIP);
 
         Map<String,String> opts = IteratorUtil.generateInitialTableProperties(true);
         for (Entry<String,String> e : opts.entrySet()) {
-          zoo.putPersistentData(defaultTableNamespace + Constants.ZNAMESPACE_CONF + "/" + e.getKey(), e.getValue().getBytes(Constants.UTF8),
-              NodeExistsPolicy.SKIP);
+          zoo.putPersistentData(defaultNamespace + Constants.ZNAMESPACE_CONF + "/" + e.getKey(), e.getValue().getBytes(Constants.UTF8), NodeExistsPolicy.SKIP);
         }
 
         for (Entry<String,String> table : Tables.getIdToNameMap(instance).entrySet()) {
           if (table.getValue().equals(MetadataTable.NAME) || table.getValue().equals(RootTable.NAME)) {
-            zoo.putPersistentData(tables + "/" + table.getKey() + Constants.ZTABLE_NAMESPACE, Constants.SYSTEM_TABLE_NAMESPACE_ID.getBytes(Constants.UTF8),
+            zoo.putPersistentData(tables + "/" + table.getKey() + Constants.ZTABLE_NAMESPACE, Constants.SYSTEM_NAMESPACE_ID.getBytes(Constants.UTF8),
                 NodeExistsPolicy.SKIP);
           } else {
-            zoo.putPersistentData(tables + "/" + table.getKey() + Constants.ZTABLE_NAMESPACE, Constants.DEFAULT_TABLE_NAMESPACE_ID.getBytes(Constants.UTF8),
+            zoo.putPersistentData(tables + "/" + table.getKey() + Constants.ZTABLE_NAMESPACE, Constants.DEFAULT_NAMESPACE_ID.getBytes(Constants.UTF8),
                 NodeExistsPolicy.SKIP);
           }
         }
@@ -370,9 +367,9 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
         String users = ZooUtil.getRoot(instance) + "/users";
         for (String user : zoo.getChildren(users)) {
           zoo.putPersistentData(users + "/" + user + "/Namespaces", new byte[0], NodeExistsPolicy.SKIP);
-          perm.grantTableNamespacePermission(user, Constants.SYSTEM_TABLE_NAMESPACE_ID, TableNamespacePermission.READ);
+          perm.grantNamespacePermission(user, Constants.SYSTEM_NAMESPACE_ID, NamespacePermission.READ);
         }
-        perm.grantTableNamespacePermission("root", Constants.SYSTEM_TABLE_NAMESPACE_ID, TableNamespacePermission.ALTER_TABLE);
+        perm.grantNamespacePermission("root", Constants.SYSTEM_NAMESPACE_ID, NamespacePermission.ALTER_TABLE);
 
       } catch (Exception ex) {
         log.fatal("Error performing upgrade", ex);
@@ -750,8 +747,8 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       }
     }
 
-    private void alterTableNamespaceProperty(TCredentials c, String namespace, String property, String value, TableOperation op)
-        throws ThriftSecurityException, ThriftTableOperationException {
+    private void alterNamespaceProperty(TCredentials c, String namespace, String property, String value, TableOperation op) throws ThriftSecurityException,
+        ThriftTableOperationException {
 
       String namespaceId = null;
       namespaceId = checkNamespaceId(namespace, op);
@@ -766,13 +763,13 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
           NamespacePropUtil.setNamespaceProperty(namespaceId, property, value);
         }
       } catch (KeeperException.NoNodeException e) {
-        // race condition... table namespace no longer exists? This call will throw an exception if the table namespace was deleted:
+        // race condition... namespace no longer exists? This call will throw an exception if the namespace was deleted:
         checkNamespaceId(namespaceId, op);
-        log.info("Error altering table namespace property", e);
-        throw new ThriftTableOperationException(namespaceId, namespace, op, TableOperationExceptionType.OTHER, "Problem altering table namespaceproperty");
+        log.info("Error altering namespace property", e);
+        throw new ThriftTableOperationException(namespaceId, namespace, op, TableOperationExceptionType.OTHER, "Problem altering namespaceproperty");
       } catch (Exception e) {
-        log.error("Problem altering table namespace property", e);
-        throw new ThriftTableOperationException(namespaceId, namespace, op, TableOperationExceptionType.OTHER, "Problem altering table namespace property");
+        log.error("Problem altering namespace property", e);
+        throw new ThriftTableOperationException(namespaceId, namespace, op, TableOperationExceptionType.OTHER, "Problem altering namespace property");
       }
     }
 
@@ -929,7 +926,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
 
           try {
             fate.seedTransaction(opid, new TraceRepo<Master>(new CreateTable(c.getPrincipal(), tableName, timeType, options)), autoCleanup);
-          } catch (TableNamespaceNotFoundException e) {
+          } catch (NamespaceNotFoundException e) {
             throw new TException(e.getMessage(), e);
           }
           break;
@@ -947,7 +944,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
 
           try {
             fate.seedTransaction(opid, new TraceRepo<Master>(new RenameTable(tableId, oldTableName, newTableName)), autoCleanup);
-          } catch (TableNamespaceNotFoundException e) {
+          } catch (NamespaceNotFoundException e) {
             throw new TException(e.getMessage(), e);
           }
 
@@ -1087,7 +1084,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
 
           try {
             fate.seedTransaction(opid, new TraceRepo<Master>(new ImportTable(c.getPrincipal(), tableName, exportDir)), autoCleanup);
-          } catch (TableNamespaceNotFoundException e) {
+          } catch (NamespaceNotFoundException e) {
             throw new TException(e.getMessage(), e);
           }
           break;
@@ -1144,12 +1141,12 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     }
 
     @Override
-    public long beginTableNamespaceOperation(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException, TException {
+    public long beginNamespaceOperation(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException, TException {
       return beginTableOperation(tinfo, credentials);
     }
 
     @Override
-    public void executeTableNamespaceOperation(TInfo tinfo, TCredentials c, long opid, org.apache.accumulo.core.master.thrift.TableOperation op,
+    public void executeNamespaceOperation(TInfo tinfo, TCredentials c, long opid, org.apache.accumulo.core.master.thrift.TableOperation op,
         List<ByteBuffer> arguments, Map<String,String> options, boolean autoCleanup) throws ThriftSecurityException, ThriftTableOperationException, TException {
       authenticate(c);
 
@@ -1160,8 +1157,8 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
             throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
           checkNotSystemNamespace(namespace, TableOperation.CREATE);
-          checkTableNamespaceName(namespace, TableOperation.CREATE);
-          fate.seedTransaction(opid, new TraceRepo<Master>(new CreateTableNamespace(c.getPrincipal(), namespace, options)), autoCleanup);
+          checkNamespaceName(namespace, TableOperation.CREATE);
+          fate.seedTransaction(opid, new TraceRepo<Master>(new CreateNamespace(c.getPrincipal(), namespace, options)), autoCleanup);
           break;
         }
         case RENAME: {
@@ -1172,11 +1169,11 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
 
           checkNotSystemNamespace(oldName, TableOperation.RENAME);
           checkNotSystemNamespace(newName, TableOperation.RENAME);
-          checkTableNamespaceName(newName, TableOperation.RENAME);
+          checkNamespaceName(newName, TableOperation.RENAME);
           if (!security.canRenameNamespace(c, namespaceId, oldName, newName))
             throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-          fate.seedTransaction(opid, new TraceRepo<Master>(new RenameTableNamespace(namespaceId, oldName, newName)), autoCleanup);
+          fate.seedTransaction(opid, new TraceRepo<Master>(new RenameNamespace(namespaceId, oldName, newName)), autoCleanup);
           break;
         }
         case DELETE: {
@@ -1186,7 +1183,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
           if (!security.canDeleteNamespace(c, namespaceId))
             throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-          fate.seedTransaction(opid, new TraceRepo<Master>(new DeleteTableNamespace(namespaceId)), autoCleanup);
+          fate.seedTransaction(opid, new TraceRepo<Master>(new DeleteNamespace(namespaceId)), autoCleanup);
           break;
         }
         default:
@@ -1196,53 +1193,53 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     }
 
     private void checkNotSystemNamespace(String namespace, TableOperation operation) throws ThriftTableOperationException {
-      if (Constants.SYSTEM_TABLE_NAMESPACE.equals(namespace)) {
-        String why = "Table namespaces cannot be == " + Constants.SYSTEM_TABLE_NAMESPACE;
+      if (Constants.SYSTEM_NAMESPACE.equals(namespace)) {
+        String why = "Namespaces cannot be == " + Constants.SYSTEM_NAMESPACE;
         log.warn(why);
         throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.OTHER, why);
       }
     }
 
-    private void checkTableNamespaceName(String namespace, TableOperation operation) throws ThriftTableOperationException {
-      if (!namespace.matches(Constants.VALID_TABLE_NAMESPACE_REGEX)) {
-        String why = "Table namespaces must only contain word characters (letters, digits, and underscores): " + namespace;
+    private void checkNamespaceName(String namespace, TableOperation operation) throws ThriftTableOperationException {
+      if (!namespace.matches(Constants.VALID_NAMESPACE_REGEX)) {
+        String why = "Namespaces must only contain word characters (letters, digits, and underscores): " + namespace;
         log.warn(why);
         throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.OTHER, why);
       }
-      if (TableNamespaces.getNameToIdMap(instance).containsKey(namespace)) {
-        String why = "Table namespace already exists: " + namespace;
+      if (Namespaces.getNameToIdMap(instance).containsKey(namespace)) {
+        String why = "Namespace already exists: " + namespace;
         throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.EXISTS, why);
       }
     }
 
     private String checkNamespaceId(String namespace, TableOperation operation) throws ThriftTableOperationException {
-      final String namespaceId = TableNamespaces.getNameToIdMap(getConfiguration().getInstance()).get(namespace);
+      final String namespaceId = Namespaces.getNameToIdMap(getConfiguration().getInstance()).get(namespace);
       if (namespaceId == null)
         throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.NOTFOUND, null);
       return namespaceId;
     }
 
     @Override
-    public String waitForTableNamespaceOperation(TInfo tinfo, TCredentials credentials, long opid) throws ThriftSecurityException,
-        ThriftTableOperationException, TException {
+    public String waitForNamespaceOperation(TInfo tinfo, TCredentials credentials, long opid) throws ThriftSecurityException, ThriftTableOperationException,
+        TException {
       return waitForTableOperation(tinfo, credentials, opid);
     }
 
     @Override
-    public void finishTableNamespaceOperation(TInfo tinfo, TCredentials credentials, long opid) throws ThriftSecurityException, TException {
+    public void finishNamespaceOperation(TInfo tinfo, TCredentials credentials, long opid) throws ThriftSecurityException, TException {
       finishTableOperation(tinfo, credentials, opid);
     }
 
     @Override
-    public void setTableNamespaceProperty(TInfo tinfo, TCredentials credentials, String ns, String property, String value) throws ThriftSecurityException,
+    public void setNamespaceProperty(TInfo tinfo, TCredentials credentials, String ns, String property, String value) throws ThriftSecurityException,
         ThriftTableOperationException, TException {
-      alterTableNamespaceProperty(credentials, ns, property, value, TableOperation.SET_PROPERTY);
+      alterNamespaceProperty(credentials, ns, property, value, TableOperation.SET_PROPERTY);
     }
 
     @Override
-    public void removeTableNamespaceProperty(TInfo tinfo, TCredentials credentials, String ns, String property) throws ThriftSecurityException,
+    public void removeNamespaceProperty(TInfo tinfo, TCredentials credentials, String ns, String property) throws ThriftSecurityException,
         ThriftTableOperationException, TException {
-      alterTableNamespaceProperty(credentials, ns, property, null, TableOperation.REMOVE_PROPERTY);
+      alterNamespaceProperty(credentials, ns, property, null, TableOperation.REMOVE_PROPERTY);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/master/src/main/java/org/apache/accumulo/master/tableOps/CancelCompactions.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CancelCompactions.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CancelCompactions.java
index 9698344..0e3af9d 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CancelCompactions.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CancelCompactions.java
@@ -64,7 +64,7 @@ public class CancelCompactions extends MasterRepo {
   
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    return Utils.reserveTableNamespace(namespaceId, tid, false, true, TableOperation.COMPACT_CANCEL)
+    return Utils.reserveNamespace(namespaceId, tid, false, true, TableOperation.COMPACT_CANCEL)
         + Utils.reserveTable(tableId, tid, false, true, TableOperation.COMPACT_CANCEL);
   }
   
@@ -100,7 +100,7 @@ public class CancelCompactions extends MasterRepo {
   
   @Override
   public void undo(long tid, Master environment) throws Exception {
-    Utils.unreserveTableNamespace(namespaceId, tid, false);
+    Utils.unreserveNamespace(namespaceId, tid, false);
     Utils.unreserveTable(tableId, tid, false);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChangeTableState.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChangeTableState.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChangeTableState.java
index ffd61e4..7859c0e 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChangeTableState.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ChangeTableState.java
@@ -46,7 +46,7 @@ public class ChangeTableState extends MasterRepo {
   @Override
   public long isReady(long tid, Master environment) throws Exception {
     // reserve the table so that this op does not run concurrently with create, clone, or delete table
-    return Utils.reserveTableNamespace(namespaceId, tid, false, true, top) + Utils.reserveTable(tableId, tid, true, true, top);
+    return Utils.reserveNamespace(namespaceId, tid, false, true, top) + Utils.reserveTable(tableId, tid, true, true, top);
   }
 
   @Override
@@ -57,7 +57,7 @@ public class ChangeTableState extends MasterRepo {
       ts = TableState.OFFLINE;
 
     TableManager.getInstance().transitionTableState(tableId, ts);
-    Utils.unreserveTableNamespace(namespaceId, tid, false);
+    Utils.unreserveNamespace(namespaceId, tid, false);
     Utils.unreserveTable(tableId, tid, true);
     Logger.getLogger(ChangeTableState.class).debug("Changed table state " + tableId + " " + ts);
     env.getEventCoordinator().event("Set table state of %s to %s", tableId, ts);
@@ -66,7 +66,7 @@ public class ChangeTableState extends MasterRepo {
 
   @Override
   public void undo(long tid, Master env) throws Exception {
-    Utils.unreserveTableNamespace(namespaceId, tid, false);
+    Utils.unreserveNamespace(namespaceId, tid, false);
     Utils.unreserveTable(tableId, tid, true);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTable.java
index 97d6234..59da70f 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTable.java
@@ -21,8 +21,8 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
-import org.apache.accumulo.core.client.impl.TableNamespaces;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.TableOperation;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
@@ -76,9 +76,9 @@ class FinishCloneTable extends MasterRepo {
     
     TableManager.getInstance().transitionTableState(cloneInfo.tableId, TableState.ONLINE);
     
-    Utils.unreserveTableNamespace(cloneInfo.srcNamespaceId, tid, false);
+    Utils.unreserveNamespace(cloneInfo.srcNamespaceId, tid, false);
     if (!cloneInfo.srcNamespaceId.equals(cloneInfo.namespaceId))
-      Utils.unreserveTableNamespace(cloneInfo.namespaceId, tid, false);
+      Utils.unreserveNamespace(cloneInfo.namespaceId, tid, false);
     Utils.unreserveTable(cloneInfo.srcTableId, tid, false);
     Utils.unreserveTable(cloneInfo.tableId, tid, true);
     
@@ -133,17 +133,17 @@ class CloneZookeeper extends MasterRepo {
   
   private CloneInfo cloneInfo;
   
-  public CloneZookeeper(CloneInfo cloneInfo) throws TableNamespaceNotFoundException {
+  public CloneZookeeper(CloneInfo cloneInfo) throws NamespaceNotFoundException {
     this.cloneInfo = cloneInfo;
     Instance inst = HdfsZooInstance.getInstance();
-    this.cloneInfo.namespaceId = TableNamespaces.getNamespaceId(inst, Tables.extractNamespace(this.cloneInfo.tableName));
+    this.cloneInfo.namespaceId = Namespaces.getNamespaceId(inst, Tables.extractNamespace(this.cloneInfo.tableName));
   }
   
   @Override
   public long isReady(long tid, Master environment) throws Exception {
     long val = 0;
     if (!cloneInfo.srcNamespaceId.equals(cloneInfo.namespaceId))
-      val += Utils.reserveTableNamespace(cloneInfo.namespaceId, tid, false, true, TableOperation.CLONE);
+      val += Utils.reserveNamespace(cloneInfo.namespaceId, tid, false, true, TableOperation.CLONE);
     val += Utils.reserveTable(cloneInfo.tableId, tid, true, false, TableOperation.CLONE);
     return val;
   }
@@ -174,7 +174,7 @@ class CloneZookeeper extends MasterRepo {
     Instance instance = HdfsZooInstance.getInstance();
     TableManager.getInstance().removeTable(cloneInfo.tableId);
     if (!cloneInfo.srcNamespaceId.equals(cloneInfo.namespaceId))
-      Utils.unreserveTableNamespace(cloneInfo.namespaceId, tid, false);
+      Utils.unreserveNamespace(cloneInfo.namespaceId, tid, false);
     Utils.unreserveTable(cloneInfo.tableId, tid, true);
     Tables.clearCache(instance);
   }
@@ -240,7 +240,7 @@ public class CloneTable extends MasterRepo {
   @Override
   public long isReady(long tid, Master environment) throws Exception {
     
-    long val = Utils.reserveTableNamespace(cloneInfo.srcNamespaceId, tid, false, true, TableOperation.CLONE);
+    long val = Utils.reserveNamespace(cloneInfo.srcNamespaceId, tid, false, true, TableOperation.CLONE);
     val += Utils.reserveTable(cloneInfo.srcTableId, tid, false, true, TableOperation.CLONE);
     return val;
   }
@@ -260,7 +260,7 @@ public class CloneTable extends MasterRepo {
   
   @Override
   public void undo(long tid, Master environment) throws Exception {
-    Utils.unreserveTableNamespace(cloneInfo.srcNamespaceId, tid, false);
+    Utils.unreserveNamespace(cloneInfo.srcNamespaceId, tid, false);
     Utils.unreserveTable(cloneInfo.srcTableId, tid, false);
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactRange.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactRange.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactRange.java
index 1c3b398..e8f65d2 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactRange.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactRange.java
@@ -304,7 +304,7 @@ public class CompactRange extends MasterRepo {
 
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    return Utils.reserveTableNamespace(namespaceId, tid, false, true, TableOperation.COMPACT)
+    return Utils.reserveNamespace(namespaceId, tid, false, true, TableOperation.COMPACT)
         + Utils.reserveTable(tableId, tid, false, true, TableOperation.COMPACT);
   }
 
@@ -387,7 +387,7 @@ public class CompactRange extends MasterRepo {
     try {
       removeIterators(tid, tableId);
     } finally {
-      Utils.unreserveTableNamespace(namespaceId, tid, false);
+      Utils.unreserveNamespace(namespaceId, tid, false);
       Utils.unreserveTable(tableId, tid, false);
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateNamespace.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateNamespace.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateNamespace.java
new file mode 100644
index 0000000..fac0cb2
--- /dev/null
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateNamespace.java
@@ -0,0 +1,196 @@
+/*
+ * 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.accumulo.master.tableOps;
+
+import java.io.Serializable;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.core.client.impl.thrift.TableOperation;
+import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.security.NamespacePermission;
+import org.apache.accumulo.fate.Repo;
+import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
+import org.apache.accumulo.master.Master;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.accumulo.server.security.SystemCredentials;
+import org.apache.accumulo.server.tables.TableManager;
+import org.apache.accumulo.server.util.NamespacePropUtil;
+import org.apache.log4j.Logger;
+
+class NamespaceInfo implements Serializable {
+
+  private static final long serialVersionUID = 1L;
+
+  String namespaceName;
+  String namespaceId;
+  String user;
+
+  public Map<String,String> props;
+}
+
+class FinishCreateNamespace extends MasterRepo {
+
+  private static final long serialVersionUID = 1L;
+
+  private NamespaceInfo namespaceInfo;
+
+  public FinishCreateNamespace(NamespaceInfo ti) {
+    this.namespaceInfo = ti;
+  }
+
+  @Override
+  public long isReady(long tid, Master environment) throws Exception {
+    return 0;
+  }
+
+  @Override
+  public Repo<Master> call(long id, Master env) throws Exception {
+
+    Utils.unreserveNamespace(namespaceInfo.namespaceId, id, true);
+
+    env.getEventCoordinator().event("Created namespace %s ", namespaceInfo.namespaceName);
+
+    Logger.getLogger(FinishCreateNamespace.class).debug("Created table " + namespaceInfo.namespaceId + " " + namespaceInfo.namespaceName);
+
+    return null;
+  }
+
+  @Override
+  public String getReturn() {
+    return namespaceInfo.namespaceId;
+  }
+
+  @Override
+  public void undo(long tid, Master env) throws Exception {}
+
+}
+
+class PopulateZookeeperWithNamespace extends MasterRepo {
+
+  private static final long serialVersionUID = 1L;
+
+  private NamespaceInfo namespaceInfo;
+
+  PopulateZookeeperWithNamespace(NamespaceInfo ti) {
+    this.namespaceInfo = ti;
+  }
+
+  @Override
+  public long isReady(long id, Master environment) throws Exception {
+    return Utils.reserveNamespace(namespaceInfo.namespaceId, id, true, false, TableOperation.CREATE);
+  }
+
+  @Override
+  public Repo<Master> call(long tid, Master master) throws Exception {
+
+    Utils.tableNameLock.lock();
+    try {
+      Instance instance = master.getInstance();
+
+      Utils.checkNamespaceDoesNotExist(instance, namespaceInfo.namespaceName, namespaceInfo.namespaceId, TableOperation.CREATE);
+
+      TableManager.getInstance().addNamespace(namespaceInfo.namespaceId, namespaceInfo.namespaceName, NodeExistsPolicy.OVERWRITE);
+
+      for (Entry<String,String> entry : namespaceInfo.props.entrySet())
+        NamespacePropUtil.setNamespaceProperty(namespaceInfo.namespaceId, entry.getKey(), entry.getValue());
+
+      Tables.clearCache(instance);
+
+      return new FinishCreateNamespace(namespaceInfo);
+    } finally {
+      Utils.tableNameLock.unlock();
+    }
+  }
+
+  @Override
+  public void undo(long tid, Master master) throws Exception {
+    TableManager.getInstance().removeNamespace(namespaceInfo.namespaceId);
+    Tables.clearCache(master.getInstance());
+    Utils.unreserveNamespace(namespaceInfo.namespaceId, tid, true);
+  }
+
+}
+
+class SetupNamespacePermissions extends MasterRepo {
+
+  private static final long serialVersionUID = 1L;
+
+  private NamespaceInfo namespaceInfo;
+
+  public SetupNamespacePermissions(NamespaceInfo ti) {
+    this.namespaceInfo = ti;
+  }
+
+  @Override
+  public Repo<Master> call(long tid, Master env) throws Exception {
+    // give all namespace permissions to the creator
+    SecurityOperation security = AuditedSecurityOperation.getInstance();
+    for (NamespacePermission permission : NamespacePermission.values()) {
+      try {
+        security.grantNamespacePermission(SystemCredentials.get().toThrift(env.getInstance()), namespaceInfo.user, namespaceInfo.namespaceId, permission);
+      } catch (ThriftSecurityException e) {
+        Logger.getLogger(FinishCreateNamespace.class).error(e.getMessage(), e);
+        throw e;
+      }
+    }
+
+    // setup permissions in zookeeper before table info in zookeeper
+    // this way concurrent users will not get a spurious permission denied
+    // error
+    return new PopulateZookeeperWithNamespace(namespaceInfo);
+  }
+}
+
+public class CreateNamespace extends MasterRepo {
+  private static final long serialVersionUID = 1L;
+
+  private NamespaceInfo namespaceInfo;
+
+  public CreateNamespace(String user, String namespaceName, Map<String,String> props) {
+    namespaceInfo = new NamespaceInfo();
+    namespaceInfo.namespaceName = namespaceName;
+    namespaceInfo.user = user;
+    namespaceInfo.props = props;
+  }
+
+  @Override
+  public long isReady(long tid, Master environment) throws Exception {
+    return 0;
+  }
+
+  @Override
+  public Repo<Master> call(long tid, Master master) throws Exception {
+    Utils.idLock.lock();
+    try {
+      namespaceInfo.namespaceId = Utils.getNextTableId(namespaceInfo.namespaceName, master.getInstance());
+      return new SetupNamespacePermissions(namespaceInfo);
+    } finally {
+      Utils.idLock.unlock();
+    }
+
+  }
+
+  @Override
+  public void undo(long tid, Master env) throws Exception {
+    // nothing to do, the namespace id was allocated!
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateTable.java
index 10eba61..09c1c11 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateTable.java
@@ -22,9 +22,9 @@ import java.util.Map.Entry;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.admin.TimeType;
-import org.apache.accumulo.core.client.impl.TableNamespaces;
+import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.TableOperation;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
@@ -82,7 +82,7 @@ class FinishCreateTable extends MasterRepo {
   public Repo<Master> call(long tid, Master env) throws Exception {
     TableManager.getInstance().transitionTableState(tableInfo.tableId, TableState.ONLINE);
     
-    Utils.unreserveTableNamespace(tableInfo.namespaceId, tid, false);
+    Utils.unreserveNamespace(tableInfo.namespaceId, tid, false);
     Utils.unreserveTable(tableInfo.tableId, tid, true);
     
     env.getEventCoordinator().event("Created table %s ", tableInfo.tableName);
@@ -281,20 +281,20 @@ public class CreateTable extends MasterRepo {
   
   private TableInfo tableInfo;
   
-  public CreateTable(String user, String tableName, TimeType timeType, Map<String,String> props) throws TableNamespaceNotFoundException {
+  public CreateTable(String user, String tableName, TimeType timeType, Map<String,String> props) throws NamespaceNotFoundException {
     tableInfo = new TableInfo();
     tableInfo.tableName = tableName;
     tableInfo.timeType = TabletTime.getTimeID(timeType);
     tableInfo.user = user;
     tableInfo.props = props;
     Instance inst = HdfsZooInstance.getInstance();
-    tableInfo.namespaceId = TableNamespaces.getNamespaceId(inst, Tables.extractNamespace(tableInfo.tableName));
+    tableInfo.namespaceId = Namespaces.getNamespaceId(inst, Tables.extractNamespace(tableInfo.tableName));
   }
   
   @Override
   public long isReady(long tid, Master environment) throws Exception {
     // reserve the table's namespace to make sure it doesn't change while the table is created
-    return Utils.reserveTableNamespace(tableInfo.namespaceId, tid, false, true, TableOperation.CREATE);
+    return Utils.reserveNamespace(tableInfo.namespaceId, tid, false, true, TableOperation.CREATE);
   }
   
   @Override
@@ -317,7 +317,7 @@ public class CreateTable extends MasterRepo {
   
   @Override
   public void undo(long tid, Master env) throws Exception {
-    Utils.unreserveTableNamespace(tableInfo.namespaceId, tid, false);
+    Utils.unreserveNamespace(tableInfo.namespaceId, tid, false);
   }
   
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateTableNamespace.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateTableNamespace.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateTableNamespace.java
deleted file mode 100644
index 138d77b..0000000
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CreateTableNamespace.java
+++ /dev/null
@@ -1,197 +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.accumulo.master.tableOps;
-
-import java.io.Serializable;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.core.client.impl.thrift.TableOperation;
-import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.security.TableNamespacePermission;
-import org.apache.accumulo.fate.Repo;
-import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
-import org.apache.accumulo.master.Master;
-import org.apache.accumulo.server.security.AuditedSecurityOperation;
-import org.apache.accumulo.server.security.SecurityOperation;
-import org.apache.accumulo.server.security.SystemCredentials;
-import org.apache.accumulo.server.tables.TableManager;
-import org.apache.accumulo.server.util.NamespacePropUtil;
-import org.apache.log4j.Logger;
-
-class TableNamespaceInfo implements Serializable {
-
-  private static final long serialVersionUID = 1L;
-
-  String namespaceName;
-  String namespaceId;
-  String user;
-
-  public Map<String,String> props;
-}
-
-class FinishCreateTableNamespace extends MasterRepo {
-
-  private static final long serialVersionUID = 1L;
-
-  private TableNamespaceInfo tableNamespaceInfo;
-
-  public FinishCreateTableNamespace(TableNamespaceInfo ti) {
-    this.tableNamespaceInfo = ti;
-  }
-
-  @Override
-  public long isReady(long tid, Master environment) throws Exception {
-    return 0;
-  }
-
-  @Override
-  public Repo<Master> call(long id, Master env) throws Exception {
-
-    Utils.unreserveTableNamespace(tableNamespaceInfo.namespaceId, id, true);
-
-    env.getEventCoordinator().event("Created table namespace %s ", tableNamespaceInfo.namespaceName);
-
-    Logger.getLogger(FinishCreateTableNamespace.class).debug("Created table " + tableNamespaceInfo.namespaceId + " " + tableNamespaceInfo.namespaceName);
-
-    return null;
-  }
-
-  @Override
-  public String getReturn() {
-    return tableNamespaceInfo.namespaceId;
-  }
-
-  @Override
-  public void undo(long tid, Master env) throws Exception {}
-
-}
-
-class PopulateZookeeperWithNamespace extends MasterRepo {
-
-  private static final long serialVersionUID = 1L;
-
-  private TableNamespaceInfo tableNamespaceInfo;
-
-  PopulateZookeeperWithNamespace(TableNamespaceInfo ti) {
-    this.tableNamespaceInfo = ti;
-  }
-
-  @Override
-  public long isReady(long id, Master environment) throws Exception {
-    return Utils.reserveTableNamespace(tableNamespaceInfo.namespaceId, id, true, false, TableOperation.CREATE);
-  }
-
-  @Override
-  public Repo<Master> call(long tid, Master master) throws Exception {
-
-    Utils.tableNameLock.lock();
-    try {
-      Instance instance = master.getInstance();
-
-      Utils.checkTableNamespaceDoesNotExist(instance, tableNamespaceInfo.namespaceName, tableNamespaceInfo.namespaceId, TableOperation.CREATE);
-
-      TableManager.getInstance().addNamespace(tableNamespaceInfo.namespaceId, tableNamespaceInfo.namespaceName, NodeExistsPolicy.OVERWRITE);
-
-      for (Entry<String,String> entry : tableNamespaceInfo.props.entrySet())
-        NamespacePropUtil.setNamespaceProperty(tableNamespaceInfo.namespaceId, entry.getKey(), entry.getValue());
-
-      Tables.clearCache(instance);
-
-      return new FinishCreateTableNamespace(tableNamespaceInfo);
-    } finally {
-      Utils.tableNameLock.unlock();
-    }
-  }
-
-  @Override
-  public void undo(long tid, Master master) throws Exception {
-    TableManager.getInstance().removeNamespace(tableNamespaceInfo.namespaceId);
-    Tables.clearCache(master.getInstance());
-    Utils.unreserveTableNamespace(tableNamespaceInfo.namespaceId, tid, true);
-  }
-
-}
-
-class SetupNamespacePermissions extends MasterRepo {
-
-  private static final long serialVersionUID = 1L;
-
-  private TableNamespaceInfo tableNamespaceInfo;
-
-  public SetupNamespacePermissions(TableNamespaceInfo ti) {
-    this.tableNamespaceInfo = ti;
-  }
-
-  @Override
-  public Repo<Master> call(long tid, Master env) throws Exception {
-    // give all table namespace permissions to the creator
-    SecurityOperation security = AuditedSecurityOperation.getInstance();
-    for (TableNamespacePermission permission : TableNamespacePermission.values()) {
-      try {
-        security.grantTableNamespacePermission(SystemCredentials.get().toThrift(env.getInstance()), tableNamespaceInfo.user, tableNamespaceInfo.namespaceId,
-            permission);
-      } catch (ThriftSecurityException e) {
-        Logger.getLogger(FinishCreateTableNamespace.class).error(e.getMessage(), e);
-        throw e;
-      }
-    }
-
-    // setup permissions in zookeeper before table info in zookeeper
-    // this way concurrent users will not get a spurious permission denied
-    // error
-    return new PopulateZookeeperWithNamespace(tableNamespaceInfo);
-  }
-}
-
-public class CreateTableNamespace extends MasterRepo {
-  private static final long serialVersionUID = 1L;
-
-  private TableNamespaceInfo tableNamespaceInfo;
-
-  public CreateTableNamespace(String user, String namespaceName, Map<String,String> props) {
-    tableNamespaceInfo = new TableNamespaceInfo();
-    tableNamespaceInfo.namespaceName = namespaceName;
-    tableNamespaceInfo.user = user;
-    tableNamespaceInfo.props = props;
-  }
-
-  @Override
-  public long isReady(long tid, Master environment) throws Exception {
-    return 0;
-  }
-
-  @Override
-  public Repo<Master> call(long tid, Master master) throws Exception {
-    Utils.idLock.lock();
-    try {
-      tableNamespaceInfo.namespaceId = Utils.getNextTableId(tableNamespaceInfo.namespaceName, master.getInstance());
-      return new SetupNamespacePermissions(tableNamespaceInfo);
-    } finally {
-      Utils.idLock.unlock();
-    }
-
-  }
-
-  @Override
-  public void undo(long tid, Master env) throws Exception {
-    // nothing to do, the namespace id was allocated!
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteNamespace.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteNamespace.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteNamespace.java
new file mode 100644
index 0000000..b6a9578
--- /dev/null
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteNamespace.java
@@ -0,0 +1,104 @@
+/*
+ * 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.accumulo.master.tableOps;
+
+import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.core.client.impl.thrift.TableOperation;
+import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
+import org.apache.accumulo.fate.Repo;
+import org.apache.accumulo.master.Master;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SystemCredentials;
+import org.apache.accumulo.server.tables.TableManager;
+import org.apache.log4j.Logger;
+
+class NamespaceCleanUp extends MasterRepo {
+
+  final private static Logger log = Logger.getLogger(CleanUp.class);
+
+  private static final long serialVersionUID = 1L;
+
+  private String namespaceId;
+
+  public NamespaceCleanUp(String namespaceId) {
+    this.namespaceId = namespaceId;
+  }
+
+  @Override
+  public long isReady(long tid, Master master) throws Exception {
+    return 0;
+  }
+
+  @Override
+  public Repo<Master> call(long id, Master master) throws Exception {
+
+    // remove from zookeeper
+    try {
+      TableManager.getInstance().removeNamespace(namespaceId);
+    } catch (Exception e) {
+      log.error("Failed to find namespace in zookeeper", e);
+    }
+    Tables.clearCache(master.getInstance());
+
+    // remove any permissions associated with this namespace
+    try {
+      AuditedSecurityOperation.getInstance().deleteNamespace(SystemCredentials.get().toThrift(master.getInstance()), namespaceId);
+    } catch (ThriftSecurityException e) {
+      log.error(e.getMessage(), e);
+    }
+
+    Utils.unreserveNamespace(namespaceId, id, true);
+
+    Logger.getLogger(CleanUp.class).debug("Deleted namespace " + namespaceId);
+
+    return null;
+  }
+
+  @Override
+  public void undo(long tid, Master environment) throws Exception {
+    // nothing to do
+  }
+
+}
+
+public class DeleteNamespace extends MasterRepo {
+
+  private static final long serialVersionUID = 1L;
+
+  private String namespaceId;
+
+  public DeleteNamespace(String namespaceId) {
+    this.namespaceId = namespaceId;
+  }
+
+  @Override
+  public long isReady(long id, Master environment) throws Exception {
+    return Utils.reserveNamespace(namespaceId, id, true, true, TableOperation.DELETE);
+  }
+
+  @Override
+  public Repo<Master> call(long tid, Master environment) throws Exception {
+    environment.getEventCoordinator().event("deleting namespace %s ", namespaceId);
+    return new NamespaceCleanUp(namespaceId);
+  }
+
+  @Override
+  public void undo(long id, Master environment) throws Exception {
+    Utils.unreserveNamespace(namespaceId, id, true);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteTable.java
index bf1d9ce..aaeaac5 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteTable.java
@@ -202,7 +202,7 @@ class CleanUp extends MasterRepo {
     }
     
     Utils.unreserveTable(tableId, tid, true);
-    Utils.unreserveTableNamespace(namespaceId, tid, false);
+    Utils.unreserveNamespace(namespaceId, tid, false);
     
     Logger.getLogger(CleanUp.class).debug("Deleted table " + tableId);
     
@@ -231,7 +231,7 @@ public class DeleteTable extends MasterRepo {
   @Override
   public long isReady(long tid, Master environment) throws Exception {
     
-    return Utils.reserveTableNamespace(namespaceId, tid, false, false, TableOperation.DELETE)
+    return Utils.reserveNamespace(namespaceId, tid, false, false, TableOperation.DELETE)
         + Utils.reserveTable(tableId, tid, true, true, TableOperation.DELETE);
   }
   
@@ -244,7 +244,7 @@ public class DeleteTable extends MasterRepo {
   
   @Override
   public void undo(long tid, Master environment) throws Exception {
-    Utils.unreserveTableNamespace(namespaceId, tid, false);
+    Utils.unreserveNamespace(namespaceId, tid, false);
     Utils.unreserveTable(tableId, tid, true);
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteTableNamespace.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteTableNamespace.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteTableNamespace.java
deleted file mode 100644
index bd8e5c3..0000000
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteTableNamespace.java
+++ /dev/null
@@ -1,104 +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.accumulo.master.tableOps;
-
-import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.core.client.impl.thrift.TableOperation;
-import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
-import org.apache.accumulo.fate.Repo;
-import org.apache.accumulo.master.Master;
-import org.apache.accumulo.server.security.AuditedSecurityOperation;
-import org.apache.accumulo.server.security.SystemCredentials;
-import org.apache.accumulo.server.tables.TableManager;
-import org.apache.log4j.Logger;
-
-class NamespaceCleanUp extends MasterRepo {
-
-  final private static Logger log = Logger.getLogger(CleanUp.class);
-
-  private static final long serialVersionUID = 1L;
-
-  private String namespaceId;
-
-  public NamespaceCleanUp(String namespaceId) {
-    this.namespaceId = namespaceId;
-  }
-
-  @Override
-  public long isReady(long tid, Master master) throws Exception {
-    return 0;
-  }
-
-  @Override
-  public Repo<Master> call(long id, Master master) throws Exception {
-
-    // remove from zookeeper
-    try {
-      TableManager.getInstance().removeNamespace(namespaceId);
-    } catch (Exception e) {
-      log.error("Failed to find table namespace in zookeeper", e);
-    }
-    Tables.clearCache(master.getInstance());
-
-    // remove any permissions associated with this table namespace
-    try {
-      AuditedSecurityOperation.getInstance().deleteTableNamespace(SystemCredentials.get().toThrift(master.getInstance()), namespaceId);
-    } catch (ThriftSecurityException e) {
-      log.error(e.getMessage(), e);
-    }
-
-    Utils.unreserveTableNamespace(namespaceId, id, true);
-
-    Logger.getLogger(CleanUp.class).debug("Deleted table namespace " + namespaceId);
-
-    return null;
-  }
-
-  @Override
-  public void undo(long tid, Master environment) throws Exception {
-    // nothing to do
-  }
-
-}
-
-public class DeleteTableNamespace extends MasterRepo {
-
-  private static final long serialVersionUID = 1L;
-
-  private String namespaceId;
-
-  public DeleteTableNamespace(String namespaceId) {
-    this.namespaceId = namespaceId;
-  }
-
-  @Override
-  public long isReady(long id, Master environment) throws Exception {
-    return Utils.reserveTableNamespace(namespaceId, id, true, true, TableOperation.DELETE);
-  }
-
-  @Override
-  public Repo<Master> call(long tid, Master environment) throws Exception {
-    environment.getEventCoordinator().event("deleting table namespace %s ", namespaceId);
-    return new NamespaceCleanUp(namespaceId);
-  }
-
-  @Override
-  public void undo(long id, Master environment) throws Exception {
-    Utils.unreserveTableNamespace(namespaceId, id, true);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/master/src/main/java/org/apache/accumulo/master/tableOps/ExportTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ExportTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ExportTable.java
index cad779c..dd2d3f4 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ExportTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ExportTable.java
@@ -94,7 +94,7 @@ class WriteExportFiles extends MasterRepo {
   @Override
   public long isReady(long tid, Master master) throws Exception {
     
-    long reserved = Utils.reserveTableNamespace(tableInfo.namespaceID, tid, false, true, TableOperation.EXPORT)
+    long reserved = Utils.reserveNamespace(tableInfo.namespaceID, tid, false, true, TableOperation.EXPORT)
         + Utils.reserveTable(tableInfo.tableID, tid, false, true, TableOperation.EXPORT);
     if (reserved > 0)
       return reserved;
@@ -137,7 +137,7 @@ class WriteExportFiles extends MasterRepo {
       throw new ThriftTableOperationException(tableInfo.tableID, tableInfo.tableName, TableOperation.EXPORT, TableOperationExceptionType.OTHER,
           "Failed to create export files " + ioe.getMessage());
     }
-    Utils.unreserveTableNamespace(tableInfo.namespaceID, tid, false);
+    Utils.unreserveNamespace(tableInfo.namespaceID, tid, false);
     Utils.unreserveTable(tableInfo.tableID, tid, false);
     Utils.unreserveHdfsDirectory(new Path(tableInfo.exportDir).toString(), tid);
     return null;
@@ -145,7 +145,7 @@ class WriteExportFiles extends MasterRepo {
   
   @Override
   public void undo(long tid, Master env) throws Exception {
-    Utils.unreserveTableNamespace(tableInfo.namespaceID, tid, false);
+    Utils.unreserveNamespace(tableInfo.namespaceID, tid, false);
     Utils.unreserveTable(tableInfo.tableID, tid, false);
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
index c9ed10f..6875bb5 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
@@ -34,9 +34,9 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.admin.TableOperationsImpl;
-import org.apache.accumulo.core.client.impl.TableNamespaces;
+import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.TableOperation;
 import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
@@ -109,7 +109,7 @@ class FinishImportTable extends MasterRepo {
     
     TableManager.getInstance().transitionTableState(tableInfo.tableId, TableState.ONLINE);
     
-    Utils.unreserveTableNamespace(tableInfo.namespaceId, tid, false);
+    Utils.unreserveNamespace(tableInfo.namespaceId, tid, false);
     Utils.unreserveTable(tableInfo.tableId, tid, true);
     
     Utils.unreserveHdfsDirectory(new Path(tableInfo.exportDir).toString(), tid);
@@ -450,7 +450,7 @@ class ImportPopulateZookeeper extends MasterRepo {
       TableManager.getInstance().addTable(tableInfo.tableId, tableInfo.tableName, NodeExistsPolicy.OVERWRITE);
       
       String namespace = Tables.extractNamespace(tableInfo.tableName);
-      String namespaceId = TableNamespaces.getNamespaceId(instance, namespace);
+      String namespaceId = Namespaces.getNamespaceId(instance, namespace);
       TableManager.getInstance().addNamespaceToTable(tableInfo.tableId, namespaceId);
       
       Tables.clearCache(instance);
@@ -521,18 +521,18 @@ public class ImportTable extends MasterRepo {
   
   private ImportedTableInfo tableInfo;
   
-  public ImportTable(String user, String tableName, String exportDir) throws TableNamespaceNotFoundException {
+  public ImportTable(String user, String tableName, String exportDir) throws NamespaceNotFoundException {
     tableInfo = new ImportedTableInfo();
     tableInfo.tableName = tableName;
     tableInfo.user = user;
     tableInfo.exportDir = exportDir;
     Instance inst = HdfsZooInstance.getInstance();
-    tableInfo.namespaceId = TableNamespaces.getNamespaceId(inst, Tables.extractNamespace(tableName));
+    tableInfo.namespaceId = Namespaces.getNamespaceId(inst, Tables.extractNamespace(tableName));
   }
   
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    return Utils.reserveHdfsDirectory(new Path(tableInfo.exportDir).toString(), tid) + Utils.reserveTableNamespace(tableInfo.namespaceId, tid, false, true, TableOperation.IMPORT);
+    return Utils.reserveHdfsDirectory(new Path(tableInfo.exportDir).toString(), tid) + Utils.reserveNamespace(tableInfo.namespaceId, tid, false, true, TableOperation.IMPORT);
   }
   
   @Override
@@ -612,6 +612,6 @@ public class ImportTable extends MasterRepo {
   @Override
   public void undo(long tid, Master env) throws Exception {
     Utils.unreserveHdfsDirectory(new Path(tableInfo.exportDir).toString(), tid);
-    Utils.unreserveTableNamespace(tableInfo.namespaceId, tid, false);
+    Utils.unreserveNamespace(tableInfo.namespaceId, tid, false);
   }
 }