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:08 UTC

[27/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/conf/TableConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
index 530b2c2..ce7a882 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
@@ -35,22 +35,22 @@ import org.apache.log4j.Logger;
 
 public class TableConfiguration extends AccumuloConfiguration {
   private static final Logger log = Logger.getLogger(TableConfiguration.class);
-  
+
   private static ZooCache tablePropCache = null;
   private final String instanceId;
-  private final TableNamespaceConfiguration parent;
-  
+  private final NamespaceConfiguration parent;
+
   private String table = null;
   private Set<ConfigurationObserver> observers;
-  
-  public TableConfiguration(String instanceId, String table, TableNamespaceConfiguration parent) {
+
+  public TableConfiguration(String instanceId, String table, NamespaceConfiguration parent) {
     this.instanceId = instanceId;
     this.table = table;
     this.parent = parent;
-    
+
     this.observers = Collections.synchronizedSet(new HashSet<ConfigurationObserver>());
   }
-  
+
   private static ZooCache getTablePropCache() {
     Instance inst = HdfsZooInstance.getInstance();
     if (tablePropCache == null)
@@ -60,7 +60,7 @@ public class TableConfiguration extends AccumuloConfiguration {
       }
     return tablePropCache;
   }
-  
+
   public void addObserver(ConfigurationObserver co) {
     if (table == null) {
       String err = "Attempt to add observer for non-table configuration";
@@ -70,7 +70,7 @@ public class TableConfiguration extends AccumuloConfiguration {
     iterator();
     observers.add(co);
   }
-  
+
   public void removeObserver(ConfigurationObserver configObserver) {
     if (table == null) {
       String err = "Attempt to remove observer for non-table configuration";
@@ -79,30 +79,30 @@ public class TableConfiguration extends AccumuloConfiguration {
     }
     observers.remove(configObserver);
   }
-  
+
   public void expireAllObservers() {
     Collection<ConfigurationObserver> copy = Collections.unmodifiableCollection(observers);
     for (ConfigurationObserver co : copy)
       co.sessionExpired();
   }
-  
+
   public void propertyChanged(String key) {
     Collection<ConfigurationObserver> copy = Collections.unmodifiableCollection(observers);
     for (ConfigurationObserver co : copy)
       co.propertyChanged(key);
   }
-  
+
   public void propertiesChanged(String key) {
     Collection<ConfigurationObserver> copy = Collections.unmodifiableCollection(observers);
     for (ConfigurationObserver co : copy)
       co.propertiesChanged();
   }
-  
+
   @Override
   public String get(Property property) {
     String key = property.getKey();
     String value = get(getTablePropCache(), key);
-    
+
     if (value == null || !property.getType().isValidFormat(value)) {
       if (value != null)
         log.error("Using default value for " + key + " due to improperly formatted " + property.getType() + ": " + value);
@@ -110,7 +110,7 @@ public class TableConfiguration extends AccumuloConfiguration {
     }
     return value;
   }
-  
+
   private String get(ZooCache zc, String key) {
     String zPath = ZooUtil.getRoot(instanceId) + Constants.ZTABLES + "/" + table + Constants.ZTABLE_CONF + "/" + key;
     byte[] v = zc.get(zPath);
@@ -137,22 +137,22 @@ public class TableConfiguration extends AccumuloConfiguration {
       }
     }
   }
-  
+
   public String getTableId() {
     return table;
   }
-  
-  /** 
-   * returns the actual TableNamespaceConfiguration that corresponds to the current parent namespace.
+
+  /**
+   * returns the actual NamespaceConfiguration that corresponds to the current parent namespace.
    */
-  public TableNamespaceConfiguration getNamespaceConfiguration() {
-    return ServerConfiguration.getTableNamespaceConfiguration(parent.inst, parent.namespaceId);
+  public NamespaceConfiguration getNamespaceConfiguration() {
+    return ServerConfiguration.getNamespaceConfiguration(parent.inst, parent.namespaceId);
   }
-  
+
   /**
    * returns the parent, which is actually a TableParentConfiguration that can change which namespace it references
    */
-  public TableNamespaceConfiguration getParentConfiguration() {
+  public NamespaceConfiguration getParentConfiguration() {
     return parent;
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/base/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfWatcher.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfWatcher.java b/server/base/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfWatcher.java
deleted file mode 100644
index b331940..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfWatcher.java
+++ /dev/null
@@ -1,107 +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.server.conf;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-
-class TableNamespaceConfWatcher implements Watcher {
-  static {
-    Logger.getLogger("org.apache.zookeeper").setLevel(Level.WARN);
-    Logger.getLogger("org.apache.hadoop.io.compress").setLevel(Level.WARN);
-  }
-
-  private static final Logger log = Logger.getLogger(TableNamespaceConfWatcher.class);
-  private Instance instance = null;
-
-  TableNamespaceConfWatcher(Instance instance) {
-    this.instance = instance;
-  }
-
-  @Override
-  public void process(WatchedEvent event) {
-    String path = event.getPath();
-    if (log.isTraceEnabled())
-      log.trace("WatchEvent : " + path + " " + event.getState() + " " + event.getType());
-
-    String namespacesPrefix = ZooUtil.getRoot(instance) + Constants.ZNAMESPACES + "/";
-
-    String namespaceId = null;
-    String key = null;
-
-    if (path != null) {
-      if (path.startsWith(namespacesPrefix)) {
-        namespaceId = path.substring(namespacesPrefix.length());
-        if (namespaceId.contains("/")) {
-          namespaceId = namespaceId.substring(0, namespaceId.indexOf('/'));
-          if (path.startsWith(namespacesPrefix + namespaceId + Constants.ZNAMESPACE_CONF + "/"))
-            key = path.substring((namespacesPrefix + namespaceId + Constants.ZNAMESPACE_CONF + "/").length());
-        }
-      }
-
-      if (namespaceId == null) {
-        log.warn("Zookeeper told me about a path I was not watching " + path + " state=" + event.getState() + " type=" + event.getType());
-        return;
-      }
-    }
-
-    switch (event.getType()) {
-      case NodeDataChanged:
-        if (log.isTraceEnabled())
-          log.trace("EventNodeDataChanged " + event.getPath());
-        if (key != null)
-          ServerConfiguration.getTableNamespaceConfiguration(instance, namespaceId).propertyChanged(key);
-        break;
-      case NodeChildrenChanged:
-        ServerConfiguration.getTableNamespaceConfiguration(instance, namespaceId).propertiesChanged(key);
-        break;
-      case NodeDeleted:
-        if (key == null) {
-          ServerConfiguration.removeNamespaceIdInstance(namespaceId);
-        }
-        break;
-      case None:
-        switch (event.getState()) {
-          case Expired:
-            ServerConfiguration.expireAllTableObservers();
-            break;
-          case SyncConnected:
-            break;
-          case Disconnected:
-            break;
-          default:
-            log.warn("EventNone event not handled path = " + event.getPath() + " state=" + event.getState());
-        }
-        break;
-      case NodeCreated:
-        switch (event.getState()) {
-          case SyncConnected:
-            break;
-          default:
-            log.warn("Event NodeCreated event not handled path = " + event.getPath() + " state=" + event.getState());
-        }
-        break;
-      default:
-        log.warn("Event not handled path = " + event.getPath() + " state=" + event.getState() + " type = " + event.getType());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/base/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfiguration.java
deleted file mode 100644
index 888c5ed..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfiguration.java
+++ /dev/null
@@ -1,174 +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.server.conf;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ConfigurationObserver;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.accumulo.fate.zookeeper.ZooCache;
-import org.apache.accumulo.server.client.HdfsZooInstance;
-import org.apache.log4j.Logger;
-
-public class TableNamespaceConfiguration extends AccumuloConfiguration {
-  private static final Logger log = Logger.getLogger(TableNamespaceConfiguration.class);
-
-  private final AccumuloConfiguration parent;
-  private static ZooCache propCache = null;
-  protected String namespaceId = null;
-  protected Instance inst = null;
-  private Set<ConfigurationObserver> observers;
-
-  public TableNamespaceConfiguration(String namespaceId, AccumuloConfiguration parent) {
-    inst = HdfsZooInstance.getInstance();
-    this.parent = parent;
-    this.namespaceId = namespaceId;
-    this.observers = Collections.synchronizedSet(new HashSet<ConfigurationObserver>());
-  }
-
-  @Override
-  public String get(Property property) {
-    String key = property.getKey();
-    String value = get(getPropCache(), key);
-
-    if (value == null || !property.getType().isValidFormat(value)) {
-      if (value != null)
-        log.error("Using default value for " + key + " due to improperly formatted " + property.getType() + ": " + value);
-      if (!(namespaceId.equals(Constants.SYSTEM_TABLE_NAMESPACE_ID) && isIteratorOrConstraint(property.getKey()))) {
-        // ignore iterators from parent if system namespace
-        value = parent.get(property);
-      }
-    }
-    return value;
-  }
-
-  private String get(ZooCache zc, String key) {
-    String zPath = ZooUtil.getRoot(inst.getInstanceID()) + Constants.ZNAMESPACES + "/" + getNamespaceId() + Constants.ZNAMESPACE_CONF + "/" + key;
-    byte[] v = zc.get(zPath);
-    String value = null;
-    if (v != null)
-      value = new String(v, Constants.UTF8);
-    return value;
-  }
-
-  private static ZooCache getPropCache() {
-    Instance inst = HdfsZooInstance.getInstance();
-    if (propCache == null)
-      synchronized (TableNamespaceConfiguration.class) {
-        if (propCache == null)
-          propCache = new ZooCache(inst.getZooKeepers(), inst.getZooKeepersSessionTimeOut(), new TableNamespaceConfWatcher(inst));
-      }
-    return propCache;
-  }
-
-  private class SystemNamespaceFilter implements PropertyFilter {
-
-    private PropertyFilter userFilter;
-
-    SystemNamespaceFilter(PropertyFilter userFilter) {
-      this.userFilter = userFilter;
-    }
-
-    @Override
-    public boolean accept(String key) {
-      if (isIteratorOrConstraint(key))
-        return false;
-      return userFilter.accept(key);
-    }
-
-  }
-
-  @Override
-  public void getProperties(Map<String,String> props, PropertyFilter filter) {
-
-    PropertyFilter parentFilter = filter;
-
-    // exclude system iterators/constraints from the system namespace
-    // so they don't affect the metadata or root tables.
-    if (this.namespaceId.equals(Constants.SYSTEM_TABLE_NAMESPACE_ID))
-      parentFilter = new SystemNamespaceFilter(filter);
-
-    parent.getProperties(props, parentFilter);
-
-    ZooCache zc = getPropCache();
-
-    List<String> children = zc.getChildren(ZooUtil.getRoot(inst.getInstanceID()) + Constants.ZNAMESPACES + "/" + getNamespaceId() + Constants.ZNAMESPACE_CONF);
-    if (children != null) {
-      for (String child : children) {
-        if (child != null && filter.accept(child)) {
-          String value = get(zc, child);
-          if (value != null)
-            props.put(child, value);
-        }
-      }
-    }
-  }
-
-  protected String getNamespaceId() {
-    return namespaceId;
-  }
-
-  public void addObserver(ConfigurationObserver co) {
-    if (namespaceId == null) {
-      String err = "Attempt to add observer for non-table-namespace configuration";
-      log.error(err);
-      throw new RuntimeException(err);
-    }
-    iterator();
-    observers.add(co);
-  }
-
-  public void removeObserver(ConfigurationObserver configObserver) {
-    if (namespaceId == null) {
-      String err = "Attempt to remove observer for non-table-namespace configuration";
-      log.error(err);
-      throw new RuntimeException(err);
-    }
-    observers.remove(configObserver);
-  }
-
-  public void expireAllObservers() {
-    Collection<ConfigurationObserver> copy = Collections.unmodifiableCollection(observers);
-    for (ConfigurationObserver co : copy)
-      co.sessionExpired();
-  }
-
-  public void propertyChanged(String key) {
-    Collection<ConfigurationObserver> copy = Collections.unmodifiableCollection(observers);
-    for (ConfigurationObserver co : copy)
-      co.propertyChanged(key);
-  }
-
-  public void propertiesChanged(String key) {
-    Collection<ConfigurationObserver> copy = Collections.unmodifiableCollection(observers);
-    for (ConfigurationObserver co : copy)
-      co.propertiesChanged();
-  }
-
-  protected boolean isIteratorOrConstraint(String key) {
-    return key.startsWith(Property.TABLE_ITERATOR_PREFIX.getKey()) || key.startsWith(Property.TABLE_CONSTRAINT_PREFIX.getKey());
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/base/src/main/java/org/apache/accumulo/server/conf/TableParentConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/TableParentConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/TableParentConfiguration.java
index 8608a3e..902ecd0 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/TableParentConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/TableParentConfiguration.java
@@ -20,9 +20,9 @@ import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 
 /**
- * Used by TableConfiguration to dynamically get the TableNamespaceConfiguration if the namespace changes
+ * Used by TableConfiguration to dynamically get the NamespaceConfiguration if the namespace changes
  */
-public class TableParentConfiguration extends TableNamespaceConfiguration {
+public class TableParentConfiguration extends NamespaceConfiguration {
 
   private String tableId;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
index 0e51b6c..61508db 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
@@ -416,16 +416,16 @@ public class Initialize {
     zoo.putPersistentData(zkInstanceRoot + Constants.ZRECOVERY, new byte[] {'0'}, NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZNAMESPACES, new byte[0], NodeExistsPolicy.FAIL);
     
-    createInitialTableNamespace(zoo, zkInstanceRoot, Constants.DEFAULT_TABLE_NAMESPACE_ID, Constants.DEFAULT_TABLE_NAMESPACE, true);
-    createInitialTableNamespace(zoo, zkInstanceRoot, Constants.SYSTEM_TABLE_NAMESPACE_ID, Constants.SYSTEM_TABLE_NAMESPACE, false);
+    createInitialNamespace(zoo, zkInstanceRoot, Constants.DEFAULT_NAMESPACE_ID, Constants.DEFAULT_NAMESPACE, true);
+    createInitialNamespace(zoo, zkInstanceRoot, Constants.SYSTEM_NAMESPACE_ID, Constants.SYSTEM_NAMESPACE, false);
     
     zoo.putPersistentData(zkInstanceRoot + Constants.ZTABLES + "/" + MetadataTable.ID + Constants.ZTABLE_NAMESPACE,
-        Constants.SYSTEM_TABLE_NAMESPACE_ID.getBytes(Constants.UTF8), NodeExistsPolicy.OVERWRITE);
+        Constants.SYSTEM_NAMESPACE_ID.getBytes(Constants.UTF8), NodeExistsPolicy.OVERWRITE);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZTABLES + "/" + RootTable.ID + Constants.ZTABLE_NAMESPACE,
-        Constants.SYSTEM_TABLE_NAMESPACE_ID.getBytes(Constants.UTF8), NodeExistsPolicy.OVERWRITE);
+        Constants.SYSTEM_NAMESPACE_ID.getBytes(Constants.UTF8), NodeExistsPolicy.OVERWRITE);
   }
   
-  private static void createInitialTableNamespace(IZooReaderWriter zoo, String root, String id, String namespace, boolean defaultOpts) throws KeeperException,
+  private static void createInitialNamespace(IZooReaderWriter zoo, String root, String id, String namespace, boolean defaultOpts) throws KeeperException,
       InterruptedException {
     String zPath = root + Constants.ZNAMESPACES + "/" + id;
     zoo.putPersistentData(zPath, new byte[0], NodeExistsPolicy.FAIL);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
index 7e7dde9..f2a7ac8 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
@@ -23,10 +23,10 @@ import java.util.Set;
 
 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.admin.SecurityOperationsImpl;
-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.SecurityErrorCode;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
@@ -42,8 +42,8 @@ import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 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.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.server.client.HdfsZooInstance;
@@ -63,21 +63,21 @@ import org.apache.log4j.Logger;
  */
 public class SecurityOperation {
   private static final Logger log = Logger.getLogger(SecurityOperationsImpl.class);
-  
+
   protected Authorizor authorizor;
   protected Authenticator authenticator;
   protected PermissionHandler permHandle;
   private static String rootUserName = null;
   private final ZooCache zooCache;
   private final String ZKUserPath;
-  
+
   protected static SecurityOperation instance;
-  
+
   public static synchronized SecurityOperation getInstance() {
     String instanceId = HdfsZooInstance.getInstance().getInstanceID();
     return getInstance(instanceId, false);
   }
-  
+
   public static synchronized SecurityOperation getInstance(String instanceId, boolean initialize) {
     if (instance == null) {
       instance = new SecurityOperation(getAuthorizor(instanceId, initialize), getAuthenticator(instanceId, initialize), getPermHandler(instanceId, initialize),
@@ -85,51 +85,51 @@ public class SecurityOperation {
     }
     return instance;
   }
-  
+
   protected static Authorizor getAuthorizor(String instanceId, boolean initialize) {
     Authorizor toRet = ServerConfiguration.getSiteConfiguration().instantiateClassProperty(Property.INSTANCE_SECURITY_AUTHORIZOR, Authorizor.class,
         ZKAuthorizor.getInstance());
     toRet.initialize(instanceId, initialize);
     return toRet;
   }
-  
+
   protected static Authenticator getAuthenticator(String instanceId, boolean initialize) {
     Authenticator toRet = ServerConfiguration.getSiteConfiguration().instantiateClassProperty(Property.INSTANCE_SECURITY_AUTHENTICATOR, Authenticator.class,
         ZKAuthenticator.getInstance());
     toRet.initialize(instanceId, initialize);
     return toRet;
   }
-  
+
   protected static PermissionHandler getPermHandler(String instanceId, boolean initialize) {
     PermissionHandler toRet = ServerConfiguration.getSiteConfiguration().instantiateClassProperty(Property.INSTANCE_SECURITY_PERMISSION_HANDLER,
         PermissionHandler.class, ZKPermHandler.getInstance());
     toRet.initialize(instanceId, initialize);
     return toRet;
   }
-  
+
   protected SecurityOperation(String instanceId) {
     ZKUserPath = Constants.ZROOT + "/" + instanceId + "/users";
     zooCache = new ZooCache();
   }
-  
+
   public SecurityOperation(Authorizor author, Authenticator authent, PermissionHandler pm, String instanceId) {
     this(instanceId);
     authorizor = author;
     authenticator = authent;
     permHandle = pm;
-    
+
     if (!authorizor.validSecurityHandlers(authenticator, pm) || !authenticator.validSecurityHandlers(authorizor, pm)
         || !permHandle.validSecurityHandlers(authent, author))
       throw new RuntimeException(authorizor + ", " + authenticator + ", and " + pm
           + " do not play nice with eachother. Please choose authentication and authorization mechanisms that are compatible with one another.");
   }
-  
+
   public void initializeSecurity(TCredentials credentials, String rootPrincipal, byte[] token) throws AccumuloSecurityException, ThriftSecurityException {
     authenticate(credentials);
-    
+
     if (!isSystemUser(credentials))
       throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-    
+
     authenticator.initializeSecurity(credentials, rootPrincipal, token);
     authorizor.initializeSecurity(credentials, rootPrincipal);
     permHandle.initializeSecurity(credentials, rootPrincipal);
@@ -140,21 +140,21 @@ public class SecurityOperation {
       throw new RuntimeException(e);
     }
   }
-  
+
   public synchronized String getRootUsername() {
     if (rootUserName == null)
       rootUserName = new String(zooCache.get(ZKUserPath));
     return rootUserName;
   }
-  
+
   public boolean isSystemUser(TCredentials credentials) {
     return SystemCredentials.get().getToken().getClass().getName().equals(credentials.getTokenClassName());
   }
-  
+
   private void authenticate(TCredentials credentials) throws ThriftSecurityException {
     if (!credentials.getInstanceId().equals(HdfsZooInstance.getInstance().getInstanceID()))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.INVALID_INSTANCEID);
-    
+
     if (isSystemUser(credentials)) {
       authenticateSystemUser(credentials);
     } else {
@@ -169,19 +169,19 @@ public class SecurityOperation {
       }
     }
   }
-  
+
   private void authenticateSystemUser(TCredentials credentials) throws ThriftSecurityException {
     if (SystemCredentials.get().getToken().equals(credentials.getToken()))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.BAD_CREDENTIALS);
   }
-  
+
   public boolean canAskAboutUser(TCredentials credentials, String user) throws ThriftSecurityException {
     // Authentication done in canPerformSystemActions
     if (!(canPerformSystemActions(credentials) || credentials.getPrincipal().equals(user)))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     return true;
   }
-  
+
   public boolean authenticateUser(TCredentials credentials, TCredentials toAuth) throws ThriftSecurityException {
     canAskAboutUser(credentials, toAuth.getPrincipal());
     // User is already authenticated from canAskAboutUser
@@ -194,7 +194,7 @@ public class SecurityOperation {
       throw e.asThriftException();
     }
   }
-  
+
   private AuthenticationToken reassembleToken(TCredentials toAuth) throws AccumuloSecurityException {
     String tokenClass = toAuth.getTokenClassName();
     if (authenticator.validTokenClass(tokenClass)) {
@@ -202,22 +202,22 @@ public class SecurityOperation {
     }
     throw new AccumuloSecurityException(toAuth.getPrincipal(), SecurityErrorCode.INVALID_TOKEN);
   }
-  
+
   public Authorizations getUserAuthorizations(TCredentials credentials, String user) throws ThriftSecurityException {
     authenticate(credentials);
-    
+
     targetUserExists(user);
-    
+
     if (!credentials.getPrincipal().equals(user) && !hasSystemPermission(credentials, SystemPermission.SYSTEM, false))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-    
+
     try {
       return authorizor.getCachedUserAuthorizations(user);
     } catch (AccumuloSecurityException e) {
       throw e.asThriftException();
     }
   }
-  
+
   public Authorizations getUserAuthorizations(TCredentials credentials) throws ThriftSecurityException {
     // system user doesn't need record-level authorizations for the tables it reads
     if (isSystemUser(credentials)) {
@@ -226,22 +226,22 @@ public class SecurityOperation {
     }
     return getUserAuthorizations(credentials, credentials.getPrincipal());
   }
-  
+
   public boolean userHasAuthorizations(TCredentials credentials, List<ByteBuffer> list) throws ThriftSecurityException {
     authenticate(credentials);
-    
+
     if (isSystemUser(credentials)) {
       // system user doesn't need record-level authorizations for the tables it reads (for now)
       return list.isEmpty();
     }
-    
+
     try {
       return authorizor.isValidAuthorizations(credentials.getPrincipal(), list);
     } catch (AccumuloSecurityException e) {
       throw e.asThriftException();
     }
   }
-  
+
   /**
    * Checks if a user has a system permission
    * 
@@ -252,7 +252,7 @@ public class SecurityOperation {
       return true;
     return _hasSystemPermission(credentials.getPrincipal(), permission, useCached);
   }
-  
+
   /**
    * Checks if a user has a system permission<br/>
    * This cannot check if a system user has permission.
@@ -262,9 +262,9 @@ public class SecurityOperation {
   private boolean _hasSystemPermission(String user, SystemPermission permission, boolean useCached) throws ThriftSecurityException {
     if (user.equals(getRootUsername()))
       return true;
-    
+
     targetUserExists(user);
-    
+
     try {
       if (useCached)
         return permHandle.hasCachedSystemPermission(user, permission);
@@ -273,7 +273,7 @@ public class SecurityOperation {
       throw e.asThriftException();
     }
   }
-  
+
   /**
    * Checks if a user has a table permission
    * 
@@ -284,7 +284,7 @@ public class SecurityOperation {
       return true;
     return _hasTablePermission(credentials.getPrincipal(), table, permission, useCached);
   }
-  
+
   /**
    * Checks if a user has a table permission<br/>
    * This cannot check if a system user has permission.
@@ -293,10 +293,10 @@ public class SecurityOperation {
    */
   protected boolean _hasTablePermission(String user, String table, TablePermission permission, boolean useCached) throws ThriftSecurityException {
     targetUserExists(user);
-    
+
     if ((table.equals(MetadataTable.ID) || table.equals(RootTable.ID)) && permission.equals(TablePermission.READ))
       return true;
-    
+
     try {
       if (useCached)
         return permHandle.hasCachedTablePermission(user, table, permission);
@@ -307,70 +307,69 @@ public class SecurityOperation {
       throw new ThriftSecurityException(user, SecurityErrorCode.TABLE_DOESNT_EXIST);
     }
   }
-  
+
   /**
-   * Checks if a user has a table namespace permission
+   * Checks if a user has a namespace permission
    * 
    * @return true if a user exists and has permission; false otherwise
    */
-  protected boolean hasTableNamespacePermission(TCredentials credentials, String tableNamespace, TableNamespacePermission permission, boolean useCached)
+  protected boolean hasNamespacePermission(TCredentials credentials, String namespace, NamespacePermission permission, boolean useCached)
       throws ThriftSecurityException {
     if (isSystemUser(credentials))
       return true;
-    return _hasTableNamespacePermission(credentials.getPrincipal(), tableNamespace, permission, useCached);
+    return _hasNamespacePermission(credentials.getPrincipal(), namespace, permission, useCached);
   }
-  
+
   /**
-   * Checks if a user has a table namespace permission given a tableId
+   * Checks if a user has a namespace permission given a tableId
    * 
    * @return true if a user exists and has permission; false otherwise
    */
-  protected boolean hasTableNamespacePermissionForTableId(TCredentials credentials, String tableId, TableNamespacePermission permission, boolean useCached)
+  protected boolean hasNamespacePermissionForTableId(TCredentials credentials, String tableId, NamespacePermission permission, boolean useCached)
       throws ThriftSecurityException {
-    String tableNamespace = Tables.getNamespace(HdfsZooInstance.getInstance(), tableId);
-    return hasTableNamespacePermission(credentials, tableNamespace, permission, useCached);
+    String namespace = Tables.getNamespace(HdfsZooInstance.getInstance(), tableId);
+    return hasNamespacePermission(credentials, namespace, permission, useCached);
   }
-  
+
   /**
-   * Checks if a user has a table namespace permission given a tableName
+   * Checks if a user has a namespace permission given a tableName
    * 
    * @return true if a user exists and has permission; false otherwise
    */
-  protected boolean hasTableNamespacePermissionForTableName(TCredentials credentials, String tableName, TableNamespacePermission permission, boolean useCached)
+  protected boolean hasNamespacePermissionForTableName(TCredentials credentials, String tableName, NamespacePermission permission, boolean useCached)
       throws ThriftSecurityException {
-    String tableNamespace = Tables.extractNamespace(tableName);
+    String namespace = Tables.extractNamespace(tableName);
     try {
-      String namespace = TableNamespaces.getNamespaceId(HdfsZooInstance.getInstance(), tableNamespace);
-      return hasTableNamespacePermission(credentials, namespace, permission, useCached);
-    } catch (TableNamespaceNotFoundException e) {
-      throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.TABLE_NAMESPACE_DOESNT_EXIST);
+      String namespaceId = Namespaces.getNamespaceId(HdfsZooInstance.getInstance(), namespace);
+      return hasNamespacePermission(credentials, namespaceId, permission, useCached);
+    } catch (NamespaceNotFoundException e) {
+      throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
     }
   }
-  
+
   /**
-   * Checks if a user has a table namespace permission<br/>
+   * Checks if a user has a namespace permission<br/>
    * This cannot check if a system user has permission.
    * 
    * @return true if a user exists and has permission; false otherwise
    */
-  protected boolean _hasTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission, boolean useCached)
-      throws ThriftSecurityException {
+  protected boolean _hasNamespacePermission(String user, String namespace, NamespacePermission permission, boolean useCached) throws ThriftSecurityException {
     targetUserExists(user);
-    
-    if (tableNamespace.equals(Constants.SYSTEM_TABLE_NAMESPACE_ID) && permission.equals(TableNamespacePermission.READ))
+
+    if (namespace.equals(Constants.SYSTEM_NAMESPACE_ID) && permission.equals(NamespacePermission.READ))
       return true;
-    
+
     try {
       if (useCached)
-        return permHandle.hasCachedTableNamespacePermission(user, tableNamespace, permission);
-      return permHandle.hasTableNamespacePermission(user, tableNamespace, permission);
+        return permHandle.hasCachedNamespacePermission(user, namespace, permission);
+      return permHandle.hasNamespacePermission(user, namespace, permission);
     } catch (AccumuloSecurityException e) {
       throw e.asThriftException();
-    } catch (TableNamespaceNotFoundException e) {
-      throw new ThriftSecurityException(user, SecurityErrorCode.TABLE_NAMESPACE_DOESNT_EXIST);
+    } catch (NamespaceNotFoundException e) {
+      throw new ThriftSecurityException(user, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
     }
   }
-  
+
   // some people just aren't allowed to ask about other users; here are those who can ask
   private boolean canAskAboutOtherUsers(TCredentials credentials, String user) throws ThriftSecurityException {
     authenticate(credentials);
@@ -378,7 +377,7 @@ public class SecurityOperation {
         || hasSystemPermission(credentials, SystemPermission.CREATE_USER, false) || hasSystemPermission(credentials, SystemPermission.ALTER_USER, false)
         || hasSystemPermission(credentials, SystemPermission.DROP_USER, false);
   }
-  
+
   private void targetUserExists(String user) throws ThriftSecurityException {
     if (user.equals(getRootUsername()))
       return;
@@ -389,46 +388,46 @@ public class SecurityOperation {
       throw e.asThriftException();
     }
   }
-  
+
   public boolean canScan(TCredentials credentials, String table) throws ThriftSecurityException {
     authenticate(credentials);
     return hasTablePermission(credentials, table, TablePermission.READ, true)
-        || hasTableNamespacePermissionForTableId(credentials, table, TableNamespacePermission.READ, true);
+        || hasNamespacePermissionForTableId(credentials, table, NamespacePermission.READ, true);
   }
-  
+
   public boolean canScan(TCredentials credentials, String table, TRange range, List<TColumn> columns, List<IterInfo> ssiList,
       Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations) throws ThriftSecurityException {
     return canScan(credentials, table);
   }
-  
+
   public boolean canScan(TCredentials credentials, String table, Map<TKeyExtent,List<TRange>> tbatch, List<TColumn> tcolumns, List<IterInfo> ssiList,
       Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations) throws ThriftSecurityException {
     return canScan(credentials, table);
   }
-  
+
   public boolean canWrite(TCredentials credentials, String table) throws ThriftSecurityException {
     authenticate(credentials);
     return hasTablePermission(credentials, table, TablePermission.WRITE, true)
-        || hasTableNamespacePermissionForTableId(credentials, table, TableNamespacePermission.WRITE, true);
+        || hasNamespacePermissionForTableId(credentials, table, NamespacePermission.WRITE, true);
   }
-  
+
   public boolean canConditionallyUpdate(TCredentials credentials, String tableID, List<ByteBuffer> authorizations) throws ThriftSecurityException {
-    
+
     authenticate(credentials);
-    
-    return (hasTablePermission(credentials, tableID, TablePermission.WRITE, true) || hasTableNamespacePermissionForTableId(credentials, tableID,
-        TableNamespacePermission.WRITE, true))
-        && (hasTablePermission(credentials, tableID, TablePermission.READ, true) || hasTableNamespacePermissionForTableId(credentials, tableID,
-            TableNamespacePermission.READ, true));
+
+    return (hasTablePermission(credentials, tableID, TablePermission.WRITE, true) || hasNamespacePermissionForTableId(credentials, tableID,
+        NamespacePermission.WRITE, true))
+        && (hasTablePermission(credentials, tableID, TablePermission.READ, true) || hasNamespacePermissionForTableId(credentials, tableID,
+            NamespacePermission.READ, true));
   }
-  
+
   public boolean canSplitTablet(TCredentials credentials, String table) throws ThriftSecurityException {
     authenticate(credentials);
     return hasSystemPermission(credentials, SystemPermission.ALTER_TABLE, false) || hasSystemPermission(credentials, SystemPermission.SYSTEM, false)
         || hasTablePermission(credentials, table, TablePermission.ALTER_TABLE, false)
-        || hasTableNamespacePermissionForTableId(credentials, table, TableNamespacePermission.ALTER_TABLE, false);
+        || hasNamespacePermissionForTableId(credentials, table, NamespacePermission.ALTER_TABLE, false);
   }
-  
+
   /**
    * This is the check to perform any system action. This includes tserver's loading of a tablet, shutting the system down, or altering system properties.
    */
@@ -436,109 +435,107 @@ public class SecurityOperation {
     authenticate(credentials);
     return hasSystemPermission(credentials, SystemPermission.SYSTEM, false);
   }
-  
+
   public boolean canFlush(TCredentials c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasTablePermission(c, tableId, TablePermission.WRITE, false) || hasTablePermission(c, tableId, TablePermission.ALTER_TABLE, false)
-        || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.ALTER_TABLE, false)
-        || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.WRITE, false);
+        || hasNamespacePermissionForTableId(c, tableId, NamespacePermission.ALTER_TABLE, false)
+        || hasNamespacePermissionForTableId(c, tableId, NamespacePermission.WRITE, false);
   }
-  
+
   public boolean canAlterTable(TCredentials c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasTablePermission(c, tableId, TablePermission.ALTER_TABLE, false) || hasSystemPermission(c, SystemPermission.ALTER_TABLE, false)
-        || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.ALTER_TABLE, false);
+        || hasNamespacePermissionForTableId(c, tableId, NamespacePermission.ALTER_TABLE, false);
   }
-  
+
   public boolean canCreateTable(TCredentials c, String tableName) throws ThriftSecurityException {
     authenticate(c);
-    return hasTableNamespacePermissionForTableName(c, tableName, TableNamespacePermission.CREATE_TABLE, false) || canCreateTable(c);
+    return hasNamespacePermissionForTableName(c, tableName, NamespacePermission.CREATE_TABLE, false) || canCreateTable(c);
   }
-  
+
   public boolean canCreateTable(TCredentials c) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c, SystemPermission.CREATE_TABLE, false);
   }
-  
+
   public boolean canRenameTable(TCredentials c, String tableId, String oldTableName, String newTableName) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c, SystemPermission.ALTER_TABLE, false) || hasTablePermission(c, tableId, TablePermission.ALTER_TABLE, false)
-        || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.ALTER_TABLE, false);
+        || hasNamespacePermissionForTableId(c, tableId, NamespacePermission.ALTER_TABLE, false);
   }
-  
+
   public boolean canCloneTable(TCredentials c, String tableId, String tableName) throws ThriftSecurityException {
     authenticate(c);
-    return (hasSystemPermission(c, SystemPermission.CREATE_TABLE, false) || hasTableNamespacePermissionForTableName(c, tableName,
-        TableNamespacePermission.CREATE_TABLE, false))
-        && (hasTablePermission(c, tableId, TablePermission.READ, false) || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.READ,
-            false));
+    return (hasSystemPermission(c, SystemPermission.CREATE_TABLE, false) || hasNamespacePermissionForTableName(c, tableName, NamespacePermission.CREATE_TABLE,
+        false))
+        && (hasTablePermission(c, tableId, TablePermission.READ, false) || hasNamespacePermissionForTableId(c, tableId, NamespacePermission.READ, false));
   }
-  
+
   public boolean canDeleteTable(TCredentials c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c, SystemPermission.DROP_TABLE, false) || hasTablePermission(c, tableId, TablePermission.DROP_TABLE, false)
-        || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.DROP_TABLE, false);
+        || hasNamespacePermissionForTableId(c, tableId, NamespacePermission.DROP_TABLE, false);
   }
-  
+
   public boolean canOnlineOfflineTable(TCredentials c, String tableId, TableOperation op) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c, SystemPermission.SYSTEM, false) || hasSystemPermission(c, SystemPermission.ALTER_TABLE, false)
         || hasTablePermission(c, tableId, TablePermission.ALTER_TABLE, false)
-        || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.ALTER_TABLE, false);
+        || hasNamespacePermissionForTableId(c, tableId, NamespacePermission.ALTER_TABLE, false);
   }
-  
+
   public boolean canMerge(TCredentials c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c, SystemPermission.SYSTEM, false) || hasSystemPermission(c, SystemPermission.ALTER_TABLE, false)
         || hasTablePermission(c, tableId, TablePermission.ALTER_TABLE, false)
-        || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.ALTER_TABLE, false);
+        || hasNamespacePermissionForTableId(c, tableId, NamespacePermission.ALTER_TABLE, false);
   }
-  
+
   public boolean canDeleteRange(TCredentials c, String tableId, String tableName, Text startRow, Text endRow) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c, SystemPermission.SYSTEM, false) || hasTablePermission(c, tableId, TablePermission.WRITE, false)
-        || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.WRITE, false);
+        || hasNamespacePermissionForTableId(c, tableId, NamespacePermission.WRITE, false);
   }
-  
+
   public boolean canBulkImport(TCredentials c, String tableId, String tableName, String dir, String failDir) throws ThriftSecurityException {
     return canBulkImport(c, tableId);
   }
-  
+
   public boolean canBulkImport(TCredentials c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasTablePermission(c, tableId, TablePermission.BULK_IMPORT, false);
   }
-  
+
   public boolean canCompact(TCredentials c, String tableId) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c, SystemPermission.ALTER_TABLE, false) || hasTablePermission(c, tableId, TablePermission.ALTER_TABLE, false)
-        || hasTablePermission(c, tableId, TablePermission.WRITE, false)
-        || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.ALTER_TABLE, false)
-        || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.WRITE, false);
+        || hasTablePermission(c, tableId, TablePermission.WRITE, false) || hasNamespacePermissionForTableId(c, tableId, NamespacePermission.ALTER_TABLE, false)
+        || hasNamespacePermissionForTableId(c, tableId, NamespacePermission.WRITE, false);
   }
-  
+
   public boolean canChangeAuthorizations(TCredentials c, String user) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c, SystemPermission.ALTER_USER, false);
   }
-  
+
   public boolean canChangePassword(TCredentials c, String user) throws ThriftSecurityException {
     authenticate(c);
     return c.getPrincipal().equals(user) || hasSystemPermission(c, SystemPermission.ALTER_USER, false);
   }
-  
+
   public boolean canCreateUser(TCredentials c, String user) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c, SystemPermission.CREATE_USER, false);
   }
-  
+
   public boolean canDropUser(TCredentials c, String user) throws ThriftSecurityException {
     authenticate(c);
     if (user.equals(getRootUsername()))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     return hasSystemPermission(c, SystemPermission.DROP_USER, false);
   }
-  
+
   public boolean canGrantSystem(TCredentials c, String user, SystemPermission sysPerm) throws ThriftSecurityException {
     authenticate(c);
     // can't grant GRANT
@@ -546,49 +543,48 @@ public class SecurityOperation {
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.GRANT_INVALID);
     return hasSystemPermission(c, SystemPermission.GRANT, false);
   }
-  
+
   public boolean canGrantTable(TCredentials c, String user, String table) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c, SystemPermission.ALTER_TABLE, false) || hasTablePermission(c, table, TablePermission.GRANT, false)
-        || hasTableNamespacePermissionForTableId(c, table, TableNamespacePermission.ALTER_TABLE, false);
+        || hasNamespacePermissionForTableId(c, table, NamespacePermission.ALTER_TABLE, false);
   }
-  
-  public boolean canGrantTableNamespace(TCredentials c, String user, String tableNamespace) throws ThriftSecurityException {
+
+  public boolean canGrantNamespace(TCredentials c, String user, String namespace) throws ThriftSecurityException {
     authenticate(c);
-    return hasSystemPermission(c, SystemPermission.ALTER_NAMESPACE, false)
-        || hasTableNamespacePermission(c, tableNamespace, TableNamespacePermission.GRANT, false);
+    return hasSystemPermission(c, SystemPermission.ALTER_NAMESPACE, false) || hasNamespacePermission(c, namespace, NamespacePermission.GRANT, false);
   }
-  
+
   public boolean canRevokeSystem(TCredentials c, String user, SystemPermission sysPerm) throws ThriftSecurityException {
     authenticate(c);
     // can't modify root user
     if (user.equals(getRootUsername()))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-    
+
     // can't revoke GRANT
     if (sysPerm.equals(SystemPermission.GRANT))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.GRANT_INVALID);
-    
+
     return hasSystemPermission(c, SystemPermission.GRANT, false);
   }
-  
+
   public boolean canRevokeTable(TCredentials c, String user, String table) throws ThriftSecurityException {
     authenticate(c);
     return hasSystemPermission(c, SystemPermission.ALTER_TABLE, false) || hasTablePermission(c, table, TablePermission.GRANT, false)
-        || hasTableNamespacePermissionForTableId(c, table, TableNamespacePermission.ALTER_TABLE, false);
+        || hasNamespacePermissionForTableId(c, table, NamespacePermission.ALTER_TABLE, false);
   }
-  
-  public boolean canRevokeTableNamespace(TCredentials c, String user, String tableNamespace) throws ThriftSecurityException {
+
+  public boolean canRevokeNamespace(TCredentials c, String user, String namespace) throws ThriftSecurityException {
     authenticate(c);
-    return hasSystemPermission(c, SystemPermission.ALTER_NAMESPACE, false) || hasTableNamespacePermission(c, tableNamespace, TableNamespacePermission.GRANT, false);
+    return hasSystemPermission(c, SystemPermission.ALTER_NAMESPACE, false) || hasNamespacePermission(c, namespace, NamespacePermission.GRANT, false);
   }
-  
+
   public void changeAuthorizations(TCredentials credentials, String user, Authorizations authorizations) throws ThriftSecurityException {
     if (!canChangeAuthorizations(credentials, user))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-    
+
     targetUserExists(user);
-    
+
     try {
       authorizor.changeAuthorizations(user, authorizations);
       log.info("Changed authorizations for user " + user + " at the request of user " + credentials.getPrincipal());
@@ -596,7 +592,7 @@ public class SecurityOperation {
       throw ase.asThriftException();
     }
   }
-  
+
   public void changePassword(TCredentials credentials, Credentials toChange) throws ThriftSecurityException {
     if (!canChangePassword(credentials, toChange.getPrincipal()))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
@@ -608,7 +604,7 @@ public class SecurityOperation {
       throw e.asThriftException();
     }
   }
-  
+
   public void createUser(TCredentials credentials, Credentials newUser, Authorizations authorizations) throws ThriftSecurityException {
     if (!canCreateUser(credentials, newUser.getPrincipal()))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
@@ -624,7 +620,7 @@ public class SecurityOperation {
       throw ase.asThriftException();
     }
   }
-  
+
   public void dropUser(TCredentials credentials, String user) throws ThriftSecurityException {
     if (!canDropUser(credentials, user))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
@@ -637,13 +633,13 @@ public class SecurityOperation {
       throw e.asThriftException();
     }
   }
-  
+
   public void grantSystemPermission(TCredentials credentials, String user, SystemPermission permissionById) throws ThriftSecurityException {
     if (!canGrantSystem(credentials, user, permissionById))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-    
+
     targetUserExists(user);
-    
+
     try {
       permHandle.grantSystemPermission(user, permissionById);
       log.info("Granted system permission " + permissionById + " for user " + user + " at the request of user " + credentials.getPrincipal());
@@ -651,13 +647,13 @@ public class SecurityOperation {
       throw e.asThriftException();
     }
   }
-  
+
   public void grantTablePermission(TCredentials c, String user, String tableId, TablePermission permission) throws ThriftSecurityException {
     if (!canGrantTable(c, user, tableId))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-    
+
     targetUserExists(user);
-    
+
     try {
       permHandle.grantTablePermission(user, tableId, permission);
       log.info("Granted table permission " + permission + " for user " + user + " on the table " + tableId + " at the request of user " + c.getPrincipal());
@@ -667,95 +663,93 @@ public class SecurityOperation {
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.TABLE_DOESNT_EXIST);
     }
   }
-  
-  public void grantTableNamespacePermission(TCredentials c, String user, String tableNamespace, TableNamespacePermission permission)
-      throws ThriftSecurityException {
-    if (!canGrantTableNamespace(c, user, tableNamespace))
+
+  public void grantNamespacePermission(TCredentials c, String user, String namespace, NamespacePermission permission) throws ThriftSecurityException {
+    if (!canGrantNamespace(c, user, namespace))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-    
+
     targetUserExists(user);
-    
+
     try {
-      permHandle.grantTableNamespacePermission(user, tableNamespace, permission);
-      log.info("Granted table namespace permission " + permission + " for user " + user + " on the table namespace " + tableNamespace
-          + " at the request of user " + c.getPrincipal());
+      permHandle.grantNamespacePermission(user, namespace, permission);
+      log.info("Granted namespace permission " + permission + " for user " + user + " on the namespace " + namespace + " at the request of user "
+          + c.getPrincipal());
     } catch (AccumuloSecurityException e) {
       throw e.asThriftException();
-    } catch (TableNamespaceNotFoundException e) {
-      throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.TABLE_NAMESPACE_DOESNT_EXIST);
+    } catch (NamespaceNotFoundException e) {
+      throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
     }
   }
-  
+
   public void revokeSystemPermission(TCredentials credentials, String user, SystemPermission permission) throws ThriftSecurityException {
     if (!canRevokeSystem(credentials, user, permission))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-    
+
     targetUserExists(user);
-    
+
     try {
       permHandle.revokeSystemPermission(user, permission);
       log.info("Revoked system permission " + permission + " for user " + user + " at the request of user " + credentials.getPrincipal());
-      
+
     } catch (AccumuloSecurityException e) {
       throw e.asThriftException();
     }
   }
-  
+
   public void revokeTablePermission(TCredentials c, String user, String tableId, TablePermission permission) throws ThriftSecurityException {
     if (!canRevokeTable(c, user, tableId))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-    
+
     targetUserExists(user);
-    
+
     try {
       permHandle.revokeTablePermission(user, tableId, permission);
       log.info("Revoked table permission " + permission + " for user " + user + " on the table " + tableId + " at the request of user " + c.getPrincipal());
-      
+
     } catch (AccumuloSecurityException e) {
       throw e.asThriftException();
     } catch (TableNotFoundException e) {
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.TABLE_DOESNT_EXIST);
     }
   }
-  
-  public void revokeTableNamespacePermission(TCredentials c, String user, String tableNamespace, TableNamespacePermission permission)
-      throws ThriftSecurityException {
-    if (!canRevokeTableNamespace(c, user, tableNamespace))
+
+  public void revokeNamespacePermission(TCredentials c, String user, String namespace, NamespacePermission permission) throws ThriftSecurityException {
+    if (!canRevokeNamespace(c, user, namespace))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-    
+
     targetUserExists(user);
-    
+
     try {
-      permHandle.revokeTableNamespacePermission(user, tableNamespace, permission);
-      log.info("Revoked table namespace permission " + permission + " for user " + user + " on the table namespace " + tableNamespace
-          + " at the request of user " + c.getPrincipal());
-      
+      permHandle.revokeNamespacePermission(user, namespace, permission);
+      log.info("Revoked namespace permission " + permission + " for user " + user + " on the namespace " + namespace + " at the request of user "
+          + c.getPrincipal());
+
     } catch (AccumuloSecurityException e) {
       throw e.asThriftException();
-    } catch (TableNamespaceNotFoundException e) {
-      throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.TABLE_NAMESPACE_DOESNT_EXIST);
+    } catch (NamespaceNotFoundException e) {
+      throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
     }
   }
-  
+
   public boolean hasSystemPermission(TCredentials credentials, String user, SystemPermission permissionById) throws ThriftSecurityException {
     if (!canAskAboutOtherUsers(credentials, user))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     return _hasSystemPermission(user, permissionById, false);
   }
-  
+
   public boolean hasTablePermission(TCredentials credentials, String user, String tableId, TablePermission permissionById) throws ThriftSecurityException {
     if (!canAskAboutOtherUsers(credentials, user))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     return _hasTablePermission(user, tableId, permissionById, false);
   }
-  
-  public boolean hasTableNamespacePermission(TCredentials credentials, String user, String tableNamespace, TableNamespacePermission permissionById)
+
+  public boolean hasNamespacePermission(TCredentials credentials, String user, String namespace, NamespacePermission permissionById)
       throws ThriftSecurityException {
     if (!canAskAboutOtherUsers(credentials, user))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-    return _hasTableNamespacePermission(user, tableNamespace, permissionById, false);
+    return _hasNamespacePermission(user, namespace, permissionById, false);
   }
-  
+
   public Set<String> listUsers(TCredentials credentials) throws ThriftSecurityException {
     authenticate(credentials);
     try {
@@ -764,7 +758,7 @@ public class SecurityOperation {
       throw e.asThriftException();
     }
   }
-  
+
   public void deleteTable(TCredentials credentials, String tableId) throws ThriftSecurityException {
     if (!canDeleteTable(credentials, tableId))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
@@ -777,62 +771,62 @@ public class SecurityOperation {
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.TABLE_DOESNT_EXIST);
     }
   }
-  
-  public void deleteTableNamespace(TCredentials credentials, String tableNamespace) throws ThriftSecurityException {
-    if (!canDeleteNamespace(credentials, tableNamespace))
+
+  public void deleteNamespace(TCredentials credentials, String namespace) throws ThriftSecurityException {
+    if (!canDeleteNamespace(credentials, namespace))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     try {
-      permHandle.cleanTableNamespacePermissions(tableNamespace);
+      permHandle.cleanNamespacePermissions(namespace);
     } catch (AccumuloSecurityException e) {
       e.setUser(credentials.getPrincipal());
       throw e.asThriftException();
-    } catch (TableNamespaceNotFoundException e) {
-      throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.TABLE_NAMESPACE_DOESNT_EXIST);
+    } catch (NamespaceNotFoundException e) {
+      throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
     }
   }
-  
+
   public boolean canExport(TCredentials credentials, String tableId, String tableName, String exportDir) throws ThriftSecurityException {
     authenticate(credentials);
     return hasTablePermission(credentials, tableId, TablePermission.READ, false)
-        || hasTableNamespacePermissionForTableId(credentials, tableId, TableNamespacePermission.READ, false);
+        || hasNamespacePermissionForTableId(credentials, tableId, NamespacePermission.READ, false);
   }
-  
+
   public boolean canImport(TCredentials credentials, String tableName, String importDir) throws ThriftSecurityException {
     authenticate(credentials);
     return hasSystemPermission(credentials, SystemPermission.CREATE_TABLE, false)
-        || hasTableNamespacePermissionForTableName(credentials, tableName, TableNamespacePermission.CREATE_TABLE, false);
+        || hasNamespacePermissionForTableName(credentials, tableName, NamespacePermission.CREATE_TABLE, false);
   }
-  
+
   public boolean canAlterNamespace(TCredentials credentials, String namespaceId) throws ThriftSecurityException {
     authenticate(credentials);
-    return hasTableNamespacePermission(credentials, namespaceId, TableNamespacePermission.ALTER_NAMESPACE, false)
+    return hasNamespacePermission(credentials, namespaceId, NamespacePermission.ALTER_NAMESPACE, false)
         || hasSystemPermission(credentials, SystemPermission.ALTER_NAMESPACE, false);
   }
-  
+
   public boolean canCreateNamespace(TCredentials credentials, String namespace) throws ThriftSecurityException {
     authenticate(credentials);
     return canCreateNamespace(credentials);
   }
-  
+
   public boolean canCreateNamespace(TCredentials credentials) throws ThriftSecurityException {
     authenticate(credentials);
     return hasSystemPermission(credentials, SystemPermission.CREATE_NAMESPACE, false);
   }
-  
+
   public boolean canDeleteNamespace(TCredentials credentials, String namespaceId) throws ThriftSecurityException {
     authenticate(credentials);
     return hasSystemPermission(credentials, SystemPermission.DROP_NAMESPACE, false);
   }
-  
+
   public boolean canRenameNamespace(TCredentials credentials, String namespaceId, String oldName, String newName) throws ThriftSecurityException {
     authenticate(credentials);
-    return hasTableNamespacePermission(credentials, namespaceId, TableNamespacePermission.ALTER_NAMESPACE, false)
+    return hasNamespacePermission(credentials, namespaceId, NamespacePermission.ALTER_NAMESPACE, false)
         || hasSystemPermission(credentials, SystemPermission.ALTER_NAMESPACE, false);
   }
-  
+
   public boolean canCloneNamespace(TCredentials credentials, String namespaceId, String namespace) throws ThriftSecurityException {
     authenticate(credentials);
-    return hasTableNamespacePermission(credentials, namespaceId, TableNamespacePermission.READ, false)
+    return hasNamespacePermission(credentials, namespaceId, NamespacePermission.READ, false)
         && hasSystemPermission(credentials, SystemPermission.CREATE_NAMESPACE, false);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/base/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java
index f1d69e8..99c3bbe 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/InsecurePermHandler.java
@@ -17,10 +17,10 @@
 package org.apache.accumulo.server.security.handler;
 
 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.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.accumulo.core.security.thrift.TCredentials;
 
@@ -103,31 +103,31 @@ public class InsecurePermHandler implements PermissionHandler {
   public void initTable(String table) throws AccumuloSecurityException {}
 
   @Override
-  public boolean hasTableNamespacePermission(String user, String namespace, TableNamespacePermission permission) throws AccumuloSecurityException,
-      TableNamespaceNotFoundException {
+  public boolean hasNamespacePermission(String user, String namespace, NamespacePermission permission) throws AccumuloSecurityException,
+      NamespaceNotFoundException {
     return true;
   }
 
   @Override
-  public boolean hasCachedTableNamespacePermission(String user, String namespace, TableNamespacePermission permission) throws AccumuloSecurityException,
-      TableNamespaceNotFoundException {
+  public boolean hasCachedNamespacePermission(String user, String namespace, NamespacePermission permission) throws AccumuloSecurityException,
+      NamespaceNotFoundException {
     return true;
   }
 
   @Override
-  public void grantTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws AccumuloSecurityException,
-      TableNamespaceNotFoundException {
+  public void grantNamespacePermission(String user, String namespace, NamespacePermission permission) throws AccumuloSecurityException,
+      NamespaceNotFoundException {
     return;
   }
 
   @Override
-  public void revokeTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws AccumuloSecurityException,
-      TableNamespaceNotFoundException {
+  public void revokeNamespacePermission(String user, String namespace, NamespacePermission permission) throws AccumuloSecurityException,
+      NamespaceNotFoundException {
     return;
   }
 
   @Override
-  public void cleanTableNamespacePermissions(String tableNamespace) throws AccumuloSecurityException, TableNamespaceNotFoundException {
+  public void cleanNamespacePermissions(String namespace) throws AccumuloSecurityException, NamespaceNotFoundException {
     return;
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/base/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java
index 63e7208..b93ff1e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/PermissionHandler.java
@@ -17,11 +17,11 @@
 package org.apache.accumulo.server.security.handler;
 
 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.ThriftSecurityException;
+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;
 
@@ -30,104 +30,107 @@ import org.apache.accumulo.core.security.thrift.TCredentials;
  * Accumulo, it should throw an AccumuloSecurityException with the error code UNSUPPORTED_OPERATION
  */
 public interface PermissionHandler {
-  
+
   /**
    * Sets up the permission handler for a new instance of Accumulo
    */
   public void initialize(String instanceId, boolean initialize);
-  
+
   /**
    * Used to validate that the Authorizor, Authenticator, and permission handler can coexist
    */
   public boolean validSecurityHandlers(Authenticator authent, Authorizor author);
-  
+
   /**
    * Used to initialize security for the root user
    */
   public void initializeSecurity(TCredentials credentials, String rootuser) throws AccumuloSecurityException, ThriftSecurityException;
-  
+
   /**
    * Used to get the system permission for the user
    */
   public boolean hasSystemPermission(String user, SystemPermission permission) throws AccumuloSecurityException;
-  
+
   /**
    * Used to get the system permission for the user, with caching due to high frequency operation. NOTE: At this time, this method is unused but is included
    * just in case we need it in the future.
    */
   public boolean hasCachedSystemPermission(String user, SystemPermission permission) throws AccumuloSecurityException;
-  
+
   /**
    * Used to get the table permission of a user for a table
    */
   public boolean hasTablePermission(String user, String table, TablePermission permission) throws AccumuloSecurityException, TableNotFoundException;
-  
+
   /**
    * Used to get the table permission of a user for a table, with caching. This method is for high frequency operations
    */
   public boolean hasCachedTablePermission(String user, String table, TablePermission permission) throws AccumuloSecurityException, TableNotFoundException;
-  
+
   /**
-   * Used to get the table namespace permission of a user for a table namespace
+   * Used to get the namespace permission of a user for a namespace
    */
-  public boolean hasTableNamespacePermission(String user, String namespace, TableNamespacePermission permission) throws AccumuloSecurityException, TableNamespaceNotFoundException;
-  
+  public boolean hasNamespacePermission(String user, String namespace, NamespacePermission permission) throws AccumuloSecurityException,
+      NamespaceNotFoundException;
+
   /**
-   * Used to get the table namespace permission of a user for a table namespace, with caching. This method is for high frequency operations
+   * Used to get the namespace permission of a user for a namespace, with caching. This method is for high frequency operations
    */
-  public boolean hasCachedTableNamespacePermission(String user, String namespace, TableNamespacePermission permission) throws AccumuloSecurityException, TableNamespaceNotFoundException;
-  
+  public boolean hasCachedNamespacePermission(String user, String namespace, NamespacePermission permission) throws AccumuloSecurityException,
+      NamespaceNotFoundException;
+
   /**
    * Gives the user the given system permission
    */
   public void grantSystemPermission(String user, SystemPermission permission) throws AccumuloSecurityException;
-  
+
   /**
    * Denies the user the given system permission
    */
   public void revokeSystemPermission(String user, SystemPermission permission) throws AccumuloSecurityException;
-  
+
   /**
    * Gives the user the given table permission
    */
   public void grantTablePermission(String user, String table, TablePermission permission) throws AccumuloSecurityException, TableNotFoundException;
-  
+
   /**
    * Denies the user the given table permission.
    */
   public void revokeTablePermission(String user, String table, TablePermission permission) throws AccumuloSecurityException, TableNotFoundException;
-  
+
   /**
-   * Gives the user the given table namespace permission
+   * Gives the user the given namespace permission
    */
-  public void grantTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws AccumuloSecurityException, TableNamespaceNotFoundException;
-  
+  public void grantNamespacePermission(String user, String namespace, NamespacePermission permission) throws AccumuloSecurityException,
+      NamespaceNotFoundException;
+
   /**
-   * Denies the user the given table namespace permission.
+   * Denies the user the given namespace permission.
    */
-  public void revokeTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws AccumuloSecurityException, TableNamespaceNotFoundException;
-  
-  
+  public void revokeNamespacePermission(String user, String namespace, NamespacePermission permission) throws AccumuloSecurityException,
+      NamespaceNotFoundException;
+
   /**
    * Cleans up the permissions for a table. Used when a table gets deleted.
    */
   public void cleanTablePermissions(String table) throws AccumuloSecurityException, TableNotFoundException;
-  
+
   /**
-   * Cleans up the permissions for a table namespace. Used when a table namespace gets deleted.
+   * Cleans up the permissions for a namespace. Used when a namespace gets deleted.
    */
-  public void cleanTableNamespacePermissions(String tableNamespace) throws AccumuloSecurityException, TableNamespaceNotFoundException;
-  
+  public void cleanNamespacePermissions(String namespace) throws AccumuloSecurityException, NamespaceNotFoundException;
+
   /**
    * Initializes a new user
    */
   public void initUser(String user) throws AccumuloSecurityException;
-  
+
   /**
    * Initializes a new user
    */
   public void initTable(String table) throws AccumuloSecurityException;
-  
+
   /**
    * Deletes a user
    */