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:58:42 UTC

[01/50] [abbrv] ACCUMULO-802 updated more shell commands to include the tableNamespaces option

Updated Branches:
  refs/heads/master c29df8ef0 -> 7b8922c70


http://git-wip-us.apache.org/repos/asf/accumulo/blob/5acd6a48/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
new file mode 100644
index 0000000..8bb33b8
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfWatcher.java
@@ -0,0 +1,107 @@
+/*
+ * 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/5acd6a48/server/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfWatcher.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfWatcher.java b/server/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfWatcher.java
deleted file mode 100644
index 8bb33b8..0000000
--- a/server/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/5acd6a48/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
index daa1f52..257881d 100644
--- a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
@@ -869,6 +869,10 @@ public class ShellServerIT extends SimpleMacIT {
     exec("tables", true, "testers3.1", false);
     exec("namespaces", true, "testers3", true);
     exec("deletenamespace testers3 -f", true);
+    input.set("true\n\n\nSTRING\n");
+    exec("setiter -tn thing2 -scan -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -n name", true);
+    exec("listiter -tn thing2 -scan", true, "Summing", true);
+    exec("deleteiter -tn thing2 -n name -scan", true);
     
     // properties override and such
     exec("config -tn thing2 -s table.file.max=44444", true);


[39/50] [abbrv] git commit: ACCUMULO-1712 Rename system tables and namespaces

Posted by ct...@apache.org.
ACCUMULO-1712 Rename system tables and namespaces

  Avoid exclamation point in names
  Adjust shell to be more useful with tables and namespaces listings
  Update documentation about metadata tables


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

Branch: refs/heads/master
Commit: 8b54ced2777beb81fce04aa8300e192eb4534a36
Parents: cd6e185
Author: Christopher Tubbs <ct...@apache.org>
Authored: Tue Dec 3 23:48:23 2013 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:11 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/core/Constants.java     |  10 +-
 .../client/admin/NamespaceOperationsImpl.java   |   2 +-
 .../core/client/admin/TableOperationsImpl.java  |  19 +-
 .../accumulo/core/client/impl/Namespaces.java   |   2 +-
 .../accumulo/core/client/impl/Tables.java       |  46 ++--
 .../client/impl/TabletServerBatchWriter.java    |   2 +-
 .../accumulo/core/client/mock/MockAccumulo.java |  38 +--
 .../client/mock/MockNamespaceOperations.java    |   2 +-
 .../core/client/mock/MockTableOperations.java   |   6 +-
 .../accumulo/core/metadata/MetadataTable.java   |   9 +-
 .../accumulo/core/metadata/RootTable.java       |  15 +-
 .../org/apache/accumulo/core/util/Merge.java    |   2 +-
 .../org/apache/accumulo/core/util/Pair.java     |  57 +++-
 .../apache/accumulo/core/util/shell/Shell.java  |  22 +-
 .../util/shell/commands/CreateTableCommand.java |   8 +-
 .../shell/commands/DeleteNamespaceCommand.java  |   6 -
 .../util/shell/commands/DeleteTableCommand.java |  22 +-
 .../util/shell/commands/NamespacesCommand.java  |  49 ++--
 .../util/shell/commands/RenameTableCommand.java |  19 +-
 .../core/util/shell/commands/TablesCommand.java |  81 +++---
 .../core/client/impl/TabletLocatorImplTest.java |  10 +-
 .../accumulo/core/util/shell/ShellTest.java     |  54 ++--
 .../chapters/administration.tex                 |   2 +-
 .../accumulo_user_manual/chapters/shell.tex     |   8 +-
 .../chapters/table_configuration.tex            |   2 +-
 .../chapters/troubleshooting.tex                |  30 ++-
 .../apache/accumulo/proxy/SimpleProxyIT.java    |   2 +-
 .../server/conf/NamespaceConfiguration.java     |   4 +-
 .../org/apache/accumulo/server/fs/FileRef.java  |   5 +-
 .../accumulo/server/fs/VolumeManager.java       |   2 +-
 .../apache/accumulo/server/init/Initialize.java |   6 +-
 .../master/state/MetaDataTableScanner.java      |   2 +-
 .../server/security/SecurityOperation.java      |   4 +-
 .../server/security/handler/ZKAuthorizor.java   |  41 +--
 .../server/security/handler/ZKPermHandler.java  |   4 +-
 .../accumulo/server/tables/TableManager.java    |  88 ++++---
 .../accumulo/server/util/MetadataTableUtil.java |   4 +-
 .../accumulo/gc/SimpleGarbageCollector.java     |   8 +-
 .../java/org/apache/accumulo/master/Master.java | 223 ++++++++--------
 .../accumulo/master/tableOps/BulkImport.java    |   2 +-
 .../accumulo/master/tableOps/CloneTable.java    |  94 +++----
 .../accumulo/master/tableOps/CreateTable.java   | 122 ++++-----
 .../accumulo/master/tableOps/ImportTable.java   | 261 ++++++++++---------
 .../accumulo/master/tableOps/RenameTable.java   |  10 +-
 .../apache/accumulo/master/TestMergeState.java  |  52 ++--
 .../src/main/resources/docs/bulkIngest.html     |   2 +-
 .../main/resources/docs/examples/README.bloom   |   9 +-
 .../main/resources/docs/examples/README.export  |   9 +-
 .../resources/docs/examples/README.visibility   |   4 +-
 .../org/apache/accumulo/tserver/Tablet.java     |  14 +-
 .../apache/accumulo/tserver/TabletServer.java   |   4 +-
 test/scale/deleteLargeTable.txt                 |   2 +-
 .../accumulo/test/functional/ZombieTServer.java |   2 +-
 .../metadata/MetadataBatchScanTest.java         |   2 +-
 .../test/performance/thrift/NullTserver.java    |   4 +-
 .../org/apache/accumulo/test/DumpConfigIT.java  |   6 +-
 .../org/apache/accumulo/test/NamespacesIT.java  |   4 +-
 .../org/apache/accumulo/test/ShellServerIT.java |  26 +-
 .../apache/accumulo/test/SplitRecoveryIT.java   |   2 +-
 .../test/functional/GarbageCollectorIT.java     |   4 +-
 test/system/bench/cloudstone1/cloudstone1.py    |   4 +-
 61 files changed, 782 insertions(+), 773 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/main/java/org/apache/accumulo/core/Constants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/Constants.java b/core/src/main/java/org/apache/accumulo/core/Constants.java
index 2909c3d..4ef9906 100644
--- a/core/src/main/java/org/apache/accumulo/core/Constants.java
+++ b/core/src/main/java/org/apache/accumulo/core/Constants.java
@@ -112,9 +112,9 @@ public class Constants {
   public static final String EXPORT_INFO_FILE = "accumulo_export_info.txt";
 
   // Variables that will be substituted with environment vars in PropertyType.PATH values
-  public static final String[] PATH_PROPERTY_ENV_VARS = new String[]{"$ACCUMULO_HOME", "$ACCUMULO_CONF_DIR"};
-  public static final String DEFAULT_NAMESPACE = "_default_";
-  public static final String SYSTEM_NAMESPACE = "_system_";
-  public static final String DEFAULT_NAMESPACE_ID = "!DEF";
-  public static final String SYSTEM_NAMESPACE_ID = "!SYS";
+  public static final String[] PATH_PROPERTY_ENV_VARS = new String[] {"$ACCUMULO_HOME", "$ACCUMULO_CONF_DIR"};
+  public static final String DEFAULT_NAMESPACE = "";
+  public static final String ACCUMULO_NAMESPACE = "accumulo";
+  public static final String DEFAULT_NAMESPACE_ID = "+default";
+  public static final String ACCUMULO_NAMESPACE_ID = "+accumulo";
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
index 6c877f8..1ddcaf7 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
@@ -220,7 +220,7 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
     ArgumentChecker.notNull(namespace);
     String namespaceId = Namespaces.getNamespaceId(instance, namespace);
 
-    if (namespaceId.equals(Constants.SYSTEM_NAMESPACE_ID) || namespaceId.equals(Constants.DEFAULT_NAMESPACE_ID)) {
+    if (namespaceId.equals(Constants.ACCUMULO_NAMESPACE_ID) || namespaceId.equals(Constants.DEFAULT_NAMESPACE_ID)) {
       log.debug(credentials.getPrincipal() + " attempted to delete the " + namespaceId + " namespace");
       throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.UNSUPPORTED_OPERATION);
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
index b21aa31..7f79fcd 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
@@ -220,15 +220,6 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
     Map<String,String> opts = new HashMap<String,String>();
 
-    String namespace = Tables.extractNamespace(tableName);
-    if (!namespaceExists(namespace)) {
-      String info = "Namespace not found while trying to create table";
-      throw new IllegalArgumentException(new NamespaceNotFoundException(null, namespace, info));
-    } else if (namespace.equals(Constants.SYSTEM_NAMESPACE)) {
-      String info = "Can't create tables in the system namespace";
-      throw new IllegalArgumentException(info);
-    }
-
     try {
       doTableOperation(TableOperation.CREATE, args, opts);
     } catch (TableNotFoundException e1) {
@@ -685,7 +676,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
     ArgumentChecker.notNull(srcTableName, newTableName);
 
-    String namespace = Tables.extractNamespace(newTableName);
+    String namespace = Tables.qualify(newTableName).getFirst();
     if (!namespaceExists(namespace)) {
       String info = "Namespace not found while cloning table";
       throw new IllegalArgumentException(new NamespaceNotFoundException(null, namespace, info));
@@ -773,12 +764,6 @@ public class TableOperationsImpl extends TableOperationsHelper {
   public void rename(String oldTableName, String newTableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException,
       TableExistsException {
 
-    String namespace = Tables.extractNamespace(newTableName);
-    if (!namespaceExists(namespace)) {
-      String info = "Namespace not found while renaming table";
-      throw new IllegalArgumentException(new NamespaceNotFoundException(null, namespace, info));
-    }
-
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldTableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(newTableName.getBytes(Constants.UTF8)));
     Map<String,String> opts = new HashMap<String,String>();
     doTableOperation(TableOperation.RENAME, args, opts);
@@ -1521,7 +1506,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       Logger.getLogger(this.getClass()).warn("Failed to check if imported table references external java classes : " + ioe.getMessage());
     }
 
-    String namespace = Tables.extractNamespace(tableName);
+    String namespace = Tables.qualify(tableName).getFirst();
     if (!namespaceExists(namespace)) {
       String info = "Namespace not found while importing to table";
       throw new RuntimeException(new NamespaceNotFoundException(null, namespace, info));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
index 0b5fff6..2480f68 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
@@ -95,7 +95,7 @@ public class Namespaces {
     List<String> ids = getTableIds(instance, namespaceId);
     List<String> names = new LinkedList<String>();
     String namespace = getNamespaceName(instance, namespaceId) + ".";
-    if (namespaceId.equals(Constants.DEFAULT_NAMESPACE_ID) || namespaceId.equals(Constants.SYSTEM_NAMESPACE_ID)) {
+    if (namespaceId.equals(Constants.DEFAULT_NAMESPACE_ID) || namespaceId.equals(Constants.ACCUMULO_NAMESPACE_ID)) {
       // default and system namespaces aren't displayed for backwards compatibility
       namespace = "";
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
index 2067762..af8c617 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
@@ -29,7 +29,7 @@ import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.log4j.Logger;
@@ -58,10 +58,10 @@ public class Tables {
       byte[] tblPath = zc.get(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_NAME);
       byte[] nId = zc.get(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_NAMESPACE);
       String name = "";
-      // create fully qualified table name if it's in a namespace other than default or system.
+      // create fully qualified table name
       if (nId != null) {
         String namespaceId = new String(nId, Constants.UTF8);
-        if (!namespaceId.equals(Constants.DEFAULT_NAMESPACE_ID) && !namespaceId.equals(Constants.SYSTEM_NAMESPACE_ID)) {
+        if (!namespaceId.equals(Constants.DEFAULT_NAMESPACE_ID)) {
           try {
             name += Namespaces.getNamespaceName(instance, namespaceId) + ".";
           } catch (NamespaceNotFoundException e) {
@@ -159,25 +159,33 @@ public class Tables {
   public static long getCacheResetCount() {
     return cacheResetCount.get();
   }
-  
-  public static String extractNamespace(String tableName) {
-    String[] s = tableName.split("\\.");
-    if (tableName.equals(MetadataTable.NAME) || tableName.equals(RootTable.NAME)) {
-      return Constants.SYSTEM_NAMESPACE;
-    } else if (s.length == 2 && !s[0].isEmpty()) {
-      return s[0];
-    } else {
-      return Constants.DEFAULT_NAMESPACE;
-    }
+
+  public static String qualified(String tableName) {
+    return qualified(tableName, Constants.DEFAULT_NAMESPACE);
+  }
+
+  public static String qualified(String tableName, String defaultNamespace) {
+    Pair<String,String> qualifiedTableName = Tables.qualify(tableName, defaultNamespace);
+    if (Constants.DEFAULT_NAMESPACE.equals(qualifiedTableName.getFirst()))
+      return qualifiedTableName.getSecond();
+    else
+      return qualifiedTableName.toString("", ".", "");
+  }
+
+  public static Pair<String,String> qualify(String tableName) {
+    return qualify(tableName, Constants.DEFAULT_NAMESPACE);
   }
 
-  public static String extractTableName(String tableName) {
-    String[] s = tableName.split("\\.");
-    if (s.length == 2 && !s[1].isEmpty() && !s[0].isEmpty()) {
-      return s[1];
-    } else {
-      return tableName;
+  public static Pair<String,String> qualify(String tableName, String defaultNamespace) {
+    if (!tableName.matches(Constants.VALID_TABLE_NAME_REGEX))
+      throw new IllegalArgumentException("Improper table name format");
+    if (MetadataTable.OLD_NAME.equals(tableName))
+      tableName = MetadataTable.NAME;
+    if (tableName.contains(".")) {
+      String[] s = tableName.split("\\.", 2);
+      return new Pair<String,String>(s[0], s[1]);
     }
+    return new Pair<String,String>(defaultNamespace, tableName);
   }
 
   public static String getNamespace(Instance instance, String tableId) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
index 8a51657..dc99d5e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
@@ -668,7 +668,7 @@ public class TabletServerBatchWriter {
       } catch (AccumuloServerException ase) {
         updateServerErrors(ase.getServer(), ase);
       } catch (AccumuloException ae) {
-        // assume an IOError communicating with !METADATA tablet
+        // assume an IOError communicating with metadata tablet
         failedMutations.add(mutationsToProcess);
       } catch (AccumuloSecurityException e) {
         updateAuthorizationFailures(Collections.singletonMap(new KeyExtent(new Text(MetadataTable.ID), null, null),

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
index 497716f..1d1853e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
@@ -31,8 +31,8 @@ import org.apache.accumulo.core.data.Mutation;
 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.SystemPermission;
 import org.apache.accumulo.core.security.NamespacePermission;
+import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.Text;
@@ -43,53 +43,53 @@ public class MockAccumulo {
   final Map<String,String> systemProperties = new HashMap<String,String>();
   Map<String,MockUser> users = new HashMap<String,MockUser>();
   final FileSystem fs;
-  
+
   MockAccumulo(FileSystem fs) {
     this.fs = fs;
   }
-  
+
   {
     MockUser root = new MockUser("root", new PasswordToken(new byte[0]), Authorizations.EMPTY);
     root.permissions.add(SystemPermission.SYSTEM);
     users.put(root.name, root);
     namespaces.put(Constants.DEFAULT_NAMESPACE, new MockNamespace());
-    namespaces.put(Constants.SYSTEM_NAMESPACE, new MockNamespace());
+    namespaces.put(Constants.ACCUMULO_NAMESPACE, new MockNamespace());
     createTable("root", RootTable.NAME, true, TimeType.LOGICAL);
     createTable("root", MetadataTable.NAME, true, TimeType.LOGICAL);
   }
-  
+
   public FileSystem getFileSystem() {
     return fs;
   }
-  
+
   void setProperty(String key, String value) {
     systemProperties.put(key, value);
   }
-  
+
   String removeProperty(String key) {
     return systemProperties.remove(key);
   }
-  
+
   void createTable(String user, String table) {
     createTable(user, table, true, TimeType.MILLIS);
   }
-  
+
   public void addMutation(String table, Mutation m) {
     MockTable t = tables.get(table);
     t.addMutation(m);
   }
-  
+
   public BatchScanner createBatchScanner(String tableName, Authorizations authorizations) {
     return new MockBatchScanner(tables.get(tableName), authorizations);
   }
-  
+
   public void createTable(String username, String tableName, boolean useVersions, TimeType timeType) {
-    String namespace = Tables.extractNamespace(tableName);
-    
+    String namespace = Tables.qualify(tableName).getFirst();
+
     if (!namespaceExists(namespace)) {
       return;
     }
-    
+
     MockNamespace n = namespaces.get(namespace);
     MockTable t = new MockTable(n, useVersions, timeType);
     t.userPermissions.put(username, EnumSet.allOf(TablePermission.class));
@@ -97,7 +97,7 @@ public class MockAccumulo {
     t.setNamespace(n);
     tables.put(tableName, t);
   }
-  
+
   public void createNamespace(String username, String namespace) {
     if (!namespaceExists(namespace)) {
       MockNamespace n = new MockNamespace();
@@ -105,19 +105,19 @@ public class MockAccumulo {
       namespaces.put(namespace, n);
     }
   }
-  
+
   public void addSplits(String tableName, SortedSet<Text> partitionKeys) {
     tables.get(tableName).addSplits(partitionKeys);
   }
-  
+
   public Collection<Text> getSplits(String tableName) {
     return tables.get(tableName).getSplits();
   }
-  
+
   public void merge(String tableName, Text start, Text end) {
     tables.get(tableName).merge(start, end);
   }
-  
+
   private boolean namespaceExists(String namespace) {
     return namespaces.containsKey(namespace);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
index 5719317..eb45d2d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
@@ -81,7 +81,7 @@ public class MockNamespaceOperations extends NamespaceOperationsHelper {
 
     MockNamespace n = acu.namespaces.get(oldNamespaceName);
     for (String t : n.getTables(acu)) {
-      String tt = newNamespaceName + "." + Tables.extractTableName(t);
+      String tt = newNamespaceName + "." + Tables.qualify(t).getSecond();
       acu.tables.put(tt, acu.tables.remove(t));
     }
     acu.namespaces.put(newNamespaceName, acu.namespaces.remove(oldNamespaceName));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
index 83748d4..6bf9904 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
@@ -96,7 +96,7 @@ public class MockTableOperations extends TableOperationsHelper {
 
   @Override
   public void create(String tableName, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException {
-    String namespace = Tables.extractNamespace(tableName);
+    String namespace = Tables.qualify(tableName).getFirst();
     if (!tableName.matches(Constants.VALID_TABLE_NAME_REGEX)) {
       throw new IllegalArgumentException();
     }
@@ -155,7 +155,7 @@ public class MockTableOperations extends TableOperationsHelper {
     if (exists(newTableName))
       throw new TableExistsException(newTableName, newTableName, "");
     MockTable t = acu.tables.remove(oldTableName);
-    String namespace = Tables.extractNamespace(newTableName);
+    String namespace = Tables.qualify(newTableName).getFirst();
     MockNamespace n = acu.namespaces.get(namespace);
     if (n == null) {
       n = new MockNamespace();
@@ -182,7 +182,7 @@ public class MockTableOperations extends TableOperationsHelper {
 
   @Override
   public Iterable<Entry<String,String>> getProperties(String tableName) throws TableNotFoundException {
-    String namespace = Tables.extractNamespace(tableName);
+    String namespace = Tables.qualify(tableName).getFirst();
 
     if (!namespaceExists(namespace)) {
       throw new IllegalArgumentException("Namespace (" + namespace + ") does not exist");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java
index 7276f6c..c31cb0d 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataTable.java
@@ -16,10 +16,13 @@
  */
 package org.apache.accumulo.core.metadata;
 
+import org.apache.accumulo.core.Constants;
 
 public class MetadataTable {
-  
+
+  public static final String OLD_NAME = "!METADATA";
+
   public static final String ID = "!0";
-  public static final String NAME = "!METADATA";
-  
+  public static final String NAME = Constants.ACCUMULO_NAMESPACE + ".metadata";
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
index c3ba6b2..527f242 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.metadata;
 
+import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.hadoop.io.Text;
 
@@ -23,15 +24,15 @@ import org.apache.hadoop.io.Text;
  * 
  */
 public class RootTable {
-  
-  public static final String ID = "!!R";
-  public static final String NAME = "!!ROOT";
-  
+
+  public static final String ID = "+r";
+  public static final String NAME = Constants.ACCUMULO_NAMESPACE + ".root";
+
   /**
    * DFS location relative to the Accumulo directory
    */
   public static final String ROOT_TABLET_LOCATION = "/root_tablet";
-  
+
   /**
    * ZK path relative to the instance directory for information about the root tablet
    */
@@ -41,8 +42,8 @@ public class RootTable {
   public static final String ZROOT_TABLET_LAST_LOCATION = ZROOT_TABLET + "/lastlocation";
   public static final String ZROOT_TABLET_WALOGS = ZROOT_TABLET + "/walogs";
   public static final String ZROOT_TABLET_PATH = ZROOT_TABLET + "/dir";
-  
+
   public static final KeyExtent EXTENT = new KeyExtent(new Text(ID), null, null);
   public static final KeyExtent OLD_EXTENT = new KeyExtent(new Text(MetadataTable.ID), KeyExtent.getMetadataEntry(new Text(MetadataTable.ID), null), null);
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/main/java/org/apache/accumulo/core/util/Merge.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/Merge.java b/core/src/main/java/org/apache/accumulo/core/util/Merge.java
index 350ccc0..13f4288 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/Merge.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/Merge.java
@@ -209,7 +209,7 @@ public class Merge {
   }
   
   protected Iterator<Size> getSizeIterator(Connector conn, String tablename, Text start, Text end) throws MergeException {
-    // open up the !METADATA table, walk through the tablets.
+    // open up metatadata, walk through the tablets.
     String tableId;
     Scanner scanner;
     try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/main/java/org/apache/accumulo/core/util/Pair.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/Pair.java b/core/src/main/java/org/apache/accumulo/core/util/Pair.java
index 575ab29..fb4ad80 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/Pair.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/Pair.java
@@ -16,33 +16,36 @@
  */
 package org.apache.accumulo.core.util;
 
+import java.util.Map;
+import java.util.Map.Entry;
+
 public class Pair<A,B> {
   A first;
   B second;
-  
+
   public Pair(A f, B s) {
     this.first = f;
     this.second = s;
   }
-  
+
   private int hashCode(Object o) {
     if (o == null)
       return 0;
     return o.hashCode();
   }
-  
+
   @Override
   public int hashCode() {
     return hashCode(first) + hashCode(second);
   }
-  
+
   private boolean equals(Object o1, Object o2) {
     if (o1 == null || o2 == null)
       return o1 == o2;
-    
+
     return o1.equals(o2);
   }
-  
+
   @Override
   public boolean equals(Object o) {
     if (o instanceof Pair<?,?>) {
@@ -51,18 +54,50 @@ public class Pair<A,B> {
     }
     return false;
   }
-  
+
   public A getFirst() {
     return first;
   }
-  
+
   public B getSecond() {
     return second;
   }
-  
+
   @Override
   public String toString() {
-    return "(" + first + "," + second + ")";
+    return toString("(", ",", ")");
+  }
+
+  public String toString(String prefix, String separator, String suffix) {
+    return prefix + first + separator + second + suffix;
+  }
+
+  public Entry<A,B> toMapEntry() {
+    return new Map.Entry<A,B>() {
+
+      @Override
+      public A getKey() {
+        return getFirst();
+      }
+
+      @Override
+      public B getValue() {
+        return getSecond();
+      }
+
+      @Override
+      public B setValue(B value) {
+        throw new UnsupportedOperationException();
+      }
+    };
   }
-  
+
+  public Pair<B,A> swap() {
+    return new Pair<B,A>(getSecond(), getFirst());
+  }
+
+  public static <K,V> Pair<K,V> fromEntry(Entry<K,V> entry) {
+    return new Pair<K,V>(entry.getKey(), entry.getValue());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java b/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
index 3a32d8e..837b4b8 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
@@ -49,6 +49,7 @@ import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
+import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
@@ -412,14 +413,14 @@ public class Shell extends ShellOptions {
       try {
         instance = getZooInstance(instanceName, hosts, options.getClientConfiguration());
       } catch (Exception e) {
-        throw new IllegalArgumentException("Unable to load client config from " +  options.getClientConfigFile(), e);
+        throw new IllegalArgumentException("Unable to load client config from " + options.getClientConfigFile(), e);
       }
     }
   }
 
   /*
-   * Takes instanceName and keepers as separate arguments, rather than just packaged into the clientConfig,
-   * so that we can fail over to accumulo-site.xml or HDFS config if they're unspecified.
+   * Takes instanceName and keepers as separate arguments, rather than just packaged into the clientConfig, so that we can fail over to accumulo-site.xml or
+   * HDFS config if they're unspecified.
    */
   private static Instance getZooInstance(String instanceName, String keepers, ClientConfiguration clientConfig) {
     UUID instanceId = null;
@@ -427,8 +428,7 @@ public class Shell extends ShellOptions {
       instanceName = clientConfig.get(ClientProperty.INSTANCE_NAME);
     }
     if (instanceName == null || keepers == null) {
-      AccumuloConfiguration conf = SiteConfiguration.getInstance(ServerConfigurationUtil.convertClientConfig(DefaultConfiguration.getInstance(),
-          clientConfig));
+      AccumuloConfiguration conf = SiteConfiguration.getInstance(ServerConfigurationUtil.convertClientConfig(DefaultConfiguration.getInstance(), clientConfig));
       if (instanceName == null) {
         Path instanceDir = new Path(conf.get(Property.INSTANCE_DFS_DIR), "instance_id");
         instanceId = UUID.fromString(ZooUtil.getInstanceIDFromHdfs(instanceDir));
@@ -493,9 +493,6 @@ public class Shell extends ShellOptions {
       log.warn("Unable to load history file at " + historyPath);
     }
 
-    // This would be a nice feature but !METADATA screws it up
-    reader.setExpandEvents(false);
-
     // Turn Ctrl+C into Exception instead of JVM exit
     reader.setHandleUserInterrupt(true);
 
@@ -749,8 +746,11 @@ public class Shell extends ShellOptions {
       modifiedTablenames.add(a.replaceAll("([\\s'\"])", "\\\\$1"));
     for (String a : userlist)
       modifiedUserlist.add(a.replaceAll("([\\s'\"])", "\\\\$1"));
-    for (String a : namespaces)
-      modifiedNamespaces.add(a.replaceAll("([\\s'\"])", "\\\\$1"));
+    for (String a : namespaces) {
+      String b = a.replaceAll("([\\s'\"])", "\\\\$1");
+      if (!b.isEmpty())
+        modifiedNamespaces.add(b);
+    }
 
     options.put(Command.CompletionSet.USERNAMES, modifiedUserlist);
     options.put(Command.CompletionSet.TABLENAMES, modifiedTablenames);
@@ -1070,7 +1070,7 @@ public class Shell extends ShellOptions {
   }
 
   public void setTableName(String tableName) {
-    this.tableName = tableName;
+    this.tableName = (tableName == null || tableName.isEmpty()) ? "" : Tables.qualified(tableName);
   }
 
   public String getTableName() {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java
index b017eaf..b0e7432 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java
@@ -31,7 +31,6 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.TimeType;
-import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.security.VisibilityConstraint;
@@ -112,12 +111,7 @@ public class CreateTableCommand extends Command {
       shellState.getConnector().tableOperations().addSplits(tableName, partitions);
     }
 
-    String n = Tables.extractNamespace(tableName);
-    String table = tableName;
-    if (n.equals(Constants.DEFAULT_NAMESPACE) || n.equals(Constants.SYSTEM_NAMESPACE)) {
-      table = Tables.extractTableName(tableName);
-    }
-    shellState.setTableName(table); // switch shell to new table context
+    shellState.setTableName(tableName); // switch shell to new table context
 
     if (cl.hasOption(createTableNoDefaultIters.getOpt())) {
       for (String key : IteratorUtil.generateInitialTableProperties(true).keySet()) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteNamespaceCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteNamespaceCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteNamespaceCommand.java
index 756e85d..01d7fc0 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteNamespaceCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteNamespaceCommand.java
@@ -20,7 +20,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.util.shell.Shell;
@@ -75,11 +74,6 @@ public class DeleteNamespaceCommand extends Command {
           shellState.getConnector().tableOperations().delete(table);
 
     shellState.getConnector().namespaceOperations().delete(namespace);
-    if (namespace.equals(Constants.SYSTEM_NAMESPACE)) {
-      shellState.getReader().println("Namespace: [" + namespace + "], can't delete system or default namespace.");
-    } else {
-      shellState.getReader().println("Namespace: [" + namespace + "] has been deleted.");
-    }
     if (resetContext) {
       shellState.setTableName("");
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteTableCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteTableCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteTableCommand.java
index 10a12d3..a5aa32a 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteTableCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteTableCommand.java
@@ -16,8 +16,6 @@
  */
 package org.apache.accumulo.core.util.shell.commands;
 
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
@@ -25,7 +23,8 @@ import org.apache.commons.cli.Options;
 
 public class DeleteTableCommand extends TableOperation {
   private Option forceOpt;
-  
+
+  @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws Exception {
     if (cl.hasOption(forceOpt.getOpt())) {
       super.force();
@@ -34,32 +33,27 @@ public class DeleteTableCommand extends TableOperation {
     }
     return super.execute(fullCommand, cl, shellState);
   }
-  
+
   @Override
   public String description() {
     return "deletes a table";
   }
-  
+
   @Override
   protected void doTableOp(final Shell shellState, final String tableName) throws Exception {
     shellState.getConnector().tableOperations().delete(tableName);
     shellState.getReader().println("Table: [" + tableName + "] has been deleted.");
-    
-    String n = Tables.extractNamespace(tableName);
-    String checkTable = tableName;
-    if (n.equals(Constants.DEFAULT_NAMESPACE) || n.equals(Constants.SYSTEM_NAMESPACE)) {
-      checkTable = Tables.extractTableName(tableName);
-    }
-    if (shellState.getTableName().equals(checkTable)) {
+
+    if (shellState.getTableName().equals(tableName)) {
       shellState.setTableName("");
     }
   }
-  
+
   @Override
   public Options getOptions() {
     forceOpt = new Option("f", "force", false, "force deletion without prompting");
     final Options opts = super.getOptions();
-    
+
     opts.addOption(forceOpt);
     return opts;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java
index 38f7c77..cbbc105 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java
@@ -18,9 +18,11 @@ package org.apache.accumulo.core.util.shell.commands;
 
 import java.io.IOException;
 import java.util.Iterator;
+import java.util.Map;
 import java.util.Map.Entry;
 import java.util.TreeMap;
 
+import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.util.shell.Shell;
@@ -28,40 +30,35 @@ import org.apache.accumulo.core.util.shell.Shell.Command;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
-import org.apache.commons.collections.iterators.AbstractIteratorDecorator;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Iterators;
 
 public class NamespacesCommand extends Command {
   private Option disablePaginationOpt, namespaceIdOption;
 
-  @SuppressWarnings("unchecked")
+  private static final String DEFAULT_NAMESPACE_DISPLAY_NAME = "(default)";
+
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException, IOException {
-    Iterator<String> names = shellState.getConnector().namespaceOperations().list().iterator();
-    Iterator<String> ids = new NamespaceIdIterator(new TreeMap<String,String>(shellState.getConnector().namespaceOperations().namespaceIdMap()).entrySet()
-        .iterator());
-
-    if (cl.hasOption(namespaceIdOption.getOpt())) {
-      shellState.printLines(ids, !cl.hasOption(disablePaginationOpt.getOpt()));
-    } else {
-      shellState.printLines(names, !cl.hasOption(disablePaginationOpt.getOpt()));
-    }
-    return 0;
-  }
+    Map<String,String> namespaces = new TreeMap<String,String>(shellState.getConnector().namespaceOperations().namespaceIdMap());
 
-  /**
-   * Decorator that formats the id and name for display.
-   */
-  private static final class NamespaceIdIterator extends AbstractIteratorDecorator {
-    public NamespaceIdIterator(Iterator<Entry<String,String>> iterator) {
-      super(iterator);
-    }
+    Iterator<String> it = Iterators.transform(namespaces.entrySet().iterator(), new Function<Entry<String,String>,String>() {
+      @Override
+      public String apply(Map.Entry<String,String> entry) {
+        String name = entry.getKey();
+        if (Constants.DEFAULT_NAMESPACE.equals(name))
+          name = DEFAULT_NAMESPACE_DISPLAY_NAME;
+        String id = entry.getValue();
+        if (cl.hasOption(namespaceIdOption.getOpt()))
+          return String.format(TablesCommand.NAME_AND_ID_FORMAT, name, id);
+        else
+          return name;
+      };
+    });
 
-    @SuppressWarnings("rawtypes")
-    @Override
-    public Object next() {
-      Entry entry = (Entry) super.next();
-      return String.format("%-15s => %10s%n", entry.getKey(), entry.getValue());
-    }
+    shellState.printLines(it, !cl.hasOption(disablePaginationOpt.getOpt()));
+    return 0;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameTableCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameTableCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameTableCommand.java
index fe86f35..38d8158 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameTableCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameTableCommand.java
@@ -19,7 +19,6 @@ package org.apache.accumulo.core.util.shell.commands;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableExistsException;
@@ -35,31 +34,27 @@ public class RenameTableCommand extends Command {
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException,
       TableNotFoundException, TableExistsException {
     shellState.getConnector().tableOperations().rename(cl.getArgs()[0], cl.getArgs()[1]);
-    if (shellState.getTableName().equals(cl.getArgs()[0])) {
-      String tableName = cl.getArgs()[1];
-      String n = Tables.extractNamespace(tableName);
-      if (n.equals(Constants.DEFAULT_NAMESPACE) || n.equals(Constants.SYSTEM_NAMESPACE)) {
-        tableName = Tables.extractTableName(tableName);
-      }
-      shellState.setTableName(tableName);
+    if (shellState.getTableName().equals(Tables.qualified(cl.getArgs()[0]))) {
+      shellState.setTableName(cl.getArgs()[1]);
     }
     return 0;
   }
-  
+
   @Override
   public String usage() {
     return getName() + " <current table name> <new table name>";
   }
-  
+
   @Override
   public String description() {
     return "renames a table";
   }
-  
+
+  @Override
   public void registerCompletion(final Token root, final Map<Command.CompletionSet,Set<String>> completionSet) {
     registerCompletionForTables(root, completionSet);
   }
-  
+
   @Override
   public int numArgs() {
     return 2;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/main/java/org/apache/accumulo/core/util/shell/commands/TablesCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/TablesCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/TablesCommand.java
index 5cdf62f..a03a986 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/TablesCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/TablesCommand.java
@@ -17,9 +17,7 @@
 package org.apache.accumulo.core.util.shell.commands;
 
 import java.io.IOException;
-import java.util.Collections;
 import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.TreeMap;
@@ -27,17 +25,21 @@ import java.util.TreeMap;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 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.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
 import org.apache.commons.collections.MapUtils;
-import org.apache.commons.collections.iterators.AbstractIteratorDecorator;
+
+import com.google.common.base.Function;
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Maps;
 
 public class TablesCommand extends Command {
-  private static final String NAME_AND_ID_FORMAT = "%-15s => %10s%n";
+  static final String NAME_AND_ID_FORMAT = "%-20s => %9s%n";
 
   private Option tableIdOption;
   private Option sortByTableIdOption;
@@ -48,53 +50,36 @@ public class TablesCommand extends Command {
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException, IOException,
       NamespaceNotFoundException {
 
-    final Iterator<String> tableNames;
-    final Iterator<String> tableIds;
-
-    if (cl.hasOption(OptUtil.namespaceOpt().getOpt())) {
-      String namespace = shellState.getConnector().namespaceOperations().namespaceIdMap().get(OptUtil.getNamespaceOpt(cl, shellState));
-      tableNames = Namespaces.getTableNames(shellState.getConnector().getInstance(), namespace).iterator();
-      List<String> tableIdStrings = Namespaces.getTableIds(shellState.getConnector().getInstance(), namespace);
-      if (cl.hasOption(sortByTableIdOption.getOpt()))
-        Collections.sort(tableIdStrings);
-      tableIds = tableIdStrings.iterator();
-    } else {
-      tableNames = shellState.getConnector().tableOperations().list().iterator();
-      tableIds = new TableIdIterator(shellState.getConnector().tableOperations().tableIdMap(), cl.hasOption(sortByTableIdOption.getOpt()));
-    }
+    final String namespace = cl.hasOption(OptUtil.namespaceOpt().getOpt()) ? OptUtil.getNamespaceOpt(cl, shellState) : null;
+    Map<String,String> tables = shellState.getConnector().tableOperations().tableIdMap();
 
-    Iterator<String> it = cl.hasOption(tableIdOption.getOpt()) ? tableIds : tableNames;
-    shellState.printLines(it, !cl.hasOption(disablePaginationOpt.getOpt()));
-    return 0;
-  }
+    // filter only specified namespace
+    tables = Maps.filterKeys(tables, new Predicate<String>() {
+      @Override
+      public boolean apply(String tableName) {
+        return namespace == null || Tables.qualify(tableName).getFirst().equals(namespace);
+      }
+    });
 
-  /**
-   * Decorator that formats table name and id for display.
-   */
-  private static final class TableIdIterator extends AbstractIteratorDecorator {
-    private final boolean sortByTableId;
+    final boolean sortByTableId = cl.hasOption(sortByTableIdOption.getOpt());
+    tables = new TreeMap<String,String>((sortByTableId ? MapUtils.invertMap(tables) : tables));
 
-    /**
-     * @param tableIdMap
-     *          tableName -> tableId
-     * @param sortByTableId
-     */
-    @SuppressWarnings("unchecked")
-    public TableIdIterator(Map<String,String> tableIdMap, boolean sortByTableId) {
-      super(new TreeMap<String,String>((sortByTableId ? MapUtils.invertMap(tableIdMap) : tableIdMap)).entrySet().iterator());
-      this.sortByTableId = sortByTableId;
-    }
+    Iterator<String> it = Iterators.transform(tables.entrySet().iterator(), new Function<Entry<String,String>,String>() {
+      @Override
+      public String apply(Map.Entry<String,String> entry) {
+        String tableName = String.valueOf(sortByTableId ? entry.getValue() : entry.getKey());
+        String tableId = String.valueOf(sortByTableId ? entry.getKey() : entry.getValue());
+        if (namespace != null)
+          tableName = Tables.qualify(tableName).getSecond();
+        if (cl.hasOption(tableIdOption.getOpt()))
+          return String.format(NAME_AND_ID_FORMAT, tableName, tableId);
+        else
+          return tableName;
+      };
+    });
 
-    @SuppressWarnings("rawtypes")
-    @Override
-    public Object next() {
-      Entry entry = (Entry) super.next();
-      if (sortByTableId) {
-        return String.format(NAME_AND_ID_FORMAT, entry.getValue(), entry.getKey());
-      } else {
-        return String.format(NAME_AND_ID_FORMAT, entry.getKey(), entry.getValue());
-      }
-    }
+    shellState.printLines(it, !cl.hasOption(disablePaginationOpt.getOpt()));
+    return 0;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
index bf92c57..b30a114 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java
@@ -766,7 +766,7 @@ public class TabletLocatorImplTest extends TestCase {
     locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver2", credentials);
     locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver3", credentials);
 
-    // simulate the !METADATA table splitting
+    // simulate the metadata table splitting
     KeyExtent mte1 = new KeyExtent(new Text(MetadataTable.ID), tab1e21.getMetadataEntry(), RTE.getEndRow());
     KeyExtent mte2 = new KeyExtent(new Text(MetadataTable.ID), null, tab1e21.getMetadataEntry());
 
@@ -805,14 +805,14 @@ public class TabletLocatorImplTest extends TestCase {
     locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credentials);
     locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credentials);
 
-    // simulate a hole in the !METADATA table, caused by a partial split
+    // simulate a hole in the metadata, caused by a partial split
     KeyExtent mte11 = new KeyExtent(new Text(MetadataTable.ID), tab1e1.getMetadataEntry(), RTE.getEndRow());
     KeyExtent mte12 = new KeyExtent(new Text(MetadataTable.ID), tab1e21.getMetadataEntry(), tab1e1.getMetadataEntry());
     deleteServer(tservers, "tserver10");
     setLocation(tservers, "tserver4", RTE, mte12, "tserver10");
     setLocation(tservers, "tserver10", mte12, tab1e21, "tserver12");
 
-    // at this point should be no info in !METADATA about tab1e1
+    // at this point should be no table1 metadata
     tab1TabletCache.invalidateCache(tab1e1);
     tab1TabletCache.invalidateCache(tab1e21);
     locateTabletTest(tab1TabletCache, "a", null, null, credentials);
@@ -1019,7 +1019,7 @@ public class TabletLocatorImplTest extends TestCase {
   }
 
   public void testBinRanges5() throws Exception {
-    // Test binning when there is a hole in the !METADATA information
+    // Test binning when there is a hole in the metadata
     Text tableName = new Text("foo");
 
     List<Range> ranges = nrl(new Range(new Text("1")));
@@ -1227,7 +1227,7 @@ public class TabletLocatorImplTest extends TestCase {
     setLocation(tservers, "tserver1", RTE, mte1, "tserver2");
     setLocation(tservers, "tserver1", RTE, mte2, "tserver3");
 
-    // create two tablets that straddle a !METADATA split point
+    // create two tablets that straddle a metadata split point
     KeyExtent ke1 = new KeyExtent(new Text("0"), new Text("0bbf20e"), null);
     KeyExtent ke2 = new KeyExtent(new Text("0"), new Text("0bc0756"), new Text("0bbf20e"));
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/core/src/test/java/org/apache/accumulo/core/util/shell/ShellTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/util/shell/ShellTest.java b/core/src/test/java/org/apache/accumulo/core/util/shell/ShellTest.java
index d4f0398..97852ca 100644
--- a/core/src/test/java/org/apache/accumulo/core/util/shell/ShellTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/util/shell/ShellTest.java
@@ -37,29 +37,29 @@ import org.junit.Test;
 public class ShellTest {
   public static class TestOutputStream extends OutputStream {
     StringBuilder sb = new StringBuilder();
-    
+
     @Override
     public void write(int b) throws IOException {
       sb.append((char) (0xff & b));
     }
-    
+
     public String get() {
       return sb.toString();
     }
-    
+
     public void clear() {
       sb.setLength(0);
     }
   }
-  
+
   private TestOutputStream output;
   private Shell shell;
-  
+
   void exec(String cmd) throws IOException {
     output.clear();
     shell.execCommand(cmd, true, true);
   }
-  
+
   void exec(String cmd, boolean expectGoodExit) throws IOException {
     exec(cmd);
     if (expectGoodExit)
@@ -67,11 +67,11 @@ public class ShellTest {
     else
       assertBadExit("", true);
   }
-  
+
   void exec(String cmd, boolean expectGoodExit, String expectString) throws IOException {
     exec(cmd, expectGoodExit, expectString, true);
   }
-  
+
   void exec(String cmd, boolean expectGoodExit, String expectString, boolean stringPresent) throws IOException {
     exec(cmd);
     if (expectGoodExit)
@@ -79,7 +79,7 @@ public class ShellTest {
     else
       assertBadExit(expectString, stringPresent);
   }
-  
+
   @Before
   public void setup() throws IOException {
     Shell.log.setLevel(Level.OFF);
@@ -88,14 +88,14 @@ public class ShellTest {
     shell.setLogErrorsToConsole();
     shell.config("--fake", "-u", "test", "-p", "secret");
   }
-  
+
   void assertGoodExit(String s, boolean stringPresent) {
     Shell.log.debug(output.get());
     assertEquals(shell.getExitCode(), 0);
     if (s.length() > 0)
       assertEquals(s + " present in " + output.get() + " was not " + stringPresent, stringPresent, output.get().contains(s));
   }
-  
+
   void assertBadExit(String s, boolean stringPresent) {
     Shell.log.debug(output.get());
     assertTrue(shell.getExitCode() > 0);
@@ -103,7 +103,7 @@ public class ShellTest {
       assertEquals(s + " present in " + output.get() + " was not " + stringPresent, stringPresent, output.get().contains(s));
     shell.resetExitCode();
   }
-  
+
   @Test
   public void aboutTest() throws IOException {
     Shell.log.debug("Starting about test -----------------------------------");
@@ -111,7 +111,7 @@ public class ShellTest {
     exec("about -v", true, "Current user:");
     exec("about arg", false, "java.lang.IllegalArgumentException: Expected 0 arguments");
   }
-  
+
   @Test
   public void addGetSplitsTest() throws IOException {
     Shell.log.debug("Starting addGetSplits test ----------------------------");
@@ -121,7 +121,7 @@ public class ShellTest {
     exec("getsplits", true, "1\n\\x80");
     exec("deletetable test -f", true, "Table: [test] has been deleted");
   }
-  
+
   @Test
   public void insertDeleteScanTest() throws IOException {
     Shell.log.debug("Starting insertDeleteScan test ------------------------");
@@ -146,7 +146,7 @@ public class ShellTest {
     exec("scan", true, "\\x90 \\xA0:\\xB0 []    \\xC0", false);
     exec("deletetable test -f", true, "Table: [test] has been deleted");
   }
-  
+
   @Test
   public void authsTest() throws Exception {
     Shell.log.debug("Starting auths test --------------------------");
@@ -161,14 +161,14 @@ public class ShellTest {
     exec("getauths", true, "a,x,y,z");
     exec("setauths -c", true);
   }
-  
+
   @Test
   public void userTest() throws Exception {
     Shell.log.debug("Starting user test --------------------------");
     // Test cannot be done via junit because createuser only prompts for password
     // exec("createuser root", false, "user exists");
   }
-  
+
   @Test
   public void duContextTest() throws Exception {
     Shell.log.debug("Starting du context test --------------------------");
@@ -176,7 +176,7 @@ public class ShellTest {
     exec("du", true, "0 [t]");
     exec("deletetable t -f", true, "Table: [t] has been deleted");
   }
-  
+
   @Test
   public void duTest() throws IOException {
     Shell.log.debug("Starting DU test --------------------------");
@@ -184,7 +184,7 @@ public class ShellTest {
     exec("du t", true, "0 [t]");
     exec("deletetable t -f", true, "Table: [t] has been deleted");
   }
-  
+
   @Test
   public void duPatternTest() throws IOException {
     Shell.log.debug("Starting DU with pattern test --------------------------");
@@ -194,7 +194,7 @@ public class ShellTest {
     exec("deletetable t -f", true, "Table: [t] has been deleted");
     exec("deletetable tt -f", true, "Table: [tt] has been deleted");
   }
-  
+
   @Test
   public void scanDateStringFormatterTest() throws IOException {
     Shell.log.debug("Starting scan dateStringFormatter test --------------------------");
@@ -205,7 +205,7 @@ public class ShellTest {
     exec("scan -fm org.apache.accumulo.core.util.format.DateStringFormatter -st", true, expected);
     exec("deletetable t -f", true, "Table: [t] has been deleted");
   }
-  
+
   @Test
   public void commentTest() throws IOException {
     Shell.log.debug("Starting comment test --------------------------");
@@ -213,7 +213,7 @@ public class ShellTest {
     exec("# foo", true, "Unknown command", false);
     exec("- foo", true, "Unknown command", true);
   }
-  
+
   @Test
   public void execFileTest() throws IOException {
     Shell.log.debug("Starting exec file test --------------------------");
@@ -221,23 +221,23 @@ public class ShellTest {
     assertEquals(0, shell.start());
     assertGoodExit("Unknown command", false);
   }
-  
+
   @Test
   public void setIterTest() throws IOException {
     Shell.log.debug("Starting setiter test --------------------------");
     exec("createtable t", true);
-    
+
     String cmdJustClass = "setiter -class VersioningIterator -p 1";
     exec(cmdJustClass, false, "java.lang.IllegalArgumentException", false);
     exec(cmdJustClass, false, "fully qualified package name", true);
-    
+
     String cmdFullPackage = "setiter -class o.a.a.foo -p 1";
     exec(cmdFullPackage, false, "java.lang.IllegalArgumentException", false);
     exec(cmdFullPackage, false, "class not found", true);
-    
+
     String cmdNoOption = "setiter -class java.lang.String -p 1";
     exec(cmdNoOption, false, "Loaded", true);
-    
+
     exec("deletetable t -f", true, "Table: [t] has been deleted");
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/docs/src/main/latex/accumulo_user_manual/chapters/administration.tex
----------------------------------------------------------------------
diff --git a/docs/src/main/latex/accumulo_user_manual/chapters/administration.tex b/docs/src/main/latex/accumulo_user_manual/chapters/administration.tex
index cc7697c..c3a9c3f 100644
--- a/docs/src/main/latex/accumulo_user_manual/chapters/administration.tex
+++ b/docs/src/main/latex/accumulo_user_manual/chapters/administration.tex
@@ -202,7 +202,7 @@ Start Accumulo using the \texttt{bin/start-all.sh} script.
 
 To verify that Accumulo is running, check the Status page as described under
 \emph{Monitoring}. In addition, the Shell can provide some information about the status of
-tables via reading the !METADATA table.
+tables via reading the metadata tables.
 
 \subsection{Stopping Accumulo}
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/docs/src/main/latex/accumulo_user_manual/chapters/shell.tex
----------------------------------------------------------------------
diff --git a/docs/src/main/latex/accumulo_user_manual/chapters/shell.tex b/docs/src/main/latex/accumulo_user_manual/chapters/shell.tex
index a184dd8..25ac8a7 100644
--- a/docs/src/main/latex/accumulo_user_manual/chapters/shell.tex
+++ b/docs/src/main/latex/accumulo_user_manual/chapters/shell.tex
@@ -52,14 +52,16 @@ table and instance specific options.
 \small
 \begin{verbatim}
 root@myinstance> tables
-!METADATA
+accumulo.metadata
+accumulo.root
 
 root@myinstance> createtable mytable
 
 root@myinstance mytable>
 
 root@myinstance mytable> tables
-!METADATA
+accumulo.metadata
+accumulo.root
 mytable
 
 root@myinstance mytable> createtable testtable
@@ -136,7 +138,7 @@ Enter current password for 'bob': *********
 
 bob@myinstance mytable> userpermissions
 System permissions: System.CREATE_TABLE
-Table permissions (!METADATA): Table.READ
+Table permissions (accumulo.metadata): Table.READ
 Table permissions (mytable): NONE
 
 bob@myinstance mytable> createtable bobstable

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/docs/src/main/latex/accumulo_user_manual/chapters/table_configuration.tex
----------------------------------------------------------------------
diff --git a/docs/src/main/latex/accumulo_user_manual/chapters/table_configuration.tex b/docs/src/main/latex/accumulo_user_manual/chapters/table_configuration.tex
index 5ecbf05..d601fae 100644
--- a/docs/src/main/latex/accumulo_user_manual/chapters/table_configuration.tex
+++ b/docs/src/main/latex/accumulo_user_manual/chapters/table_configuration.tex
@@ -401,7 +401,7 @@ table.cache.index.enable: Determines whether index cache is enabled.
 \end{verbatim}
 
 The block cache can have a significant effect on alleviating hot spots, as well as reducing query latency.
-It is enabled by default for the !METADATA table.
+It is enabled by default for the metadata tables.
 
 \section{Compaction}
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/docs/src/main/latex/accumulo_user_manual/chapters/troubleshooting.tex
----------------------------------------------------------------------
diff --git a/docs/src/main/latex/accumulo_user_manual/chapters/troubleshooting.tex b/docs/src/main/latex/accumulo_user_manual/chapters/troubleshooting.tex
index 7f62885..91fb156 100644
--- a/docs/src/main/latex/accumulo_user_manual/chapters/troubleshooting.tex
+++ b/docs/src/main/latex/accumulo_user_manual/chapters/troubleshooting.tex
@@ -367,7 +367,7 @@ Here's what the output means:
 An unassigned tablet with write-ahead logs is probably waiting for
 logs to be sorted for efficient recovery.
 
-Q. How can I be sure that the !METADATA table is up and consistent?
+Q. How can I be sure that the metadata tables are up and consistent?
 
 A. \texttt{CheckForMetadataProblems} will verify the start/end of
 every tablet matches, and the start and stop for the table is empty:
@@ -446,13 +446,14 @@ INFO : Using ZooKeepers localhost:2181
 \end{verbatim}
 \normalsize
 
-\section{METADATA Table}
+\section{System Metadata Tables}
 
-Accumulo tracks information about all other tables in the !METADATA
-table.  The !METADATA table information is tracked in a very simple
-table that always consists of a single tablet, called the !!ROOT table.
-The root table information, such as its location and write-ahead logs
-are stored in Zookeeper.
+Accumulo tracks information about tables in metadata tables. The metadata for
+most tables is contained within the metadata table in the accumulo namespace,
+while metadata for that table is contained in the root table in the accumulo
+namespace. The root table is composed of a single tablet, which does not
+split, so it is also called the root tablet. Information about the root
+table, such as its location and write-ahead logs, are stored in ZooKeeper.
 
 Let's create a table and put some data into it:
 
@@ -460,19 +461,20 @@ Let's create a table and put some data into it:
 \begin{verbatim}
 shell> createtable test
 shell> tables -l
-!METADATA       =>         !0
-test            =>          3
-trace           =>          1
+accumulo.metadata    =>        !0
+accumulo.root        =>        +r
+test                 =>         2
+trace                =>         1
 shell> insert a b c d
 shell> flush -w
 \end{verbatim}
 \normalsize
 
-Now let's take a look at the !METADATA table information for this table:
+Now let's take a look at the metadata for this table:
 
 \small
 \begin{verbatim}
-shell> table !METADATA
+shell> table accumulo.metadata
 shell> scan -b 3; -e 3<
 3< file:/default_tablet/F000009y.rf []    186,1
 3< last:13fe86cd27101e5 []    127.0.0.1:9997
@@ -496,7 +498,7 @@ Let's decode this little session:
 \item{\texttt{log:127.0.0.1+9997/0cb7ce52-ac46-4bf7-ae1d-acdcfaa97995 []    127.0.0.1+9997/0cb7ce52-ac46-4bf7-ae1d-acdcfaa97995|6} This tablet has a reference to a single write-ahead log.  This file can be found in /accumulo/wal/127.0.0.1+9997/0cb7ce52-ac46-4bf7-ae1d-acdcfaa97995.  The value of this entry could refer to multiple files.  This tablet's data is encoded as ``6'' within the log.}
 \item{\texttt{srv:dir []    /default\_tablet} Files written for this tablet will be placed into /accumulo/tables/3/default\_tablet.}
 \item{\texttt{srv:flush []    1} Flush id.  This table has successfully completed the flush with the id of ``1''. }
-\item{\texttt{srv:lock []    tservers/127.0.0.1:9997/zlock-0000000001\$13fe86cd27101e5}  This is the lock information for the tablet holding the present lock.  This information is checked against zookeeper whenever this is updated, which prevents a !METADATA table update from a tablet server that no longer holds its lock.}
+\item{\texttt{srv:lock []    tservers/127.0.0.1:9997/zlock-0000000001\$13fe86cd27101e5}  This is the lock information for the tablet holding the present lock.  This information is checked against zookeeper whenever this is updated, which prevents a metadata update from a tablet server that no longer holds its lock.}
 \item{\texttt{srv:time []    M1373998392323} }
 \item{\texttt{~tab:~pr []    \\x00} The end-row marker for the previous tablet (prev-row).  The first byte indicates the presence of a prev-row.  This tablet has the range (-inf, +inf), so it has no prev-row (or end row). }
 \end{enumerate}
@@ -505,7 +507,7 @@ Besides these columns, you may see:
 
 \begin{enumerate}
 \item{\texttt{rowId future:zooKeeperID location} Tablet has been assigned to a tablet, but not yet loaded.}
-\item{\texttt{~del:filename} When a tablet server is done use a file, it will create a delete marker in the !METADATA table, unassociated with any table.  The garbage collector will remove the marker, and the file, when no other reference to the file exists.}
+\item{\texttt{~del:filename} When a tablet server is done use a file, it will create a delete marker in the appropriate metadata table, unassociated with any tablet.  The garbage collector will remove the marker, and the file, when no other reference to the file exists.}
 \item{\texttt{~blip:txid} Bulk-Load In Progress marker}
 \item{\texttt{rowId loaded:filename} A file has been bulk-loaded into this tablet, however the bulk load has not yet completed on other tablets, so this is marker prevents the file from being loaded multiple times.}
 \item{\texttt{rowId !cloned} A marker that indicates that this tablet has been successfully cloned.}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/proxy/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java
----------------------------------------------------------------------
diff --git a/proxy/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java b/proxy/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java
index 31005f3..ecb1c89 100644
--- a/proxy/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java
+++ b/proxy/src/test/java/org/apache/accumulo/proxy/SimpleProxyIT.java
@@ -1459,7 +1459,7 @@ public class SimpleProxyIT {
     assertEquals(val, ByteBufferUtil.toString(keyValue.value));
   }
 
-  // scan !METADATA table for file entries for the given table
+  // scan metadata for file entries for the given table
   private int countFiles(String table) throws Exception {
     Map<String,String> tableIdMap = client.tableIdMap(creds);
     String tableId = tableIdMap.get(table);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
index d300366..131426e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
@@ -57,7 +57,7 @@ public class NamespaceConfiguration extends AccumuloConfiguration {
     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_NAMESPACE_ID) && isIteratorOrConstraint(property.getKey()))) {
+      if (!(namespaceId.equals(Constants.ACCUMULO_NAMESPACE_ID) && isIteratorOrConstraint(property.getKey()))) {
         // ignore iterators from parent if system namespace
         value = parent.get(property);
       }
@@ -108,7 +108,7 @@ public class NamespaceConfiguration extends AccumuloConfiguration {
 
     // exclude system iterators/constraints from the system namespace
     // so they don't affect the metadata or root tables.
-    if (this.namespaceId.equals(Constants.SYSTEM_NAMESPACE_ID))
+    if (this.namespaceId.equals(Constants.ACCUMULO_NAMESPACE_ID))
       parentFilter = new SystemNamespaceFilter(filter);
 
     parent.getProperties(props, parentFilter);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/server/base/src/main/java/org/apache/accumulo/server/fs/FileRef.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/FileRef.java b/server/base/src/main/java/org/apache/accumulo/server/fs/FileRef.java
index b4bea4a..dade501 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/FileRef.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/FileRef.java
@@ -23,8 +23,9 @@ import org.apache.hadoop.io.Text;
 
 /**
  * This is a glue object, to convert short file references to long references.
- * The !METADATA table may contain old relative file references.  This class keeps 
- * track of the short file reference, so it can be removed properly from the !METADATA table.
+ * The metadata may contain old relative file references.  This class keeps 
+ * track of the short file reference, so it can be removed properly from the
+ * metadata tables.
  */
 public class FileRef implements Comparable<FileRef> {
   String metaReference;  // something like ../2/d-00000/A00001.rf

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java
index b7787c9..00e86d3 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManager.java
@@ -123,7 +123,7 @@ public interface VolumeManager {
   // forward to the appropriate FileSystem object
   FileStatus[] globStatus(Path path) throws IOException;
 
-  // Convert a file or directory !METADATA reference into a path
+  // Convert a file or directory metadata reference into a path
   Path getFullPath(Key key);
   
   Path getFullPath(String tableId, String path);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/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 61508db..b7f9bdb 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
@@ -417,12 +417,12 @@ public class Initialize {
     zoo.putPersistentData(zkInstanceRoot + Constants.ZNAMESPACES, new byte[0], NodeExistsPolicy.FAIL);
     
     createInitialNamespace(zoo, zkInstanceRoot, Constants.DEFAULT_NAMESPACE_ID, Constants.DEFAULT_NAMESPACE, true);
-    createInitialNamespace(zoo, zkInstanceRoot, Constants.SYSTEM_NAMESPACE_ID, Constants.SYSTEM_NAMESPACE, false);
+    createInitialNamespace(zoo, zkInstanceRoot, Constants.ACCUMULO_NAMESPACE_ID, Constants.ACCUMULO_NAMESPACE, false);
     
     zoo.putPersistentData(zkInstanceRoot + Constants.ZTABLES + "/" + MetadataTable.ID + Constants.ZTABLE_NAMESPACE,
-        Constants.SYSTEM_NAMESPACE_ID.getBytes(Constants.UTF8), NodeExistsPolicy.OVERWRITE);
+        Constants.ACCUMULO_NAMESPACE_ID.getBytes(Constants.UTF8), NodeExistsPolicy.OVERWRITE);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZTABLES + "/" + RootTable.ID + Constants.ZTABLE_NAMESPACE,
-        Constants.SYSTEM_NAMESPACE_ID.getBytes(Constants.UTF8), NodeExistsPolicy.OVERWRITE);
+        Constants.ACCUMULO_NAMESPACE_ID.getBytes(Constants.UTF8), NodeExistsPolicy.OVERWRITE);
   }
   
   private static void createInitialNamespace(IZooReaderWriter zoo, String root, String id, String namespace, boolean defaultOpts) throws KeeperException,

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
index 5471fac..f1c934a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
@@ -125,7 +125,7 @@ public class MetaDataTableScanner implements Iterator<TabletLocationState> {
     try {
       return fetch();
     } catch (RuntimeException ex) {
-      // something is wrong with the records in the !METADATA table, just skip over it
+      // something is wrong with the metadata records, just skip over it
       log.error(ex, ex);
       mdScanner.close();
       return null;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/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 29db1d8..010efa8 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
@@ -338,7 +338,7 @@ public class SecurityOperation {
    */
   protected boolean hasNamespacePermissionForTableName(TCredentials credentials, String tableName, NamespacePermission permission, boolean useCached)
       throws ThriftSecurityException {
-    String namespace = Tables.extractNamespace(tableName);
+    String namespace = Tables.qualify(tableName).getFirst();
     try {
       String namespaceId = Namespaces.getNamespaceId(HdfsZooInstance.getInstance(), namespace);
       return hasNamespacePermission(credentials, namespaceId, permission, useCached);
@@ -356,7 +356,7 @@ public class SecurityOperation {
   protected boolean _hasNamespacePermission(String user, String namespace, NamespacePermission permission, boolean useCached) throws ThriftSecurityException {
     targetUserExists(user);
 
-    if (namespace.equals(Constants.SYSTEM_NAMESPACE_ID) && permission.equals(NamespacePermission.READ))
+    if (namespace.equals(Constants.ACCUMULO_NAMESPACE_ID) && permission.equals(NamespacePermission.READ))
       return true;
 
     try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
index 71274cc..482a669 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
@@ -28,6 +28,7 @@ import java.util.TreeSet;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 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.Authorizations;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
@@ -43,55 +44,58 @@ import org.apache.zookeeper.KeeperException;
 public class ZKAuthorizor implements Authorizor {
   private static final Logger log = Logger.getLogger(ZKAuthorizor.class);
   private static Authorizor zkAuthorizorInstance = null;
-  
+
   private final String ZKUserAuths = "/Authorizations";
-  
+
   private String ZKUserPath;
   private final ZooCache zooCache;
-  
+
   public static synchronized Authorizor getInstance() {
     if (zkAuthorizorInstance == null)
       zkAuthorizorInstance = new ZKAuthorizor();
     return zkAuthorizorInstance;
   }
-  
+
   public ZKAuthorizor() {
     zooCache = new ZooCache();
   }
-  
+
+  @Override
   public void initialize(String instanceId, boolean initialize) {
     ZKUserPath = ZKSecurityTool.getInstancePath(instanceId) + "/users";
   }
-  
+
+  @Override
   public Authorizations getCachedUserAuthorizations(String user) {
     byte[] authsBytes = zooCache.get(ZKUserPath + "/" + user + ZKUserAuths);
     if (authsBytes != null)
       return ZKSecurityTool.convertAuthorizations(authsBytes);
     return Authorizations.EMPTY;
   }
-  
+
   @Override
   public boolean validSecurityHandlers(Authenticator auth, PermissionHandler pm) {
     return true;
   }
-  
+
   @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())
       rootPerms.add(p);
     Map<String,Set<TablePermission>> tablePerms = new HashMap<String,Set<TablePermission>>();
-    // Allow the root user to flush the !METADATA table
+    // Allow the root user to flush the metadata tables
     tablePerms.put(MetadataTable.ID, Collections.singleton(TablePermission.ALTER_TABLE));
-    
+    tablePerms.put(RootTable.ID, Collections.singleton(TablePermission.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 + ZKUserAuths, ZKSecurityTool.convertAuthorizations(Authorizations.EMPTY), NodeExistsPolicy.FAIL);
     } catch (KeeperException e) {
@@ -102,11 +106,12 @@ public class ZKAuthorizor implements Authorizor {
       throw new RuntimeException(e);
     }
   }
-  
+
   /**
    * @param user
    * @throws AccumuloSecurityException
    */
+  @Override
   public void initUser(String user) throws AccumuloSecurityException {
     IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
     try {
@@ -119,7 +124,7 @@ public class ZKAuthorizor implements Authorizor {
       throw new RuntimeException(e);
     }
   }
-  
+
   @Override
   public void dropUser(String user) throws AccumuloSecurityException {
     try {
@@ -136,10 +141,10 @@ public class ZKAuthorizor implements Authorizor {
       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 void changeAuthorizations(String user, Authorizations authorizations) throws AccumuloSecurityException {
     try {
@@ -156,7 +161,7 @@ public class ZKAuthorizor implements Authorizor {
       throw new RuntimeException(e);
     }
   }
-  
+
   @Override
   public boolean isValidAuthorizations(String user, List<ByteBuffer> auths) throws AccumuloSecurityException {
     Collection<ByteBuffer> userauths = getCachedUserAuthorizations(user).getAuthorizationsBB();
@@ -165,5 +170,5 @@ public class ZKAuthorizor implements Authorizor {
         return false;
     return true;
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/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 4b19edb..1fb0b16 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
@@ -386,8 +386,8 @@ public class ZKPermHandler implements PermissionHandler {
     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<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));
+    namespacePerms.put(Constants.ACCUMULO_NAMESPACE_ID, Collections.singleton(NamespacePermission.ALTER_NAMESPACE));
+    namespacePerms.put(Constants.ACCUMULO_NAMESPACE_ID, Collections.singleton(NamespacePermission.ALTER_TABLE));
 
     try {
       // prep parent node of users with root username


[12/50] [abbrv] git commit: ACCUMULO-1479 fixed minor problems that caused tests to fail, permissions still not fully implemented

Posted by ct...@apache.org.
ACCUMULO-1479 fixed minor problems that caused tests to fail, permissions still not fully implemented


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

Branch: refs/heads/master
Commit: ad9abf462eab4108394e510ee2d614b14a3a501e
Parents: dfdf511
Author: Sean Hickey <ta...@gmail.com>
Authored: Thu Aug 1 15:15:39 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:10 2013 -0500

----------------------------------------------------------------------
 .../accumulo/server/security/SecurityOperation.java      | 11 ++++++++---
 1 file changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/ad9abf46/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 cad84d0..6dcaf9d 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
@@ -26,6 +26,7 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
 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.Tables;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
@@ -338,7 +339,12 @@ public class SecurityOperation {
   protected boolean hasTableNamespacePermissionForTableName(TCredentials credentials, String tableName, TableNamespacePermission permission, boolean useCached)
       throws ThriftSecurityException {
     String tableNamespace = Tables.extractNamespace(tableName);
-    return hasTableNamespacePermission(credentials, tableNamespace, permission, useCached);
+    try {
+      String namespace = TableNamespaces.getNamespaceId(HdfsZooInstance.getInstance(), tableNamespace);
+      return hasTableNamespacePermission(credentials, namespace, permission, useCached);
+    } catch (TableNamespaceNotFoundException e) {
+      return false;
+    }
   }
   
   /**
@@ -792,8 +798,7 @@ public class SecurityOperation {
   
   public boolean canImport(TCredentials credentials, String tableName, String importDir) throws ThriftSecurityException {
     authenticate(credentials);
-    String tableId = Tables.getNamespace(HdfsZooInstance.getInstance(), tableName);
     return hasSystemPermission(credentials, SystemPermission.CREATE_TABLE, false)
-        || hasTableNamespacePermissionForTableId(credentials, tableId, TableNamespacePermission.CREATE_TABLE, false);
+        || hasTableNamespacePermissionForTableName(credentials, tableName, TableNamespacePermission.CREATE_TABLE, false);
   }
 }


[49/50] [abbrv] git commit: ACCUMULO-802 fix more issues from ReviewBoard

Posted by ct...@apache.org.
ACCUMULO-802 fix more issues from ReviewBoard


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

Branch: refs/heads/master
Commit: 859cf2087c2506f60178efff1e08d8ad91af3d2d
Parents: 8b54ced
Author: Christopher Tubbs <ct...@apache.org>
Authored: Wed Dec 4 18:45:51 2013 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:47:21 2013 -0500

----------------------------------------------------------------------
 .../core/client/admin/NamespaceOperations.java  |  5 ++++-
 .../client/admin/NamespaceOperationsImpl.java   |  8 +++-----
 .../core/client/admin/TableOperationsImpl.java  |  8 +-------
 .../accumulo/core/client/impl/Tables.java       | 21 +++++++++++++-------
 .../shell/commands/CreateNamespaceCommand.java  |  3 ---
 .../randomwalk/concurrent/RenameNamespace.java  |  2 +-
 6 files changed, 23 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/859cf208/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java
index 4ee670c..535287d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java
@@ -32,6 +32,9 @@ import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 /**
  * Provides an API for administering namespaces
  * 
+ * All tables exist in a namespace. The default namespace has no name, and is used if an explicit namespace is not specified. Fully qualified table names look
+ * like "namespaceName.tableName". Tables in the default namespace are fully qualified simply as "tableName".
+ * 
  * @since 1.6.0
  */
 public interface NamespaceOperations {
@@ -63,7 +66,7 @@ public interface NamespaceOperations {
   public boolean exists(String namespace) throws AccumuloException, AccumuloSecurityException;
 
   /**
-   * Create an empty namespace with no initial configuration
+   * Create an empty namespace with no initial configuration. Valid names for a namespace contain letters, numbers, and the underscore character.
    * 
    * @param namespace
    *          the name of the namespace

http://git-wip-us.apache.org/repos/asf/accumulo/blob/859cf208/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
index 1ddcaf7..e5e18dc 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
@@ -98,9 +98,8 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
 
     try {
       doNamespaceOperation(TableOperation.CREATE, Arrays.asList(ByteBuffer.wrap(namespace.getBytes())), Collections.<String,String> emptyMap());
-    } catch (NamespaceNotFoundException e1) {
-      // should not happen
-      throw new RuntimeException(e1);
+    } catch (NamespaceNotFoundException e) {
+      throw new AssertionError("Shouldn't happen: " + e.getMessage());
     }
   }
 
@@ -235,8 +234,7 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
     try {
       doNamespaceOperation(TableOperation.DELETE, args, opts);
     } catch (NamespaceExistsException e) {
-      // should not happen
-      throw new RuntimeException(e);
+      throw new AssertionError("Shouldn't happen: " + e.getMessage());
     }
 
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/859cf208/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
index 7f79fcd..ef2bd09 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
@@ -1506,13 +1506,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       Logger.getLogger(this.getClass()).warn("Failed to check if imported table references external java classes : " + ioe.getMessage());
     }
 
-    String namespace = Tables.qualify(tableName).getFirst();
-    if (!namespaceExists(namespace)) {
-      String info = "Namespace not found while importing to table";
-      throw new RuntimeException(new NamespaceNotFoundException(null, namespace, info));
-    }
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes()), ByteBuffer.wrap(importDir.getBytes()));
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(importDir.getBytes()));
 
     Map<String,String> opts = Collections.emptyMap();
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/859cf208/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
index af8c617..45db491 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
@@ -17,6 +17,7 @@
 package org.apache.accumulo.core.client.impl;
 
 import java.security.SecurityPermission;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.SortedMap;
@@ -54,28 +55,34 @@ public class Tables {
 
     TreeMap<String,String> tableMap = new TreeMap<String,String>();
 
+    Map<String,String> namespaceIdToNameMap = new HashMap<String,String>();
+
     for (String tableId : tableIds) {
-      byte[] tblPath = zc.get(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_NAME);
+      byte[] tableName = zc.get(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_NAME);
       byte[] nId = zc.get(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_NAMESPACE);
-      String name = "";
+      String namespaceName = Constants.DEFAULT_NAMESPACE;
       // create fully qualified table name
       if (nId != null) {
         String namespaceId = new String(nId, Constants.UTF8);
         if (!namespaceId.equals(Constants.DEFAULT_NAMESPACE_ID)) {
           try {
-            name += Namespaces.getNamespaceName(instance, namespaceId) + ".";
+            namespaceName = namespaceIdToNameMap.get(namespaceId);
+            if (namespaceName == null) {
+              namespaceName = Namespaces.getNamespaceName(instance, namespaceId);
+              namespaceIdToNameMap.put(namespaceId, namespaceName);
+            }
           } catch (NamespaceNotFoundException e) {
             log.error("Table (" + tableId + ") contains reference to namespace (" + namespaceId + ") that doesn't exist");
             continue;
           }
         }
       }
-      if (tblPath != null) {
-        name += new String(tblPath, Constants.UTF8);
+      if (tableName != null) {
+        String tableNameStr = qualified(new String(tableName, Constants.UTF8), namespaceName);
         if (nameAsKey)
-          tableMap.put(name, tableId);
+          tableMap.put(tableNameStr, tableId);
         else
-          tableMap.put(tableId, name);
+          tableMap.put(tableId, tableNameStr);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/859cf208/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateNamespaceCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateNamespaceCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateNamespaceCommand.java
index 908a9cc..d59f2ef 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateNamespaceCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateNamespaceCommand.java
@@ -35,7 +35,6 @@ import org.apache.commons.cli.Options;
 
 public class CreateNamespaceCommand extends Command {
   private Option createTableOptCopyConfig, createNamespaceOptCopyConfig;
-  private Option base64Opt;
 
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException,
@@ -93,8 +92,6 @@ public class CreateNamespaceCommand extends Command {
     createTableOptCopyConfig = new Option("ctc", "copy-table-config", true, "table to copy configuration from");
     createTableOptCopyConfig.setArgName("tableName");
 
-    base64Opt = new Option("b64", "base64encoded", false, "decode encoded split points");
-    o.addOption(base64Opt);
     OptionGroup ogp = new OptionGroup();
     ogp.addOption(createTableOptCopyConfig);
     ogp.addOption(createNamespaceOptCopyConfig);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/859cf208/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameNamespace.java
index bf22ad6..5aa21ed 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameNamespace.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameNamespace.java
@@ -46,7 +46,7 @@ public class RenameNamespace extends Test {
     } catch (NamespaceExistsException e) {
       log.debug("Rename namespace " + srcName + " failed, " + newName + " exists");
     } catch (NamespaceNotFoundException e) {
-      log.debug("Rename namespace " + srcName + " failed, doesnt exist");
+      log.debug("Rename namespace " + srcName + " failed, doesn't exist");
     }
   }
 }


[44/50] [abbrv] git commit: ACCUMULO-1907 Prevent rename across namespace

Posted by ct...@apache.org.
ACCUMULO-1907 Prevent rename across namespace

  Moving between namespaces is restricted to clone, followed by a
  deleted, which has more predictable behavior.


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

Branch: refs/heads/master
Commit: cd6e1852b51d023718066d4e34c07dce4cf5b483
Parents: 3c92094
Author: Christopher Tubbs <ct...@apache.org>
Authored: Tue Nov 26 20:00:02 2013 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:11 2013 -0500

----------------------------------------------------------------------
 .../core/client/admin/TableOperationsImpl.java  |   2 +-
 .../accumulo/master/tableOps/RenameTable.java   |  25 +--
 .../org/apache/accumulo/test/NamespacesIT.java  | 166 ++++++++-----------
 3 files changed, 77 insertions(+), 116 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/cd6e1852/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
index c398e6e..b21aa31 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
@@ -779,7 +779,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       throw new IllegalArgumentException(new NamespaceNotFoundException(null, namespace, info));
     }
 
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldTableName.getBytes()), ByteBuffer.wrap(newTableName.getBytes()));
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldTableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(newTableName.getBytes(Constants.UTF8)));
     Map<String,String> opts = new HashMap<String,String>();
     doTableOperation(TableOperation.RENAME, args, opts);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cd6e1852/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
index a0ddb8d..9fa736d 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
@@ -30,7 +30,6 @@ import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter.Mutator;
 import org.apache.accumulo.master.Master;
 import org.apache.accumulo.server.client.HdfsZooInstance;
-import org.apache.accumulo.server.tables.TableManager;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.log4j.Logger;
 
@@ -40,14 +39,11 @@ public class RenameTable extends MasterRepo {
   private String tableId;
   private String oldTableName;
   private String newTableName;
-  private String oldNamespaceId;
-  private String newNamespaceId;
+  private String namespaceId;
 
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    return Utils.reserveNamespace(oldNamespaceId, tid, false, true, TableOperation.RENAME)
-        + Utils.reserveNamespace(newNamespaceId, tid, false, true, TableOperation.RENAME)
-        + Utils.reserveTable(tableId, tid, true, true, TableOperation.RENAME);
+    return Utils.reserveNamespace(namespaceId, tid, false, true, TableOperation.RENAME) + Utils.reserveTable(tableId, tid, true, true, TableOperation.RENAME);
   }
 
   public RenameTable(String tableId, String oldTableName, String newTableName) throws NamespaceNotFoundException {
@@ -55,19 +51,16 @@ public class RenameTable extends MasterRepo {
     this.oldTableName = oldTableName;
     this.newTableName = newTableName;
     Instance inst = HdfsZooInstance.getInstance();
-    this.oldNamespaceId = Tables.getNamespace(inst, tableId);
-    this.newNamespaceId = Namespaces.getNamespaceId(inst, Tables.extractNamespace(newTableName));
+    this.namespaceId = Tables.getNamespace(inst, tableId);
   }
 
   @Override
   public Repo<Master> call(long tid, Master master) throws Exception {
 
     Instance instance = master.getInstance();
-
-    if (!newNamespaceId.equals(oldNamespaceId)) {
-      TableManager tm = TableManager.getInstance();
-      tm.addNamespaceToTable(tableId, newNamespaceId);
-    }
+    // ensure no attempt is made to rename across namespaces
+    if (newTableName.contains(".") && !namespaceId.equals(Namespaces.getNamespaceId(instance, Tables.extractNamespace(newTableName))))
+      throw new IllegalArgumentException("Namespace in new table name does not match the old table name");
 
     IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
 
@@ -97,8 +90,7 @@ public class RenameTable extends MasterRepo {
     } finally {
       Utils.tableNameLock.unlock();
       Utils.unreserveTable(tableId, tid, true);
-      Utils.unreserveNamespace(this.oldNamespaceId, tid, false);
-      Utils.unreserveNamespace(this.newNamespaceId, tid, false);
+      Utils.unreserveNamespace(this.namespaceId, tid, false);
     }
 
     Logger.getLogger(RenameTable.class).debug("Renamed table " + tableId + " " + oldTableName + " " + newTableName);
@@ -108,9 +100,8 @@ public class RenameTable extends MasterRepo {
 
   @Override
   public void undo(long tid, Master env) throws Exception {
-    Utils.unreserveNamespace(newNamespaceId, tid, false);
-    Utils.unreserveNamespace(oldNamespaceId, tid, false);
     Utils.unreserveTable(tableId, tid, true);
+    Utils.unreserveNamespace(namespaceId, tid, false);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cd6e1852/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java b/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
index 5695c51..8341762 100644
--- a/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
@@ -53,31 +53,12 @@ import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.examples.simple.constraints.NumericValueConstraint;
-import org.apache.accumulo.minicluster.MiniAccumuloCluster;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.apache.accumulo.test.functional.SimpleMacIT;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
 
-public class NamespacesIT {
+public class NamespacesIT extends SimpleMacIT {
 
   Random random = new Random();
-  public static TemporaryFolder folder = new TemporaryFolder();
-  static private MiniAccumuloCluster accumulo;
-  static private String secret = "secret";
-
-  @BeforeClass
-  static public void setUp() throws Exception {
-    folder.create();
-    accumulo = new MiniAccumuloCluster(folder.getRoot(), secret);
-    accumulo.start();
-  }
-
-  @AfterClass
-  static public void tearDown() throws Exception {
-    accumulo.stop();
-    folder.delete();
-  }
 
   /**
    * This test creates a table without specifying a namespace. In this case, it puts the table into the default namespace.
@@ -85,7 +66,7 @@ public class NamespacesIT {
   @Test
   public void testDefaultNamespace() throws Exception {
     String tableName = "test";
-    Connector c = accumulo.getConnector("root", secret);
+    Connector c = getConnector();
 
     assertTrue(c.namespaceOperations().exists(Constants.DEFAULT_NAMESPACE));
     c.tableOperations().create(tableName);
@@ -103,7 +84,7 @@ public class NamespacesIT {
     String tableName1 = namespace + ".table1";
     String tableName2 = namespace + ".table2";
 
-    Connector c = accumulo.getConnector("root", secret);
+    Connector c = getConnector();
 
     c.namespaceOperations().create(namespace);
     assertTrue(c.namespaceOperations().exists(namespace));
@@ -154,7 +135,7 @@ public class NamespacesIT {
     String propKey = Property.TABLE_SCAN_MAXMEM.getKey();
     String propVal = "42K";
 
-    Connector c = accumulo.getConnector("root", secret);
+    Connector c = getConnector();
 
     c.namespaceOperations().create(namespace);
     c.tableOperations().create(tableName1);
@@ -213,29 +194,24 @@ public class NamespacesIT {
    * 
    */
   @Test
-  public void testRenameAndCloneTableToNewNamespace() throws Exception {
-    String namespace1 = "renamed";
-    String namespace2 = "cloned";
-    String tableName = "table";
-    String tableName1 = "renamed.table1";
-    String tableName2 = "cloned.table2";
+  public void testCloneTableToNewNamespace() throws Exception {
+    Connector c = getConnector();
 
-    Connector c = accumulo.getConnector("root", secret);
+    String[] uniqueNames = getTableNames(2);
+    String namespace1 = uniqueNames[0];
+    String namespace2 = uniqueNames[1];
+    String tableName1 = namespace1 + ".table1";
+    String tableName2 = namespace2 + ".table2";
 
-    c.tableOperations().create(tableName);
     c.namespaceOperations().create(namespace1);
-    c.namespaceOperations().create(namespace2);
-
-    c.tableOperations().rename(tableName, tableName1);
-
+    c.tableOperations().create(tableName1);
     assertTrue(c.tableOperations().exists(tableName1));
-    assertTrue(!c.tableOperations().exists(tableName));
 
+    c.namespaceOperations().create(namespace2);
     c.tableOperations().clone(tableName1, tableName2, false, null, null);
 
     assertTrue(c.tableOperations().exists(tableName1));
     assertTrue(c.tableOperations().exists(tableName2));
-    return;
   }
 
   /**
@@ -247,7 +223,7 @@ public class NamespacesIT {
     String namespace2 = "n2";
     String table = "t";
 
-    Connector c = accumulo.getConnector("root", secret);
+    Connector c = getConnector();
     Instance instance = c.getInstance();
 
     c.namespaceOperations().create(namespace1);
@@ -269,8 +245,9 @@ public class NamespacesIT {
    */
   @Test
   public void testCloneTableProperties() throws Exception {
-    String n1 = "namespace1";
-    String n2 = "namespace2";
+    String[] uniqueNames = getTableNames(2);
+    String n1 = uniqueNames[0];
+    String n2 = uniqueNames[1];
     String t1 = n1 + ".table";
     String t2 = n2 + ".table";
 
@@ -278,7 +255,7 @@ public class NamespacesIT {
     String propVal1 = "55";
     String propVal2 = "66";
 
-    Connector c = accumulo.getConnector("root", secret);
+    Connector c = getConnector();
 
     c.namespaceOperations().create(n1);
     c.tableOperations().create(t1);
@@ -295,6 +272,8 @@ public class NamespacesIT {
 
     assertTrue(checkTableHasProp(c, t2, propKey, propVal2));
 
+    c.tableOperations().delete(t1);
+    c.tableOperations().delete(t2);
     c.namespaceOperations().delete(n1);
     c.namespaceOperations().delete(n2);
   }
@@ -304,7 +283,7 @@ public class NamespacesIT {
    */
   @Test
   public void testNamespaceIteratorsAndConstraints() throws Exception {
-    Connector c = accumulo.getConnector("root", secret);
+    Connector c = getConnector();
 
     String namespace = "iterator";
     String tableName = namespace + ".table";
@@ -349,42 +328,31 @@ public class NamespacesIT {
   }
 
   /**
-   * Tests that when a table moves to a new namespace that it's properties inherit from the new namespace and not the old one
+   * Tests disallowed rename across namespaces
    */
   @Test
-  public void testRenameToNewNamespaceProperties() throws Exception {
-    Connector c = accumulo.getConnector("root", secret);
-
-    String namespace1 = "moveToNewNamespace1";
-    String namespace2 = "moveToNewNamespace2";
-    String tableName1 = namespace1 + ".table";
-    String tableName2 = namespace2 + ".table";
+  public void testRenameTable() throws Exception {
+    Connector c = getConnector();
 
-    String propKey = Property.TABLE_FILE_MAX.getKey();
-    String propVal = "42";
+    String[] uniqueNames = getTableNames(4);
+    String namespace1 = uniqueNames[0];
+    String namespace2 = uniqueNames[1];
+    String tableName1 = namespace1 + "." + uniqueNames[2];
+    String tableName2 = namespace2 + "." + uniqueNames[3];
+    String tableName3 = namespace1 + "." + uniqueNames[3];
 
     c.namespaceOperations().create(namespace1);
     c.namespaceOperations().create(namespace2);
     c.tableOperations().create(tableName1);
 
-    c.namespaceOperations().setProperty(namespace1, propKey, propVal);
-    boolean hasProp = false;
-    for (Entry<String,String> p : c.tableOperations().getProperties(tableName1)) {
-      if (p.getKey().equals(propKey) && p.getValue().equals(propVal)) {
-        hasProp = true;
-      }
+    try {
+      c.tableOperations().rename(tableName1, tableName2);
+      fail();
+    } catch (AccumuloException e) {
+      // this is expected, because we don't allow renames across namespaces
     }
-    assertTrue(hasProp);
-
-    c.tableOperations().rename(tableName1, tableName2);
 
-    hasProp = false;
-    for (Entry<String,String> p : c.tableOperations().getProperties(tableName2)) {
-      if (p.getKey().equals(propKey) && p.getValue().equals(propVal)) {
-        hasProp = true;
-      }
-    }
-    assertTrue(!hasProp);
+    c.tableOperations().rename(tableName1, tableName3);
   }
 
   /**
@@ -393,53 +361,57 @@ public class NamespacesIT {
    */
   @Test
   public void testPermissions() throws Exception {
-    Connector c = accumulo.getConnector("root", secret);
-
-    PasswordToken pass = new PasswordToken(secret);
-
-    String n1 = "spaceOfTheName";
-
-    String user1 = "dude";
+    Connector c = getConnector();
+
+    String[] uniqueNames = getTableNames(8);
+    String user1 = uniqueNames[0];
+    String user2 = uniqueNames[1];
+    PasswordToken pass = new PasswordToken(uniqueNames[2]);
+    String n1 = uniqueNames[3];
+    String n2 = uniqueNames[4];
+    String t1 = uniqueNames[5];
+    String t2 = uniqueNames[6];
+    String t3 = uniqueNames[7];
 
     c.namespaceOperations().create(n1);
-    c.tableOperations().create(n1 + ".table1");
+    c.tableOperations().create(n1 + "." + t1);
 
     c.securityOperations().createLocalUser(user1, pass);
 
-    Connector user1Con = accumulo.getConnector(user1, secret);
+    Connector user1Con = getConnector().getInstance().getConnector(user1, pass);
 
     try {
-      user1Con.tableOperations().create(n1 + ".table2");
+      user1Con.tableOperations().create(n1 + "." + t2);
       fail();
     } catch (AccumuloSecurityException e) {
       // supposed to happen
     }
 
     c.securityOperations().grantNamespacePermission(user1, n1, NamespacePermission.CREATE_TABLE);
-    user1Con.tableOperations().create(n1 + ".table2");
-    assertTrue(c.tableOperations().list().contains(n1 + ".table2"));
+    user1Con.tableOperations().create(n1 + "." + t2);
+    assertTrue(c.tableOperations().list().contains(n1 + "." + t2));
     c.securityOperations().revokeNamespacePermission(user1, n1, NamespacePermission.CREATE_TABLE);
 
     try {
-      user1Con.tableOperations().delete(n1 + ".table1");
+      user1Con.tableOperations().delete(n1 + "." + t1);
       fail();
     } catch (AccumuloSecurityException e) {
       // should happen
     }
 
     c.securityOperations().grantNamespacePermission(user1, n1, NamespacePermission.DROP_TABLE);
-    user1Con.tableOperations().delete(n1 + ".table1");
-    assertTrue(!c.tableOperations().list().contains(n1 + ".table1"));
+    user1Con.tableOperations().delete(n1 + "." + t1);
+    assertTrue(!c.tableOperations().list().contains(n1 + "." + t1));
     c.securityOperations().revokeNamespacePermission(user1, n1, NamespacePermission.DROP_TABLE);
 
-    c.tableOperations().create(n1 + ".t");
-    BatchWriter bw = c.createBatchWriter(n1 + ".t", null);
+    c.tableOperations().create(n1 + "." + t3);
+    BatchWriter bw = c.createBatchWriter(n1 + "." + t3, null);
     Mutation m = new Mutation("row");
     m.put("cf", "cq", "value");
     bw.addMutation(m);
     bw.close();
 
-    Iterator<Entry<Key,Value>> i = user1Con.createScanner(n1 + ".t", new Authorizations()).iterator();
+    Iterator<Entry<Key,Value>> i = user1Con.createScanner(n1 + "." + t3, new Authorizations()).iterator();
     try {
       i.next();
       fail();
@@ -447,9 +419,9 @@ public class NamespacesIT {
       // yup
     }
 
-    m = new Mutation("user1");
+    m = new Mutation(user1);
     m.put("cf", "cq", "turtles");
-    bw = user1Con.createBatchWriter(n1 + ".t", null);
+    bw = user1Con.createBatchWriter(n1 + "." + t3, null);
     try {
       bw.addMutation(m);
       bw.close();
@@ -459,26 +431,26 @@ public class NamespacesIT {
     }
 
     c.securityOperations().grantNamespacePermission(user1, n1, NamespacePermission.READ);
-    i = user1Con.createScanner(n1 + ".t", new Authorizations()).iterator();
+    i = user1Con.createScanner(n1 + "." + t3, new Authorizations()).iterator();
     assertTrue(i.hasNext());
     c.securityOperations().revokeNamespacePermission(user1, n1, NamespacePermission.READ);
 
     c.securityOperations().grantNamespacePermission(user1, n1, NamespacePermission.WRITE);
-    m = new Mutation("user1");
+    m = new Mutation(user1);
     m.put("cf", "cq", "turtles");
-    bw = user1Con.createBatchWriter(n1 + ".t", null);
+    bw = user1Con.createBatchWriter(n1 + "." + t3, null);
     bw.addMutation(m);
     bw.close();
     c.securityOperations().revokeNamespacePermission(user1, n1, NamespacePermission.WRITE);
 
     try {
-      user1Con.tableOperations().setProperty(n1 + ".t", Property.TABLE_FILE_MAX.getKey(), "42");
+      user1Con.tableOperations().setProperty(n1 + "." + t3, Property.TABLE_FILE_MAX.getKey(), "42");
       fail();
     } catch (AccumuloSecurityException e) {}
 
     c.securityOperations().grantNamespacePermission(user1, n1, NamespacePermission.ALTER_TABLE);
-    user1Con.tableOperations().setProperty(n1 + ".t", Property.TABLE_FILE_MAX.getKey(), "42");
-    user1Con.tableOperations().removeProperty(n1 + ".t", Property.TABLE_FILE_MAX.getKey());
+    user1Con.tableOperations().setProperty(n1 + "." + t3, Property.TABLE_FILE_MAX.getKey(), "42");
+    user1Con.tableOperations().removeProperty(n1 + "." + t3, Property.TABLE_FILE_MAX.getKey());
     c.securityOperations().revokeNamespacePermission(user1, n1, NamespacePermission.ALTER_TABLE);
 
     try {
@@ -491,7 +463,6 @@ public class NamespacesIT {
     user1Con.namespaceOperations().removeProperty(n1, Property.TABLE_FILE_MAX.getKey());
     c.securityOperations().revokeNamespacePermission(user1, n1, NamespacePermission.ALTER_NAMESPACE);
 
-    String user2 = "guy";
     c.securityOperations().createLocalUser(user2, pass);
     try {
       user1Con.securityOperations().grantNamespacePermission(user2, n1, NamespacePermission.ALTER_NAMESPACE);
@@ -503,7 +474,6 @@ public class NamespacesIT {
     user1Con.securityOperations().revokeNamespacePermission(user2, n1, NamespacePermission.ALTER_NAMESPACE);
     c.securityOperations().revokeNamespacePermission(user1, n1, NamespacePermission.GRANT);
 
-    String n2 = "namespace2";
     try {
       user1Con.namespaceOperations().create(n2);
       fail();
@@ -538,7 +508,7 @@ public class NamespacesIT {
    */
   @Test
   public void excludeSystemIterConst() throws Exception {
-    Connector c = accumulo.getConnector("root", secret);
+    Connector c = getConnector();
 
     c.instanceOperations().setProperty("table.iterator.scan.sum", "20," + SimpleFilter.class.getName());
     assertTrue(c.instanceOperations().getSystemConfiguration().containsValue("20," + SimpleFilter.class.getName()));


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

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java
index ce72ecc..2d618be 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java
@@ -28,7 +28,7 @@ import jline.console.ConsoleReader;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.IteratorSetting;
-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.conf.Property;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
@@ -52,19 +52,19 @@ import org.apache.commons.cli.Options;
 import org.apache.commons.vfs2.FileSystemException;
 
 public class SetIterCommand extends Command {
-  
+
   private Option allScopeOpt, mincScopeOpt, majcScopeOpt, scanScopeOpt, nameOpt, priorityOpt;
   private Option aggTypeOpt, ageoffTypeOpt, regexTypeOpt, versionTypeOpt, reqvisTypeOpt, classnameTypeOpt;
-  
+
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException,
       TableNotFoundException, IOException, ShellCommandException {
-    
+
     boolean tables = cl.hasOption(OptUtil.tableOpt().getOpt()) || !shellState.getTableName().isEmpty();
-    boolean namespaces = cl.hasOption(OptUtil.tableNamespaceOpt().getOpt());
-    
+    boolean namespaces = cl.hasOption(OptUtil.namespaceOpt().getOpt());
+
     final int priority = Integer.parseInt(cl.getOptionValue(priorityOpt.getOpt()));
-    
+
     final Map<String,String> options = new HashMap<String,String>();
     String classname = cl.getOptionValue(classnameTypeOpt.getOpt());
     if (cl.hasOption(aggTypeOpt.getOpt())) {
@@ -81,56 +81,56 @@ public class SetIterCommand extends Command {
     } else if (cl.hasOption(reqvisTypeOpt.getOpt())) {
       classname = ReqVisFilter.class.getName();
     }
-    
+
     ClassLoader classloader = getClassLoader(cl, shellState);
-    
+
     final String name = cl.getOptionValue(nameOpt.getOpt(), setUpOptions(classloader, shellState.getReader(), classname, options));
-    
+
     if (namespaces) {
       try {
-        setTableNamespaceProperties(cl, shellState, priority, options, classname, name);
-      } catch (TableNamespaceNotFoundException e) {
+        setNamespaceProperties(cl, shellState, priority, options, classname, name);
+      } catch (NamespaceNotFoundException e) {
         throw new IllegalArgumentException(e);
       }
     } else if (tables) {
       setTableProperties(cl, shellState, priority, options, classname, name);
     } else {
-      throw new IllegalArgumentException("No table or table namespace specified");
+      throw new IllegalArgumentException("No table or namespace specified");
     }
     return 0;
   }
-  
+
   private ClassLoader getClassLoader(final CommandLine cl, final Shell shellState) throws AccumuloException, TableNotFoundException, AccumuloSecurityException,
       IOException, FileSystemException {
-    
+
     boolean tables = cl.hasOption(OptUtil.tableOpt().getOpt()) || !shellState.getTableName().isEmpty();
-    boolean namespaces = cl.hasOption(OptUtil.tableNamespaceOpt().getOpt());
-    
+    boolean namespaces = cl.hasOption(OptUtil.namespaceOpt().getOpt());
+
     String classpath = null;
     Iterable<Entry<String,String>> tableProps;
-    
+
     if (namespaces) {
       try {
-        tableProps = shellState.getConnector().tableNamespaceOperations().getProperties(OptUtil.getTableNamespaceOpt(cl, shellState));
-      } catch (TableNamespaceNotFoundException e) {
+        tableProps = shellState.getConnector().namespaceOperations().getProperties(OptUtil.getNamespaceOpt(cl, shellState));
+      } catch (NamespaceNotFoundException e) {
         throw new IllegalArgumentException(e);
       }
     } else if (tables) {
       tableProps = shellState.getConnector().tableOperations().getProperties(OptUtil.getTableOpt(cl, shellState));
     } else {
-      throw new IllegalArgumentException("No table or table namespace specified");
+      throw new IllegalArgumentException("No table or namespace specified");
     }
     for (Entry<String,String> entry : tableProps) {
       if (entry.getKey().equals(Property.TABLE_CLASSPATH.getKey())) {
         classpath = entry.getValue();
       }
     }
-    
+
     ClassLoader classloader;
-    
+
     if (classpath != null && !classpath.equals("")) {
       shellState.getConnector().instanceOperations().getSystemConfiguration().get(Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + classpath);
-      
+
       try {
         AccumuloVFSClassLoader.getContextManager().setContextConfig(new ContextManager.DefaultContextsConfig(new Iterable<Map.Entry<String,String>>() {
           @Override
@@ -145,25 +145,25 @@ public class SetIterCommand extends Command {
           }
         }));
       } catch (IllegalStateException ise) {}
-      
+
       classloader = AccumuloVFSClassLoader.getContextManager().getClassLoader(classpath);
     } else {
       classloader = AccumuloVFSClassLoader.getClassLoader();
     }
     return classloader;
   }
-  
+
   protected void setTableProperties(final CommandLine cl, final Shell shellState, final int priority, final Map<String,String> options, final String classname,
       final String name) throws AccumuloException, AccumuloSecurityException, ShellCommandException, TableNotFoundException {
     // remove empty values
-    
+
     final String tableName = OptUtil.getTableOpt(cl, shellState);
-    
+
     if (!shellState.getConnector().tableOperations().testClassLoad(tableName, classname, SortedKeyValueIterator.class.getName())) {
       throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Servers are unable to load " + classname + " as type "
           + SortedKeyValueIterator.class.getName());
     }
-    
+
     final String aggregatorClass = options.get("aggregatorClass");
     @SuppressWarnings("deprecation")
     String deprecatedAggregatorClassName = org.apache.accumulo.core.iterators.aggregation.Aggregator.class.getName();
@@ -171,7 +171,7 @@ public class SetIterCommand extends Command {
       throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Servers are unable to load " + aggregatorClass + " as type "
           + deprecatedAggregatorClassName);
     }
-    
+
     for (Iterator<Entry<String,String>> i = options.entrySet().iterator(); i.hasNext();) {
       final Entry<String,String> entry = i.next();
       if (entry.getValue() == null || entry.getValue().isEmpty()) {
@@ -194,27 +194,26 @@ public class SetIterCommand extends Command {
     final IteratorSetting setting = new IteratorSetting(priority, name, classname, options);
     shellState.getConnector().tableOperations().attachIterator(tableName, setting, scopes);
   }
-  
-  protected void setTableNamespaceProperties(final CommandLine cl, final Shell shellState, final int priority, final Map<String,String> options,
-      final String classname, final String name) throws AccumuloException, AccumuloSecurityException, ShellCommandException, TableNamespaceNotFoundException {
+
+  protected void setNamespaceProperties(final CommandLine cl, final Shell shellState, final int priority, final Map<String,String> options,
+      final String classname, final String name) throws AccumuloException, AccumuloSecurityException, ShellCommandException, NamespaceNotFoundException {
     // remove empty values
-    
-    final String namespace = OptUtil.getTableNamespaceOpt(cl, shellState);
-    
-    if (!shellState.getConnector().tableNamespaceOperations().testClassLoad(namespace, classname, SortedKeyValueIterator.class.getName())) {
+
+    final String namespace = OptUtil.getNamespaceOpt(cl, shellState);
+
+    if (!shellState.getConnector().namespaceOperations().testClassLoad(namespace, classname, SortedKeyValueIterator.class.getName())) {
       throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Servers are unable to load " + classname + " as type "
           + SortedKeyValueIterator.class.getName());
     }
-    
+
     final String aggregatorClass = options.get("aggregatorClass");
     @SuppressWarnings("deprecation")
     String deprecatedAggregatorClassName = org.apache.accumulo.core.iterators.aggregation.Aggregator.class.getName();
-    if (aggregatorClass != null
-        && !shellState.getConnector().tableNamespaceOperations().testClassLoad(namespace, aggregatorClass, deprecatedAggregatorClassName)) {
+    if (aggregatorClass != null && !shellState.getConnector().namespaceOperations().testClassLoad(namespace, aggregatorClass, deprecatedAggregatorClassName)) {
       throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Servers are unable to load " + aggregatorClass + " as type "
           + deprecatedAggregatorClassName);
     }
-    
+
     for (Iterator<Entry<String,String>> i = options.entrySet().iterator(); i.hasNext();) {
       final Entry<String,String> entry = i.next();
       if (entry.getValue() == null || entry.getValue().isEmpty()) {
@@ -235,9 +234,9 @@ public class SetIterCommand extends Command {
       throw new IllegalArgumentException("You must select at least one scope to configure");
     }
     final IteratorSetting setting = new IteratorSetting(priority, name, classname, options);
-    shellState.getConnector().tableNamespaceOperations().attachIterator(namespace, setting, scopes);
+    shellState.getConnector().namespaceOperations().attachIterator(namespace, setting, scopes);
   }
-  
+
   private static String setUpOptions(ClassLoader classloader, final ConsoleReader reader, final String className, final Map<String,String> options)
       throws IOException, ShellCommandException {
     String input;
@@ -265,7 +264,7 @@ public class SetIterCommand extends Command {
       msg.append("; use 'config -s' instead.");
       throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, msg.toString());
     }
-    
+
     final IteratorOptions itopts = skvi.describeOptions();
     if (itopts.getName() == null) {
       throw new IllegalArgumentException(className + " described its default distinguishing name as null");
@@ -281,9 +280,9 @@ public class SetIterCommand extends Command {
         options.remove(key);
       }
       localOptions.clear();
-      
+
       reader.println(itopts.getDescription());
-      
+
       String prompt;
       if (itopts.getNamedOptions() != null) {
         for (Entry<String,String> e : itopts.getNamedOptions().entrySet()) {
@@ -302,14 +301,14 @@ public class SetIterCommand extends Command {
           localOptions.put(e.getKey(), input);
         }
       }
-      
+
       if (itopts.getUnnamedOptionDescriptions() != null) {
         for (String desc : itopts.getUnnamedOptionDescriptions()) {
           reader.println(Shell.repeat("-", 10) + "> entering options: " + desc);
           input = "start";
           while (true) {
             prompt = Shell.repeat("-", 10) + "> set " + shortClassName + " option (<name> <value>, hit enter to skip): ";
-            
+
             reader.flush();
             input = reader.readLine(prompt);
             if (input == null) {
@@ -318,45 +317,45 @@ public class SetIterCommand extends Command {
             } else {
               input = new String(input);
             }
-            
+
             if (input.length() == 0)
               break;
-            
+
             String[] sa = input.split(" ", 2);
             localOptions.put(sa[0], sa[1]);
           }
         }
       }
-      
+
       options.putAll(localOptions);
       if (!skvi.validateOptions(options))
         reader.println("invalid options for " + clazz.getName());
-      
+
     } while (!skvi.validateOptions(options));
     return itopts.getName();
   }
-  
+
   @Override
   public String description() {
-    return "sets a table-specific or table-namespace-specific iterator";
+    return "sets a table-specific or namespace-specific iterator";
   }
-  
+
   @Override
   public Options getOptions() {
     final Options o = new Options();
-    
+
     priorityOpt = new Option("p", "priority", true, "the order in which the iterator is applied");
     priorityOpt.setArgName("pri");
     priorityOpt.setRequired(true);
-    
+
     nameOpt = new Option("n", "name", true, "iterator to set");
     nameOpt.setArgName("itername");
-    
+
     allScopeOpt = new Option("all", "all-scopes", false, "applied at scan time, minor and major compactions");
     mincScopeOpt = new Option(IteratorScope.minc.name(), "minor-compaction", false, "applied at minor compaction");
     majcScopeOpt = new Option(IteratorScope.majc.name(), "major-compaction", false, "applied at major compaction");
     scanScopeOpt = new Option(IteratorScope.scan.name(), "scan-time", false, "applied at scan time");
-    
+
     final OptionGroup typeGroup = new OptionGroup();
     classnameTypeOpt = new Option("class", "class-name", true, "a java class that implements SortedKeyValueIterator");
     classnameTypeOpt.setArgName("name");
@@ -365,7 +364,7 @@ public class SetIterCommand extends Command {
     versionTypeOpt = new Option("vers", "version", false, "a versioning iterator");
     reqvisTypeOpt = new Option("reqvis", "require-visibility", false, "an iterator that omits entries with empty visibilities");
     ageoffTypeOpt = new Option("ageoff", "ageoff", false, "an aging off iterator");
-    
+
     typeGroup.addOption(classnameTypeOpt);
     typeGroup.addOption(aggTypeOpt);
     typeGroup.addOption(regexTypeOpt);
@@ -373,11 +372,11 @@ public class SetIterCommand extends Command {
     typeGroup.addOption(reqvisTypeOpt);
     typeGroup.addOption(ageoffTypeOpt);
     typeGroup.setRequired(true);
-    
+
     final OptionGroup tableGroup = new OptionGroup();
     tableGroup.addOption(OptUtil.tableOpt("table to configure iterators on"));
-    tableGroup.addOption(OptUtil.tableNamespaceOpt("tableNamespace to configure iterators on"));
-    
+    tableGroup.addOption(OptUtil.namespaceOpt("namespace to configure iterators on"));
+
     o.addOption(priorityOpt);
     o.addOption(nameOpt);
     o.addOption(allScopeOpt);
@@ -388,7 +387,7 @@ public class SetIterCommand extends Command {
     o.addOptionGroup(tableGroup);
     return o;
   }
-  
+
   @Override
   public int numArgs() {
     return 0;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/commands/TableOperation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/TableOperation.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/TableOperation.java
index e2ec71f..27946b8 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/TableOperation.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/TableOperation.java
@@ -23,7 +23,7 @@ import java.util.TreeSet;
 
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.TableNotFoundException;
-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.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
@@ -34,11 +34,12 @@ import org.apache.commons.cli.OptionGroup;
 import org.apache.commons.cli.Options;
 
 public abstract class TableOperation extends Command {
-  
-  protected Option optTablePattern, optTableName, optTableNamespace;
+
+  protected Option optTablePattern, optTableName, optNamespace;
   private boolean force = true;
   private boolean useCommandLine = true;
-  
+
+  @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws Exception {
     // populate the tableSet set with the tables you want to operate on
     final SortedSet<String> tableSet = new TreeSet<String>();
@@ -49,10 +50,10 @@ public abstract class TableOperation extends Command {
         }
     } else if (cl.hasOption(optTableName.getOpt())) {
       tableSet.add(cl.getOptionValue(optTableName.getOpt()));
-    } else if (cl.hasOption(optTableNamespace.getOpt())) {
+    } else if (cl.hasOption(optNamespace.getOpt())) {
       Instance instance = shellState.getInstance();
-      String namespaceId = TableNamespaces.getNamespaceId(instance, cl.getOptionValue(optTableNamespace.getOpt()));
-      for (String tableId : TableNamespaces.getTableIds(instance, namespaceId)) {
+      String namespaceId = Namespaces.getNamespaceId(instance, cl.getOptionValue(optNamespace.getOpt()));
+      for (String tableId : Namespaces.getTableIds(instance, namespaceId)) {
         tableSet.add(Tables.getTableName(instance, tableId));
       }
     } else if (useCommandLine && cl.getArgs().length > 0) {
@@ -63,10 +64,10 @@ public abstract class TableOperation extends Command {
       shellState.checkTableState();
       tableSet.add(shellState.getTableName());
     }
-    
+
     if (tableSet.isEmpty())
       Shell.log.warn("No tables found that match your criteria");
-    
+
     boolean more = true;
     // flush the tables
     for (String tableName : tableSet) {
@@ -87,63 +88,63 @@ public abstract class TableOperation extends Command {
         doTableOp(shellState, tableName);
       }
     }
-    
+
     return 0;
   }
-  
+
   protected abstract void doTableOp(Shell shellState, String tableName) throws Exception;
-  
+
   @Override
   public String description() {
     return "makes a best effort to flush tables from memory to disk";
   }
-  
+
   @Override
   public Options getOptions() {
     final Options o = new Options();
-    
+
     optTablePattern = new Option("p", "pattern", true, "regex pattern of table names to operate on");
     optTablePattern.setArgName("pattern");
-    
+
     optTableName = new Option(Shell.tableOption, "table", true, "name of a table to operate on");
     optTableName.setArgName("tableName");
-    
-    optTableNamespace = new Option(Shell.tableNamespaceOption, "table-namespace", true, "name of a table namespace to operate on");
-    optTableNamespace.setArgName("tableNamespace");
-    
+
+    optNamespace = new Option(Shell.namespaceOption, "namespace", true, "name of a namespace to operate on");
+    optNamespace.setArgName("namespace");
+
     final OptionGroup opg = new OptionGroup();
-    
+
     opg.addOption(optTablePattern);
     opg.addOption(optTableName);
-    opg.addOption(optTableNamespace);
-    
+    opg.addOption(optNamespace);
+
     o.addOptionGroup(opg);
-    
+
     return o;
   }
-  
+
   @Override
   public int numArgs() {
     return useCommandLine ? Shell.NO_FIXED_ARG_LENGTH_CHECK : 0;
   }
-  
+
   protected void force() {
     force = true;
   }
-  
+
   protected void noForce() {
     force = false;
   }
-  
+
   protected void disableUnflaggedTableOptions() {
     useCommandLine = false;
   }
-  
+
   @Override
   public String usage() {
     return getName() + " [<table>{ <table>}]";
   }
-  
+
   @Override
   public void registerCompletion(final Token root, final Map<Command.CompletionSet,Set<String>> special) {
     if (useCommandLine)

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/commands/TablesCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/TablesCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/TablesCommand.java
index 5d9b087..5cdf62f 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/TablesCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/TablesCommand.java
@@ -26,8 +26,8 @@ import java.util.TreeMap;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-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.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
 import org.apache.commons.cli.CommandLine;
@@ -46,15 +46,15 @@ public class TablesCommand extends Command {
   @SuppressWarnings("unchecked")
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException, IOException,
-      TableNamespaceNotFoundException {
+      NamespaceNotFoundException {
 
     final Iterator<String> tableNames;
     final Iterator<String> tableIds;
 
-    if (cl.hasOption(OptUtil.tableNamespaceOpt().getOpt())) {
-      String namespace = shellState.getConnector().tableNamespaceOperations().namespaceIdMap().get(OptUtil.getTableNamespaceOpt(cl, shellState));
-      tableNames = TableNamespaces.getTableNames(shellState.getConnector().getInstance(), namespace).iterator();
-      List<String> tableIdStrings = TableNamespaces.getTableIds(shellState.getConnector().getInstance(), namespace);
+    if (cl.hasOption(OptUtil.namespaceOpt().getOpt())) {
+      String namespace = shellState.getConnector().namespaceOperations().namespaceIdMap().get(OptUtil.getNamespaceOpt(cl, shellState));
+      tableNames = Namespaces.getTableNames(shellState.getConnector().getInstance(), namespace).iterator();
+      List<String> tableIdStrings = Namespaces.getTableIds(shellState.getConnector().getInstance(), namespace);
       if (cl.hasOption(sortByTableIdOption.getOpt()))
         Collections.sort(tableIdStrings);
       tableIds = tableIdStrings.iterator();
@@ -111,7 +111,7 @@ public class TablesCommand extends Command {
     o.addOption(sortByTableIdOption);
     disablePaginationOpt = new Option("np", "no-pagination", false, "disable pagination of output");
     o.addOption(disablePaginationOpt);
-    o.addOption(OptUtil.tableNamespaceOpt("name of table namespace to list only its tables"));
+    o.addOption(OptUtil.namespaceOpt("name of namespace to list only its tables"));
     return o;
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserPermissionsCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserPermissionsCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserPermissionsCommand.java
index 1b6377a..48b1853 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserPermissionsCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserPermissionsCommand.java
@@ -20,8 +20,8 @@ import java.io.IOException;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+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.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
@@ -32,11 +32,11 @@ import org.apache.commons.cli.Options;
 public class UserPermissionsCommand extends Command {
   private Option userOpt;
   private static int runOnce = 0;
-  
+
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException, IOException {
     final String user = cl.getOptionValue(userOpt.getOpt(), shellState.getConnector().whoami());
-    
+
     String delim = "";
     shellState.getReader().print("System permissions: ");
     for (SystemPermission p : SystemPermission.values()) {
@@ -46,13 +46,13 @@ public class UserPermissionsCommand extends Command {
       }
     }
     shellState.getReader().println();
-    
-    for (String n : shellState.getConnector().tableNamespaceOperations().list()) {
+
+    for (String n : shellState.getConnector().namespaceOperations().list()) {
       delim = "";
-      for (TableNamespacePermission p : TableNamespacePermission.values()) {
-        if (p != null && shellState.getConnector().securityOperations().hasTableNamespacePermission(user, n, p)) {
+      for (NamespacePermission p : NamespacePermission.values()) {
+        if (p != null && shellState.getConnector().securityOperations().hasNamespacePermission(user, n, p)) {
           if (runOnce == 0) {
-            shellState.getReader().print("\nTable Namespace permissions (" + n + "): ");
+            shellState.getReader().print("\nNamespace permissions (" + n + "): ");
             runOnce++;
           }
           shellState.getReader().print(delim + "Namespace." + p.name());
@@ -62,7 +62,7 @@ public class UserPermissionsCommand extends Command {
       runOnce = 0;
     }
     shellState.getReader().println();
-    
+
     for (String t : shellState.getConnector().tableOperations().list()) {
       delim = "";
       for (TablePermission p : TablePermission.values()) {
@@ -74,20 +74,20 @@ public class UserPermissionsCommand extends Command {
           shellState.getReader().print(delim + "Table." + p.name());
           delim = ", ";
         }
-        
+
       }
       runOnce = 0;
     }
     shellState.getReader().println();
-    
+
     return 0;
   }
-  
+
   @Override
   public String description() {
-    return "displays a user's system, table, and table namespace permissions";
+    return "displays a user's system, table, and namespace permissions";
   }
-  
+
   @Override
   public Options getOptions() {
     Options o = new Options();
@@ -96,7 +96,7 @@ public class UserPermissionsCommand extends Command {
     o.addOption(userOpt);
     return o;
   }
-  
+
   @Override
   public int numArgs() {
     return 0;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/thrift/client.thrift
----------------------------------------------------------------------
diff --git a/core/src/main/thrift/client.thrift b/core/src/main/thrift/client.thrift
index 7c340d7..601a7b2 100644
--- a/core/src/main/thrift/client.thrift
+++ b/core/src/main/thrift/client.thrift
@@ -75,7 +75,7 @@ enum SecurityErrorCode {
     TOKEN_EXPIRED = 15,
     SERIALIZATION_ERROR = 16,
     INSUFFICIENT_PROPERTIES = 17,
-    TABLE_NAMESPACE_DOESNT_EXIST = 18;
+    NAMESPACE_DOESNT_EXIST = 18;
 }
 
 exception ThriftSecurityException {
@@ -128,21 +128,21 @@ service ClientService {
     // permissions-related methods
     bool hasSystemPermission(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:byte sysPerm) throws (1:ThriftSecurityException sec)
     bool hasTablePermission(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:string tableName, 4:byte tblPerm) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
-    bool hasTableNamespacePermission(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string principal, 4:string tableNamespace, 5:byte tblNspcPerm) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+    bool hasNamespacePermission(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string principal, 4:string ns, 5:byte tblNspcPerm) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
     void grantSystemPermission(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:byte permission) throws (1:ThriftSecurityException sec)
     void revokeSystemPermission(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:byte permission) throws (1:ThriftSecurityException sec)
     void grantTablePermission(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:string tableName, 4:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
     void revokeTablePermission(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:string tableName, 4:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
-    void grantTableNamespacePermission(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string principal, 4:string tableNamespace, 5:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
-    void revokeTableNamespacePermission(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string principal, 4:string tableNamespace, 5:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+    void grantNamespacePermission(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string principal, 4:string ns, 5:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+    void revokeNamespacePermission(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string principal, 4:string ns, 5:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
 
     // configuration methods
     map<string, string> getConfiguration(2:trace.TInfo tinfo, 3:security.TCredentials credentials, 1:ConfigurationType type);
     map<string, string> getTableConfiguration(1:trace.TInfo tinfo, 3:security.TCredentials credentials, 2:string tableName) throws (1:ThriftTableOperationException tope);
-    map<string, string> getTableNamespaceConfiguration(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string ns) throws (1:ThriftTableOperationException tope);
+    map<string, string> getNamespaceConfiguration(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string ns) throws (1:ThriftTableOperationException tope);
     bool checkClass(1:trace.TInfo tinfo, 4:security.TCredentials credentials, 2:string className, 3:string interfaceMatch);
     bool checkTableClass(1:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string tableId, 3:string className, 4:string interfaceMatch) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope);
-    bool checkTableNamespaceClass(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string namespaceId, 4:string className, 5:string interfaceMatch) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope);
+    bool checkNamespaceClass(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string namespaceId, 4:string className, 5:string interfaceMatch) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope);
 }
 
 // Only used for a unit test

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/thrift/master.thrift
----------------------------------------------------------------------
diff --git a/core/src/main/thrift/master.thrift b/core/src/main/thrift/master.thrift
index 6270f33..1ec5c33 100644
--- a/core/src/main/thrift/master.thrift
+++ b/core/src/main/thrift/master.thrift
@@ -138,8 +138,8 @@ service MasterClientService {
     void setTableProperty(5:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string tableName, 3:string property, 4:string value) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
     void removeTableProperty(4:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string tableName, 3:string property) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
     
-    void setTableNamespaceProperty(5:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string ns, 3:string property, 4:string value) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
-    void removeTableNamespaceProperty(4:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string ns, 3:string property) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
+    void setNamespaceProperty(5:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string ns, 3:string property, 4:string value) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
+    void removeNamespaceProperty(4:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:string ns, 3:string property) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
 
     // system management methods
     void setMasterGoalState(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:MasterGoalState state) throws (1:client.ThriftSecurityException sec);
@@ -161,10 +161,10 @@ service MasterClientService {
    string waitForTableOperation(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:i64 opid) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
    void finishTableOperation(3:trace.TInfo tinfo, 1:security.TCredentials credentials, 2:i64 opid) throws (1:client.ThriftSecurityException sec)
 
-   //table namespace operations
-   i64 beginTableNamespaceOperation(1:trace.TInfo tinfo, 2:security.TCredentials credentials) throws (1:client.ThriftSecurityException sec)
-   void executeTableNamespaceOperation(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:i64 opid, 4:TableOperation op, 5:list<binary> arguments, 6:map<string, string> options, 7:bool autoClean) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
-   string waitForTableNamespaceOperation(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:i64 opid) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
-   void finishTableNamespaceOperation(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:i64 opid) throws (1:client.ThriftSecurityException sec)
+   //namespace operations
+   i64 beginNamespaceOperation(1:trace.TInfo tinfo, 2:security.TCredentials credentials) throws (1:client.ThriftSecurityException sec)
+   void executeNamespaceOperation(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:i64 opid, 4:TableOperation op, 5:list<binary> arguments, 6:map<string, string> options, 7:bool autoClean) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
+   string waitForNamespaceOperation(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:i64 opid) throws (1:client.ThriftSecurityException sec, 2:client.ThriftTableOperationException tope)
+   void finishNamespaceOperation(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:i64 opid) throws (1:client.ThriftSecurityException sec)
    
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java b/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java
new file mode 100644
index 0000000..cbca7c3
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java
@@ -0,0 +1,313 @@
+/*
+ * 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.core.client.mock;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.Map.Entry;
+import java.util.Random;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.NamespaceNotEmptyException;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.NamespaceOperations;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.Filter;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.security.Authorizations;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class MockNamespacesTest {
+
+  Random random = new Random();
+  public static TemporaryFolder folder = new TemporaryFolder();
+
+  /**
+   * This test creates a table without specifying a namespace. In this case, it puts the table into the default namespace.
+   * 
+   * @throws Exception
+   */
+  @Test
+  public void testDefaultNamespace() throws Exception {
+    String tableName = "test";
+    Instance instance = new MockInstance("default");
+    Connector c = instance.getConnector("user", new PasswordToken("pass"));
+
+    assertTrue(c.namespaceOperations().exists(Constants.DEFAULT_NAMESPACE));
+    c.tableOperations().create(tableName);
+    assertTrue(c.tableOperations().exists(tableName));
+  }
+
+  /**
+   * This test creates a new namespace "testing" and a table "testing.table1" which puts "table1" into the "testing" namespace. Then we create "testing.table2"
+   * which creates "table2" and puts it into "testing" as well. Then we make sure that you can't delete a namespace with tables in it, and then we delete the
+   * tables and delete the namespace.
+   * 
+   * @throws Exception
+   */
+  @Test
+  public void testCreateAndDeleteNamespace() throws Exception {
+    String namespace = "testing";
+    String tableName1 = namespace + ".table1";
+    String tableName2 = namespace + ".table2";
+
+    Instance instance = new MockInstance("createdelete");
+    Connector c = instance.getConnector("user", new PasswordToken("pass"));
+
+    c.namespaceOperations().create(namespace);
+    assertTrue(c.namespaceOperations().exists(namespace));
+
+    c.tableOperations().create(tableName1);
+    assertTrue(c.tableOperations().exists(tableName1));
+
+    c.tableOperations().create(tableName2);
+    assertTrue(c.tableOperations().exists(tableName2));
+
+    // deleting
+    try {
+      // can't delete a namespace with tables in it
+      c.namespaceOperations().delete(namespace);
+      fail();
+    } catch (NamespaceNotEmptyException e) {
+      // ignore, supposed to happen
+    }
+    assertTrue(c.namespaceOperations().exists(namespace));
+    assertTrue(c.tableOperations().exists(tableName1));
+    assertTrue(c.tableOperations().exists(tableName2));
+
+    c.tableOperations().delete(tableName2);
+    assertTrue(!c.tableOperations().exists(tableName2));
+    assertTrue(c.namespaceOperations().exists(namespace));
+
+    c.tableOperations().delete(tableName1);
+    assertTrue(!c.tableOperations().exists(tableName1));
+    c.namespaceOperations().delete(namespace);
+    assertTrue(!c.namespaceOperations().exists(namespace));
+  }
+
+  /**
+   * This test creates a namespace, modifies it's properties, and checks to make sure that those properties are applied to its tables. To do something on a
+   * namespace-wide level, use {@link NamespaceOperations}.
+   * 
+   * Checks to make sure namespace-level properties are overridden by table-level properties.
+   * 
+   * Checks to see if the default namespace's properties work as well.
+   * 
+   * @throws Exception
+   */
+
+  @Test
+  public void testNamespaceProperties() throws Exception {
+    String namespace = "propchange";
+    String tableName1 = namespace + ".table1";
+    String tableName2 = namespace + ".table2";
+
+    String propKey = Property.TABLE_SCAN_MAXMEM.getKey();
+    String propVal = "42K";
+
+    Instance instance = new MockInstance("props");
+    Connector c = instance.getConnector("user", new PasswordToken("pass"));
+
+    c.namespaceOperations().create(namespace);
+    c.tableOperations().create(tableName1);
+    c.namespaceOperations().setProperty(namespace, propKey, propVal);
+
+    // check the namespace has the property
+    assertTrue(checkNamespaceHasProp(c, namespace, propKey, propVal));
+
+    // check that the table gets it from the namespace
+    assertTrue(checkTableHasProp(c, tableName1, propKey, propVal));
+
+    // test a second table to be sure the first wasn't magical
+    // (also, changed the order, the namespace has the property already)
+    c.tableOperations().create(tableName2);
+    assertTrue(checkTableHasProp(c, tableName2, propKey, propVal));
+
+    // test that table properties override namespace properties
+    String propKey2 = Property.TABLE_FILE_MAX.getKey();
+    String propVal2 = "42";
+    String tablePropVal = "13";
+
+    c.tableOperations().setProperty(tableName2, propKey2, tablePropVal);
+    c.namespaceOperations().setProperty("propchange", propKey2, propVal2);
+
+    assertTrue(checkTableHasProp(c, tableName2, propKey2, tablePropVal));
+
+    // now check that you can change the default namespace's properties
+    propVal = "13K";
+    String tableName = "some_table";
+    c.tableOperations().create(tableName);
+    c.namespaceOperations().setProperty(Constants.DEFAULT_NAMESPACE, propKey, propVal);
+
+    assertTrue(checkTableHasProp(c, tableName, propKey, propVal));
+
+    // test the properties server-side by configuring an iterator.
+    // should not show anything with column-family = 'a'
+    String tableName3 = namespace + ".table3";
+    c.tableOperations().create(tableName3);
+
+    IteratorSetting setting = new IteratorSetting(250, "thing", SimpleFilter.class.getName());
+    c.namespaceOperations().attachIterator(namespace, setting);
+
+    BatchWriter bw = c.createBatchWriter(tableName3, new BatchWriterConfig());
+    Mutation m = new Mutation("r");
+    m.put("a", "b", new Value("abcde".getBytes()));
+    bw.addMutation(m);
+    bw.flush();
+    bw.close();
+
+    // Scanner s = c.createScanner(tableName3, Authorizations.EMPTY);
+    // do scanners work correctly in mock?
+    // assertTrue(!s.iterator().hasNext());
+  }
+
+  /**
+   * This test renames and clones two separate table into different namespaces. different namespace.
+   * 
+   * @throws Exception
+   */
+  @Test
+  public void testRenameAndCloneTableToNewNamespace() throws Exception {
+    String namespace1 = "renamed";
+    String namespace2 = "cloned";
+    String tableName = "table";
+    String tableName1 = "renamed.table1";
+    // String tableName2 = "cloned.table2";
+
+    Instance instance = new MockInstance("renameclone");
+    Connector c = instance.getConnector("user", new PasswordToken("pass"));
+
+    c.tableOperations().create(tableName);
+    c.namespaceOperations().create(namespace1);
+    c.namespaceOperations().create(namespace2);
+
+    c.tableOperations().rename(tableName, tableName1);
+
+    assertTrue(c.tableOperations().exists(tableName1));
+    assertTrue(!c.tableOperations().exists(tableName));
+
+    // TODO implement clone in mock
+    /*
+     * c.tableOperations().clone(tableName1, tableName2, false, null, null);
+     * 
+     * assertTrue(c.tableOperations().exists(tableName1)); assertTrue(c.tableOperations().exists(tableName2));
+     */
+    return;
+  }
+
+  /**
+   * This test renames a namespace and ensures that its tables are still correct
+   */
+  @Test
+  public void testNamespaceRename() throws Exception {
+    String namespace1 = "n1";
+    String namespace2 = "n2";
+    String table = "t";
+
+    Instance instance = new MockInstance("rename");
+    Connector c = instance.getConnector("user", new PasswordToken("pass"));
+
+    c.namespaceOperations().create(namespace1);
+    c.tableOperations().create(namespace1 + "." + table);
+
+    c.namespaceOperations().rename(namespace1, namespace2);
+
+    assertTrue(!c.namespaceOperations().exists(namespace1));
+    assertTrue(c.namespaceOperations().exists(namespace2));
+    assertTrue(!c.tableOperations().exists(namespace1 + "." + table));
+    assertTrue(c.tableOperations().exists(namespace2 + "." + table));
+  }
+
+  /**
+   * This tests adding iterators to a namespace, listing them, and removing them
+   */
+  @Test
+  public void testNamespaceIterators() throws Exception {
+    Instance instance = new MockInstance("Iterators");
+    Connector c = instance.getConnector("user", new PasswordToken("pass"));
+
+    String namespace = "iterator";
+    String tableName = namespace + ".table";
+    String iter = "thing";
+
+    c.namespaceOperations().create(namespace);
+    c.tableOperations().create(tableName);
+
+    IteratorSetting setting = new IteratorSetting(250, iter, SimpleFilter.class.getName());
+    HashSet<IteratorScope> scope = new HashSet<IteratorScope>();
+    scope.add(IteratorScope.scan);
+    c.namespaceOperations().attachIterator(namespace, setting, EnumSet.copyOf(scope));
+
+    BatchWriter bw = c.createBatchWriter(tableName, new BatchWriterConfig());
+    Mutation m = new Mutation("r");
+    m.put("a", "b", new Value("abcde".getBytes(Constants.UTF8)));
+    bw.addMutation(m);
+    bw.flush();
+
+    Scanner s = c.createScanner(tableName, Authorizations.EMPTY);
+    System.out.println(s.iterator().next());
+    // do scanners work correctly in mock?
+    // assertTrue(!s.iterator().hasNext());
+
+    assertTrue(c.namespaceOperations().listIterators(namespace).containsKey(iter));
+    c.namespaceOperations().removeIterator(namespace, iter, EnumSet.copyOf(scope));
+  }
+
+  private boolean checkTableHasProp(Connector c, String t, String propKey, String propVal) throws AccumuloException, TableNotFoundException {
+    for (Entry<String,String> e : c.tableOperations().getProperties(t)) {
+      if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private boolean checkNamespaceHasProp(Connector c, String n, String propKey, String propVal) throws AccumuloException, NamespaceNotFoundException {
+    for (Entry<String,String> e : c.namespaceOperations().getProperties(n)) {
+      if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  public static class SimpleFilter extends Filter {
+    @Override
+    public boolean accept(Key k, Value v) {
+      if (k.getColumnFamily().toString().equals("a"))
+        return false;
+      return true;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableNamespacesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableNamespacesTest.java b/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableNamespacesTest.java
deleted file mode 100644
index 2cf0541..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableNamespacesTest.java
+++ /dev/null
@@ -1,312 +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.core.client.mock;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.util.EnumSet;
-import java.util.HashSet;
-import java.util.Map.Entry;
-import java.util.Random;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableNamespaceNotEmptyException;
-import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.iterators.Filter;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-import org.apache.accumulo.core.security.Authorizations;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-public class MockTableNamespacesTest {
-
-  Random random = new Random();
-  public static TemporaryFolder folder = new TemporaryFolder();
-
-  /**
-   * This test creates a table without specifying a namespace. In this case, it puts the table into the default namespace.
-   * 
-   * @throws Exception
-   */
-  @Test
-  public void testDefaultNamespace() throws Exception {
-    String tableName = "test";
-    Instance instance = new MockInstance("default");
-    Connector c = instance.getConnector("user", new PasswordToken("pass"));
-
-    assertTrue(c.tableNamespaceOperations().exists(Constants.DEFAULT_TABLE_NAMESPACE));
-    c.tableOperations().create(tableName);
-    assertTrue(c.tableOperations().exists(tableName));
-  }
-
-  /**
-   * This test creates a new namespace "testing" and a table "testing.table1" which puts "table1" into the "testing" namespace. Then we create "testing.table2"
-   * which creates "table2" and puts it into "testing" as well. Then we make sure that you can't delete a namespace with tables in it, and then we delete the
-   * tables and delete the namespace.
-   * 
-   * @throws Exception
-   */
-  @Test
-  public void testCreateAndDeleteNamespace() throws Exception {
-    String namespace = "testing";
-    String tableName1 = namespace + ".table1";
-    String tableName2 = namespace + ".table2";
-
-    Instance instance = new MockInstance("createdelete");
-    Connector c = instance.getConnector("user", new PasswordToken("pass"));
-
-    c.tableNamespaceOperations().create(namespace);
-    assertTrue(c.tableNamespaceOperations().exists(namespace));
-
-    c.tableOperations().create(tableName1);
-    assertTrue(c.tableOperations().exists(tableName1));
-
-    c.tableOperations().create(tableName2);
-    assertTrue(c.tableOperations().exists(tableName2));
-
-    // deleting
-    try {
-      // can't delete a namespace with tables in it
-      c.tableNamespaceOperations().delete(namespace);
-      fail();
-    } catch (TableNamespaceNotEmptyException e) {
-      // ignore, supposed to happen
-    }
-    assertTrue(c.tableNamespaceOperations().exists(namespace));
-    assertTrue(c.tableOperations().exists(tableName1));
-    assertTrue(c.tableOperations().exists(tableName2));
-
-    c.tableOperations().delete(tableName2);
-    assertTrue(!c.tableOperations().exists(tableName2));
-    assertTrue(c.tableNamespaceOperations().exists(namespace));
-
-    c.tableOperations().delete(tableName1);
-    assertTrue(!c.tableOperations().exists(tableName1));
-    c.tableNamespaceOperations().delete(namespace);
-    assertTrue(!c.tableNamespaceOperations().exists(namespace));
-  }
-
-  /**
-   * This test creates a namespace, modifies it's properties, and checks to make sure that those properties are applied to its tables. To do something on a
-   * namespace-wide level, use TableNamespaceOperations.
-   * 
-   * Checks to make sure namespace-level properties are overridden by table-level properties.
-   * 
-   * Checks to see if the default namespace's properties work as well.
-   * 
-   * @throws Exception
-   */
-
-  @Test
-  public void testNamespaceProperties() throws Exception {
-    String namespace = "propchange";
-    String tableName1 = namespace + ".table1";
-    String tableName2 = namespace + ".table2";
-
-    String propKey = Property.TABLE_SCAN_MAXMEM.getKey();
-    String propVal = "42K";
-
-    Instance instance = new MockInstance("props");
-    Connector c = instance.getConnector("user", new PasswordToken("pass"));
-
-    c.tableNamespaceOperations().create(namespace);
-    c.tableOperations().create(tableName1);
-    c.tableNamespaceOperations().setProperty(namespace, propKey, propVal);
-
-    // check the namespace has the property
-    assertTrue(checkTableNamespaceHasProp(c, namespace, propKey, propVal));
-
-    // check that the table gets it from the namespace
-    assertTrue(checkTableHasProp(c, tableName1, propKey, propVal));
-
-    // test a second table to be sure the first wasn't magical
-    // (also, changed the order, the namespace has the property already)
-    c.tableOperations().create(tableName2);
-    assertTrue(checkTableHasProp(c, tableName2, propKey, propVal));
-
-    // test that table properties override namespace properties
-    String propKey2 = Property.TABLE_FILE_MAX.getKey();
-    String propVal2 = "42";
-    String tablePropVal = "13";
-
-    c.tableOperations().setProperty(tableName2, propKey2, tablePropVal);
-    c.tableNamespaceOperations().setProperty("propchange", propKey2, propVal2);
-
-    assertTrue(checkTableHasProp(c, tableName2, propKey2, tablePropVal));
-
-    // now check that you can change the default namespace's properties
-    propVal = "13K";
-    String tableName = "some_table";
-    c.tableOperations().create(tableName);
-    c.tableNamespaceOperations().setProperty(Constants.DEFAULT_TABLE_NAMESPACE, propKey, propVal);
-
-    assertTrue(checkTableHasProp(c, tableName, propKey, propVal));
-
-    // test the properties server-side by configuring an iterator.
-    // should not show anything with column-family = 'a'
-    String tableName3 = namespace + ".table3";
-    c.tableOperations().create(tableName3);
-
-    IteratorSetting setting = new IteratorSetting(250, "thing", SimpleFilter.class.getName());
-    c.tableNamespaceOperations().attachIterator(namespace, setting);
-
-    BatchWriter bw = c.createBatchWriter(tableName3, new BatchWriterConfig());
-    Mutation m = new Mutation("r");
-    m.put("a", "b", new Value("abcde".getBytes()));
-    bw.addMutation(m);
-    bw.flush();
-    bw.close();
-
-    // Scanner s = c.createScanner(tableName3, Authorizations.EMPTY);
-    // do scanners work correctly in mock?
-    // assertTrue(!s.iterator().hasNext());
-  }
-
-  /**
-   * This test renames and clones two separate table into different namespaces. different namespace.
-   * 
-   * @throws Exception
-   */
-  @Test
-  public void testRenameAndCloneTableToNewNamespace() throws Exception {
-    String namespace1 = "renamed";
-    String namespace2 = "cloned";
-    String tableName = "table";
-    String tableName1 = "renamed.table1";
-    // String tableName2 = "cloned.table2";
-
-    Instance instance = new MockInstance("renameclone");
-    Connector c = instance.getConnector("user", new PasswordToken("pass"));
-
-    c.tableOperations().create(tableName);
-    c.tableNamespaceOperations().create(namespace1);
-    c.tableNamespaceOperations().create(namespace2);
-
-    c.tableOperations().rename(tableName, tableName1);
-
-    assertTrue(c.tableOperations().exists(tableName1));
-    assertTrue(!c.tableOperations().exists(tableName));
-
-    // TODO implement clone in mock
-    /*
-     * c.tableOperations().clone(tableName1, tableName2, false, null, null);
-     * 
-     * assertTrue(c.tableOperations().exists(tableName1)); assertTrue(c.tableOperations().exists(tableName2));
-     */
-    return;
-  }
-
-  /**
-   * This test renames a table namespace and ensures that its tables are still correct
-   */
-  @Test
-  public void testNamespaceRename() throws Exception {
-    String namespace1 = "n1";
-    String namespace2 = "n2";
-    String table = "t";
-
-    Instance instance = new MockInstance("rename");
-    Connector c = instance.getConnector("user", new PasswordToken("pass"));
-
-    c.tableNamespaceOperations().create(namespace1);
-    c.tableOperations().create(namespace1 + "." + table);
-
-    c.tableNamespaceOperations().rename(namespace1, namespace2);
-
-    assertTrue(!c.tableNamespaceOperations().exists(namespace1));
-    assertTrue(c.tableNamespaceOperations().exists(namespace2));
-    assertTrue(!c.tableOperations().exists(namespace1 + "." + table));
-    assertTrue(c.tableOperations().exists(namespace2 + "." + table));
-  }
-
-  /**
-   * This tests adding iterators to a namespace, listing them, and removing them
-   */
-  @Test
-  public void testNamespaceIterators() throws Exception {
-    Instance instance = new MockInstance("Iterators");
-    Connector c = instance.getConnector("user", new PasswordToken("pass"));
-
-    String namespace = "iterator";
-    String tableName = namespace + ".table";
-    String iter = "thing";
-
-    c.tableNamespaceOperations().create(namespace);
-    c.tableOperations().create(tableName);
-
-    IteratorSetting setting = new IteratorSetting(250, iter, SimpleFilter.class.getName());
-    HashSet<IteratorScope> scope = new HashSet<IteratorScope>();
-    scope.add(IteratorScope.scan);
-    c.tableNamespaceOperations().attachIterator(namespace, setting, EnumSet.copyOf(scope));
-
-    BatchWriter bw = c.createBatchWriter(tableName, new BatchWriterConfig());
-    Mutation m = new Mutation("r");
-    m.put("a", "b", new Value("abcde".getBytes(Constants.UTF8)));
-    bw.addMutation(m);
-    bw.flush();
-
-    Scanner s = c.createScanner(tableName, Authorizations.EMPTY);
-    System.out.println(s.iterator().next());
-    // do scanners work correctly in mock?
-    // assertTrue(!s.iterator().hasNext());
-
-    assertTrue(c.tableNamespaceOperations().listIterators(namespace).containsKey(iter));
-    c.tableNamespaceOperations().removeIterator(namespace, iter, EnumSet.copyOf(scope));
-  }
-
-  private boolean checkTableHasProp(Connector c, String t, String propKey, String propVal) throws AccumuloException, TableNotFoundException {
-    for (Entry<String,String> e : c.tableOperations().getProperties(t)) {
-      if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  private boolean checkTableNamespaceHasProp(Connector c, String n, String propKey, String propVal) throws AccumuloException, TableNamespaceNotFoundException {
-    for (Entry<String,String> e : c.tableNamespaceOperations().getProperties(n)) {
-      if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  public static class SimpleFilter extends Filter {
-    @Override
-    public boolean accept(Key k, Value v) {
-      if (k.getColumnFamily().toString().equals("a"))
-        return false;
-      return true;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
index 1deaa04..d522582 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
@@ -33,8 +33,8 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
 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.ClientService;
 import org.apache.accumulo.core.client.impl.thrift.ConfigurationType;
@@ -51,8 +51,8 @@ import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 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.conf.ServerConfiguration;
@@ -91,14 +91,14 @@ public class ClientServiceHandler implements ClientService.Iface {
     return tableId;
   }
 
-  protected String checkTableNamespaceId(String tableNamespace, TableOperation operation) throws ThriftTableOperationException {
-    String namespaceId = TableNamespaces.getNameToIdMap(instance).get(tableNamespace);
+  protected String checkNamespaceId(String namespace, TableOperation operation) throws ThriftTableOperationException {
+    String namespaceId = Namespaces.getNameToIdMap(instance).get(namespace);
     if (namespaceId == null) {
-      // maybe the table namespace exists, but the cache was not updated yet... so try to clear the cache and check again
+      // maybe the namespace exists, but the cache was not updated yet... so try to clear the cache and check again
       Tables.clearCache(instance);
-      namespaceId = TableNamespaces.getNameToIdMap(instance).get(tableNamespace);
+      namespaceId = Namespaces.getNameToIdMap(instance).get(namespace);
       if (namespaceId == null)
-        throw new ThriftTableOperationException(null, tableNamespace, operation, TableOperationExceptionType.NOTFOUND, null);
+        throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.NOTFOUND, null);
     }
     return namespaceId;
   }
@@ -186,10 +186,10 @@ public class ClientServiceHandler implements ClientService.Iface {
   }
 
   @Override
-  public void grantTableNamespacePermission(TInfo tinfo, TCredentials credentials, String user, String tableNamespace, byte permission)
-      throws ThriftSecurityException, ThriftTableOperationException {
-    String namespaceId = checkTableNamespaceId(tableNamespace, TableOperation.PERMISSION);
-    security.grantTableNamespacePermission(credentials, user, namespaceId, TableNamespacePermission.getPermissionById(permission));
+  public void grantNamespacePermission(TInfo tinfo, TCredentials credentials, String user, String ns, byte permission) throws ThriftSecurityException,
+      ThriftTableOperationException {
+    String namespaceId = checkNamespaceId(ns, TableOperation.PERMISSION);
+    security.grantNamespacePermission(credentials, user, namespaceId, NamespacePermission.getPermissionById(permission));
   }
 
   @Override
@@ -217,17 +217,17 @@ public class ClientServiceHandler implements ClientService.Iface {
   }
 
   @Override
-  public boolean hasTableNamespacePermission(TInfo tinfo, TCredentials credentials, String user, String tableNamespace, byte perm)
-      throws ThriftSecurityException, ThriftTableOperationException {
-    String namespaceId = checkTableNamespaceId(tableNamespace, TableOperation.PERMISSION);
-    return security.hasTableNamespacePermission(credentials, user, namespaceId, TableNamespacePermission.getPermissionById(perm));
+  public boolean hasNamespacePermission(TInfo tinfo, TCredentials credentials, String user, String ns, byte perm) throws ThriftSecurityException,
+      ThriftTableOperationException {
+    String namespaceId = checkNamespaceId(ns, TableOperation.PERMISSION);
+    return security.hasNamespacePermission(credentials, user, namespaceId, NamespacePermission.getPermissionById(perm));
   }
 
   @Override
-  public void revokeTableNamespacePermission(TInfo tinfo, TCredentials credentials, String user, String tableNamespace, byte permission)
-      throws ThriftSecurityException, ThriftTableOperationException {
-    String namespaceId = checkTableNamespaceId(tableNamespace, TableOperation.PERMISSION);
-    security.revokeTableNamespacePermission(credentials, user, namespaceId, TableNamespacePermission.getPermissionById(permission));
+  public void revokeNamespacePermission(TInfo tinfo, TCredentials credentials, String user, String ns, byte permission) throws ThriftSecurityException,
+      ThriftTableOperationException {
+    String namespaceId = checkNamespaceId(ns, TableOperation.PERMISSION);
+    security.revokeNamespacePermission(credentials, user, namespaceId, NamespacePermission.getPermissionById(permission));
   }
 
   @Override
@@ -355,21 +355,21 @@ public class ClientServiceHandler implements ClientService.Iface {
   }
 
   @Override
-  public boolean checkTableNamespaceClass(TInfo tinfo, TCredentials credentials, String tableNamespace, String className, String interfaceMatch)
-      throws TException, ThriftTableOperationException, ThriftSecurityException {
+  public boolean checkNamespaceClass(TInfo tinfo, TCredentials credentials, String ns, String className, String interfaceMatch) throws TException,
+      ThriftTableOperationException, ThriftSecurityException {
 
     security.authenticateUser(credentials, credentials);
 
-    String tableNamespaceId = checkTableNamespaceId(tableNamespace, null);
+    String namespaceId = checkNamespaceId(ns, null);
 
     ClassLoader loader = getClass().getClassLoader();
     Class<?> shouldMatch;
     try {
       shouldMatch = loader.loadClass(interfaceMatch);
 
-      new ServerConfiguration(instance).getTableNamespaceConfiguration(tableNamespaceId);
+      new ServerConfiguration(instance).getNamespaceConfiguration(namespaceId);
 
-      String context = new ServerConfiguration(instance).getTableNamespaceConfiguration(tableNamespaceId).get(Property.TABLE_CLASSPATH);
+      String context = new ServerConfiguration(instance).getNamespaceConfiguration(namespaceId).get(Property.TABLE_CLASSPATH);
 
       ClassLoader currentLoader;
 
@@ -422,15 +422,15 @@ public class ClientServiceHandler implements ClientService.Iface {
   }
 
   @Override
-  public Map<String,String> getTableNamespaceConfiguration(TInfo tinfo, TCredentials credentials, String ns) throws ThriftTableOperationException, TException {
+  public Map<String,String> getNamespaceConfiguration(TInfo tinfo, TCredentials credentials, String ns) throws ThriftTableOperationException, TException {
     String namespaceId;
     try {
-      namespaceId = TableNamespaces.getNamespaceId(instance, ns);
-    } catch (TableNamespaceNotFoundException e) {
-      String why = "Could not find table namespace while getting configuration.";
+      namespaceId = Namespaces.getNamespaceId(instance, ns);
+    } catch (NamespaceNotFoundException e) {
+      String why = "Could not find namespace while getting configuration.";
       throw new ThriftTableOperationException(null, ns, null, TableOperationExceptionType.NOTFOUND, why);
     }
-    AccumuloConfiguration config = ServerConfiguration.getTableNamespaceConfiguration(instance, namespaceId);
+    AccumuloConfiguration config = ServerConfiguration.getNamespaceConfiguration(instance, namespaceId);
     return conf(credentials, config);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfWatcher.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfWatcher.java b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfWatcher.java
new file mode 100644
index 0000000..7f3d73e
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfWatcher.java
@@ -0,0 +1,107 @@
+/*
+ * 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 NamespaceConfWatcher 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(NamespaceConfWatcher.class);
+  private Instance instance = null;
+
+  NamespaceConfWatcher(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.getNamespaceConfiguration(instance, namespaceId).propertyChanged(key);
+        break;
+      case NodeChildrenChanged:
+        ServerConfiguration.getNamespaceConfiguration(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/NamespaceConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
new file mode 100644
index 0000000..d300366
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
@@ -0,0 +1,174 @@
+/*
+ * 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 NamespaceConfiguration extends AccumuloConfiguration {
+  private static final Logger log = Logger.getLogger(NamespaceConfiguration.class);
+
+  private final AccumuloConfiguration parent;
+  private static ZooCache propCache = null;
+  protected String namespaceId = null;
+  protected Instance inst = null;
+  private Set<ConfigurationObserver> observers;
+
+  public NamespaceConfiguration(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_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 (NamespaceConfiguration.class) {
+        if (propCache == null)
+          propCache = new ZooCache(inst.getZooKeepers(), inst.getZooKeepersSessionTimeOut(), new NamespaceConfWatcher(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_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-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-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/ServerConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java
index 5093025..2115f3f 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java
@@ -32,8 +32,8 @@ import org.apache.accumulo.core.data.KeyExtent;
 public class ServerConfiguration {
 
   private static final Map<String,TableConfiguration> tableInstances = new HashMap<String,TableConfiguration>(1);
-  private static final Map<String,TableNamespaceConfiguration> tableNamespaceInstances = new HashMap<String,TableNamespaceConfiguration>(1);
-  private static final Map<String,TableNamespaceConfiguration> tableParentInstances = new HashMap<String,TableNamespaceConfiguration>(1);
+  private static final Map<String,NamespaceConfiguration> namespaceInstances = new HashMap<String,NamespaceConfiguration>(1);
+  private static final Map<String,NamespaceConfiguration> tableParentInstances = new HashMap<String,NamespaceConfiguration>(1);
   private static SecurityPermission CONFIGURATION_PERMISSION = new SecurityPermission("configurationPermission");
 
   public static synchronized SiteConfiguration getSiteConfiguration() {
@@ -62,10 +62,10 @@ public class ServerConfiguration {
     return getZooConfiguration(instance);
   }
 
-  public static TableNamespaceConfiguration getTableNamespaceConfigurationForTable(Instance instance, String tableId) {
+  public static NamespaceConfiguration getNamespaceConfigurationForTable(Instance instance, String tableId) {
     checkPermissions();
     synchronized (tableParentInstances) {
-      TableNamespaceConfiguration conf = tableParentInstances.get(tableId);
+      NamespaceConfiguration conf = tableParentInstances.get(tableId);
       if (conf == null) {
         conf = new TableParentConfiguration(tableId, getSystemConfiguration(instance));
         ConfigSanityCheck.validate(conf);
@@ -75,14 +75,14 @@ public class ServerConfiguration {
     }
   }
 
-  public static TableNamespaceConfiguration getTableNamespaceConfiguration(Instance instance, String namespaceId) {
+  public static NamespaceConfiguration getNamespaceConfiguration(Instance instance, String namespaceId) {
     checkPermissions();
-    synchronized (tableNamespaceInstances) {
-      TableNamespaceConfiguration conf = tableNamespaceInstances.get(namespaceId);
+    synchronized (namespaceInstances) {
+      NamespaceConfiguration conf = namespaceInstances.get(namespaceId);
       if (conf == null) {
-        conf = new TableNamespaceConfiguration(namespaceId, getSystemConfiguration(instance));
+        conf = new NamespaceConfiguration(namespaceId, getSystemConfiguration(instance));
         ConfigSanityCheck.validate(conf);
-        tableNamespaceInstances.put(namespaceId, conf);
+        namespaceInstances.put(namespaceId, conf);
       }
       return conf;
     }
@@ -93,7 +93,7 @@ public class ServerConfiguration {
     synchronized (tableInstances) {
       TableConfiguration conf = tableInstances.get(tableId);
       if (conf == null && Tables.exists(instance, tableId)) {
-        conf = new TableConfiguration(instance.getInstanceID(), tableId, getTableNamespaceConfigurationForTable(instance, tableId));
+        conf = new TableConfiguration(instance.getInstanceID(), tableId, getNamespaceConfigurationForTable(instance, tableId));
         ConfigSanityCheck.validate(conf);
         tableInstances.put(tableId, conf);
       }
@@ -108,8 +108,8 @@ public class ServerConfiguration {
   }
 
   static void removeNamespaceIdInstance(String namespaceId) {
-    synchronized (tableNamespaceInstances) {
-      tableNamespaceInstances.remove(namespaceId);
+    synchronized (namespaceInstances) {
+      namespaceInstances.remove(namespaceId);
     }
   }
 
@@ -135,8 +135,8 @@ public class ServerConfiguration {
     return getTableConfiguration(extent.getTableId().toString());
   }
 
-  public TableNamespaceConfiguration getTableNamespaceConfiguration(String namespaceId) {
-    return getTableNamespaceConfiguration(instance, namespaceId);
+  public NamespaceConfiguration getNamespaceConfiguration(String namespaceId) {
+    return getNamespaceConfiguration(instance, namespaceId);
   }
 
   public synchronized AccumuloConfiguration getConfiguration() {


[23/50] [abbrv] git commit: ACCUMULO-802 Fix exception handling in RPC call for config

Posted by ct...@apache.org.
ACCUMULO-802 Fix exception handling in RPC call for config


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

Branch: refs/heads/master
Commit: ab50556c6b8017df0e11d9dfd3592aab273dfbb1
Parents: 71bed4d
Author: Christopher Tubbs <ct...@apache.org>
Authored: Thu Nov 14 16:22:28 2013 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:11 2013 -0500

----------------------------------------------------------------------
 .../server/client/ClientServiceHandler.java     | 22 ++++++++++----------
 1 file changed, 11 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/ab50556c/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
index 3148a3d..1deaa04 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
@@ -52,8 +52,8 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.security.SystemPermission;
-import org.apache.accumulo.core.security.TablePermission;
 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.conf.ServerConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -186,12 +186,12 @@ public class ClientServiceHandler implements ClientService.Iface {
   }
 
   @Override
-  public void grantTableNamespacePermission(TInfo tinfo, TCredentials credentials, String user, String tableNamespace, byte permission) throws ThriftSecurityException,
-      ThriftTableOperationException {
+  public void grantTableNamespacePermission(TInfo tinfo, TCredentials credentials, String user, String tableNamespace, byte permission)
+      throws ThriftSecurityException, ThriftTableOperationException {
     String namespaceId = checkTableNamespaceId(tableNamespace, TableOperation.PERMISSION);
     security.grantTableNamespacePermission(credentials, user, namespaceId, TableNamespacePermission.getPermissionById(permission));
   }
-  
+
   @Override
   public void revokeSystemPermission(TInfo tinfo, TCredentials credentials, String user, byte permission) throws ThriftSecurityException {
     security.revokeSystemPermission(credentials, user, SystemPermission.getPermissionById(permission));
@@ -217,19 +217,19 @@ public class ClientServiceHandler implements ClientService.Iface {
   }
 
   @Override
-  public boolean hasTableNamespacePermission(TInfo tinfo, TCredentials credentials, String user, String tableNamespace, byte perm) throws ThriftSecurityException,
-      ThriftTableOperationException {
+  public boolean hasTableNamespacePermission(TInfo tinfo, TCredentials credentials, String user, String tableNamespace, byte perm)
+      throws ThriftSecurityException, ThriftTableOperationException {
     String namespaceId = checkTableNamespaceId(tableNamespace, TableOperation.PERMISSION);
     return security.hasTableNamespacePermission(credentials, user, namespaceId, TableNamespacePermission.getPermissionById(perm));
   }
-  
+
   @Override
-  public void revokeTableNamespacePermission(TInfo tinfo, TCredentials credentials, String user, String tableNamespace, byte permission) throws ThriftSecurityException,
-      ThriftTableOperationException {
+  public void revokeTableNamespacePermission(TInfo tinfo, TCredentials credentials, String user, String tableNamespace, byte permission)
+      throws ThriftSecurityException, ThriftTableOperationException {
     String namespaceId = checkTableNamespaceId(tableNamespace, TableOperation.PERMISSION);
     security.revokeTableNamespacePermission(credentials, user, namespaceId, TableNamespacePermission.getPermissionById(permission));
   }
-  
+
   @Override
   public Set<String> listLocalUsers(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException {
     return security.listUsers(credentials);
@@ -428,7 +428,7 @@ public class ClientServiceHandler implements ClientService.Iface {
       namespaceId = TableNamespaces.getNamespaceId(instance, ns);
     } catch (TableNamespaceNotFoundException e) {
       String why = "Could not find table namespace while getting configuration.";
-      throw new ThriftTableOperationException(null, ns, null, null, why);
+      throw new ThriftTableOperationException(null, ns, null, TableOperationExceptionType.NOTFOUND, why);
     }
     AccumuloConfiguration config = ServerConfiguration.getTableNamespaceConfiguration(instance, namespaceId);
     return conf(credentials, config);


[43/50] [abbrv] git commit: ACCUMULO-1906 Remove ops on tables in a namespace

Posted by ct...@apache.org.
ACCUMULO-1906 Remove ops on tables in a namespace

  Removing operations on tables in a namespace where the same operation
  can be easily accomplished with a short sequence of API calls on
  existing methods. Also, clean up Javadocs.


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

Branch: refs/heads/master
Commit: 3c920943bf92e2c15a5eeb4945165b3573003f33
Parents: ea8ec19
Author: Christopher Tubbs <ct...@apache.org>
Authored: Mon Nov 18 16:45:14 2013 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:11 2013 -0500

----------------------------------------------------------------------
 .../core/client/admin/NamespaceOperations.java  | 268 ++++++++-----------
 .../client/admin/NamespaceOperationsHelper.java |   6 +-
 .../client/admin/NamespaceOperationsImpl.java   | 250 +----------------
 .../core/client/admin/TableOperations.java      | 107 ++++----
 .../accumulo/core/client/impl/Tables.java       |  48 ++--
 .../client/mock/MockNamespaceOperations.java    |  74 +----
 .../shell/commands/DeleteNamespaceCommand.java  |  11 +-
 .../core/util/shell/commands/OptUtil.java       |   5 +-
 .../core/client/mock/MockNamespacesTest.java    |   4 +-
 .../server/security/SecurityOperation.java      |   5 -
 .../randomwalk/concurrent/DeleteNamespace.java  |   2 +-
 .../randomwalk/concurrent/OfflineNamespace.java |  53 ----
 .../org/apache/accumulo/test/NamespacesIT.java  |   7 +-
 .../org/apache/accumulo/test/ShellServerIT.java |  49 ++--
 .../accumulo/test/functional/RestartIT.java     |  18 +-
 15 files changed, 257 insertions(+), 650 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/3c920943/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java
index c82b42e..4ee670c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java
@@ -17,7 +17,6 @@
 package org.apache.accumulo.core.client.admin;
 
 import java.util.EnumSet;
-import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.SortedSet;
@@ -28,78 +27,43 @@ import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.NamespaceExistsException;
 import org.apache.accumulo.core.client.NamespaceNotEmptyException;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
-import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 
 /**
- * Provides a class for administering namespaces
+ * Provides an API for administering namespaces
  * 
+ * @since 1.6.0
  */
-
 public interface NamespaceOperations {
 
   /**
    * Retrieve a list of namespaces in Accumulo.
    * 
    * @return List of namespaces in accumulo
-   */
-  public SortedSet<String> list();
-
-  /**
-   * A method to check if a namespace exists in Accumulo.
-   * 
-   * @param namespace
-   *          the name of the namespace
-   * @return true if the namespace exists
-   */
-  public boolean exists(String namespace);
-
-  /**
-   * Create a namespace with no special configuration
-   * 
-   * @param namespace
-   *          the name of the namespace
    * @throws AccumuloException
    *           if a general error occurs
    * @throws AccumuloSecurityException
    *           if the user does not have permission
-   * @throws NamespaceExistsException
-   *           if the namespace already exists
-   */
-  public void create(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException;
-
-  /**
-   * @param namespace
-   *          the name of the namespace
-   * @param limitVersion
-   *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws NamespaceExistsException
-   *           if the namespace already exists
+   * @since 1.6.0
    */
-  public void create(String namespace, boolean limitVersion) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException;
+  public SortedSet<String> list() throws AccumuloException, AccumuloSecurityException;
 
   /**
+   * A method to check if a namespace exists in Accumulo.
+   * 
    * @param namespace
    *          the name of the namespace
-   * @param versioningIter
-   *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
-   * @param timeType
-   *          specifies logical or real-time based time recording for entries in the table
+   * @return true if the namespace exists
    * @throws AccumuloException
    *           if a general error occurs
    * @throws AccumuloSecurityException
    *           if the user does not have permission
-   * @throws NamespaceExistsException
-   *           if the namespace already exists
+   * @since 1.6.0
    */
-  public void create(String namespace, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException;
+  public boolean exists(String namespace) throws AccumuloException, AccumuloSecurityException;
 
   /**
-   * Delete a namespace if it is empty
+   * Create an empty namespace with no initial configuration
    * 
    * @param namespace
    *          the name of the namespace
@@ -107,36 +71,28 @@ public interface NamespaceOperations {
    *           if a general error occurs
    * @throws AccumuloSecurityException
    *           if the user does not have permission
-   * @throws NamespaceNotFoundException
-   *           if the namespace does not exist
-   * @throws NamespaceNotEmptyException
-   *           if the namespaces still contains tables
-   * @throws TableNotFoundException
-   *           if table not found while deleting
+   * @throws NamespaceExistsException
+   *           if the specified namespace already exists
+   * @since 1.6.0
    */
-  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException, NamespaceNotEmptyException,
-      TableNotFoundException;
+  public void create(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException;
 
   /**
-   * Delete a namespace
+   * Delete an empty namespace
    * 
    * @param namespace
    *          the name of the namespace
-   * @param deleteTables
-   *          boolean, if true deletes all the tables in the namespace in addition to deleting the namespace.
    * @throws AccumuloException
    *           if a general error occurs
    * @throws AccumuloSecurityException
    *           if the user does not have permission
    * @throws NamespaceNotFoundException
-   *           if the namespace does not exist
+   *           if the specified namespace doesn't exist
    * @throws NamespaceNotEmptyException
    *           if the namespaces still contains tables
-   * @throws TableNotFoundException
-   *           if table not found while deleting
+   * @since 1.6.0
    */
-  public void delete(String namespace, boolean deleteTables) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException,
-      NamespaceNotEmptyException, TableNotFoundException;
+  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException, NamespaceNotEmptyException;
 
   /**
    * Rename a namespace
@@ -153,13 +109,13 @@ public interface NamespaceOperations {
    *           if the old namespace does not exist
    * @throws NamespaceExistsException
    *           if the new namespace already exists
+   * @since 1.6.0
    */
-  public void rename(String oldNamespaceName, String newNamespaceName) throws AccumuloSecurityException, NamespaceNotFoundException, AccumuloException,
+  public void rename(String oldNamespaceName, String newNamespaceName) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException,
       NamespaceExistsException;
 
   /**
-   * Sets a property on a namespace which applies to all tables in the namespace. Note that it may take a short period of time (a second) to propagate the
-   * change everywhere.
+   * Sets a property on a namespace which applies to all tables in the namespace. Note that it may take a few seconds to propagate the change everywhere.
    * 
    * @param namespace
    *          the name of the namespace
@@ -171,11 +127,14 @@ public interface NamespaceOperations {
    *           if a general error occurs
    * @throws AccumuloSecurityException
    *           if the user does not have permission
+   * @throws NamespaceNotFoundException
+   *           if the specified namespace doesn't exist
+   * @since 1.6.0
    */
-  public void setProperty(String namespace, String property, String value) throws AccumuloException, AccumuloSecurityException;
+  public void setProperty(String namespace, String property, String value) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException;
 
   /**
-   * Removes a property from a namespace. Note that it may take a short period of time (a second) to propagate the change everywhere.
+   * Removes a property from a namespace. Note that it may take a few seconds to propagate the change everywhere.
    * 
    * @param namespace
    *          the name of the namespace
@@ -185,66 +144,39 @@ public interface NamespaceOperations {
    *           if a general error occurs
    * @throws AccumuloSecurityException
    *           if the user does not have permission
-   */
-  public void removeProperty(String namespace, String property) throws AccumuloException, AccumuloSecurityException;
-
-  /**
-   * Gets properties of a namespace. Note that recently changed properties may not be available immediately.
-   * 
-   * @param namespace
-   *          the name of the namespace
-   * @return all properties visible by this table (system and per-table properties). Note that recently changed properties may not be visible immediately.
    * @throws NamespaceNotFoundException
-   *           if the namespace does not exist
+   *           if the specified namespace doesn't exist
+   * @since 1.6.0
    */
-  public Iterable<Entry<String,String>> getProperties(String namespace) throws AccumuloException, NamespaceNotFoundException;
+  public void removeProperty(String namespace, String property) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException;
 
   /**
+   * Gets properties of a namespace, which are inherited by tables in this namespace. Note that recently changed properties may not be available immediately.
    * 
    * @param namespace
-   *          the namespace to take offline
-   * @throws AccumuloException
-   *           when there is a general accumulo error
-   * @throws AccumuloSecurityException
-   *           when the user does not have the proper permissions
-   * @throws NamespaceNotFoundException
-   *           if the namespace does not exist
-   */
-  public void offline(String namespace) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException;
-
-  /**
-   * 
-   * @param namespace
-   *          the namespace to take online
+   *          the name of the namespace
+   * @return all properties visible by this namespace (system and per-table properties). Note that recently changed properties may not be visible immediately.
    * @throws AccumuloException
-   *           when there is a general accumulo error
+   *           if a general error occurs
    * @throws AccumuloSecurityException
-   *           when the user does not have the proper permissions
+   *           if the user does not have permission
    * @throws NamespaceNotFoundException
-   *           if the namespace does not exist
+   *           if the specified namespace doesn't exist
+   * @since 1.6.0
    */
-  public void online(String namespace) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException;
+  public Iterable<Entry<String,String>> getProperties(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException;
 
   /**
    * Get a mapping of namespace name to internal namespace id.
    * 
    * @return the map from namespace name to internal namespace id
-   */
-  public Map<String,String> namespaceIdMap();
-
-  /**
-   * Gets the number of bytes being used in the files for the set of tables in this namespace
-   * 
-   * @param namespace
-   *          the namespace to get the set of tables from
-   * 
-   * @return a list of disk usage objects containing linked table names and sizes
    * @throws AccumuloException
-   *           when there is a general accumulo error
+   *           if a general error occurs
    * @throws AccumuloSecurityException
-   *           when the user does not have the proper permissions
+   *           if the user does not have permission
+   * @since 1.6.0
    */
-  public List<DiskUsage> getDiskUsage(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException;
+  public Map<String,String> namespaceIdMap() throws AccumuloException, AccumuloSecurityException;
 
   /**
    * Add an iterator to a namespace on all scopes.
@@ -253,15 +185,15 @@ public interface NamespaceOperations {
    *          the name of the namespace
    * @param setting
    *          object specifying the properties of the iterator
-   * @throws AccumuloSecurityException
-   *           thrown if the user does not have the ability to set properties on the namespace
    * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
    * @throws NamespaceNotFoundException
-   *           throw if the namespace no longer exists
-   * @throws IllegalArgumentException
-   *           if the setting conflicts with any existing iterators
+   *           if the specified namespace doesn't exist
+   * @since 1.6.0
    */
-  public void attachIterator(String namespace, IteratorSetting setting) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException;
+  public void attachIterator(String namespace, IteratorSetting setting) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException;
 
   /**
    * Add an iterator to a namespace on the given scopes.
@@ -270,15 +202,17 @@ public interface NamespaceOperations {
    *          the name of the namespace
    * @param setting
    *          object specifying the properties of the iterator
-   * @throws AccumuloSecurityException
-   *           thrown if the user does not have the ability to set properties on the namespace
+   * @param scopes
+   *          the set of scopes the iterator should apply to
    * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
    * @throws NamespaceNotFoundException
-   *           throw if the namespace no longer exists
-   * @throws IllegalArgumentException
-   *           if the setting conflicts with any existing iterators
+   *           if the specified namespace doesn't exist
+   * @since 1.6.0
    */
-  public void attachIterator(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+  public void attachIterator(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException, AccumuloSecurityException,
       NamespaceNotFoundException;
 
   /**
@@ -290,13 +224,15 @@ public interface NamespaceOperations {
    *          the name of the iterator
    * @param scopes
    *          the scopes of the iterator
-   * @throws AccumuloSecurityException
-   *           thrown if the user does not have the ability to set properties on the namespace
    * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
    * @throws NamespaceNotFoundException
-   *           thrown if the namespace no longer exists
+   *           if the specified namespace doesn't exist
+   * @since 1.6.0
    */
-  public void removeIterator(String namespace, String name, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+  public void removeIterator(String namespace, String name, EnumSet<IteratorScope> scopes) throws AccumuloException, AccumuloSecurityException,
       NamespaceNotFoundException;
 
   /**
@@ -309,14 +245,16 @@ public interface NamespaceOperations {
    * @param scope
    *          the scope of the iterator
    * @return the settings for this iterator
-   * @throws AccumuloSecurityException
-   *           thrown if the user does not have the ability to set properties on the namespace
    * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
    * @throws NamespaceNotFoundException
-   *           thrown if the namespace no longer exists
+   *           if the specified namespace doesn't exist
+   * @since 1.6.0
    */
-  public IteratorSetting getIteratorSetting(String namespace, String name, IteratorScope scope) throws AccumuloSecurityException, AccumuloException,
-      NumberFormatException, NamespaceNotFoundException;
+  public IteratorSetting getIteratorSetting(String namespace, String name, IteratorScope scope) throws AccumuloException, AccumuloSecurityException,
+      NamespaceNotFoundException;
 
   /**
    * Get a list of iterators for this namespace.
@@ -324,30 +262,36 @@ public interface NamespaceOperations {
    * @param namespace
    *          the name of the namespace
    * @return a set of iterator names
-   * @throws AccumuloSecurityException
-   *           thrown if the user does not have the ability to set properties on the namespace
    * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
    * @throws NamespaceNotFoundException
-   *           thrown if the namespace no longer exists
+   *           if the specified namespace doesn't exist
+   * @since 1.6.0
    */
-  public Map<String,EnumSet<IteratorScope>> listIterators(String namespace) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException;
+  public Map<String,EnumSet<IteratorScope>> listIterators(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException;
 
   /**
    * Check whether a given iterator configuration conflicts with existing configuration; in particular, determine if the name or priority are already in use for
-   * the specified scopes.
+   * the specified scopes. If not, an IllegalArgumentException is thrown, wrapped in an AccumuloException.
    * 
    * @param namespace
    *          the name of the namespace
    * @param setting
    *          object specifying the properties of the iterator
+   * @param scopes
+   *          the scopes of the iterator
    * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
    * @throws NamespaceNotFoundException
-   *           thrown if the namespace no longer exists
-   * @throws IllegalStateException
-   *           if the setting conflicts with any existing iterators
+   *           if the specified namespace doesn't exist
+   * @since 1.6.0
    */
   public void checkIteratorConflicts(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException,
-      NamespaceNotFoundException;
+      AccumuloSecurityException, NamespaceNotFoundException;
 
   /**
    * Add a new constraint to a namespace.
@@ -356,13 +300,14 @@ public interface NamespaceOperations {
    *          the name of the namespace
    * @param constraintClassName
    *          the full name of the constraint class
-   * @return the unique number assigned to the constraint
+   * @return the unique id number assigned to the constraint
    * @throws AccumuloException
-   *           thrown if the constraint has already been added to the table or if there are errors in the configuration of existing constraints
+   *           if a general error occurs
    * @throws AccumuloSecurityException
-   *           thrown if the user doesn't have permission to add the constraint
+   *           if the user does not have permission
    * @throws NamespaceNotFoundException
-   *           thrown if the namespace no longer exists
+   *           if the specified namespace doesn't exist
+   * @since 1.6.0
    */
   public int addConstraint(String namespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException;
 
@@ -371,13 +316,17 @@ public interface NamespaceOperations {
    * 
    * @param namespace
    *          the name of the namespace
-   * @param number
-   *          the unique number assigned to the constraint
+   * @param id
+   *          the unique id number assigned to the constraint
    * @throws AccumuloException
+   *           if a general error occurs
    * @throws AccumuloSecurityException
-   *           thrown if the user doesn't have permission to remove the constraint
+   *           if the user does not have permission
+   * @throws NamespaceNotFoundException
+   *           if the specified namespace doesn't exist
+   * @since 1.6.0
    */
-  public void removeConstraint(String namespace, int number) throws AccumuloException, AccumuloSecurityException;
+  public void removeConstraint(String namespace, int id) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException;
 
   /**
    * List constraints on a namespace with their assigned numbers.
@@ -386,16 +335,33 @@ public interface NamespaceOperations {
    *          the name of the namespace
    * @return a map from constraint class name to assigned number
    * @throws AccumuloException
-   *           thrown if there are errors in the configuration of existing constraints
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
    * @throws NamespaceNotFoundException
-   *           thrown if the namespace no longer exists
+   *           if the specified namespace doesn't exist
+   * @since 1.6.0
    */
-  public Map<String,Integer> listConstraints(String namespace) throws AccumuloException, NamespaceNotFoundException;
+  public Map<String,Integer> listConstraints(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException;
 
   /**
    * Test to see if the instance can load the given class as the given type. This check uses the table classpath property if it is set.
    * 
+   * @param namespace
+   *          the name of the namespace
+   * @param className
+   *          the class to try to load
+   * @param asTypeName
+   *          the interface or superclass the given class is attempted to load as
    * @return true if the instance can load the given class as the given type, false otherwise
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   * @throws NamespaceNotFoundException
+   *           if the specified namespace doesn't exist
+   * @since 1.6.0
    */
-  boolean testClassLoad(String namespace, String className, String asTypeName) throws NamespaceNotFoundException, AccumuloException, AccumuloSecurityException;
+  public boolean testClassLoad(String namespace, String className, String asTypeName) throws AccumuloException, AccumuloSecurityException,
+      NamespaceNotFoundException;
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3c920943/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsHelper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsHelper.java
index 007461f..7ecc4e3 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsHelper.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsHelper.java
@@ -119,7 +119,7 @@ public abstract class NamespaceOperationsHelper implements NamespaceOperations {
 
   @Override
   public void checkIteratorConflicts(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException,
-      NamespaceNotFoundException {
+      NamespaceNotFoundException, AccumuloSecurityException {
     if (!exists(namespace))
       throw new NamespaceNotFoundException(null, namespace, null);
     for (IteratorScope scope : scopes) {
@@ -179,12 +179,12 @@ public abstract class NamespaceOperationsHelper implements NamespaceOperations {
   }
 
   @Override
-  public void removeConstraint(String namespace, int number) throws AccumuloException, AccumuloSecurityException {
+  public void removeConstraint(String namespace, int number) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
     this.removeProperty(namespace, Property.TABLE_CONSTRAINT_PREFIX.toString() + number);
   }
 
   @Override
-  public Map<String,Integer> listConstraints(String namespace) throws AccumuloException, NamespaceNotFoundException {
+  public Map<String,Integer> listConstraints(String namespace) throws AccumuloException, NamespaceNotFoundException, AccumuloSecurityException {
     Map<String,Integer> constraints = new TreeMap<String,Integer>();
     for (Entry<String,String> property : this.getProperties(namespace)) {
       if (property.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.toString())) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3c920943/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
index 9036845..6c877f8 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
@@ -18,18 +18,15 @@ package org.apache.accumulo.core.client.admin;
 
 import java.nio.ByteBuffer;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
-import jline.internal.Log;
-
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -38,7 +35,6 @@ import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.NamespaceExistsException;
 import org.apache.accumulo.core.client.NamespaceNotEmptyException;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
-import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.client.impl.ClientExec;
 import org.apache.accumulo.core.client.impl.ClientExecReturn;
@@ -51,7 +47,6 @@ import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.core.constraints.Constraint;
-import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
@@ -67,33 +62,18 @@ import org.apache.log4j.Logger;
 import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransportException;
 
-/**
- * Provides a class for administering namespaces
- * 
- */
 public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
   private Instance instance;
   private Credentials credentials;
 
   private static final Logger log = Logger.getLogger(TableOperations.class);
 
-  /**
-   * @param instance
-   *          the connection information for this instance
-   * @param credentials
-   *          the username/password for this connection
-   */
   public NamespaceOperationsImpl(Instance instance, Credentials credentials) {
     ArgumentChecker.notNull(instance, credentials);
     this.instance = instance;
     this.credentials = credentials;
   }
 
-  /**
-   * Retrieve a list of namespaces in Accumulo.
-   * 
-   * @return List of namespaces in accumulo
-   */
   @Override
   public SortedSet<String> list() {
     OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Fetching list of namespaces...");
@@ -102,13 +82,6 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
     return namespaces;
   }
 
-  /**
-   * A method to check if a namespace exists in Accumulo.
-   * 
-   * @param namespace
-   *          the name of the namespace
-   * @return true if the namespace exists
-   */
   @Override
   public boolean exists(String namespace) {
     ArgumentChecker.notNull(namespace);
@@ -119,52 +92,12 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
     return exists;
   }
 
-  /**
-   * Create a namespace with no special configuration
-   * 
-   * @param namespace
-   *          the name of the namespace
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws NamespaceExistsException
-   *           if the namespace already exists
-   */
   @Override
   public void create(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException {
-    create(namespace, true, TimeType.MILLIS);
-  }
-
-  /**
-   * @param namespace
-   *          the name of the namespace
-   * @param limitVersion
-   *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
-   */
-  @Override
-  public void create(String namespace, boolean limitVersion) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException {
-    create(namespace, limitVersion, TimeType.MILLIS);
-  }
-
-  /**
-   * @param namespace
-   *          the name of the namespace
-   * @param timeType
-   *          specifies logical or real-time based time recording for entries in the table
-   * @param limitVersion
-   *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
-   */
-  @Override
-  public void create(String namespace, boolean limitVersion, TimeType timeType) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException {
-    ArgumentChecker.notNull(namespace, timeType);
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(namespace.getBytes()), ByteBuffer.wrap(timeType.name().getBytes()));
-
-    Map<String,String> opts = IteratorUtil.generateInitialTableProperties(limitVersion);
+    ArgumentChecker.notNull(namespace);
 
     try {
-      doNamespaceOperation(TableOperation.CREATE, args, opts);
+      doNamespaceOperation(TableOperation.CREATE, Arrays.asList(ByteBuffer.wrap(namespace.getBytes())), Collections.<String,String> emptyMap());
     } catch (NamespaceNotFoundException e1) {
       // should not happen
       throw new RuntimeException(e1);
@@ -282,49 +215,8 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
     }
   }
 
-  /**
-   * Delete a namespace if empty
-   * 
-   * @param namespace
-   *          the name of the namespace
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws NamespaceNotFoundException
-   *           if the namespace does not exist
-   * @throws NamespaceNotEmptyException
-   *           if the namespaces still contains tables
-   * @throws TableNotFoundException
-   *           if table not found while deleting
-   */
   @Override
-  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException, NamespaceNotEmptyException,
-      TableNotFoundException {
-    delete(namespace, false);
-  }
-
-  /**
-   * Delete a namespace
-   * 
-   * @param namespace
-   *          the name of the namespace
-   * @param deleteTables
-   *          boolean, if true deletes all the tables in the namespace in addition to deleting the namespace.
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws NamespaceNotFoundException
-   *           if the namespace does not exist
-   * @throws NamespaceNotEmptyException
-   *           if the namespaces still contains tables
-   * @throws TableNotFoundException
-   *           if table not found while deleting
-   */
-  @Override
-  public void delete(String namespace, boolean deleteTables) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException,
-      NamespaceNotEmptyException, TableNotFoundException {
+  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException, NamespaceNotEmptyException {
     ArgumentChecker.notNull(namespace);
     String namespaceId = Namespaces.getNamespaceId(instance, namespace);
 
@@ -334,16 +226,7 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
     }
 
     if (Namespaces.getTableIds(instance, namespaceId).size() > 0) {
-      if (!deleteTables) {
-        throw new NamespaceNotEmptyException(namespaceId, namespace, null);
-      }
-      for (String table : Namespaces.getTableNames(instance, namespaceId)) {
-        try {
-          getTableOperations().delete(table);
-        } catch (TableNotFoundException e) {
-          log.debug("Table (" + table + ") not found while deleting namespace, probably deleted while we were deleting the rest of the tables");
-        }
-      }
+      throw new NamespaceNotEmptyException(namespaceId, namespace, null);
     }
 
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(namespace.getBytes()));
@@ -358,22 +241,6 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
 
   }
 
-  /**
-   * Rename a namespace
-   * 
-   * @param oldNamespaceName
-   *          the old namespace
-   * @param newNamespaceName
-   *          the new namespace
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws NamespaceNotFoundException
-   *           if the old namespace name does not exist
-   * @throws NamespaceExistsException
-   *           if the new namespace name already exists
-   */
   @Override
   public void rename(String oldNamespaceName, String newNamespaceName) throws AccumuloSecurityException, NamespaceNotFoundException, AccumuloException,
       NamespaceExistsException {
@@ -383,20 +250,6 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
     doNamespaceOperation(TableOperation.RENAME, args, opts);
   }
 
-  /**
-   * Sets a property on a namespace which will apply to all tables in the namespace
-   * 
-   * @param namespace
-   *          the name of the namespace
-   * @param property
-   *          the name of a per-table property
-   * @param value
-   *          the value to set a per-table property to
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   */
   @Override
   public void setProperty(final String namespace, final String property, final String value) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(namespace, property, value);
@@ -409,18 +262,6 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
     });
   }
 
-  /**
-   * Removes a property from a namespace
-   * 
-   * @param namespace
-   *          the name of the namespace
-   * @param property
-   *          the name of a per-table property
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   */
   @Override
   public void removeProperty(final String namespace, final String property) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(namespace, property);
@@ -433,15 +274,6 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
     });
   }
 
-  /**
-   * Gets properties of a namespace
-   * 
-   * @param namespace
-   *          the name of the namespace
-   * @return all properties visible by this namespace (system and per-namespace properties)
-   * @throws NamespaceNotFoundException
-   *           if the namespace does not exist
-   */
   @Override
   public Iterable<Entry<String,String>> getProperties(final String namespace) throws AccumuloException, NamespaceNotFoundException {
     ArgumentChecker.notNull(namespace);
@@ -468,84 +300,12 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
 
   }
 
-  /**
-   * 
-   * @param namespace
-   *          the namespace to take offline
-   * @throws AccumuloException
-   *           when there is a general accumulo error
-   * @throws AccumuloSecurityException
-   *           when the user does not have the proper permissions
-   * @throws NamespaceNotFoundException
-   *           if the namespace does not exist
-   */
-  @Override
-  public void offline(String namespace) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException {
-
-    ArgumentChecker.notNull(namespace);
-    String namespaceId = Namespaces.getNamespaceId(instance, namespace);
-    try {
-      for (String table : Namespaces.getTableNames(instance, namespaceId)) {
-        getTableOperations().offline(table);
-      }
-    } catch (TableNotFoundException e) {
-      Log.error("Namespace (" + namespaceId + ") contains reference to table that doesn't exist");
-    }
-  }
-
-  /**
-   * 
-   * @param namespace
-   *          the namespace to take online
-   * @throws AccumuloException
-   *           when there is a general accumulo error
-   * @throws AccumuloSecurityException
-   *           when the user does not have the proper permissions
-   * @throws NamespaceNotFoundException
-   *           if the namespace does not exist
-   */
-  @Override
-  public void online(String namespace) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException {
-    ArgumentChecker.notNull(namespace);
-    String namespaceId = Namespaces.getNamespaceId(instance, namespace);
-    try {
-      for (String table : Namespaces.getTableNames(instance, namespaceId)) {
-        getTableOperations().online(table);
-      }
-    } catch (TableNotFoundException e) {
-      Log.warn("Namespace (" + namespaceId + ") contains a reference to a table that doesn't exist");
-    }
-  }
-
-  /**
-   * Get a mapping of namespace name to internal namespace id.
-   * 
-   * @return the map from namespace name to internal namespace id
-   */
   @Override
   public Map<String,String> namespaceIdMap() {
     return Namespaces.getNameToIdMap(instance);
   }
 
   @Override
-  public List<DiskUsage> getDiskUsage(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
-    Set<String> tables = new HashSet<String>();
-    String namespaceId = Namespaces.getNamespaceId(instance, namespace);
-    tables.addAll(Namespaces.getTableNames(instance, namespaceId));
-    List<DiskUsage> du = null;
-    try {
-      du = getTableOperations().getDiskUsage(tables);
-    } catch (TableNotFoundException e) {
-      log.warn("Could not find table (" + e.getTableName() + ") reference in namespace (" + namespace + ")");
-    }
-    return du;
-  }
-
-  private TableOperations getTableOperations() throws AccumuloException, AccumuloSecurityException {
-    return new TableOperationsImpl(instance, credentials);
-  }
-
-  @Override
   public void attachIterator(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
       NamespaceNotFoundException {
     testClassLoad(namespace, setting.getIteratorClass(), SortedKeyValueIterator.class.getName());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3c920943/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
index dc95250..97066c4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
@@ -41,14 +41,14 @@ import org.apache.hadoop.io.Text;
  */
 
 public interface TableOperations {
-  
+
   /**
    * Retrieve a list of tables in Accumulo.
    * 
    * @return List of tables in accumulo
    */
   public SortedSet<String> list();
-  
+
   /**
    * A method to check if a table exists in Accumulo.
    * 
@@ -57,7 +57,7 @@ public interface TableOperations {
    * @return true if the table exists
    */
   public boolean exists(String tableName);
-  
+
   /**
    * Create a table with no special configuration
    * 
@@ -71,7 +71,7 @@ public interface TableOperations {
    *           if the table already exists
    */
   public void create(String tableName) throws AccumuloException, AccumuloSecurityException, TableExistsException;
-  
+
   /**
    * @param tableName
    *          the name of the table
@@ -85,7 +85,7 @@ public interface TableOperations {
    *           if the table already exists
    */
   public void create(String tableName, boolean limitVersion) throws AccumuloException, AccumuloSecurityException, TableExistsException;
-  
+
   /**
    * @param tableName
    *          the name of the table
@@ -101,7 +101,7 @@ public interface TableOperations {
    *           if the table already exists
    */
   public void create(String tableName, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException;
-  
+
   /**
    * Imports a table exported via exportTable and copied via hadoop distcp.
    * 
@@ -115,7 +115,7 @@ public interface TableOperations {
    * @since 1.5.0
    */
   public void importTable(String tableName, String importDir) throws TableExistsException, AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * Exports a table. The tables data is not exported, just table metadata and a list of files to distcp. The table being exported must be offline and stay
    * offline for the duration of distcp. To avoid losing access to a table it can be cloned and the clone taken offline for export.
@@ -147,7 +147,7 @@ public interface TableOperations {
    *           if the table does not exist
    */
   public void addSplits(String tableName, SortedSet<Text> partitionKeys) throws TableNotFoundException, AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * @param tableName
    *          the name of the table
@@ -158,7 +158,7 @@ public interface TableOperations {
    */
   @Deprecated
   public Collection<Text> getSplits(String tableName) throws TableNotFoundException;
-  
+
   /**
    * @param tableName
    *          the name of the table
@@ -172,7 +172,7 @@ public interface TableOperations {
    * @since 1.5.0
    */
   public Collection<Text> listSplits(String tableName) throws TableNotFoundException, AccumuloSecurityException, AccumuloException;
-  
+
   /**
    * @param tableName
    *          the name of the table
@@ -184,7 +184,7 @@ public interface TableOperations {
    */
   @Deprecated
   public Collection<Text> getSplits(String tableName, int maxSplits) throws TableNotFoundException;
-  
+
   /**
    * @param tableName
    *          the name of the table
@@ -199,7 +199,7 @@ public interface TableOperations {
    * @since 1.5.0
    */
   public Collection<Text> listSplits(String tableName, int maxSplits) throws TableNotFoundException, AccumuloSecurityException, AccumuloException;
-  
+
   /**
    * Finds the max row within a given range. To find the max row in a table, pass null for start and end row.
    * 
@@ -223,7 +223,7 @@ public interface TableOperations {
    */
   public Text getMaxRow(String tableName, Authorizations auths, Text startRow, boolean startInclusive, Text endRow, boolean endInclusive)
       throws TableNotFoundException, AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * Merge tablets between (start, end]
    * 
@@ -235,7 +235,7 @@ public interface TableOperations {
    *          last tablet to be merged contains this row, null means the last tablet
    */
   public void merge(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
-  
+
   /**
    * Delete rows between (start, end]
    * 
@@ -247,7 +247,7 @@ public interface TableOperations {
    *          last row to be deleted, inclusive, null means the last row of the table
    */
   public void deleteRows(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
-  
+
   /**
    * Starts a full major compaction of the tablets in the range (start, end]. The compaction is preformed even for tablets that have only one file.
    * 
@@ -264,7 +264,7 @@ public interface TableOperations {
    */
   public void compact(String tableName, Text start, Text end, boolean flush, boolean wait) throws AccumuloSecurityException, TableNotFoundException,
       AccumuloException;
-  
+
   /**
    * Starts a full major compaction of the tablets in the range (start, end]. The compaction is preformed even for tablets that have only one file.
    * 
@@ -284,7 +284,7 @@ public interface TableOperations {
    */
   public void compact(String tableName, Text start, Text end, List<IteratorSetting> iterators, boolean flush, boolean wait) throws AccumuloSecurityException,
       TableNotFoundException, AccumuloException;
-  
+
   /**
    * Cancels a user initiated major compaction of a table initiated with {@link #compact(String, Text, Text, boolean, boolean)} or
    * {@link #compact(String, Text, Text, List, boolean, boolean)}. Compactions of tablets that are currently running may finish, but new compactions of tablets
@@ -301,7 +301,7 @@ public interface TableOperations {
    * @since 1.5.0
    */
   public void cancelCompaction(String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException;
-  
+
   /**
    * Delete a table
    * 
@@ -315,7 +315,7 @@ public interface TableOperations {
    *           if the table does not exist
    */
   public void delete(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
-  
+
   /**
    * Clone a table from an existing table. The cloned table will have the same data as the source table it was created from. After cloning, the two tables can
    * mutate independently. Initially the cloned table should not use any extra space, however as the source table and cloned table major compact extra space
@@ -334,10 +334,10 @@ public interface TableOperations {
    * @param propertiesToExclude
    *          do not copy these properties from the source table, just revert to system defaults
    */
-  
+
   public void clone(String srcTableName, String newTableName, boolean flush, Map<String,String> propertiesToSet, Set<String> propertiesToExclude)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException, TableExistsException;
-  
+
   /**
    * Rename a table
    * 
@@ -356,7 +356,7 @@ public interface TableOperations {
    */
   public void rename(String oldTableName, String newTableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException,
       TableExistsException;
-  
+
   /**
    * Initiate a flush of a table's data that is in memory
    * 
@@ -371,7 +371,7 @@ public interface TableOperations {
    */
   @Deprecated
   public void flush(String tableName) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * Flush a table's data that is currently in memory.
    * 
@@ -387,7 +387,7 @@ public interface TableOperations {
    * @throws TableNotFoundException
    */
   public void flush(String tableName, Text start, Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
-  
+
   /**
    * Sets a property on a table. Note that it may take a short period of time (a second) to propagate the change everywhere.
    * 
@@ -403,9 +403,9 @@ public interface TableOperations {
    *           if the user does not have permission
    */
   public void setProperty(String tableName, String property, String value) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
-   * Removes a property from a table.  Note that it may take a short period of time (a second) to propagate the change everywhere.
+   * Removes a property from a table. Note that it may take a short period of time (a second) to propagate the change everywhere.
    * 
    * @param tableName
    *          the name of the table
@@ -417,19 +417,18 @@ public interface TableOperations {
    *           if the user does not have permission
    */
   public void removeProperty(String tableName, String property) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
-   * Gets properties of a table.  Note that recently changed properties may not be available immediately.
+   * Gets properties of a table. Note that recently changed properties may not be available immediately.
    * 
    * @param tableName
    *          the name of the table
-   * @return all properties visible by this table (system and per-table properties).  Note that recently changed 
-   *         properties may not be visible immediately. 
+   * @return all properties visible by this table (system and per-table properties). Note that recently changed properties may not be visible immediately.
    * @throws TableNotFoundException
    *           if the table does not exist
    */
   public Iterable<Entry<String,String>> getProperties(String tableName) throws AccumuloException, TableNotFoundException;
-  
+
   /**
    * Sets a table's locality groups. A table's locality groups can be changed at any time.
    * 
@@ -445,7 +444,7 @@ public interface TableOperations {
    *           if the table does not exist
    */
   public void setLocalityGroups(String tableName, Map<String,Set<Text>> groups) throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
-  
+
   /**
    * 
    * Gets the locality groups currently set for a table.
@@ -459,7 +458,7 @@ public interface TableOperations {
    *           if the table does not exist
    */
   public Map<String,Set<Text>> getLocalityGroups(String tableName) throws AccumuloException, TableNotFoundException;
-  
+
   /**
    * @param tableName
    *          the name of the table
@@ -477,7 +476,7 @@ public interface TableOperations {
    */
   public Set<Range> splitRangeByTablets(String tableName, Range range, int maxSplits) throws AccumuloException, AccumuloSecurityException,
       TableNotFoundException;
-  
+
   /**
    * Bulk import all the files in a directory into a table.
    * 
@@ -488,7 +487,7 @@ public interface TableOperations {
    * @param failureDir
    *          the HDFS directory to place files that failed to be imported, must exist and be empty
    * @param setTime
-   *          override the time values in the input files, and use the current time for all mutations 
+   *          override the time values in the input files, and use the current time for all mutations
    * @throws IOException
    *           when there is an error reading/writing to HDFS
    * @throws AccumuloException
@@ -501,7 +500,7 @@ public interface TableOperations {
    */
   public void importDirectory(String tableName, String dir, String failureDir, boolean setTime) throws TableNotFoundException, IOException, AccumuloException,
       AccumuloSecurityException;
-  
+
   /**
    * Initiates taking a table offline, but does not wait for action to complete
    * 
@@ -514,7 +513,7 @@ public interface TableOperations {
    * @throws TableNotFoundException
    */
   public void offline(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException;
-  
+
   /**
    * 
    * @param tableName
@@ -529,7 +528,7 @@ public interface TableOperations {
    * @since 1.6.0
    */
   public void offline(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException;
-  
+
   /**
    * Initiates bringing a table online, but does not wait for action to complete
    * 
@@ -542,7 +541,7 @@ public interface TableOperations {
    * @throws TableNotFoundException
    */
   public void online(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException;
- 
+
   /**
    * 
    * @param tableName
@@ -557,7 +556,7 @@ public interface TableOperations {
    * @since 1.6.0
    */
   public void online(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException;
-  
+
   /**
    * Clears the tablet locator cache for a specified table
    * 
@@ -567,14 +566,14 @@ public interface TableOperations {
    *           if table does not exist
    */
   public void clearLocatorCache(String tableName) throws TableNotFoundException;
-  
+
   /**
    * Get a mapping of table name to internal table id.
    * 
    * @return the map from table name to internal table id
    */
   public Map<String,String> tableIdMap();
-  
+
   /**
    * Add an iterator to a table on all scopes.
    * 
@@ -591,7 +590,7 @@ public interface TableOperations {
    *           if the setting conflicts with any existing iterators
    */
   public void attachIterator(String tableName, IteratorSetting setting) throws AccumuloSecurityException, AccumuloException, TableNotFoundException;
-  
+
   /**
    * Add an iterator to a table on the given scopes.
    * 
@@ -609,7 +608,7 @@ public interface TableOperations {
    */
   public void attachIterator(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
       TableNotFoundException;
-  
+
   /**
    * Remove an iterator from a table by name.
    * 
@@ -627,7 +626,7 @@ public interface TableOperations {
    */
   public void removeIterator(String tableName, String name, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
       TableNotFoundException;
-  
+
   /**
    * Get the settings for an iterator.
    * 
@@ -646,7 +645,7 @@ public interface TableOperations {
    */
   public IteratorSetting getIteratorSetting(String tableName, String name, IteratorScope scope) throws AccumuloSecurityException, AccumuloException,
       TableNotFoundException;
-  
+
   /**
    * Get a list of iterators for this table.
    * 
@@ -658,7 +657,7 @@ public interface TableOperations {
    * @throws TableNotFoundException
    */
   public Map<String,EnumSet<IteratorScope>> listIterators(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException;
-  
+
   /**
    * Check whether a given iterator configuration conflicts with existing configuration; in particular, determine if the name or priority are already in use for
    * the specified scopes.
@@ -669,11 +668,9 @@ public interface TableOperations {
    *          object specifying the properties of the iterator
    * @throws AccumuloException
    * @throws TableNotFoundException
-   * @throws IllegalStateException
-   *           if the setting conflicts with any existing iterators
    */
   public void checkIteratorConflicts(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException, TableNotFoundException;
-  
+
   /**
    * Add a new constraint to a table.
    * 
@@ -690,7 +687,7 @@ public interface TableOperations {
    * @since 1.5.0
    */
   public int addConstraint(String tableName, String constraintClassName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
-  
+
   /**
    * Remove a constraint from a table.
    * 
@@ -704,7 +701,7 @@ public interface TableOperations {
    * @since 1.5.0
    */
   public void removeConstraint(String tableName, int number) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * List constraints on a table with their assigned numbers.
    * 
@@ -720,15 +717,15 @@ public interface TableOperations {
 
   /**
    * Gets the number of bytes being used in the files for a set of tables
-   *
+   * 
    * @param tables
    *          a set of tables
-   * @return  a list of disk usage objects containing linked table names and sizes
+   * @return a list of disk usage objects containing linked table names and sizes
    * @throws AccumuloException
    * @throws AccumuloSecurityException
    */
   public List<DiskUsage> getDiskUsage(Set<String> tables) throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
-  
+
   /**
    * Test to see if the instance can load the given class as the given type. This check uses the table classpath if it is set.
    * 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3c920943/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
index f017df4..2067762 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
@@ -23,8 +23,6 @@ import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicLong;
 
-import jline.internal.Log;
-
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
@@ -34,11 +32,13 @@ import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.log4j.Logger;
 
 public class Tables {
   private static SecurityPermission TABLES_PERMISSION = new SecurityPermission("tablesPermission");
   private static AtomicLong cacheResetCount = new AtomicLong(0);
-  
+  private static final Logger log = Logger.getLogger(Tables.class);
+
   private static ZooCache getZooCache(Instance instance) {
     SecurityManager sm = System.getSecurityManager();
     if (sm != null) {
@@ -46,14 +46,14 @@ public class Tables {
     }
     return ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
   }
-  
+
   private static SortedMap<String,String> getMap(Instance instance, boolean nameAsKey) {
     ZooCache zc = getZooCache(instance);
-    
+
     List<String> tableIds = zc.getChildren(ZooUtil.getRoot(instance) + Constants.ZTABLES);
-    
+
     TreeMap<String,String> tableMap = new TreeMap<String,String>();
-    
+
     for (String tableId : tableIds) {
       byte[] tblPath = zc.get(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_NAME);
       byte[] nId = zc.get(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_NAMESPACE);
@@ -65,7 +65,7 @@ public class Tables {
           try {
             name += Namespaces.getNamespaceName(instance, namespaceId) + ".";
           } catch (NamespaceNotFoundException e) {
-            Log.error("Table (" + tableId + ") contains reference to namespace (" + namespaceId + ") that doesn't exist");
+            log.error("Table (" + tableId + ") contains reference to namespace (" + namespaceId + ") that doesn't exist");
             continue;
           }
         }
@@ -78,54 +78,54 @@ public class Tables {
           tableMap.put(tableId, name);
       }
     }
-    
+
     return tableMap;
   }
-  
+
   public static String getTableId(Instance instance, String tableName) throws TableNotFoundException {
     String tableId = getNameToIdMap(instance).get(tableName);
     if (tableId == null)
       throw new TableNotFoundException(tableId, tableName, null);
     return tableId;
   }
-  
+
   public static String getTableName(Instance instance, String tableId) throws TableNotFoundException {
     String tableName = getIdToNameMap(instance).get(tableId);
     if (tableName == null)
       throw new TableNotFoundException(tableId, tableName, null);
     return tableName;
   }
-  
+
   public static SortedMap<String,String> getNameToIdMap(Instance instance) {
     return getMap(instance, true);
   }
-  
+
   public static SortedMap<String,String> getIdToNameMap(Instance instance) {
     return getMap(instance, false);
   }
-  
+
   public static boolean exists(Instance instance, String tableId) {
     ZooCache zc = getZooCache(instance);
     List<String> tableIds = zc.getChildren(ZooUtil.getRoot(instance) + Constants.ZTABLES);
     return tableIds.contains(tableId);
   }
-  
+
   public static void clearCache(Instance instance) {
     cacheResetCount.incrementAndGet();
     getZooCache(instance).clear(ZooUtil.getRoot(instance) + Constants.ZTABLES);
     getZooCache(instance).clear(ZooUtil.getRoot(instance) + Constants.ZNAMESPACES);
   }
-  
+
   public static String getPrintableTableNameFromId(Map<String,String> tidToNameMap, String tableId) {
     String tableName = tidToNameMap.get(tableId);
     return tableName == null ? "(ID:" + tableId + ")" : tableName;
   }
-  
+
   public static String getPrintableTableIdFromName(Map<String,String> nameToIdMap, String tableName) {
     String tableId = nameToIdMap.get(tableName);
     return tableId == null ? "(NAME:" + tableName + ")" : tableId;
   }
-  
+
   public static String getPrintableTableInfoFromId(Instance instance, String tableId) {
     String tableName = null;
     try {
@@ -135,7 +135,7 @@ public class Tables {
     }
     return tableName == null ? String.format("?(ID:%s)", tableId) : String.format("%s(ID:%s)", tableName, tableId);
   }
-  
+
   public static String getPrintableTableInfoFromName(Instance instance, String tableName) {
     String tableId = null;
     try {
@@ -145,17 +145,17 @@ public class Tables {
     }
     return tableId == null ? String.format("%s(?)", tableName) : String.format("%s(ID:%s)", tableName, tableId);
   }
-  
+
   public static TableState getTableState(Instance instance, String tableId) {
     String statePath = ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_STATE;
     ZooCache zc = getZooCache(instance);
     byte[] state = zc.get(statePath);
     if (state == null)
       return TableState.UNKNOWN;
-    
+
     return TableState.valueOf(new String(state));
   }
-  
+
   public static long getCacheResetCount() {
     return cacheResetCount.get();
   }
@@ -170,7 +170,7 @@ public class Tables {
       return Constants.DEFAULT_NAMESPACE;
     }
   }
-  
+
   public static String extractTableName(String tableName) {
     String[] s = tableName.split("\\.");
     if (s.length == 2 && !s[1].isEmpty() && !s[0].isEmpty()) {
@@ -179,7 +179,7 @@ public class Tables {
       return tableName;
     }
   }
-  
+
   public static String getNamespace(Instance instance, String tableId) {
     ZooCache zc = getZooCache(instance);
     byte[] n = zc.get(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_NAMESPACE);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3c920943/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
index fc76daa..5719317 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
@@ -16,9 +16,7 @@
  */
 package org.apache.accumulo.core.client.mock;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.SortedSet;
@@ -30,10 +28,7 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.NamespaceExistsException;
 import org.apache.accumulo.core.client.NamespaceNotEmptyException;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.DiskUsage;
 import org.apache.accumulo.core.client.admin.NamespaceOperationsHelper;
-import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
 
@@ -53,57 +48,25 @@ public class MockNamespaceOperations extends NamespaceOperationsHelper {
   }
 
   @Override
-  public boolean exists(String tableName) {
-    return acu.namespaces.containsKey(tableName);
+  public boolean exists(String namespace) {
+    return acu.namespaces.containsKey(namespace);
   }
 
   @Override
-  public void create(String tableName) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException {
-    create(tableName, true, TimeType.MILLIS);
-  }
-
-  @Override
-  public void create(String tableName, boolean versioningIter) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException {
-    create(tableName, versioningIter, TimeType.MILLIS);
-  }
-
-  @Override
-  public void create(String namespace, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException,
-      NamespaceExistsException {
-    if (!namespace.matches(Constants.VALID_NAMESPACE_REGEX)) {
+  public void create(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException {
+    if (!namespace.matches(Constants.VALID_NAMESPACE_REGEX))
       throw new IllegalArgumentException();
-    }
 
     if (exists(namespace))
       throw new NamespaceExistsException(namespace, namespace, "");
-
-    if (!exists(namespace)) {
+    else
       acu.createNamespace(username, namespace);
-    }
-    acu.createTable(username, namespace, versioningIter, timeType);
   }
 
   @Override
-  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException, NamespaceNotEmptyException,
-      TableNotFoundException {
-    delete(namespace, false);
-  }
-
-  @Override
-  public void delete(String namespace, boolean deleteTables) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException,
-      NamespaceNotEmptyException, TableNotFoundException {
-    if (!exists(namespace))
-      throw new NamespaceNotFoundException(namespace, namespace, "");
-
-    MockNamespace n = acu.namespaces.get(namespace);
-    if (!deleteTables) {
-      if (n.getTables(acu).size() > 0) {
-        throw new NamespaceNotEmptyException(null, namespace, null);
-      }
-    } else {
-      for (String t : n.getTables(acu)) {
-        new MockTableOperations(acu, username).delete(t);
-      }
+  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException, NamespaceNotEmptyException {
+    if (acu.namespaces.get(namespace).getTables(acu).size() > 0) {
+      throw new NamespaceNotEmptyException(null, namespace, null);
     }
     acu.namespaces.remove(namespace);
   }
@@ -144,18 +107,6 @@ public class MockNamespaceOperations extends NamespaceOperationsHelper {
   }
 
   @Override
-  public void offline(String namespace) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException {
-    if (!exists(namespace))
-      throw new NamespaceNotFoundException(namespace, namespace, "");
-  }
-
-  @Override
-  public void online(String namespace) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException {
-    if (!exists(namespace))
-      throw new NamespaceNotFoundException(namespace, namespace, "");
-  }
-
-  @Override
   public Map<String,String> namespaceIdMap() {
     Map<String,String> result = new HashMap<String,String>();
     for (String table : acu.tables.keySet()) {
@@ -165,15 +116,6 @@ public class MockNamespaceOperations extends NamespaceOperationsHelper {
   }
 
   @Override
-  public List<DiskUsage> getDiskUsage(String namespace) throws AccumuloException, AccumuloSecurityException {
-
-    List<DiskUsage> diskUsages = new ArrayList<DiskUsage>();
-    diskUsages.add(new DiskUsage(new TreeSet<String>(acu.namespaces.get(namespace).getTables(acu)), 0l));
-
-    return diskUsages;
-  }
-
-  @Override
   public boolean testClassLoad(String namespace, String className, String asTypeName) throws AccumuloException, AccumuloSecurityException,
       NamespaceNotFoundException {
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3c920943/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteNamespaceCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteNamespaceCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteNamespaceCommand.java
index 26473ed..756e85d 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteNamespaceCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteNamespaceCommand.java
@@ -69,11 +69,12 @@ public class DeleteNamespaceCommand extends Command {
     List<String> tables = Namespaces.getTableNames(shellState.getInstance(), namespaceId);
     resetContext = tables.contains(currentTable);
 
-    if (force) {
-      shellState.getConnector().namespaceOperations().delete(namespace, true);
-    } else {
-      shellState.getConnector().namespaceOperations().delete(namespace);
-    }
+    if (force)
+      for (String table : shellState.getConnector().tableOperations().list())
+        if (table.startsWith(namespace + "."))
+          shellState.getConnector().tableOperations().delete(table);
+
+    shellState.getConnector().namespaceOperations().delete(namespace);
     if (namespace.equals(Constants.SYSTEM_NAMESPACE)) {
       shellState.getReader().println("Namespace: [" + namespace + "], can't delete system or default namespace.");
     } else {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3c920943/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OptUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OptUtil.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OptUtil.java
index 24b6b9d..9915bdf 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OptUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OptUtil.java
@@ -18,6 +18,8 @@ package org.apache.accumulo.core.util.shell.commands;
 
 import java.io.UnsupportedEncodingException;
 
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.util.shell.Shell;
@@ -46,7 +48,8 @@ public abstract class OptUtil {
     return tableName;
   }
 
-  public static String getNamespaceOpt(final CommandLine cl, final Shell shellState) throws NamespaceNotFoundException {
+  public static String getNamespaceOpt(final CommandLine cl, final Shell shellState) throws NamespaceNotFoundException, AccumuloException,
+      AccumuloSecurityException {
     String namespace = null;
     if (cl.hasOption(Shell.namespaceOption)) {
       namespace = cl.getOptionValue(Shell.namespaceOption);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3c920943/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java b/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java
index cbca7c3..ad03551 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java
@@ -27,6 +27,7 @@ import java.util.Random;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
@@ -293,7 +294,8 @@ public class MockNamespacesTest {
     return false;
   }
 
-  private boolean checkNamespaceHasProp(Connector c, String n, String propKey, String propVal) throws AccumuloException, NamespaceNotFoundException {
+  private boolean checkNamespaceHasProp(Connector c, String n, String propKey, String propVal) throws AccumuloException, NamespaceNotFoundException,
+      AccumuloSecurityException {
     for (Entry<String,String> e : c.namespaceOperations().getProperties(n)) {
       if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) {
         return true;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3c920943/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 f2a7ac8..29db1d8 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
@@ -824,9 +824,4 @@ public class SecurityOperation {
         || hasSystemPermission(credentials, SystemPermission.ALTER_NAMESPACE, false);
   }
 
-  public boolean canCloneNamespace(TCredentials credentials, String namespaceId, String namespace) throws ThriftSecurityException {
-    authenticate(credentials);
-    return hasNamespacePermission(credentials, namespaceId, NamespacePermission.READ, false)
-        && hasSystemPermission(credentials, SystemPermission.CREATE_NAMESPACE, false);
-  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3c920943/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteNamespace.java
index f7c813a..c31e6b5 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteNamespace.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteNamespace.java
@@ -39,7 +39,7 @@ public class DeleteNamespace extends Test {
     String namespace = namespaces.get(rand.nextInt(namespaces.size()));
 
     try {
-      conn.namespaceOperations().delete(namespace, true);
+      conn.namespaceOperations().delete(namespace);
       log.debug("Deleted namespace " + namespace);
     } catch (NamespaceNotFoundException e) {
       log.debug("Delete namespace " + namespace + " failed, doesnt exist");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3c920943/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineNamespace.java
deleted file mode 100644
index e689921..0000000
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineNamespace.java
+++ /dev/null
@@ -1,53 +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.test.randomwalk.concurrent;
-
-import java.util.List;
-import java.util.Properties;
-import java.util.Random;
-
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.NamespaceNotFoundException;
-import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.test.randomwalk.State;
-import org.apache.accumulo.test.randomwalk.Test;
-
-public class OfflineNamespace extends Test {
-
-  @Override
-  public void visit(State state, Properties props) throws Exception {
-    Connector conn = state.getConnector();
-
-    Random rand = (Random) state.get("rand");
-
-    @SuppressWarnings("unchecked")
-    List<String> namespaces = (List<String>) state.get("namespaces");
-
-    String namespace = namespaces.get(rand.nextInt(namespaces.size()));
-
-    try {
-      conn.namespaceOperations().offline(namespace);
-      log.debug("Offlined namespace " + namespace);
-      UtilWaitThread.sleep(rand.nextInt(200));
-      conn.namespaceOperations().online(namespace);
-      log.debug("Onlined namespace " + namespace);
-    } catch (NamespaceNotFoundException tne) {
-      log.debug("offline or online failed " + namespace + ", doesnt exist");
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3c920943/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java b/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
index 5b7d11d..5695c51 100644
--- a/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
@@ -295,8 +295,8 @@ public class NamespacesIT {
 
     assertTrue(checkTableHasProp(c, t2, propKey, propVal2));
 
-    c.namespaceOperations().delete(n1, true);
-    c.namespaceOperations().delete(n2, true);
+    c.namespaceOperations().delete(n1);
+    c.namespaceOperations().delete(n2);
   }
 
   /**
@@ -564,7 +564,8 @@ public class NamespacesIT {
     return false;
   }
 
-  private boolean checkNamespaceHasProp(Connector c, String n, String propKey, String propVal) throws AccumuloException, NamespaceNotFoundException {
+  private boolean checkNamespaceHasProp(Connector c, String n, String propKey, String propVal) throws AccumuloException, NamespaceNotFoundException,
+      AccumuloSecurityException {
     for (Entry<String,String> e : c.namespaceOperations().getProperties(n)) {
       if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) {
         return true;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3c920943/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
index f21dfd9..7951c80 100644
--- a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
@@ -273,7 +273,7 @@ public class ShellServerIT extends SimpleMacIT {
     shell.execCommand("du -h", false, false);
     String o = output.get();
     // for some reason, there's a bit of fluctuation
-    assertTrue("Output did not match regex: '" + o + "'", o.matches(".*[1-9][0-9][0-9]\\s\\[t\\]\\n")); 
+    assertTrue("Output did not match regex: '" + o + "'", o.matches(".*[1-9][0-9][0-9]\\s\\[t\\]\\n"));
     exec("deletetable -f t");
   }
 
@@ -784,12 +784,12 @@ public class ShellServerIT extends SimpleMacIT {
     exec("config -t ptc -s " + Property.TABLE_CLASSPATH.getKey() + "=cx1", true);
 
     UtilWaitThread.sleep(200);
-    
-    // We can't use the setiter command as Filter implements OptionDescriber which 
+
+    // We can't use the setiter command as Filter implements OptionDescriber which
     // forces us to enter more input that I don't know how to input
     // Instead, we can just manually set the property on the table.
     exec("config -t ptc -s " + Property.TABLE_ITERATOR_PREFIX.getKey() + "scan.foo=10,org.apache.accumulo.test.FooFilter");
-    
+
     exec("insert foo f q v", true);
 
     UtilWaitThread.sleep(100);
@@ -804,7 +804,7 @@ public class ShellServerIT extends SimpleMacIT {
     exec("table ptc", true);
     exec("insert foo f q v", false);
     exec("insert ok foo q v", true);
-    
+
     exec("deletetable -f ptc", true);
     exec("config -d " + Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + "cx1");
 
@@ -850,60 +850,55 @@ public class ShellServerIT extends SimpleMacIT {
     exec("y");
     exec("namespaces", true, "thing2", true);
 
-    exec("clonenamespace thing2 testers -e table.file.max", true);
-    exec("namespaces", true, "testers", true);
-    exec("tables", true, "testers.thingy", true);
-    exec("clonenamespace thing2 testers2 -s table.file.max=42", true);
-
-    exec("du -tn thing2", true, "thing2.thingy", true);
+    exec("du -ns thing2", true, "thing2.thingy", true);
 
     // all "TableOperation" commands can take a namespace
-    exec("offline -tn thing2", true);
-    exec("online -tn thing2", true);
-    exec("flush -tn thing2", true);
-    exec("compact -tn thing2", true);
+    exec("offline -ns thing2", true);
+    exec("online -ns thing2", true);
+    exec("flush -ns thing2", true);
+    exec("compact -ns thing2", true);
     exec("createnamespace testers3", true);
     exec("createtable testers3.1", true);
     exec("createtable testers3.2", true);
-    exec("deletetable -tn testers3 -f", true);
+    exec("deletetable -ns testers3 -f", true);
     exec("tables", true, "testers3.1", false);
     exec("namespaces", true, "testers3", true);
     exec("deletenamespace testers3 -f", true);
     input.set("true\n\n\nSTRING\n");
-    exec("setiter -tn thing2 -scan -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -n name", true);
-    exec("listiter -tn thing2 -scan", true, "Summing", true);
-    exec("deleteiter -tn thing2 -n name -scan", true);
+    exec("setiter -ns thing2 -scan -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -n name", true);
+    exec("listiter -ns thing2 -scan", true, "Summing", true);
+    exec("deleteiter -ns thing2 -n name -scan", true);
     exec("createuser dude");
     exec("pass");
     exec("pass");
-    exec("grant Namespace.CREATE_TABLE -tn thing2 -u dude", true);
-    exec("revoke Namespace.CREATE_TABLE -tn thing2 -u dude", true);
+    exec("grant Namespace.CREATE_TABLE -ns thing2 -u dude", true);
+    exec("revoke Namespace.CREATE_TABLE -ns thing2 -u dude", true);
 
     // properties override and such
-    exec("config -tn thing2 -s table.file.max=44444", true);
-    exec("config -tn thing2", true, "44444", true);
+    exec("config -ns thing2 -s table.file.max=44444", true);
+    exec("config -ns thing2", true, "44444", true);
     exec("config -t thing2.thingy", true, "44444", true);
     exec("config -t thing2.thingy -s table.file.max=55555", true);
     exec("config -t thing2.thingy", true, "55555", true);
 
     // can copy properties when creating
     exec("createnamespace thing3 -cc thing2", true);
-    exec("config -tn thing3", true, "44444", true);
+    exec("config -ns thing3", true, "44444", true);
     exec("createnamespace thing4 -ctc thing2.thingy", true);
-    exec("config -tn thing4", true, "55555", true);
+    exec("config -ns thing4", true, "55555", true);
 
     exec("deletenamespace -f thing2", true);
     exec("namespaces", true, "thing2", false);
     exec("tables", true, "thing2.thingy", false);
 
     // put constraints on a namespace
-    exec("constraint -tn thing4 -a org.apache.accumulo.examples.simple.constraints.NumericValueConstraint", true);
+    exec("constraint -ns thing4 -a org.apache.accumulo.examples.simple.constraints.NumericValueConstraint", true);
     exec("createtable thing4.constrained", true);
     exec("table thing4.constrained", true);
     // should fail
     exec("insert r cf cq abc", false);
     exec("constraint -l", true, "NumericValueConstraint", true);
-    exec("constraint -tn thing4 -d 2");
+    exec("constraint -ns thing4 -d 1");
     exec("sleep 1");
     exec("insert r cf cq abc", true);
   }


[10/50] [abbrv] git commit: ACCUMULO-802 fixed problem when cloning a table, its namespace properties are excluded but those could conflict with explicitly set properties

Posted by ct...@apache.org.
ACCUMULO-802 fixed problem when cloning a table, its namespace properties are excluded but those could conflict with explicitly set properties


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

Branch: refs/heads/master
Commit: 45f59ca7fa62d7643f924f56b1215c8d89979cce
Parents: 88d44bc
Author: Sean Hickey <ta...@gmail.com>
Authored: Thu Aug 8 13:18:02 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:10 2013 -0500

----------------------------------------------------------------------
 .../accumulo/core/client/admin/TableOperationsImpl.java  | 11 ++++++++---
 .../org/apache/accumulo/master/tableOps/CreateTable.java |  2 +-
 2 files changed, 9 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/45f59ca7/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
index d5e0659..b1ab058 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
@@ -703,7 +703,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
     if (propertiesToSet == null)
       propertiesToSet = Collections.emptyMap();
 
-    HashSet<String> excludeProps = getUniqueNamespaceProperties(namespace, srcTableName);
+    HashSet<String> excludeProps = getUniqueNamespaceProperties(namespace, srcTableName, propertiesToSet);
     for (String p : propertiesToExclude) {
       excludeProps.add(p);
     }
@@ -726,8 +726,9 @@ public class TableOperationsImpl extends TableOperationsHelper {
     doTableOperation(TableOperation.CLONE, args, opts);
   }
 
-  // get the properties that are only in the table namespace so that we can exclude them when copying table properties
-  private HashSet<String> getUniqueNamespaceProperties(String namespace, String table) throws TableNotFoundException, AccumuloException {
+  // get the properties that are only in the table namespace so that we can exclude them when copying table properties.
+  // also, don't exclude properties that are going to be explicitly set.
+  private HashSet<String> getUniqueNamespaceProperties(String namespace, String table, Map<String,String> propsToSet) throws TableNotFoundException, AccumuloException {
     HashSet<String> props = new HashSet<String>();
     try {
       Iterable<Entry<String,String>> n = new TableNamespaceOperationsImpl(instance, credentials).getProperties(namespace);
@@ -745,6 +746,10 @@ public class TableOperationsImpl extends TableOperationsHelper {
     } catch (TableNamespaceNotFoundException e) {
       throw new IllegalStateException(new TableNamespaceNotFoundException(null, namespace, null));
     }
+
+    for (Entry<String,String> e : propsToSet.entrySet()) {
+      props.remove(e.getKey());
+    }
     return props;
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/45f59ca7/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 a2c0344..217bfda 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
@@ -291,7 +291,7 @@ public class CreateTable extends MasterRepo {
   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
     tableInfo.namespaceId = TableNamespaces.getNamespaceId(environment.getInstance(), Tables.extractNamespace(tableInfo.tableName));
-    return Utils.reserveTableNamespace(tableInfo.namespaceId, tid, false, false, TableOperation.CREATE);
+    return Utils.reserveTableNamespace(tableInfo.namespaceId, tid, false, true, TableOperation.CREATE);
   }
   
   @Override


[35/50] [abbrv] git commit: ACCUMULO-802 Renamed "TableNamespace" to "Namespace"

Posted by ct...@apache.org.
ACCUMULO-802 Renamed "TableNamespace" to "Namespace"

 based on ReviewBoard suggestion


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

Branch: refs/heads/master
Commit: ea8ec1939af2d73ab3727e6fa4f16f7bde0d51c6
Parents: ab50556
Author: Christopher Tubbs <ct...@apache.org>
Authored: Fri Nov 15 20:31:42 2013 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:11 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/core/Constants.java     |   58 +-
 .../apache/accumulo/core/client/Connector.java  |   42 +-
 .../core/client/NamespaceExistsException.java   |   67 +
 .../core/client/NamespaceNotEmptyException.java |   74 ++
 .../core/client/NamespaceNotFoundException.java |   77 ++
 .../client/TableNamespaceExistsException.java   |   67 -
 .../client/TableNamespaceNotEmptyException.java |   74 --
 .../client/TableNamespaceNotFoundException.java |   77 --
 .../core/client/admin/NamespaceOperations.java  |  401 ++++++
 .../client/admin/NamespaceOperationsHelper.java |  203 +++
 .../client/admin/NamespaceOperationsImpl.java   |  589 +++++++++
 .../core/client/admin/SecurityOperations.java   |   76 +-
 .../client/admin/SecurityOperationsImpl.java    |   73 +-
 .../client/admin/TableNamespaceOperations.java  |  406 ------
 .../admin/TableNamespaceOperationsHelper.java   |  206 ---
 .../admin/TableNamespaceOperationsImpl.java     |  590 ---------
 .../core/client/admin/TableOperationsImpl.java  |   34 +-
 .../core/client/impl/ConnectorImpl.java         |   10 +-
 .../accumulo/core/client/impl/Namespaces.java   |  107 ++
 .../core/client/impl/TableNamespaces.java       |  107 --
 .../accumulo/core/client/impl/Tables.java       |   12 +-
 .../core/client/impl/thrift/ClientService.java  | 1230 +++++++++---------
 .../client/impl/thrift/SecurityErrorCode.java   |    4 +-
 .../accumulo/core/client/mock/MockAccumulo.java |   14 +-
 .../core/client/mock/MockConnector.java         |    6 +-
 .../core/client/mock/MockNamespace.java         |   54 +
 .../client/mock/MockNamespaceOperations.java    |  188 +++
 .../client/mock/MockSecurityOperations.java     |   39 +-
 .../accumulo/core/client/mock/MockTable.java    |    8 +-
 .../core/client/mock/MockTableNamespace.java    |   54 -
 .../mock/MockTableNamespaceOperations.java      |  188 ---
 .../core/client/mock/MockTableOperations.java   |    8 +-
 .../core/client/security/SecurityErrorCode.java |    2 +-
 .../core/master/thrift/MasterClientService.java | 1072 +++++++--------
 .../core/security/NamespacePermission.java      |   66 +
 .../core/security/TableNamespacePermission.java |   66 -
 .../apache/accumulo/core/util/shell/Shell.java  |   26 +-
 .../core/util/shell/ShellCompletor.java         |   57 +-
 .../accumulo/core/util/shell/ShellOptions.java  |    4 +-
 .../core/util/shell/commands/ConfigCommand.java |  103 +-
 .../util/shell/commands/ConstraintCommand.java  |   58 +-
 .../shell/commands/CreateNamespaceCommand.java  |   30 +-
 .../util/shell/commands/CreateTableCommand.java |    4 +-
 .../core/util/shell/commands/DUCommand.java     |   56 +-
 .../util/shell/commands/DeleteIterCommand.java  |   44 +-
 .../shell/commands/DeleteNamespaceCommand.java  |   26 +-
 .../util/shell/commands/DeleteTableCommand.java |    2 +-
 .../core/util/shell/commands/GrantCommand.java  |   50 +-
 .../util/shell/commands/ListIterCommand.java    |   43 +-
 .../commands/NamespacePermissionsCommand.java   |    6 +-
 .../util/shell/commands/NamespacesCommand.java  |    8 +-
 .../core/util/shell/commands/OptUtil.java       |   62 +-
 .../shell/commands/RenameNamespaceCommand.java  |   20 +-
 .../util/shell/commands/RenameTableCommand.java |    2 +-
 .../core/util/shell/commands/RevokeCommand.java |   52 +-
 .../util/shell/commands/SetIterCommand.java     |  127 +-
 .../util/shell/commands/TableOperation.java     |   59 +-
 .../core/util/shell/commands/TablesCommand.java |   16 +-
 .../shell/commands/UserPermissionsCommand.java  |   30 +-
 core/src/main/thrift/client.thrift              |   12 +-
 core/src/main/thrift/master.thrift              |   14 +-
 .../core/client/mock/MockNamespacesTest.java    |  313 +++++
 .../client/mock/MockTableNamespacesTest.java    |  312 -----
 .../server/client/ClientServiceHandler.java     |   60 +-
 .../server/conf/NamespaceConfWatcher.java       |  107 ++
 .../server/conf/NamespaceConfiguration.java     |  174 +++
 .../server/conf/ServerConfiguration.java        |   28 +-
 .../server/conf/TableConfiguration.java         |   46 +-
 .../server/conf/TableNamespaceConfWatcher.java  |  107 --
 .../conf/TableNamespaceConfiguration.java       |  174 ---
 .../server/conf/TableParentConfiguration.java   |    4 +-
 .../apache/accumulo/server/init/Initialize.java |   10 +-
 .../server/security/SecurityOperation.java      |  400 +++---
 .../security/handler/InsecurePermHandler.java   |   22 +-
 .../security/handler/PermissionHandler.java     |   69 +-
 .../server/security/handler/ZKPermHandler.java  |  157 +--
 .../server/security/handler/ZKSecurityTool.java |   12 +-
 .../accumulo/server/tables/TableManager.java    |    4 +-
 .../java/org/apache/accumulo/master/Master.java |  111 +-
 .../master/tableOps/CancelCompactions.java      |    4 +-
 .../master/tableOps/ChangeTableState.java       |    6 +-
 .../accumulo/master/tableOps/CloneTable.java    |   20 +-
 .../accumulo/master/tableOps/CompactRange.java  |    4 +-
 .../master/tableOps/CreateNamespace.java        |  196 +++
 .../accumulo/master/tableOps/CreateTable.java   |   14 +-
 .../master/tableOps/CreateTableNamespace.java   |  197 ---
 .../master/tableOps/DeleteNamespace.java        |  104 ++
 .../accumulo/master/tableOps/DeleteTable.java   |    6 +-
 .../master/tableOps/DeleteTableNamespace.java   |  104 --
 .../accumulo/master/tableOps/ExportTable.java   |    6 +-
 .../accumulo/master/tableOps/ImportTable.java   |   16 +-
 .../master/tableOps/RenameNamespace.java        |   92 ++
 .../accumulo/master/tableOps/RenameTable.java   |   20 +-
 .../master/tableOps/RenameTableNamespace.java   |   91 --
 .../accumulo/master/tableOps/TableRangeOp.java  |    6 +-
 .../apache/accumulo/master/tableOps/Utils.java  |   56 +-
 .../concurrent/ChangePermissions.java           |   30 +-
 .../randomwalk/concurrent/CheckPermission.java  |   26 +-
 .../test/randomwalk/concurrent/Config.java      |   14 +-
 .../randomwalk/concurrent/CreateNamespace.java  |   48 +
 .../concurrent/CreateTableNamespace.java        |   48 -
 .../randomwalk/concurrent/DeleteNamespace.java  |   48 +
 .../concurrent/DeleteTableNamespace.java        |   48 -
 .../randomwalk/concurrent/OfflineNamespace.java |   53 +
 .../concurrent/OfflineTableNamespace.java       |   53 -
 .../randomwalk/concurrent/RenameNamespace.java  |   52 +
 .../concurrent/RenameTableNamespace.java        |   52 -
 .../randomwalk/security/WalkingSecurity.java    |  182 +--
 .../org/apache/accumulo/test/NamespacesIT.java  |  584 +++++++++
 .../org/apache/accumulo/test/ShellServerIT.java |   32 +-
 .../apache/accumulo/test/TableNamespacesIT.java |  584 ---------
 .../accumulo/test/functional/PermissionsIT.java |  100 +-
 .../randomwalk/conf/modules/Concurrent.xml      |   16 +-
 113 files changed, 6139 insertions(+), 6143 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/Constants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/Constants.java b/core/src/main/java/org/apache/accumulo/core/Constants.java
index 4eb7471..2909c3d 100644
--- a/core/src/main/java/org/apache/accumulo/core/Constants.java
+++ b/core/src/main/java/org/apache/accumulo/core/Constants.java
@@ -23,11 +23,11 @@ import org.apache.accumulo.core.security.Authorizations;
 public class Constants {
   public static final Charset UTF8 = Charset.forName("UTF-8");
   public static final String VERSION = FilteredConstants.VERSION;
-  
+
   // Zookeeper locations
   public static final String ZROOT = "/accumulo";
   public static final String ZINSTANCES = "/instances";
-  
+
   public static final String ZTABLES = "/tables";
   public static final byte[] ZTABLES_INITIAL_ID = new byte[] {'0'};
   public static final String ZTABLE_NAME = "/name";
@@ -45,76 +45,76 @@ public class Constants {
   public static final String ZMASTERS = "/masters";
   public static final String ZMASTER_LOCK = ZMASTERS + "/lock";
   public static final String ZMASTER_GOAL_STATE = ZMASTERS + "/goal_state";
-  
+
   public static final String ZGC = "/gc";
   public static final String ZGC_LOCK = ZGC + "/lock";
-  
+
   public static final String ZMONITOR = "/monitor";
   public static final String ZMONITOR_LOG4J_PORT = ZMONITOR + "/log4j_port";
-  
+
   public static final String ZCONFIG = "/config";
-  
+
   public static final String ZTSERVERS = "/tservers";
-  
+
   public static final String ZDEAD = "/dead";
   public static final String ZDEADTSERVERS = ZDEAD + "/tservers";
-  
+
   public static final String ZTRACERS = "/tracers";
-  
+
   public static final String ZPROBLEMS = "/problems";
-  
+
   public static final String BULK_ARBITRATOR_TYPE = "bulkTx";
-  
+
   public static final String ZFATE = "/fate";
-  
+
   public static final String ZNEXT_FILE = "/next_file";
-  
+
   public static final String ZBULK_FAILED_COPYQ = "/bulk_failed_copyq";
-  
+
   public static final String ZHDFS_RESERVATIONS = "/hdfs_reservations";
   public static final String ZRECOVERY = "/recovery";
-  
+
   /**
    * Initial tablet directory name for the default tablet in all tables
    */
   public static final String DEFAULT_TABLET_LOCATION = "/default_tablet";
-  
+
   public static final String ZTABLE_LOCKS = "/table_locks";
-  
+
   public static final String BULK_PREFIX = "b-";
-  
+
   // this affects the table client caching of metadata
   public static final int SCAN_BATCH_SIZE = 1000;
-  
+
   // Scanners will default to fetching 3 batches of Key/Value pairs before asynchronously
   // fetching the next batch.
   public static final long SCANNER_DEFAULT_READAHEAD_THRESHOLD = 3l;
-  
+
   // Security configuration
   public static final String PW_HASH_ALGORITHM = "SHA-256";
-  
+
   /**
    * @deprecated since 1.6.0; Use {@link Authorizations#EMPTY} instead
    */
   @Deprecated
   public static final Authorizations NO_AUTHS = Authorizations.EMPTY;
-  
+
   public static final int MAX_DATA_TO_PRINT = 64;
   public static final String CORE_PACKAGE_NAME = "org.apache.accumulo.core";
   public static final String VALID_TABLE_NAME_REGEX = "^(\\w+\\.)?(\\w+)$";
-  public static final String VALID_TABLE_NAMESPACE_REGEX = "^\\w+$";
+  public static final String VALID_NAMESPACE_REGEX = "^\\w+$";
   public static final String MAPFILE_EXTENSION = "map";
   public static final String GENERATED_TABLET_DIRECTORY_PREFIX = "t-";
-  
+
   public static final String EXPORT_METADATA_FILE = "metadata.bin";
   public static final String EXPORT_TABLE_CONFIG_FILE = "table_config.txt";
   public static final String EXPORT_FILE = "exportMetadata.zip";
   public static final String EXPORT_INFO_FILE = "accumulo_export_info.txt";
-  
+
   // Variables that will be substituted with environment vars in PropertyType.PATH values
   public static final String[] PATH_PROPERTY_ENV_VARS = new String[]{"$ACCUMULO_HOME", "$ACCUMULO_CONF_DIR"};
-  public static final String DEFAULT_TABLE_NAMESPACE = "_default_";
-  public static final String SYSTEM_TABLE_NAMESPACE = "_system_";
-  public static final String DEFAULT_TABLE_NAMESPACE_ID = "!DEF";
-  public static final String SYSTEM_TABLE_NAMESPACE_ID = "!SYS";
+  public static final String DEFAULT_NAMESPACE = "_default_";
+  public static final String SYSTEM_NAMESPACE = "_system_";
+  public static final String DEFAULT_NAMESPACE_ID = "!DEF";
+  public static final String SYSTEM_NAMESPACE_ID = "!SYS";
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/Connector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Connector.java b/core/src/main/java/org/apache/accumulo/core/client/Connector.java
index 06b74a8..92a1184 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Connector.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Connector.java
@@ -17,8 +17,8 @@
 package org.apache.accumulo.core.client;
 
 import org.apache.accumulo.core.client.admin.InstanceOperations;
+import org.apache.accumulo.core.client.admin.NamespaceOperations;
 import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.admin.TableNamespaceOperations;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.security.Authorizations;
 
@@ -29,7 +29,7 @@ import org.apache.accumulo.core.security.Authorizations;
  * The Connector enforces security on the client side by forcing all API calls to be accompanied by user credentials.
  */
 public abstract class Connector {
-  
+
   /**
    * Factory method to create a BatchScanner connected to Accumulo.
    * 
@@ -47,7 +47,7 @@ public abstract class Connector {
    *           when the specified table doesn't exist
    */
   public abstract BatchScanner createBatchScanner(String tableName, Authorizations authorizations, int numQueryThreads) throws TableNotFoundException;
-  
+
   /**
    * Factory method to create a BatchDeleter connected to Accumulo.
    * 
@@ -74,7 +74,7 @@ public abstract class Connector {
   @Deprecated
   public abstract BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations, int numQueryThreads, long maxMemory, long maxLatency,
       int maxWriteThreads) throws TableNotFoundException;
-  
+
   /**
    * 
    * @param tableName
@@ -91,10 +91,10 @@ public abstract class Connector {
    * @throws TableNotFoundException
    * @since 1.5.0
    */
-  
+
   public abstract BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations, int numQueryThreads, BatchWriterConfig config)
       throws TableNotFoundException;
-  
+
   /**
    * Factory method to create a BatchWriter connected to Accumulo.
    * 
@@ -114,7 +114,7 @@ public abstract class Connector {
    */
   @Deprecated
   public abstract BatchWriter createBatchWriter(String tableName, long maxMemory, long maxLatency, int maxWriteThreads) throws TableNotFoundException;
-  
+
   /**
    * Factory method to create a BatchWriter connected to Accumulo.
    * 
@@ -126,9 +126,9 @@ public abstract class Connector {
    * @throws TableNotFoundException
    * @since 1.5.0
    */
-  
+
   public abstract BatchWriter createBatchWriter(String tableName, BatchWriterConfig config) throws TableNotFoundException;
-  
+
   /**
    * Factory method to create a Multi-Table BatchWriter connected to Accumulo. Multi-table batch writers can queue data for multiple tables, which is good for
    * ingesting data into multiple tables from the same source
@@ -145,7 +145,7 @@ public abstract class Connector {
    */
   @Deprecated
   public abstract MultiTableBatchWriter createMultiTableBatchWriter(long maxMemory, long maxLatency, int maxWriteThreads);
-  
+
   /**
    * Factory method to create a Multi-Table BatchWriter connected to Accumulo. Multi-table batch writers can queue data for multiple tables. Also data for
    * multiple tables can be sent to a server in a single batch. Its an efficient way to ingest data into multiple tables from a single process.
@@ -155,9 +155,9 @@ public abstract class Connector {
    * @return MultiTableBatchWriter object for configuring and writing data to
    * @since 1.5.0
    */
-  
+
   public abstract MultiTableBatchWriter createMultiTableBatchWriter(BatchWriterConfig config);
-  
+
   /**
    * Factory method to create a Scanner connected to Accumulo.
    * 
@@ -173,7 +173,7 @@ public abstract class Connector {
    *           when the specified table doesn't exist
    */
   public abstract Scanner createScanner(String tableName, Authorizations authorizations) throws TableNotFoundException;
-  
+
   /**
    * Factory method to create a ConditionalWriter connected to Accumulo.
    * 
@@ -195,35 +195,35 @@ public abstract class Connector {
    * @return the internal instance object
    */
   public abstract Instance getInstance();
-  
+
   /**
    * Get the current user for this connector
    * 
    * @return the user name
    */
   public abstract String whoami();
-  
+
   /**
    * Retrieves a TableOperations object to perform table functions, such as create and delete.
    * 
    * @return an object to manipulate tables
    */
   public abstract TableOperations tableOperations();
-  
+
   /**
-   * Retrieves a TableNamespaceOperations object to perform table namespace functions, such as create and delete.
+   * Retrieves a NamespaceOperations object to perform namespace functions, such as create and delete.
    * 
-   * @return an object to manipulate table namespaces
+   * @return an object to manipulate namespaces
    */
-  public abstract TableNamespaceOperations tableNamespaceOperations();
-  
+  public abstract NamespaceOperations namespaceOperations();
+
   /**
    * Retrieves a SecurityOperations object to perform user security operations, such as creating users.
    * 
    * @return an object to modify users and permissions
    */
   public abstract SecurityOperations securityOperations();
-  
+
   /**
    * Retrieves an InstanceOperations object to modify instance configuration.
    * 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/NamespaceExistsException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/NamespaceExistsException.java b/core/src/main/java/org/apache/accumulo/core/client/NamespaceExistsException.java
new file mode 100644
index 0000000..d2cb607
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/NamespaceExistsException.java
@@ -0,0 +1,67 @@
+/*
+ * 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.core.client;
+
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+
+/**
+ * Thrown when the namespace specified already exists, and it was expected that it didn't
+ */
+public class NamespaceExistsException extends Exception {
+  /**
+   * Exception to throw if an operation is attempted on a namespace that already exists.
+   * 
+   */
+  private static final long serialVersionUID = 1L;
+
+  /**
+   * @param namespaceId
+   *          the internal id of the namespace that exists
+   * @param namespaceName
+   *          the visible name of the namespace that exists
+   * @param description
+   *          the specific reason why it failed
+   */
+  public NamespaceExistsException(String namespaceId, String namespaceName, String description) {
+    super("Namespace" + (namespaceName != null && !namespaceName.isEmpty() ? " " + namespaceName : "")
+        + (namespaceId != null && !namespaceId.isEmpty() ? " (Id=" + namespaceId + ")" : "") + " exists"
+        + (description != null && !description.isEmpty() ? " (" + description + ")" : ""));
+  }
+
+  /**
+   * @param namespaceId
+   *          the internal id of the namespace that exists
+   * @param namespaceName
+   *          the visible name of the namespace that exists
+   * @param description
+   *          the specific reason why it failed
+   * @param cause
+   *          the exception that caused this failure
+   */
+  public NamespaceExistsException(String namespaceId, String namespaceName, String description, Throwable cause) {
+    this(namespaceId, namespaceName, description);
+    super.initCause(cause);
+  }
+
+  /**
+   * @param e
+   *          constructs an exception from a thrift exception
+   */
+  public NamespaceExistsException(ThriftTableOperationException e) {
+    this(e.getTableId(), e.getTableName(), e.getDescription(), e);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/NamespaceNotEmptyException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/NamespaceNotEmptyException.java b/core/src/main/java/org/apache/accumulo/core/client/NamespaceNotEmptyException.java
new file mode 100644
index 0000000..d5542bf
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/NamespaceNotEmptyException.java
@@ -0,0 +1,74 @@
+/*
+ * 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.core.client;
+
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+
+/**
+ * Thrown when the namespace specified contains tables
+ */
+public class NamespaceNotEmptyException extends Exception {
+
+  private static final long serialVersionUID = 1L;
+
+  private String namespace;
+
+  /**
+   * @param namespaceId
+   *          the internal id of the namespace
+   * @param namespaceName
+   *          the visible name of the namespace
+   * @param description
+   *          the specific reason why it failed
+   */
+  public NamespaceNotEmptyException(String namespaceId, String namespaceName, String description) {
+    super("Namespace" + (namespaceName != null && !namespaceName.isEmpty() ? " " + namespaceName : "")
+        + (namespaceId != null && !namespaceId.isEmpty() ? " (Id=" + namespaceId + ")" : "") + " it not empty, contains at least one table"
+        + (description != null && !description.isEmpty() ? " (" + description + ")" : ""));
+    this.namespace = namespaceName;
+  }
+
+  /**
+   * @param namespaceId
+   *          the internal id of the namespace
+   * @param namespaceName
+   *          the visible name of the namespace
+   * @param description
+   *          the specific reason why it failed
+   * @param cause
+   *          the exception that caused this failure
+   */
+  public NamespaceNotEmptyException(String namespaceId, String namespaceName, String description, Throwable cause) {
+    this(namespaceId, namespaceName, description);
+    super.initCause(cause);
+  }
+
+  /**
+   * @param e
+   *          constructs an exception from a thrift exception
+   */
+  public NamespaceNotEmptyException(ThriftTableOperationException e) {
+    this(e.getTableId(), e.getTableName(), e.getDescription(), e);
+  }
+
+  /**
+   * @return the name of the namespace
+   */
+  public String getNamespaceName() {
+    return namespace;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/NamespaceNotFoundException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/NamespaceNotFoundException.java b/core/src/main/java/org/apache/accumulo/core/client/NamespaceNotFoundException.java
new file mode 100644
index 0000000..031cf16
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/NamespaceNotFoundException.java
@@ -0,0 +1,77 @@
+/*
+ * 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.core.client;
+
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+
+/**
+ * Thrown when the namespace specified doesn't exist when it was expected to
+ */
+public class NamespaceNotFoundException extends Exception {
+  /**
+   * Exception to throw if an operation is attempted on a namespace that doesn't exist.
+   * 
+   */
+  private static final long serialVersionUID = 1L;
+
+  private String namespace;
+
+  /**
+   * @param namespaceId
+   *          the internal id of the namespace that was sought
+   * @param namespaceName
+   *          the visible name of the namespace that was sought
+   * @param description
+   *          the specific reason why it failed
+   */
+  public NamespaceNotFoundException(String namespaceId, String namespaceName, String description) {
+    super("Namespace" + (namespaceName != null && !namespaceName.isEmpty() ? " " + namespaceName : "")
+        + (namespaceId != null && !namespaceId.isEmpty() ? " (Id=" + namespaceId + ")" : "") + " does not exist"
+        + (description != null && !description.isEmpty() ? " (" + description + ")" : ""));
+    this.namespace = namespaceName;
+  }
+
+  /**
+   * @param namespaceId
+   *          the internal id of the namespace that was sought
+   * @param namespaceName
+   *          the visible name of the namespace that was sought
+   * @param description
+   *          the specific reason why it failed
+   * @param cause
+   *          the exception that caused this failure
+   */
+  public NamespaceNotFoundException(String namespaceId, String namespaceName, String description, Throwable cause) {
+    this(namespaceId, namespaceName, description);
+    super.initCause(cause);
+  }
+
+  /**
+   * @param e
+   *          constructs an exception from a thrift exception
+   */
+  public NamespaceNotFoundException(ThriftTableOperationException e) {
+    this(e.getTableId(), e.getTableName(), e.getDescription(), e);
+  }
+
+  /**
+   * @return the name of the namespace sought
+   */
+  public String getNamespaceName() {
+    return namespace;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceExistsException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceExistsException.java b/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceExistsException.java
deleted file mode 100644
index c6a8591..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceExistsException.java
+++ /dev/null
@@ -1,67 +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.core.client;
-
-import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
-
-/**
- * Thrown when the table namespace specified already exists, and it was expected that it didn't
- */
-public class TableNamespaceExistsException extends Exception {
-  /**
-   * Exception to throw if an operation is attempted on a table namespace that already exists.
-   * 
-   */
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * @param namespaceId
-   *          the internal id of the table namespace that exists
-   * @param namespaceName
-   *          the visible name of the table namespace that exists
-   * @param description
-   *          the specific reason why it failed
-   */
-  public TableNamespaceExistsException(String namespaceId, String namespaceName, String description) {
-    super("Table namespace" + (namespaceName != null && !namespaceName.isEmpty() ? " " + namespaceName : "")
-        + (namespaceId != null && !namespaceId.isEmpty() ? " (Id=" + namespaceId + ")" : "") + " exists"
-        + (description != null && !description.isEmpty() ? " (" + description + ")" : ""));
-  }
-
-  /**
-   * @param namespaceId
-   *          the internal id of the table namespace that exists
-   * @param namespaceName
-   *          the visible name of the table namespace that exists
-   * @param description
-   *          the specific reason why it failed
-   * @param cause
-   *          the exception that caused this failure
-   */
-  public TableNamespaceExistsException(String namespaceId, String namespaceName, String description, Throwable cause) {
-    this(namespaceId, namespaceName, description);
-    super.initCause(cause);
-  }
-
-  /**
-   * @param e
-   *          constructs an exception from a thrift exception
-   */
-  public TableNamespaceExistsException(ThriftTableOperationException e) {
-    this(e.getTableId(), e.getTableName(), e.getDescription(), e);
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotEmptyException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotEmptyException.java b/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotEmptyException.java
deleted file mode 100644
index 04195e3..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotEmptyException.java
+++ /dev/null
@@ -1,74 +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.core.client;
-
-import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
-
-/**
- * Thrown when the table namespace specified contains tables
- */
-public class TableNamespaceNotEmptyException extends Exception {
-
-  private static final long serialVersionUID = 1L;
-
-  private String namespace;
-
-  /**
-   * @param namespaceId
-   *          the internal id of the table namespace
-   * @param namespaceName
-   *          the visible name of the table namespace
-   * @param description
-   *          the specific reason why it failed
-   */
-  public TableNamespaceNotEmptyException(String namespaceId, String namespaceName, String description) {
-    super("Table namespace" + (namespaceName != null && !namespaceName.isEmpty() ? " " + namespaceName : "")
-        + (namespaceId != null && !namespaceId.isEmpty() ? " (Id=" + namespaceId + ")" : "") + " it not empty, contains at least one table"
-        + (description != null && !description.isEmpty() ? " (" + description + ")" : ""));
-    this.namespace = namespaceName;
-  }
-
-  /**
-   * @param namespaceId
-   *          the internal id of the table namespace
-   * @param namespaceName
-   *          the visible name of the table namespace
-   * @param description
-   *          the specific reason why it failed
-   * @param cause
-   *          the exception that caused this failure
-   */
-  public TableNamespaceNotEmptyException(String namespaceId, String namespaceName, String description, Throwable cause) {
-    this(namespaceId, namespaceName, description);
-    super.initCause(cause);
-  }
-
-  /**
-   * @param e
-   *          constructs an exception from a thrift exception
-   */
-  public TableNamespaceNotEmptyException(ThriftTableOperationException e) {
-    this(e.getTableId(), e.getTableName(), e.getDescription(), e);
-  }
-
-  /**
-   * @return the name of the table namespace
-   */
-  public String getNamespaceName() {
-    return namespace;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotFoundException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotFoundException.java b/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotFoundException.java
deleted file mode 100644
index c4db89b..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotFoundException.java
+++ /dev/null
@@ -1,77 +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.core.client;
-
-import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
-
-/**
- * Thrown when the table namespace specified doesn't exist when it was expected to
- */
-public class TableNamespaceNotFoundException extends Exception {
-  /**
-   * Exception to throw if an operation is attempted on a table namespace that doesn't exist.
-   * 
-   */
-  private static final long serialVersionUID = 1L;
-
-  private String namespace;
-
-  /**
-   * @param namespaceId
-   *          the internal id of the table namespace that was sought
-   * @param namespaceName
-   *          the visible name of the table namespace that was sought
-   * @param description
-   *          the specific reason why it failed
-   */
-  public TableNamespaceNotFoundException(String namespaceId, String namespaceName, String description) {
-    super("Table namespace" + (namespaceName != null && !namespaceName.isEmpty() ? " " + namespaceName : "")
-        + (namespaceId != null && !namespaceId.isEmpty() ? " (Id=" + namespaceId + ")" : "") + " does not exist"
-        + (description != null && !description.isEmpty() ? " (" + description + ")" : ""));
-    this.namespace = namespaceName;
-  }
-
-  /**
-   * @param namespaceId
-   *          the internal id of the table namespace that was sought
-   * @param namespaceName
-   *          the visible name of the table namespace that was sought
-   * @param description
-   *          the specific reason why it failed
-   * @param cause
-   *          the exception that caused this failure
-   */
-  public TableNamespaceNotFoundException(String namespaceId, String namespaceName, String description, Throwable cause) {
-    this(namespaceId, namespaceName, description);
-    super.initCause(cause);
-  }
-
-  /**
-   * @param e
-   *          constructs an exception from a thrift exception
-   */
-  public TableNamespaceNotFoundException(ThriftTableOperationException e) {
-    this(e.getTableId(), e.getTableName(), e.getDescription(), e);
-  }
-
-  /**
-   * @return the name of the table namespace sought
-   */
-  public String getNamespaceName() {
-    return namespace;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java
new file mode 100644
index 0000000..c82b42e
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java
@@ -0,0 +1,401 @@
+/*
+ * 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.core.client.admin;
+
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.SortedSet;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.NamespaceExistsException;
+import org.apache.accumulo.core.client.NamespaceNotEmptyException;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+
+/**
+ * Provides a class for administering namespaces
+ * 
+ */
+
+public interface NamespaceOperations {
+
+  /**
+   * Retrieve a list of namespaces in Accumulo.
+   * 
+   * @return List of namespaces in accumulo
+   */
+  public SortedSet<String> list();
+
+  /**
+   * A method to check if a namespace exists in Accumulo.
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @return true if the namespace exists
+   */
+  public boolean exists(String namespace);
+
+  /**
+   * Create a namespace with no special configuration
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   * @throws NamespaceExistsException
+   *           if the namespace already exists
+   */
+  public void create(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException;
+
+  /**
+   * @param namespace
+   *          the name of the namespace
+   * @param limitVersion
+   *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   * @throws NamespaceExistsException
+   *           if the namespace already exists
+   */
+  public void create(String namespace, boolean limitVersion) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException;
+
+  /**
+   * @param namespace
+   *          the name of the namespace
+   * @param versioningIter
+   *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
+   * @param timeType
+   *          specifies logical or real-time based time recording for entries in the table
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   * @throws NamespaceExistsException
+   *           if the namespace already exists
+   */
+  public void create(String namespace, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException;
+
+  /**
+   * Delete a namespace if it is empty
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   * @throws NamespaceNotFoundException
+   *           if the namespace does not exist
+   * @throws NamespaceNotEmptyException
+   *           if the namespaces still contains tables
+   * @throws TableNotFoundException
+   *           if table not found while deleting
+   */
+  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException, NamespaceNotEmptyException,
+      TableNotFoundException;
+
+  /**
+   * Delete a namespace
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @param deleteTables
+   *          boolean, if true deletes all the tables in the namespace in addition to deleting the namespace.
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   * @throws NamespaceNotFoundException
+   *           if the namespace does not exist
+   * @throws NamespaceNotEmptyException
+   *           if the namespaces still contains tables
+   * @throws TableNotFoundException
+   *           if table not found while deleting
+   */
+  public void delete(String namespace, boolean deleteTables) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException,
+      NamespaceNotEmptyException, TableNotFoundException;
+
+  /**
+   * Rename a namespace
+   * 
+   * @param oldNamespaceName
+   *          the old namespace name
+   * @param newNamespaceName
+   *          the new namespace name
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   * @throws NamespaceNotFoundException
+   *           if the old namespace does not exist
+   * @throws NamespaceExistsException
+   *           if the new namespace already exists
+   */
+  public void rename(String oldNamespaceName, String newNamespaceName) throws AccumuloSecurityException, NamespaceNotFoundException, AccumuloException,
+      NamespaceExistsException;
+
+  /**
+   * Sets a property on a namespace which applies to all tables in the namespace. Note that it may take a short period of time (a second) to propagate the
+   * change everywhere.
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @param property
+   *          the name of a per-table property
+   * @param value
+   *          the value to set a per-table property to
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   */
+  public void setProperty(String namespace, String property, String value) throws AccumuloException, AccumuloSecurityException;
+
+  /**
+   * Removes a property from a namespace. Note that it may take a short period of time (a second) to propagate the change everywhere.
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @param property
+   *          the name of a per-table property
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   */
+  public void removeProperty(String namespace, String property) throws AccumuloException, AccumuloSecurityException;
+
+  /**
+   * Gets properties of a namespace. Note that recently changed properties may not be available immediately.
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @return all properties visible by this table (system and per-table properties). Note that recently changed properties may not be visible immediately.
+   * @throws NamespaceNotFoundException
+   *           if the namespace does not exist
+   */
+  public Iterable<Entry<String,String>> getProperties(String namespace) throws AccumuloException, NamespaceNotFoundException;
+
+  /**
+   * 
+   * @param namespace
+   *          the namespace to take offline
+   * @throws AccumuloException
+   *           when there is a general accumulo error
+   * @throws AccumuloSecurityException
+   *           when the user does not have the proper permissions
+   * @throws NamespaceNotFoundException
+   *           if the namespace does not exist
+   */
+  public void offline(String namespace) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException;
+
+  /**
+   * 
+   * @param namespace
+   *          the namespace to take online
+   * @throws AccumuloException
+   *           when there is a general accumulo error
+   * @throws AccumuloSecurityException
+   *           when the user does not have the proper permissions
+   * @throws NamespaceNotFoundException
+   *           if the namespace does not exist
+   */
+  public void online(String namespace) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException;
+
+  /**
+   * Get a mapping of namespace name to internal namespace id.
+   * 
+   * @return the map from namespace name to internal namespace id
+   */
+  public Map<String,String> namespaceIdMap();
+
+  /**
+   * Gets the number of bytes being used in the files for the set of tables in this namespace
+   * 
+   * @param namespace
+   *          the namespace to get the set of tables from
+   * 
+   * @return a list of disk usage objects containing linked table names and sizes
+   * @throws AccumuloException
+   *           when there is a general accumulo error
+   * @throws AccumuloSecurityException
+   *           when the user does not have the proper permissions
+   */
+  public List<DiskUsage> getDiskUsage(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException;
+
+  /**
+   * Add an iterator to a namespace on all scopes.
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @param setting
+   *          object specifying the properties of the iterator
+   * @throws AccumuloSecurityException
+   *           thrown if the user does not have the ability to set properties on the namespace
+   * @throws AccumuloException
+   * @throws NamespaceNotFoundException
+   *           throw if the namespace no longer exists
+   * @throws IllegalArgumentException
+   *           if the setting conflicts with any existing iterators
+   */
+  public void attachIterator(String namespace, IteratorSetting setting) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException;
+
+  /**
+   * Add an iterator to a namespace on the given scopes.
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @param setting
+   *          object specifying the properties of the iterator
+   * @throws AccumuloSecurityException
+   *           thrown if the user does not have the ability to set properties on the namespace
+   * @throws AccumuloException
+   * @throws NamespaceNotFoundException
+   *           throw if the namespace no longer exists
+   * @throws IllegalArgumentException
+   *           if the setting conflicts with any existing iterators
+   */
+  public void attachIterator(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+      NamespaceNotFoundException;
+
+  /**
+   * Remove an iterator from a namespace by name.
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @param name
+   *          the name of the iterator
+   * @param scopes
+   *          the scopes of the iterator
+   * @throws AccumuloSecurityException
+   *           thrown if the user does not have the ability to set properties on the namespace
+   * @throws AccumuloException
+   * @throws NamespaceNotFoundException
+   *           thrown if the namespace no longer exists
+   */
+  public void removeIterator(String namespace, String name, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+      NamespaceNotFoundException;
+
+  /**
+   * Get the settings for an iterator.
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @param name
+   *          the name of the iterator
+   * @param scope
+   *          the scope of the iterator
+   * @return the settings for this iterator
+   * @throws AccumuloSecurityException
+   *           thrown if the user does not have the ability to set properties on the namespace
+   * @throws AccumuloException
+   * @throws NamespaceNotFoundException
+   *           thrown if the namespace no longer exists
+   */
+  public IteratorSetting getIteratorSetting(String namespace, String name, IteratorScope scope) throws AccumuloSecurityException, AccumuloException,
+      NumberFormatException, NamespaceNotFoundException;
+
+  /**
+   * Get a list of iterators for this namespace.
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @return a set of iterator names
+   * @throws AccumuloSecurityException
+   *           thrown if the user does not have the ability to set properties on the namespace
+   * @throws AccumuloException
+   * @throws NamespaceNotFoundException
+   *           thrown if the namespace no longer exists
+   */
+  public Map<String,EnumSet<IteratorScope>> listIterators(String namespace) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException;
+
+  /**
+   * Check whether a given iterator configuration conflicts with existing configuration; in particular, determine if the name or priority are already in use for
+   * the specified scopes.
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @param setting
+   *          object specifying the properties of the iterator
+   * @throws AccumuloException
+   * @throws NamespaceNotFoundException
+   *           thrown if the namespace no longer exists
+   * @throws IllegalStateException
+   *           if the setting conflicts with any existing iterators
+   */
+  public void checkIteratorConflicts(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException,
+      NamespaceNotFoundException;
+
+  /**
+   * Add a new constraint to a namespace.
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @param constraintClassName
+   *          the full name of the constraint class
+   * @return the unique number assigned to the constraint
+   * @throws AccumuloException
+   *           thrown if the constraint has already been added to the table or if there are errors in the configuration of existing constraints
+   * @throws AccumuloSecurityException
+   *           thrown if the user doesn't have permission to add the constraint
+   * @throws NamespaceNotFoundException
+   *           thrown if the namespace no longer exists
+   */
+  public int addConstraint(String namespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException;
+
+  /**
+   * Remove a constraint from a namespace.
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @param number
+   *          the unique number assigned to the constraint
+   * @throws AccumuloException
+   * @throws AccumuloSecurityException
+   *           thrown if the user doesn't have permission to remove the constraint
+   */
+  public void removeConstraint(String namespace, int number) throws AccumuloException, AccumuloSecurityException;
+
+  /**
+   * List constraints on a namespace with their assigned numbers.
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @return a map from constraint class name to assigned number
+   * @throws AccumuloException
+   *           thrown if there are errors in the configuration of existing constraints
+   * @throws NamespaceNotFoundException
+   *           thrown if the namespace no longer exists
+   */
+  public Map<String,Integer> listConstraints(String namespace) throws AccumuloException, NamespaceNotFoundException;
+
+  /**
+   * Test to see if the instance can load the given class as the given type. This check uses the table classpath property if it is set.
+   * 
+   * @return true if the instance can load the given class as the given type, false otherwise
+   */
+  boolean testClassLoad(String namespace, String className, String asTypeName) throws NamespaceNotFoundException, AccumuloException, AccumuloSecurityException;
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsHelper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsHelper.java
new file mode 100644
index 0000000..007461f
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsHelper.java
@@ -0,0 +1,203 @@
+/*
+ * 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.core.client.admin;
+
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+
+public abstract class NamespaceOperationsHelper implements NamespaceOperations {
+
+  @Override
+  public void attachIterator(String namespace, IteratorSetting setting) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException {
+    attachIterator(namespace, setting, EnumSet.allOf(IteratorScope.class));
+  }
+
+  @Override
+  public void attachIterator(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+      NamespaceNotFoundException {
+    checkIteratorConflicts(namespace, setting, scopes);
+    for (IteratorScope scope : scopes) {
+      String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), setting.getName());
+      for (Entry<String,String> prop : setting.getOptions().entrySet()) {
+        this.setProperty(namespace, root + ".opt." + prop.getKey(), prop.getValue());
+      }
+      this.setProperty(namespace, root, setting.getPriority() + "," + setting.getIteratorClass());
+    }
+  }
+
+  @Override
+  public void removeIterator(String namespace, String name, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+      NamespaceNotFoundException {
+    if (!exists(namespace))
+      throw new NamespaceNotFoundException(null, namespace, null);
+    Map<String,String> copy = new TreeMap<String,String>();
+    for (Entry<String,String> property : this.getProperties(namespace)) {
+      copy.put(property.getKey(), property.getValue());
+    }
+    for (IteratorScope scope : scopes) {
+      String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), name);
+      for (Entry<String,String> property : copy.entrySet()) {
+        if (property.getKey().equals(root) || property.getKey().startsWith(root + ".opt."))
+          this.removeProperty(namespace, property.getKey());
+      }
+    }
+  }
+
+  @Override
+  public IteratorSetting getIteratorSetting(String namespace, String name, IteratorScope scope) throws AccumuloSecurityException, AccumuloException,
+      NamespaceNotFoundException {
+    if (!exists(namespace))
+      throw new NamespaceNotFoundException(null, namespace, null);
+    int priority = -1;
+    String classname = null;
+    Map<String,String> settings = new HashMap<String,String>();
+
+    String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), name);
+    String opt = root + ".opt.";
+    for (Entry<String,String> property : this.getProperties(namespace)) {
+      if (property.getKey().equals(root)) {
+        String parts[] = property.getValue().split(",");
+        if (parts.length != 2) {
+          throw new AccumuloException("Bad value for iterator setting: " + property.getValue());
+        }
+        priority = Integer.parseInt(parts[0]);
+        classname = parts[1];
+      } else if (property.getKey().startsWith(opt)) {
+        settings.put(property.getKey().substring(opt.length()), property.getValue());
+      }
+    }
+    if (priority <= 0 || classname == null) {
+      return null;
+    }
+    return new IteratorSetting(priority, name, classname, settings);
+  }
+
+  @Override
+  public Map<String,EnumSet<IteratorScope>> listIterators(String namespace) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException {
+    if (!exists(namespace))
+      throw new NamespaceNotFoundException(null, namespace, null);
+    Map<String,EnumSet<IteratorScope>> result = new TreeMap<String,EnumSet<IteratorScope>>();
+    for (Entry<String,String> property : this.getProperties(namespace)) {
+      String name = property.getKey();
+      String[] parts = name.split("\\.");
+      if (parts.length == 4) {
+        if (parts[0].equals("table") && parts[1].equals("iterator")) {
+          IteratorScope scope = IteratorScope.valueOf(parts[2]);
+          if (!result.containsKey(parts[3]))
+            result.put(parts[3], EnumSet.noneOf(IteratorScope.class));
+          result.get(parts[3]).add(scope);
+        }
+      }
+    }
+    return result;
+  }
+
+  @Override
+  public void checkIteratorConflicts(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException,
+      NamespaceNotFoundException {
+    if (!exists(namespace))
+      throw new NamespaceNotFoundException(null, namespace, null);
+    for (IteratorScope scope : scopes) {
+      String scopeStr = String.format("%s%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase());
+      String nameStr = String.format("%s.%s", scopeStr, setting.getName());
+      String optStr = String.format("%s.opt.", nameStr);
+      Map<String,String> optionConflicts = new TreeMap<String,String>();
+      for (Entry<String,String> property : this.getProperties(namespace)) {
+        if (property.getKey().startsWith(scopeStr)) {
+          if (property.getKey().equals(nameStr))
+            throw new AccumuloException(new IllegalArgumentException("iterator name conflict for " + setting.getName() + ": " + property.getKey() + "="
+                + property.getValue()));
+          if (property.getKey().startsWith(optStr))
+            optionConflicts.put(property.getKey(), property.getValue());
+          if (property.getKey().contains(".opt."))
+            continue;
+          String parts[] = property.getValue().split(",");
+          if (parts.length != 2)
+            throw new AccumuloException("Bad value for existing iterator setting: " + property.getKey() + "=" + property.getValue());
+          try {
+            if (Integer.parseInt(parts[0]) == setting.getPriority())
+              throw new AccumuloException(new IllegalArgumentException("iterator priority conflict: " + property.getKey() + "=" + property.getValue()));
+          } catch (NumberFormatException e) {
+            throw new AccumuloException("Bad value for existing iterator setting: " + property.getKey() + "=" + property.getValue());
+          }
+        }
+      }
+      if (optionConflicts.size() > 0)
+        throw new AccumuloException(new IllegalArgumentException("iterator options conflict for " + setting.getName() + ": " + optionConflicts));
+    }
+  }
+
+  @Override
+  public int addConstraint(String namespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
+    TreeSet<Integer> constraintNumbers = new TreeSet<Integer>();
+    TreeMap<String,Integer> constraintClasses = new TreeMap<String,Integer>();
+    int i;
+    for (Entry<String,String> property : this.getProperties(namespace)) {
+      if (property.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.toString())) {
+        try {
+          i = Integer.parseInt(property.getKey().substring(Property.TABLE_CONSTRAINT_PREFIX.toString().length()));
+        } catch (NumberFormatException e) {
+          throw new AccumuloException("Bad key for existing constraint: " + property.toString());
+        }
+        constraintNumbers.add(i);
+        constraintClasses.put(property.getValue(), i);
+      }
+    }
+    i = 1;
+    while (constraintNumbers.contains(i))
+      i++;
+    if (constraintClasses.containsKey(constraintClassName))
+      throw new AccumuloException("Constraint " + constraintClassName + " already exists for namespace " + namespace + " with number "
+          + constraintClasses.get(constraintClassName));
+    this.setProperty(namespace, Property.TABLE_CONSTRAINT_PREFIX.toString() + i, constraintClassName);
+    return i;
+  }
+
+  @Override
+  public void removeConstraint(String namespace, int number) throws AccumuloException, AccumuloSecurityException {
+    this.removeProperty(namespace, Property.TABLE_CONSTRAINT_PREFIX.toString() + number);
+  }
+
+  @Override
+  public Map<String,Integer> listConstraints(String namespace) throws AccumuloException, NamespaceNotFoundException {
+    Map<String,Integer> constraints = new TreeMap<String,Integer>();
+    for (Entry<String,String> property : this.getProperties(namespace)) {
+      if (property.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.toString())) {
+        if (constraints.containsKey(property.getValue()))
+          throw new AccumuloException("Same constraint configured twice: " + property.getKey() + "=" + Property.TABLE_CONSTRAINT_PREFIX
+              + constraints.get(property.getValue()) + "=" + property.getKey());
+        try {
+          constraints.put(property.getValue(), Integer.parseInt(property.getKey().substring(Property.TABLE_CONSTRAINT_PREFIX.toString().length())));
+        } catch (NumberFormatException e) {
+          throw new AccumuloException("Bad key for existing constraint: " + property.toString());
+        }
+      }
+    }
+    return constraints;
+  }
+}


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

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/ClientService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/ClientService.java b/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/ClientService.java
index 3ccb234..195f372 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/ClientService.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/ClientService.java
@@ -84,7 +84,7 @@ import org.slf4j.LoggerFactory;
 
     public boolean hasTablePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableName, byte tblPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
 
-    public boolean hasTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte tblNspcPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+    public boolean hasNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String ns, byte tblNspcPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
 
     public void grantSystemPermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, byte permission) throws ThriftSecurityException, org.apache.thrift.TException;
 
@@ -94,21 +94,21 @@ import org.slf4j.LoggerFactory;
 
     public void revokeTablePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableName, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
 
-    public void grantTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+    public void grantNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
 
-    public void revokeTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+    public void revokeNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
 
     public Map<String,String> getConfiguration(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, ConfigurationType type) throws org.apache.thrift.TException;
 
     public Map<String,String> getTableConfiguration(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String tableName) throws ThriftTableOperationException, org.apache.thrift.TException;
 
-    public Map<String,String> getTableNamespaceConfiguration(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns) throws ThriftTableOperationException, org.apache.thrift.TException;
+    public Map<String,String> getNamespaceConfiguration(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns) throws ThriftTableOperationException, org.apache.thrift.TException;
 
     public boolean checkClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String className, String interfaceMatch) throws org.apache.thrift.TException;
 
     public boolean checkTableClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String tableId, String className, String interfaceMatch) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
 
-    public boolean checkTableNamespaceClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String namespaceId, String className, String interfaceMatch) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+    public boolean checkNamespaceClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String namespaceId, String className, String interfaceMatch) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
 
   }
 
@@ -148,7 +148,7 @@ import org.slf4j.LoggerFactory;
 
     public void hasTablePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableName, byte tblPerm, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.hasTablePermission_call> resultHandler) throws org.apache.thrift.TException;
 
-    public void hasTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte tblNspcPerm, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.hasTableNamespacePermission_call> resultHandler) throws org.apache.thrift.TException;
+    public void hasNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String ns, byte tblNspcPerm, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.hasNamespacePermission_call> resultHandler) throws org.apache.thrift.TException;
 
     public void grantSystemPermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.grantSystemPermission_call> resultHandler) throws org.apache.thrift.TException;
 
@@ -158,21 +158,21 @@ import org.slf4j.LoggerFactory;
 
     public void revokeTablePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.revokeTablePermission_call> resultHandler) throws org.apache.thrift.TException;
 
-    public void grantTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.grantTableNamespacePermission_call> resultHandler) throws org.apache.thrift.TException;
+    public void grantNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.grantNamespacePermission_call> resultHandler) throws org.apache.thrift.TException;
 
-    public void revokeTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.revokeTableNamespacePermission_call> resultHandler) throws org.apache.thrift.TException;
+    public void revokeNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.revokeNamespacePermission_call> resultHandler) throws org.apache.thrift.TException;
 
     public void getConfiguration(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, ConfigurationType type, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.getConfiguration_call> resultHandler) throws org.apache.thrift.TException;
 
     public void getTableConfiguration(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String tableName, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.getTableConfiguration_call> resultHandler) throws org.apache.thrift.TException;
 
-    public void getTableNamespaceConfiguration(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.getTableNamespaceConfiguration_call> resultHandler) throws org.apache.thrift.TException;
+    public void getNamespaceConfiguration(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.getNamespaceConfiguration_call> resultHandler) throws org.apache.thrift.TException;
 
     public void checkClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String className, String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.checkClass_call> resultHandler) throws org.apache.thrift.TException;
 
     public void checkTableClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String tableId, String className, String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.checkTableClass_call> resultHandler) throws org.apache.thrift.TException;
 
-    public void checkTableNamespaceClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String namespaceId, String className, String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.checkTableNamespaceClass_call> resultHandler) throws org.apache.thrift.TException;
+    public void checkNamespaceClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String namespaceId, String className, String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.checkNamespaceClass_call> resultHandler) throws org.apache.thrift.TException;
 
   }
 
@@ -649,27 +649,27 @@ import org.slf4j.LoggerFactory;
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "hasTablePermission failed: unknown result");
     }
 
-    public boolean hasTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte tblNspcPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    public boolean hasNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String ns, byte tblNspcPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
     {
-      send_hasTableNamespacePermission(tinfo, credentials, principal, tableNamespace, tblNspcPerm);
-      return recv_hasTableNamespacePermission();
+      send_hasNamespacePermission(tinfo, credentials, principal, ns, tblNspcPerm);
+      return recv_hasNamespacePermission();
     }
 
-    public void send_hasTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte tblNspcPerm) throws org.apache.thrift.TException
+    public void send_hasNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String ns, byte tblNspcPerm) throws org.apache.thrift.TException
     {
-      hasTableNamespacePermission_args args = new hasTableNamespacePermission_args();
+      hasNamespacePermission_args args = new hasNamespacePermission_args();
       args.setTinfo(tinfo);
       args.setCredentials(credentials);
       args.setPrincipal(principal);
-      args.setTableNamespace(tableNamespace);
+      args.setNs(ns);
       args.setTblNspcPerm(tblNspcPerm);
-      sendBase("hasTableNamespacePermission", args);
+      sendBase("hasNamespacePermission", args);
     }
 
-    public boolean recv_hasTableNamespacePermission() throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    public boolean recv_hasNamespacePermission() throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
     {
-      hasTableNamespacePermission_result result = new hasTableNamespacePermission_result();
-      receiveBase(result, "hasTableNamespacePermission");
+      hasNamespacePermission_result result = new hasNamespacePermission_result();
+      receiveBase(result, "hasNamespacePermission");
       if (result.isSetSuccess()) {
         return result.success;
       }
@@ -679,7 +679,7 @@ import org.slf4j.LoggerFactory;
       if (result.tope != null) {
         throw result.tope;
       }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "hasTableNamespacePermission failed: unknown result");
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "hasNamespacePermission failed: unknown result");
     }
 
     public void grantSystemPermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, byte permission) throws ThriftSecurityException, org.apache.thrift.TException
@@ -794,27 +794,27 @@ import org.slf4j.LoggerFactory;
       return;
     }
 
-    public void grantTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    public void grantNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
     {
-      send_grantTableNamespacePermission(tinfo, credentials, principal, tableNamespace, permission);
-      recv_grantTableNamespacePermission();
+      send_grantNamespacePermission(tinfo, credentials, principal, ns, permission);
+      recv_grantNamespacePermission();
     }
 
-    public void send_grantTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission) throws org.apache.thrift.TException
+    public void send_grantNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String ns, byte permission) throws org.apache.thrift.TException
     {
-      grantTableNamespacePermission_args args = new grantTableNamespacePermission_args();
+      grantNamespacePermission_args args = new grantNamespacePermission_args();
       args.setTinfo(tinfo);
       args.setCredentials(credentials);
       args.setPrincipal(principal);
-      args.setTableNamespace(tableNamespace);
+      args.setNs(ns);
       args.setPermission(permission);
-      sendBase("grantTableNamespacePermission", args);
+      sendBase("grantNamespacePermission", args);
     }
 
-    public void recv_grantTableNamespacePermission() throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    public void recv_grantNamespacePermission() throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
     {
-      grantTableNamespacePermission_result result = new grantTableNamespacePermission_result();
-      receiveBase(result, "grantTableNamespacePermission");
+      grantNamespacePermission_result result = new grantNamespacePermission_result();
+      receiveBase(result, "grantNamespacePermission");
       if (result.sec != null) {
         throw result.sec;
       }
@@ -824,27 +824,27 @@ import org.slf4j.LoggerFactory;
       return;
     }
 
-    public void revokeTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    public void revokeNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String ns, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
     {
-      send_revokeTableNamespacePermission(tinfo, credentials, principal, tableNamespace, permission);
-      recv_revokeTableNamespacePermission();
+      send_revokeNamespacePermission(tinfo, credentials, principal, ns, permission);
+      recv_revokeNamespacePermission();
     }
 
-    public void send_revokeTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission) throws org.apache.thrift.TException
+    public void send_revokeNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String ns, byte permission) throws org.apache.thrift.TException
     {
-      revokeTableNamespacePermission_args args = new revokeTableNamespacePermission_args();
+      revokeNamespacePermission_args args = new revokeNamespacePermission_args();
       args.setTinfo(tinfo);
       args.setCredentials(credentials);
       args.setPrincipal(principal);
-      args.setTableNamespace(tableNamespace);
+      args.setNs(ns);
       args.setPermission(permission);
-      sendBase("revokeTableNamespacePermission", args);
+      sendBase("revokeNamespacePermission", args);
     }
 
-    public void recv_revokeTableNamespacePermission() throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    public void recv_revokeNamespacePermission() throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
     {
-      revokeTableNamespacePermission_result result = new revokeTableNamespacePermission_result();
-      receiveBase(result, "revokeTableNamespacePermission");
+      revokeNamespacePermission_result result = new revokeNamespacePermission_result();
+      receiveBase(result, "revokeNamespacePermission");
       if (result.sec != null) {
         throw result.sec;
       }
@@ -907,32 +907,32 @@ import org.slf4j.LoggerFactory;
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTableConfiguration failed: unknown result");
     }
 
-    public Map<String,String> getTableNamespaceConfiguration(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns) throws ThriftTableOperationException, org.apache.thrift.TException
+    public Map<String,String> getNamespaceConfiguration(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns) throws ThriftTableOperationException, org.apache.thrift.TException
     {
-      send_getTableNamespaceConfiguration(tinfo, credentials, ns);
-      return recv_getTableNamespaceConfiguration();
+      send_getNamespaceConfiguration(tinfo, credentials, ns);
+      return recv_getNamespaceConfiguration();
     }
 
-    public void send_getTableNamespaceConfiguration(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns) throws org.apache.thrift.TException
+    public void send_getNamespaceConfiguration(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns) throws org.apache.thrift.TException
     {
-      getTableNamespaceConfiguration_args args = new getTableNamespaceConfiguration_args();
+      getNamespaceConfiguration_args args = new getNamespaceConfiguration_args();
       args.setTinfo(tinfo);
       args.setCredentials(credentials);
       args.setNs(ns);
-      sendBase("getTableNamespaceConfiguration", args);
+      sendBase("getNamespaceConfiguration", args);
     }
 
-    public Map<String,String> recv_getTableNamespaceConfiguration() throws ThriftTableOperationException, org.apache.thrift.TException
+    public Map<String,String> recv_getNamespaceConfiguration() throws ThriftTableOperationException, org.apache.thrift.TException
     {
-      getTableNamespaceConfiguration_result result = new getTableNamespaceConfiguration_result();
-      receiveBase(result, "getTableNamespaceConfiguration");
+      getNamespaceConfiguration_result result = new getNamespaceConfiguration_result();
+      receiveBase(result, "getNamespaceConfiguration");
       if (result.isSetSuccess()) {
         return result.success;
       }
       if (result.tope != null) {
         throw result.tope;
       }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTableNamespaceConfiguration failed: unknown result");
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getNamespaceConfiguration failed: unknown result");
     }
 
     public boolean checkClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String className, String interfaceMatch) throws org.apache.thrift.TException
@@ -994,27 +994,27 @@ import org.slf4j.LoggerFactory;
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "checkTableClass failed: unknown result");
     }
 
-    public boolean checkTableNamespaceClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String namespaceId, String className, String interfaceMatch) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    public boolean checkNamespaceClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String namespaceId, String className, String interfaceMatch) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
     {
-      send_checkTableNamespaceClass(tinfo, credentials, namespaceId, className, interfaceMatch);
-      return recv_checkTableNamespaceClass();
+      send_checkNamespaceClass(tinfo, credentials, namespaceId, className, interfaceMatch);
+      return recv_checkNamespaceClass();
     }
 
-    public void send_checkTableNamespaceClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String namespaceId, String className, String interfaceMatch) throws org.apache.thrift.TException
+    public void send_checkNamespaceClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String namespaceId, String className, String interfaceMatch) throws org.apache.thrift.TException
     {
-      checkTableNamespaceClass_args args = new checkTableNamespaceClass_args();
+      checkNamespaceClass_args args = new checkNamespaceClass_args();
       args.setTinfo(tinfo);
       args.setCredentials(credentials);
       args.setNamespaceId(namespaceId);
       args.setClassName(className);
       args.setInterfaceMatch(interfaceMatch);
-      sendBase("checkTableNamespaceClass", args);
+      sendBase("checkNamespaceClass", args);
     }
 
-    public boolean recv_checkTableNamespaceClass() throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    public boolean recv_checkNamespaceClass() throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
     {
-      checkTableNamespaceClass_result result = new checkTableNamespaceClass_result();
-      receiveBase(result, "checkTableNamespaceClass");
+      checkNamespaceClass_result result = new checkNamespaceClass_result();
+      receiveBase(result, "checkNamespaceClass");
       if (result.isSetSuccess()) {
         return result.success;
       }
@@ -1024,7 +1024,7 @@ import org.slf4j.LoggerFactory;
       if (result.tope != null) {
         throw result.tope;
       }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "checkTableNamespaceClass failed: unknown result");
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "checkNamespaceClass failed: unknown result");
     }
 
   }
@@ -1676,35 +1676,35 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public void hasTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte tblNspcPerm, org.apache.thrift.async.AsyncMethodCallback<hasTableNamespacePermission_call> resultHandler) throws org.apache.thrift.TException {
+    public void hasNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String ns, byte tblNspcPerm, org.apache.thrift.async.AsyncMethodCallback<hasNamespacePermission_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      hasTableNamespacePermission_call method_call = new hasTableNamespacePermission_call(tinfo, credentials, principal, tableNamespace, tblNspcPerm, resultHandler, this, ___protocolFactory, ___transport);
+      hasNamespacePermission_call method_call = new hasNamespacePermission_call(tinfo, credentials, principal, ns, tblNspcPerm, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class hasTableNamespacePermission_call extends org.apache.thrift.async.TAsyncMethodCall {
+    public static class hasNamespacePermission_call extends org.apache.thrift.async.TAsyncMethodCall {
       private org.apache.accumulo.trace.thrift.TInfo tinfo;
       private org.apache.accumulo.core.security.thrift.TCredentials credentials;
       private String principal;
-      private String tableNamespace;
+      private String ns;
       private byte tblNspcPerm;
-      public hasTableNamespacePermission_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte tblNspcPerm, org.apache.thrift.async.AsyncMethodCallback<hasTableNamespacePermission_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public hasNamespacePermission_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String ns, byte tblNspcPerm, org.apache.thrift.async.AsyncMethodCallback<hasNamespacePermission_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
         this.principal = principal;
-        this.tableNamespace = tableNamespace;
+        this.ns = ns;
         this.tblNspcPerm = tblNspcPerm;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("hasTableNamespacePermission", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        hasTableNamespacePermission_args args = new hasTableNamespacePermission_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("hasNamespacePermission", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        hasNamespacePermission_args args = new hasNamespacePermission_args();
         args.setTinfo(tinfo);
         args.setCredentials(credentials);
         args.setPrincipal(principal);
-        args.setTableNamespace(tableNamespace);
+        args.setNs(ns);
         args.setTblNspcPerm(tblNspcPerm);
         args.write(prot);
         prot.writeMessageEnd();
@@ -1716,7 +1716,7 @@ import org.slf4j.LoggerFactory;
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_hasTableNamespacePermission();
+        return (new Client(prot)).recv_hasNamespacePermission();
       }
     }
 
@@ -1890,35 +1890,35 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public void grantTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission, org.apache.thrift.async.AsyncMethodCallback<grantTableNamespacePermission_call> resultHandler) throws org.apache.thrift.TException {
+    public void grantNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<grantNamespacePermission_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      grantTableNamespacePermission_call method_call = new grantTableNamespacePermission_call(tinfo, credentials, principal, tableNamespace, permission, resultHandler, this, ___protocolFactory, ___transport);
+      grantNamespacePermission_call method_call = new grantNamespacePermission_call(tinfo, credentials, principal, ns, permission, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class grantTableNamespacePermission_call extends org.apache.thrift.async.TAsyncMethodCall {
+    public static class grantNamespacePermission_call extends org.apache.thrift.async.TAsyncMethodCall {
       private org.apache.accumulo.trace.thrift.TInfo tinfo;
       private org.apache.accumulo.core.security.thrift.TCredentials credentials;
       private String principal;
-      private String tableNamespace;
+      private String ns;
       private byte permission;
-      public grantTableNamespacePermission_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission, org.apache.thrift.async.AsyncMethodCallback<grantTableNamespacePermission_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public grantNamespacePermission_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<grantNamespacePermission_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
         this.principal = principal;
-        this.tableNamespace = tableNamespace;
+        this.ns = ns;
         this.permission = permission;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("grantTableNamespacePermission", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        grantTableNamespacePermission_args args = new grantTableNamespacePermission_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("grantNamespacePermission", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        grantNamespacePermission_args args = new grantNamespacePermission_args();
         args.setTinfo(tinfo);
         args.setCredentials(credentials);
         args.setPrincipal(principal);
-        args.setTableNamespace(tableNamespace);
+        args.setNs(ns);
         args.setPermission(permission);
         args.write(prot);
         prot.writeMessageEnd();
@@ -1930,39 +1930,39 @@ import org.slf4j.LoggerFactory;
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        (new Client(prot)).recv_grantTableNamespacePermission();
+        (new Client(prot)).recv_grantNamespacePermission();
       }
     }
 
-    public void revokeTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission, org.apache.thrift.async.AsyncMethodCallback<revokeTableNamespacePermission_call> resultHandler) throws org.apache.thrift.TException {
+    public void revokeNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<revokeNamespacePermission_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      revokeTableNamespacePermission_call method_call = new revokeTableNamespacePermission_call(tinfo, credentials, principal, tableNamespace, permission, resultHandler, this, ___protocolFactory, ___transport);
+      revokeNamespacePermission_call method_call = new revokeNamespacePermission_call(tinfo, credentials, principal, ns, permission, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class revokeTableNamespacePermission_call extends org.apache.thrift.async.TAsyncMethodCall {
+    public static class revokeNamespacePermission_call extends org.apache.thrift.async.TAsyncMethodCall {
       private org.apache.accumulo.trace.thrift.TInfo tinfo;
       private org.apache.accumulo.core.security.thrift.TCredentials credentials;
       private String principal;
-      private String tableNamespace;
+      private String ns;
       private byte permission;
-      public revokeTableNamespacePermission_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission, org.apache.thrift.async.AsyncMethodCallback<revokeTableNamespacePermission_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public revokeNamespacePermission_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String ns, byte permission, org.apache.thrift.async.AsyncMethodCallback<revokeNamespacePermission_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
         this.principal = principal;
-        this.tableNamespace = tableNamespace;
+        this.ns = ns;
         this.permission = permission;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("revokeTableNamespacePermission", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        revokeTableNamespacePermission_args args = new revokeTableNamespacePermission_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("revokeNamespacePermission", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        revokeNamespacePermission_args args = new revokeNamespacePermission_args();
         args.setTinfo(tinfo);
         args.setCredentials(credentials);
         args.setPrincipal(principal);
-        args.setTableNamespace(tableNamespace);
+        args.setNs(ns);
         args.setPermission(permission);
         args.write(prot);
         prot.writeMessageEnd();
@@ -1974,7 +1974,7 @@ import org.slf4j.LoggerFactory;
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        (new Client(prot)).recv_revokeTableNamespacePermission();
+        (new Client(prot)).recv_revokeNamespacePermission();
       }
     }
 
@@ -2054,18 +2054,18 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public void getTableNamespaceConfiguration(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, org.apache.thrift.async.AsyncMethodCallback<getTableNamespaceConfiguration_call> resultHandler) throws org.apache.thrift.TException {
+    public void getNamespaceConfiguration(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, org.apache.thrift.async.AsyncMethodCallback<getNamespaceConfiguration_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      getTableNamespaceConfiguration_call method_call = new getTableNamespaceConfiguration_call(tinfo, credentials, ns, resultHandler, this, ___protocolFactory, ___transport);
+      getNamespaceConfiguration_call method_call = new getNamespaceConfiguration_call(tinfo, credentials, ns, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class getTableNamespaceConfiguration_call extends org.apache.thrift.async.TAsyncMethodCall {
+    public static class getNamespaceConfiguration_call extends org.apache.thrift.async.TAsyncMethodCall {
       private org.apache.accumulo.trace.thrift.TInfo tinfo;
       private org.apache.accumulo.core.security.thrift.TCredentials credentials;
       private String ns;
-      public getTableNamespaceConfiguration_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, org.apache.thrift.async.AsyncMethodCallback<getTableNamespaceConfiguration_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public getNamespaceConfiguration_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, org.apache.thrift.async.AsyncMethodCallback<getNamespaceConfiguration_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
@@ -2073,8 +2073,8 @@ import org.slf4j.LoggerFactory;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTableNamespaceConfiguration", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getTableNamespaceConfiguration_args args = new getTableNamespaceConfiguration_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getNamespaceConfiguration", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getNamespaceConfiguration_args args = new getNamespaceConfiguration_args();
         args.setTinfo(tinfo);
         args.setCredentials(credentials);
         args.setNs(ns);
@@ -2088,7 +2088,7 @@ import org.slf4j.LoggerFactory;
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_getTableNamespaceConfiguration();
+        return (new Client(prot)).recv_getNamespaceConfiguration();
       }
     }
 
@@ -2177,20 +2177,20 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public void checkTableNamespaceClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String namespaceId, String className, String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<checkTableNamespaceClass_call> resultHandler) throws org.apache.thrift.TException {
+    public void checkNamespaceClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String namespaceId, String className, String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<checkNamespaceClass_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      checkTableNamespaceClass_call method_call = new checkTableNamespaceClass_call(tinfo, credentials, namespaceId, className, interfaceMatch, resultHandler, this, ___protocolFactory, ___transport);
+      checkNamespaceClass_call method_call = new checkNamespaceClass_call(tinfo, credentials, namespaceId, className, interfaceMatch, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class checkTableNamespaceClass_call extends org.apache.thrift.async.TAsyncMethodCall {
+    public static class checkNamespaceClass_call extends org.apache.thrift.async.TAsyncMethodCall {
       private org.apache.accumulo.trace.thrift.TInfo tinfo;
       private org.apache.accumulo.core.security.thrift.TCredentials credentials;
       private String namespaceId;
       private String className;
       private String interfaceMatch;
-      public checkTableNamespaceClass_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String namespaceId, String className, String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<checkTableNamespaceClass_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public checkNamespaceClass_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String namespaceId, String className, String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<checkNamespaceClass_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
@@ -2200,8 +2200,8 @@ import org.slf4j.LoggerFactory;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("checkTableNamespaceClass", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        checkTableNamespaceClass_args args = new checkTableNamespaceClass_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("checkNamespaceClass", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        checkNamespaceClass_args args = new checkNamespaceClass_args();
         args.setTinfo(tinfo);
         args.setCredentials(credentials);
         args.setNamespaceId(namespaceId);
@@ -2217,7 +2217,7 @@ import org.slf4j.LoggerFactory;
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_checkTableNamespaceClass();
+        return (new Client(prot)).recv_checkNamespaceClass();
       }
     }
 
@@ -2251,19 +2251,19 @@ import org.slf4j.LoggerFactory;
       processMap.put("getUserAuthorizations", new getUserAuthorizations());
       processMap.put("hasSystemPermission", new hasSystemPermission());
       processMap.put("hasTablePermission", new hasTablePermission());
-      processMap.put("hasTableNamespacePermission", new hasTableNamespacePermission());
+      processMap.put("hasNamespacePermission", new hasNamespacePermission());
       processMap.put("grantSystemPermission", new grantSystemPermission());
       processMap.put("revokeSystemPermission", new revokeSystemPermission());
       processMap.put("grantTablePermission", new grantTablePermission());
       processMap.put("revokeTablePermission", new revokeTablePermission());
-      processMap.put("grantTableNamespacePermission", new grantTableNamespacePermission());
-      processMap.put("revokeTableNamespacePermission", new revokeTableNamespacePermission());
+      processMap.put("grantNamespacePermission", new grantNamespacePermission());
+      processMap.put("revokeNamespacePermission", new revokeNamespacePermission());
       processMap.put("getConfiguration", new getConfiguration());
       processMap.put("getTableConfiguration", new getTableConfiguration());
-      processMap.put("getTableNamespaceConfiguration", new getTableNamespaceConfiguration());
+      processMap.put("getNamespaceConfiguration", new getNamespaceConfiguration());
       processMap.put("checkClass", new checkClass());
       processMap.put("checkTableClass", new checkTableClass());
-      processMap.put("checkTableNamespaceClass", new checkTableNamespaceClass());
+      processMap.put("checkNamespaceClass", new checkNamespaceClass());
       return processMap;
     }
 
@@ -2670,23 +2670,23 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public static class hasTableNamespacePermission<I extends Iface> extends org.apache.thrift.ProcessFunction<I, hasTableNamespacePermission_args> {
-      public hasTableNamespacePermission() {
-        super("hasTableNamespacePermission");
+    public static class hasNamespacePermission<I extends Iface> extends org.apache.thrift.ProcessFunction<I, hasNamespacePermission_args> {
+      public hasNamespacePermission() {
+        super("hasNamespacePermission");
       }
 
-      public hasTableNamespacePermission_args getEmptyArgsInstance() {
-        return new hasTableNamespacePermission_args();
+      public hasNamespacePermission_args getEmptyArgsInstance() {
+        return new hasNamespacePermission_args();
       }
 
       protected boolean isOneway() {
         return false;
       }
 
-      public hasTableNamespacePermission_result getResult(I iface, hasTableNamespacePermission_args args) throws org.apache.thrift.TException {
-        hasTableNamespacePermission_result result = new hasTableNamespacePermission_result();
+      public hasNamespacePermission_result getResult(I iface, hasNamespacePermission_args args) throws org.apache.thrift.TException {
+        hasNamespacePermission_result result = new hasNamespacePermission_result();
         try {
-          result.success = iface.hasTableNamespacePermission(args.tinfo, args.credentials, args.principal, args.tableNamespace, args.tblNspcPerm);
+          result.success = iface.hasNamespacePermission(args.tinfo, args.credentials, args.principal, args.ns, args.tblNspcPerm);
           result.setSuccessIsSet(true);
         } catch (ThriftSecurityException sec) {
           result.sec = sec;
@@ -2797,23 +2797,23 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public static class grantTableNamespacePermission<I extends Iface> extends org.apache.thrift.ProcessFunction<I, grantTableNamespacePermission_args> {
-      public grantTableNamespacePermission() {
-        super("grantTableNamespacePermission");
+    public static class grantNamespacePermission<I extends Iface> extends org.apache.thrift.ProcessFunction<I, grantNamespacePermission_args> {
+      public grantNamespacePermission() {
+        super("grantNamespacePermission");
       }
 
-      public grantTableNamespacePermission_args getEmptyArgsInstance() {
-        return new grantTableNamespacePermission_args();
+      public grantNamespacePermission_args getEmptyArgsInstance() {
+        return new grantNamespacePermission_args();
       }
 
       protected boolean isOneway() {
         return false;
       }
 
-      public grantTableNamespacePermission_result getResult(I iface, grantTableNamespacePermission_args args) throws org.apache.thrift.TException {
-        grantTableNamespacePermission_result result = new grantTableNamespacePermission_result();
+      public grantNamespacePermission_result getResult(I iface, grantNamespacePermission_args args) throws org.apache.thrift.TException {
+        grantNamespacePermission_result result = new grantNamespacePermission_result();
         try {
-          iface.grantTableNamespacePermission(args.tinfo, args.credentials, args.principal, args.tableNamespace, args.permission);
+          iface.grantNamespacePermission(args.tinfo, args.credentials, args.principal, args.ns, args.permission);
         } catch (ThriftSecurityException sec) {
           result.sec = sec;
         } catch (ThriftTableOperationException tope) {
@@ -2823,23 +2823,23 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public static class revokeTableNamespacePermission<I extends Iface> extends org.apache.thrift.ProcessFunction<I, revokeTableNamespacePermission_args> {
-      public revokeTableNamespacePermission() {
-        super("revokeTableNamespacePermission");
+    public static class revokeNamespacePermission<I extends Iface> extends org.apache.thrift.ProcessFunction<I, revokeNamespacePermission_args> {
+      public revokeNamespacePermission() {
+        super("revokeNamespacePermission");
       }
 
-      public revokeTableNamespacePermission_args getEmptyArgsInstance() {
-        return new revokeTableNamespacePermission_args();
+      public revokeNamespacePermission_args getEmptyArgsInstance() {
+        return new revokeNamespacePermission_args();
       }
 
       protected boolean isOneway() {
         return false;
       }
 
-      public revokeTableNamespacePermission_result getResult(I iface, revokeTableNamespacePermission_args args) throws org.apache.thrift.TException {
-        revokeTableNamespacePermission_result result = new revokeTableNamespacePermission_result();
+      public revokeNamespacePermission_result getResult(I iface, revokeNamespacePermission_args args) throws org.apache.thrift.TException {
+        revokeNamespacePermission_result result = new revokeNamespacePermission_result();
         try {
-          iface.revokeTableNamespacePermission(args.tinfo, args.credentials, args.principal, args.tableNamespace, args.permission);
+          iface.revokeNamespacePermission(args.tinfo, args.credentials, args.principal, args.ns, args.permission);
         } catch (ThriftSecurityException sec) {
           result.sec = sec;
         } catch (ThriftTableOperationException tope) {
@@ -2893,23 +2893,23 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public static class getTableNamespaceConfiguration<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getTableNamespaceConfiguration_args> {
-      public getTableNamespaceConfiguration() {
-        super("getTableNamespaceConfiguration");
+    public static class getNamespaceConfiguration<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getNamespaceConfiguration_args> {
+      public getNamespaceConfiguration() {
+        super("getNamespaceConfiguration");
       }
 
-      public getTableNamespaceConfiguration_args getEmptyArgsInstance() {
-        return new getTableNamespaceConfiguration_args();
+      public getNamespaceConfiguration_args getEmptyArgsInstance() {
+        return new getNamespaceConfiguration_args();
       }
 
       protected boolean isOneway() {
         return false;
       }
 
-      public getTableNamespaceConfiguration_result getResult(I iface, getTableNamespaceConfiguration_args args) throws org.apache.thrift.TException {
-        getTableNamespaceConfiguration_result result = new getTableNamespaceConfiguration_result();
+      public getNamespaceConfiguration_result getResult(I iface, getNamespaceConfiguration_args args) throws org.apache.thrift.TException {
+        getNamespaceConfiguration_result result = new getNamespaceConfiguration_result();
         try {
-          result.success = iface.getTableNamespaceConfiguration(args.tinfo, args.credentials, args.ns);
+          result.success = iface.getNamespaceConfiguration(args.tinfo, args.credentials, args.ns);
         } catch (ThriftTableOperationException tope) {
           result.tope = tope;
         }
@@ -2965,23 +2965,23 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public static class checkTableNamespaceClass<I extends Iface> extends org.apache.thrift.ProcessFunction<I, checkTableNamespaceClass_args> {
-      public checkTableNamespaceClass() {
-        super("checkTableNamespaceClass");
+    public static class checkNamespaceClass<I extends Iface> extends org.apache.thrift.ProcessFunction<I, checkNamespaceClass_args> {
+      public checkNamespaceClass() {
+        super("checkNamespaceClass");
       }
 
-      public checkTableNamespaceClass_args getEmptyArgsInstance() {
-        return new checkTableNamespaceClass_args();
+      public checkNamespaceClass_args getEmptyArgsInstance() {
+        return new checkNamespaceClass_args();
       }
 
       protected boolean isOneway() {
         return false;
       }
 
-      public checkTableNamespaceClass_result getResult(I iface, checkTableNamespaceClass_args args) throws org.apache.thrift.TException {
-        checkTableNamespaceClass_result result = new checkTableNamespaceClass_result();
+      public checkNamespaceClass_result getResult(I iface, checkNamespaceClass_args args) throws org.apache.thrift.TException {
+        checkNamespaceClass_result result = new checkNamespaceClass_result();
         try {
-          result.success = iface.checkTableNamespaceClass(args.tinfo, args.credentials, args.namespaceId, args.className, args.interfaceMatch);
+          result.success = iface.checkNamespaceClass(args.tinfo, args.credentials, args.namespaceId, args.className, args.interfaceMatch);
           result.setSuccessIsSet(true);
         } catch (ThriftSecurityException sec) {
           result.sec = sec;
@@ -19546,25 +19546,25 @@ import org.slf4j.LoggerFactory;
 
   }
 
-  public static class hasTableNamespacePermission_args implements org.apache.thrift.TBase<hasTableNamespacePermission_args, hasTableNamespacePermission_args._Fields>, java.io.Serializable, Cloneable   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("hasTableNamespacePermission_args");
+  public static class hasNamespacePermission_args implements org.apache.thrift.TBase<hasNamespacePermission_args, hasNamespacePermission_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("hasNamespacePermission_args");
 
     private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
     private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
     private static final org.apache.thrift.protocol.TField PRINCIPAL_FIELD_DESC = new org.apache.thrift.protocol.TField("principal", org.apache.thrift.protocol.TType.STRING, (short)3);
-    private static final org.apache.thrift.protocol.TField TABLE_NAMESPACE_FIELD_DESC = new org.apache.thrift.protocol.TField("tableNamespace", org.apache.thrift.protocol.TType.STRING, (short)4);
+    private static final org.apache.thrift.protocol.TField NS_FIELD_DESC = new org.apache.thrift.protocol.TField("ns", org.apache.thrift.protocol.TType.STRING, (short)4);
     private static final org.apache.thrift.protocol.TField TBL_NSPC_PERM_FIELD_DESC = new org.apache.thrift.protocol.TField("tblNspcPerm", org.apache.thrift.protocol.TType.BYTE, (short)5);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new hasTableNamespacePermission_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new hasTableNamespacePermission_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new hasNamespacePermission_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new hasNamespacePermission_argsTupleSchemeFactory());
     }
 
     public org.apache.accumulo.trace.thrift.TInfo tinfo; // required
     public org.apache.accumulo.core.security.thrift.TCredentials credentials; // required
     public String principal; // required
-    public String tableNamespace; // required
+    public String ns; // required
     public byte tblNspcPerm; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -19572,7 +19572,7 @@ import org.slf4j.LoggerFactory;
       TINFO((short)1, "tinfo"),
       CREDENTIALS((short)2, "credentials"),
       PRINCIPAL((short)3, "principal"),
-      TABLE_NAMESPACE((short)4, "tableNamespace"),
+      NS((short)4, "ns"),
       TBL_NSPC_PERM((short)5, "tblNspcPerm");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -19594,8 +19594,8 @@ import org.slf4j.LoggerFactory;
             return CREDENTIALS;
           case 3: // PRINCIPAL
             return PRINCIPAL;
-          case 4: // TABLE_NAMESPACE
-            return TABLE_NAMESPACE;
+          case 4: // NS
+            return NS;
           case 5: // TBL_NSPC_PERM
             return TBL_NSPC_PERM;
           default:
@@ -19649,29 +19649,29 @@ import org.slf4j.LoggerFactory;
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.security.thrift.TCredentials.class)));
       tmpMap.put(_Fields.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.TABLE_NAMESPACE, new org.apache.thrift.meta_data.FieldMetaData("tableNamespace", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+      tmpMap.put(_Fields.NS, new org.apache.thrift.meta_data.FieldMetaData("ns", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
       tmpMap.put(_Fields.TBL_NSPC_PERM, new org.apache.thrift.meta_data.FieldMetaData("tblNspcPerm", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BYTE)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(hasTableNamespacePermission_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(hasNamespacePermission_args.class, metaDataMap);
     }
 
-    public hasTableNamespacePermission_args() {
+    public hasNamespacePermission_args() {
     }
 
-    public hasTableNamespacePermission_args(
+    public hasNamespacePermission_args(
       org.apache.accumulo.trace.thrift.TInfo tinfo,
       org.apache.accumulo.core.security.thrift.TCredentials credentials,
       String principal,
-      String tableNamespace,
+      String ns,
       byte tblNspcPerm)
     {
       this();
       this.tinfo = tinfo;
       this.credentials = credentials;
       this.principal = principal;
-      this.tableNamespace = tableNamespace;
+      this.ns = ns;
       this.tblNspcPerm = tblNspcPerm;
       setTblNspcPermIsSet(true);
     }
@@ -19679,7 +19679,7 @@ import org.slf4j.LoggerFactory;
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public hasTableNamespacePermission_args(hasTableNamespacePermission_args other) {
+    public hasNamespacePermission_args(hasNamespacePermission_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
         this.tinfo = new org.apache.accumulo.trace.thrift.TInfo(other.tinfo);
@@ -19690,14 +19690,14 @@ import org.slf4j.LoggerFactory;
       if (other.isSetPrincipal()) {
         this.principal = other.principal;
       }
-      if (other.isSetTableNamespace()) {
-        this.tableNamespace = other.tableNamespace;
+      if (other.isSetNs()) {
+        this.ns = other.ns;
       }
       this.tblNspcPerm = other.tblNspcPerm;
     }
 
-    public hasTableNamespacePermission_args deepCopy() {
-      return new hasTableNamespacePermission_args(this);
+    public hasNamespacePermission_args deepCopy() {
+      return new hasNamespacePermission_args(this);
     }
 
     @Override
@@ -19705,7 +19705,7 @@ import org.slf4j.LoggerFactory;
       this.tinfo = null;
       this.credentials = null;
       this.principal = null;
-      this.tableNamespace = null;
+      this.ns = null;
       setTblNspcPermIsSet(false);
       this.tblNspcPerm = 0;
     }
@@ -19714,7 +19714,7 @@ import org.slf4j.LoggerFactory;
       return this.tinfo;
     }
 
-    public hasTableNamespacePermission_args setTinfo(org.apache.accumulo.trace.thrift.TInfo tinfo) {
+    public hasNamespacePermission_args setTinfo(org.apache.accumulo.trace.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -19738,7 +19738,7 @@ import org.slf4j.LoggerFactory;
       return this.credentials;
     }
 
-    public hasTableNamespacePermission_args setCredentials(org.apache.accumulo.core.security.thrift.TCredentials credentials) {
+    public hasNamespacePermission_args setCredentials(org.apache.accumulo.core.security.thrift.TCredentials credentials) {
       this.credentials = credentials;
       return this;
     }
@@ -19762,7 +19762,7 @@ import org.slf4j.LoggerFactory;
       return this.principal;
     }
 
-    public hasTableNamespacePermission_args setPrincipal(String principal) {
+    public hasNamespacePermission_args setPrincipal(String principal) {
       this.principal = principal;
       return this;
     }
@@ -19782,27 +19782,27 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public String getTableNamespace() {
-      return this.tableNamespace;
+    public String getNs() {
+      return this.ns;
     }
 
-    public hasTableNamespacePermission_args setTableNamespace(String tableNamespace) {
-      this.tableNamespace = tableNamespace;
+    public hasNamespacePermission_args setNs(String ns) {
+      this.ns = ns;
       return this;
     }
 
-    public void unsetTableNamespace() {
-      this.tableNamespace = null;
+    public void unsetNs() {
+      this.ns = null;
     }
 
-    /** Returns true if field tableNamespace is set (has been assigned a value) and false otherwise */
-    public boolean isSetTableNamespace() {
-      return this.tableNamespace != null;
+    /** Returns true if field ns is set (has been assigned a value) and false otherwise */
+    public boolean isSetNs() {
+      return this.ns != null;
     }
 
-    public void setTableNamespaceIsSet(boolean value) {
+    public void setNsIsSet(boolean value) {
       if (!value) {
-        this.tableNamespace = null;
+        this.ns = null;
       }
     }
 
@@ -19810,7 +19810,7 @@ import org.slf4j.LoggerFactory;
       return this.tblNspcPerm;
     }
 
-    public hasTableNamespacePermission_args setTblNspcPerm(byte tblNspcPerm) {
+    public hasNamespacePermission_args setTblNspcPerm(byte tblNspcPerm) {
       this.tblNspcPerm = tblNspcPerm;
       setTblNspcPermIsSet(true);
       return this;
@@ -19855,11 +19855,11 @@ import org.slf4j.LoggerFactory;
         }
         break;
 
-      case TABLE_NAMESPACE:
+      case NS:
         if (value == null) {
-          unsetTableNamespace();
+          unsetNs();
         } else {
-          setTableNamespace((String)value);
+          setNs((String)value);
         }
         break;
 
@@ -19885,8 +19885,8 @@ import org.slf4j.LoggerFactory;
       case PRINCIPAL:
         return getPrincipal();
 
-      case TABLE_NAMESPACE:
-        return getTableNamespace();
+      case NS:
+        return getNs();
 
       case TBL_NSPC_PERM:
         return Byte.valueOf(getTblNspcPerm());
@@ -19908,8 +19908,8 @@ import org.slf4j.LoggerFactory;
         return isSetCredentials();
       case PRINCIPAL:
         return isSetPrincipal();
-      case TABLE_NAMESPACE:
-        return isSetTableNamespace();
+      case NS:
+        return isSetNs();
       case TBL_NSPC_PERM:
         return isSetTblNspcPerm();
       }
@@ -19920,12 +19920,12 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof hasTableNamespacePermission_args)
-        return this.equals((hasTableNamespacePermission_args)that);
+      if (that instanceof hasNamespacePermission_args)
+        return this.equals((hasNamespacePermission_args)that);
       return false;
     }
 
-    public boolean equals(hasTableNamespacePermission_args that) {
+    public boolean equals(hasNamespacePermission_args that) {
       if (that == null)
         return false;
 
@@ -19956,12 +19956,12 @@ import org.slf4j.LoggerFactory;
           return false;
       }
 
-      boolean this_present_tableNamespace = true && this.isSetTableNamespace();
-      boolean that_present_tableNamespace = true && that.isSetTableNamespace();
-      if (this_present_tableNamespace || that_present_tableNamespace) {
-        if (!(this_present_tableNamespace && that_present_tableNamespace))
+      boolean this_present_ns = true && this.isSetNs();
+      boolean that_present_ns = true && that.isSetNs();
+      if (this_present_ns || that_present_ns) {
+        if (!(this_present_ns && that_present_ns))
           return false;
-        if (!this.tableNamespace.equals(that.tableNamespace))
+        if (!this.ns.equals(that.ns))
           return false;
       }
 
@@ -19982,13 +19982,13 @@ import org.slf4j.LoggerFactory;
       return 0;
     }
 
-    public int compareTo(hasTableNamespacePermission_args other) {
+    public int compareTo(hasNamespacePermission_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
-      hasTableNamespacePermission_args typedOther = (hasTableNamespacePermission_args)other;
+      hasNamespacePermission_args typedOther = (hasNamespacePermission_args)other;
 
       lastComparison = Boolean.valueOf(isSetTinfo()).compareTo(typedOther.isSetTinfo());
       if (lastComparison != 0) {
@@ -20020,12 +20020,12 @@ import org.slf4j.LoggerFactory;
           return lastComparison;
         }
       }
-      lastComparison = Boolean.valueOf(isSetTableNamespace()).compareTo(typedOther.isSetTableNamespace());
+      lastComparison = Boolean.valueOf(isSetNs()).compareTo(typedOther.isSetNs());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetTableNamespace()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableNamespace, typedOther.tableNamespace);
+      if (isSetNs()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ns, typedOther.ns);
         if (lastComparison != 0) {
           return lastComparison;
         }
@@ -20057,7 +20057,7 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("hasTableNamespacePermission_args(");
+      StringBuilder sb = new StringBuilder("hasNamespacePermission_args(");
       boolean first = true;
 
       sb.append("tinfo:");
@@ -20084,11 +20084,11 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
       if (!first) sb.append(", ");
-      sb.append("tableNamespace:");
-      if (this.tableNamespace == null) {
+      sb.append("ns:");
+      if (this.ns == null) {
         sb.append("null");
       } else {
-        sb.append(this.tableNamespace);
+        sb.append(this.ns);
       }
       first = false;
       if (!first) sb.append(", ");
@@ -20128,15 +20128,15 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    private static class hasTableNamespacePermission_argsStandardSchemeFactory implements SchemeFactory {
-      public hasTableNamespacePermission_argsStandardScheme getScheme() {
-        return new hasTableNamespacePermission_argsStandardScheme();
+    private static class hasNamespacePermission_argsStandardSchemeFactory implements SchemeFactory {
+      public hasNamespacePermission_argsStandardScheme getScheme() {
+        return new hasNamespacePermission_argsStandardScheme();
       }
     }
 
-    private static class hasTableNamespacePermission_argsStandardScheme extends StandardScheme<hasTableNamespacePermission_args> {
+    private static class hasNamespacePermission_argsStandardScheme extends StandardScheme<hasNamespacePermission_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, hasTableNamespacePermission_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, hasNamespacePermission_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -20172,10 +20172,10 @@ import org.slf4j.LoggerFactory;
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
-            case 4: // TABLE_NAMESPACE
+            case 4: // NS
               if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.tableNamespace = iprot.readString();
-                struct.setTableNamespaceIsSet(true);
+                struct.ns = iprot.readString();
+                struct.setNsIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
@@ -20199,7 +20199,7 @@ import org.slf4j.LoggerFactory;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, hasTableNamespacePermission_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, hasNamespacePermission_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -20218,9 +20218,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeString(struct.principal);
           oprot.writeFieldEnd();
         }
-        if (struct.tableNamespace != null) {
-          oprot.writeFieldBegin(TABLE_NAMESPACE_FIELD_DESC);
-          oprot.writeString(struct.tableNamespace);
+        if (struct.ns != null) {
+          oprot.writeFieldBegin(NS_FIELD_DESC);
+          oprot.writeString(struct.ns);
           oprot.writeFieldEnd();
         }
         oprot.writeFieldBegin(TBL_NSPC_PERM_FIELD_DESC);
@@ -20232,16 +20232,16 @@ import org.slf4j.LoggerFactory;
 
     }
 
-    private static class hasTableNamespacePermission_argsTupleSchemeFactory implements SchemeFactory {
-      public hasTableNamespacePermission_argsTupleScheme getScheme() {
-        return new hasTableNamespacePermission_argsTupleScheme();
+    private static class hasNamespacePermission_argsTupleSchemeFactory implements SchemeFactory {
+      public hasNamespacePermission_argsTupleScheme getScheme() {
+        return new hasNamespacePermission_argsTupleScheme();
       }
     }
 
-    private static class hasTableNamespacePermission_argsTupleScheme extends TupleScheme<hasTableNamespacePermission_args> {
+    private static class hasNamespacePermission_argsTupleScheme extends TupleScheme<hasNamespacePermission_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, hasTableNamespacePermission_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, hasNamespacePermission_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetTinfo()) {
@@ -20253,7 +20253,7 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPrincipal()) {
           optionals.set(2);
         }
-        if (struct.isSetTableNamespace()) {
+        if (struct.isSetNs()) {
           optionals.set(3);
         }
         if (struct.isSetTblNspcPerm()) {
@@ -20269,8 +20269,8 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPrincipal()) {
           oprot.writeString(struct.principal);
         }
-        if (struct.isSetTableNamespace()) {
-          oprot.writeString(struct.tableNamespace);
+        if (struct.isSetNs()) {
+          oprot.writeString(struct.ns);
         }
         if (struct.isSetTblNspcPerm()) {
           oprot.writeByte(struct.tblNspcPerm);
@@ -20278,7 +20278,7 @@ import org.slf4j.LoggerFactory;
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, hasTableNamespacePermission_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, hasNamespacePermission_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
@@ -20296,8 +20296,8 @@ import org.slf4j.LoggerFactory;
           struct.setPrincipalIsSet(true);
         }
         if (incoming.get(3)) {
-          struct.tableNamespace = iprot.readString();
-          struct.setTableNamespaceIsSet(true);
+          struct.ns = iprot.readString();
+          struct.setNsIsSet(true);
         }
         if (incoming.get(4)) {
           struct.tblNspcPerm = iprot.readByte();
@@ -20308,8 +20308,8 @@ import org.slf4j.LoggerFactory;
 
   }
 
-  public static class hasTableNamespacePermission_result implements org.apache.thrift.TBase<hasTableNamespacePermission_result, hasTableNamespacePermission_result._Fields>, java.io.Serializable, Cloneable   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("hasTableNamespacePermission_result");
+  public static class hasNamespacePermission_result implements org.apache.thrift.TBase<hasNamespacePermission_result, hasNamespacePermission_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("hasNamespacePermission_result");
 
     private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0);
     private static final org.apache.thrift.protocol.TField SEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sec", org.apache.thrift.protocol.TType.STRUCT, (short)1);
@@ -20317,8 +20317,8 @@ import org.slf4j.LoggerFactory;
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new hasTableNamespacePermission_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new hasTableNamespacePermission_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new hasNamespacePermission_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new hasNamespacePermission_resultTupleSchemeFactory());
     }
 
     public boolean success; // required
@@ -20402,13 +20402,13 @@ import org.slf4j.LoggerFactory;
       tmpMap.put(_Fields.TOPE, new org.apache.thrift.meta_data.FieldMetaData("tope", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(hasTableNamespacePermission_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(hasNamespacePermission_result.class, metaDataMap);
     }
 
-    public hasTableNamespacePermission_result() {
+    public hasNamespacePermission_result() {
     }
 
-    public hasTableNamespacePermission_result(
+    public hasNamespacePermission_result(
       boolean success,
       ThriftSecurityException sec,
       ThriftTableOperationException tope)
@@ -20423,7 +20423,7 @@ import org.slf4j.LoggerFactory;
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public hasTableNamespacePermission_result(hasTableNamespacePermission_result other) {
+    public hasNamespacePermission_result(hasNamespacePermission_result other) {
       __isset_bitfield = other.__isset_bitfield;
       this.success = other.success;
       if (other.isSetSec()) {
@@ -20434,8 +20434,8 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public hasTableNamespacePermission_result deepCopy() {
-      return new hasTableNamespacePermission_result(this);
+    public hasNamespacePermission_result deepCopy() {
+      return new hasNamespacePermission_result(this);
     }
 
     @Override
@@ -20450,7 +20450,7 @@ import org.slf4j.LoggerFactory;
       return this.success;
     }
 
-    public hasTableNamespacePermission_result setSuccess(boolean success) {
+    public hasNamespacePermission_result setSuccess(boolean success) {
       this.success = success;
       setSuccessIsSet(true);
       return this;
@@ -20473,7 +20473,7 @@ import org.slf4j.LoggerFactory;
       return this.sec;
     }
 
-    public hasTableNamespacePermission_result setSec(ThriftSecurityException sec) {
+    public hasNamespacePermission_result setSec(ThriftSecurityException sec) {
       this.sec = sec;
       return this;
     }
@@ -20497,7 +20497,7 @@ import org.slf4j.LoggerFactory;
       return this.tope;
     }
 
-    public hasTableNamespacePermission_result setTope(ThriftTableOperationException tope) {
+    public hasNamespacePermission_result setTope(ThriftTableOperationException tope) {
       this.tope = tope;
       return this;
     }
@@ -20582,12 +20582,12 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof hasTableNamespacePermission_result)
-        return this.equals((hasTableNamespacePermission_result)that);
+      if (that instanceof hasNamespacePermission_result)
+        return this.equals((hasNamespacePermission_result)that);
       return false;
     }
 
-    public boolean equals(hasTableNamespacePermission_result that) {
+    public boolean equals(hasNamespacePermission_result that) {
       if (that == null)
         return false;
 
@@ -20626,13 +20626,13 @@ import org.slf4j.LoggerFactory;
       return 0;
     }
 
-    public int compareTo(hasTableNamespacePermission_result other) {
+    public int compareTo(hasNamespacePermission_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
-      hasTableNamespacePermission_result typedOther = (hasTableNamespacePermission_result)other;
+      hasNamespacePermission_result typedOther = (hasNamespacePermission_result)other;
 
       lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess());
       if (lastComparison != 0) {
@@ -20681,7 +20681,7 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("hasTableNamespacePermission_result(");
+      StringBuilder sb = new StringBuilder("hasNamespacePermission_result(");
       boolean first = true;
 
       sb.append("success:");
@@ -20730,15 +20730,15 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    private static class hasTableNamespacePermission_resultStandardSchemeFactory implements SchemeFactory {
-      public hasTableNamespacePermission_resultStandardScheme getScheme() {
-        return new hasTableNamespacePermission_resultStandardScheme();
+    private static class hasNamespacePermission_resultStandardSchemeFactory implements SchemeFactory {
+      public hasNamespacePermission_resultStandardScheme getScheme() {
+        return new hasNamespacePermission_resultStandardScheme();
       }
     }
 
-    private static class hasTableNamespacePermission_resultStandardScheme extends StandardScheme<hasTableNamespacePermission_result> {
+    private static class hasNamespacePermission_resultStandardScheme extends StandardScheme<hasNamespacePermission_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, hasTableNamespacePermission_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, hasNamespacePermission_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -20785,7 +20785,7 @@ import org.slf4j.LoggerFactory;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, hasTableNamespacePermission_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, hasNamespacePermission_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -20810,16 +20810,16 @@ import org.slf4j.LoggerFactory;
 
     }
 
-    private static class hasTableNamespacePermission_resultTupleSchemeFactory implements SchemeFactory {
-      public hasTableNamespacePermission_resultTupleScheme getScheme() {
-        return new hasTableNamespacePermission_resultTupleScheme();
+    private static class hasNamespacePermission_resultTupleSchemeFactory implements SchemeFactory {
+      public hasNamespacePermission_resultTupleScheme getScheme() {
+        return new hasNamespacePermission_resultTupleScheme();
       }
     }
 
-    private static class hasTableNamespacePermission_resultTupleScheme extends TupleScheme<hasTableNamespacePermission_result> {
+    private static class hasNamespacePermission_resultTupleScheme extends TupleScheme<hasNamespacePermission_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, hasTableNamespacePermission_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, hasNamespacePermission_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetSuccess()) {
@@ -20844,7 +20844,7 @@ import org.slf4j.LoggerFactory;
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, hasTableNamespacePermission_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, hasNamespacePermission_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
@@ -25342,25 +25342,25 @@ import org.slf4j.LoggerFactory;
 
   }
 
-  public static class grantTableNamespacePermission_args implements org.apache.thrift.TBase<grantTableNamespacePermission_args, grantTableNamespacePermission_args._Fields>, java.io.Serializable, Cloneable   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("grantTableNamespacePermission_args");
+  public static class grantNamespacePermission_args implements org.apache.thrift.TBase<grantNamespacePermission_args, grantNamespacePermission_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("grantNamespacePermission_args");
 
     private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
     private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
     private static final org.apache.thrift.protocol.TField PRINCIPAL_FIELD_DESC = new org.apache.thrift.protocol.TField("principal", org.apache.thrift.protocol.TType.STRING, (short)3);
-    private static final org.apache.thrift.protocol.TField TABLE_NAMESPACE_FIELD_DESC = new org.apache.thrift.protocol.TField("tableNamespace", org.apache.thrift.protocol.TType.STRING, (short)4);
+    private static final org.apache.thrift.protocol.TField NS_FIELD_DESC = new org.apache.thrift.protocol.TField("ns", org.apache.thrift.protocol.TType.STRING, (short)4);
     private static final org.apache.thrift.protocol.TField PERMISSION_FIELD_DESC = new org.apache.thrift.protocol.TField("permission", org.apache.t

<TRUNCATED>

[02/50] [abbrv] ACCUMULO-802 updated more shell commands to include the tableNamespaces option

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/5acd6a48/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/ClientService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/ClientService.java b/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/ClientService.java
index 9eac584..b857c27 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/ClientService.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/ClientService.java
@@ -102,6 +102,8 @@ import org.slf4j.LoggerFactory;
 
     public boolean checkTableClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String tableId, String className, String interfaceMatch) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
 
+    public boolean checkTableNamespaceClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String namespaceId, String className, String interfaceMatch) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+
   }
 
   public interface AsyncIface {
@@ -158,6 +160,8 @@ import org.slf4j.LoggerFactory;
 
     public void checkTableClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String tableId, String className, String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.checkTableClass_call> resultHandler) throws org.apache.thrift.TException;
 
+    public void checkTableNamespaceClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String namespaceId, String className, String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.checkTableNamespaceClass_call> resultHandler) throws org.apache.thrift.TException;
+
   }
 
   public static class Client extends org.apache.thrift.TServiceClient implements Iface {
@@ -885,6 +889,39 @@ import org.slf4j.LoggerFactory;
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "checkTableClass failed: unknown result");
     }
 
+    public boolean checkTableNamespaceClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String namespaceId, String className, String interfaceMatch) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    {
+      send_checkTableNamespaceClass(tinfo, credentials, namespaceId, className, interfaceMatch);
+      return recv_checkTableNamespaceClass();
+    }
+
+    public void send_checkTableNamespaceClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String namespaceId, String className, String interfaceMatch) throws org.apache.thrift.TException
+    {
+      checkTableNamespaceClass_args args = new checkTableNamespaceClass_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setNamespaceId(namespaceId);
+      args.setClassName(className);
+      args.setInterfaceMatch(interfaceMatch);
+      sendBase("checkTableNamespaceClass", args);
+    }
+
+    public boolean recv_checkTableNamespaceClass() throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    {
+      checkTableNamespaceClass_result result = new checkTableNamespaceClass_result();
+      receiveBase(result, "checkTableNamespaceClass");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.sec != null) {
+        throw result.sec;
+      }
+      if (result.tope != null) {
+        throw result.tope;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "checkTableNamespaceClass failed: unknown result");
+    }
+
   }
   public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
     public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
@@ -1903,6 +1940,50 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public void checkTableNamespaceClass(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String namespaceId, String className, String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<checkTableNamespaceClass_call> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      checkTableNamespaceClass_call method_call = new checkTableNamespaceClass_call(tinfo, credentials, namespaceId, className, interfaceMatch, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class checkTableNamespaceClass_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private org.apache.accumulo.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
+      private String namespaceId;
+      private String className;
+      private String interfaceMatch;
+      public checkTableNamespaceClass_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String namespaceId, String className, String interfaceMatch, org.apache.thrift.async.AsyncMethodCallback<checkTableNamespaceClass_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.namespaceId = namespaceId;
+        this.className = className;
+        this.interfaceMatch = interfaceMatch;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("checkTableNamespaceClass", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        checkTableNamespaceClass_args args = new checkTableNamespaceClass_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setNamespaceId(namespaceId);
+        args.setClassName(className);
+        args.setInterfaceMatch(interfaceMatch);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public boolean getResult() throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_checkTableNamespaceClass();
+      }
+    }
+
   }
 
   public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
@@ -1942,6 +2023,7 @@ import org.slf4j.LoggerFactory;
       processMap.put("getTableNamespaceConfiguration", new getTableNamespaceConfiguration());
       processMap.put("checkClass", new checkClass());
       processMap.put("checkTableClass", new checkTableClass());
+      processMap.put("checkTableNamespaceClass", new checkTableNamespaceClass());
       return processMap;
     }
 
@@ -2564,6 +2646,33 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public static class checkTableNamespaceClass<I extends Iface> extends org.apache.thrift.ProcessFunction<I, checkTableNamespaceClass_args> {
+      public checkTableNamespaceClass() {
+        super("checkTableNamespaceClass");
+      }
+
+      public checkTableNamespaceClass_args getEmptyArgsInstance() {
+        return new checkTableNamespaceClass_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public checkTableNamespaceClass_result getResult(I iface, checkTableNamespaceClass_args args) throws org.apache.thrift.TException {
+        checkTableNamespaceClass_result result = new checkTableNamespaceClass_result();
+        try {
+          result.success = iface.checkTableNamespaceClass(args.tinfo, args.credentials, args.namespaceId, args.className, args.interfaceMatch);
+          result.setSuccessIsSet(true);
+        } catch (ThriftSecurityException sec) {
+          result.sec = sec;
+        } catch (ThriftTableOperationException tope) {
+          result.tope = tope;
+        }
+        return result;
+      }
+    }
+
   }
 
   public static class getRootTabletLocation_args implements org.apache.thrift.TBase<getRootTabletLocation_args, getRootTabletLocation_args._Fields>, java.io.Serializable, Cloneable   {
@@ -29097,4 +29206,1326 @@ import org.slf4j.LoggerFactory;
 
   }
 
+  public static class checkTableNamespaceClass_args implements org.apache.thrift.TBase<checkTableNamespaceClass_args, checkTableNamespaceClass_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("checkTableNamespaceClass_args");
+
+    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+    private static final org.apache.thrift.protocol.TField NAMESPACE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("namespaceId", org.apache.thrift.protocol.TType.STRING, (short)3);
+    private static final org.apache.thrift.protocol.TField CLASS_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("className", org.apache.thrift.protocol.TType.STRING, (short)4);
+    private static final org.apache.thrift.protocol.TField INTERFACE_MATCH_FIELD_DESC = new org.apache.thrift.protocol.TField("interfaceMatch", org.apache.thrift.protocol.TType.STRING, (short)5);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new checkTableNamespaceClass_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new checkTableNamespaceClass_argsTupleSchemeFactory());
+    }
+
+    public org.apache.accumulo.trace.thrift.TInfo tinfo; // required
+    public org.apache.accumulo.core.security.thrift.TCredentials credentials; // required
+    public String namespaceId; // required
+    public String className; // required
+    public String interfaceMatch; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      TINFO((short)1, "tinfo"),
+      CREDENTIALS((short)2, "credentials"),
+      NAMESPACE_ID((short)3, "namespaceId"),
+      CLASS_NAME((short)4, "className"),
+      INTERFACE_MATCH((short)5, "interfaceMatch");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // TINFO
+            return TINFO;
+          case 2: // CREDENTIALS
+            return CREDENTIALS;
+          case 3: // NAMESPACE_ID
+            return NAMESPACE_ID;
+          case 4: // CLASS_NAME
+            return CLASS_NAME;
+          case 5: // INTERFACE_MATCH
+            return INTERFACE_MATCH;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.trace.thrift.TInfo.class)));
+      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.security.thrift.TCredentials.class)));
+      tmpMap.put(_Fields.NAMESPACE_ID, new org.apache.thrift.meta_data.FieldMetaData("namespaceId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.CLASS_NAME, new org.apache.thrift.meta_data.FieldMetaData("className", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.INTERFACE_MATCH, new org.apache.thrift.meta_data.FieldMetaData("interfaceMatch", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(checkTableNamespaceClass_args.class, metaDataMap);
+    }
+
+    public checkTableNamespaceClass_args() {
+    }
+
+    public checkTableNamespaceClass_args(
+      org.apache.accumulo.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.security.thrift.TCredentials credentials,
+      String namespaceId,
+      String className,
+      String interfaceMatch)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.namespaceId = namespaceId;
+      this.className = className;
+      this.interfaceMatch = interfaceMatch;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public checkTableNamespaceClass_args(checkTableNamespaceClass_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.trace.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.security.thrift.TCredentials(other.credentials);
+      }
+      if (other.isSetNamespaceId()) {
+        this.namespaceId = other.namespaceId;
+      }
+      if (other.isSetClassName()) {
+        this.className = other.className;
+      }
+      if (other.isSetInterfaceMatch()) {
+        this.interfaceMatch = other.interfaceMatch;
+      }
+    }
+
+    public checkTableNamespaceClass_args deepCopy() {
+      return new checkTableNamespaceClass_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.namespaceId = null;
+      this.className = null;
+      this.interfaceMatch = null;
+    }
+
+    public org.apache.accumulo.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public checkTableNamespaceClass_args setTinfo(org.apache.accumulo.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    public org.apache.accumulo.core.security.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public checkTableNamespaceClass_args setCredentials(org.apache.accumulo.core.security.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    public String getNamespaceId() {
+      return this.namespaceId;
+    }
+
+    public checkTableNamespaceClass_args setNamespaceId(String namespaceId) {
+      this.namespaceId = namespaceId;
+      return this;
+    }
+
+    public void unsetNamespaceId() {
+      this.namespaceId = null;
+    }
+
+    /** Returns true if field namespaceId is set (has been assigned a value) and false otherwise */
+    public boolean isSetNamespaceId() {
+      return this.namespaceId != null;
+    }
+
+    public void setNamespaceIdIsSet(boolean value) {
+      if (!value) {
+        this.namespaceId = null;
+      }
+    }
+
+    public String getClassName() {
+      return this.className;
+    }
+
+    public checkTableNamespaceClass_args setClassName(String className) {
+      this.className = className;
+      return this;
+    }
+
+    public void unsetClassName() {
+      this.className = null;
+    }
+
+    /** Returns true if field className is set (has been assigned a value) and false otherwise */
+    public boolean isSetClassName() {
+      return this.className != null;
+    }
+
+    public void setClassNameIsSet(boolean value) {
+      if (!value) {
+        this.className = null;
+      }
+    }
+
+    public String getInterfaceMatch() {
+      return this.interfaceMatch;
+    }
+
+    public checkTableNamespaceClass_args setInterfaceMatch(String interfaceMatch) {
+      this.interfaceMatch = interfaceMatch;
+      return this;
+    }
+
+    public void unsetInterfaceMatch() {
+      this.interfaceMatch = null;
+    }
+
+    /** Returns true if field interfaceMatch is set (has been assigned a value) and false otherwise */
+    public boolean isSetInterfaceMatch() {
+      return this.interfaceMatch != null;
+    }
+
+    public void setInterfaceMatchIsSet(boolean value) {
+      if (!value) {
+        this.interfaceMatch = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.security.thrift.TCredentials)value);
+        }
+        break;
+
+      case NAMESPACE_ID:
+        if (value == null) {
+          unsetNamespaceId();
+        } else {
+          setNamespaceId((String)value);
+        }
+        break;
+
+      case CLASS_NAME:
+        if (value == null) {
+          unsetClassName();
+        } else {
+          setClassName((String)value);
+        }
+        break;
+
+      case INTERFACE_MATCH:
+        if (value == null) {
+          unsetInterfaceMatch();
+        } else {
+          setInterfaceMatch((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      case NAMESPACE_ID:
+        return getNamespaceId();
+
+      case CLASS_NAME:
+        return getClassName();
+
+      case INTERFACE_MATCH:
+        return getInterfaceMatch();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case NAMESPACE_ID:
+        return isSetNamespaceId();
+      case CLASS_NAME:
+        return isSetClassName();
+      case INTERFACE_MATCH:
+        return isSetInterfaceMatch();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof checkTableNamespaceClass_args)
+        return this.equals((checkTableNamespaceClass_args)that);
+      return false;
+    }
+
+    public boolean equals(checkTableNamespaceClass_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      boolean this_present_namespaceId = true && this.isSetNamespaceId();
+      boolean that_present_namespaceId = true && that.isSetNamespaceId();
+      if (this_present_namespaceId || that_present_namespaceId) {
+        if (!(this_present_namespaceId && that_present_namespaceId))
+          return false;
+        if (!this.namespaceId.equals(that.namespaceId))
+          return false;
+      }
+
+      boolean this_present_className = true && this.isSetClassName();
+      boolean that_present_className = true && that.isSetClassName();
+      if (this_present_className || that_present_className) {
+        if (!(this_present_className && that_present_className))
+          return false;
+        if (!this.className.equals(that.className))
+          return false;
+      }
+
+      boolean this_present_interfaceMatch = true && this.isSetInterfaceMatch();
+      boolean that_present_interfaceMatch = true && that.isSetInterfaceMatch();
+      if (this_present_interfaceMatch || that_present_interfaceMatch) {
+        if (!(this_present_interfaceMatch && that_present_interfaceMatch))
+          return false;
+        if (!this.interfaceMatch.equals(that.interfaceMatch))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return 0;
+    }
+
+    public int compareTo(checkTableNamespaceClass_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      checkTableNamespaceClass_args typedOther = (checkTableNamespaceClass_args)other;
+
+      lastComparison = Boolean.valueOf(isSetTinfo()).compareTo(typedOther.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, typedOther.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetCredentials()).compareTo(typedOther.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, typedOther.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetNamespaceId()).compareTo(typedOther.isSetNamespaceId());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetNamespaceId()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.namespaceId, typedOther.namespaceId);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetClassName()).compareTo(typedOther.isSetClassName());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetClassName()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.className, typedOther.className);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetInterfaceMatch()).compareTo(typedOther.isSetInterfaceMatch());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetInterfaceMatch()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.interfaceMatch, typedOther.interfaceMatch);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("checkTableNamespaceClass_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("namespaceId:");
+      if (this.namespaceId == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.namespaceId);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("className:");
+      if (this.className == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.className);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("interfaceMatch:");
+      if (this.interfaceMatch == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.interfaceMatch);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class checkTableNamespaceClass_argsStandardSchemeFactory implements SchemeFactory {
+      public checkTableNamespaceClass_argsStandardScheme getScheme() {
+        return new checkTableNamespaceClass_argsStandardScheme();
+      }
+    }
+
+    private static class checkTableNamespaceClass_argsStandardScheme extends StandardScheme<checkTableNamespaceClass_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, checkTableNamespaceClass_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.trace.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // CREDENTIALS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.credentials = new org.apache.accumulo.core.security.thrift.TCredentials();
+                struct.credentials.read(iprot);
+                struct.setCredentialsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // NAMESPACE_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.namespaceId = iprot.readString();
+                struct.setNamespaceIdIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 4: // CLASS_NAME
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.className = iprot.readString();
+                struct.setClassNameIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 5: // INTERFACE_MATCH
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.interfaceMatch = iprot.readString();
+                struct.setInterfaceMatchIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, checkTableNamespaceClass_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.namespaceId != null) {
+          oprot.writeFieldBegin(NAMESPACE_ID_FIELD_DESC);
+          oprot.writeString(struct.namespaceId);
+          oprot.writeFieldEnd();
+        }
+        if (struct.className != null) {
+          oprot.writeFieldBegin(CLASS_NAME_FIELD_DESC);
+          oprot.writeString(struct.className);
+          oprot.writeFieldEnd();
+        }
+        if (struct.interfaceMatch != null) {
+          oprot.writeFieldBegin(INTERFACE_MATCH_FIELD_DESC);
+          oprot.writeString(struct.interfaceMatch);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class checkTableNamespaceClass_argsTupleSchemeFactory implements SchemeFactory {
+      public checkTableNamespaceClass_argsTupleScheme getScheme() {
+        return new checkTableNamespaceClass_argsTupleScheme();
+      }
+    }
+
+    private static class checkTableNamespaceClass_argsTupleScheme extends TupleScheme<checkTableNamespaceClass_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, checkTableNamespaceClass_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        if (struct.isSetNamespaceId()) {
+          optionals.set(2);
+        }
+        if (struct.isSetClassName()) {
+          optionals.set(3);
+        }
+        if (struct.isSetInterfaceMatch()) {
+          optionals.set(4);
+        }
+        oprot.writeBitSet(optionals, 5);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetNamespaceId()) {
+          oprot.writeString(struct.namespaceId);
+        }
+        if (struct.isSetClassName()) {
+          oprot.writeString(struct.className);
+        }
+        if (struct.isSetInterfaceMatch()) {
+          oprot.writeString(struct.interfaceMatch);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, checkTableNamespaceClass_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(5);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.trace.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.credentials = new org.apache.accumulo.core.security.thrift.TCredentials();
+          struct.credentials.read(iprot);
+          struct.setCredentialsIsSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.namespaceId = iprot.readString();
+          struct.setNamespaceIdIsSet(true);
+        }
+        if (incoming.get(3)) {
+          struct.className = iprot.readString();
+          struct.setClassNameIsSet(true);
+        }
+        if (incoming.get(4)) {
+          struct.interfaceMatch = iprot.readString();
+          struct.setInterfaceMatchIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class checkTableNamespaceClass_result implements org.apache.thrift.TBase<checkTableNamespaceClass_result, checkTableNamespaceClass_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("checkTableNamespaceClass_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0);
+    private static final org.apache.thrift.protocol.TField SEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sec", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField TOPE_FIELD_DESC = new org.apache.thrift.protocol.TField("tope", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new checkTableNamespaceClass_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new checkTableNamespaceClass_resultTupleSchemeFactory());
+    }
+
+    public boolean success; // required
+    public ThriftSecurityException sec; // required
+    public ThriftTableOperationException tope; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success"),
+      SEC((short)1, "sec"),
+      TOPE((short)2, "tope");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          case 1: // SEC
+            return SEC;
+          case 2: // TOPE
+            return TOPE;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    private static final int __SUCCESS_ISSET_ID = 0;
+    private byte __isset_bitfield = 0;
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+      tmpMap.put(_Fields.SEC, new org.apache.thrift.meta_data.FieldMetaData("sec", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      tmpMap.put(_Fields.TOPE, new org.apache.thrift.meta_data.FieldMetaData("tope", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(checkTableNamespaceClass_result.class, metaDataMap);
+    }
+
+    public checkTableNamespaceClass_result() {
+    }
+
+    public checkTableNamespaceClass_result(
+      boolean success,
+      ThriftSecurityException sec,
+      ThriftTableOperationException tope)
+    {
+      this();
+      this.success = success;
+      setSuccessIsSet(true);
+      this.sec = sec;
+      this.tope = tope;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public checkTableNamespaceClass_result(checkTableNamespaceClass_result other) {
+      __isset_bitfield = other.__isset_bitfield;
+      this.success = other.success;
+      if (other.isSetSec()) {
+        this.sec = new ThriftSecurityException(other.sec);
+      }
+      if (other.isSetTope()) {
+        this.tope = new ThriftTableOperationException(other.tope);
+      }
+    }
+
+    public checkTableNamespaceClass_result deepCopy() {
+      return new checkTableNamespaceClass_result(this);
+    }
+
+    @Override
+    public void clear() {
+      setSuccessIsSet(false);
+      this.success = false;
+      this.sec = null;
+      this.tope = null;
+    }
+
+    public boolean isSuccess() {
+      return this.success;
+    }
+
+    public checkTableNamespaceClass_result setSuccess(boolean success) {
+      this.success = success;
+      setSuccessIsSet(true);
+      return this;
+    }
+
+    public void unsetSuccess() {
+      __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value);
+    }
+
+    public ThriftSecurityException getSec() {
+      return this.sec;
+    }
+
+    public checkTableNamespaceClass_result setSec(ThriftSecurityException sec) {
+      this.sec = sec;
+      return this;
+    }
+
+    public void unsetSec() {
+      this.sec = null;
+    }
+
+    /** Returns true if field sec is set (has been assigned a value) and false otherwise */
+    public boolean isSetSec() {
+      return this.sec != null;
+    }
+
+    public void setSecIsSet(boolean value) {
+      if (!value) {
+        this.sec = null;
+      }
+    }
+
+    public ThriftTableOperationException getTope() {
+      return this.tope;
+    }
+
+    public checkTableNamespaceClass_result setTope(ThriftTableOperationException tope) {
+      this.tope = tope;
+      return this;
+    }
+
+    public void unsetTope() {
+      this.tope = null;
+    }
+
+    /** Returns true if field tope is set (has been assigned a value) and false otherwise */
+    public boolean isSetTope() {
+      return this.tope != null;
+    }
+
+    public void setTopeIsSet(boolean value) {
+      if (!value) {
+        this.tope = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((Boolean)value);
+        }
+        break;
+
+      case SEC:
+        if (value == null) {
+          unsetSec();
+        } else {
+          setSec((ThriftSecurityException)value);
+        }
+        break;
+
+      case TOPE:
+        if (value == null) {
+          unsetTope();
+        } else {
+          setTope((ThriftTableOperationException)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return Boolean.valueOf(isSuccess());
+
+      case SEC:
+        return getSec();
+
+      case TOPE:
+        return getTope();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      case SEC:
+        return isSetSec();
+      case TOPE:
+        return isSetTope();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof checkTableNamespaceClass_result)
+        return this.equals((checkTableNamespaceClass_result)that);
+      return false;
+    }
+
+    public boolean equals(checkTableNamespaceClass_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true;
+      boolean that_present_success = true;
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (this.success != that.success)
+          return false;
+      }
+
+      boolean this_present_sec = true && this.isSetSec();
+      boolean that_present_sec = true && that.isSetSec();
+      if (this_present_sec || that_present_sec) {
+        if (!(this_present_sec && that_present_sec))
+          return false;
+        if (!this.sec.equals(that.sec))
+          return false;
+      }
+
+      boolean this_present_tope = true && this.isSetTope();
+      boolean that_present_tope = true && that.isSetTope();
+      if (this_present_tope || that_present_tope) {
+        if (!(this_present_tope && that_present_tope))
+          return false;
+        if (!this.tope.equals(that.tope))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return 0;
+    }
+
+    public int compareTo(checkTableNamespaceClass_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      checkTableNamespaceClass_result typedOther = (checkTableNamespaceClass_result)other;
+
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetSec()).compareTo(typedOther.isSetSec());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSec()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sec, typedOther.sec);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetTope()).compareTo(typedOther.isSetTope());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTope()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tope, typedOther.tope);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("checkTableNamespaceClass_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      sb.append(this.success);
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("sec:");
+      if (this.sec == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.sec);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("tope:");
+      if (this.tope == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tope);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bitfield = 0;
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class checkTableNamespaceClass_resultStandardSchemeFactory implements SchemeFactory {
+      public checkTableNamespaceClass_resultStandardScheme getScheme() {
+        return new checkTableNamespaceClass_resultStandardScheme();
+      }
+    }
+
+    private static class checkTableNamespaceClass_resultStandardScheme extends StandardScheme<checkTableNamespaceClass_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, checkTableNamespaceClass_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+                struct.success = iprot.readBool();
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // SEC
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.sec = new ThriftSecurityException();
+                struct.sec.read(iprot);
+                struct.setSecIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // TOPE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tope = new ThriftTableOperationException();
+                struct.tope.read(iprot);
+                struct.setTopeIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, checkTableNamespaceClass_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.isSetSuccess()) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          oprot.writeBool(struct.success);
+          oprot.writeFieldEnd();
+        }
+        if (struct.sec != null) {
+          oprot.writeFieldBegin(SEC_FIELD_DESC);
+          struct.sec.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.tope != null) {
+          oprot.writeFieldBegin(TOPE_FIELD_DESC);
+          struct.tope.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class checkTableNamespaceClass_resultTupleSchemeFactory implements SchemeFactory {
+      public checkTableNamespaceClass_resultTupleScheme getScheme() {
+        return new checkTableNamespaceClass_resultTupleScheme();
+      }
+    }
+
+    private static class checkTableNamespaceClass_resultTupleScheme extends TupleScheme<checkTableNamespaceClass_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, checkTableNamespaceClass_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        if (struct.isSetSec()) {
+          optionals.set(1);
+        }
+        if (struct.isSetTope()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.isSetSuccess()) {
+          oprot.writeBool(struct.success);
+        }
+        if (struct.isSetSec()) {
+          struct.sec.write(oprot);
+        }
+        if (struct.isSetTope()) {
+          struct.tope.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, checkTableNamespaceClass_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(3);
+        if (incoming.get(0)) {
+          struct.success = iprot.readBool();
+          struct.setSuccessIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.sec = new ThriftSecurityException();
+          struct.sec.read(iprot);
+          struct.setSecIsSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.tope = new ThriftTableOperationException();
+          struct.tope.read(iprot);
+          struct.setTopeIsSet(true);
+        }
+      }
+    }
+
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5acd6a48/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java
index 94fdb4b..28f2edc 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java
@@ -36,9 +36,11 @@ import org.apache.accumulo.core.client.admin.DiskUsage;
 import org.apache.accumulo.core.client.admin.TableNamespaceOperationsHelper;
 import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.core.security.Credentials;
+import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
 import org.apache.commons.lang.NotImplementedException;
 
+
+
 public class MockTableNamespaceOperations extends TableNamespaceOperationsHelper {
 
   final private MockAccumulo acu;
@@ -184,4 +186,17 @@ public class MockTableNamespaceOperations extends TableNamespaceOperationsHelper
     // TODO Implement clone in Mock
     throw new NotImplementedException();
   }
+  
+  @Override
+  public boolean testClassLoad(String namespace, String className, String asTypeName) throws AccumuloException, AccumuloSecurityException,
+      TableNamespaceNotFoundException {
+    
+    try {
+      AccumuloVFSClassLoader.loadClass(className, Class.forName(asTypeName));
+    } catch (ClassNotFoundException e) {
+      e.printStackTrace();
+      return false;
+    }
+    return true;
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5acd6a48/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ConstraintCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ConstraintCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ConstraintCommand.java
index f72fa32..c91f29e 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ConstraintCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ConstraintCommand.java
@@ -19,7 +19,6 @@ package org.apache.accumulo.core.util.shell.commands;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.core.constraints.Constraint;
-import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
 import org.apache.accumulo.core.util.shell.ShellCommandException;
@@ -48,20 +47,23 @@ public class ConstraintCommand extends Command {
     } else {
       tableName = null;
     }
-    
+      
     int i;
-    
     switch (OptUtil.getAldOpt(cl)) {
       case ADD:
         for (String constraint : cl.getArgs()) {
-          if (!shellState.getConnector().tableOperations().testClassLoad(MetadataTable.NAME, constraint, Constraint.class.getName())) {
-            throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Servers are unable to load " + constraint + " as type "
-                + Constraint.class.getName());
-          }
           if (namespace != null) {
+            if (!shellState.getConnector().tableNamespaceOperations().testClassLoad(namespace, constraint, Constraint.class.getName())) {
+              throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Servers are unable to load " + constraint + " as type "
+                  + Constraint.class.getName());
+            }
             i = shellState.getConnector().tableNamespaceOperations().addConstraint(namespace, constraint);
             shellState.getReader().println("Added constraint " + constraint + " to table namespace " + namespace + " with number " + i);
-          } else if (tableName != null){
+          } else if (tableName != null && !tableName.isEmpty()){
+            if (!shellState.getConnector().tableOperations().testClassLoad(tableName, constraint, Constraint.class.getName())) {
+              throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Servers are unable to load " + constraint + " as type "
+                  + Constraint.class.getName());
+            }
             i = shellState.getConnector().tableOperations().addConstraint(tableName, constraint);
             shellState.getReader().println("Added constraint " + constraint + " to table " + tableName + " with number " + i);
           } else {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5acd6a48/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteIterCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteIterCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteIterCommand.java
index d064482..4f2f46f 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteIterCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteIterCommand.java
@@ -23,18 +23,31 @@ import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionGroup;
 import org.apache.commons.cli.Options;
 
 public class DeleteIterCommand extends Command {
   private Option allScopeOpt, mincScopeOpt, majcScopeOpt, scanScopeOpt, nameOpt;
   
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws Exception {
-    final String tableName = OptUtil.getTableOpt(cl, shellState);
+    
+    boolean tables = cl.hasOption(OptUtil.tableOpt().getOpt()) || !shellState.getTableName().isEmpty();
+    boolean namespaces = cl.hasOption(OptUtil.tableNamespaceOpt().getOpt());
     
     final String name = cl.getOptionValue(nameOpt.getOpt());
-    if (!shellState.getConnector().tableOperations().listIterators(tableName).containsKey(name)) {
-      Shell.log.warn("no iterators found that match your criteria");
-      return 0;
+    
+    if (namespaces) {
+      if (!shellState.getConnector().tableNamespaceOperations().listIterators(OptUtil.getTableNamespaceOpt(cl, shellState)).containsKey(name)) {
+        Shell.log.warn("no iterators found that match your criteria");
+        return 0;
+      }
+    } else if (tables) {
+      if (!shellState.getConnector().tableOperations().listIterators(OptUtil.getTableOpt(cl, shellState)).containsKey(name)) {
+        Shell.log.warn("no iterators found that match your criteria");
+        return 0;
+      }
+    } else {
+      throw new IllegalArgumentException("No table or table namespace specified");
     }
     
     final EnumSet<IteratorScope> scopes = EnumSet.noneOf(IteratorScope.class);
@@ -50,13 +63,20 @@ public class DeleteIterCommand extends Command {
     if (scopes.isEmpty()) {
       throw new IllegalArgumentException("You must select at least one scope to configure");
     }
-    shellState.getConnector().tableOperations().removeIterator(tableName, name, scopes);
+    
+    if (namespaces) {
+      shellState.getConnector().tableNamespaceOperations().removeIterator(OptUtil.getTableNamespaceOpt(cl, shellState), name, scopes);
+    } else if (tables) {
+      shellState.getConnector().tableOperations().removeIterator(OptUtil.getTableOpt(cl, shellState), name, scopes);
+    } else {
+      throw new IllegalArgumentException("No table or table namespace specified");
+    }
     return 0;
   }
   
   @Override
   public String description() {
-    return "deletes a table-specific iterator";
+    return "deletes a table-specific or table-namespace-specific iterator";
   }
   
   public Options getOptions() {
@@ -71,7 +91,10 @@ public class DeleteIterCommand extends Command {
     majcScopeOpt = new Option(IteratorScope.majc.name(), "major-compaction", false, "remove from major compaction scope");
     scanScopeOpt = new Option(IteratorScope.scan.name(), "scan-time", false, "remove from scan scope");
     
-    o.addOption(OptUtil.tableOpt("table to delete the iterator from"));
+    OptionGroup grp = new OptionGroup();
+    grp.addOption(OptUtil.tableOpt("table to delete the iterator from"));
+    grp.addOption(OptUtil.tableNamespaceOpt("table namespace to delete the iterator from"));
+    o.addOptionGroup(grp);
     o.addOption(nameOpt);
     
     o.addOption(allScopeOpt);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5acd6a48/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ListIterCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ListIterCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ListIterCommand.java
index e2d4d91..5e5018f 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ListIterCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ListIterCommand.java
@@ -27,6 +27,7 @@ import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionGroup;
 import org.apache.commons.cli.Options;
 
 public class ListIterCommand extends Command {
@@ -35,9 +36,18 @@ public class ListIterCommand extends Command {
   
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws Exception {
-    final String tableName = OptUtil.getTableOpt(cl, shellState);
     
-    final Map<String,EnumSet<IteratorScope>> iterators = shellState.getConnector().tableOperations().listIterators(tableName);
+    boolean tables = cl.hasOption(OptUtil.tableOpt().getOpt()) || !shellState.getTableName().isEmpty();
+    boolean namespaces = cl.hasOption(OptUtil.tableNamespaceOpt().getOpt());
+    
+    final Map<String,EnumSet<IteratorScope>> iterators;
+    if (namespaces) {
+      iterators = shellState.getConnector().tableNamespaceOperations().listIterators(OptUtil.getTableNamespaceOpt(cl, shellState));
+    } else if (tables) {
+      iterators = shellState.getConnector().tableOperations().listIterators(OptUtil.getTableOpt(cl, shellState));
+    } else {
+      throw new IllegalArgumentException("No table or table namespace specified");
+    }
     
     if (cl.hasOption(nameOpt.getOpt())) {
       final String name = cl.getOptionValue(nameOpt.getOpt());
@@ -57,12 +67,19 @@ public class ListIterCommand extends Command {
     }
     if (!hasScope) {
       throw new IllegalArgumentException("You must select at least one scope to configure");
-    }    
+    }
     final StringBuilder sb = new StringBuilder("-\n");
     for (String name : iterators.keySet()) {
       for (IteratorScope scope : iterators.get(name)) {
         if (cl.hasOption(scopeOpts.get(scope).getOpt())) {
-          IteratorSetting setting = shellState.getConnector().tableOperations().getIteratorSetting(tableName, name, scope);
+          IteratorSetting setting;
+          if (namespaces) {
+            setting = shellState.getConnector().tableNamespaceOperations().getIteratorSetting(OptUtil.getTableNamespaceOpt(cl, shellState), name, scope);
+          } else if (tables) {
+            setting = shellState.getConnector().tableOperations().getIteratorSetting(OptUtil.getTableOpt(cl, shellState), name, scope);
+          } else {
+            throw new IllegalArgumentException("No table or table namespace specified");
+          }
           sb.append("-    Iterator ").append(setting.getName()).append(", ").append(scope).append(" scope options:\n");
           sb.append("-        ").append("iteratorPriority").append(" = ").append(setting.getPriority()).append("\n");
           sb.append("-        ").append("iteratorClassName").append(" = ").append(setting.getIteratorClass()).append("\n");
@@ -79,7 +96,7 @@ public class ListIterCommand extends Command {
   }
   
   public String description() {
-    return "lists table-specific iterators configured in this shell session";
+    return "lists table-specific or table-namespace-specific iterators configured in this shell session";
   }
   
   @Override
@@ -99,7 +116,10 @@ public class ListIterCommand extends Command {
     scopeOpts.put(IteratorScope.majc, new Option(IteratorScope.majc.name(), "major-compaction", false, "list iterator for major compaction scope"));
     scopeOpts.put(IteratorScope.scan, new Option(IteratorScope.scan.name(), "scan-time", false, "list iterator for scan scope"));
     
-    o.addOption(OptUtil.tableOpt("table to list the configured iterators on"));
+    OptionGroup grp = new OptionGroup();
+    grp.addOption(OptUtil.tableOpt("table to list the configured iterators on"));
+    grp.addOption(OptUtil.tableNamespaceOpt("table namespace to list the configured iterators on"));
+    o.addOptionGroup(grp);
     o.addOption(nameOpt);
     
     for (Option opt : scopeOpts.values()) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5acd6a48/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OptUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OptUtil.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OptUtil.java
index f0c14d4..e92be4a 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OptUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OptUtil.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.core.util.shell.commands;
 
 import java.io.UnsupportedEncodingException;
 
+import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.commons.cli.CommandLine;
@@ -45,6 +46,19 @@ public abstract class OptUtil {
     return tableName;
   }
   
+  public static String getTableNamespaceOpt(final CommandLine cl, final Shell shellState) throws TableNamespaceNotFoundException {
+    String namespace = null;
+    if (cl.hasOption(Shell.tableNamespaceOption)) {
+      namespace = cl.getOptionValue(Shell.tableNamespaceOption);
+      if (!shellState.getConnector().tableNamespaceOperations().exists(namespace)) {
+        throw new TableNamespaceNotFoundException(namespace, namespace, "specified table namespace that doesn't exist");
+      }
+    } else {
+      throw new TableNamespaceNotFoundException(null, null, "no table namespace specified");
+    }
+    return namespace;
+  }
+  
   public static Option tableOpt() {
     return tableOpt("tableName");
   }
@@ -56,6 +70,17 @@ public abstract class OptUtil {
     return tableOpt;
   }
   
+  public static Option tableNamespaceOpt() {
+    return tableNamespaceOpt("tableNamespace");
+  }
+  
+  public static Option tableNamespaceOpt(final String description) {
+    final Option tableNamespaceOpt = new Option(Shell.tableNamespaceOption, "tableNamespace", true, description);
+    tableNamespaceOpt.setArgName("tableNamespace");
+    tableNamespaceOpt.setRequired(false);
+    return tableNamespaceOpt;
+  }
+  
   public static enum AdlOpt {
     ADD("a"), DELETE("d"), LIST("l");
     

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5acd6a48/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java
index 3a71556..ce72ecc 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java
@@ -28,6 +28,7 @@ import jline.console.ConsoleReader;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
@@ -59,6 +60,9 @@ public class SetIterCommand extends Command {
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException,
       TableNotFoundException, IOException, ShellCommandException {
     
+    boolean tables = cl.hasOption(OptUtil.tableOpt().getOpt()) || !shellState.getTableName().isEmpty();
+    boolean namespaces = cl.hasOption(OptUtil.tableNamespaceOpt().getOpt());
+    
     final int priority = Integer.parseInt(cl.getOptionValue(priorityOpt.getOpt()));
     
     final Map<String,String> options = new HashMap<String,String>();
@@ -79,17 +83,43 @@ public class SetIterCommand extends Command {
     }
     
     ClassLoader classloader = getClassLoader(cl, shellState);
-
+    
     final String name = cl.getOptionValue(nameOpt.getOpt(), setUpOptions(classloader, shellState.getReader(), classname, options));
     
-    setTableProperties(cl, shellState, priority, options, classname, name);
+    if (namespaces) {
+      try {
+        setTableNamespaceProperties(cl, shellState, priority, options, classname, name);
+      } catch (TableNamespaceNotFoundException e) {
+        throw new IllegalArgumentException(e);
+      }
+    } else if (tables) {
+      setTableProperties(cl, shellState, priority, options, classname, name);
+    } else {
+      throw new IllegalArgumentException("No table or table namespace specified");
+    }
     return 0;
   }
-
+  
   private ClassLoader getClassLoader(final CommandLine cl, final Shell shellState) throws AccumuloException, TableNotFoundException, AccumuloSecurityException,
       IOException, FileSystemException {
+    
+    boolean tables = cl.hasOption(OptUtil.tableOpt().getOpt()) || !shellState.getTableName().isEmpty();
+    boolean namespaces = cl.hasOption(OptUtil.tableNamespaceOpt().getOpt());
+    
     String classpath = null;
-    Iterable<Entry<String,String>> tableProps = shellState.getConnector().tableOperations().getProperties(OptUtil.getTableOpt(cl, shellState));
+    Iterable<Entry<String,String>> tableProps;
+    
+    if (namespaces) {
+      try {
+        tableProps = shellState.getConnector().tableNamespaceOperations().getProperties(OptUtil.getTableNamespaceOpt(cl, shellState));
+      } catch (TableNamespaceNotFoundException e) {
+        throw new IllegalArgumentException(e);
+      }
+    } else if (tables) {
+      tableProps = shellState.getConnector().tableOperations().getProperties(OptUtil.getTableOpt(cl, shellState));
+    } else {
+      throw new IllegalArgumentException("No table or table namespace specified");
+    }
     for (Entry<String,String> entry : tableProps) {
       if (entry.getKey().equals(Property.TABLE_CLASSPATH.getKey())) {
         classpath = entry.getValue();
@@ -97,7 +127,7 @@ public class SetIterCommand extends Command {
     }
     
     ClassLoader classloader;
-
+    
     if (classpath != null && !classpath.equals("")) {
       shellState.getConnector().instanceOperations().getSystemConfiguration().get(Property.VFS_CONTEXT_CLASSPATH_PROPERTY.getKey() + classpath);
       
@@ -115,7 +145,7 @@ public class SetIterCommand extends Command {
           }
         }));
       } catch (IllegalStateException ise) {}
-
+      
       classloader = AccumuloVFSClassLoader.getContextManager().getClassLoader(classpath);
     } else {
       classloader = AccumuloVFSClassLoader.getClassLoader();
@@ -128,7 +158,7 @@ public class SetIterCommand extends Command {
     // remove empty values
     
     final String tableName = OptUtil.getTableOpt(cl, shellState);
-
+    
     if (!shellState.getConnector().tableOperations().testClassLoad(tableName, classname, SortedKeyValueIterator.class.getName())) {
       throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Servers are unable to load " + classname + " as type "
           + SortedKeyValueIterator.class.getName());
@@ -165,9 +195,51 @@ public class SetIterCommand extends Command {
     shellState.getConnector().tableOperations().attachIterator(tableName, setting, scopes);
   }
   
+  protected void setTableNamespaceProperties(final CommandLine cl, final Shell shellState, final int priority, final Map<String,String> options,
+      final String classname, final String name) throws AccumuloException, AccumuloSecurityException, ShellCommandException, TableNamespaceNotFoundException {
+    // remove empty values
+    
+    final String namespace = OptUtil.getTableNamespaceOpt(cl, shellState);
+    
+    if (!shellState.getConnector().tableNamespaceOperations().testClassLoad(namespace, classname, SortedKeyValueIterator.class.getName())) {
+      throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Servers are unable to load " + classname + " as type "
+          + SortedKeyValueIterator.class.getName());
+    }
+    
+    final String aggregatorClass = options.get("aggregatorClass");
+    @SuppressWarnings("deprecation")
+    String deprecatedAggregatorClassName = org.apache.accumulo.core.iterators.aggregation.Aggregator.class.getName();
+    if (aggregatorClass != null
+        && !shellState.getConnector().tableNamespaceOperations().testClassLoad(namespace, aggregatorClass, deprecatedAggregatorClassName)) {
+      throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Servers are unable to load " + aggregatorClass + " as type "
+          + deprecatedAggregatorClassName);
+    }
+    
+    for (Iterator<Entry<String,String>> i = options.entrySet().iterator(); i.hasNext();) {
+      final Entry<String,String> entry = i.next();
+      if (entry.getValue() == null || entry.getValue().isEmpty()) {
+        i.remove();
+      }
+    }
+    final EnumSet<IteratorScope> scopes = EnumSet.noneOf(IteratorScope.class);
+    if (cl.hasOption(allScopeOpt.getOpt()) || cl.hasOption(mincScopeOpt.getOpt())) {
+      scopes.add(IteratorScope.minc);
+    }
+    if (cl.hasOption(allScopeOpt.getOpt()) || cl.hasOption(majcScopeOpt.getOpt())) {
+      scopes.add(IteratorScope.majc);
+    }
+    if (cl.hasOption(allScopeOpt.getOpt()) || cl.hasOption(scanScopeOpt.getOpt())) {
+      scopes.add(IteratorScope.scan);
+    }
+    if (scopes.isEmpty()) {
+      throw new IllegalArgumentException("You must select at least one scope to configure");
+    }
+    final IteratorSetting setting = new IteratorSetting(priority, name, classname, options);
+    shellState.getConnector().tableNamespaceOperations().attachIterator(namespace, setting, scopes);
+  }
+  
   private static String setUpOptions(ClassLoader classloader, final ConsoleReader reader, final String className, final Map<String,String> options)
-      throws IOException,
-      ShellCommandException {
+      throws IOException, ShellCommandException {
     String input;
     OptionDescriber skvi;
     Class<? extends OptionDescriber> clazz;
@@ -266,7 +338,7 @@ public class SetIterCommand extends Command {
   
   @Override
   public String description() {
-    return "sets a table-specific iterator";
+    return "sets a table-specific or table-namespace-specific iterator";
   }
   
   @Override
@@ -302,7 +374,10 @@ public class SetIterCommand extends Command {
     typeGroup.addOption(ageoffTypeOpt);
     typeGroup.setRequired(true);
     
-    o.addOption(OptUtil.tableOpt("table to configure iterators on"));
+    final OptionGroup tableGroup = new OptionGroup();
+    tableGroup.addOption(OptUtil.tableOpt("table to configure iterators on"));
+    tableGroup.addOption(OptUtil.tableNamespaceOpt("tableNamespace to configure iterators on"));
+    
     o.addOption(priorityOpt);
     o.addOption(nameOpt);
     o.addOption(allScopeOpt);
@@ -310,6 +385,7 @@ public class SetIterCommand extends Command {
     o.addOption(majcScopeOpt);
     o.addOption(scanScopeOpt);
     o.addOptionGroup(typeGroup);
+    o.addOptionGroup(tableGroup);
     return o;
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5acd6a48/core/src/main/java/org/apache/accumulo/core/util/shell/commands/TablesCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/TablesCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/TablesCommand.java
index 8449e65..5d9b087 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/TablesCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/TablesCommand.java
@@ -42,7 +42,6 @@ public class TablesCommand extends Command {
   private Option tableIdOption;
   private Option sortByTableIdOption;
   private Option disablePaginationOpt;
-  private Option tableNamespaceOpt;
 
   @SuppressWarnings("unchecked")
   @Override
@@ -52,8 +51,8 @@ public class TablesCommand extends Command {
     final Iterator<String> tableNames;
     final Iterator<String> tableIds;
 
-    if (cl.hasOption(tableNamespaceOpt.getOpt())) {
-      String namespace = shellState.getConnector().tableNamespaceOperations().namespaceIdMap().get(cl.getOptionValue(tableNamespaceOpt.getOpt()));
+    if (cl.hasOption(OptUtil.tableNamespaceOpt().getOpt())) {
+      String namespace = shellState.getConnector().tableNamespaceOperations().namespaceIdMap().get(OptUtil.getTableNamespaceOpt(cl, shellState));
       tableNames = TableNamespaces.getTableNames(shellState.getConnector().getInstance(), namespace).iterator();
       List<String> tableIdStrings = TableNamespaces.getTableIds(shellState.getConnector().getInstance(), namespace);
       if (cl.hasOption(sortByTableIdOption.getOpt()))
@@ -112,8 +111,7 @@ public class TablesCommand extends Command {
     o.addOption(sortByTableIdOption);
     disablePaginationOpt = new Option("np", "no-pagination", false, "disable pagination of output");
     o.addOption(disablePaginationOpt);
-    tableNamespaceOpt = new Option(Shell.tableNamespaceOption, "table-namespace", true, "name of table namespace to list only its tables");
-    o.addOption(tableNamespaceOpt);
+    o.addOption(OptUtil.tableNamespaceOpt("name of table namespace to list only its tables"));
     return o;
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5acd6a48/core/src/main/thrift/client.thrift
----------------------------------------------------------------------
diff --git a/core/src/main/thrift/client.thrift b/core/src/main/thrift/client.thrift
index 1e6a145..67f43d8 100644
--- a/core/src/main/thrift/client.thrift
+++ b/core/src/main/thrift/client.thrift
@@ -138,6 +138,7 @@ service ClientService {
     map<string, string> getTableNamespaceConfiguration(1:trace.TInfo tinfo, 3:security.TCredentials credentials, 2:string ns) throws (1:ThriftTableOperationException tope);
     bool checkClass(1:trace.TInfo tinfo, 4:security.TCredentials credentials, 2:string className, 3:string interfaceMatch);
     bool checkTableClass(1:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string tableId, 3:string className, 4:string interfaceMatch) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope);
+    bool checkTableNamespaceClass(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string namespaceId, 4:string className, 5:string interfaceMatch) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope);
 }
 
 // Only used for a unit test

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5acd6a48/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
index d0d83cb..0421ec1 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
@@ -90,6 +90,18 @@ public class ClientServiceHandler implements ClientService.Iface {
     return tableId;
   }
 
+  protected String checkTableNamespaceId(String tableNamespace, TableOperation operation) throws ThriftTableOperationException {
+    String namespaceId = TableNamespaces.getNameToIdMap(instance).get(tableNamespace);
+    if (namespaceId == null) {
+      // maybe the table namespace exists, but the cache was not updated yet... so try to clear the cache and check again
+      Tables.clearCache(instance);
+      namespaceId = TableNamespaces.getNameToIdMap(instance).get(tableNamespace);
+      if (namespaceId == null)
+        throw new ThriftTableOperationException(null, tableNamespace, operation, TableOperationExceptionType.NOTFOUND, null);
+    }
+    return namespaceId;
+  }
+
   @Override
   public String getInstanceId() {
     return instance.getInstanceID();
@@ -321,6 +333,40 @@ public class ClientServiceHandler implements ClientService.Iface {
   }
 
   @Override
+  public boolean checkTableNamespaceClass(TInfo tinfo, TCredentials credentials, String tableNamespace, String className, String interfaceMatch)
+      throws TException, ThriftTableOperationException, ThriftSecurityException {
+
+    security.authenticateUser(credentials, credentials);
+
+    String tableNamespaceId = checkTableNamespaceId(tableNamespace, null);
+
+    ClassLoader loader = getClass().getClassLoader();
+    Class<?> shouldMatch;
+    try {
+      shouldMatch = loader.loadClass(interfaceMatch);
+
+      new ServerConfiguration(instance).getTableNamespaceConfiguration(tableNamespaceId);
+
+      String context = new ServerConfiguration(instance).getTableNamespaceConfiguration(tableNamespaceId).get(Property.TABLE_CLASSPATH);
+
+      ClassLoader currentLoader;
+
+      if (context != null && !context.equals("")) {
+        currentLoader = AccumuloVFSClassLoader.getContextManager().getClassLoader(context);
+      } else {
+        currentLoader = AccumuloVFSClassLoader.getClassLoader();
+      }
+
+      Class<?> test = currentLoader.loadClass(className).asSubclass(shouldMatch);
+      test.newInstance();
+      return true;
+    } catch (Exception e) {
+      log.warn("Error checking object types", e);
+      return false;
+    }
+  }
+
+  @Override
   public List<TDiskUsage> getDiskUsage(Set<String> tables, TCredentials credentials) throws ThriftTableOperationException, ThriftSecurityException, TException {
     try {
       AuthenticationToken token = AuthenticationTokenSerializer.deserialize(credentials.getTokenClassName(), credentials.getToken());


[06/50] [abbrv] ACCUMULO-1479 implemented most of Table Namespace Permissions, doesnt entirely work, not well tested

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/SecurityErrorCode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/SecurityErrorCode.java b/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/SecurityErrorCode.java
index b706ce8..9bf554f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/SecurityErrorCode.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/SecurityErrorCode.java
@@ -45,7 +45,8 @@ import org.apache.thrift.TEnum;
   PERMISSIONHANDLER_FAILED(14),
   TOKEN_EXPIRED(15),
   SERIALIZATION_ERROR(16),
-  INSUFFICIENT_PROPERTIES(17);
+  INSUFFICIENT_PROPERTIES(17),
+  TABLE_NAMESPACE_DOESNT_EXIST(18);
 
   private final int value;
 
@@ -102,6 +103,8 @@ import org.apache.thrift.TEnum;
         return SERIALIZATION_ERROR;
       case 17:
         return INSUFFICIENT_PROPERTIES;
+      case 18:
+        return TABLE_NAMESPACE_DOESNT_EXIST;
       default:
         return null;
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
index 37e8379..f96abad 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
@@ -32,6 +32,7 @@ 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.SystemPermission;
+import org.apache.accumulo.core.security.TableNamespacePermission;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.Text;
@@ -100,7 +101,7 @@ public class MockAccumulo {
   public void createNamespace(String username, String namespace) {
     if (!namespaceExists(namespace)) {
       MockTableNamespace n = new MockTableNamespace();
-      n.userPermissions.put(username, EnumSet.allOf(TablePermission.class));
+      n.userPermissions.put(username, EnumSet.allOf(TableNamespacePermission.class));
       namespaces.put(namespace, n);
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
index 765cda9..dd48b52 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
@@ -27,6 +27,7 @@ import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 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.TablePermission;
 
 public class MockSecurityOperations implements SecurityOperations {
@@ -128,6 +129,17 @@ public class MockSecurityOperations implements SecurityOperations {
   }
   
   @Override
+  public boolean hasTableNamespacePermission(String principal, String tableNamespace, TableNamespacePermission perm) throws AccumuloException, AccumuloSecurityException {
+    MockTableNamespace namespace = acu.namespaces.get(tableNamespace);
+    if (namespace == null)
+      throw new AccumuloSecurityException(tableNamespace, SecurityErrorCode.TABLE_DOESNT_EXIST);
+    EnumSet<TableNamespacePermission> perms = namespace.userPermissions.get(principal);
+    if (perms == null)
+      return false;
+    return perms.contains(perm);
+  }
+  
+  @Override
   public void grantSystemPermission(String principal, SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
     MockUser user = acu.users.get(principal);
     if (user != null)
@@ -151,6 +163,20 @@ public class MockSecurityOperations implements SecurityOperations {
   }
   
   @Override
+  public void grantTableNamespacePermission(String principal, String tableNamespace, TableNamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
+    if (acu.users.get(principal) == null)
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+    MockTableNamespace namespace = acu.namespaces.get(tableNamespace);
+    if (namespace == null)
+      throw new AccumuloSecurityException(tableNamespace, SecurityErrorCode.TABLE_DOESNT_EXIST);
+    EnumSet<TableNamespacePermission> perms = namespace.userPermissions.get(principal);
+    if (perms == null)
+      namespace.userPermissions.put(principal, EnumSet.of(permission));
+    else
+      perms.add(permission);
+  }
+  
+  @Override
   public void revokeSystemPermission(String principal, SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
     MockUser user = acu.users.get(principal);
     if (user != null)
@@ -172,6 +198,19 @@ public class MockSecurityOperations implements SecurityOperations {
     
   }
   
+  @Override
+  public void revokeTableNamespacePermission(String principal, String tableNamespace, TableNamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
+    if (acu.users.get(principal) == null)
+      throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
+    MockTableNamespace namespace = acu.namespaces.get(tableNamespace);
+    if (namespace == null)
+      throw new AccumuloSecurityException(tableNamespace, SecurityErrorCode.TABLE_DOESNT_EXIST);
+    EnumSet<TableNamespacePermission> perms = namespace.userPermissions.get(principal);
+    if (perms != null)
+      perms.remove(permission);
+    
+  }
+  
   @Deprecated
   @Override
   public Set<String> listUsers() throws AccumuloException, AccumuloSecurityException {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java
index 2a32165..beec4db 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java
@@ -25,13 +25,13 @@ import java.util.Map.Entry;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.core.security.TableNamespacePermission;
 
 
 public class MockTableNamespace {
   
   final Map<String,String> settings;
-  Map<String,EnumSet<TablePermission>> userPermissions = new HashMap<String,EnumSet<TablePermission>>();
+  Map<String,EnumSet<TableNamespacePermission>> userPermissions = new HashMap<String,EnumSet<TableNamespacePermission>>();
   
   public MockTableNamespace() {
     settings = new HashMap<String,String>();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/core/src/main/java/org/apache/accumulo/core/client/security/SecurityErrorCode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/security/SecurityErrorCode.java b/core/src/main/java/org/apache/accumulo/core/client/security/SecurityErrorCode.java
index fb51387..f1ea539 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/security/SecurityErrorCode.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/security/SecurityErrorCode.java
@@ -37,5 +37,6 @@ public enum SecurityErrorCode {
   PERMISSIONHANDLER_FAILED,
   TOKEN_EXPIRED,
   SERIALIZATION_ERROR,
-  INSUFFICIENT_PROPERTIES;
+  INSUFFICIENT_PROPERTIES,
+  TABLE_NAMESPACE_DOESNT_EXIST;
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/core/src/main/java/org/apache/accumulo/core/security/SystemPermission.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/security/SystemPermission.java b/core/src/main/java/org/apache/accumulo/core/security/SystemPermission.java
index 699396e..6bdb8e6 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/SystemPermission.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/SystemPermission.java
@@ -29,7 +29,10 @@ public enum SystemPermission {
   CREATE_USER((byte) 4),
   DROP_USER((byte) 5),
   ALTER_USER((byte) 6),
-  SYSTEM((byte) 7);
+  SYSTEM((byte) 7),
+  CREATE_NAMESPACE((byte) 8),
+  DROP_NAMESPACE((byte) 9),
+  ALTER_NAMESPACE((byte) 10);
   
   private byte permID;
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/core/src/main/java/org/apache/accumulo/core/security/TableNamespacePermission.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/security/TableNamespacePermission.java b/core/src/main/java/org/apache/accumulo/core/security/TableNamespacePermission.java
new file mode 100644
index 0000000..565a81a
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/security/TableNamespacePermission.java
@@ -0,0 +1,67 @@
+/*
+ * 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.core.security;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public enum TableNamespacePermission {
+  // One can add new permissions, with new numbers, but please don't change or use numbers previously assigned
+  READ((byte) 0),
+  WRITE((byte) 1),
+  ALTER_TABLE_NAMESPACE((byte) 2),
+  GRANT((byte) 3),
+  ALTER_TABLE((byte) 4),
+  CREATE_TABLE((byte) 5),
+  DROP_TABLE((byte) 6),
+  BULK_IMPORT((byte) 7);
+  
+  final private byte permID;
+  
+  final private static TableNamespacePermission mapping[] = new TableNamespacePermission[8];
+  static {
+    for (TableNamespacePermission perm : TableNamespacePermission.values())
+      mapping[perm.permID] = perm;
+  }
+  
+  private TableNamespacePermission(byte id) {
+    this.permID = id;
+  }
+  
+  public byte getId() {
+    return this.permID;
+  }
+  
+  public static List<String> printableValues() {
+    TableNamespacePermission[] a = TableNamespacePermission.values();
+    
+    List<String> list = new ArrayList<String>(a.length);
+    
+    for (TableNamespacePermission p : a)
+      list.add("Namespace." + p);
+    
+    return list;
+  }
+  
+  public static TableNamespacePermission getPermissionById(byte id) {
+    TableNamespacePermission result = mapping[id];
+    if (result != null)
+      return result;
+    throw new IndexOutOfBoundsException("No such permission");
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GrantCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GrantCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GrantCommand.java
index f3e7200..9f6e7e4 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GrantCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GrantCommand.java
@@ -21,6 +21,7 @@ import java.util.Set;
 
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.core.security.TableNamespacePermission;
 import org.apache.accumulo.core.util.BadArgumentException;
 import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
@@ -53,6 +54,17 @@ public class GrantCommand extends TableOperation {
       }
     } else if (permission[0].equalsIgnoreCase("Table")) {
       super.execute(fullCommand, cl, shellState);
+    } else if (permission[0].equalsIgnoreCase("Namespace")) {
+      if (cl.hasOption(optTableNamespace.getOpt())) {
+        try {
+          shellState.getConnector().securityOperations()
+              .grantTableNamespacePermission(user, cl.getOptionValue(optTableNamespace.getOpt()), TableNamespacePermission.valueOf(permission[1]));
+        } catch (IllegalArgumentException e) {
+          throw new BadArgumentException("No such table namespace permission", fullCommand, fullCommand.indexOf(cl.getArgs()[0]));
+        }
+      } else {
+        throw new BadArgumentException("No Table Namespace specified to apply permission to", fullCommand, fullCommand.indexOf(cl.getArgs()[0]));
+      }
     } else {
       throw new BadArgumentException("Unrecognized permission", fullCommand, fullCommand.indexOf(cl.getArgs()[0]));
     }
@@ -71,7 +83,7 @@ public class GrantCommand extends TableOperation {
   
   @Override
   public String description() {
-    return "grants system or table permissions for a user";
+    return "grants system, table, or table namespace permissions for a user";
   }
   
   @Override
@@ -84,6 +96,7 @@ public class GrantCommand extends TableOperation {
     final Token cmd = new Token(getName());
     cmd.addSubcommand(new Token(TablePermission.printableValues()));
     cmd.addSubcommand(new Token(SystemPermission.printableValues()));
+    cmd.addSubcommand(new Token(TableNamespacePermission.printableValues()));
     root.addSubcommand(cmd);
   }
   
@@ -96,9 +109,13 @@ public class GrantCommand extends TableOperation {
     
     systemOpt = new Option("s", "system", false, "grant a system permission");
     
+    optTableNamespace = new Option(Shell.tableNamespaceOption, "table-namespace", true, "name of a table namespace to operate on");
+    optTableNamespace.setArgName("tableNamespace");
+    
     group.addOption(systemOpt);
     group.addOption(optTableName);
     group.addOption(optTablePattern);
+    group.addOption(optTableNamespace);
     
     o.addOptionGroup(group);
     userOpt = new Option(Shell.userOption, "user", true, "user to operate on");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RevokeCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RevokeCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RevokeCommand.java
index 676284a..cc9cd93 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RevokeCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RevokeCommand.java
@@ -20,6 +20,7 @@ import java.util.Map;
 import java.util.Set;
 
 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.util.BadArgumentException;
 import org.apache.accumulo.core.util.shell.Shell;
@@ -53,7 +54,18 @@ public class RevokeCommand extends TableOperation {
       }
     } else if (permission[0].equalsIgnoreCase("Table")) {
       super.execute(fullCommand, cl, shellState);
-    } else {
+    } else if (permission[0].equalsIgnoreCase("Namespace")) {
+      if (cl.hasOption(optTableNamespace.getOpt())) {
+        try {
+          shellState.getConnector().securityOperations()
+              .revokeTableNamespacePermission(user, cl.getOptionValue(optTableNamespace.getOpt()), TableNamespacePermission.valueOf(permission[1]));
+        } catch (IllegalArgumentException e) {
+          throw new BadArgumentException("No such table namespace permission", fullCommand, fullCommand.indexOf(cl.getArgs()[0]));
+        }
+      } else {
+        throw new BadArgumentException("No Table Namespace specified to apply permission to", fullCommand, fullCommand.indexOf(cl.getArgs()[0]));
+      } 
+    }else {
       throw new BadArgumentException("Unrecognized permission", fullCommand, fullCommand.indexOf(cl.getArgs()[0]));
     }
     return 0;
@@ -84,6 +96,7 @@ public class RevokeCommand extends TableOperation {
     final Token cmd = new Token(getName());
     cmd.addSubcommand(new Token(TablePermission.printableValues()));
     cmd.addSubcommand(new Token(SystemPermission.printableValues()));
+    cmd.addSubcommand(new Token(TableNamespacePermission.printableValues()));
     root.addSubcommand(cmd);
   }
   
@@ -96,9 +109,13 @@ public class RevokeCommand extends TableOperation {
     
     systemOpt = new Option("s", "system", false, "revoke a system permission");
     
+    optTableNamespace = new Option(Shell.tableNamespaceOption, "table-namespace", true, "name of a table namespace to operate on");
+    optTableNamespace.setArgName("tableNamespace");
+        
     group.addOption(systemOpt);
     group.addOption(optTableName);
     group.addOption(optTablePattern);
+    group.addOption(optTableNamespace);
     
     o.addOptionGroup(group);
     userOpt = new Option(Shell.userOption, "user", true, "user to operate on");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserPermissionsCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserPermissionsCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserPermissionsCommand.java
index 79d78da..25d9d54 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserPermissionsCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserPermissionsCommand.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 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.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
@@ -62,12 +63,29 @@ public class UserPermissionsCommand extends Command {
       runOnce = 0;
     }
     shellState.getReader().println();
+    
+    for (String n : shellState.getConnector().tableNamespaceOperations().list()) {
+      delim = "";
+      for (TableNamespacePermission p : TableNamespacePermission.values()) {
+        if (p != null && shellState.getConnector().securityOperations().hasTableNamespacePermission(user, n, p)) {
+          if (runOnce == 0) {
+            shellState.getReader().print("\nTable Namespace permissions (" + n + "): ");
+            runOnce++;
+          }
+          shellState.getReader().print(delim + "Namespace." + p.name());
+          delim = ", ";
+        }
+      }
+      runOnce = 0;
+    }
+    shellState.getReader().println();
+    
     return 0;
   }
   
   @Override
   public String description() {
-    return "displays a user's system and table permissions";
+    return "displays a user's system, table, and table namespace permissions";
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/core/src/main/thrift/client.thrift
----------------------------------------------------------------------
diff --git a/core/src/main/thrift/client.thrift b/core/src/main/thrift/client.thrift
index 67f43d8..7c340d7 100644
--- a/core/src/main/thrift/client.thrift
+++ b/core/src/main/thrift/client.thrift
@@ -72,9 +72,10 @@ enum SecurityErrorCode {
     AUTHENTICATOR_FAILED = 12,
     AUTHORIZOR_FAILED = 13,
     PERMISSIONHANDLER_FAILED = 14,
-    TOKEN_EXPIRED = 15
-    SERIALIZATION_ERROR = 16;
-    INSUFFICIENT_PROPERTIES = 17;
+    TOKEN_EXPIRED = 15,
+    SERIALIZATION_ERROR = 16,
+    INSUFFICIENT_PROPERTIES = 17,
+    TABLE_NAMESPACE_DOESNT_EXIST = 18;
 }
 
 exception ThriftSecurityException {
@@ -127,15 +128,18 @@ service ClientService {
     // permissions-related methods
     bool hasSystemPermission(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:byte sysPerm) throws (1:ThriftSecurityException sec)
     bool hasTablePermission(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:string tableName, 4:byte tblPerm) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+    bool hasTableNamespacePermission(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string principal, 4:string tableNamespace, 5:byte tblNspcPerm) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
     void grantSystemPermission(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:byte permission) throws (1:ThriftSecurityException sec)
     void revokeSystemPermission(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:byte permission) throws (1:ThriftSecurityException sec)
     void grantTablePermission(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:string tableName, 4:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
     void revokeTablePermission(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:string tableName, 4:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+    void grantTableNamespacePermission(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string principal, 4:string tableNamespace, 5:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+    void revokeTableNamespacePermission(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string principal, 4:string tableNamespace, 5:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
 
     // configuration methods
     map<string, string> getConfiguration(2:trace.TInfo tinfo, 3:security.TCredentials credentials, 1:ConfigurationType type);
     map<string, string> getTableConfiguration(1:trace.TInfo tinfo, 3:security.TCredentials credentials, 2:string tableName) throws (1:ThriftTableOperationException tope);
-    map<string, string> getTableNamespaceConfiguration(1:trace.TInfo tinfo, 3:security.TCredentials credentials, 2:string ns) throws (1:ThriftTableOperationException tope);
+    map<string, string> getTableNamespaceConfiguration(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string ns) throws (1:ThriftTableOperationException tope);
     bool checkClass(1:trace.TInfo tinfo, 4:security.TCredentials credentials, 2:string className, 3:string interfaceMatch);
     bool checkTableClass(1:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string tableId, 3:string className, 4:string interfaceMatch) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope);
     bool checkTableNamespaceClass(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string namespaceId, 4:string className, 5:string interfaceMatch) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
index 0421ec1..3148a3d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
@@ -53,6 +53,7 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.core.security.TableNamespacePermission;
 import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -185,6 +186,13 @@ public class ClientServiceHandler implements ClientService.Iface {
   }
 
   @Override
+  public void grantTableNamespacePermission(TInfo tinfo, TCredentials credentials, String user, String tableNamespace, byte permission) throws ThriftSecurityException,
+      ThriftTableOperationException {
+    String namespaceId = checkTableNamespaceId(tableNamespace, TableOperation.PERMISSION);
+    security.grantTableNamespacePermission(credentials, user, namespaceId, TableNamespacePermission.getPermissionById(permission));
+  }
+  
+  @Override
   public void revokeSystemPermission(TInfo tinfo, TCredentials credentials, String user, byte permission) throws ThriftSecurityException {
     security.revokeSystemPermission(credentials, user, SystemPermission.getPermissionById(permission));
   }
@@ -209,6 +217,20 @@ public class ClientServiceHandler implements ClientService.Iface {
   }
 
   @Override
+  public boolean hasTableNamespacePermission(TInfo tinfo, TCredentials credentials, String user, String tableNamespace, byte perm) throws ThriftSecurityException,
+      ThriftTableOperationException {
+    String namespaceId = checkTableNamespaceId(tableNamespace, TableOperation.PERMISSION);
+    return security.hasTableNamespacePermission(credentials, user, namespaceId, TableNamespacePermission.getPermissionById(perm));
+  }
+  
+  @Override
+  public void revokeTableNamespacePermission(TInfo tinfo, TCredentials credentials, String user, String tableNamespace, byte permission) throws ThriftSecurityException,
+      ThriftTableOperationException {
+    String namespaceId = checkTableNamespaceId(tableNamespace, TableOperation.PERMISSION);
+    security.revokeTableNamespacePermission(credentials, user, namespaceId, TableNamespacePermission.getPermissionById(permission));
+  }
+  
+  @Override
   public Set<String> listLocalUsers(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException {
     return security.listUsers(credentials);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/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 f00159c..cad84d0 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,8 +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.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.SecurityOperationsImpl;
+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;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
@@ -40,6 +42,7 @@ 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.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;
@@ -304,6 +307,64 @@ public class SecurityOperation {
     }
   }
   
+  /**
+   * Checks if a user has a table namespace permission
+   * 
+   * @return true if a user exists and has permission; false otherwise
+   */
+  protected boolean hasTableNamespacePermission(TCredentials credentials, String tableNamespace, TableNamespacePermission permission, boolean useCached)
+      throws ThriftSecurityException {
+    if (isSystemUser(credentials))
+      return true;
+    return _hasTableNamespacePermission(credentials.getPrincipal(), tableNamespace, permission, useCached);
+  }
+  
+  /**
+   * Checks if a user has a table 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)
+      throws ThriftSecurityException {
+    String tableNamespace = Tables.getNamespace(HdfsZooInstance.getInstance(), tableId);
+    return hasTableNamespacePermission(credentials, tableNamespace, permission, useCached);
+  }
+  
+  /**
+   * Checks if a user has a table 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)
+      throws ThriftSecurityException {
+    String tableNamespace = Tables.extractNamespace(tableName);
+    return hasTableNamespacePermission(credentials, tableNamespace, permission, useCached);
+  }
+  
+  /**
+   * Checks if a user has a table 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 {
+    targetUserExists(user);
+    
+    if (tableNamespace.equals(Constants.SYSTEM_TABLE_NAMESPACE_ID) && permission.equals(TableNamespacePermission.READ))
+      return true;
+    
+    try {
+      if (useCached)
+        return permHandle.hasCachedTableNamespacePermission(user, tableNamespace, permission);
+      return permHandle.hasTableNamespacePermission(user, tableNamespace, permission);
+    } catch (AccumuloSecurityException e) {
+      throw e.asThriftException();
+    } catch (TableNamespaceNotFoundException e) {
+      throw new ThriftSecurityException(user, SecurityErrorCode.TABLE_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);
@@ -325,7 +386,8 @@ public class SecurityOperation {
   
   public boolean canScan(TCredentials credentials, String table) throws ThriftSecurityException {
     authenticate(credentials);
-    return hasTablePermission(credentials, table, TablePermission.READ, true);
+    return hasTablePermission(credentials, table, TablePermission.READ, true)
+        || hasTableNamespacePermissionForTableId(credentials, table, TableNamespacePermission.READ, true);
   }
   
   public boolean canScan(TCredentials credentials, String table, TRange range, List<TColumn> columns, List<IterInfo> ssiList,
@@ -340,20 +402,25 @@ public class SecurityOperation {
   
   public boolean canWrite(TCredentials credentials, String table) throws ThriftSecurityException {
     authenticate(credentials);
-    return hasTablePermission(credentials, table, TablePermission.WRITE, true);
+    return hasTablePermission(credentials, table, TablePermission.WRITE, true)
+        || hasTableNamespacePermissionForTableId(credentials, table, TableNamespacePermission.WRITE, true);
   }
   
   public boolean canConditionallyUpdate(TCredentials credentials, String tableID, List<ByteBuffer> authorizations) throws ThriftSecurityException {
     
     authenticate(credentials);
     
-    return hasTablePermission(credentials, tableID, TablePermission.WRITE, true) && hasTablePermission(credentials, tableID, TablePermission.READ, true);
+    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));
   }
   
   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);
+        || hasTablePermission(credentials, table, TablePermission.ALTER_TABLE, false)
+        || hasTableNamespacePermissionForTableId(credentials, table, TableNamespacePermission.ALTER_TABLE, false);
   }
   
   /**
@@ -366,16 +433,19 @@ public class SecurityOperation {
   
   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);
+    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);
   }
   
   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);
+    return hasTablePermission(c, tableId, TablePermission.ALTER_TABLE, false) || hasSystemPermission(c, SystemPermission.ALTER_TABLE, false)
+        || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.ALTER_TABLE, false);
   }
   
   public boolean canCreateTable(TCredentials c, String tableName) throws ThriftSecurityException {
-    return canCreateTable(c);
+    return canCreateTable(c) || hasTableNamespacePermissionForTableName(c, tableName, TableNamespacePermission.CREATE_TABLE, false);
   }
   
   public boolean canCreateTable(TCredentials c) throws ThriftSecurityException {
@@ -385,34 +455,41 @@ public class SecurityOperation {
   
   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);
+    return hasSystemPermission(c, SystemPermission.ALTER_TABLE, false) || hasTablePermission(c, tableId, TablePermission.ALTER_TABLE, false)
+        || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.ALTER_TABLE, false);
   }
   
   public boolean canCloneTable(TCredentials c, String tableId, String tableName) throws ThriftSecurityException {
     authenticate(c);
-    return hasSystemPermission(c, SystemPermission.CREATE_TABLE, false) && hasTablePermission(c, tableId, TablePermission.READ, false);
+    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));
   }
   
   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);
+    return hasSystemPermission(c, SystemPermission.DROP_TABLE, false) || hasTablePermission(c, tableId, TablePermission.DROP_TABLE, false)
+        || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.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);
+        || hasTablePermission(c, tableId, TablePermission.ALTER_TABLE, false)
+        || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.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);
+        || hasTablePermission(c, tableId, TablePermission.ALTER_TABLE, false)
+        || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.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);
+    return hasSystemPermission(c, SystemPermission.SYSTEM, false) || hasTablePermission(c, tableId, TablePermission.WRITE, false)
+        || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.WRITE, false);
   }
   
   public boolean canBulkImport(TCredentials c, String tableId, String tableName, String dir, String failDir) throws ThriftSecurityException {
@@ -421,13 +498,16 @@ public class SecurityOperation {
   
   public boolean canBulkImport(TCredentials c, String tableId) throws ThriftSecurityException {
     authenticate(c);
-    return hasTablePermission(c, tableId, TablePermission.BULK_IMPORT, false);
+    return hasTablePermission(c, tableId, TablePermission.BULK_IMPORT, false)
+        || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.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);
+        || hasTablePermission(c, tableId, TablePermission.WRITE, false)
+        || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.ALTER_TABLE, false)
+        || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.WRITE, false);
   }
   
   public boolean canChangeAuthorizations(TCredentials c, String user) throws ThriftSecurityException {
@@ -462,7 +542,14 @@ public class SecurityOperation {
   
   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);
+    return hasSystemPermission(c, SystemPermission.ALTER_TABLE, false) || hasTablePermission(c, table, TablePermission.GRANT, false)
+        || hasTableNamespacePermissionForTableId(c, table, TableNamespacePermission.ALTER_TABLE, false);
+  }
+  
+  public boolean canGrantTableNamespace(TCredentials c, String user, String tableNamespace) throws ThriftSecurityException {
+    authenticate(c);
+    return hasSystemPermission(c, SystemPermission.ALTER_NAMESPACE, false)
+        || hasTableNamespacePermission(c, tableNamespace, TableNamespacePermission.GRANT, false);
   }
   
   public boolean canRevokeSystem(TCredentials c, String user, SystemPermission sysPerm) throws ThriftSecurityException {
@@ -480,7 +567,13 @@ public class SecurityOperation {
   
   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);
+    return hasSystemPermission(c, SystemPermission.ALTER_TABLE, false) || hasTablePermission(c, table, TablePermission.GRANT, false)
+        || hasTableNamespacePermissionForTableId(c, table, TableNamespacePermission.ALTER_TABLE, false);
+  }
+  
+  public boolean canRevokeTableNamespace(TCredentials c, String user, String tableNamespace) throws ThriftSecurityException {
+    authenticate(c);
+    return hasSystemPermission(c, SystemPermission.ALTER_NAMESPACE, false) || hasTablePermission(c, tableNamespace, TablePermission.GRANT, false);
   }
   
   public void changeAuthorizations(TCredentials credentials, String user, Authorizations authorizations) throws ThriftSecurityException {
@@ -568,6 +661,24 @@ public class SecurityOperation {
     }
   }
   
+  public void grantTableNamespacePermission(TCredentials c, String user, String tableNamespace, TableNamespacePermission permission)
+      throws ThriftSecurityException {
+    if (!canGrantTableNamespace(c, user, tableNamespace))
+      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());
+    } catch (AccumuloSecurityException e) {
+      throw e.asThriftException();
+    } catch (TableNamespaceNotFoundException e) {
+      throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.TABLE_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);
@@ -600,6 +711,25 @@ public class SecurityOperation {
     }
   }
   
+  public void revokeTableNamespacePermission(TCredentials c, String user, String tableNamespace, TableNamespacePermission permission)
+      throws ThriftSecurityException {
+    if (!canRevokeTableNamespace(c, user, tableNamespace))
+      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());
+      
+    } catch (AccumuloSecurityException e) {
+      throw e.asThriftException();
+    } catch (TableNamespaceNotFoundException e) {
+      throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.TABLE_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);
@@ -612,6 +742,13 @@ public class SecurityOperation {
     return _hasTablePermission(user, tableId, permissionById, false);
   }
   
+  public boolean hasTableNamespacePermission(TCredentials credentials, String user, String tableNamespace, TableNamespacePermission permissionById)
+      throws ThriftSecurityException {
+    if (!canAskAboutOtherUsers(credentials, user))
+      throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+    return _hasTableNamespacePermission(user, tableNamespace, permissionById, false);
+  }
+  
   public Set<String> listUsers(TCredentials credentials) throws ThriftSecurityException {
     authenticate(credentials);
     try {
@@ -634,13 +771,29 @@ public class SecurityOperation {
     }
   }
   
+  public void deleteTableNamespace(TCredentials credentials, String tableNamespace) throws ThriftSecurityException {
+    if (!canDeleteTable(credentials, tableNamespace))
+      throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+    try {
+      permHandle.cleanTableNamespacePermissions(tableNamespace);
+    } catch (AccumuloSecurityException e) {
+      e.setUser(credentials.getPrincipal());
+      throw e.asThriftException();
+    } catch (TableNamespaceNotFoundException e) {
+      throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.TABLE_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);
+    return hasTablePermission(credentials, tableId, TablePermission.READ, false)
+        || hasTableNamespacePermissionForTableId(credentials, tableId, TableNamespacePermission.READ, false);
   }
   
   public boolean canImport(TCredentials credentials, String tableName, String importDir) throws ThriftSecurityException {
     authenticate(credentials);
-    return hasSystemPermission(credentials, SystemPermission.CREATE_TABLE, false);
+    String tableId = Tables.getNamespace(HdfsZooInstance.getInstance(), tableName);
+    return hasSystemPermission(credentials, SystemPermission.CREATE_TABLE, false)
+        || hasTableNamespacePermissionForTableId(credentials, tableId, TableNamespacePermission.CREATE_TABLE, false);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/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 b57abfe..f1d69e8 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,8 +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.TableNotFoundException;
 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;
 
@@ -99,5 +101,34 @@ public class InsecurePermHandler implements PermissionHandler {
   
   @Override
   public void initTable(String table) throws AccumuloSecurityException {}
+
+  @Override
+  public boolean hasTableNamespacePermission(String user, String namespace, TableNamespacePermission permission) throws AccumuloSecurityException,
+      TableNamespaceNotFoundException {
+    return true;
+  }
+
+  @Override
+  public boolean hasCachedTableNamespacePermission(String user, String namespace, TableNamespacePermission permission) throws AccumuloSecurityException,
+      TableNamespaceNotFoundException {
+    return true;
+  }
+
+  @Override
+  public void grantTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws AccumuloSecurityException,
+      TableNamespaceNotFoundException {
+    return;
+  }
+
+  @Override
+  public void revokeTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws AccumuloSecurityException,
+      TableNamespaceNotFoundException {
+    return;
+  }
+
+  @Override
+  public void cleanTableNamespacePermissions(String tableNamespace) throws AccumuloSecurityException, TableNamespaceNotFoundException {
+    return;
+  }
   
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/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 72c64b5..63e7208 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,9 +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.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 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;
 
@@ -66,6 +68,16 @@ public interface PermissionHandler {
   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
+   */
+  public boolean hasTableNamespacePermission(String user, String namespace, TableNamespacePermission permission) throws AccumuloSecurityException, TableNamespaceNotFoundException;
+  
+  /**
+   * Used to get the table namespace permission of a user for a table namespace, with caching. This method is for high frequency operations
+   */
+  public boolean hasCachedTableNamespacePermission(String user, String namespace, TableNamespacePermission permission) throws AccumuloSecurityException, TableNamespaceNotFoundException;
+  
+  /**
    * Gives the user the given system permission
    */
   public void grantSystemPermission(String user, SystemPermission permission) throws AccumuloSecurityException;
@@ -86,11 +98,27 @@ public interface PermissionHandler {
   public void revokeTablePermission(String user, String table, TablePermission permission) throws AccumuloSecurityException, TableNotFoundException;
   
   /**
+   * Gives the user the given table namespace permission
+   */
+  public void grantTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws AccumuloSecurityException, TableNamespaceNotFoundException;
+  
+  /**
+   * Denies the user the given table namespace permission.
+   */
+  public void revokeTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws AccumuloSecurityException, TableNamespaceNotFoundException;
+  
+  
+  /**
    * 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.
+   */
+  public void cleanTableNamespacePermissions(String tableNamespace) throws AccumuloSecurityException, TableNamespaceNotFoundException;
+  
+  /**
    * Initializes a new user
    */
   public void initUser(String user) throws AccumuloSecurityException;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/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 f219603..78b79a1 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
@@ -24,11 +24,13 @@ import java.util.Set;
 import java.util.TreeSet;
 
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
 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.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,9 +51,11 @@ public class ZKPermHandler implements PermissionHandler {
   
   private String ZKUserPath;
   private String ZKTablePath;
+  private String ZKNamespacePath;
   private final ZooCache zooCache;
   private final String ZKUserSysPerms = "/System";
   private final String ZKUserTablePerms = "/Tables";
+  private final String ZKUserNamespacePerms = "/Namespaces";
   
   public static synchronized PermissionHandler getInstance() {
     if (zkPermHandlerInstance == null)
@@ -63,6 +67,7 @@ public class ZKPermHandler implements PermissionHandler {
   public void initialize(String instanceId, boolean initialize) {
     ZKUserPath = ZKSecurityTool.getInstancePath(instanceId) + "/users";
     ZKTablePath = ZKSecurityTool.getInstancePath(instanceId) + "/tables";
+    ZKNamespacePath = ZKSecurityTool.getInstancePath(instanceId) + "/namespaces";
   }
   
   public ZKPermHandler() {
@@ -118,6 +123,54 @@ public class ZKPermHandler implements PermissionHandler {
   }
   
   @Override
+  public boolean hasTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws TableNamespaceNotFoundException {
+    byte[] serializedPerms;
+    try {
+      String path = ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + tableNamespace;
+      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?
+        try {
+          // check for existence:
+          ZooReaderWriter.getRetryingInstance().getData(ZKNamespacePath + "/" + tableNamespace, 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);
+          return false;
+        } catch (KeeperException ex) {
+          // not there, throw an informative exception
+          if (e.code() == Code.NONODE) {
+            throw new TableNamespaceNotFoundException(null, tableNamespace, "while checking permissions");
+          }
+          log.warn("Unhandled InterruptedException, failing closed for table permission check", e);
+        }
+        return false;
+      }
+      log.warn("Unhandled KeeperException, failing closed for table permission check", e);
+      return false;
+    } catch (InterruptedException e) {
+      log.warn("Unhandled InterruptedException, failing closed for table permission check", e);
+      return false;
+    }
+    if (serializedPerms != null) {
+      return ZKSecurityTool.convertTableNamespacePermissions(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);
+    if (serializedPerms != null) {
+      return ZKSecurityTool.convertTableNamespacePermissions(serializedPerms).contains(permission);
+    }
+    return false;
+  }
+  
+  @Override
   public void grantSystemPermission(String user, SystemPermission permission) throws AccumuloSecurityException {
     try {
       byte[] permBytes = zooCache.get(ZKUserPath + "/" + user + ZKUserSysPerms);
@@ -172,6 +225,33 @@ public class ZKPermHandler implements PermissionHandler {
   }
   
   @Override
+  public void grantTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws AccumuloSecurityException {
+    Set<TableNamespacePermission> tableNamespacePerms;
+    byte[] serializedPerms = zooCache.get(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + tableNamespace);
+    if (serializedPerms != null)
+      tableNamespacePerms = ZKSecurityTool.convertTableNamespacePermissions(serializedPerms);
+    else
+      tableNamespacePerms = new TreeSet<TableNamespacePermission>();
+    
+    try {
+      if (tableNamespacePerms.add(permission)) {
+        synchronized (zooCache) {
+          zooCache.clear(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + tableNamespace);
+          IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
+          zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + tableNamespace, ZKSecurityTool.convertTableNamespacePermissions(tableNamespacePerms),
+              NodeExistsPolicy.OVERWRITE);
+        }
+      }
+    } catch (KeeperException e) {
+      log.error(e, e);
+      throw new AccumuloSecurityException(user, SecurityErrorCode.CONNECTION_ERROR, e);
+    } catch (InterruptedException e) {
+      log.error(e, e);
+      throw new RuntimeException(e);
+    }
+  }
+  
+  @Override
   public void revokeSystemPermission(String user, SystemPermission permission) throws AccumuloSecurityException {
     byte[] sysPermBytes = zooCache.get(ZKUserPath + "/" + user + ZKUserSysPerms);
     
@@ -227,6 +307,34 @@ public class ZKPermHandler implements PermissionHandler {
   }
   
   @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.
+    if (serializedPerms == null)
+      return;
+    
+    Set<TableNamespacePermission> tableNamespacePerms = ZKSecurityTool.convertTableNamespacePermissions(serializedPerms);
+    try {
+      if (tableNamespacePerms.remove(permission)) {
+        zooCache.clear();
+        IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
+        if (tableNamespacePerms.size() == 0)
+          zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + tableNamespace, NodeMissingPolicy.SKIP);
+        else
+          zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + tableNamespace, ZKSecurityTool.convertTableNamespacePermissions(tableNamespacePerms),
+              NodeExistsPolicy.OVERWRITE);
+      }
+    } catch (KeeperException e) {
+      log.error(e, e);
+      throw new AccumuloSecurityException(user, SecurityErrorCode.CONNECTION_ERROR, e);
+    } catch (InterruptedException e) {
+      log.error(e, e);
+      throw new RuntimeException(e);
+    }
+  }
+  
+  @Override
   public void cleanTablePermissions(String table) throws AccumuloSecurityException {
     try {
       synchronized (zooCache) {
@@ -245,6 +353,24 @@ public class ZKPermHandler implements PermissionHandler {
   }
   
   @Override
+  public void cleanTableNamespacePermissions(String tableNamespace) 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);
+      }
+    } catch (KeeperException e) {
+      log.error(e, e);
+      throw new AccumuloSecurityException("unknownUser", SecurityErrorCode.CONNECTION_ERROR, e);
+    } catch (InterruptedException e) {
+      log.error(e, e);
+      throw new RuntimeException(e);
+    }
+  }
+  
+  @Override
   public void initializeSecurity(TCredentials itw, String rootuser) throws AccumuloSecurityException {
     IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
     
@@ -285,6 +411,7 @@ public class ZKPermHandler implements PermissionHandler {
     try {
       zoo.putPersistentData(ZKUserPath + "/" + user, new byte[0], NodeExistsPolicy.SKIP);
       zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserTablePerms, new byte[0], NodeExistsPolicy.SKIP);
+      zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserNamespacePerms, new byte[0], NodeExistsPolicy.SKIP);
     } catch (KeeperException e) {
       log.error(e, e);
       throw new AccumuloSecurityException(user, SecurityErrorCode.CONNECTION_ERROR, e);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/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 3b9d8b2..ce62b7e 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,6 +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.TablePermission;
 import org.apache.log4j.Logger;
 
@@ -149,6 +150,26 @@ class ZKSecurityTool {
     return toReturn;
   }
   
+  public static byte[] convertTableNamespacePermissions(Set<TableNamespacePermission> namespacepermissions) {
+    ByteArrayOutputStream bytes = new ByteArrayOutputStream(namespacepermissions.size());
+    DataOutputStream out = new DataOutputStream(bytes);
+    try {
+      for (TableNamespacePermission tnp : namespacepermissions)
+        out.writeByte(tnp.getId());
+    } catch (IOException e) {
+      log.error(e, e);
+      throw new RuntimeException(e); // this is impossible with ByteArrayOutputStream; crash hard if this happens
+    }
+    return bytes.toByteArray();
+  }
+  
+  public static Set<TableNamespacePermission> convertTableNamespacePermissions(byte[] namespacepermissions) {
+    Set<TableNamespacePermission> toReturn = new HashSet<TableNamespacePermission>();
+    for (byte b : namespacepermissions)
+      toReturn.add(TableNamespacePermission.getPermissionById(b));
+    return toReturn;
+  }
+  
   public static String getInstancePath(String instanceId) {
     return Constants.ZROOT + "/" + instanceId;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/test/src/main/java/org/apache/accumulo/test/randomwalk/security/WalkingSecurity.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/security/WalkingSecurity.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/security/WalkingSecurity.java
index 0ddb752..08fd395 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/security/WalkingSecurity.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/security/WalkingSecurity.java
@@ -27,6 +27,7 @@ import java.util.Set;
 import java.util.TreeSet;
 
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
@@ -35,6 +36,7 @@ import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.Credentials;
 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.core.util.CachedConfiguration;
@@ -59,6 +61,7 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
   private static final String userPass = "UserPass";
   private static final String userExists = "UserExists";
   private static final String tableExists = "TableExists";
+  private static final String tableNamespaceExists = "TableNamespaceExists";
   
   private static final String connector = "UserConnection";
   
@@ -205,6 +208,16 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
   }
   
   @Override
+  public boolean hasTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws AccumuloSecurityException, TableNamespaceNotFoundException {
+    return Boolean.parseBoolean(state.getString("Nsp-" + user + '-' + permission.name()));
+  }
+  
+  @Override
+  public boolean hasCachedTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws AccumuloSecurityException, TableNamespaceNotFoundException {
+    return hasTableNamespacePermission(user, tableNamespace, permission);
+  }
+  
+  @Override
   public void grantSystemPermission(String user, SystemPermission permission) throws AccumuloSecurityException {
     setSysPerm(state, user, permission, true);
   }
@@ -239,6 +252,25 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
   }
   
   @Override
+  public void grantTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws AccumuloSecurityException, TableNamespaceNotFoundException {
+    setNspPerm(state, user, permission, tableNamespace, true);
+  }
+  
+  private void setNspPerm(State state, String userName, TableNamespacePermission tnp, String tableNamespace, boolean value) {
+    if (tableNamespace.equals(userName))
+      throw new RuntimeException("I don't even know");
+    log.debug((value ? "Gave" : "Took") + " the table permission " + tnp.name() + (value ? " to" : " from") + " user " + userName);
+    state.set("Nsp-" + userName + '-' + tnp.name(), Boolean.toString(value));
+    if (tnp.equals(TableNamespacePermission.READ) || tnp.equals(TableNamespacePermission.WRITE))
+      state.set("Nsp-" + userName + '-' + tnp.name() + '-' + "time", System.currentTimeMillis());
+  }
+  
+  @Override
+  public void revokeTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws AccumuloSecurityException, TableNamespaceNotFoundException {
+    setNspPerm(state, user, permission, tableNamespace, false);
+  }
+  
+  @Override
   public void cleanTablePermissions(String table) throws AccumuloSecurityException, TableNotFoundException {
     for (String user : new String[] {getSysUserName(), getTabUserName()}) {
       for (TablePermission tp : TablePermission.values()) {
@@ -249,6 +281,16 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
   }
   
   @Override
+  public void cleanTableNamespacePermissions(String tableNamespace) throws AccumuloSecurityException, TableNamespaceNotFoundException {
+    for (String user : new String[] {getSysUserName(), getNspUserName()}) {
+      for (TableNamespacePermission tnp : TableNamespacePermission.values()) {
+        revokeTableNamespacePermission(user, tableNamespace, tnp);
+      }
+    }
+    state.set(tableNamespaceExists, Boolean.toString(false));
+  }
+  
+  @Override
   public void cleanUser(String user) throws AccumuloSecurityException {
     if (getTableExists())
       for (TablePermission tp : TablePermission.values())
@@ -267,11 +309,20 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
     return state.getString("system" + userName);
   }
   
+  public String getNspUserName() {
+    return state.getString("namespace" + userName);
+  }
+  
   public void setTabUserName(String name) {
     state.set("table" + userName, name);
     state.set(name + userExists, Boolean.toString(false));
   }
   
+  public void setNspUserName(String name) {
+    state.set("namespace" + userName, name);
+    state.set(name + userExists, Boolean.toString(false));
+  }
+  
   public void setSysUserName(String name) {
     state.set("system" + userName, name);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java b/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
index 584991e..b779152 100644
--- a/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
@@ -30,6 +30,7 @@ import java.util.Set;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
@@ -42,6 +43,7 @@ import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.TableNamespaces;
 import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -49,6 +51,7 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.Filter;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.TableNamespacePermission;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.examples.simple.constraints.NumericValueConstraint;
 import org.apache.accumulo.minicluster.MiniAccumuloCluster;
@@ -404,12 +407,12 @@ public class TableNamespacesIT {
     } catch (MutationsRejectedException e) {
       // supposed to be thrown
     }
-    
     int num = c.tableNamespaceOperations().listConstraints(namespace).get(NumericValueConstraint.class.getName());
     c.tableNamespaceOperations().removeConstraint(namespace, num);
   }
   
   /**
+<<<<<<< HEAD
    * Tests that when a table moves to a new namespace that it's properties inherit from the new namespace and not the old one
    */
   @Test
@@ -447,6 +450,38 @@ public class TableNamespacesIT {
     }
     assertTrue(!hasProp);
   }
+  /**
+   *  Tests new Namespace permissions as well as modifications to Table permissions because of namespaces 
+   */
+  @Test
+  public void testPermissions() throws Exception {
+    Connector c = accumulo.getConnector("root", secret);
+    
+    PasswordToken pass = new PasswordToken(secret);
+    
+    String n1 = "namespace1";
+
+    String user1 = "dude";
+
+    c.tableNamespaceOperations().create(n1);
+    c.tableOperations().create(n1 + ".table1");
+    
+    c.securityOperations().createLocalUser(user1, pass);
+    
+    Connector user1Con = accumulo.getConnector(user1, secret);
+    
+    try {
+      user1Con.tableOperations().create(n1 + ".table2");
+      fail();
+    } catch (AccumuloSecurityException e) {
+      // supposed to happen
+    }
+    
+    c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.CREATE_TABLE);
+    
+    user1Con.tableOperations().create(n1 + ".table2");
+    assertTrue(c.tableOperations().list().contains(n1 + ".table2"));
+  }
   
   private boolean checkTableHasProp(Connector c, String t, String propKey, String propVal) throws AccumuloException, TableNotFoundException {
     for (Entry<String,String> e : c.tableOperations().getProperties(t)) {


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

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
new file mode 100644
index 0000000..9036845
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java
@@ -0,0 +1,589 @@
+/*
+ * 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.core.client.admin;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import jline.internal.Log;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.NamespaceExistsException;
+import org.apache.accumulo.core.client.NamespaceNotEmptyException;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.TableOfflineException;
+import org.apache.accumulo.core.client.impl.ClientExec;
+import org.apache.accumulo.core.client.impl.ClientExecReturn;
+import org.apache.accumulo.core.client.impl.MasterClient;
+import org.apache.accumulo.core.client.impl.Namespaces;
+import org.apache.accumulo.core.client.impl.ServerClient;
+import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.core.client.impl.thrift.ClientService;
+import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+import org.apache.accumulo.core.constraints.Constraint;
+import org.apache.accumulo.core.iterators.IteratorUtil;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.master.thrift.MasterClientService;
+import org.apache.accumulo.core.master.thrift.TableOperation;
+import org.apache.accumulo.core.security.Credentials;
+import org.apache.accumulo.core.util.ArgumentChecker;
+import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.core.util.OpTimer;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.trace.instrument.Tracer;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransportException;
+
+/**
+ * Provides a class for administering namespaces
+ * 
+ */
+public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
+  private Instance instance;
+  private Credentials credentials;
+
+  private static final Logger log = Logger.getLogger(TableOperations.class);
+
+  /**
+   * @param instance
+   *          the connection information for this instance
+   * @param credentials
+   *          the username/password for this connection
+   */
+  public NamespaceOperationsImpl(Instance instance, Credentials credentials) {
+    ArgumentChecker.notNull(instance, credentials);
+    this.instance = instance;
+    this.credentials = credentials;
+  }
+
+  /**
+   * Retrieve a list of namespaces in Accumulo.
+   * 
+   * @return List of namespaces in accumulo
+   */
+  @Override
+  public SortedSet<String> list() {
+    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Fetching list of namespaces...");
+    TreeSet<String> namespaces = new TreeSet<String>(Namespaces.getNameToIdMap(instance).keySet());
+    opTimer.stop("Fetched " + namespaces.size() + " namespaces in %DURATION%");
+    return namespaces;
+  }
+
+  /**
+   * A method to check if a namespace exists in Accumulo.
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @return true if the namespace exists
+   */
+  @Override
+  public boolean exists(String namespace) {
+    ArgumentChecker.notNull(namespace);
+
+    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Checking if namespace " + namespace + " exists...");
+    boolean exists = Namespaces.getNameToIdMap(instance).containsKey(namespace);
+    opTimer.stop("Checked existance of " + exists + " in %DURATION%");
+    return exists;
+  }
+
+  /**
+   * Create a namespace with no special configuration
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   * @throws NamespaceExistsException
+   *           if the namespace already exists
+   */
+  @Override
+  public void create(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException {
+    create(namespace, true, TimeType.MILLIS);
+  }
+
+  /**
+   * @param namespace
+   *          the name of the namespace
+   * @param limitVersion
+   *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
+   */
+  @Override
+  public void create(String namespace, boolean limitVersion) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException {
+    create(namespace, limitVersion, TimeType.MILLIS);
+  }
+
+  /**
+   * @param namespace
+   *          the name of the namespace
+   * @param timeType
+   *          specifies logical or real-time based time recording for entries in the table
+   * @param limitVersion
+   *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
+   */
+  @Override
+  public void create(String namespace, boolean limitVersion, TimeType timeType) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException {
+    ArgumentChecker.notNull(namespace, timeType);
+
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(namespace.getBytes()), ByteBuffer.wrap(timeType.name().getBytes()));
+
+    Map<String,String> opts = IteratorUtil.generateInitialTableProperties(limitVersion);
+
+    try {
+      doNamespaceOperation(TableOperation.CREATE, args, opts);
+    } catch (NamespaceNotFoundException e1) {
+      // should not happen
+      throw new RuntimeException(e1);
+    }
+  }
+
+  private long beginNamespaceOperation() throws ThriftSecurityException, TException {
+    while (true) {
+      MasterClientService.Iface client = null;
+      try {
+        client = MasterClient.getConnectionWithRetry(instance);
+        return client.beginNamespaceOperation(Tracer.traceInfo(), credentials.toThrift(instance));
+      } catch (TTransportException tte) {
+        log.debug("Failed to call beginNamespaceOperation(), retrying ... ", tte);
+        UtilWaitThread.sleep(100);
+      } finally {
+        MasterClient.close(client);
+      }
+    }
+  }
+
+  private void executeNamespaceOperation(long opid, TableOperation op, List<ByteBuffer> args, Map<String,String> opts, boolean autoCleanUp)
+      throws ThriftSecurityException, TException, ThriftTableOperationException {
+    while (true) {
+      MasterClientService.Iface client = null;
+      try {
+        client = MasterClient.getConnectionWithRetry(instance);
+        client.executeNamespaceOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid, op, args, opts, autoCleanUp);
+        break;
+      } catch (TTransportException tte) {
+        log.debug("Failed to call executeTableOperation(), retrying ... ", tte);
+        UtilWaitThread.sleep(100);
+      } finally {
+        MasterClient.close(client);
+      }
+    }
+  }
+
+  private String waitForNamespaceOperation(long opid) throws ThriftSecurityException, TException, ThriftTableOperationException {
+    while (true) {
+      MasterClientService.Iface client = null;
+      try {
+        client = MasterClient.getConnectionWithRetry(instance);
+        return client.waitForNamespaceOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid);
+      } catch (TTransportException tte) {
+        log.debug("Failed to call waitForTableOperation(), retrying ... ", tte);
+        UtilWaitThread.sleep(100);
+      } finally {
+        MasterClient.close(client);
+      }
+    }
+  }
+
+  private void finishNamespaceOperation(long opid) throws ThriftSecurityException, TException {
+    while (true) {
+      MasterClientService.Iface client = null;
+      try {
+        client = MasterClient.getConnectionWithRetry(instance);
+        client.finishNamespaceOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid);
+        break;
+      } catch (TTransportException tte) {
+        log.debug("Failed to call finishTableOperation(), retrying ... ", tte);
+        UtilWaitThread.sleep(100);
+      } finally {
+        MasterClient.close(client);
+      }
+    }
+  }
+
+  private String doNamespaceOperation(TableOperation op, List<ByteBuffer> args, Map<String,String> opts) throws AccumuloSecurityException,
+      NamespaceExistsException, NamespaceNotFoundException, AccumuloException {
+    return doNamespaceOperation(op, args, opts, true);
+  }
+
+  private String doNamespaceOperation(TableOperation op, List<ByteBuffer> args, Map<String,String> opts, boolean wait) throws AccumuloSecurityException,
+      NamespaceExistsException, NamespaceNotFoundException, AccumuloException {
+    Long opid = null;
+
+    try {
+      opid = beginNamespaceOperation();
+      executeNamespaceOperation(opid, op, args, opts, !wait);
+      if (!wait) {
+        opid = null;
+        return null;
+      }
+      String ret = waitForNamespaceOperation(opid);
+      Tables.clearCache(instance);
+      return ret;
+    } catch (ThriftSecurityException e) {
+      String tableName = ByteBufferUtil.toString(args.get(0));
+      String tableInfo = Tables.getPrintableTableInfoFromName(instance, tableName);
+      throw new AccumuloSecurityException(e.user, e.code, tableInfo, e);
+    } catch (ThriftTableOperationException e) {
+      switch (e.getType()) {
+        case EXISTS:
+          throw new NamespaceExistsException(e);
+        case NOTFOUND:
+          throw new NamespaceNotFoundException(e);
+        case OFFLINE:
+          throw new TableOfflineException(instance, null);
+        case OTHER:
+        default:
+          throw new AccumuloException(e.description, e);
+      }
+    } catch (Exception e) {
+      throw new AccumuloException(e.getMessage(), e);
+    } finally {
+      // always finish table op, even when exception
+      if (opid != null)
+        try {
+          finishNamespaceOperation(opid);
+        } catch (Exception e) {
+          log.warn(e.getMessage(), e);
+        }
+    }
+  }
+
+  /**
+   * Delete a namespace if empty
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   * @throws NamespaceNotFoundException
+   *           if the namespace does not exist
+   * @throws NamespaceNotEmptyException
+   *           if the namespaces still contains tables
+   * @throws TableNotFoundException
+   *           if table not found while deleting
+   */
+  @Override
+  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException, NamespaceNotEmptyException,
+      TableNotFoundException {
+    delete(namespace, false);
+  }
+
+  /**
+   * Delete a namespace
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @param deleteTables
+   *          boolean, if true deletes all the tables in the namespace in addition to deleting the namespace.
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   * @throws NamespaceNotFoundException
+   *           if the namespace does not exist
+   * @throws NamespaceNotEmptyException
+   *           if the namespaces still contains tables
+   * @throws TableNotFoundException
+   *           if table not found while deleting
+   */
+  @Override
+  public void delete(String namespace, boolean deleteTables) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException,
+      NamespaceNotEmptyException, TableNotFoundException {
+    ArgumentChecker.notNull(namespace);
+    String namespaceId = Namespaces.getNamespaceId(instance, namespace);
+
+    if (namespaceId.equals(Constants.SYSTEM_NAMESPACE_ID) || namespaceId.equals(Constants.DEFAULT_NAMESPACE_ID)) {
+      log.debug(credentials.getPrincipal() + " attempted to delete the " + namespaceId + " namespace");
+      throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.UNSUPPORTED_OPERATION);
+    }
+
+    if (Namespaces.getTableIds(instance, namespaceId).size() > 0) {
+      if (!deleteTables) {
+        throw new NamespaceNotEmptyException(namespaceId, namespace, null);
+      }
+      for (String table : Namespaces.getTableNames(instance, namespaceId)) {
+        try {
+          getTableOperations().delete(table);
+        } catch (TableNotFoundException e) {
+          log.debug("Table (" + table + ") not found while deleting namespace, probably deleted while we were deleting the rest of the tables");
+        }
+      }
+    }
+
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(namespace.getBytes()));
+    Map<String,String> opts = new HashMap<String,String>();
+
+    try {
+      doNamespaceOperation(TableOperation.DELETE, args, opts);
+    } catch (NamespaceExistsException e) {
+      // should not happen
+      throw new RuntimeException(e);
+    }
+
+  }
+
+  /**
+   * Rename a namespace
+   * 
+   * @param oldNamespaceName
+   *          the old namespace
+   * @param newNamespaceName
+   *          the new namespace
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   * @throws NamespaceNotFoundException
+   *           if the old namespace name does not exist
+   * @throws NamespaceExistsException
+   *           if the new namespace name already exists
+   */
+  @Override
+  public void rename(String oldNamespaceName, String newNamespaceName) throws AccumuloSecurityException, NamespaceNotFoundException, AccumuloException,
+      NamespaceExistsException {
+
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldNamespaceName.getBytes()), ByteBuffer.wrap(newNamespaceName.getBytes()));
+    Map<String,String> opts = new HashMap<String,String>();
+    doNamespaceOperation(TableOperation.RENAME, args, opts);
+  }
+
+  /**
+   * Sets a property on a namespace which will apply to all tables in the namespace
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @param property
+   *          the name of a per-table property
+   * @param value
+   *          the value to set a per-table property to
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   */
+  @Override
+  public void setProperty(final String namespace, final String property, final String value) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(namespace, property, value);
+
+    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
+      @Override
+      public void execute(MasterClientService.Client client) throws Exception {
+        client.setNamespaceProperty(Tracer.traceInfo(), credentials.toThrift(instance), namespace, property, value);
+      }
+    });
+  }
+
+  /**
+   * Removes a property from a namespace
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @param property
+   *          the name of a per-table property
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission
+   */
+  @Override
+  public void removeProperty(final String namespace, final String property) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(namespace, property);
+
+    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
+      @Override
+      public void execute(MasterClientService.Client client) throws Exception {
+        client.removeNamespaceProperty(Tracer.traceInfo(), credentials.toThrift(instance), namespace, property);
+      }
+    });
+  }
+
+  /**
+   * Gets properties of a namespace
+   * 
+   * @param namespace
+   *          the name of the namespace
+   * @return all properties visible by this namespace (system and per-namespace properties)
+   * @throws NamespaceNotFoundException
+   *           if the namespace does not exist
+   */
+  @Override
+  public Iterable<Entry<String,String>> getProperties(final String namespace) throws AccumuloException, NamespaceNotFoundException {
+    ArgumentChecker.notNull(namespace);
+    try {
+      return ServerClient.executeRaw(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
+        @Override
+        public Map<String,String> execute(ClientService.Client client) throws Exception {
+          return client.getNamespaceConfiguration(Tracer.traceInfo(), credentials.toThrift(instance), namespace);
+        }
+      }).entrySet();
+    } catch (ThriftTableOperationException e) {
+      switch (e.getType()) {
+        case NOTFOUND:
+          throw new NamespaceNotFoundException(e);
+        case OTHER:
+        default:
+          throw new AccumuloException(e.description, e);
+      }
+    } catch (AccumuloException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new AccumuloException(e);
+    }
+
+  }
+
+  /**
+   * 
+   * @param namespace
+   *          the namespace to take offline
+   * @throws AccumuloException
+   *           when there is a general accumulo error
+   * @throws AccumuloSecurityException
+   *           when the user does not have the proper permissions
+   * @throws NamespaceNotFoundException
+   *           if the namespace does not exist
+   */
+  @Override
+  public void offline(String namespace) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException {
+
+    ArgumentChecker.notNull(namespace);
+    String namespaceId = Namespaces.getNamespaceId(instance, namespace);
+    try {
+      for (String table : Namespaces.getTableNames(instance, namespaceId)) {
+        getTableOperations().offline(table);
+      }
+    } catch (TableNotFoundException e) {
+      Log.error("Namespace (" + namespaceId + ") contains reference to table that doesn't exist");
+    }
+  }
+
+  /**
+   * 
+   * @param namespace
+   *          the namespace to take online
+   * @throws AccumuloException
+   *           when there is a general accumulo error
+   * @throws AccumuloSecurityException
+   *           when the user does not have the proper permissions
+   * @throws NamespaceNotFoundException
+   *           if the namespace does not exist
+   */
+  @Override
+  public void online(String namespace) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException {
+    ArgumentChecker.notNull(namespace);
+    String namespaceId = Namespaces.getNamespaceId(instance, namespace);
+    try {
+      for (String table : Namespaces.getTableNames(instance, namespaceId)) {
+        getTableOperations().online(table);
+      }
+    } catch (TableNotFoundException e) {
+      Log.warn("Namespace (" + namespaceId + ") contains a reference to a table that doesn't exist");
+    }
+  }
+
+  /**
+   * Get a mapping of namespace name to internal namespace id.
+   * 
+   * @return the map from namespace name to internal namespace id
+   */
+  @Override
+  public Map<String,String> namespaceIdMap() {
+    return Namespaces.getNameToIdMap(instance);
+  }
+
+  @Override
+  public List<DiskUsage> getDiskUsage(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
+    Set<String> tables = new HashSet<String>();
+    String namespaceId = Namespaces.getNamespaceId(instance, namespace);
+    tables.addAll(Namespaces.getTableNames(instance, namespaceId));
+    List<DiskUsage> du = null;
+    try {
+      du = getTableOperations().getDiskUsage(tables);
+    } catch (TableNotFoundException e) {
+      log.warn("Could not find table (" + e.getTableName() + ") reference in namespace (" + namespace + ")");
+    }
+    return du;
+  }
+
+  private TableOperations getTableOperations() throws AccumuloException, AccumuloSecurityException {
+    return new TableOperationsImpl(instance, credentials);
+  }
+
+  @Override
+  public void attachIterator(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+      NamespaceNotFoundException {
+    testClassLoad(namespace, setting.getIteratorClass(), SortedKeyValueIterator.class.getName());
+    super.attachIterator(namespace, setting, scopes);
+  }
+
+  @Override
+  public int addConstraint(String namespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException {
+    testClassLoad(namespace, constraintClassName, Constraint.class.getName());
+    return super.addConstraint(namespace, constraintClassName);
+  }
+
+  @Override
+  public boolean testClassLoad(final String namespace, final String className, final String asTypeName) throws NamespaceNotFoundException, AccumuloException,
+      AccumuloSecurityException {
+    ArgumentChecker.notNull(namespace, className, asTypeName);
+
+    try {
+      return ServerClient.executeRaw(instance, new ClientExecReturn<Boolean,ClientService.Client>() {
+        @Override
+        public Boolean execute(ClientService.Client client) throws Exception {
+          return client.checkNamespaceClass(Tracer.traceInfo(), credentials.toThrift(instance), namespace, className, asTypeName);
+        }
+      });
+    } catch (ThriftTableOperationException e) {
+      switch (e.getType()) {
+        case NOTFOUND:
+          throw new NamespaceNotFoundException(e);
+        case OTHER:
+        default:
+          throw new AccumuloException(e.description, e);
+      }
+    } catch (ThriftSecurityException e) {
+      throw new AccumuloSecurityException(e.user, e.code, e);
+    } catch (AccumuloException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new AccumuloException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java
index 8a93efa..b3178fd 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java
@@ -23,15 +23,15 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.security.Authorizations;
+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;
 
 /**
  * Provides a class for managing users and permissions
  */
 public interface SecurityOperations {
-  
+
   /**
    * Create a user
    * 
@@ -49,7 +49,7 @@ public interface SecurityOperations {
    */
   @Deprecated
   public void createUser(String user, byte[] password, Authorizations authorizations) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * Create a user
    * 
@@ -64,7 +64,7 @@ public interface SecurityOperations {
    * @since 1.5.0
    */
   public void createLocalUser(String principal, PasswordToken password) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * Delete a user
    * 
@@ -78,7 +78,7 @@ public interface SecurityOperations {
    */
   @Deprecated
   public void dropUser(String user) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * Delete a user
    * 
@@ -91,7 +91,7 @@ public interface SecurityOperations {
    * @since 1.5.0
    */
   public void dropLocalUser(String principal) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * Verify a username/password combination is valid
    * 
@@ -108,7 +108,7 @@ public interface SecurityOperations {
    */
   @Deprecated
   public boolean authenticateUser(String user, byte[] password) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * Verify a username/password combination is valid
    * 
@@ -124,7 +124,7 @@ public interface SecurityOperations {
    * @since 1.5.0
    */
   public boolean authenticateUser(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * Set the user's password
    * 
@@ -141,7 +141,7 @@ public interface SecurityOperations {
    */
   @Deprecated
   public void changeUserPassword(String user, byte[] password) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * Set the user's password
    * 
@@ -156,7 +156,7 @@ public interface SecurityOperations {
    * @since 1.5.0
    */
   public void changeLocalUserPassword(String principal, PasswordToken token) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * Set the user's record-level authorizations
    * 
@@ -170,7 +170,7 @@ public interface SecurityOperations {
    *           if the user does not have permission to modify a user
    */
   public void changeUserAuthorizations(String principal, Authorizations authorizations) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * Retrieves the user's authorizations for scanning
    * 
@@ -183,7 +183,7 @@ public interface SecurityOperations {
    *           if the user does not have permission to query a user
    */
   public Authorizations getUserAuthorizations(String principal) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * Verify the user has a particular system permission
    * 
@@ -198,7 +198,7 @@ public interface SecurityOperations {
    *           if the user does not have permission to query a user
    */
   public boolean hasSystemPermission(String principal, SystemPermission perm) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * Verify the user has a particular table permission
    * 
@@ -215,24 +215,24 @@ public interface SecurityOperations {
    *           if the user does not have permission to query a user
    */
   public boolean hasTablePermission(String principal, String table, TablePermission perm) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
-   * Verify the user has a particular table namespace permission
+   * Verify the user has a particular namespace permission
    * 
    * @param principal
    *          the name of the user to query
-   * @param tableNamespace
-   *          the name of the table namespace to query about
+   * @param namespace
+   *          the name of the namespace to query about
    * @param perm
-   *          the table namespace permission to check for
+   *          the namespace permission to check for
    * @return true if user has that permission; false otherwise
    * @throws AccumuloException
    *           if a general error occurs
    * @throws AccumuloSecurityException
    *           if the user does not have permission to query a user
    */
-  public boolean hasTableNamespacePermission(String principal, String tableNamespace, TableNamespacePermission perm) throws AccumuloException, AccumuloSecurityException;
-  
+  public boolean hasNamespacePermission(String principal, String namespace, NamespacePermission perm) throws AccumuloException, AccumuloSecurityException;
+
   /**
    * Grant a user a system permission
    * 
@@ -246,7 +246,7 @@ public interface SecurityOperations {
    *           if the user does not have permission to grant a user permissions
    */
   public void grantSystemPermission(String principal, SystemPermission permission) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * Grant a user a specific permission for a specific table
    * 
@@ -262,23 +262,23 @@ public interface SecurityOperations {
    *           if the user does not have permission to grant a user permissions
    */
   public void grantTablePermission(String principal, String table, TablePermission permission) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
-   * Grant a user a specific permission for a specific table namespace
+   * Grant a user a specific permission for a specific namespace
    * 
    * @param principal
    *          the name of the user to modify
-   * @param tableNamespace
-   *          the name of the table namespace to modify for the user
+   * @param namespace
+   *          the name of the namespace to modify for the user
    * @param permission
-   *          the table namespace permission to grant to the user
+   *          the namespace permission to grant to the user
    * @throws AccumuloException
    *           if a general error occurs
    * @throws AccumuloSecurityException
    *           if the user does not have permission to grant a user permissions
    */
-  public void grantTableNamespacePermission(String principal, String tableNamespace, TableNamespacePermission permission) throws AccumuloException, AccumuloSecurityException;
-  
+  public void grantNamespacePermission(String principal, String namespace, NamespacePermission permission) throws AccumuloException, AccumuloSecurityException;
+
   /**
    * Revoke a system permission from a user
    * 
@@ -292,7 +292,7 @@ public interface SecurityOperations {
    *           if the user does not have permission to revoke a user's permissions
    */
   public void revokeSystemPermission(String principal, SystemPermission permission) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * Revoke a table permission for a specific user on a specific table
    * 
@@ -308,23 +308,23 @@ public interface SecurityOperations {
    *           if the user does not have permission to revoke a user's permissions
    */
   public void revokeTablePermission(String principal, String table, TablePermission permission) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
-   * Revoke a table namespace permission for a specific user on a specific table namespace
+   * Revoke a namespace permission for a specific user on a specific namespace
    * 
    * @param principal
    *          the name of the user to modify
-   * @param tableNamespace
-   *          the name of the table namespace to modify for the user
+   * @param namespace
+   *          the name of the namespace to modify for the user
    * @param permission
-   *          the table namespace permission to revoke for the user
+   *          the namespace permission to revoke for the user
    * @throws AccumuloException
    *           if a general error occurs
    * @throws AccumuloSecurityException
    *           if the user does not have permission to revoke a user's permissions
    */
-  public void revokeTableNamespacePermission(String principal, String tableNamespace, TableNamespacePermission permission) throws AccumuloException, AccumuloSecurityException;
-  
+  public void revokeNamespacePermission(String principal, String namespace, NamespacePermission permission) throws AccumuloException, AccumuloSecurityException;
+
   /**
    * Return a list of users in accumulo
    * 
@@ -337,7 +337,7 @@ public interface SecurityOperations {
    */
   @Deprecated
   public Set<String> listUsers() throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * Return a list of users in accumulo
    * 
@@ -349,5 +349,5 @@ public interface SecurityOperations {
    * @since 1.5.0
    */
   public Set<String> listLocalUsers() throws AccumuloException, AccumuloSecurityException;
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
index 107ba74..ebd79ad 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
@@ -34,18 +34,18 @@ import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 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.util.ArgumentChecker;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.trace.instrument.Tracer;
 
 public class SecurityOperationsImpl implements SecurityOperations {
-  
+
   private Instance instance;
   private Credentials credentials;
-  
+
   private void execute(ClientExec<ClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
     try {
       ServerClient.executeRaw(instance, exec);
@@ -63,7 +63,7 @@ public class SecurityOperationsImpl implements SecurityOperations {
       throw new AccumuloException(e);
     }
   }
-  
+
   private <T> T execute(ClientExecReturn<T,ClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
     try {
       return ServerClient.executeRaw(instance, exec);
@@ -81,20 +81,20 @@ public class SecurityOperationsImpl implements SecurityOperations {
       throw new AccumuloException(e);
     }
   }
-  
+
   public SecurityOperationsImpl(Instance instance, Credentials credentials) {
     ArgumentChecker.notNull(instance, credentials);
     this.instance = instance;
     this.credentials = credentials;
   }
-  
+
   @Deprecated
   @Override
   public void createUser(String user, byte[] password, final Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
     createLocalUser(user, new PasswordToken(password));
     changeUserAuthorizations(user, authorizations);
   }
-  
+
   @Override
   public void createLocalUser(final String principal, final PasswordToken password) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(principal, password);
@@ -105,13 +105,13 @@ public class SecurityOperationsImpl implements SecurityOperations {
       }
     });
   }
-  
+
   @Deprecated
   @Override
   public void dropUser(final String user) throws AccumuloException, AccumuloSecurityException {
     dropLocalUser(user);
   }
-  
+
   @Override
   public void dropLocalUser(final String principal) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(principal);
@@ -122,13 +122,13 @@ public class SecurityOperationsImpl implements SecurityOperations {
       }
     });
   }
-  
+
   @Deprecated
   @Override
   public boolean authenticateUser(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
     return authenticateUser(user, new PasswordToken(password));
   }
-  
+
   @Override
   public boolean authenticateUser(final String principal, final AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(principal, token);
@@ -140,13 +140,13 @@ public class SecurityOperationsImpl implements SecurityOperations {
       }
     });
   }
-  
+
   @Override
   @Deprecated
   public void changeUserPassword(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
     changeLocalUserPassword(user, new PasswordToken(password));
   }
-  
+
   @Override
   public void changeLocalUserPassword(final String principal, final PasswordToken token) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(principal, token);
@@ -161,7 +161,7 @@ public class SecurityOperationsImpl implements SecurityOperations {
       this.credentials = toChange;
     }
   }
-  
+
   @Override
   public void changeUserAuthorizations(final String principal, final Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(principal, authorizations);
@@ -173,7 +173,7 @@ public class SecurityOperationsImpl implements SecurityOperations {
       }
     });
   }
-  
+
   @Override
   public Authorizations getUserAuthorizations(final String principal) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(principal);
@@ -184,7 +184,7 @@ public class SecurityOperationsImpl implements SecurityOperations {
       }
     });
   }
-  
+
   @Override
   public boolean hasSystemPermission(final String principal, final SystemPermission perm) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(principal, perm);
@@ -195,7 +195,7 @@ public class SecurityOperationsImpl implements SecurityOperations {
       }
     });
   }
-  
+
   @Override
   public boolean hasTablePermission(final String principal, final String table, final TablePermission perm) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(principal, table, perm);
@@ -206,18 +206,19 @@ public class SecurityOperationsImpl implements SecurityOperations {
       }
     });
   }
-  
+
   @Override
-  public boolean hasTableNamespacePermission(final String principal, final String tableNamespace, final TableNamespacePermission perm) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, tableNamespace, perm);
+  public boolean hasNamespacePermission(final String principal, final String namespace, final NamespacePermission permission) throws AccumuloException,
+      AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, namespace, permission);
     return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
       @Override
       public Boolean execute(ClientService.Client client) throws Exception {
-        return client.hasTableNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, tableNamespace, perm.getId());
+        return client.hasNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, namespace, permission.getId());
       }
     });
   }
-  
+
   @Override
   public void grantSystemPermission(final String principal, final SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(principal, permission);
@@ -228,7 +229,7 @@ public class SecurityOperationsImpl implements SecurityOperations {
       }
     });
   }
-  
+
   @Override
   public void grantTablePermission(final String principal, final String table, final TablePermission permission) throws AccumuloException,
       AccumuloSecurityException {
@@ -240,19 +241,19 @@ public class SecurityOperationsImpl implements SecurityOperations {
       }
     });
   }
-  
+
   @Override
-  public void grantTableNamespacePermission(final String principal, final String tableNamespace, final TableNamespacePermission permission) throws AccumuloException,
+  public void grantNamespacePermission(final String principal, final String namespace, final NamespacePermission permission) throws AccumuloException,
       AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, tableNamespace, permission);
+    ArgumentChecker.notNull(principal, namespace, permission);
     execute(new ClientExec<ClientService.Client>() {
       @Override
       public void execute(ClientService.Client client) throws Exception {
-        client.grantTableNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, tableNamespace, permission.getId());
+        client.grantNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, namespace, permission.getId());
       }
     });
   }
-  
+
   @Override
   public void revokeSystemPermission(final String principal, final SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(principal, permission);
@@ -263,7 +264,7 @@ public class SecurityOperationsImpl implements SecurityOperations {
       }
     });
   }
-  
+
   @Override
   public void revokeTablePermission(final String principal, final String table, final TablePermission permission) throws AccumuloException,
       AccumuloSecurityException {
@@ -275,25 +276,25 @@ public class SecurityOperationsImpl implements SecurityOperations {
       }
     });
   }
-  
+
   @Override
-  public void revokeTableNamespacePermission(final String principal, final String tableNamespace, final TableNamespacePermission permission) throws AccumuloException,
+  public void revokeNamespacePermission(final String principal, final String namespace, final NamespacePermission permission) throws AccumuloException,
       AccumuloSecurityException {
-    ArgumentChecker.notNull(principal, tableNamespace, permission);
+    ArgumentChecker.notNull(principal, namespace, permission);
     execute(new ClientExec<ClientService.Client>() {
       @Override
       public void execute(ClientService.Client client) throws Exception {
-        client.revokeTableNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, tableNamespace, permission.getId());
+        client.revokeNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, namespace, permission.getId());
       }
     });
   }
-  
+
   @Deprecated
   @Override
   public Set<String> listUsers() throws AccumuloException, AccumuloSecurityException {
     return listLocalUsers();
   }
-  
+
   @Override
   public Set<String> listLocalUsers() throws AccumuloException, AccumuloSecurityException {
     return execute(new ClientExecReturn<Set<String>,ClientService.Client>() {
@@ -303,5 +304,5 @@ public class SecurityOperationsImpl implements SecurityOperations {
       }
     });
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
deleted file mode 100644
index d966f3c..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
+++ /dev/null
@@ -1,406 +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.core.client.admin;
-
-import java.util.EnumSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.SortedSet;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.TableNamespaceExistsException;
-import org.apache.accumulo.core.client.TableNamespaceNotEmptyException;
-import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-
-/**
- * Provides a class for administering table namespaces
- * 
- */
-
-public interface TableNamespaceOperations {
-
-  /**
-   * Retrieve a list of table namespaces in Accumulo.
-   * 
-   * @return List of table namespaces in accumulo
-   */
-  public SortedSet<String> list();
-
-  /**
-   * A method to check if a table namespace exists in Accumulo.
-   * 
-   * @param namespace
-   *          the name of the table namespace
-   * @return true if the table namespace exists
-   */
-  public boolean exists(String namespace);
-
-  /**
-   * Create a table namespace with no special configuration
-   * 
-   * @param namespace
-   *          the name of the table namespace
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableNamespaceExistsException
-   *           if the table namespace already exists
-   */
-  public void create(String namespace) throws AccumuloException, AccumuloSecurityException, TableNamespaceExistsException;
-
-  /**
-   * @param namespace
-   *          the name of the table namespace
-   * @param limitVersion
-   *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableNamespaceExistsException
-   *           if the table namespace already exists
-   */
-  public void create(String namespace, boolean limitVersion) throws AccumuloException, AccumuloSecurityException, TableNamespaceExistsException;
-
-  /**
-   * @param namespace
-   *          the name of the table namespace
-   * @param versioningIter
-   *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
-   * @param timeType
-   *          specifies logical or real-time based time recording for entries in the table
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableNamespaceExistsException
-   *           if the table namespace already exists
-   */
-  public void create(String namespace, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException,
-      TableNamespaceExistsException;
-
-  /**
-   * Delete a table namespace if it is empty
-   * 
-   * @param namespace
-   *          the name of the table namespace
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableNamespaceNotFoundException
-   *           if the table namespace does not exist
-   * @throws TableNamespaceNotEmptyException
-   *           if the table namespaces still contains tables
-   * @throws TableNotFoundException
-   *           if table not found while deleting
-   */
-  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException, TableNamespaceNotEmptyException,
-      TableNotFoundException;
-
-  /**
-   * Delete a table namespace
-   * 
-   * @param namespace
-   *          the name of the table namespace
-   * @param deleteTables
-   *          boolean, if true deletes all the tables in the namespace in addition to deleting the namespace.
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableNamespaceNotFoundException
-   *           if the table namespace does not exist
-   * @throws TableNamespaceNotEmptyException
-   *           if the table namespaces still contains tables
-   * @throws TableNotFoundException
-   *           if table not found while deleting
-   */
-  public void delete(String namespace, boolean deleteTables) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException,
-      TableNamespaceNotEmptyException, TableNotFoundException;
-
-  /**
-   * Rename a table namespace
-   * 
-   * @param oldNamespaceName
-   *          the old table namespace name
-   * @param newNamespaceName
-   *          the new table namespace name
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableNamespaceNotFoundException
-   *           if the old table namespace does not exist
-   * @throws TableNamespaceExistsException
-   *           if the new table namespace already exists
-   */
-  public void rename(String oldNamespaceName, String newNamespaceName) throws AccumuloSecurityException, TableNamespaceNotFoundException, AccumuloException,
-      TableNamespaceExistsException;
-
-  /**
-   * Sets a property on a table namespace which applies to all tables in the namespace. Note that it may take a short period of time (a second) to propagate the
-   * change everywhere.
-   * 
-   * @param namespace
-   *          the name of the table namespace
-   * @param property
-   *          the name of a per-table property
-   * @param value
-   *          the value to set a per-table property to
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   */
-  public void setProperty(String namespace, String property, String value) throws AccumuloException, AccumuloSecurityException;
-
-  /**
-   * Removes a property from a table namespace. Note that it may take a short period of time (a second) to propagate the change everywhere.
-   * 
-   * @param namespace
-   *          the name of the table namespace
-   * @param property
-   *          the name of a per-table property
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   */
-  public void removeProperty(String namespace, String property) throws AccumuloException, AccumuloSecurityException;
-
-  /**
-   * Gets properties of a table namespace. Note that recently changed properties may not be available immediately.
-   * 
-   * @param namespace
-   *          the name of the table namespace
-   * @return all properties visible by this table (system and per-table properties). Note that recently changed properties may not be visible immediately.
-   * @throws TableNamespaceNotFoundException
-   *           if the table namespace does not exist
-   */
-  public Iterable<Entry<String,String>> getProperties(String namespace) throws AccumuloException, TableNamespaceNotFoundException;
-
-  /**
-   * 
-   * @param namespace
-   *          the table namespace to take offline
-   * @throws AccumuloException
-   *           when there is a general accumulo error
-   * @throws AccumuloSecurityException
-   *           when the user does not have the proper permissions
-   * @throws TableNamespaceNotFoundException
-   *           if the table namespace does not exist
-   */
-  public void offline(String namespace) throws AccumuloSecurityException, AccumuloException, TableNamespaceNotFoundException;
-
-  /**
-   * 
-   * @param namespace
-   *          the table namespace to take online
-   * @throws AccumuloException
-   *           when there is a general accumulo error
-   * @throws AccumuloSecurityException
-   *           when the user does not have the proper permissions
-   * @throws TableNamespaceNotFoundException
-   *           if the table namespace does not exist
-   */
-  public void online(String namespace) throws AccumuloSecurityException, AccumuloException, TableNamespaceNotFoundException;
-
-  /**
-   * Get a mapping of table namespace name to internal table namespace id.
-   * 
-   * @return the map from table namespace name to internal table namespace id
-   */
-  public Map<String,String> namespaceIdMap();
-
-  /**
-   * Gets the number of bytes being used in the files for the set of tables in this namespace
-   * 
-   * @param namespace
-   *          the table namespace to get the set of tables from
-   * 
-   * @return a list of disk usage objects containing linked table names and sizes
-   * @throws AccumuloException
-   *           when there is a general accumulo error
-   * @throws AccumuloSecurityException
-   *           when the user does not have the proper permissions
-   */
-  public List<DiskUsage> getDiskUsage(String namespace) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException;
-
-  /**
-   * Add an iterator to a table namespace on all scopes.
-   * 
-   * @param tableNamespace
-   *          the name of the table namespace
-   * @param setting
-   *          object specifying the properties of the iterator
-   * @throws AccumuloSecurityException
-   *           thrown if the user does not have the ability to set properties on the table namespace
-   * @throws AccumuloException
-   * @throws TableNamespaceNotFoundException
-   *           throw if the table namespace no longer exists
-   * @throws IllegalArgumentException
-   *           if the setting conflicts with any existing iterators
-   */
-  public void attachIterator(String tableNamespace, IteratorSetting setting) throws AccumuloSecurityException, AccumuloException,
-      TableNamespaceNotFoundException;
-
-  /**
-   * Add an iterator to a table namespace on the given scopes.
-   * 
-   * @param tableNamespace
-   *          the name of the table namespace
-   * @param setting
-   *          object specifying the properties of the iterator
-   * @throws AccumuloSecurityException
-   *           thrown if the user does not have the ability to set properties on the table namespace
-   * @throws AccumuloException
-   * @throws TableNamespaceNotFoundException
-   *           throw if the table namespace no longer exists
-   * @throws IllegalArgumentException
-   *           if the setting conflicts with any existing iterators
-   */
-  public void attachIterator(String tableNamespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException,
-      AccumuloException, TableNamespaceNotFoundException;
-
-  /**
-   * Remove an iterator from a table namespace by name.
-   * 
-   * @param tableNamespace
-   *          the name of the table namespace
-   * @param name
-   *          the name of the iterator
-   * @param scopes
-   *          the scopes of the iterator
-   * @throws AccumuloSecurityException
-   *           thrown if the user does not have the ability to set properties on the table namespace
-   * @throws AccumuloException
-   * @throws TableNamespaceNotFoundException
-   *           thrown if the table namespace no longer exists
-   */
-  public void removeIterator(String tableNamespace, String name, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
-      TableNamespaceNotFoundException;
-
-  /**
-   * Get the settings for an iterator.
-   * 
-   * @param tableNamespace
-   *          the name of the table namespace
-   * @param name
-   *          the name of the iterator
-   * @param scope
-   *          the scope of the iterator
-   * @return the settings for this iterator
-   * @throws AccumuloSecurityException
-   *           thrown if the user does not have the ability to set properties on the table namespace
-   * @throws AccumuloException
-   * @throws TableNamespaceNotFoundException
-   *           thrown if the table namespace no longer exists
-   */
-  public IteratorSetting getIteratorSetting(String tableNamespace, String name, IteratorScope scope) throws AccumuloSecurityException, AccumuloException,
-      NumberFormatException, TableNamespaceNotFoundException;
-
-  /**
-   * Get a list of iterators for this table namespace.
-   * 
-   * @param tableNamespace
-   *          the name of the table namespace
-   * @return a set of iterator names
-   * @throws AccumuloSecurityException
-   *           thrown if the user does not have the ability to set properties on the table namespace
-   * @throws AccumuloException
-   * @throws TableNamespaceNotFoundException
-   *           thrown if the table namespace no longer exists
-   */
-  public Map<String,EnumSet<IteratorScope>> listIterators(String tableNamespace) throws AccumuloSecurityException, AccumuloException,
-      TableNamespaceNotFoundException;
-
-  /**
-   * Check whether a given iterator configuration conflicts with existing configuration; in particular, determine if the name or priority are already in use for
-   * the specified scopes.
-   * 
-   * @param tableNamespace
-   *          the name of the table namespace
-   * @param setting
-   *          object specifying the properties of the iterator
-   * @throws AccumuloException
-   * @throws TableNamespaceNotFoundException
-   *           thrown if the table namespace no longer exists
-   * @throws IllegalStateException
-   *           if the setting conflicts with any existing iterators
-   */
-  public void checkIteratorConflicts(String tableNamespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException,
-      TableNamespaceNotFoundException;
-
-  /**
-   * Add a new constraint to a table namespace.
-   * 
-   * @param tableNamespace
-   *          the name of the table namespace
-   * @param constraintClassName
-   *          the full name of the constraint class
-   * @return the unique number assigned to the constraint
-   * @throws AccumuloException
-   *           thrown if the constraint has already been added to the table or if there are errors in the configuration of existing constraints
-   * @throws AccumuloSecurityException
-   *           thrown if the user doesn't have permission to add the constraint
-   * @throws TableNamespaceNotFoundException
-   *           thrown if the table namespace no longer exists
-   */
-  public int addConstraint(String tableNamespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException,
-      TableNamespaceNotFoundException;
-
-  /**
-   * Remove a constraint from a table namespace.
-   * 
-   * @param tableNamespace
-   *          the name of the table namespace
-   * @param number
-   *          the unique number assigned to the constraint
-   * @throws AccumuloException
-   * @throws AccumuloSecurityException
-   *           thrown if the user doesn't have permission to remove the constraint
-   */
-  public void removeConstraint(String tableNamespace, int number) throws AccumuloException, AccumuloSecurityException;
-
-  /**
-   * List constraints on a table namespace with their assigned numbers.
-   * 
-   * @param tableNamespace
-   *          the name of the table namespace
-   * @return a map from constraint class name to assigned number
-   * @throws AccumuloException
-   *           thrown if there are errors in the configuration of existing constraints
-   * @throws TableNamespaceNotFoundException
-   *           thrown if the table namespace no longer exists
-   */
-  public Map<String,Integer> listConstraints(String tableNamespace) throws AccumuloException, TableNamespaceNotFoundException;
-
-  /**
-   * Test to see if the instance can load the given class as the given type. This check uses the table classpath property if it is set.
-   * 
-   * @return true if the instance can load the given class as the given type, false otherwise
-   */
-  boolean testClassLoad(String namespace, String className, String asTypeName) throws TableNamespaceNotFoundException, AccumuloException,
-      AccumuloSecurityException;
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsHelper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsHelper.java
deleted file mode 100644
index c73d8e9..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsHelper.java
+++ /dev/null
@@ -1,206 +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.core.client.admin;
-
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.TreeMap;
-import java.util.TreeSet;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-
-public abstract class TableNamespaceOperationsHelper implements TableNamespaceOperations {
-
-  @Override
-  public void attachIterator(String tableNamespace, IteratorSetting setting) throws AccumuloSecurityException, AccumuloException,
-      TableNamespaceNotFoundException {
-    attachIterator(tableNamespace, setting, EnumSet.allOf(IteratorScope.class));
-  }
-
-  @Override
-  public void attachIterator(String tableNamespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException,
-      AccumuloException, TableNamespaceNotFoundException {
-    checkIteratorConflicts(tableNamespace, setting, scopes);
-    for (IteratorScope scope : scopes) {
-      String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), setting.getName());
-      for (Entry<String,String> prop : setting.getOptions().entrySet()) {
-        this.setProperty(tableNamespace, root + ".opt." + prop.getKey(), prop.getValue());
-      }
-      this.setProperty(tableNamespace, root, setting.getPriority() + "," + setting.getIteratorClass());
-    }
-  }
-
-  @Override
-  public void removeIterator(String tableNamespace, String name, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
-      TableNamespaceNotFoundException {
-    if (!exists(tableNamespace))
-      throw new TableNamespaceNotFoundException(null, tableNamespace, null);
-    Map<String,String> copy = new TreeMap<String,String>();
-    for (Entry<String,String> property : this.getProperties(tableNamespace)) {
-      copy.put(property.getKey(), property.getValue());
-    }
-    for (IteratorScope scope : scopes) {
-      String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), name);
-      for (Entry<String,String> property : copy.entrySet()) {
-        if (property.getKey().equals(root) || property.getKey().startsWith(root + ".opt."))
-          this.removeProperty(tableNamespace, property.getKey());
-      }
-    }
-  }
-
-  @Override
-  public IteratorSetting getIteratorSetting(String tableNamespace, String name, IteratorScope scope) throws AccumuloSecurityException, AccumuloException,
-      TableNamespaceNotFoundException {
-    if (!exists(tableNamespace))
-      throw new TableNamespaceNotFoundException(null, tableNamespace, null);
-    int priority = -1;
-    String classname = null;
-    Map<String,String> settings = new HashMap<String,String>();
-
-    String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), name);
-    String opt = root + ".opt.";
-    for (Entry<String,String> property : this.getProperties(tableNamespace)) {
-      if (property.getKey().equals(root)) {
-        String parts[] = property.getValue().split(",");
-        if (parts.length != 2) {
-          throw new AccumuloException("Bad value for iterator setting: " + property.getValue());
-        }
-        priority = Integer.parseInt(parts[0]);
-        classname = parts[1];
-      } else if (property.getKey().startsWith(opt)) {
-        settings.put(property.getKey().substring(opt.length()), property.getValue());
-      }
-    }
-    if (priority <= 0 || classname == null) {
-      return null;
-    }
-    return new IteratorSetting(priority, name, classname, settings);
-  }
-
-  @Override
-  public Map<String,EnumSet<IteratorScope>> listIterators(String tableNamespace) throws AccumuloSecurityException, AccumuloException,
-      TableNamespaceNotFoundException {
-    if (!exists(tableNamespace))
-      throw new TableNamespaceNotFoundException(null, tableNamespace, null);
-    Map<String,EnumSet<IteratorScope>> result = new TreeMap<String,EnumSet<IteratorScope>>();
-    for (Entry<String,String> property : this.getProperties(tableNamespace)) {
-      String name = property.getKey();
-      String[] parts = name.split("\\.");
-      if (parts.length == 4) {
-        if (parts[0].equals("table") && parts[1].equals("iterator")) {
-          IteratorScope scope = IteratorScope.valueOf(parts[2]);
-          if (!result.containsKey(parts[3]))
-            result.put(parts[3], EnumSet.noneOf(IteratorScope.class));
-          result.get(parts[3]).add(scope);
-        }
-      }
-    }
-    return result;
-  }
-
-  @Override
-  public void checkIteratorConflicts(String tableNamespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException,
-      TableNamespaceNotFoundException {
-    if (!exists(tableNamespace))
-      throw new TableNamespaceNotFoundException(null, tableNamespace, null);
-    for (IteratorScope scope : scopes) {
-      String scopeStr = String.format("%s%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase());
-      String nameStr = String.format("%s.%s", scopeStr, setting.getName());
-      String optStr = String.format("%s.opt.", nameStr);
-      Map<String,String> optionConflicts = new TreeMap<String,String>();
-      for (Entry<String,String> property : this.getProperties(tableNamespace)) {
-        if (property.getKey().startsWith(scopeStr)) {
-          if (property.getKey().equals(nameStr))
-            throw new AccumuloException(new IllegalArgumentException("iterator name conflict for " + setting.getName() + ": " + property.getKey() + "="
-                + property.getValue()));
-          if (property.getKey().startsWith(optStr))
-            optionConflicts.put(property.getKey(), property.getValue());
-          if (property.getKey().contains(".opt."))
-            continue;
-          String parts[] = property.getValue().split(",");
-          if (parts.length != 2)
-            throw new AccumuloException("Bad value for existing iterator setting: " + property.getKey() + "=" + property.getValue());
-          try {
-            if (Integer.parseInt(parts[0]) == setting.getPriority())
-              throw new AccumuloException(new IllegalArgumentException("iterator priority conflict: " + property.getKey() + "=" + property.getValue()));
-          } catch (NumberFormatException e) {
-            throw new AccumuloException("Bad value for existing iterator setting: " + property.getKey() + "=" + property.getValue());
-          }
-        }
-      }
-      if (optionConflicts.size() > 0)
-        throw new AccumuloException(new IllegalArgumentException("iterator options conflict for " + setting.getName() + ": " + optionConflicts));
-    }
-  }
-
-  @Override
-  public int addConstraint(String tableNamespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException,
-      TableNamespaceNotFoundException {
-    TreeSet<Integer> constraintNumbers = new TreeSet<Integer>();
-    TreeMap<String,Integer> constraintClasses = new TreeMap<String,Integer>();
-    int i;
-    for (Entry<String,String> property : this.getProperties(tableNamespace)) {
-      if (property.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.toString())) {
-        try {
-          i = Integer.parseInt(property.getKey().substring(Property.TABLE_CONSTRAINT_PREFIX.toString().length()));
-        } catch (NumberFormatException e) {
-          throw new AccumuloException("Bad key for existing constraint: " + property.toString());
-        }
-        constraintNumbers.add(i);
-        constraintClasses.put(property.getValue(), i);
-      }
-    }
-    i = 1;
-    while (constraintNumbers.contains(i))
-      i++;
-    if (constraintClasses.containsKey(constraintClassName))
-      throw new AccumuloException("Constraint " + constraintClassName + " already exists for table namespace " + tableNamespace + " with number "
-          + constraintClasses.get(constraintClassName));
-    this.setProperty(tableNamespace, Property.TABLE_CONSTRAINT_PREFIX.toString() + i, constraintClassName);
-    return i;
-  }
-
-  @Override
-  public void removeConstraint(String tableNamespace, int number) throws AccumuloException, AccumuloSecurityException {
-    this.removeProperty(tableNamespace, Property.TABLE_CONSTRAINT_PREFIX.toString() + number);
-  }
-
-  @Override
-  public Map<String,Integer> listConstraints(String tableNamespace) throws AccumuloException, TableNamespaceNotFoundException {
-    Map<String,Integer> constraints = new TreeMap<String,Integer>();
-    for (Entry<String,String> property : this.getProperties(tableNamespace)) {
-      if (property.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.toString())) {
-        if (constraints.containsKey(property.getValue()))
-          throw new AccumuloException("Same constraint configured twice: " + property.getKey() + "=" + Property.TABLE_CONSTRAINT_PREFIX
-              + constraints.get(property.getValue()) + "=" + property.getKey());
-        try {
-          constraints.put(property.getValue(), Integer.parseInt(property.getKey().substring(Property.TABLE_CONSTRAINT_PREFIX.toString().length())));
-        } catch (NumberFormatException e) {
-          throw new AccumuloException("Bad key for existing constraint: " + property.toString());
-        }
-      }
-    }
-    return constraints;
-  }
-}


[18/50] [abbrv] git commit: ACCUMULO-1479 added tests for the namespace system permissions in PermissionsIT to keep it from failing, already have tests in TableNamespacesIT

Posted by ct...@apache.org.
ACCUMULO-1479 added tests for the namespace system permissions in PermissionsIT to keep it from failing, already have tests in TableNamespacesIT


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

Branch: refs/heads/master
Commit: 51f07eb9d37926bb37f9715413290e0103e5365d
Parents: 9db79fc
Author: Sean Hickey <ta...@gmail.com>
Authored: Fri Aug 9 10:19:47 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:10 2013 -0500

----------------------------------------------------------------------
 .../accumulo/test/functional/PermissionsIT.java | 93 +++++++++++++++++++-
 1 file changed, 89 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/51f07eb9/test/src/test/java/org/apache/accumulo/test/functional/PermissionsIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/PermissionsIT.java b/test/src/test/java/org/apache/accumulo/test/functional/PermissionsIT.java
index b8e1a4f..b05ad18 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/PermissionsIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/PermissionsIT.java
@@ -34,6 +34,9 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableExistsException;
+import org.apache.accumulo.core.client.TableNamespaceExistsException;
+import org.apache.accumulo.core.client.TableNamespaceNotEmptyException;
+import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.security.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
@@ -105,8 +108,9 @@ public class PermissionsIT extends SimpleMacIT {
   }
 
   private static void testMissingSystemPermission(String tableNamePrefix, Connector root_conn, Connector test_user_conn, SystemPermission perm)
-      throws AccumuloException, TableExistsException, AccumuloSecurityException, TableNotFoundException {
-    String tableName, user, password = "password";
+      throws AccumuloException, TableExistsException, AccumuloSecurityException, TableNotFoundException, TableNamespaceExistsException,
+      TableNamespaceNotFoundException, TableNamespaceNotEmptyException {
+    String tableName, user, password = "password", tableNamespace;
     log.debug("Confirming that the lack of the " + perm + " permission properly restricts the user");
 
     // test permission prior to granting it
@@ -199,14 +203,66 @@ public class PermissionsIT extends SimpleMacIT {
       case SYSTEM:
         // test for system permission would go here
         break;
+      case CREATE_NAMESPACE:
+        tableNamespace = "__CREATE_TABLE_NAMESPACE WITHOUT_PERM_TEST__";
+        try {
+          test_user_conn.tableNamespaceOperations().create(tableNamespace);
+          throw new IllegalStateException("Should NOT be able to create a table namespace");
+        } catch (AccumuloSecurityException e) {
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED || root_conn.tableNamespaceOperations().list().contains(tableNamespace))
+            throw e;
+        }
+        break;
+      case DROP_NAMESPACE:
+        tableNamespace = "__DROP_TABLE_NAMESPACE_WITHOUT_PERM_TEST__";
+        root_conn.tableNamespaceOperations().create(tableNamespace);
+        try {
+          test_user_conn.tableNamespaceOperations().delete(tableNamespace);
+          throw new IllegalStateException("Should NOT be able to delete a table namespace");
+        } catch (AccumuloSecurityException e) {
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED || !root_conn.tableNamespaceOperations().list().contains(tableNamespace))
+            throw e;
+        }
+        break;
+      case ALTER_NAMESPACE:
+        tableNamespace = "__ALTER_TABLE_NAMESPACE_WITHOUT_PERM_TEST__";
+        root_conn.tableNamespaceOperations().create(tableNamespace);
+        try {
+          test_user_conn.tableNamespaceOperations().setProperty(tableNamespace, Property.TABLE_BLOOM_ERRORRATE.getKey(), "003.14159%");
+          throw new IllegalStateException("Should NOT be able to set a table namespace property");
+        } catch (AccumuloSecurityException e) {
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED
+              || map(root_conn.tableNamespaceOperations().getProperties(tableNamespace)).get(Property.TABLE_BLOOM_ERRORRATE.getKey()).equals("003.14159%"))
+            throw e;
+        }
+        root_conn.tableNamespaceOperations().setProperty(tableNamespace, Property.TABLE_BLOOM_ERRORRATE.getKey(), "003.14159%");
+        try {
+          test_user_conn.tableNamespaceOperations().removeProperty(tableNamespace, Property.TABLE_BLOOM_ERRORRATE.getKey());
+          throw new IllegalStateException("Should NOT be able to remove a table namespace property");
+        } catch (AccumuloSecurityException e) {
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED
+              || !map(root_conn.tableNamespaceOperations().getProperties(tableNamespace)).get(Property.TABLE_BLOOM_ERRORRATE.getKey()).equals("003.14159%"))
+            throw e;
+        }
+        String tableNamespace2 = tableNamespace + "2";
+        try {
+          test_user_conn.tableNamespaceOperations().rename(tableNamespace, tableNamespace2);
+          throw new IllegalStateException("Should NOT be able to rename a table namespace");
+        } catch (AccumuloSecurityException e) {
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED || !root_conn.tableNamespaceOperations().list().contains(tableNamespace)
+              || root_conn.tableNamespaceOperations().list().contains(tableNamespace2))
+            throw e;
+        }
+        break;
       default:
         throw new IllegalArgumentException("Unrecognized System Permission: " + perm);
     }
   }
 
   private static void testGrantedSystemPermission(String tableNamePrefix, Connector root_conn, Connector test_user_conn, SystemPermission perm)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException, TableExistsException {
-    String tableName, user, password = "password";
+      throws AccumuloException, AccumuloSecurityException, TableNotFoundException, TableExistsException, TableNamespaceExistsException,
+      TableNamespaceNotFoundException, TableNamespaceNotEmptyException {
+    String tableName, user, password = "password", tableNamespace;
     log.debug("Confirming that the presence of the " + perm + " permission properly permits the user");
 
     // test permission after granting it
@@ -263,6 +319,35 @@ public class PermissionsIT extends SimpleMacIT {
       case SYSTEM:
         // test for system permission would go here
         break;
+      case CREATE_NAMESPACE:
+        tableNamespace = "__CREATE_TABLE_NAMESPACE_WITH_PERM_TEST__";
+        test_user_conn.tableNamespaceOperations().create(tableNamespace);
+        if (!root_conn.tableNamespaceOperations().list().contains(tableNamespace))
+          throw new IllegalStateException("Should be able to create a table namespace");
+        break;
+      case DROP_NAMESPACE:
+        tableNamespace = "__DROP_TABLE_NAMESPACE_WITH_PERM_TEST__";
+        root_conn.tableNamespaceOperations().create(tableNamespace);
+        test_user_conn.tableNamespaceOperations().delete(tableNamespace);
+        if (root_conn.tableNamespaceOperations().list().contains(tableNamespace))
+          throw new IllegalStateException("Should be able to delete a table namespace");
+        break;
+      case ALTER_NAMESPACE:
+        tableNamespace = "__ALTER_TABLE_NAMESPACE_WITH_PERM_TEST__";
+        String tableNamespace2 = tableNamespace + "2";
+        root_conn.tableNamespaceOperations().create(tableNamespace);
+        test_user_conn.tableNamespaceOperations().setProperty(tableNamespace, Property.TABLE_BLOOM_ERRORRATE.getKey(), "003.14159%");
+        Map<String,String> propies = map(root_conn.tableNamespaceOperations().getProperties(tableNamespace));
+        if (!propies.get(Property.TABLE_BLOOM_ERRORRATE.getKey()).equals("003.14159%"))
+          throw new IllegalStateException("Should be able to set a table property");
+        test_user_conn.tableNamespaceOperations().removeProperty(tableNamespace, Property.TABLE_BLOOM_ERRORRATE.getKey());
+        propies = map(root_conn.tableNamespaceOperations().getProperties(tableNamespace));
+        if (propies.get(Property.TABLE_BLOOM_ERRORRATE.getKey()).equals("003.14159%"))
+          throw new IllegalStateException("Should be able to remove a table property");
+        test_user_conn.tableNamespaceOperations().rename(tableNamespace, tableNamespace2);
+        if (root_conn.tableNamespaceOperations().list().contains(tableNamespace) || !root_conn.tableNamespaceOperations().list().contains(tableNamespace2))
+          throw new IllegalStateException("Should be able to rename a table");
+        break;
       default:
         throw new IllegalArgumentException("Unrecognized System Permission: " + perm);
     }


[41/50] [abbrv] git commit: ACCUMULO-802 fixed problems with some TableOps not locking correctly, fixed null pointer from some tables missing their namespace during randomwalk

Posted by ct...@apache.org.
ACCUMULO-802 fixed problems with some TableOps not locking correctly, fixed null pointer from some tables missing their namespace during randomwalk


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

Branch: refs/heads/master
Commit: bf7c1db8ff3ee7d6514295ac3f3ff4b24254b285
Parents: 7d8d688
Author: Sean Hickey <ta...@gmail.com>
Authored: Thu Aug 15 10:58:23 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:11 2013 -0500

----------------------------------------------------------------------
 .../admin/TableNamespaceOperationsImpl.java     |  6 ++++-
 .../accumulo/core/client/impl/Tables.java       |  2 +-
 .../server/conf/TableParentConfiguration.java   |  3 +--
 .../accumulo/server/tables/TableManager.java    |  2 +-
 .../java/org/apache/accumulo/master/Master.java | 12 +++++++--
 .../accumulo/master/tableOps/CloneTable.java    | 10 +++++---
 .../master/tableOps/CreateTableNamespace.java   |  1 +
 .../accumulo/master/tableOps/ImportTable.java   | 11 +++++---
 .../accumulo/master/tableOps/RenameTable.java   | 27 ++++++++++----------
 9 files changed, 46 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/bf7c1db8/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
index 694ba89..35fa8ae 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
@@ -340,7 +340,11 @@ public class TableNamespaceOperationsImpl extends TableNamespaceOperationsHelper
         throw new TableNamespaceNotEmptyException(namespaceId, namespace, null);
       }
       for (String table : TableNamespaces.getTableNames(instance, namespaceId)) {
-        getTableOperations().delete(table);
+        try {
+          getTableOperations().delete(table);
+        } catch (TableNotFoundException e) {
+          log.debug("Table (" + table + ") not found while deleting namespace, probably deleted while we were deleting the rest of the tables");
+        }
       }
     }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bf7c1db8/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
index 248bc6b..f80f506 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
@@ -63,7 +63,7 @@ public class Tables {
         String namespaceId = new String(nId, Constants.UTF8);
         if (!namespaceId.equals(Constants.DEFAULT_TABLE_NAMESPACE_ID) && !namespaceId.equals(Constants.SYSTEM_TABLE_NAMESPACE_ID)) {
           try {
-            name += TableNamespaces.getNamespaceName(instance, new String(namespaceId)) + ".";
+            name += TableNamespaces.getNamespaceName(instance, namespaceId) + ".";
           } catch (TableNamespaceNotFoundException e) {
             Log.error("Table (" + tableId + ") contains reference to namespace (" + namespaceId + ") that doesn't exist");
             continue;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bf7c1db8/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 7590d76..f8d8411 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
@@ -35,7 +35,6 @@ public class TableParentConfiguration extends TableNamespaceConfiguration {
   
   @Override
   protected String getNamespaceId() {
-    this.namespaceId = Tables.getNamespace(inst, tableId); 
-    return this.namespaceId;
+    return Tables.getNamespace(inst, tableId); 
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bf7c1db8/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 09bf92f..6d1cf08 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
@@ -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.getBytes(Constants.UTF8), NodeExistsPolicy.OVERWRITE);
+    ZooReaderWriter.getRetryingInstance().putPersistentData(zPath, Constants.DEFAULT_TABLE_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/bf7c1db8/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 aa0d17a..e0c1398 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
@@ -946,7 +946,11 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
           if (!security.canRenameTable(c, tableId, oldTableName, newTableName))
             throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-          fate.seedTransaction(opid, new TraceRepo<Master>(new RenameTable(tableId, oldTableName, newTableName)), autoCleanup);
+          try {
+            fate.seedTransaction(opid, new TraceRepo<Master>(new RenameTable(tableId, oldTableName, newTableName)), autoCleanup);
+          } catch (TableNamespaceNotFoundException e) {
+            throw new TException(e.getMessage(), e);
+          }
           
           break;
         }
@@ -1082,7 +1086,11 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
           checkNotMetadataTable(tableName, TableOperation.CREATE);
           checkTableName(tableName, TableOperation.CREATE);
 
-          fate.seedTransaction(opid, new TraceRepo<Master>(new ImportTable(c.getPrincipal(), tableName, exportDir)), autoCleanup);
+          try {
+            fate.seedTransaction(opid, new TraceRepo<Master>(new ImportTable(c.getPrincipal(), tableName, exportDir)), autoCleanup);
+          } catch (TableNamespaceNotFoundException e) {
+            throw new TException(e.getMessage(), e);
+          }
           break;
         }
         case EXPORT: {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bf7c1db8/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 f370a3a..97d6234 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,6 +21,7 @@ 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.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.TableOperation;
@@ -76,7 +77,7 @@ class FinishCloneTable extends MasterRepo {
     TableManager.getInstance().transitionTableState(cloneInfo.tableId, TableState.ONLINE);
     
     Utils.unreserveTableNamespace(cloneInfo.srcNamespaceId, tid, false);
-    if (!cloneInfo.namespaceId.equals(cloneInfo.srcNamespaceId))
+    if (!cloneInfo.srcNamespaceId.equals(cloneInfo.namespaceId))
       Utils.unreserveTableNamespace(cloneInfo.namespaceId, tid, false);
     Utils.unreserveTable(cloneInfo.srcTableId, tid, false);
     Utils.unreserveTable(cloneInfo.tableId, tid, true);
@@ -132,13 +133,14 @@ class CloneZookeeper extends MasterRepo {
   
   private CloneInfo cloneInfo;
   
-  public CloneZookeeper(CloneInfo cloneInfo) {
+  public CloneZookeeper(CloneInfo cloneInfo) throws TableNamespaceNotFoundException {
     this.cloneInfo = cloneInfo;
+    Instance inst = HdfsZooInstance.getInstance();
+    this.cloneInfo.namespaceId = TableNamespaces.getNamespaceId(inst, Tables.extractNamespace(this.cloneInfo.tableName));
   }
   
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    cloneInfo.namespaceId = TableNamespaces.getNamespaceId(environment.getInstance(), Tables.extractNamespace(cloneInfo.tableName));
     long val = 0;
     if (!cloneInfo.srcNamespaceId.equals(cloneInfo.namespaceId))
       val += Utils.reserveTableNamespace(cloneInfo.namespaceId, tid, false, true, TableOperation.CLONE);
@@ -171,7 +173,7 @@ class CloneZookeeper extends MasterRepo {
   public void undo(long tid, Master environment) throws Exception {
     Instance instance = HdfsZooInstance.getInstance();
     TableManager.getInstance().removeTable(cloneInfo.tableId);
-    if (!cloneInfo.namespaceId.equals(cloneInfo.srcNamespaceId))
+    if (!cloneInfo.srcNamespaceId.equals(cloneInfo.namespaceId))
       Utils.unreserveTableNamespace(cloneInfo.namespaceId, tid, false);
     Utils.unreserveTable(cloneInfo.tableId, tid, true);
     Tables.clearCache(instance);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bf7c1db8/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
index 247ae56..138d77b 100644
--- 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
@@ -124,6 +124,7 @@ class PopulateZookeeperWithNamespace extends MasterRepo {
   public void undo(long tid, Master master) throws Exception {
     TableManager.getInstance().removeNamespace(tableNamespaceInfo.namespaceId);
     Tables.clearCache(master.getInstance());
+    Utils.unreserveTableNamespace(tableNamespaceInfo.namespaceId, tid, true);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bf7c1db8/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 4405387..c9ed10f 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,6 +34,7 @@ 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.admin.TableOperationsImpl;
 import org.apache.accumulo.core.client.impl.TableNamespaces;
 import org.apache.accumulo.core.client.impl.Tables;
@@ -419,7 +420,7 @@ class ImportPopulateZookeeper extends MasterRepo {
   
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    return Utils.reserveTableNamespace(tableInfo.namespaceId, tid, false, true, TableOperation.IMPORT) + Utils.reserveTable(tableInfo.tableId, tid, true, false, TableOperation.IMPORT);
+    return Utils.reserveTable(tableInfo.tableId, tid, true, false, TableOperation.IMPORT);
   }
   
   private Map<String,String> getExportedProps(VolumeManager fs) throws Exception {
@@ -470,7 +471,6 @@ class ImportPopulateZookeeper extends MasterRepo {
   public void undo(long tid, Master env) throws Exception {
     Instance instance = HdfsZooInstance.getInstance();
     TableManager.getInstance().removeTable(tableInfo.tableId);
-    Utils.unreserveTableNamespace(tableInfo.namespaceId, tid, false);
     Utils.unreserveTable(tableInfo.tableId, tid, true);
     Tables.clearCache(instance);
   }
@@ -521,16 +521,18 @@ public class ImportTable extends MasterRepo {
   
   private ImportedTableInfo tableInfo;
   
-  public ImportTable(String user, String tableName, String exportDir) {
+  public ImportTable(String user, String tableName, String exportDir) throws TableNamespaceNotFoundException {
     tableInfo = new ImportedTableInfo();
     tableInfo.tableName = tableName;
     tableInfo.user = user;
     tableInfo.exportDir = exportDir;
+    Instance inst = HdfsZooInstance.getInstance();
+    tableInfo.namespaceId = TableNamespaces.getNamespaceId(inst, Tables.extractNamespace(tableName));
   }
   
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    return Utils.reserveHdfsDirectory(new Path(tableInfo.exportDir).toString(), tid);
+    return Utils.reserveHdfsDirectory(new Path(tableInfo.exportDir).toString(), tid) + Utils.reserveTableNamespace(tableInfo.namespaceId, tid, false, true, TableOperation.IMPORT);
   }
   
   @Override
@@ -610,5 +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);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bf7c1db8/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
index c34d082..9044247 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.master.tableOps;
 
 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.impl.TableNamespaces;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.TableOperation;
@@ -39,20 +40,23 @@ public class RenameTable extends MasterRepo {
   private String tableId;
   private String oldTableName;
   private String newTableName;
-  private String namespaceId;
+  private String oldNamespaceId;
+  private String newNamespaceId;
 
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    return Utils.reserveTableNamespace(namespaceId, tid, false, true, TableOperation.RENAME)
+    return Utils.reserveTableNamespace(oldNamespaceId, tid, false, true, TableOperation.RENAME)
+        + Utils.reserveTableNamespace(newNamespaceId, tid, false, true, TableOperation.RENAME)
         + Utils.reserveTable(tableId, tid, true, true, TableOperation.RENAME);
   }
 
-  public RenameTable(String tableId, String oldTableName, String newTableName) {
+  public RenameTable(String tableId, String oldTableName, String newTableName) throws TableNamespaceNotFoundException {
     this.tableId = tableId;
     this.oldTableName = oldTableName;
     this.newTableName = newTableName;
     Instance inst = HdfsZooInstance.getInstance();
-    this.namespaceId = Tables.getNamespace(inst, tableId);
+    this.oldNamespaceId = Tables.getNamespace(inst, tableId);
+    this.newNamespaceId = TableNamespaces.getNamespaceId(inst, Tables.extractNamespace(newTableName));
   }
 
   @Override
@@ -60,14 +64,9 @@ public class RenameTable extends MasterRepo {
 
     Instance instance = master.getInstance();
 
-    final String namespace = Tables.extractNamespace(newTableName);
-    String namespaceId = TableNamespaces.getNamespaceId(instance, namespace);
-    final String oldNamespace = Tables.extractNamespace(oldTableName);
-    String oldNamespaceId = TableNamespaces.getNamespaceId(instance, oldNamespace);
-
-    if (!namespaceId.equals(oldNamespaceId)) {
+    if (!newNamespaceId.equals(oldNamespaceId)) {
       TableManager tm = TableManager.getInstance();
-      tm.addNamespaceToTable(tableId, namespaceId);
+      tm.addNamespaceToTable(tableId, newNamespaceId);
     }
 
     IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
@@ -98,7 +97,8 @@ public class RenameTable extends MasterRepo {
     } finally {
       Utils.tableNameLock.unlock();
       Utils.unreserveTable(tableId, tid, true);
-      Utils.unreserveTableNamespace(this.namespaceId, tid, false);
+      Utils.unreserveTableNamespace(this.oldNamespaceId, tid, false);
+      Utils.unreserveTableNamespace(this.newNamespaceId, tid, false);
     }
 
     Logger.getLogger(RenameTable.class).debug("Renamed table " + tableId + " " + oldTableName + " " + newTableName);
@@ -108,8 +108,9 @@ public class RenameTable extends MasterRepo {
 
   @Override
   public void undo(long tid, Master env) throws Exception {
+    Utils.unreserveTableNamespace(newNamespaceId, tid, false);
+    Utils.unreserveTableNamespace(oldNamespaceId, tid, false);
     Utils.unreserveTable(tableId, tid, true);
-    Utils.unreserveTableNamespace(namespaceId, tid, false);
   }
 
 }


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

Posted by ct...@apache.org.
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
    */


[13/50] [abbrv] git commit: ACCUMULO-802 added namespace locking to the rest of the tableOps and put them in a consistent order (namespace then table), switched to using HdfsZooInstance to get the instance for the namespaceId

Posted by ct...@apache.org.
ACCUMULO-802 added namespace locking to the rest of the tableOps and put them in a consistent order (namespace then table), switched to using HdfsZooInstance to get the instance for the namespaceId


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

Branch: refs/heads/master
Commit: 3d707c8645619e349fefd9323a2c2c22c803d4d4
Parents: c2ed43e
Author: Sean Hickey <ta...@gmail.com>
Authored: Wed Aug 14 08:04:55 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:10 2013 -0500

----------------------------------------------------------------------
 .../core/client/admin/TableOperationsImpl.java  |  5 +-
 .../java/org/apache/accumulo/master/Master.java |  8 +-
 .../master/tableOps/CancelCompactions.java      | 17 ++--
 .../master/tableOps/ChangeTableState.java       | 26 ++++--
 .../accumulo/master/tableOps/CloneTable.java    | 11 +--
 .../accumulo/master/tableOps/CompactRange.java  | 95 +++++++++++++++++++-
 .../accumulo/master/tableOps/CreateTable.java   |  8 +-
 .../accumulo/master/tableOps/DeleteTable.java   |  6 +-
 .../accumulo/master/tableOps/ExportTable.java   | 10 ++-
 .../accumulo/master/tableOps/ImportTable.java   |  5 +-
 .../accumulo/master/tableOps/RenameTable.java   |  4 +-
 .../master/tableOps/RenameTableNamespace.java   |  2 +-
 .../accumulo/master/tableOps/TableRangeOp.java  | 56 +++++++-----
 .../test/randomwalk/concurrent/Merge.java       |  2 +-
 .../concurrent/RenameTableNamespace.java        |  2 +-
 15 files changed, 197 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
index cfaa657..aa4c83e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
@@ -129,7 +129,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   private Credentials credentials;
 
   public static final String CLONE_EXCLUDE_PREFIX = "!";
-
+  
   private static final Logger log = Logger.getLogger(TableOperations.class);
 
   /**
@@ -727,7 +727,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
   // get the properties that are only in the table namespace so that we can exclude them when copying table properties.
   // also, don't exclude properties that are going to be explicitly set.
-  private HashSet<String> getUniqueNamespaceProperties(String namespace, String table, Map<String,String> propsToSet) throws TableNotFoundException, AccumuloException {
+  private HashSet<String> getUniqueNamespaceProperties(String namespace, String table, Map<String,String> propsToSet) throws TableNotFoundException,
+      AccumuloException {
     HashSet<String> props = new HashSet<String>();
     try {
       Iterable<Entry<String,String>> n = new TableNamespaceOperationsImpl(instance, credentials).getProperties(namespace);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/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 338550a..5cfd3c3 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
@@ -929,7 +929,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
           TimeType timeType = TimeType.valueOf(ByteBufferUtil.toString(arguments.get(1)));
 
           try {
-            fate.seedTransaction(opid, new TraceRepo<Master>(new CreateTable(c.getPrincipal(), tableName, timeType, options, getInstance())), autoCleanup);
+            fate.seedTransaction(opid, new TraceRepo<Master>(new CreateTable(c.getPrincipal(), tableName, timeType, options)), autoCleanup);
           } catch (TableNamespaceNotFoundException e) {
             throw new ThriftTableOperationException(null, tableName, TableOperation.CREATE, TableOperationExceptionType.NOTFOUND, e.getMessage());
           }
@@ -946,7 +946,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
           if (!security.canRenameTable(c, tableId, oldTableName, newTableName))
             throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-          fate.seedTransaction(opid, new TraceRepo<Master>(new RenameTable(tableId, oldTableName, newTableName, getInstance())), autoCleanup);
+          fate.seedTransaction(opid, new TraceRepo<Master>(new RenameTable(tableId, oldTableName, newTableName)), autoCleanup);
           
           break;
         }
@@ -975,7 +975,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
             propertiesToSet.put(entry.getKey(), entry.getValue());
           }
 
-          fate.seedTransaction(opid, new TraceRepo<Master>(new CloneTable(c.getPrincipal(), srcTableId, tableName, propertiesToSet, propertiesToExclude, getInstance())),
+          fate.seedTransaction(opid, new TraceRepo<Master>(new CloneTable(c.getPrincipal(), srcTableId, tableName, propertiesToSet, propertiesToExclude)),
               autoCleanup);
 
           break;
@@ -986,7 +986,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
           checkNotMetadataTable(tableName, TableOperation.DELETE);
           if (!security.canDeleteTable(c, tableId))
             throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-          fate.seedTransaction(opid, new TraceRepo<Master>(new DeleteTable(tableId, getInstance())), autoCleanup);
+          fate.seedTransaction(opid, new TraceRepo<Master>(new DeleteTable(tableId)), autoCleanup);
           break;
         }
         case ONLINE: {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/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 dd4c229..9698344 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
@@ -17,6 +17,8 @@
 package org.apache.accumulo.master.tableOps;
 
 import org.apache.accumulo.core.Constants;
+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.fate.Repo;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
@@ -32,7 +34,7 @@ class FinishCancelCompaction extends MasterRepo {
   public FinishCancelCompaction(String tableId) {
     this.tableId = tableId;
   }
-
+  
   @Override
   public Repo<Master> call(long tid, Master environment) throws Exception {
     Utils.getReadLock(tableId, tid).unlock();
@@ -52,14 +54,18 @@ public class CancelCompactions extends MasterRepo {
   
   private static final long serialVersionUID = 1L;
   private String tableId;
+  private String namespaceId;
   
   public CancelCompactions(String tableId) {
     this.tableId = tableId;
+    Instance inst = HdfsZooInstance.getInstance();
+    this.namespaceId = Tables.getNamespace(inst, tableId);
   }
-
+  
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    return Utils.reserveTable(tableId, tid, false, true, TableOperation.COMPACT_CANCEL);
+    return Utils.reserveTableNamespace(namespaceId, tid, false, true, TableOperation.COMPACT_CANCEL)
+        + Utils.reserveTable(tableId, tid, false, true, TableOperation.COMPACT_CANCEL);
   }
   
   @Override
@@ -85,15 +91,16 @@ public class CancelCompactions extends MasterRepo {
           return (flushID + "").getBytes();
         else
           return (cid + "").getBytes();
-
+        
       }
     });
-
+    
     return new FinishCancelCompaction(tableId);
   }
   
   @Override
   public void undo(long tid, Master environment) throws Exception {
+    Utils.unreserveTableNamespace(namespaceId, tid, false);
     Utils.unreserveTable(tableId, tid, false);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/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 697c15e..ffd61e4 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
@@ -16,49 +16,57 @@
  */
 package org.apache.accumulo.master.tableOps;
 
+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.master.state.tables.TableState;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.master.Master;
+import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.tables.TableManager;
 import org.apache.log4j.Logger;
 
 public class ChangeTableState extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
   private String tableId;
   private TableOperation top;
-  
+  private String namespaceId;
+
   public ChangeTableState(String tableId, TableOperation top) {
     this.tableId = tableId;
     this.top = top;
-    
+    Instance inst = HdfsZooInstance.getInstance();
+    this.namespaceId = Tables.getNamespace(inst, tableId);
+
     if (top != TableOperation.ONLINE && top != TableOperation.OFFLINE)
       throw new IllegalArgumentException(top.toString());
   }
-  
+
   @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.reserveTable(tableId, tid, true, true, top);
+    return Utils.reserveTableNamespace(namespaceId, tid, false, true, top) + Utils.reserveTable(tableId, tid, true, true, top);
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master env) throws Exception {
-    
+
     TableState ts = TableState.ONLINE;
     if (top == TableOperation.OFFLINE)
       ts = TableState.OFFLINE;
-    
+
     TableManager.getInstance().transitionTableState(tableId, ts);
+    Utils.unreserveTableNamespace(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);
     return null;
   }
-  
+
   @Override
   public void undo(long tid, Master env) throws Exception {
+    Utils.unreserveTableNamespace(namespaceId, tid, false);
     Utils.unreserveTable(tableId, tid, true);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/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 46edc25..f370a3a 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
@@ -75,11 +75,11 @@ class FinishCloneTable extends MasterRepo {
     
     TableManager.getInstance().transitionTableState(cloneInfo.tableId, TableState.ONLINE);
     
-    Utils.unreserveTable(cloneInfo.srcTableId, tid, false);
-    Utils.unreserveTable(cloneInfo.tableId, tid, true);
     Utils.unreserveTableNamespace(cloneInfo.srcNamespaceId, tid, false);
     if (!cloneInfo.namespaceId.equals(cloneInfo.srcNamespaceId))
       Utils.unreserveTableNamespace(cloneInfo.namespaceId, tid, false);
+    Utils.unreserveTable(cloneInfo.srcTableId, tid, false);
+    Utils.unreserveTable(cloneInfo.tableId, tid, true);
     
     environment.getEventCoordinator().event("Cloned table %s from %s", cloneInfo.tableName, cloneInfo.srcTableId);
     
@@ -171,9 +171,9 @@ class CloneZookeeper extends MasterRepo {
   public void undo(long tid, Master environment) throws Exception {
     Instance instance = HdfsZooInstance.getInstance();
     TableManager.getInstance().removeTable(cloneInfo.tableId);
-    Utils.unreserveTable(cloneInfo.tableId, tid, true);
     if (!cloneInfo.namespaceId.equals(cloneInfo.srcNamespaceId))
       Utils.unreserveTableNamespace(cloneInfo.namespaceId, tid, false);
+    Utils.unreserveTable(cloneInfo.tableId, tid, true);
     Tables.clearCache(instance);
   }
   
@@ -224,13 +224,14 @@ public class CloneTable extends MasterRepo {
   private static final long serialVersionUID = 1L;
   private CloneInfo cloneInfo;
   
-  public CloneTable(String user, String srcTableId, String tableName, Map<String,String> propertiesToSet, Set<String> propertiesToExclude, Instance inst) {
+  public CloneTable(String user, String srcTableId, String tableName, Map<String,String> propertiesToSet, Set<String> propertiesToExclude) {
     cloneInfo = new CloneInfo();
     cloneInfo.user = user;
     cloneInfo.srcTableId = srcTableId;
     cloneInfo.tableName = tableName;
     cloneInfo.propertiesToExclude = propertiesToExclude;
     cloneInfo.propertiesToSet = propertiesToSet;
+    Instance inst = HdfsZooInstance.getInstance();
     cloneInfo.srcNamespaceId = Tables.getNamespace(inst, cloneInfo.srcTableId);
   }
   
@@ -257,8 +258,8 @@ public class CloneTable extends MasterRepo {
   
   @Override
   public void undo(long tid, Master environment) throws Exception {
-    Utils.unreserveTable(cloneInfo.srcTableId, tid, false);
     Utils.unreserveTableNamespace(cloneInfo.srcNamespaceId, tid, false);
+    Utils.unreserveTable(cloneInfo.srcTableId, tid, false);
   }
   
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/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 160fc7e..1c3b398 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
@@ -16,6 +16,10 @@
  */
 package org.apache.accumulo.master.tableOps;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
@@ -49,10 +53,10 @@ import org.apache.accumulo.master.Master;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.master.LiveTServerSet.TServerConnection;
 import org.apache.accumulo.server.master.state.TServerInstance;
-import org.apache.accumulo.server.master.tableOps.CompactionIterators;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.commons.codec.binary.Hex;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.log4j.Logger;
 import org.apache.thrift.TException;
@@ -66,6 +70,7 @@ class CompactionDriver extends MasterRepo {
   private String tableId;
   private byte[] startRow;
   private byte[] endRow;
+  private String namespaceId;
 
   public CompactionDriver(long compactId, String tableId, byte[] startRow, byte[] endRow) {
 
@@ -73,6 +78,8 @@ class CompactionDriver extends MasterRepo {
     this.tableId = tableId;
     this.startRow = startRow;
     this.endRow = endRow;
+    Instance inst = HdfsZooInstance.getInstance();
+    this.namespaceId = Tables.getNamespace(inst, tableId);
   }
 
   @Override
@@ -179,6 +186,7 @@ class CompactionDriver extends MasterRepo {
   public Repo<Master> call(long tid, Master environment) throws Exception {
     CompactRange.removeIterators(tid, tableId);
     Utils.getReadLock(tableId, tid).unlock();
+    Utils.getReadLock(namespaceId, tid).unlock();
     return null;
   }
 
@@ -196,11 +204,92 @@ public class CompactRange extends MasterRepo {
   private byte[] startRow;
   private byte[] endRow;
   private byte[] iterators;
+  private String namespaceId;
+
+  public static class CompactionIterators implements Writable {
+    byte[] startRow;
+    byte[] endRow;
+    List<IteratorSetting> iterators;
+
+    public CompactionIterators(byte[] startRow, byte[] endRow, List<IteratorSetting> iterators) {
+      this.startRow = startRow;
+      this.endRow = endRow;
+      this.iterators = iterators;
+    }
+
+    public CompactionIterators() {
+      startRow = null;
+      endRow = null;
+      iterators = Collections.emptyList();
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      out.writeBoolean(startRow != null);
+      if (startRow != null) {
+        out.writeInt(startRow.length);
+        out.write(startRow);
+      }
+
+      out.writeBoolean(endRow != null);
+      if (endRow != null) {
+        out.writeInt(endRow.length);
+        out.write(endRow);
+      }
+
+      out.writeInt(iterators.size());
+      for (IteratorSetting is : iterators) {
+        is.write(out);
+      }
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      if (in.readBoolean()) {
+        startRow = new byte[in.readInt()];
+        in.readFully(startRow);
+      } else {
+        startRow = null;
+      }
+
+      if (in.readBoolean()) {
+        endRow = new byte[in.readInt()];
+        in.readFully(endRow);
+      } else {
+        endRow = null;
+      }
+
+      int num = in.readInt();
+      iterators = new ArrayList<IteratorSetting>(num);
+
+      for (int i = 0; i < num; i++) {
+        iterators.add(new IteratorSetting(in));
+      }
+    }
+
+    public Text getEndRow() {
+      if (endRow == null)
+        return null;
+      return new Text(endRow);
+    }
+
+    public Text getStartRow() {
+      if (startRow == null)
+        return null;
+      return new Text(startRow);
+    }
+
+    public List<IteratorSetting> getIterators() {
+      return iterators;
+    }
+  }
 
   public CompactRange(String tableId, byte[] startRow, byte[] endRow, List<IteratorSetting> iterators) throws ThriftTableOperationException {
     this.tableId = tableId;
     this.startRow = startRow.length == 0 ? null : startRow;
     this.endRow = endRow.length == 0 ? null : endRow;
+    Instance inst = HdfsZooInstance.getInstance();
+    this.namespaceId = Tables.getNamespace(inst, tableId);
 
     if (iterators.size() > 0) {
       this.iterators = WritableUtils.toByteArray(new CompactionIterators(this.startRow, this.endRow, iterators));
@@ -215,7 +304,8 @@ public class CompactRange extends MasterRepo {
 
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    return Utils.reserveTable(tableId, tid, false, true, TableOperation.COMPACT);
+    return Utils.reserveTableNamespace(namespaceId, tid, false, true, TableOperation.COMPACT)
+        + Utils.reserveTable(tableId, tid, false, true, TableOperation.COMPACT);
   }
 
   @Override
@@ -297,6 +387,7 @@ public class CompactRange extends MasterRepo {
     try {
       removeIterators(tid, tableId);
     } finally {
+      Utils.unreserveTableNamespace(namespaceId, tid, false);
       Utils.unreserveTable(tableId, tid, false);
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/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 f657a1b..10eba61 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
@@ -35,6 +35,7 @@ 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.ServerConstants;
+import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.security.AuditedSecurityOperation;
 import org.apache.accumulo.server.security.SecurityOperation;
@@ -81,8 +82,8 @@ class FinishCreateTable extends MasterRepo {
   public Repo<Master> call(long tid, Master env) throws Exception {
     TableManager.getInstance().transitionTableState(tableInfo.tableId, TableState.ONLINE);
     
-    Utils.unreserveTable(tableInfo.tableId, tid, true);
     Utils.unreserveTableNamespace(tableInfo.namespaceId, tid, false);
+    Utils.unreserveTable(tableInfo.tableId, tid, true);
     
     env.getEventCoordinator().event("Created table %s ", tableInfo.tableName);
     
@@ -280,12 +281,13 @@ public class CreateTable extends MasterRepo {
   
   private TableInfo tableInfo;
   
-  public CreateTable(String user, String tableName, TimeType timeType, Map<String,String> props, Instance inst) throws TableNamespaceNotFoundException {
+  public CreateTable(String user, String tableName, TimeType timeType, Map<String,String> props) throws TableNamespaceNotFoundException {
     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));
   }
   
@@ -296,7 +298,7 @@ public class CreateTable extends MasterRepo {
   }
   
   @Override
-  public Repo<Master> call(long tid, Master master) throws Exception {   
+  public Repo<Master> call(long tid, Master master) throws Exception {
     // first step is to reserve a table id.. if the machine fails during this step
     // it is ok to retry... the only side effect is that a table id may not be used
     // or skipped

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/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 98d1ae9..bf1d9ce 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
@@ -41,6 +41,7 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.master.Master;
 import org.apache.accumulo.server.ServerConstants;
+import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.master.state.MetaDataTableScanner;
 import org.apache.accumulo.server.master.state.TabletLocationState;
@@ -221,8 +222,9 @@ public class DeleteTable extends MasterRepo {
   
   private String tableId, namespaceId;
   
-  public DeleteTable(String tableId, Instance inst) {
+  public DeleteTable(String tableId) {
     this.tableId = tableId;
+    Instance inst = HdfsZooInstance.getInstance();
     this.namespaceId = Tables.getNamespace(inst, tableId);
   }
   
@@ -242,8 +244,8 @@ public class DeleteTable extends MasterRepo {
   
   @Override
   public void undo(long tid, Master environment) throws Exception {
-    Utils.unreserveTable(tableId, tid, true);
     Utils.unreserveTableNamespace(namespaceId, tid, false);
+    Utils.unreserveTable(tableId, tid, true);
   }
   
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/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 22df3b3..cad779c 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
@@ -32,6 +32,7 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.Tables;
@@ -53,6 +54,7 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.master.Master;
 import org.apache.accumulo.server.ServerConstants;
+import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -67,6 +69,7 @@ class ExportInfo implements Serializable {
   public String tableName;
   public String tableID;
   public String exportDir;
+  public String namespaceID;
 }
 
 class WriteExportFiles extends MasterRepo {
@@ -91,7 +94,8 @@ class WriteExportFiles extends MasterRepo {
   @Override
   public long isReady(long tid, Master master) throws Exception {
     
-    long reserved = Utils.reserveTable(tableInfo.tableID, tid, false, true, TableOperation.EXPORT);
+    long reserved = Utils.reserveTableNamespace(tableInfo.namespaceID, tid, false, true, TableOperation.EXPORT)
+        + Utils.reserveTable(tableInfo.tableID, tid, false, true, TableOperation.EXPORT);
     if (reserved > 0)
       return reserved;
     
@@ -133,6 +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.unreserveTable(tableInfo.tableID, tid, false);
     Utils.unreserveHdfsDirectory(new Path(tableInfo.exportDir).toString(), tid);
     return null;
@@ -140,6 +145,7 @@ class WriteExportFiles extends MasterRepo {
   
   @Override
   public void undo(long tid, Master env) throws Exception {
+    Utils.unreserveTableNamespace(tableInfo.namespaceID, tid, false);
     Utils.unreserveTable(tableInfo.tableID, tid, false);
   }
   
@@ -283,6 +289,8 @@ public class ExportTable extends MasterRepo {
     tableInfo.tableName = tableName;
     tableInfo.exportDir = exportDir;
     tableInfo.tableID = tableId;
+    Instance inst = HdfsZooInstance.getInstance();
+    tableInfo.namespaceID = Tables.getNamespace(inst, tableId);
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/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 4032ce5..4405387 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
@@ -83,6 +83,7 @@ class ImportedTableInfo implements Serializable {
   public String tableName;
   public String tableId;
   public String importDir;
+  public String namespaceId;
 }
 
 class FinishImportTable extends MasterRepo {
@@ -107,6 +108,7 @@ class FinishImportTable extends MasterRepo {
     
     TableManager.getInstance().transitionTableState(tableInfo.tableId, TableState.ONLINE);
     
+    Utils.unreserveTableNamespace(tableInfo.namespaceId, tid, false);
     Utils.unreserveTable(tableInfo.tableId, tid, true);
     
     Utils.unreserveHdfsDirectory(new Path(tableInfo.exportDir).toString(), tid);
@@ -417,7 +419,7 @@ class ImportPopulateZookeeper extends MasterRepo {
   
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    return Utils.reserveTable(tableInfo.tableId, tid, true, false, TableOperation.IMPORT);
+    return Utils.reserveTableNamespace(tableInfo.namespaceId, tid, false, true, TableOperation.IMPORT) + Utils.reserveTable(tableInfo.tableId, tid, true, false, TableOperation.IMPORT);
   }
   
   private Map<String,String> getExportedProps(VolumeManager fs) throws Exception {
@@ -468,6 +470,7 @@ class ImportPopulateZookeeper extends MasterRepo {
   public void undo(long tid, Master env) throws Exception {
     Instance instance = HdfsZooInstance.getInstance();
     TableManager.getInstance().removeTable(tableInfo.tableId);
+    Utils.unreserveTableNamespace(tableInfo.namespaceId, tid, false);
     Utils.unreserveTable(tableInfo.tableId, tid, true);
     Tables.clearCache(instance);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
index 871c0a0..c34d082 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
@@ -28,6 +28,7 @@ import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter.Mutator;
 import org.apache.accumulo.master.Master;
+import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.tables.TableManager;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.log4j.Logger;
@@ -46,10 +47,11 @@ public class RenameTable extends MasterRepo {
         + Utils.reserveTable(tableId, tid, true, true, TableOperation.RENAME);
   }
 
-  public RenameTable(String tableId, String oldTableName, String newTableName, Instance inst) {
+  public RenameTable(String tableId, String oldTableName, String newTableName) {
     this.tableId = tableId;
     this.oldTableName = oldTableName;
     this.newTableName = newTableName;
+    Instance inst = HdfsZooInstance.getInstance();
     this.namespaceId = Tables.getNamespace(inst, tableId);
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTableNamespace.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTableNamespace.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTableNamespace.java
index 2da5d9f..9662fbb 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTableNamespace.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTableNamespace.java
@@ -75,7 +75,7 @@ public class RenameTableNamespace extends MasterRepo {
       Tables.clearCache(instance);
     } finally {
       Utils.tableNameLock.unlock();
-      Utils.unreserveTable(namespaceId, id, true);
+      Utils.unreserveTableNamespace(namespaceId, id, true);
     }
 
     Logger.getLogger(RenameTableNamespace.class).debug("Renamed table namespace " + namespaceId + " " + oldName + " " + newName);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableRangeOp.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableRangeOp.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableRangeOp.java
index 0ad2196..77a6606 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableRangeOp.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableRangeOp.java
@@ -16,6 +16,8 @@
  */
 package org.apache.accumulo.master.tableOps;
 
+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.TableOperationExceptionType;
 import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
@@ -24,9 +26,10 @@ import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.master.Master;
+import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.master.state.MergeInfo;
-import org.apache.accumulo.server.master.state.MergeState;
 import org.apache.accumulo.server.master.state.MergeInfo.Operation;
+import org.apache.accumulo.server.master.state.MergeState;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -43,14 +46,17 @@ import org.apache.hadoop.io.Text;
  * and prevent merges (writes) while they run. Merge operations will lock out some operations while they run.
  */
 class TableRangeOpWait extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
   private String tableId;
-  
+  private String namespaceId;
+
   public TableRangeOpWait(String tableId) {
     this.tableId = tableId;
+    Instance inst = HdfsZooInstance.getInstance();
+    this.namespaceId = Tables.getNamespace(inst, tableId);
   }
-  
+
   @Override
   public long isReady(long tid, Master env) throws Exception {
     Text tableIdText = new Text(tableId);
@@ -59,69 +65,74 @@ class TableRangeOpWait extends MasterRepo {
     }
     return 0;
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master master) throws Exception {
     Text tableIdText = new Text(tableId);
     MergeInfo mergeInfo = master.getMergeInfo(tableIdText);
     log.info("removing merge information " + mergeInfo);
     master.clearMergeState(tableIdText);
+    Utils.unreserveTableNamespace(namespaceId, tid, false);
     Utils.unreserveTable(tableId, tid, true);
     return null;
   }
-  
+
 }
 
 public class TableRangeOp extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
-  
+
   private String tableId;
   private byte[] startRow;
   private byte[] endRow;
   private Operation op;
-  
+  private String namespaceId;
+
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    return Utils.reserveTable(tableId, tid, true, true, TableOperation.MERGE);
+    return Utils.reserveTableNamespace(namespaceId, tid, false, true, TableOperation.MERGE)
+        + Utils.reserveTable(tableId, tid, true, true, TableOperation.MERGE);
   }
-  
+
   public TableRangeOp(MergeInfo.Operation op, String tableId, Text startRow, Text endRow) throws ThriftTableOperationException {
-    
+
     this.tableId = tableId;
     this.startRow = TextUtil.getBytes(startRow);
     this.endRow = TextUtil.getBytes(endRow);
     this.op = op;
+    Instance inst = HdfsZooInstance.getInstance();
+    this.namespaceId = Tables.getNamespace(inst, tableId);
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master env) throws Exception {
-    
+
     if (RootTable.ID.equals(tableId) && TableOperation.MERGE.equals(op)) {
       log.warn("Attempt to merge tablets for " + RootTable.NAME + " does nothing. It is not splittable.");
     }
-    
+
     Text start = startRow.length == 0 ? null : new Text(startRow);
     Text end = endRow.length == 0 ? null : new Text(endRow);
     Text tableIdText = new Text(tableId);
-    
+
     if (start != null && end != null)
       if (start.compareTo(end) >= 0)
         throw new ThriftTableOperationException(tableId, null, TableOperation.MERGE, TableOperationExceptionType.BAD_RANGE,
             "start row must be less than end row");
-    
+
     env.mustBeOnline(tableId);
-    
+
     MergeInfo info = env.getMergeInfo(tableIdText);
-    
+
     if (info.getState() == MergeState.NONE) {
       KeyExtent range = new KeyExtent(tableIdText, end, start);
       env.setMergeState(new MergeInfo(range, op), MergeState.STARTED);
     }
-    
+
     return new TableRangeOpWait(tableId);
   }
-  
+
   @Override
   public void undo(long tid, Master env) throws Exception {
     // Not sure this is a good thing to do. The Master state engine should be the one to remove it.
@@ -130,7 +141,8 @@ public class TableRangeOp extends MasterRepo {
     if (mergeInfo.getState() != MergeState.NONE)
       log.info("removing merge information " + mergeInfo);
     env.clearMergeState(tableIdText);
+    Utils.unreserveTableNamespace(namespaceId, tid, false);
     Utils.unreserveTable(tableId, tid, true);
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Merge.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Merge.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Merge.java
index 8fcfab5..a16f6a6 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Merge.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Merge.java
@@ -42,7 +42,7 @@ public class Merge extends Test {
     tableNames = new ArrayList<String>(tableNames);
     tableNames.add(MetadataTable.NAME);
     String tableName = tableNames.get(rand.nextInt(tableNames.size()));
-        
+    
     List<Text> range = ConcurrentFixture.generateRange(rand);
     
     try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3d707c86/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java
index 01329d0..fdd4cef 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java
@@ -35,7 +35,7 @@ public class RenameTableNamespace extends Test {
     Random rand = (Random) state.get("rand");
     
     @SuppressWarnings("unchecked")
-    List<String> namespaces = (List<String>) state.get("tables");
+    List<String> namespaces = (List<String>) state.get("namespaces");
     
     String srcName = namespaces.get(rand.nextInt(namespaces.size()));
     String newName = namespaces.get(rand.nextInt(namespaces.size()));


[38/50] [abbrv] ACCUMULO-1712 Rename system tables and namespaces

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/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 bd3b6a9..dc154d7 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
@@ -28,6 +28,7 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.master.state.tables.TableState;
+import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter.Mutator;
@@ -46,20 +47,21 @@ import org.apache.zookeeper.Watcher.Event.EventType;
 
 public class TableManager {
   private static SecurityPermission TABLE_MANAGER_PERMISSION = new SecurityPermission("tableManagerPermission");
-  
+
   private static final Logger log = Logger.getLogger(TableManager.class);
   private static final Set<TableObserver> observers = Collections.synchronizedSet(new HashSet<TableObserver>());
   private static final Map<String,TableState> tableStateCache = Collections.synchronizedMap(new HashMap<String,TableState>());
-  
+
   private static TableManager tableManager = null;
-  
+
   private final Instance instance;
   private ZooCache zooStateCache;
-  
+
   public static void prepareNewTableState(String instanceId, String tableId, String tableName, TableState state, NodeExistsPolicy existsPolicy)
       throws KeeperException, InterruptedException {
     // state gets created last
-    tableName = Tables.extractTableName(tableName);
+    Pair<String,String> qualifiedTableName = Tables.qualify(tableName);
+    tableName = qualifiedTableName.getSecond();
     String zTablePath = Constants.ZROOT + "/" + instanceId + Constants.ZTABLES + "/" + tableId;
     IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
     zoo.putPersistentData(zTablePath, new byte[0], existsPolicy);
@@ -69,9 +71,9 @@ 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_NAMESPACE_ID.getBytes(Constants.UTF8), existsPolicy);
+    zoo.putPersistentData(zTablePath + Constants.ZTABLE_NAMESPACE, qualifiedTableName.getFirst().getBytes(Constants.UTF8), existsPolicy);
   }
-  
+
   public synchronized static TableManager getInstance() {
     SecurityManager sm = System.getSecurityManager();
     if (sm != null) {
@@ -81,43 +83,43 @@ public class TableManager {
       tableManager = new TableManager();
     return tableManager;
   }
-  
+
   private TableManager() {
     instance = HdfsZooInstance.getInstance();
     zooStateCache = new ZooCache(new TableStateWatcher());
     updateTableStateCache();
   }
-  
+
   public TableState getTableState(String tableId) {
     return tableStateCache.get(tableId);
   }
-  
+
   public static class IllegalTableTransitionException extends Exception {
     private static final long serialVersionUID = 1L;
-    
+
     final TableState oldState;
     final TableState newState;
-    
+
     public IllegalTableTransitionException(TableState oldState, TableState newState) {
       this.oldState = oldState;
       this.newState = newState;
     }
-    
+
     public TableState getOldState() {
       return oldState;
     }
-    
+
     public TableState getNewState() {
       return newState;
     }
-    
+
   }
-  
+
   public synchronized void transitionTableState(final String tableId, final TableState newState) {
     String statePath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_STATE;
-    
+
     try {
-      ZooReaderWriter.getRetryingInstance().mutate(statePath, (byte[]) newState.name().getBytes(), ZooUtil.PUBLIC, new Mutator() {
+      ZooReaderWriter.getRetryingInstance().mutate(statePath, newState.name().getBytes(), ZooUtil.PUBLIC, new Mutator() {
         @Override
         public byte[] mutate(byte[] oldData) throws Exception {
           TableState oldState = TableState.UNKNOWN;
@@ -152,7 +154,7 @@ public class TableManager {
       throw new RuntimeException(e);
     }
   }
-  
+
   private void updateTableStateCache() {
     synchronized (tableStateCache) {
       for (String tableId : zooStateCache.getChildren(ZooUtil.getRoot(instance) + Constants.ZTABLES))
@@ -160,7 +162,7 @@ public class TableManager {
           updateTableStateCache(tableId);
     }
   }
-  
+
   public TableState updateTableStateCache(String tableId) {
     synchronized (tableStateCache) {
       TableState tState = TableState.UNKNOWN;
@@ -177,29 +179,29 @@ public class TableManager {
       return tState;
     }
   }
-  
+
   public void addTable(String tableId, String tableName, NodeExistsPolicy existsPolicy) throws KeeperException, InterruptedException {
     prepareNewTableState(instance.getInstanceID(), tableId, tableName, TableState.NEW, existsPolicy);
     updateTableStateCache(tableId);
   }
-  
+
   public void cloneTable(String srcTable, String tableId, String tableName, Map<String,String> propertiesToSet, Set<String> propertiesToExclude,
       NodeExistsPolicy existsPolicy) throws KeeperException, InterruptedException {
     prepareNewTableState(instance.getInstanceID(), tableId, tableName, TableState.NEW, existsPolicy);
     String srcTablePath = Constants.ZROOT + "/" + instance.getInstanceID() + Constants.ZTABLES + "/" + srcTable + Constants.ZTABLE_CONF;
     String newTablePath = Constants.ZROOT + "/" + instance.getInstanceID() + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_CONF;
     ZooReaderWriter.getRetryingInstance().recursiveCopyPersistent(srcTablePath, newTablePath, NodeExistsPolicy.OVERWRITE);
-    
+
     for (Entry<String,String> entry : propertiesToSet.entrySet())
       TablePropUtil.setTableProperty(tableId, entry.getKey(), entry.getValue());
-    
+
     for (String prop : propertiesToExclude)
       ZooReaderWriter.getRetryingInstance().recursiveDelete(
           Constants.ZROOT + "/" + instance.getInstanceID() + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_CONF + "/" + prop, NodeMissingPolicy.SKIP);
-    
+
     updateTableStateCache(tableId);
   }
-  
+
   public void removeTable(String tableId) throws KeeperException, InterruptedException {
     synchronized (tableStateCache) {
       tableStateCache.remove(tableId);
@@ -208,7 +210,7 @@ public class TableManager {
       ZooReaderWriter.getRetryingInstance().recursiveDelete(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + tableId, NodeMissingPolicy.SKIP);
     }
   }
-  
+
   public boolean addObserver(TableObserver to) {
     synchronized (observers) {
       synchronized (tableStateCache) {
@@ -217,23 +219,23 @@ public class TableManager {
       }
     }
   }
-  
+
   public boolean removeObserver(TableObserver to) {
     return observers.remove(to);
   }
-  
+
   private class TableStateWatcher implements Watcher {
     @Override
     public void process(WatchedEvent event) {
       if (log.isTraceEnabled())
         log.trace(event);
-      
+
       final String zPath = event.getPath();
       final EventType zType = event.getType();
-      
+
       String tablesPrefix = ZooUtil.getRoot(instance) + Constants.ZTABLES;
       String tableId = null;
-      
+
       if (zPath != null && zPath.startsWith(tablesPrefix + "/")) {
         String suffix = zPath.substring(tablesPrefix.length() + 1);
         if (suffix.contains("/")) {
@@ -246,7 +248,7 @@ public class TableManager {
           return;
         }
       }
-      
+
       switch (zType) {
         case NodeChildrenChanged:
           if (zPath != null && zPath.equals(tablesPrefix)) {
@@ -293,48 +295,48 @@ public class TableManager {
       }
     }
   }
-  
+
   public void addNamespace(String namespaceId, String namespace, NodeExistsPolicy existsPolicy) throws KeeperException, InterruptedException {
     // state gets created last
     String zPath = Constants.ZROOT + "/" + instance.getInstanceID() + Constants.ZNAMESPACES + "/" + namespaceId;
-    
+
     IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
-    
+
     zoo.putPersistentData(zPath, new byte[0], existsPolicy);
     zoo.putPersistentData(zPath + Constants.ZNAMESPACE_NAME, namespace.getBytes(Constants.UTF8), existsPolicy);
     zoo.putPersistentData(zPath + Constants.ZNAMESPACE_CONF, new byte[0], existsPolicy);
   }
-  
+
   public void removeNamespace(String namespaceId) throws KeeperException, InterruptedException {
     ZooReaderWriter.getRetryingInstance().recursiveDelete(ZooUtil.getRoot(instance) + Constants.ZNAMESPACES + "/" + namespaceId, NodeMissingPolicy.SKIP);
   }
-  
+
   public void addNamespaceToTable(String tableId, String namespaceId) throws KeeperException, InterruptedException {
     String zPath = Constants.ZROOT + "/" + instance.getInstanceID() + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_NAMESPACE;
     ZooReaderWriter.getRetryingInstance().putPersistentData(zPath, namespaceId.getBytes(Constants.UTF8), NodeExistsPolicy.OVERWRITE);
   }
-  
+
   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_NAMESPACE_ID.getBytes(Constants.UTF8), NodeExistsPolicy.OVERWRITE);
   }
-  
+
   public void cloneNamespace(String srcId, String newId, String namespaceName, Map<String,String> propertiesToSet, Set<String> propertiesToExclude,
       NodeExistsPolicy existsPolicy) throws KeeperException, InterruptedException {
     String srcPath = Constants.ZROOT + "/" + instance.getInstanceID() + Constants.ZNAMESPACES + "/" + srcId + Constants.ZNAMESPACE_CONF;
     String newPath = Constants.ZROOT + "/" + instance.getInstanceID() + Constants.ZNAMESPACES + "/" + newId + Constants.ZNAMESPACE_CONF;
     ZooReaderWriter.getRetryingInstance().recursiveCopyPersistent(srcPath, newPath, NodeExistsPolicy.OVERWRITE);
-    
+
     for (Entry<String,String> entry : propertiesToSet.entrySet())
       NamespacePropUtil.setNamespaceProperty(newId, entry.getKey(), entry.getValue());
-    
+
     for (String prop : propertiesToExclude)
       ZooReaderWriter.getRetryingInstance().recursiveDelete(
           Constants.ZROOT + "/" + instance.getInstanceID() + Constants.ZNAMESPACES + "/" + newId + Constants.ZNAMESPACE_CONF + "/" + prop,
           NodeMissingPolicy.SKIP);
   }
-  
+
   /*
    * private static boolean verifyTabletAssignments(String tableId) { log.info( "Sending message to load balancer to verify assignment of tablets with tableId="
    * + tableId); // Return true only if transitions to other states did not interrupt // this process. (like deleting the table) return true; }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
index a3b8011..f61955a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
@@ -346,7 +346,7 @@ public class MetadataTableUtil {
     Mutation m = null;
     ms.setRange(new KeyExtent(tableIdText, null, null).toMetadataRange());
 
-    // insert deletes before deleting data from !METADATA... this makes the code fault tolerant
+    // insert deletes before deleting data from metadata... this makes the code fault tolerant
     if (insertDeletes) {
 
       ms.fetchColumnFamily(DataFileColumnFamily.NAME);
@@ -528,7 +528,7 @@ public class MetadataTableUtil {
         LogEntry e = new LogEntry();
         try {
           e.fromBytes(zoo.getData(root + "/" + child, null));
-          // upgrade from !0;!0<< -> !!R<<
+          // upgrade from !0;!0<< -> +r<<
           e.extent = RootTable.EXTENT;
           result.add(e);
         } catch (KeeperException.NoNodeException ex) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
----------------------------------------------------------------------
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
index ca1fb03..9286a37 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
@@ -305,16 +305,14 @@ public class SimpleGarbageCollector implements Iface {
                   ++status.current.deleted;
                 }
               } else if (fs.exists(fullPath)) {
-                // leave the entry in the METADATA table; we'll try again
-                // later
+                // leave the entry in the metadata; we'll try again later
                 removeFlag = false;
                 synchronized (SimpleGarbageCollector.this) {
                   ++status.current.errors;
                 }
                 log.warn("File exists, but was not deleted for an unknown reason: " + fullPath);
               } else {
-                // this failure, we still want to remove the METADATA table
-                // entry
+                // this failure, we still want to remove the metadata entry
                 removeFlag = true;
                 synchronized (SimpleGarbageCollector.this) {
                   ++status.current.errors;
@@ -470,7 +468,7 @@ public class SimpleGarbageCollector implements Iface {
       }
       gcSpan.stop();
       
-      // we just made a lot of changes to the !METADATA table: flush them out
+      // we just made a lot of metadata changes: flush them out
       try {
         Connector connector = instance.getConnector(credentials.getPrincipal(), credentials.getToken());
         connector.tableOperations().compact(MetadataTable.NAME, null, null, true, true);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/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 fcfdc8c..eb1a492 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
@@ -80,6 +80,7 @@ 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.TablePermission;
 import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.Daemon;
@@ -317,6 +318,8 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
         if (!Tables.exists(instance, RootTable.ID)) {
           TableManager.prepareNewTableState(instance.getInstanceID(), RootTable.ID, RootTable.NAME, TableState.ONLINE, NodeExistsPolicy.SKIP);
           Initialize.initMetadataConfig(RootTable.ID);
+          // ensure root user can flush root table
+          security.grantTablePermission(SystemCredentials.get().toThrift(instance), security.getRootUsername(), RootTable.ID, TablePermission.ALTER_TABLE);
         }
 
         moveRootTabletToRootTable(zoo);
@@ -334,7 +337,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
         // 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 systemNamespace = ZooUtil.getRoot(instance) + Constants.ZNAMESPACES + "/" + Constants.ACCUMULO_NAMESPACE_ID;
         String tables = ZooUtil.getRoot(instance) + Constants.ZTABLES;
         zoo.putPersistentData(namespaces, new byte[0], NodeExistsPolicy.SKIP);
 
@@ -344,7 +347,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
 
         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);
+        zoo.putPersistentData(systemNamespace + Constants.ZNAMESPACE_NAME, Constants.ACCUMULO_NAMESPACE.getBytes(Constants.UTF8), NodeExistsPolicy.SKIP);
 
         Map<String,String> opts = IteratorUtil.generateInitialTableProperties(true);
         for (Entry<String,String> e : opts.entrySet()) {
@@ -353,7 +356,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
 
         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_NAMESPACE_ID.getBytes(Constants.UTF8),
+            zoo.putPersistentData(tables + "/" + table.getKey() + Constants.ZTABLE_NAMESPACE, Constants.ACCUMULO_NAMESPACE_ID.getBytes(Constants.UTF8),
                 NodeExistsPolicy.SKIP);
           } else {
             zoo.putPersistentData(tables + "/" + table.getKey() + Constants.ZTABLE_NAMESPACE, Constants.DEFAULT_NAMESPACE_ID.getBytes(Constants.UTF8),
@@ -367,9 +370,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.grantNamespacePermission(user, Constants.SYSTEM_NAMESPACE_ID, NamespacePermission.READ);
+          perm.grantNamespacePermission(user, Constants.ACCUMULO_NAMESPACE_ID, NamespacePermission.READ);
         }
-        perm.grantNamespacePermission("root", Constants.SYSTEM_NAMESPACE_ID, NamespacePermission.ALTER_TABLE);
+        perm.grantNamespacePermission("root", Constants.ACCUMULO_NAMESPACE_ID, NamespacePermission.ALTER_TABLE);
 
       } catch (Exception ex) {
         log.fatal("Error performing upgrade", ex);
@@ -402,8 +405,8 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
           }
         };
 
-        // need to run this in a separate thread because a lock is held that prevents !METADATA tablets from being assigned and this task writes to the
-        // !METADATA table
+        // need to run this in a separate thread because a lock is held that prevents metadata tablets from being assigned and this task writes to the
+        // metadata table
         new Thread(upgradeTask).start();
       }
     }
@@ -461,7 +464,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
         }
         break;
       case SAFE_MODE:
-        // Count offline tablets for the METADATA table
+        // Count offline tablets for the metadata table
         for (TabletGroupWatcher watcher : watchers) {
           result += watcher.getStats(METADATA_TABLE_ID).unassigned();
         }
@@ -478,7 +481,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     return result;
   }
 
-  private void checkNotRootID(String tableId, TableOperation operation) throws ThriftTableOperationException {
+  private static void checkNotRootID(String tableId, TableOperation operation) throws ThriftTableOperationException {
     if (RootTable.ID.equals(tableId)) {
       String why = "Table name cannot be == " + RootTable.NAME;
       log.warn(why);
@@ -486,7 +489,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     }
   }
 
-  private void checkNotMetadataTable(String tableName, TableOperation operation) throws ThriftTableOperationException {
+  private static void checkNotMetadataTable(String tableName, TableOperation operation) throws ThriftTableOperationException {
     if (MetadataTable.NAME.equals(tableName) || RootTable.NAME.equals(tableName)) {
       String why = "Table names cannot be == " + RootTable.NAME + " or " + MetadataTable.NAME;
       log.warn(why);
@@ -494,17 +497,59 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     }
   }
 
+  private static void checkNotSystemNamespace(String namespace, TableOperation operation) throws ThriftTableOperationException {
+    if (Constants.ACCUMULO_NAMESPACE.equals(namespace)) {
+      String why = "Namespaces cannot be == " + Constants.ACCUMULO_NAMESPACE;
+      log.warn(why);
+      throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.OTHER, why);
+    }
+  }
+
   private void checkTableName(String tableName, TableOperation operation) throws ThriftTableOperationException {
     if (!tableName.matches(Constants.VALID_TABLE_NAME_REGEX)) {
       String why = "Table names must only contain word characters (letters, digits, and underscores): " + tableName;
       log.warn(why);
       throw new ThriftTableOperationException(null, tableName, operation, TableOperationExceptionType.OTHER, why);
     }
-    if (Tables.getNameToIdMap(HdfsZooInstance.getInstance()).containsKey(tableName)) {
+    if (Tables.getNameToIdMap(instance).containsKey(tableName)) {
       String why = "Table name already exists: " + tableName;
       throw new ThriftTableOperationException(null, tableName, operation, TableOperationExceptionType.EXISTS, why);
     }
+  }
 
+  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 (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 = Namespaces.getNameToIdMap(instance).get(namespace);
+    if (namespaceId == null)
+      throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.NOTFOUND, null);
+    return namespaceId;
+  }
+
+  private void authenticate(TCredentials c) throws ThriftSecurityException {
+    if (!security.authenticateUser(c, c))
+      throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.BAD_CREDENTIALS);
+
+  }
+
+  private void updatePlugins(String property) {
+    if (property.equals(Property.MASTER_TABLET_BALANCER.getKey())) {
+      TabletBalancer balancer = ServerConfiguration.getSystemConfiguration(instance).instantiateClassProperty(Property.MASTER_TABLET_BALANCER,
+          TabletBalancer.class, new DefaultLoadBalancer());
+      balancer.init(serverConfig);
+      tabletBalancer = balancer;
+      log.info("tablet balancer changed to " + tabletBalancer.getClass().getName());
+    }
   }
 
   public void mustBeOnline(final String tableId) throws ThriftTableOperationException {
@@ -513,6 +558,62 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       throw new ThriftTableOperationException(tableId, null, TableOperation.MERGE, TableOperationExceptionType.OFFLINE, "table is not online");
   }
 
+  private void alterTableProperty(TCredentials c, String tableName, String property, String value, TableOperation op) throws ThriftSecurityException,
+      ThriftTableOperationException {
+    final String tableId = checkTableId(tableName, op);
+    if (!security.canAlterTable(c, tableId))
+      throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
+    try {
+      if (value == null || value.isEmpty()) {
+        TablePropUtil.removeTableProperty(tableId, property);
+      } else if (!TablePropUtil.setTableProperty(tableId, property, value)) {
+        throw new Exception("Invalid table property.");
+      }
+    } catch (KeeperException.NoNodeException e) {
+      // race condition... table no longer exists? This call will throw an exception if the table was deleted:
+      checkTableId(tableName, op);
+      log.info("Error altering table property", e);
+      throw new ThriftTableOperationException(tableId, tableName, op, TableOperationExceptionType.OTHER, "Problem altering table property");
+    } catch (Exception e) {
+      log.error("Problem altering table property", e);
+      throw new ThriftTableOperationException(tableId, tableName, op, TableOperationExceptionType.OTHER, "Problem altering table property");
+    }
+  }
+
+  protected String checkTableId(String tableName, TableOperation operation) throws ThriftTableOperationException {
+    final String tableId = Tables.getNameToIdMap(getConfiguration().getInstance()).get(tableName);
+    if (tableId == null)
+      throw new ThriftTableOperationException(null, tableName, operation, TableOperationExceptionType.NOTFOUND, null);
+    return tableId;
+  }
+
+  private void alterNamespaceProperty(TCredentials c, String namespace, String property, String value, TableOperation op) throws ThriftSecurityException,
+      ThriftTableOperationException {
+
+    String namespaceId = null;
+    namespaceId = checkNamespaceId(namespace, op);
+
+    if (!security.canAlterNamespace(c, namespaceId))
+      throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
+    try {
+      if (value == null) {
+        NamespacePropUtil.removeNamespaceProperty(namespaceId, property);
+      } else {
+        NamespacePropUtil.setNamespaceProperty(namespaceId, property, value);
+      }
+    } catch (KeeperException.NoNodeException e) {
+      // race condition... namespace no longer exists? This call will throw an exception if the namespace was deleted:
+      checkNamespaceId(namespaceId, op);
+      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 namespace property", e);
+      throw new ThriftTableOperationException(namespaceId, namespace, op, TableOperationExceptionType.OTHER, "Problem altering namespace property");
+    }
+  }
+
   public Connector getConnector() throws AccumuloException, AccumuloSecurityException {
     return instance.getConnector(SystemCredentials.get().getPrincipal(), SystemCredentials.get().getToken());
   }
@@ -548,13 +649,6 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
 
   private class MasterClientServiceHandler implements MasterClientService.Iface {
 
-    protected String checkTableId(String tableName, TableOperation operation) throws ThriftTableOperationException {
-      final String tableId = Tables.getNameToIdMap(getConfiguration().getInstance()).get(tableName);
-      if (tableId == null)
-        throw new ThriftTableOperationException(null, tableName, operation, TableOperationExceptionType.NOTFOUND, null);
-      return tableId;
-    }
-
     @Override
     public long initiateFlush(TInfo tinfo, TCredentials c, String tableId) throws ThriftSecurityException, ThriftTableOperationException, TException {
       security.canFlush(c, tableId);
@@ -724,55 +818,6 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       return result;
     }
 
-    private void alterTableProperty(TCredentials c, String tableName, String property, String value, TableOperation op) throws ThriftSecurityException,
-        ThriftTableOperationException {
-      final String tableId = checkTableId(tableName, op);
-      if (!security.canAlterTable(c, tableId))
-        throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-
-      try {
-        if (value == null || value.isEmpty()) {
-          TablePropUtil.removeTableProperty(tableId, property);
-        } else if (!TablePropUtil.setTableProperty(tableId, property, value)) {
-          throw new Exception("Invalid table property.");
-        }
-      } catch (KeeperException.NoNodeException e) {
-        // race condition... table no longer exists? This call will throw an exception if the table was deleted:
-        checkTableId(tableName, op);
-        log.info("Error altering table property", e);
-        throw new ThriftTableOperationException(tableId, tableName, op, TableOperationExceptionType.OTHER, "Problem altering table property");
-      } catch (Exception e) {
-        log.error("Problem altering table property", e);
-        throw new ThriftTableOperationException(tableId, tableName, op, TableOperationExceptionType.OTHER, "Problem altering table property");
-      }
-    }
-
-    private void alterNamespaceProperty(TCredentials c, String namespace, String property, String value, TableOperation op) throws ThriftSecurityException,
-        ThriftTableOperationException {
-
-      String namespaceId = null;
-      namespaceId = checkNamespaceId(namespace, op);
-
-      if (!security.canAlterNamespace(c, namespaceId))
-        throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-
-      try {
-        if (value == null) {
-          NamespacePropUtil.removeNamespaceProperty(namespaceId, property);
-        } else {
-          NamespacePropUtil.setNamespaceProperty(namespaceId, property, value);
-        }
-      } catch (KeeperException.NoNodeException e) {
-        // race condition... namespace no longer exists? This call will throw an exception if the namespace was deleted:
-        checkNamespaceId(namespaceId, op);
-        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 namespace property", e);
-        throw new ThriftTableOperationException(namespaceId, namespace, op, TableOperationExceptionType.OTHER, "Problem altering namespace property");
-      }
-    }
-
     @Override
     public void removeTableProperty(TInfo info, TCredentials credentials, String tableName, String property) throws ThriftSecurityException,
         ThriftTableOperationException, TException {
@@ -860,17 +905,6 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       Master.this.setMasterGoalState(state);
     }
 
-    private void updatePlugins(String property) {
-      if (property.equals(Property.MASTER_TABLET_BALANCER.getKey())) {
-        TabletBalancer balancer = ServerConfiguration.getSystemConfiguration(instance).instantiateClassProperty(Property.MASTER_TABLET_BALANCER,
-            TabletBalancer.class,
-            new DefaultLoadBalancer());
-        balancer.init(serverConfig);
-        tabletBalancer = balancer;
-        log.info("tablet balancer changed to " + tabletBalancer.getClass().getName());
-      }
-    }
-
     @Override
     public void removeSystemProperty(TInfo info, TCredentials c, String property) throws ThriftSecurityException, TException {
       security.canPerformSystemActions(c);
@@ -897,12 +931,6 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       }
     }
 
-    private void authenticate(TCredentials c) throws ThriftSecurityException {
-      if (!security.authenticateUser(c, c))
-        throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.BAD_CREDENTIALS);
-
-    }
-
     @Override
     public long beginTableOperation(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException, TException {
       authenticate(credentials);
@@ -1192,33 +1220,6 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
 
     }
 
-    private void checkNotSystemNamespace(String namespace, TableOperation operation) throws ThriftTableOperationException {
-      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 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 (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 = Namespaces.getNameToIdMap(getConfiguration().getInstance()).get(namespace);
-      if (namespaceId == null)
-        throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.NOTFOUND, null);
-      return namespaceId;
-    }
-
     @Override
     public String waitForNamespaceOperation(TInfo tinfo, TCredentials credentials, long opid) throws ThriftSecurityException, ThriftTableOperationException,
         TException {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/server/master/src/main/java/org/apache/accumulo/master/tableOps/BulkImport.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/BulkImport.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/BulkImport.java
index a340b50..2426898 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/BulkImport.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/BulkImport.java
@@ -545,7 +545,7 @@ class LoadFiles extends MasterRepo {
             try {
               // get a connection to a random tablet server, do not prefer cached connections because
               // this is running on the master and there are lots of connections to tablet servers
-              // serving the !METADATA tablets
+              // serving the metadata tablets
               long timeInMillis = master.getConfiguration().getConfiguration().getTimeInMillis(Property.MASTER_BULK_TIMEOUT);
               Pair<String,Client> pair = ServerClient.getConnection(master.getInstance(), false, timeInMillis);
               client = pair.getSecond();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/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 59da70f..534a6c8 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
@@ -39,9 +39,9 @@ import org.apache.accumulo.server.util.MetadataTableUtil;
 import org.apache.log4j.Logger;
 
 class CloneInfo implements Serializable {
-  
+
   private static final long serialVersionUID = 1L;
-  
+
   String srcTableId;
   String tableName;
   String tableId;
@@ -49,65 +49,65 @@ class CloneInfo implements Serializable {
   String srcNamespaceId;
   Map<String,String> propertiesToSet;
   Set<String> propertiesToExclude;
-  
+
   public String user;
 }
 
 class FinishCloneTable extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
   private CloneInfo cloneInfo;
-  
+
   public FinishCloneTable(CloneInfo cloneInfo) {
     this.cloneInfo = cloneInfo;
   }
-  
+
   @Override
   public long isReady(long tid, Master environment) throws Exception {
     return 0;
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master environment) throws Exception {
     // directories are intentionally not created.... this is done because directories should be unique
     // because they occupy a different namespace than normal tablet directories... also some clones
     // may never create files.. therefore there is no need to consume namenode space w/ directories
     // that are not used... tablet will create directories as needed
-    
+
     TableManager.getInstance().transitionTableState(cloneInfo.tableId, TableState.ONLINE);
-    
+
     Utils.unreserveNamespace(cloneInfo.srcNamespaceId, tid, false);
     if (!cloneInfo.srcNamespaceId.equals(cloneInfo.namespaceId))
       Utils.unreserveNamespace(cloneInfo.namespaceId, tid, false);
     Utils.unreserveTable(cloneInfo.srcTableId, tid, false);
     Utils.unreserveTable(cloneInfo.tableId, tid, true);
-    
+
     environment.getEventCoordinator().event("Cloned table %s from %s", cloneInfo.tableName, cloneInfo.srcTableId);
-    
+
     Logger.getLogger(FinishCloneTable.class).debug("Cloned table " + cloneInfo.srcTableId + " " + cloneInfo.tableId + " " + cloneInfo.tableName);
-    
+
     return null;
   }
-  
+
   @Override
   public void undo(long tid, Master environment) throws Exception {}
-  
+
 }
 
 class CloneMetadata extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
   private CloneInfo cloneInfo;
-  
+
   public CloneMetadata(CloneInfo cloneInfo) {
     this.cloneInfo = cloneInfo;
   }
-  
+
   @Override
   public long isReady(long tid, Master environment) throws Exception {
     return 0;
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master environment) throws Exception {
     Logger.getLogger(CloneMetadata.class).info(
@@ -119,26 +119,26 @@ class CloneMetadata extends MasterRepo {
     MetadataTableUtil.cloneTable(instance, cloneInfo.srcTableId, cloneInfo.tableId, environment.getFileSystem());
     return new FinishCloneTable(cloneInfo);
   }
-  
+
   @Override
   public void undo(long tid, Master environment) throws Exception {
     MetadataTableUtil.deleteTable(cloneInfo.tableId, false, SystemCredentials.get(), environment.getMasterLock());
   }
-  
+
 }
 
 class CloneZookeeper extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
-  
+
   private CloneInfo cloneInfo;
-  
+
   public CloneZookeeper(CloneInfo cloneInfo) throws NamespaceNotFoundException {
     this.cloneInfo = cloneInfo;
     Instance inst = HdfsZooInstance.getInstance();
-    this.cloneInfo.namespaceId = Namespaces.getNamespaceId(inst, Tables.extractNamespace(this.cloneInfo.tableName));
+    this.cloneInfo.namespaceId = Namespaces.getNamespaceId(inst, Tables.qualify(this.cloneInfo.tableName).getFirst());
   }
-  
+
   @Override
   public long isReady(long tid, Master environment) throws Exception {
     long val = 0;
@@ -147,28 +147,28 @@ class CloneZookeeper extends MasterRepo {
     val += Utils.reserveTable(cloneInfo.tableId, tid, true, false, TableOperation.CLONE);
     return val;
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master environment) throws Exception {
     Utils.tableNameLock.lock();
     try {
       // write tableName & tableId to zookeeper
       Instance instance = HdfsZooInstance.getInstance();
-      
+
       Utils.checkTableDoesNotExist(instance, cloneInfo.tableName, cloneInfo.tableId, TableOperation.CLONE);
-      
+
       TableManager.getInstance().cloneTable(cloneInfo.srcTableId, cloneInfo.tableId, cloneInfo.tableName, cloneInfo.propertiesToSet,
           cloneInfo.propertiesToExclude, NodeExistsPolicy.OVERWRITE);
       Tables.clearCache(instance);
-      
+
       TableManager.getInstance().addNamespaceToTable(cloneInfo.tableId, cloneInfo.namespaceId);
-      
+
       return new CloneMetadata(cloneInfo);
     } finally {
       Utils.tableNameLock.unlock();
     }
   }
-  
+
   @Override
   public void undo(long tid, Master environment) throws Exception {
     Instance instance = HdfsZooInstance.getInstance();
@@ -178,24 +178,24 @@ class CloneZookeeper extends MasterRepo {
     Utils.unreserveTable(cloneInfo.tableId, tid, true);
     Tables.clearCache(instance);
   }
-  
+
 }
 
 class ClonePermissions extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
-  
+
   private CloneInfo cloneInfo;
-  
+
   public ClonePermissions(CloneInfo cloneInfo) {
     this.cloneInfo = cloneInfo;
   }
-  
+
   @Override
   public long isReady(long tid, Master environment) throws Exception {
     return 0;
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master environment) throws Exception {
     // give all table permissions to the creator
@@ -208,13 +208,13 @@ class ClonePermissions extends MasterRepo {
         throw e;
       }
     }
-    
+
     // setup permissions in zookeeper before table info in zookeeper
     // this way concurrent users will not get a spurious pemission denied
     // error
     return new CloneZookeeper(cloneInfo);
   }
-  
+
   @Override
   public void undo(long tid, Master environment) throws Exception {
     AuditedSecurityOperation.getInstance().deleteTable(SystemCredentials.get().toThrift(environment.getInstance()), cloneInfo.tableId);
@@ -222,10 +222,10 @@ class ClonePermissions extends MasterRepo {
 }
 
 public class CloneTable extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
   private CloneInfo cloneInfo;
-  
+
   public CloneTable(String user, String srcTableId, String tableName, Map<String,String> propertiesToSet, Set<String> propertiesToExclude) {
     cloneInfo = new CloneInfo();
     cloneInfo.user = user;
@@ -236,18 +236,18 @@ public class CloneTable extends MasterRepo {
     Instance inst = HdfsZooInstance.getInstance();
     cloneInfo.srcNamespaceId = Tables.getNamespace(inst, cloneInfo.srcTableId);
   }
-  
+
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    
+
     long val = Utils.reserveNamespace(cloneInfo.srcNamespaceId, tid, false, true, TableOperation.CLONE);
     val += Utils.reserveTable(cloneInfo.srcTableId, tid, false, true, TableOperation.CLONE);
     return val;
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master environment) throws Exception {
-    
+
     Utils.idLock.lock();
     try {
       Instance instance = HdfsZooInstance.getInstance();
@@ -257,11 +257,11 @@ public class CloneTable extends MasterRepo {
       Utils.idLock.unlock();
     }
   }
-  
+
   @Override
   public void undo(long tid, Master environment) throws Exception {
     Utils.unreserveNamespace(cloneInfo.srcNamespaceId, tid, false);
     Utils.unreserveTable(cloneInfo.srcTableId, tid, false);
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/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 09c1c11..671bf23 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
@@ -49,111 +49,111 @@ import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
 
 class TableInfo implements Serializable {
-  
+
   private static final long serialVersionUID = 1L;
-  
+
   String tableName;
   String tableId;
   String namespaceId;
   char timeType;
   String user;
-  
+
   public Map<String,String> props;
 
   public String dir = null;
 }
 
 class FinishCreateTable extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
-  
+
   private TableInfo tableInfo;
-  
+
   public FinishCreateTable(TableInfo ti) {
     this.tableInfo = ti;
   }
-  
+
   @Override
   public long isReady(long tid, Master environment) throws Exception {
     return 0;
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master env) throws Exception {
     TableManager.getInstance().transitionTableState(tableInfo.tableId, TableState.ONLINE);
-    
+
     Utils.unreserveNamespace(tableInfo.namespaceId, tid, false);
     Utils.unreserveTable(tableInfo.tableId, tid, true);
-    
+
     env.getEventCoordinator().event("Created table %s ", tableInfo.tableName);
-    
+
     Logger.getLogger(FinishCreateTable.class).debug("Created table " + tableInfo.tableId + " " + tableInfo.tableName);
-    
+
     return null;
   }
-  
+
   @Override
   public String getReturn() {
     return tableInfo.tableId;
   }
-  
+
   @Override
   public void undo(long tid, Master env) throws Exception {}
-  
+
 }
 
 class PopulateMetadata extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
-  
+
   private TableInfo tableInfo;
-  
+
   PopulateMetadata(TableInfo ti) {
     this.tableInfo = ti;
   }
-  
+
   @Override
   public long isReady(long tid, Master environment) throws Exception {
     return 0;
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master environment) throws Exception {
     KeyExtent extent = new KeyExtent(new Text(tableInfo.tableId), null, null);
     MetadataTableUtil.addTablet(extent, tableInfo.dir, SystemCredentials.get(), tableInfo.timeType, environment.getMasterLock());
-    
+
     return new FinishCreateTable(tableInfo);
-    
+
   }
-  
+
   @Override
   public void undo(long tid, Master environment) throws Exception {
     MetadataTableUtil.deleteTable(tableInfo.tableId, false, SystemCredentials.get(), environment.getMasterLock());
   }
-  
+
 }
 
 class CreateDir extends MasterRepo {
   private static final long serialVersionUID = 1L;
-  
+
   private TableInfo tableInfo;
-  
+
   CreateDir(TableInfo ti) {
     this.tableInfo = ti;
   }
-  
+
   @Override
   public long isReady(long tid, Master environment) throws Exception {
     return 0;
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master master) throws Exception {
     VolumeManager fs = master.getFileSystem();
     fs.mkdirs(new Path(tableInfo.dir));
     return new PopulateMetadata(tableInfo);
   }
-  
+
   @Override
   public void undo(long tid, Master master) throws Exception {
     VolumeManager fs = master.getFileSystem();
@@ -185,51 +185,51 @@ class ChooseDir extends MasterRepo {
 
   @Override
   public void undo(long tid, Master master) throws Exception {
-    
+
   }
 }
 
 class PopulateZookeeper extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
-  
+
   private TableInfo tableInfo;
-  
+
   PopulateZookeeper(TableInfo ti) {
     this.tableInfo = ti;
   }
-  
+
   @Override
   public long isReady(long tid, Master environment) throws Exception {
     return Utils.reserveTable(tableInfo.tableId, tid, true, false, TableOperation.CREATE);
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master master) throws Exception {
     // reserve the table name in zookeeper or fail
-    
+
     Utils.tableNameLock.lock();
     try {
       // write tableName & tableId to zookeeper
       Instance instance = master.getInstance();
-      
+
       Utils.checkTableDoesNotExist(instance, tableInfo.tableName, tableInfo.tableId, TableOperation.CREATE);
-      
+
       TableManager.getInstance().addTable(tableInfo.tableId, tableInfo.tableName, NodeExistsPolicy.OVERWRITE);
-      
+
       TableManager.getInstance().addNamespaceToTable(tableInfo.tableId, tableInfo.namespaceId);
-      
+
       for (Entry<String,String> entry : tableInfo.props.entrySet())
         TablePropUtil.setTableProperty(tableInfo.tableId, entry.getKey(), entry.getValue());
-      
+
       Tables.clearCache(instance);
       return new ChooseDir(tableInfo);
     } finally {
       Utils.tableNameLock.unlock();
     }
-    
+
   }
-  
+
   @Override
   public void undo(long tid, Master master) throws Exception {
     Instance instance = master.getInstance();
@@ -237,19 +237,19 @@ class PopulateZookeeper extends MasterRepo {
     Utils.unreserveTable(tableInfo.tableId, tid, true);
     Tables.clearCache(instance);
   }
-  
+
 }
 
 class SetupPermissions extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
-  
+
   private TableInfo tableInfo;
-  
+
   public SetupPermissions(TableInfo ti) {
     this.tableInfo = ti;
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master env) throws Exception {
     // give all table permissions to the creator
@@ -262,25 +262,25 @@ class SetupPermissions extends MasterRepo {
         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 PopulateZookeeper(tableInfo);
   }
-  
+
   @Override
   public void undo(long tid, Master env) throws Exception {
     AuditedSecurityOperation.getInstance().deleteTable(SystemCredentials.get().toThrift(env.getInstance()), tableInfo.tableId);
   }
-  
+
 }
 
 public class CreateTable extends MasterRepo {
   private static final long serialVersionUID = 1L;
-  
+
   private TableInfo tableInfo;
-  
+
   public CreateTable(String user, String tableName, TimeType timeType, Map<String,String> props) throws NamespaceNotFoundException {
     tableInfo = new TableInfo();
     tableInfo.tableName = tableName;
@@ -288,23 +288,23 @@ public class CreateTable extends MasterRepo {
     tableInfo.user = user;
     tableInfo.props = props;
     Instance inst = HdfsZooInstance.getInstance();
-    tableInfo.namespaceId = Namespaces.getNamespaceId(inst, Tables.extractNamespace(tableInfo.tableName));
+    tableInfo.namespaceId = Namespaces.getNamespaceId(inst, Tables.qualify(tableInfo.tableName).getFirst());
   }
-  
+
   @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.reserveNamespace(tableInfo.namespaceId, tid, false, true, TableOperation.CREATE);
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master master) throws Exception {
     // first step is to reserve a table id.. if the machine fails during this step
     // it is ok to retry... the only side effect is that a table id may not be used
     // or skipped
-    
+
     // assuming only the master process is creating tables
-    
+
     Utils.idLock.lock();
     try {
       tableInfo.tableId = Utils.getNextTableId(tableInfo.tableName, master.getInstance());
@@ -312,12 +312,12 @@ public class CreateTable extends MasterRepo {
     } finally {
       Utils.idLock.unlock();
     }
-    
+
   }
-  
+
   @Override
   public void undo(long tid, Master env) throws Exception {
     Utils.unreserveNamespace(tableInfo.namespaceId, tid, false);
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/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 6875bb5..5ddf129 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
@@ -76,9 +76,9 @@ import org.apache.log4j.Logger;
  * 
  */
 class ImportedTableInfo implements Serializable {
-  
+
   private static final long serialVersionUID = 1L;
-  
+
   public String exportDir;
   public String user;
   public String tableName;
@@ -88,82 +88,82 @@ class ImportedTableInfo implements Serializable {
 }
 
 class FinishImportTable extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
-  
+
   private ImportedTableInfo tableInfo;
-  
+
   public FinishImportTable(ImportedTableInfo ti) {
     this.tableInfo = ti;
   }
-  
+
   @Override
   public long isReady(long tid, Master environment) throws Exception {
     return 0;
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master env) throws Exception {
-    
+
     env.getFileSystem().deleteRecursively(new Path(tableInfo.importDir, "mappings.txt"));
-    
+
     TableManager.getInstance().transitionTableState(tableInfo.tableId, TableState.ONLINE);
-    
+
     Utils.unreserveNamespace(tableInfo.namespaceId, tid, false);
     Utils.unreserveTable(tableInfo.tableId, tid, true);
-    
+
     Utils.unreserveHdfsDirectory(new Path(tableInfo.exportDir).toString(), tid);
-    
+
     env.getEventCoordinator().event("Imported table %s ", tableInfo.tableName);
-    
+
     Logger.getLogger(FinishImportTable.class).debug("Imported table " + tableInfo.tableId + " " + tableInfo.tableName);
-    
+
     return null;
   }
-  
+
   @Override
   public String getReturn() {
     return tableInfo.tableId;
   }
-  
+
   @Override
   public void undo(long tid, Master env) throws Exception {}
-  
+
 }
 
 class MoveExportedFiles extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
-  
+
   private ImportedTableInfo tableInfo;
-  
+
   MoveExportedFiles(ImportedTableInfo ti) {
     this.tableInfo = ti;
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master master) throws Exception {
     try {
       VolumeManager fs = master.getFileSystem();
-      
+
       Map<String,String> fileNameMappings = PopulateMetadataTable.readMappingFile(fs, tableInfo);
-      
+
       for (String oldFileName : fileNameMappings.keySet()) {
         if (!fs.exists(new Path(tableInfo.exportDir, oldFileName))) {
           throw new ThriftTableOperationException(tableInfo.tableId, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
               "File referenced by exported table does not exists " + oldFileName);
         }
       }
-      
+
       FileStatus[] files = fs.listStatus(new Path(tableInfo.exportDir));
-      
+
       for (FileStatus fileStatus : files) {
         String newName = fileNameMappings.get(fileStatus.getPath().getName());
-        
+
         if (newName != null)
           fs.rename(fileStatus.getPath(), new Path(tableInfo.importDir, newName));
       }
-      
+
       return new FinishImportTable(tableInfo);
     } catch (IOException ioe) {
       log.warn(ioe.getMessage(), ioe);
@@ -174,78 +174,78 @@ class MoveExportedFiles extends MasterRepo {
 }
 
 class PopulateMetadataTable extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
-  
+
   private ImportedTableInfo tableInfo;
-  
+
   PopulateMetadataTable(ImportedTableInfo ti) {
     this.tableInfo = ti;
   }
-  
+
   static Map<String,String> readMappingFile(VolumeManager fs, ImportedTableInfo tableInfo) throws Exception {
     BufferedReader in = new BufferedReader(new InputStreamReader(fs.open(new Path(tableInfo.importDir, "mappings.txt"))));
-    
+
     try {
       Map<String,String> map = new HashMap<String,String>();
-      
+
       String line = null;
       while ((line = in.readLine()) != null) {
         String sa[] = line.split(":", 2);
         map.put(sa[0], sa[1]);
       }
-      
+
       return map;
     } finally {
       in.close();
     }
-    
+
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master master) throws Exception {
-    
+
     Path path = new Path(tableInfo.exportDir, Constants.EXPORT_FILE);
-    
+
     BatchWriter mbw = null;
     ZipInputStream zis = null;
-    
+
     try {
       VolumeManager fs = master.getFileSystem();
-      
+
       mbw = master.getConnector().createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-      
+
       zis = new ZipInputStream(fs.open(path));
-      
+
       Map<String,String> fileNameMappings = readMappingFile(fs, tableInfo);
-      
+
       String bulkDir = new Path(tableInfo.importDir).getName();
-      
+
       ZipEntry zipEntry;
       while ((zipEntry = zis.getNextEntry()) != null) {
         if (zipEntry.getName().equals(Constants.EXPORT_METADATA_FILE)) {
           DataInputStream in = new DataInputStream(new BufferedInputStream(zis));
-          
+
           Key key = new Key();
           Value val = new Value();
-          
+
           Mutation m = null;
           Text currentRow = null;
           int dirCount = 0;
-          
+
           while (true) {
             key.readFields(in);
             val.readFields(in);
-            
+
             Text endRow = new KeyExtent(key.getRow(), (Text) null).getEndRow();
             Text metadataRow = new KeyExtent(new Text(tableInfo.tableId), endRow, null).getMetadataEntry();
-            
+
             Text cq;
-            
+
             if (key.getColumnFamily().equals(DataFileColumnFamily.NAME)) {
               String oldName = new Path(key.getColumnQualifier().toString()).getName();
               String newName = fileNameMappings.get(oldName);
-              
+
               if (newName == null) {
                 throw new ThriftTableOperationException(tableInfo.tableId, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
                     "File " + oldName + " does not exist in import dir");
@@ -255,31 +255,31 @@ class PopulateMetadataTable extends MasterRepo {
             } else {
               cq = key.getColumnQualifier();
             }
-            
+
             if (m == null) {
               m = new Mutation(metadataRow);
               TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(m, new Value(FastFormat.toZeroPaddedString(dirCount++, 8, 16, "/c-".getBytes())));
               currentRow = metadataRow;
             }
-            
+
             if (!currentRow.equals(metadataRow)) {
               mbw.addMutation(m);
               m = new Mutation(metadataRow);
               TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(m, new Value(FastFormat.toZeroPaddedString(dirCount++, 8, 16, "/c-".getBytes())));
             }
-            
+
             m.put(key.getColumnFamily(), cq, val);
-            
+
             if (endRow == null && TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key)) {
               mbw.addMutation(m);
               break; // its the last column in the last row
             }
           }
-          
+
           break;
         }
       }
-      
+
       return new MoveExportedFiles(tableInfo);
     } catch (IOException ioe) {
       log.warn(ioe.getMessage(), ioe);
@@ -293,13 +293,13 @@ class PopulateMetadataTable extends MasterRepo {
           log.warn("Failed to close zip file ", ioe);
         }
       }
-      
+
       if (mbw != null) {
         mbw.close();
       }
     }
   }
-  
+
   @Override
   public void undo(long tid, Master environment) throws Exception {
     MetadataTableUtil.deleteTable(tableInfo.tableId, false, SystemCredentials.get(), environment.getMasterLock());
@@ -307,33 +307,33 @@ class PopulateMetadataTable extends MasterRepo {
 }
 
 class MapImportFileNames extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
-  
+
   private ImportedTableInfo tableInfo;
-  
+
   MapImportFileNames(ImportedTableInfo ti) {
     this.tableInfo = ti;
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master environment) throws Exception {
-    
+
     Path path = new Path(tableInfo.importDir, "mappings.txt");
-    
+
     BufferedWriter mappingsWriter = null;
-    
+
     try {
       VolumeManager fs = environment.getFileSystem();
-      
+
       fs.mkdirs(new Path(tableInfo.importDir));
-      
+
       FileStatus[] files = fs.listStatus(new Path(tableInfo.exportDir));
-      
+
       UniqueNameAllocator namer = UniqueNameAllocator.getInstance();
-      
+
       mappingsWriter = new BufferedWriter(new OutputStreamWriter(fs.create(path)));
-      
+
       for (FileStatus fileStatus : files) {
         String fileName = fileStatus.getPath().getName();
         log.info("filename " + fileStatus.getPath().toString());
@@ -341,7 +341,7 @@ class MapImportFileNames extends MasterRepo {
         String extension = "";
         if (sa.length > 1) {
           extension = sa[sa.length - 1];
-          
+
           if (!FileOperations.getValidExtensions().contains(extension)) {
             continue;
           }
@@ -349,18 +349,18 @@ class MapImportFileNames extends MasterRepo {
           // assume it is a map file
           extension = Constants.MAPFILE_EXTENSION;
         }
-        
+
         String newName = "I" + namer.getNextName() + "." + extension;
-        
+
         mappingsWriter.append(fileName);
         mappingsWriter.append(':');
         mappingsWriter.append(newName);
         mappingsWriter.newLine();
       }
-      
+
       mappingsWriter.close();
       mappingsWriter = null;
-      
+
       return new PopulateMetadataTable(tableInfo);
     } catch (IOException ioe) {
       log.warn(ioe.getMessage(), ioe);
@@ -375,7 +375,7 @@ class MapImportFileNames extends MasterRepo {
         }
     }
   }
-  
+
   @Override
   public void undo(long tid, Master env) throws Exception {
     env.getFileSystem().deleteRecursively(new Path(tableInfo.importDir));
@@ -383,50 +383,50 @@ class MapImportFileNames extends MasterRepo {
 }
 
 class CreateImportDir extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
-  
+
   private ImportedTableInfo tableInfo;
-  
+
   CreateImportDir(ImportedTableInfo ti) {
     this.tableInfo = ti;
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master master) throws Exception {
-    
+
     UniqueNameAllocator namer = UniqueNameAllocator.getInstance();
-    
+
     Path base = master.getFileSystem().matchingFileSystem(new Path(tableInfo.exportDir), ServerConstants.getTablesDirs());
     Path directory = new Path(base, tableInfo.tableId);
-    
+
     Path newBulkDir = new Path(directory, Constants.BULK_PREFIX + namer.getNextName());
-    
+
     tableInfo.importDir = newBulkDir.toString();
-    
+
     return new MapImportFileNames(tableInfo);
   }
 }
 
 class ImportPopulateZookeeper extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
-  
+
   private ImportedTableInfo tableInfo;
-  
+
   ImportPopulateZookeeper(ImportedTableInfo ti) {
     this.tableInfo = ti;
   }
-  
+
   @Override
   public long isReady(long tid, Master environment) throws Exception {
     return Utils.reserveTable(tableInfo.tableId, tid, true, false, TableOperation.IMPORT);
   }
-  
+
   private Map<String,String> getExportedProps(VolumeManager fs) throws Exception {
-    
+
     Path path = new Path(tableInfo.exportDir, Constants.EXPORT_FILE);
-    
+
     try {
       FileSystem ns = fs.getFileSystemByPath(path);
       return TableOperationsImpl.getExportedProps(ns, path);
@@ -435,38 +435,38 @@ class ImportPopulateZookeeper extends MasterRepo {
           "Error reading table props from " + path + " " + ioe.getMessage());
     }
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master env) throws Exception {
     // reserve the table name in zookeeper or fail
-    
+
     Utils.tableNameLock.lock();
     try {
       // write tableName & tableId to zookeeper
       Instance instance = HdfsZooInstance.getInstance();
-      
+
       Utils.checkTableDoesNotExist(instance, tableInfo.tableName, tableInfo.tableId, TableOperation.CREATE);
-      
+
       TableManager.getInstance().addTable(tableInfo.tableId, tableInfo.tableName, NodeExistsPolicy.OVERWRITE);
-      
-      String namespace = Tables.extractNamespace(tableInfo.tableName);
+
+      String namespace = Tables.qualify(tableInfo.tableName).getFirst();
       String namespaceId = Namespaces.getNamespaceId(instance, namespace);
       TableManager.getInstance().addNamespaceToTable(tableInfo.tableId, namespaceId);
-      
+
       Tables.clearCache(instance);
     } finally {
       Utils.tableNameLock.unlock();
     }
-    
+
     for (Entry<String,String> entry : getExportedProps(env.getFileSystem()).entrySet())
       if (!TablePropUtil.setTableProperty(tableInfo.tableId, entry.getKey(), entry.getValue())) {
         throw new ThriftTableOperationException(tableInfo.tableId, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
             "Invalid table property " + entry.getKey());
       }
-    
+
     return new CreateImportDir(tableInfo);
   }
-  
+
   @Override
   public void undo(long tid, Master env) throws Exception {
     Instance instance = HdfsZooInstance.getInstance();
@@ -477,20 +477,20 @@ class ImportPopulateZookeeper extends MasterRepo {
 }
 
 class ImportSetupPermissions extends MasterRepo {
-  
+
   private static final long serialVersionUID = 1L;
-  
+
   private ImportedTableInfo tableInfo;
-  
+
   public ImportSetupPermissions(ImportedTableInfo ti) {
     this.tableInfo = ti;
   }
-  
+
   @Override
   public long isReady(long tid, Master environment) throws Exception {
     return 0;
   }
-  
+
   @Override
   public Repo<Master> call(long tid, Master env) throws Exception {
     // give all table permissions to the creator
@@ -503,13 +503,13 @@ class ImportSetupPermissions extends MasterRepo {
         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 ImportPopulateZookeeper(tableInfo);
   }
-  
+
   @Override
   public void undo(long tid, Master env) throws Exception {
     AuditedSecurityOperation.getInstance().deleteTable(SystemCredentials.get().toThrift(env.getInstance()), tableInfo.tableId);
@@ -518,33 +518,34 @@ class ImportSetupPermissions extends MasterRepo {
 
 public class ImportTable extends MasterRepo {
   private static final long serialVersionUID = 1L;
-  
+
   private ImportedTableInfo tableInfo;
-  
+
   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 = Namespaces.getNamespaceId(inst, Tables.extractNamespace(tableName));
+    tableInfo.namespaceId = Namespaces.getNamespaceId(inst, Tables.qualify(tableName).getFirst());
   }
-  
+
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    return Utils.reserveHdfsDirectory(new Path(tableInfo.exportDir).toString(), tid) + Utils.reserveNamespace(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
   public Repo<Master> call(long tid, Master env) throws Exception {
     checkVersions(env);
-    
+
     // first step is to reserve a table id.. if the machine fails during this step
     // it is ok to retry... the only side effect is that a table id may not be used
     // or skipped
-    
+
     // assuming only the master process is creating tables
-    
+
     Utils.idLock.lock();
     try {
       Instance instance = HdfsZooInstance.getInstance();
@@ -554,18 +555,18 @@ public class ImportTable extends MasterRepo {
       Utils.idLock.unlock();
     }
   }
-  
+
   public void checkVersions(Master env) throws ThriftTableOperationException {
     Path path = new Path(tableInfo.exportDir, Constants.EXPORT_FILE);
-    
+
     ZipInputStream zis = null;
-    
+
     try {
       zis = new ZipInputStream(env.getFileSystem().open(path));
-      
+
       Integer exportVersion = null;
       Integer dataVersion = null;
-      
+
       ZipEntry zipEntry;
       while ((zipEntry = zis.getNextEntry()) != null) {
         if (zipEntry.getName().equals(Constants.EXPORT_INFO_FILE)) {
@@ -579,22 +580,22 @@ public class ImportTable extends MasterRepo {
               dataVersion = Integer.parseInt(sa[1]);
             }
           }
-          
+
           break;
         }
       }
-      
+
       zis.close();
       zis = null;
-      
+
       if (exportVersion == null || exportVersion > ExportTable.VERSION)
         throw new ThriftTableOperationException(null, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
             "Incompatible export version " + exportVersion);
-      
+
       if (dataVersion == null || dataVersion > ServerConstants.DATA_VERSION)
         throw new ThriftTableOperationException(null, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
             "Incompatible data version " + exportVersion);
-      
+
     } catch (IOException ioe) {
       log.warn(ioe.getMessage(), ioe);
       throw new ThriftTableOperationException(null, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
@@ -608,7 +609,7 @@ public class ImportTable extends MasterRepo {
         }
     }
   }
-  
+
   @Override
   public void undo(long tid, Master env) throws Exception {
     Utils.unreserveHdfsDirectory(new Path(tableInfo.exportDir).toString(), tid);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
index 9fa736d..eee05aa 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
@@ -24,6 +24,7 @@ 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;
 import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
@@ -58,8 +59,11 @@ public class RenameTable extends MasterRepo {
   public Repo<Master> call(long tid, Master master) throws Exception {
 
     Instance instance = master.getInstance();
+    Pair<String,String> qualifiedOldTableName = Tables.qualify(oldTableName);
+    Pair<String,String> qualifiedNewTableName = Tables.qualify(newTableName);
+
     // ensure no attempt is made to rename across namespaces
-    if (newTableName.contains(".") && !namespaceId.equals(Namespaces.getNamespaceId(instance, Tables.extractNamespace(newTableName))))
+    if (newTableName.contains(".") && !namespaceId.equals(Namespaces.getNamespaceId(instance, qualifiedNewTableName.getFirst())))
       throw new IllegalArgumentException("Namespace in new table name does not match the old table name");
 
     IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
@@ -68,8 +72,8 @@ public class RenameTable extends MasterRepo {
     try {
       Utils.checkTableDoesNotExist(instance, newTableName, tableId, TableOperation.RENAME);
 
-      final String newName = Tables.extractTableName(newTableName);
-      final String oldName = Tables.extractTableName(oldTableName);
+      final String newName = qualifiedNewTableName.getSecond();
+      final String oldName = qualifiedOldTableName.getSecond();
 
       final String tap = ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_NAME;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java b/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java
index f435062..bea9381 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java
@@ -57,44 +57,44 @@ import com.google.common.net.HostAndPort;
  * 
  */
 public class TestMergeState {
-  
+
   class MockCurrentState implements CurrentState {
-    
+
     TServerInstance someTServer = new TServerInstance(HostAndPort.fromParts("127.0.0.1", 1234), 0x123456);
     MergeInfo mergeInfo;
-    
+
     MockCurrentState(MergeInfo info) {
       this.mergeInfo = info;
     }
-    
+
     @Override
     public Set<String> onlineTables() {
       return Collections.singleton("t");
     }
-    
+
     @Override
     public Set<TServerInstance> onlineTabletServers() {
       return Collections.singleton(someTServer);
     }
-    
+
     @Override
     public Collection<MergeInfo> merges() {
       return Collections.singleton(mergeInfo);
     }
   }
-  
+
   private static void update(Connector c, Mutation m) throws TableNotFoundException, MutationsRejectedException {
     BatchWriter bw = c.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
     bw.addMutation(m);
     bw.close();
   }
-  
+
   @Test
   public void test() throws Exception {
     Instance instance = new MockInstance();
     Connector connector = instance.getConnector("root", new PasswordToken(""));
-    BatchWriter bw = connector.createBatchWriter("!METADATA", new BatchWriterConfig());
-    
+    BatchWriter bw = connector.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+
     // Create a fake METADATA table with these splits
     String splits[] = {"a", "e", "j", "o", "t", "z"};
     // create metadata for a table "t" with the splits above
@@ -113,11 +113,11 @@ public class TestMergeState {
     defaultTablet.put(TabletsSection.CurrentLocationColumnFamily.NAME, new Text("123456"), new Value("127.0.0.1:1234".getBytes()));
     bw.addMutation(defaultTablet);
     bw.close();
-    
+
     // Read out the TabletLocationStates
     MockCurrentState state = new MockCurrentState(new MergeInfo(new KeyExtent(tableId, new Text("p"), new Text("e")), MergeInfo.Operation.MERGE));
     Credentials credentials = new Credentials("root", new PasswordToken(new byte[0]));
-    
+
     // Verify the tablet state: hosted, and count
     MetaDataStateStore metaDataStateStore = new MetaDataStateStore(instance, credentials, state);
     int count = 0;
@@ -126,59 +126,59 @@ public class TestMergeState {
       count++;
     }
     Assert.assertEquals(splits.length + 1, count);
-    
+
     // Create the hole
     // Split the tablet at one end of the range
     Mutation m = new KeyExtent(tableId, new Text("t"), new Text("p")).getPrevRowUpdateMutation();
     TabletsSection.TabletColumnFamily.SPLIT_RATIO_COLUMN.put(m, new Value("0.5".getBytes()));
     TabletsSection.TabletColumnFamily.OLD_PREV_ROW_COLUMN.put(m, KeyExtent.encodePrevEndRow(new Text("o")));
     update(connector, m);
-    
+
     // do the state check
     MergeStats stats = scan(state, metaDataStateStore);
     MergeState newState = stats.nextMergeState(connector, state);
     Assert.assertEquals(MergeState.WAITING_FOR_OFFLINE, newState);
-    
+
     // unassign the tablets
-    BatchDeleter deleter = connector.createBatchDeleter("!METADATA", Authorizations.EMPTY, 1000, new BatchWriterConfig());
+    BatchDeleter deleter = connector.createBatchDeleter(MetadataTable.NAME, Authorizations.EMPTY, 1000, new BatchWriterConfig());
     deleter.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
     deleter.setRanges(Collections.singletonList(new Range()));
     deleter.delete();
-    
-    // now we should be ready to merge but, we have an inconsistent !METADATA table
+
+    // now we should be ready to merge but, we have inconsistent metadata
     stats = scan(state, metaDataStateStore);
     Assert.assertEquals(MergeState.WAITING_FOR_OFFLINE, stats.nextMergeState(connector, state));
-    
+
     // finish the split
     KeyExtent tablet = new KeyExtent(tableId, new Text("p"), new Text("o"));
     m = tablet.getPrevRowUpdateMutation();
     TabletsSection.TabletColumnFamily.SPLIT_RATIO_COLUMN.put(m, new Value("0.5".getBytes()));
     update(connector, m);
     metaDataStateStore.setLocations(Collections.singletonList(new Assignment(tablet, state.someTServer)));
-    
+
     // onos... there's a new tablet online
     stats = scan(state, metaDataStateStore);
     Assert.assertEquals(MergeState.WAITING_FOR_CHOPPED, stats.nextMergeState(connector, state));
-    
+
     // chop it
     m = tablet.getPrevRowUpdateMutation();
     ChoppedColumnFamily.CHOPPED_COLUMN.put(m, new Value("junk".getBytes()));
     update(connector, m);
-    
+
     stats = scan(state, metaDataStateStore);
     Assert.assertEquals(MergeState.WAITING_FOR_OFFLINE, stats.nextMergeState(connector, state));
-    
+
     // take it offline
     m = tablet.getPrevRowUpdateMutation();
     Collection<Collection<String>> walogs = Collections.emptyList();
     metaDataStateStore.unassign(Collections.singletonList(new TabletLocationState(tablet, null, state.someTServer, null, walogs, false)));
-    
+
     // now we can split
     stats = scan(state, metaDataStateStore);
     Assert.assertEquals(MergeState.MERGING, stats.nextMergeState(connector, state));
-    
+
   }
-  
+
   /**
    * @param state
    * @param metaDataStateStore

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/server/monitor/src/main/resources/docs/bulkIngest.html
----------------------------------------------------------------------
diff --git a/server/monitor/src/main/resources/docs/bulkIngest.html b/server/monitor/src/main/resources/docs/bulkIngest.html
index 0c0ce70..86cdb71 100644
--- a/server/monitor/src/main/resources/docs/bulkIngest.html
+++ b/server/monitor/src/main/resources/docs/bulkIngest.html
@@ -61,7 +61,7 @@ from live ingest in the following ways:
 
 <p>In the first case, adding the file requires telling a single tablet server about a single file.  Even if the file
 is 20G in size, it is one call to the tablet server.  The tablet server makes one extra file entry in the
-!METADATA table, and the data is now part of the tablet.
+tablet's metadata, and the data is now part of the tablet.
 
 <p>In the second case, an request must be made for each tablet for each file to be added.  If there
 100 files and 100 tablets, this will be 10K requests, and the number of files needed to be opened

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/server/monitor/src/main/resources/docs/examples/README.bloom
----------------------------------------------------------------------
diff --git a/server/monitor/src/main/resources/docs/examples/README.bloom b/server/monitor/src/main/resources/docs/examples/README.bloom
index ca6a15c..a7330da 100644
--- a/server/monitor/src/main/resources/docs/examples/README.bloom
+++ b/server/monitor/src/main/resources/docs/examples/README.bloom
@@ -174,10 +174,11 @@ you will need the table ID, because this is used in HDFS instead of the table
 name.  The following command will show table ids.
 
     $ ./bin/accumulo shell -u username -p password -e 'tables -l'
-    !METADATA       =>         !0
-    bloom_test1     =>         o7
-    bloom_test2     =>         o8
-    trace           =>          1
+    accumulo.metadata    =>        !0
+    accumulo.root        =>        +r
+    bloom_test1          =>        o7
+    bloom_test2          =>        o8
+    trace                =>         1
 
 So the table id for bloom_test2 is o8.  The command below shows what files this
 table has in HDFS.  This assumes Accumulo is at the default location in HDFS. 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/server/monitor/src/main/resources/docs/examples/README.export
----------------------------------------------------------------------
diff --git a/server/monitor/src/main/resources/docs/examples/README.export b/server/monitor/src/main/resources/docs/examples/README.export
index d45c202..6430449 100644
--- a/server/monitor/src/main/resources/docs/examples/README.export
+++ b/server/monitor/src/main/resources/docs/examples/README.export
@@ -79,10 +79,11 @@ preserved.
     table    |    @override ........... | 100M
     ---------+--------------------------+-------------------------------------------
     root@test15> tables -l
-    !METADATA       =>         !0
-    trace           =>          1
-    table1_copy     =>          5
-    root@test15 table1_copy> scan -t !METADATA -b 5 -c srv:time
+    accumulo.metadata    =>        !0
+    accumulo.root        =>        +r
+    table1_copy          =>         5
+    trace                =>         1
+    root@test15 table1_copy> scan -t accumulo.metadata -b 5 -c srv:time
     5;b srv:time []    M1343224500467
     5;r srv:time []    M1343224500467
     5< srv:time []    M1343224500467

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/server/monitor/src/main/resources/docs/examples/README.visibility
----------------------------------------------------------------------
diff --git a/server/monitor/src/main/resources/docs/examples/README.visibility b/server/monitor/src/main/resources/docs/examples/README.visibility
index 7ddbf1d..ba0b44d 100644
--- a/server/monitor/src/main/resources/docs/examples/README.visibility
+++ b/server/monitor/src/main/resources/docs/examples/README.visibility
@@ -28,7 +28,7 @@ Notice:    Licensed to the Apache Software Foundation (ASF) under one
     username@instance> userpermissions
     System permissions: 
     
-    Table permissions (!METADATA): Table.READ
+    Table permissions (accumulo.metadata): Table.READ
     username@instance> 
 
 A user does not by default have permission to create a table.
@@ -44,7 +44,7 @@ A user does not by default have permission to create a table.
     username@instance> userpermissions
     System permissions: System.CREATE_TABLE
     
-    Table permissions (!METADATA): Table.READ
+    Table permissions (accumulo.metadata): Table.READ
     Table permissions (vistest): Table.READ, Table.WRITE, Table.BULK_IMPORT, Table.ALTER_TABLE, Table.GRANT, Table.DROP_TABLE
     username@instance vistest> 
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
index e5a6dbb..7daa21a 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
@@ -849,14 +849,14 @@ public class Tablet {
         filesInUseByScans = Collections.singleton(absMergeFile);
 
       // very important to write delete entries outside of log lock, because
-      // this !METADATA write does not go up... it goes sideways or to itself
+      // this metadata write does not go up... it goes sideways or to itself
       if (absMergeFile != null)
         MetadataTableUtil.addDeleteEntries(extent, Collections.singleton(absMergeFile), SystemCredentials.get());
 
       Set<String> unusedWalLogs = beginClearingUnusedLogs();
       try {
-        // the order of writing to !METADATA and walog is important in the face of machine/process failures
-        // need to write to !METADATA before writing to walog, when things are done in the reverse order
+        // the order of writing to metadata and walog is important in the face of machine/process failures
+        // need to write to metadata before writing to walog, when things are done in the reverse order
         // data could be lost... the minor compaction start even should be written before the following metadata
         // write is made
 
@@ -3194,7 +3194,7 @@ public class Tablet {
 
       Pair<Long,List<IteratorSetting>> compactionId = null;
       if (!propogateDeletes) {
-        // compacting everything, so update the compaction id in !METADATA
+        // compacting everything, so update the compaction id in metadata
         try {
           compactionId = getCompactionID();
         } catch (NoNodeException e) {
@@ -3562,7 +3562,7 @@ public class Tablet {
 
       // it is possible that some of the bulk loading flags will be deleted after being read below because the bulk load
       // finishes.... therefore split could propogate load flags for a finished bulk load... there is a special iterator
-      // on the !METADATA table to clean up this type of garbage
+      // on the metadata table to clean up this type of garbage
       Map<FileRef,Long> bulkLoadedFiles = MetadataTableUtil.getBulkFilesLoaded(SystemCredentials.get(), extent);
 
       MetadataTableUtil.splitTablet(high, extent.getPrevEndRow(), splitRatio, SystemCredentials.get(), tabletServer.getLock());
@@ -3740,7 +3740,7 @@ public class Tablet {
   private Set<DfsLogger> otherLogs = Collections.emptySet();
   private boolean removingLogs = false;
 
-  // this lock is basically used to synchronize writing of log info to !METADATA
+  // this lock is basically used to synchronize writing of log info to metadata
   private final ReentrantLock logLock = new ReentrantLock();
 
   public synchronized int getLogCount() {
@@ -3776,7 +3776,7 @@ public class Tablet {
           if (otherLogs.size() != 0)
             throw new IllegalStateException("Expect other logs to be 0 when min finish, but its " + otherLogs);
 
-          // when writing a minc finish event, there is no need to add the log to !METADATA
+          // when writing a minc finish event, there is no need to add the log to metadata
           // if nothing has been logged for the tablet since the minor compaction started
           if (currentLogs.size() == 0)
             return false;


[42/50] [abbrv] ACCUMULO-1906 Remove ops on tables in a namespace

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/3c920943/test/src/test/java/org/apache/accumulo/test/functional/RestartIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/RestartIT.java b/test/src/test/java/org/apache/accumulo/test/functional/RestartIT.java
index 37ad947..9fe2841 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/RestartIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/RestartIT.java
@@ -58,14 +58,13 @@ public class RestartIT extends ConfigurableMacIT {
   static {
     OPTS.rows = VOPTS.rows = 10 * 1000;
   }
-  
+
   @Test(timeout = 2 * 60 * 1000)
   public void restartMaster() throws Exception {
     Connector c = getConnector();
     c.tableOperations().create("test_ingest");
-    Process ingest = cluster.exec(TestIngest.class, 
-        "-u", "root", "-p", ROOT_PASSWORD, 
-        "-i", cluster.getInstanceName(), "-z", cluster.getZooKeepers(), "--rows", ""+OPTS.rows);
+    Process ingest = cluster.exec(TestIngest.class, "-u", "root", "-p", ROOT_PASSWORD, "-i", cluster.getInstanceName(), "-z", cluster.getZooKeepers(),
+        "--rows", "" + OPTS.rows);
     for (ProcessReference master : cluster.getProcesses().get(ServerType.MASTER)) {
       cluster.killProcess(ServerType.MASTER, master);
     }
@@ -74,7 +73,7 @@ public class RestartIT extends ConfigurableMacIT {
     VerifyIngest.verifyIngest(c, VOPTS, SOPTS);
     ingest.destroy();
   }
-  
+
   @Test(timeout = 8 * 60 * 1000)
   public void restartMasterRecovery() throws Exception {
     Connector c = getConnector();
@@ -99,9 +98,8 @@ public class RestartIT extends ConfigurableMacIT {
     Connector c = getConnector();
     c.tableOperations().create("test_ingest");
     c.tableOperations().setProperty("test_ingest", Property.TABLE_SPLIT_THRESHOLD.getKey(), "5K");
-    Process ingest = cluster.exec(TestIngest.class, 
-        "-u", "root", "-p", ROOT_PASSWORD, 
-        "-i", cluster.getInstanceName(), "-z", cluster.getZooKeepers(), "--rows", ""+VOPTS.rows);
+    Process ingest = cluster.exec(TestIngest.class, "-u", "root", "-p", ROOT_PASSWORD, "-i", cluster.getInstanceName(), "-z", cluster.getZooKeepers(),
+        "--rows", "" + VOPTS.rows);
     for (ProcessReference master : cluster.getProcesses().get(ServerType.MASTER)) {
       cluster.killProcess(ServerType.MASTER, master);
     }
@@ -110,7 +108,7 @@ public class RestartIT extends ConfigurableMacIT {
     VerifyIngest.verifyIngest(c, VOPTS, SOPTS);
     ingest.destroy();
   }
-  
+
   @Test(timeout = 10 * 60 * 1000)
   public void killedTabletServer() throws Exception {
     Connector c = getConnector();
@@ -150,7 +148,7 @@ public class RestartIT extends ConfigurableMacIT {
     assertEquals(0, cluster.exec(Admin.class, "stopAll").waitFor());
   }
 
-  @Test(timeout = 2 * 60 * 1000)
+  @Test(timeout = 10 * 60 * 1000)
   public void shutdownDuringCompactingSplitting() throws Exception {
     Connector c = getConnector();
     c.tableOperations().create("test_ingest");


[08/50] [abbrv] git commit: ACCUMULO-1479 implemented most of Table Namespace Permissions, doesnt entirely work, not well tested

Posted by ct...@apache.org.
ACCUMULO-1479 implemented most of Table Namespace Permissions, doesnt entirely work, not well tested


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

Branch: refs/heads/master
Commit: dfdf5113837d79ef9f21a5a64de0a3750611b5c8
Parents: 45f59ca
Author: Sean Hickey <ta...@gmail.com>
Authored: Thu Aug 1 14:16:19 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:10 2013 -0500

----------------------------------------------------------------------
 .../core/client/admin/SecurityOperations.java   |   50 +
 .../client/admin/SecurityOperationsImpl.java    |   36 +
 .../core/client/impl/thrift/ClientService.java  | 5009 ++++++++++++++++--
 .../client/impl/thrift/SecurityErrorCode.java   |    5 +-
 .../accumulo/core/client/mock/MockAccumulo.java |    3 +-
 .../client/mock/MockSecurityOperations.java     |   39 +
 .../core/client/mock/MockTableNamespace.java    |    4 +-
 .../core/client/security/SecurityErrorCode.java |    3 +-
 .../core/security/SystemPermission.java         |    5 +-
 .../core/security/TableNamespacePermission.java |   67 +
 .../core/util/shell/commands/GrantCommand.java  |   19 +-
 .../core/util/shell/commands/RevokeCommand.java |   19 +-
 .../shell/commands/UserPermissionsCommand.java  |   20 +-
 core/src/main/thrift/client.thrift              |   12 +-
 .../server/client/ClientServiceHandler.java     |   22 +
 .../server/security/SecurityOperation.java      |  191 +-
 .../security/handler/InsecurePermHandler.java   |   31 +
 .../security/handler/PermissionHandler.java     |   28 +
 .../server/security/handler/ZKPermHandler.java  |  127 +
 .../server/security/handler/ZKSecurityTool.java |   21 +
 .../randomwalk/security/WalkingSecurity.java    |   51 +
 .../apache/accumulo/test/TableNamespacesIT.java |   37 +-
 22 files changed, 5301 insertions(+), 498 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java
index 86a3ff2..8a93efa 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperations.java
@@ -24,6 +24,7 @@ import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 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.TablePermission;
 
 /**
@@ -216,6 +217,23 @@ public interface SecurityOperations {
   public boolean hasTablePermission(String principal, String table, TablePermission perm) throws AccumuloException, AccumuloSecurityException;
   
   /**
+   * Verify the user has a particular table namespace permission
+   * 
+   * @param principal
+   *          the name of the user to query
+   * @param tableNamespace
+   *          the name of the table namespace to query about
+   * @param perm
+   *          the table namespace permission to check for
+   * @return true if user has that permission; false otherwise
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission to query a user
+   */
+  public boolean hasTableNamespacePermission(String principal, String tableNamespace, TableNamespacePermission perm) throws AccumuloException, AccumuloSecurityException;
+  
+  /**
    * Grant a user a system permission
    * 
    * @param principal
@@ -246,6 +264,22 @@ public interface SecurityOperations {
   public void grantTablePermission(String principal, String table, TablePermission permission) throws AccumuloException, AccumuloSecurityException;
   
   /**
+   * Grant a user a specific permission for a specific table namespace
+   * 
+   * @param principal
+   *          the name of the user to modify
+   * @param tableNamespace
+   *          the name of the table namespace to modify for the user
+   * @param permission
+   *          the table namespace permission to grant to the user
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission to grant a user permissions
+   */
+  public void grantTableNamespacePermission(String principal, String tableNamespace, TableNamespacePermission permission) throws AccumuloException, AccumuloSecurityException;
+  
+  /**
    * Revoke a system permission from a user
    * 
    * @param principal
@@ -276,6 +310,22 @@ public interface SecurityOperations {
   public void revokeTablePermission(String principal, String table, TablePermission permission) throws AccumuloException, AccumuloSecurityException;
   
   /**
+   * Revoke a table namespace permission for a specific user on a specific table namespace
+   * 
+   * @param principal
+   *          the name of the user to modify
+   * @param tableNamespace
+   *          the name of the table namespace to modify for the user
+   * @param permission
+   *          the table namespace permission to revoke for the user
+   * @throws AccumuloException
+   *           if a general error occurs
+   * @throws AccumuloSecurityException
+   *           if the user does not have permission to revoke a user's permissions
+   */
+  public void revokeTableNamespacePermission(String principal, String tableNamespace, TableNamespacePermission permission) throws AccumuloException, AccumuloSecurityException;
+  
+  /**
    * Return a list of users in accumulo
    * 
    * @return a set of user names

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
index 0f0e998..107ba74 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/SecurityOperationsImpl.java
@@ -35,6 +35,7 @@ import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.Credentials;
 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.util.ArgumentChecker;
 import org.apache.accumulo.core.util.ByteBufferUtil;
@@ -207,6 +208,17 @@ public class SecurityOperationsImpl implements SecurityOperations {
   }
   
   @Override
+  public boolean hasTableNamespacePermission(final String principal, final String tableNamespace, final TableNamespacePermission perm) throws AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, tableNamespace, perm);
+    return execute(new ClientExecReturn<Boolean,ClientService.Client>() {
+      @Override
+      public Boolean execute(ClientService.Client client) throws Exception {
+        return client.hasTableNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, tableNamespace, perm.getId());
+      }
+    });
+  }
+  
+  @Override
   public void grantSystemPermission(final String principal, final SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(principal, permission);
     execute(new ClientExec<ClientService.Client>() {
@@ -230,6 +242,18 @@ public class SecurityOperationsImpl implements SecurityOperations {
   }
   
   @Override
+  public void grantTableNamespacePermission(final String principal, final String tableNamespace, final TableNamespacePermission permission) throws AccumuloException,
+      AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, tableNamespace, permission);
+    execute(new ClientExec<ClientService.Client>() {
+      @Override
+      public void execute(ClientService.Client client) throws Exception {
+        client.grantTableNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, tableNamespace, permission.getId());
+      }
+    });
+  }
+  
+  @Override
   public void revokeSystemPermission(final String principal, final SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(principal, permission);
     execute(new ClientExec<ClientService.Client>() {
@@ -252,6 +276,18 @@ public class SecurityOperationsImpl implements SecurityOperations {
     });
   }
   
+  @Override
+  public void revokeTableNamespacePermission(final String principal, final String tableNamespace, final TableNamespacePermission permission) throws AccumuloException,
+      AccumuloSecurityException {
+    ArgumentChecker.notNull(principal, tableNamespace, permission);
+    execute(new ClientExec<ClientService.Client>() {
+      @Override
+      public void execute(ClientService.Client client) throws Exception {
+        client.revokeTableNamespacePermission(Tracer.traceInfo(), credentials.toThrift(instance), principal, tableNamespace, permission.getId());
+      }
+    });
+  }
+  
   @Deprecated
   @Override
   public Set<String> listUsers() throws AccumuloException, AccumuloSecurityException {


[19/50] [abbrv] git commit: ACCUMULO-802 removed default initial properties from system namespace, plus minor documentation fixes

Posted by ct...@apache.org.
ACCUMULO-802 removed default initial properties from system namespace, plus minor documentation fixes


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

Branch: refs/heads/master
Commit: 5262bd501d1ba1a2bbb71009080fd1847e151f41
Parents: d10feb7
Author: Sean Hickey <ta...@gmail.com>
Authored: Mon Aug 12 09:37:42 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:10 2013 -0500

----------------------------------------------------------------------
 .../client/TableNamespaceNotEmptyException.java    | 15 ++++++---------
 .../client/TableNamespaceNotFoundException.java    |  2 +-
 .../client/admin/TableNamespaceOperations.java     | 17 ++++++++++-------
 .../client/admin/TableNamespaceOperationsImpl.java | 11 +++++++----
 .../apache/accumulo/server/init/Initialize.java    | 16 +++++++++-------
 .../java/org/apache/accumulo/master/Master.java    |  4 +---
 6 files changed, 34 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/5262bd50/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotEmptyException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotEmptyException.java b/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotEmptyException.java
index 81ef03a..e333693 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotEmptyException.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotEmptyException.java
@@ -22,19 +22,16 @@ import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException
  * Thrown when the table namespace specified contains tables
  */
 public class TableNamespaceNotEmptyException extends Exception {
-  /**
-   * Exception to throw if an operation is attempted on a table that doesn't exist.
-   * 
-   */
+
   private static final long serialVersionUID = 1L;
   
   private String namespace;
   
   /**
    * @param namespaceId
-   *          the internal id of the table namespace that was sought
+   *          the internal id of the table namespace
    * @param namespaceName
-   *          the visible name of the table namespace that was sought
+   *          the visible name of the table namespace
    * @param description
    *          the specific reason why it failed
    */
@@ -46,9 +43,9 @@ public class TableNamespaceNotEmptyException extends Exception {
   
   /**
    * @param namespaceId
-   *          the internal id of the table namespace that was sought
+   *          the internal id of the table namespace
    * @param namespaceName
-   *          the visible name of the table namespace that was sought
+   *          the visible name of the table namespace
    * @param description
    *          the specific reason why it failed
    * @param cause
@@ -68,7 +65,7 @@ public class TableNamespaceNotEmptyException extends Exception {
   }
   
   /**
-   * @return the name of the table namespace sought
+   * @return the name of the table namespace
    */
   public String getNamespaceName() {
     return namespace;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5262bd50/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotFoundException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotFoundException.java b/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotFoundException.java
index 3dd0740..30e63c3 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotFoundException.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotFoundException.java
@@ -23,7 +23,7 @@ import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException
  */
 public class TableNamespaceNotFoundException extends Exception {
   /**
-   * Exception to throw if an operation is attempted on a table that doesn't exist.
+   * Exception to throw if an operation is attempted on a table namespace that doesn't exist.
    * 
    */
   private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5262bd50/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
index 33f9dc0..136f79b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
@@ -113,10 +113,11 @@ public interface TableNamespaceOperations {
    *           if the table namespace does not exist
    * @throws TableNamespaceNotEmptyException
    *           if the table namespaces still contains tables
-   * @throws TableNotFoundException 
+   * @throws TableNotFoundException
    *           if table not found while deleting
    */
-  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException, TableNamespaceNotEmptyException, TableNotFoundException;
+  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException, TableNamespaceNotEmptyException,
+      TableNotFoundException;
   
   /**
    * Delete a table namespace
@@ -133,7 +134,7 @@ public interface TableNamespaceOperations {
    *           if the table namespace does not exist
    * @throws TableNamespaceNotEmptyException
    *           if the table namespaces still contains tables
-   * @throws TableNotFoundException 
+   * @throws TableNotFoundException
    *           if table not found while deleting
    */
   public void delete(String namespace, boolean deleteTables) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException,
@@ -159,7 +160,8 @@ public interface TableNamespaceOperations {
       TableNamespaceExistsException;
   
   /**
-   * Sets a property on a table namespace. Note that it may take a short period of time (a second) to propagate the change everywhere.
+   * Sets a property on a table namespace which applies to all tables in the namespace. Note that it may take a short period of time (a second) to propagate the
+   * change everywhere.
    * 
    * @param namespace
    *          the name of the table namespace
@@ -195,7 +197,7 @@ public interface TableNamespaceOperations {
    *          the name of the table namespace
    * @return all properties visible by this table (system and per-table properties). Note that recently changed properties may not be visible immediately.
    * @throws TableNamespaceNotFoundException
-   *           if the table does not exist
+   *           if the table namespace does not exist
    */
   public Iterable<Entry<String,String>> getProperties(String namespace) throws AccumuloException, TableNamespaceNotFoundException;
   
@@ -208,6 +210,7 @@ public interface TableNamespaceOperations {
    * @throws AccumuloSecurityException
    *           when the user does not have the proper permissions
    * @throws TableNamespaceNotFoundException
+   *           if the table namespace does not exist
    */
   public void offline(String namespace) throws AccumuloSecurityException, AccumuloException, TableNamespaceNotFoundException;
   
@@ -220,6 +223,7 @@ public interface TableNamespaceOperations {
    * @throws AccumuloSecurityException
    *           when the user does not have the proper permissions
    * @throws TableNamespaceNotFoundException
+   *           if the table namespace does not exist
    */
   public void online(String namespace) throws AccumuloSecurityException, AccumuloException, TableNamespaceNotFoundException;
   
@@ -279,7 +283,7 @@ public interface TableNamespaceOperations {
    * @param setting
    *          object specifying the properties of the iterator
    * @throws AccumuloSecurityException
-   *           thrown if the user does not have the ability to set properties on the table
+   *           thrown if the user does not have the ability to set properties on the table namespace
    * @throws AccumuloException
    * @throws TableNamespaceNotFoundException
    *           throw if the table namespace no longer exists
@@ -419,7 +423,6 @@ public interface TableNamespaceOperations {
    *           thrown if the table namespace no longer exists
    */
   public Map<String,Integer> listConstraints(String tableNamespace) throws AccumuloException, TableNamespaceNotFoundException;
-
   
   /**
    * Test to see if the instance can load the given class as the given type. This check uses the table classpath property if it is set.

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5262bd50/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
index c35319d..694ba89 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
@@ -297,11 +297,12 @@ public class TableNamespaceOperationsImpl extends TableNamespaceOperationsHelper
    *           if the table namespace does not exist
    * @throws TableNamespaceNotEmptyException
    *           if the table namespaces still contains tables
-   * @throws TableNotFoundException 
+   * @throws TableNotFoundException
    *           if table not found while deleting
    */
   @Override
-  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException, TableNamespaceNotEmptyException, TableNotFoundException {
+  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException, TableNamespaceNotEmptyException,
+      TableNotFoundException {
     delete(namespace, false);
   }
 
@@ -320,7 +321,7 @@ public class TableNamespaceOperationsImpl extends TableNamespaceOperationsHelper
    *           if the table namespace does not exist
    * @throws TableNamespaceNotEmptyException
    *           if the table namespaces still contains tables
-   * @throws TableNotFoundException 
+   * @throws TableNotFoundException
    *           if table not found while deleting
    */
   @Override
@@ -446,7 +447,7 @@ public class TableNamespaceOperationsImpl extends TableNamespaceOperationsHelper
   }
 
   /**
-   * Sets a property on a table namespace
+   * Sets a property on a table namespace which will apply to all tables in the namespace
    * 
    * @param namespace
    *          the name of the table namespace
@@ -539,6 +540,7 @@ public class TableNamespaceOperationsImpl extends TableNamespaceOperationsHelper
    * @throws AccumuloSecurityException
    *           when the user does not have the proper permissions
    * @throws TableNamespaceNotFoundException
+   *           if the table namespace does not exist
    */
   @Override
   public void offline(String namespace) throws AccumuloSecurityException, AccumuloException, TableNamespaceNotFoundException {
@@ -563,6 +565,7 @@ public class TableNamespaceOperationsImpl extends TableNamespaceOperationsHelper
    * @throws AccumuloSecurityException
    *           when the user does not have the proper permissions
    * @throws TableNamespaceNotFoundException
+   *           if the table namespace does not exist
    */
   @Override
   public void online(String namespace) throws AccumuloSecurityException, AccumuloException, TableNamespaceNotFoundException {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5262bd50/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 f54816f..042811a 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,8 +416,8 @@ 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);
-    createInitialTableNamespace(zoo, zkInstanceRoot, Constants.SYSTEM_TABLE_NAMESPACE_ID, Constants.SYSTEM_TABLE_NAMESPACE);
+    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);
     
     zoo.putPersistentData(zkInstanceRoot + Constants.ZTABLES + "/" + MetadataTable.ID + Constants.ZTABLE_NAMESPACE,
         Constants.SYSTEM_TABLE_NAMESPACE_ID.getBytes(Constants.UTF8), NodeExistsPolicy.FAIL);
@@ -425,16 +425,18 @@ public class Initialize {
         Constants.SYSTEM_TABLE_NAMESPACE_ID.getBytes(Constants.UTF8), NodeExistsPolicy.FAIL);
   }
   
-  private static void createInitialTableNamespace(IZooReaderWriter zoo, String root, String id, String namespace) throws KeeperException, InterruptedException {
+  private static void createInitialTableNamespace(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);
     zoo.putPersistentData(zPath + Constants.ZNAMESPACE_NAME, namespace.getBytes(Constants.UTF8), NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zPath + Constants.ZNAMESPACE_CONF, new byte[0], NodeExistsPolicy.FAIL);
     
-    Map<String,String> opts = IteratorUtil.generateInitialTableProperties(true);
-    for (Entry<String,String> e : opts.entrySet()) {
-      zoo.putPersistentData(zPath + Constants.ZNAMESPACE_CONF + "/" + e.getKey(), e.getValue().getBytes(Constants.UTF8),
-          NodeExistsPolicy.SKIP);
+    if (defaultOpts) {
+      Map<String,String> opts = IteratorUtil.generateInitialTableProperties(true);
+      for (Entry<String,String> e : opts.entrySet()) {
+        zoo.putPersistentData(zPath + Constants.ZNAMESPACE_CONF + "/" + e.getKey(), e.getValue().getBytes(Constants.UTF8), NodeExistsPolicy.SKIP);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5262bd50/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 0d508dd..0af32d4 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
@@ -352,8 +352,6 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
         for (Entry<String,String> e : opts.entrySet()) {
           zoo.putPersistentData(defaultTableNamespace + Constants.ZNAMESPACE_CONF + "/" + e.getKey(), e.getValue().getBytes(Constants.UTF8),
               NodeExistsPolicy.SKIP);
-          zoo.putPersistentData(systemTableNamespace + Constants.ZNAMESPACE_CONF + "/" + e.getKey(), e.getValue().getBytes(Constants.UTF8),
-              NodeExistsPolicy.SKIP);
         }
 
         for (Entry<String,String> table : Tables.getIdToNameMap(instance).entrySet()) {
@@ -944,7 +942,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
             throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
           fate.seedTransaction(opid, new TraceRepo<Master>(new RenameTable(tableId, oldTableName, newTableName)), autoCleanup);
-
+          
           break;
         }
         case CLONE: {


[04/50] [abbrv] git commit: ACCUMULO-802 added a few more namespace-specific randomwalk tests

Posted by ct...@apache.org.
ACCUMULO-802 added a few more namespace-specific randomwalk tests


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

Branch: refs/heads/master
Commit: 431f4e2c72a45e86c5f4a59fa3ec9119349b4906
Parents: be1ff27
Author: Sean Hickey <ta...@gmail.com>
Authored: Wed Jul 31 12:50:06 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:09 2013 -0500

----------------------------------------------------------------------
 .../conf/TableNamespaceConfiguration.java       |  37 ++++---
 .../test/randomwalk/concurrent/Config.java      | 111 ++++++++++++++-----
 .../concurrent/OfflineTableNamespace.java       |  53 +++++++++
 .../randomwalk/conf/modules/Concurrent.xml      |   5 +
 4 files changed, 162 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/431f4e2c/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
index e3c58f3..60da78a 100644
--- 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
@@ -16,6 +16,9 @@
  */
 package org.apache.accumulo.server.conf;
 
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
@@ -25,6 +28,7 @@ import java.util.TreeMap;
 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;
@@ -33,7 +37,7 @@ 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;
   private String namespaceId = null;
@@ -46,12 +50,12 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
     this.namespaceId = namespaceId;
     this.observers = Collections.synchronizedSet(new HashSet<ConfigurationObserver>());
   }
-  
+
   @Override
   public String get(Property property) {
     String key = property.getKey();
     String value = get(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);
@@ -59,17 +63,16 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
     }
     return value;
   }
-  
+
   private String get(String key) {
-    String zPath = ZooUtil.getRoot(inst.getInstanceID()) + Constants.ZNAMESPACES + "/" + getNamespaceId() + Constants.ZNAMESPACE_CONF + "/"
-        + key;
+    String zPath = ZooUtil.getRoot(inst.getInstanceID()) + Constants.ZNAMESPACES + "/" + getNamespaceId() + Constants.ZNAMESPACE_CONF + "/" + key;
     byte[] v = getPropCache().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)
@@ -79,14 +82,14 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
       }
     return propCache;
   }
-  
+
   @Override
   public Iterator<Entry<String,String>> iterator() {
     TreeMap<String,String> entries = new TreeMap<String,String>();
-    
+
     for (Entry<String,String> parentEntry : parent)
       entries.put(parentEntry.getKey(), parentEntry.getValue());
-    
+
     List<String> children = getPropCache().getChildren(
         ZooUtil.getRoot(inst.getInstanceID()) + Constants.ZNAMESPACES + "/" + getNamespaceId() + Constants.ZNAMESPACE_CONF);
     if (children != null) {
@@ -96,14 +99,14 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
           entries.put(child, value);
       }
     }
-    
+
     return entries.entrySet().iterator();
   }
-  
+
   private String getNamespaceId() {
     return namespaceId;
   }
-  
+
   public void addObserver(ConfigurationObserver co) {
     if (namespaceId == null) {
       String err = "Attempt to add observer for non-table-namespace configuration";
@@ -113,7 +116,7 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
     iterator();
     observers.add(co);
   }
-  
+
   public void removeObserver(ConfigurationObserver configObserver) {
     if (namespaceId == null) {
       String err = "Attempt to remove observer for non-table-namespace configuration";
@@ -122,19 +125,19 @@ public class TableNamespaceConfiguration 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)

http://git-wip-us.apache.org/repos/asf/accumulo/blob/431f4e2c/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
index b47f1a9..5a2008f 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
@@ -29,24 +29,30 @@ import org.apache.commons.math.random.RandomData;
 import org.apache.commons.math.random.RandomDataImpl;
 
 public class Config extends Test {
-  
+
   private static final String LAST_SETTING = "lastSetting";
 
   private static final String LAST_TABLE_SETTING = "lastTableSetting";
 
+  private static final String LAST_NAMESPACE_SETTING = "lastNamespaceSetting";
+
   static class Setting {
     public Property property;
     public long min;
     public long max;
+
     public Setting(Property property, long min, long max) {
       this.property = property;
       this.min = min;
       this.max = max;
     }
   }
+
   static Setting s(Property property, long min, long max) {
     return new Setting(property, min, max);
   }
+
+  /* @formatter:off */
   Setting[] settings = {
       s(Property.TSERV_BLOOM_LOAD_MAXCONCURRENT, 1, 10),
       s(Property.TSERV_BULK_PROCESS_THREADS, 1, 10),
@@ -61,19 +67,19 @@ public class Config extends Test {
       s(Property.TSERV_MAJC_THREAD_MAXOPEN, 3, 100),
       s(Property.TSERV_MINC_MAXCONCURRENT, 1, 10),
       s(Property.TSERV_DEFAULT_BLOCKSIZE, 100000, 10000000L),
-      s(Property.TSERV_MAX_IDLE, 10000, 500*1000),
-      s(Property.TSERV_MAXMEM, 1000000, 3*1024*1024*1024L),
+      s(Property.TSERV_MAX_IDLE, 10000, 500 * 1000),
+      s(Property.TSERV_MAXMEM, 1000000, 3 * 1024 * 1024 * 1024L),
       s(Property.TSERV_READ_AHEAD_MAXCONCURRENT, 1, 25),
       s(Property.TSERV_MIGRATE_MAXCONCURRENT, 1, 10),
-      s(Property.TSERV_MUTATION_QUEUE_MAX, 10000, 1024*1024),
+      s(Property.TSERV_MUTATION_QUEUE_MAX, 10000, 1024 * 1024),
       s(Property.TSERV_RECOVERY_MAX_CONCURRENT, 1, 100),
       s(Property.TSERV_SCAN_MAX_OPENFILES, 10, 1000),
       s(Property.TSERV_THREADCHECK, 100, 10000),
       s(Property.TSERV_MINTHREADS, 1, 100),
-      s(Property.TSERV_SESSION_MAXIDLE, 100, 5*60*1000),
-      s(Property.TSERV_SORT_BUFFER_SIZE, 1024*1024, 1024*1024*1024L),
+      s(Property.TSERV_SESSION_MAXIDLE, 100, 5 * 60 * 1000),
+      s(Property.TSERV_SORT_BUFFER_SIZE, 1024 * 1024, 1024 * 1024 * 1024L),
       s(Property.TSERV_TABLET_SPLIT_FINDMIDPOINT_MAXOPEN, 5, 100),
-      s(Property.TSERV_WAL_BLOCKSIZE, 1024*1024, 1024*1024*1024*10L),
+      s(Property.TSERV_WAL_BLOCKSIZE, 1024 * 1024, 1024 * 1024 * 1024 * 10L),
       s(Property.TSERV_WORKQ_THREADS, 1, 10),
       s(Property.MASTER_BULK_THREADPOOL_SIZE, 1, 10),
       s(Property.MASTER_BULK_RETRIES, 1, 10),
@@ -83,21 +89,21 @@ public class Config extends Test {
       s(Property.MASTER_LEASE_RECOVERY_WAITING_PERIOD, 0, 10),
       s(Property.MASTER_RECOVERY_MAXTIME, 10, 1000),
       s(Property.MASTER_THREADCHECK, 100, 10000),
-      s(Property.MASTER_MINTHREADS, 1, 200),
-  };
-  
+      s(Property.MASTER_MINTHREADS, 1, 200),};
+
   Setting[] tableSettings = {
       s(Property.TABLE_MAJC_RATIO, 1, 10),
-      s(Property.TABLE_MAJC_COMPACTALL_IDLETIME, 100, 10*60*60*1000L),
-      s(Property.TABLE_SPLIT_THRESHOLD, 10*1024, 10L*1024*1024*1024),
-      s(Property.TABLE_MINC_COMPACT_IDLETIME, 100, 100*60*60*1000L),
-      s(Property.TABLE_SCAN_MAXMEM, 10*1024, 10*1024*1024),
-      s(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE, 10*1024, 10*1024*1024L), 
-      s(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX, 10*1024, 10*1024*1024L),
+      s(Property.TABLE_MAJC_COMPACTALL_IDLETIME, 100, 10 * 60 * 60 * 1000L),
+      s(Property.TABLE_SPLIT_THRESHOLD, 10 * 1024, 10L * 1024 * 1024 * 1024),
+      s(Property.TABLE_MINC_COMPACT_IDLETIME, 100, 100 * 60 * 60 * 1000L),
+      s(Property.TABLE_SCAN_MAXMEM, 10 * 1024, 10 * 1024 * 1024),
+      s(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE, 10 * 1024, 10 * 1024 * 1024L),
+      s(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX, 10 * 1024, 10 * 1024 * 1024L),
       s(Property.TABLE_FILE_REPLICATION, 0, 5),
-      s(Property.TABLE_FILE_MAX, 2, 50),
-  };
-  
+      s(Property.TABLE_FILE_MAX, 2, 50),};
+
+  /* @formatter:on */
+
   @Override
   public void visit(State state, Properties props) throws Exception {
     // reset any previous setting
@@ -120,7 +126,27 @@ public class Config extends Test {
           state.getConnector().tableOperations().setProperty(table, property.getKey(), property.getDefaultValue());
         } catch (AccumuloException ex) {
           if (ex.getCause() instanceof ThriftTableOperationException) {
-            ThriftTableOperationException ttoe = (ThriftTableOperationException)ex.getCause();
+            ThriftTableOperationException ttoe = (ThriftTableOperationException) ex.getCause();
+            if (ttoe.type == TableOperationExceptionType.NOTFOUND)
+              return;
+          }
+          throw ex;
+        }
+      }
+    }
+    lastSetting = state.getMap().get(LAST_NAMESPACE_SETTING);
+    if (lastSetting != null) {
+      String parts[] = lastSetting.toString().split(",");
+      String namespace = parts[0];
+      int choice = Integer.parseInt(parts[1]);
+      Property property = tableSettings[choice].property;
+      if (state.getConnector().tableNamespaceOperations().exists(namespace)) {
+        log.debug("Setting " + property.getKey() + " on " + namespace + " back to " + property.getDefaultValue());
+        try {
+          state.getConnector().tableNamespaceOperations().setProperty(namespace, property.getKey(), property.getDefaultValue());
+        } catch (AccumuloException ex) {
+          if (ex.getCause() instanceof ThriftTableOperationException) {
+            ThriftTableOperationException ttoe = (ThriftTableOperationException) ex.getCause();
             if (ttoe.type == TableOperationExceptionType.NOTFOUND)
               return;
           }
@@ -130,9 +156,13 @@ public class Config extends Test {
     }
     state.getMap().remove(LAST_SETTING);
     state.getMap().remove(LAST_TABLE_SETTING);
+    state.getMap().remove(LAST_NAMESPACE_SETTING);
     RandomData random = new RandomDataImpl();
-    if (random.nextInt(0, 1) == 0) {
+    int dice = random.nextInt(0, 2);
+    if (dice == 0) {
       changeTableSetting(random, state, props);
+    } else if (dice == 1) {
+      changeTableNamespaceSetting(random, state, props);
     } else {
       changeSetting(random, state, props);
     }
@@ -142,13 +172,13 @@ public class Config extends Test {
     // pick a random property
     int choice = random.nextInt(0, tableSettings.length - 1);
     Setting setting = tableSettings[choice];
-    
+
     // pick a random table
     SortedSet<String> tables = state.getConnector().tableOperations().list().tailSet("ctt").headSet("ctu");
     if (tables.isEmpty())
       return;
     String table = random.nextSample(tables, 1)[0].toString();
-    
+
     // generate a random value
     long newValue = random.nextLong(setting.min, setting.max);
     state.getMap().put(LAST_TABLE_SETTING, table + "," + choice);
@@ -157,14 +187,41 @@ public class Config extends Test {
       state.getConnector().tableOperations().setProperty(table, setting.property.getKey(), "" + newValue);
     } catch (AccumuloException ex) {
       if (ex.getCause() instanceof ThriftTableOperationException) {
-        ThriftTableOperationException ttoe = (ThriftTableOperationException)ex.getCause();
+        ThriftTableOperationException ttoe = (ThriftTableOperationException) ex.getCause();
+        if (ttoe.type == TableOperationExceptionType.NOTFOUND)
+          return;
+      }
+      throw ex;
+    }
+  }
+
+  private void changeTableNamespaceSetting(RandomData random, State state, Properties props) throws Exception {
+    // pick a random property
+    int choice = random.nextInt(0, tableSettings.length - 1);
+    Setting setting = tableSettings[choice];
+
+    // pick a random table
+    SortedSet<String> namespaces = state.getConnector().tableNamespaceOperations().list();
+    if (namespaces.isEmpty())
+      return;
+    String namespace = random.nextSample(namespaces, 1)[0].toString();
+
+    // generate a random value
+    long newValue = random.nextLong(setting.min, setting.max);
+    state.getMap().put(LAST_NAMESPACE_SETTING, namespace + "," + choice);
+    log.debug("Setting " + setting.property.getKey() + " on table namespace " + namespace + " to " + newValue);
+    try {
+      state.getConnector().tableNamespaceOperations().setProperty(namespace, setting.property.getKey(), "" + newValue);
+    } catch (AccumuloException ex) {
+      if (ex.getCause() instanceof ThriftTableOperationException) {
+        ThriftTableOperationException ttoe = (ThriftTableOperationException) ex.getCause();
         if (ttoe.type == TableOperationExceptionType.NOTFOUND)
           return;
       }
       throw ex;
     }
   }
-  
+
   private void changeSetting(RandomData random, State state, Properties props) throws Exception {
     // pick a random property
     int choice = random.nextInt(0, settings.length - 1);
@@ -173,7 +230,7 @@ public class Config extends Test {
     long newValue = random.nextLong(setting.min, setting.max);
     state.getMap().put(LAST_SETTING, "" + choice);
     log.debug("Setting " + setting.property.getKey() + " to " + newValue);
-    state.getConnector().instanceOperations().setProperty(setting.property.getKey(), ""+newValue);
+    state.getConnector().instanceOperations().setProperty(setting.property.getKey(), "" + newValue);
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/431f4e2c/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTableNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTableNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTableNamespace.java
new file mode 100644
index 0000000..c895d04
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTableNamespace.java
@@ -0,0 +1,53 @@
+/*
+ * 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.test.randomwalk.concurrent;
+
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.test.randomwalk.State;
+import org.apache.accumulo.test.randomwalk.Test;
+
+public class OfflineTableNamespace extends Test {
+  
+  @Override
+  public void visit(State state, Properties props) throws Exception {
+    Connector conn = state.getConnector();
+    
+    Random rand = (Random) state.get("rand");
+    
+    @SuppressWarnings("unchecked")
+    List<String> namespaces = (List<String>) state.get("namespaces");
+    
+    String namespace = namespaces.get(rand.nextInt(namespaces.size()));
+    
+    try {
+      conn.tableNamespaceOperations().offline(namespace);
+      log.debug("Offlined namespace " + namespace);
+      UtilWaitThread.sleep(rand.nextInt(200));
+      conn.tableNamespaceOperations().online(namespace);
+      log.debug("Onlined namespace " + namespace);
+    } catch (TableNamespaceNotFoundException tne) {
+      log.debug("offline or online failed " + namespace + ", doesnt exist");
+    }
+    
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/431f4e2c/test/system/randomwalk/conf/modules/Concurrent.xml
----------------------------------------------------------------------
diff --git a/test/system/randomwalk/conf/modules/Concurrent.xml b/test/system/randomwalk/conf/modules/Concurrent.xml
index 89b1a88..cce8675 100644
--- a/test/system/randomwalk/conf/modules/Concurrent.xml
+++ b/test/system/randomwalk/conf/modules/Concurrent.xml
@@ -53,6 +53,7 @@
   <edge id="ct.DeleteTableNamespace" weight="100"/>
   <edge id="ct.RenameTableNamespace" weight="100"/>
   <edge id="ct.CloneTableNamespace" weight="100"/>
+  <edge id="ct.OfflineTableNamespace" weight="100"/>
   <edge id="ct.Apocalypse" weight="10"/>
   <edge id="END" weight="1"/>
 </node>
@@ -183,4 +184,8 @@
   <edge id="ct.StartAll" weight="1"/>
 </node>
 
+<node id="ct.OfflineTableNamespace">
+  <edge id="ct.StartAll" weight="1"/>
+</node>
+
 </module>


[17/50] [abbrv] git commit: ACCUMULO-802 added -l option to namespaces command

Posted by ct...@apache.org.
ACCUMULO-802 added -l option to namespaces command


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

Branch: refs/heads/master
Commit: d10feb7c7f7ae9eb5affe69ab18959e8bd7c16d7
Parents: 51f07eb
Author: Sean Hickey <ta...@gmail.com>
Authored: Mon Aug 12 08:11:31 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:10 2013 -0500

----------------------------------------------------------------------
 .../util/shell/commands/NamespacesCommand.java  | 34 ++++++++++++++++++--
 1 file changed, 32 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/d10feb7c/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java
index dcb4bd7..ad30699 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java
@@ -17,6 +17,9 @@
 package org.apache.accumulo.core.util.shell.commands;
 
 import java.io.IOException;
+import java.util.Iterator;
+import java.util.TreeMap;
+import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -25,16 +28,41 @@ import org.apache.accumulo.core.util.shell.Shell.Command;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
+import org.apache.commons.collections.iterators.AbstractIteratorDecorator;
 
 public class NamespacesCommand extends Command {
-  private Option disablePaginationOpt;
+  private Option disablePaginationOpt, namespaceIdOption;
   
+  @SuppressWarnings("unchecked")
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException, IOException {
-    shellState.printLines(shellState.getConnector().tableNamespaceOperations().list().iterator(), !cl.hasOption(disablePaginationOpt.getOpt()));
+      Iterator<String> names = shellState.getConnector().tableNamespaceOperations().list().iterator();
+      Iterator<String> ids = new NamespaceIdIterator(new TreeMap<String,String>(shellState.getConnector().tableNamespaceOperations().namespaceIdMap()).entrySet().iterator());
+    
+    if (cl.hasOption(namespaceIdOption.getOpt())) {
+      shellState.printLines(ids, !cl.hasOption(disablePaginationOpt.getOpt()));
+    } else {
+      shellState.printLines(names, !cl.hasOption(disablePaginationOpt.getOpt()));
+    }
     return 0;
   }
   
+  /**
+   * Decorator that formats the id and name for display.
+   */
+  private static final class NamespaceIdIterator extends AbstractIteratorDecorator {
+    public NamespaceIdIterator(Iterator<Entry<String,String>> iterator) {
+      super(iterator);
+    }
+    
+    @SuppressWarnings("rawtypes")
+    @Override
+    public Object next() {
+      Entry entry = (Entry) super.next();
+      return String.format("%-15s => %10s%n", entry.getKey(), entry.getValue());
+    }
+  }
+  
   @Override
   public String description() {
     return "displays a list of all existing table namespaces";
@@ -43,6 +71,8 @@ public class NamespacesCommand extends Command {
   @Override
   public Options getOptions() {
     final Options o = new Options();
+    namespaceIdOption = new Option("l", "list-ids", false, "display internal table namespace ids along with the name");
+    o.addOption(namespaceIdOption);
     disablePaginationOpt = new Option("np", "no-pagination", false, "disable pagination of output");
     o.addOption(disablePaginationOpt);
     return o;


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

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
deleted file mode 100644
index a7d0c34..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
+++ /dev/null
@@ -1,590 +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.core.client.admin;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-import jline.internal.Log;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.TableNamespaceExistsException;
-import org.apache.accumulo.core.client.TableNamespaceNotEmptyException;
-import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.TableOfflineException;
-import org.apache.accumulo.core.client.impl.ClientExec;
-import org.apache.accumulo.core.client.impl.ClientExecReturn;
-import org.apache.accumulo.core.client.impl.MasterClient;
-import org.apache.accumulo.core.client.impl.ServerClient;
-import org.apache.accumulo.core.client.impl.TableNamespaces;
-import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.core.client.impl.thrift.ClientService;
-import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
-import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
-import org.apache.accumulo.core.constraints.Constraint;
-import org.apache.accumulo.core.iterators.IteratorUtil;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.master.thrift.MasterClientService;
-import org.apache.accumulo.core.master.thrift.TableOperation;
-import org.apache.accumulo.core.security.Credentials;
-import org.apache.accumulo.core.util.ArgumentChecker;
-import org.apache.accumulo.core.util.ByteBufferUtil;
-import org.apache.accumulo.core.util.OpTimer;
-import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.trace.instrument.Tracer;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransportException;
-
-/**
- * Provides a class for administering table namespaces
- * 
- */
-public class TableNamespaceOperationsImpl extends TableNamespaceOperationsHelper {
-  private Instance instance;
-  private Credentials credentials;
-
-  private static final Logger log = Logger.getLogger(TableOperations.class);
-
-  /**
-   * @param instance
-   *          the connection information for this instance
-   * @param credentials
-   *          the username/password for this connection
-   */
-  public TableNamespaceOperationsImpl(Instance instance, Credentials credentials) {
-    ArgumentChecker.notNull(instance, credentials);
-    this.instance = instance;
-    this.credentials = credentials;
-  }
-
-  /**
-   * Retrieve a list of table namespaces in Accumulo.
-   * 
-   * @return List of table namespaces in accumulo
-   */
-  @Override
-  public SortedSet<String> list() {
-    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Fetching list of table namespaces...");
-    TreeSet<String> namespaces = new TreeSet<String>(TableNamespaces.getNameToIdMap(instance).keySet());
-    opTimer.stop("Fetched " + namespaces.size() + " table namespaces in %DURATION%");
-    return namespaces;
-  }
-
-  /**
-   * A method to check if a table namespace exists in Accumulo.
-   * 
-   * @param namespace
-   *          the name of the table namespace
-   * @return true if the table namespace exists
-   */
-  @Override
-  public boolean exists(String namespace) {
-    ArgumentChecker.notNull(namespace);
-
-    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Checking if table namespace " + namespace + " exists...");
-    boolean exists = TableNamespaces.getNameToIdMap(instance).containsKey(namespace);
-    opTimer.stop("Checked existance of " + exists + " in %DURATION%");
-    return exists;
-  }
-
-  /**
-   * Create a table namespace with no special configuration
-   * 
-   * @param namespace
-   *          the name of the table namespace
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableNamespaceExistsException
-   *           if the table namespace already exists
-   */
-  @Override
-  public void create(String namespace) throws AccumuloException, AccumuloSecurityException, TableNamespaceExistsException {
-    create(namespace, true, TimeType.MILLIS);
-  }
-
-  /**
-   * @param namespace
-   *          the name of the table namespace
-   * @param limitVersion
-   *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
-   */
-  @Override
-  public void create(String namespace, boolean limitVersion) throws AccumuloException, AccumuloSecurityException, TableNamespaceExistsException {
-    create(namespace, limitVersion, TimeType.MILLIS);
-  }
-
-  /**
-   * @param namespace
-   *          the name of the table namespace
-   * @param timeType
-   *          specifies logical or real-time based time recording for entries in the table
-   * @param limitVersion
-   *          Enables/disables the versioning iterator, which will limit the number of Key versions kept.
-   */
-  @Override
-  public void create(String namespace, boolean limitVersion, TimeType timeType) throws AccumuloException, AccumuloSecurityException,
-      TableNamespaceExistsException {
-    ArgumentChecker.notNull(namespace, timeType);
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(namespace.getBytes()), ByteBuffer.wrap(timeType.name().getBytes()));
-
-    Map<String,String> opts = IteratorUtil.generateInitialTableProperties(limitVersion);
-
-    try {
-      doTableNamespaceOperation(TableOperation.CREATE, args, opts);
-    } catch (TableNamespaceNotFoundException e1) {
-      // should not happen
-      throw new RuntimeException(e1);
-    }
-  }
-
-  private long beginTableNamespaceOperation() throws ThriftSecurityException, TException {
-    while (true) {
-      MasterClientService.Iface client = null;
-      try {
-        client = MasterClient.getConnectionWithRetry(instance);
-        return client.beginTableNamespaceOperation(Tracer.traceInfo(), credentials.toThrift(instance));
-      } catch (TTransportException tte) {
-        log.debug("Failed to call beginTableOperation(), retrying ... ", tte);
-        UtilWaitThread.sleep(100);
-      } finally {
-        MasterClient.close(client);
-      }
-    }
-  }
-
-  private void executeTableNamespaceOperation(long opid, TableOperation op, List<ByteBuffer> args, Map<String,String> opts, boolean autoCleanUp)
-      throws ThriftSecurityException, TException, ThriftTableOperationException {
-    while (true) {
-      MasterClientService.Iface client = null;
-      try {
-        client = MasterClient.getConnectionWithRetry(instance);
-        client.executeTableNamespaceOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid, op, args, opts, autoCleanUp);
-        break;
-      } catch (TTransportException tte) {
-        log.debug("Failed to call executeTableOperation(), retrying ... ", tte);
-        UtilWaitThread.sleep(100);
-      } finally {
-        MasterClient.close(client);
-      }
-    }
-  }
-
-  private String waitForTableNamespaceOperation(long opid) throws ThriftSecurityException, TException, ThriftTableOperationException {
-    while (true) {
-      MasterClientService.Iface client = null;
-      try {
-        client = MasterClient.getConnectionWithRetry(instance);
-        return client.waitForTableNamespaceOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid);
-      } catch (TTransportException tte) {
-        log.debug("Failed to call waitForTableOperation(), retrying ... ", tte);
-        UtilWaitThread.sleep(100);
-      } finally {
-        MasterClient.close(client);
-      }
-    }
-  }
-
-  private void finishTableNamespaceOperation(long opid) throws ThriftSecurityException, TException {
-    while (true) {
-      MasterClientService.Iface client = null;
-      try {
-        client = MasterClient.getConnectionWithRetry(instance);
-        client.finishTableNamespaceOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid);
-        break;
-      } catch (TTransportException tte) {
-        log.debug("Failed to call finishTableOperation(), retrying ... ", tte);
-        UtilWaitThread.sleep(100);
-      } finally {
-        MasterClient.close(client);
-      }
-    }
-  }
-
-  private String doTableNamespaceOperation(TableOperation op, List<ByteBuffer> args, Map<String,String> opts) throws AccumuloSecurityException,
-      TableNamespaceExistsException, TableNamespaceNotFoundException, AccumuloException {
-    return doTableNamespaceOperation(op, args, opts, true);
-  }
-
-  private String doTableNamespaceOperation(TableOperation op, List<ByteBuffer> args, Map<String,String> opts, boolean wait) throws AccumuloSecurityException,
-      TableNamespaceExistsException, TableNamespaceNotFoundException, AccumuloException {
-    Long opid = null;
-
-    try {
-      opid = beginTableNamespaceOperation();
-      executeTableNamespaceOperation(opid, op, args, opts, !wait);
-      if (!wait) {
-        opid = null;
-        return null;
-      }
-      String ret = waitForTableNamespaceOperation(opid);
-      Tables.clearCache(instance);
-      return ret;
-    } catch (ThriftSecurityException e) {
-      String tableName = ByteBufferUtil.toString(args.get(0));
-      String tableInfo = Tables.getPrintableTableInfoFromName(instance, tableName);
-      throw new AccumuloSecurityException(e.user, e.code, tableInfo, e);
-    } catch (ThriftTableOperationException e) {
-      switch (e.getType()) {
-        case EXISTS:
-          throw new TableNamespaceExistsException(e);
-        case NOTFOUND:
-          throw new TableNamespaceNotFoundException(e);
-        case OFFLINE:
-          throw new TableOfflineException(instance, null);
-        case OTHER:
-        default:
-          throw new AccumuloException(e.description, e);
-      }
-    } catch (Exception e) {
-      throw new AccumuloException(e.getMessage(), e);
-    } finally {
-      // always finish table op, even when exception
-      if (opid != null)
-        try {
-          finishTableNamespaceOperation(opid);
-        } catch (Exception e) {
-          log.warn(e.getMessage(), e);
-        }
-    }
-  }
-
-  /**
-   * Delete a table namespace if empty
-   * 
-   * @param namespace
-   *          the name of the table namespace
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableNamespaceNotFoundException
-   *           if the table namespace does not exist
-   * @throws TableNamespaceNotEmptyException
-   *           if the table namespaces still contains tables
-   * @throws TableNotFoundException
-   *           if table not found while deleting
-   */
-  @Override
-  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException, TableNamespaceNotEmptyException,
-      TableNotFoundException {
-    delete(namespace, false);
-  }
-
-  /**
-   * Delete a table namespace
-   * 
-   * @param namespace
-   *          the name of the table namespace
-   * @param deleteTables
-   *          boolean, if true deletes all the tables in the namespace in addition to deleting the namespace.
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableNamespaceNotFoundException
-   *           if the table namespace does not exist
-   * @throws TableNamespaceNotEmptyException
-   *           if the table namespaces still contains tables
-   * @throws TableNotFoundException
-   *           if table not found while deleting
-   */
-  @Override
-  public void delete(String namespace, boolean deleteTables) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException,
-      TableNamespaceNotEmptyException, TableNotFoundException {
-    ArgumentChecker.notNull(namespace);
-    String namespaceId = TableNamespaces.getNamespaceId(instance, namespace);
-
-    if (namespaceId.equals(Constants.SYSTEM_TABLE_NAMESPACE_ID) || namespaceId.equals(Constants.DEFAULT_TABLE_NAMESPACE_ID)) {
-      log.debug(credentials.getPrincipal() + " attempted to delete the " + namespaceId + " table namespace");
-      throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.UNSUPPORTED_OPERATION);
-    }
-
-    if (TableNamespaces.getTableIds(instance, namespaceId).size() > 0) {
-      if (!deleteTables) {
-        throw new TableNamespaceNotEmptyException(namespaceId, namespace, null);
-      }
-      for (String table : TableNamespaces.getTableNames(instance, namespaceId)) {
-        try {
-          getTableOperations().delete(table);
-        } catch (TableNotFoundException e) {
-          log.debug("Table (" + table + ") not found while deleting namespace, probably deleted while we were deleting the rest of the tables");
-        }
-      }
-    }
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(namespace.getBytes()));
-    Map<String,String> opts = new HashMap<String,String>();
-
-    try {
-      doTableNamespaceOperation(TableOperation.DELETE, args, opts);
-    } catch (TableNamespaceExistsException e) {
-      // should not happen
-      throw new RuntimeException(e);
-    }
-
-  }
-
-  /**
-   * Rename a table namespace
-   * 
-   * @param oldNamespaceName
-   *          the old table namespace
-   * @param newNamespaceName
-   *          the new table namespace
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   * @throws TableNamespaceNotFoundException
-   *           if the old table namespace name does not exist
-   * @throws TableNamespaceExistsException
-   *           if the new table namespace name already exists
-   */
-  @Override
-  public void rename(String oldNamespaceName, String newNamespaceName) throws AccumuloSecurityException, TableNamespaceNotFoundException, AccumuloException,
-      TableNamespaceExistsException {
-
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldNamespaceName.getBytes()), ByteBuffer.wrap(newNamespaceName.getBytes()));
-    Map<String,String> opts = new HashMap<String,String>();
-    doTableNamespaceOperation(TableOperation.RENAME, args, opts);
-  }
-
-  /**
-   * Sets a property on a table namespace which will apply to all tables in the namespace
-   * 
-   * @param namespace
-   *          the name of the table namespace
-   * @param property
-   *          the name of a per-table property
-   * @param value
-   *          the value to set a per-table property to
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   */
-  @Override
-  public void setProperty(final String namespace, final String property, final String value) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(namespace, property, value);
-
-    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
-      @Override
-      public void execute(MasterClientService.Client client) throws Exception {
-        client.setTableNamespaceProperty(Tracer.traceInfo(), credentials.toThrift(instance), namespace, property, value);
-      }
-    });
-  }
-
-  /**
-   * Removes a property from a table namespace
-   * 
-   * @param namespace
-   *          the name of the table namespace
-   * @param property
-   *          the name of a per-table property
-   * @throws AccumuloException
-   *           if a general error occurs
-   * @throws AccumuloSecurityException
-   *           if the user does not have permission
-   */
-  @Override
-  public void removeProperty(final String namespace, final String property) throws AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(namespace, property);
-
-    MasterClient.execute(instance, new ClientExec<MasterClientService.Client>() {
-      @Override
-      public void execute(MasterClientService.Client client) throws Exception {
-        client.removeTableNamespaceProperty(Tracer.traceInfo(), credentials.toThrift(instance), namespace, property);
-      }
-    });
-  }
-
-  /**
-   * Gets properties of a table namespace
-   * 
-   * @param namespace
-   *          the name of the table namespace
-   * @return all properties visible by this table namespace (system and per-namespace properties)
-   * @throws TableNamespaceNotFoundException
-   *           if the table namespace does not exist
-   */
-  @Override
-  public Iterable<Entry<String,String>> getProperties(final String namespace) throws AccumuloException, TableNamespaceNotFoundException {
-    ArgumentChecker.notNull(namespace);
-    try {
-      return ServerClient.executeRaw(instance, new ClientExecReturn<Map<String,String>,ClientService.Client>() {
-        @Override
-        public Map<String,String> execute(ClientService.Client client) throws Exception {
-          return client.getTableNamespaceConfiguration(Tracer.traceInfo(), credentials.toThrift(instance), namespace);
-        }
-      }).entrySet();
-    } catch (ThriftTableOperationException e) {
-      switch (e.getType()) {
-        case NOTFOUND:
-          throw new TableNamespaceNotFoundException(e);
-        case OTHER:
-        default:
-          throw new AccumuloException(e.description, e);
-      }
-    } catch (AccumuloException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new AccumuloException(e);
-    }
-
-  }
-
-  /**
-   * 
-   * @param namespace
-   *          the table namespace to take offline
-   * @throws AccumuloException
-   *           when there is a general accumulo error
-   * @throws AccumuloSecurityException
-   *           when the user does not have the proper permissions
-   * @throws TableNamespaceNotFoundException
-   *           if the table namespace does not exist
-   */
-  @Override
-  public void offline(String namespace) throws AccumuloSecurityException, AccumuloException, TableNamespaceNotFoundException {
-
-    ArgumentChecker.notNull(namespace);
-    String namespaceId = TableNamespaces.getNamespaceId(instance, namespace);
-    try {
-      for (String table : TableNamespaces.getTableNames(instance, namespaceId)) {
-        getTableOperations().offline(table);
-      }
-    } catch (TableNotFoundException e) {
-      Log.error("Table namespace (" + namespaceId + ") contains reference to table that doesn't exist");
-    }
-  }
-
-  /**
-   * 
-   * @param namespace
-   *          the table namespace to take online
-   * @throws AccumuloException
-   *           when there is a general accumulo error
-   * @throws AccumuloSecurityException
-   *           when the user does not have the proper permissions
-   * @throws TableNamespaceNotFoundException
-   *           if the table namespace does not exist
-   */
-  @Override
-  public void online(String namespace) throws AccumuloSecurityException, AccumuloException, TableNamespaceNotFoundException {
-    ArgumentChecker.notNull(namespace);
-    String namespaceId = TableNamespaces.getNamespaceId(instance, namespace);
-    try {
-      for (String table : TableNamespaces.getTableNames(instance, namespaceId)) {
-        getTableOperations().online(table);
-      }
-    } catch (TableNotFoundException e) {
-      Log.warn("Table namespace (" + namespaceId + ") contains a reference to a table that doesn't exist");
-    }
-  }
-
-  /**
-   * Get a mapping of table namespace name to internal table namespace id.
-   * 
-   * @return the map from table namespace name to internal table namespace id
-   */
-  @Override
-  public Map<String,String> namespaceIdMap() {
-    return TableNamespaces.getNameToIdMap(instance);
-  }
-
-  @Override
-  public List<DiskUsage> getDiskUsage(String namespace) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException {
-    Set<String> tables = new HashSet<String>();
-    String namespaceId = TableNamespaces.getNamespaceId(instance, namespace);
-    tables.addAll(TableNamespaces.getTableNames(instance, namespaceId));
-    List<DiskUsage> du = null;
-    try {
-      du = getTableOperations().getDiskUsage(tables);
-    } catch (TableNotFoundException e) {
-      log.warn("Could not find table (" + e.getTableName() + ") reference in namespace (" + namespace + ")");
-    }
-    return du;
-  }
-
-  private TableOperations getTableOperations() throws AccumuloException, AccumuloSecurityException {
-    return new TableOperationsImpl(instance, credentials);
-  }
-
-  @Override
-  public void attachIterator(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
-      TableNamespaceNotFoundException {
-    testClassLoad(namespace, setting.getIteratorClass(), SortedKeyValueIterator.class.getName());
-    super.attachIterator(namespace, setting, scopes);
-  }
-
-  @Override
-  public int addConstraint(String namespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException {
-    testClassLoad(namespace, constraintClassName, Constraint.class.getName());
-    return super.addConstraint(namespace, constraintClassName);
-  }
-
-  @Override
-  public boolean testClassLoad(final String namespace, final String className, final String asTypeName) throws TableNamespaceNotFoundException,
-      AccumuloException, AccumuloSecurityException {
-    ArgumentChecker.notNull(namespace, className, asTypeName);
-
-    try {
-      return ServerClient.executeRaw(instance, new ClientExecReturn<Boolean,ClientService.Client>() {
-        @Override
-        public Boolean execute(ClientService.Client client) throws Exception {
-          return client.checkTableNamespaceClass(Tracer.traceInfo(), credentials.toThrift(instance), namespace, className, asTypeName);
-        }
-      });
-    } catch (ThriftTableOperationException e) {
-      switch (e.getType()) {
-        case NOTFOUND:
-          throw new TableNamespaceNotFoundException(e);
-        case OTHER:
-        default:
-          throw new AccumuloException(e.description, e);
-      }
-    } catch (ThriftSecurityException e) {
-      throw new AccumuloSecurityException(e.user, e.code, e);
-    } catch (AccumuloException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new AccumuloException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
index aa4c83e..c398e6e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
@@ -50,20 +50,20 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 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.TableDeletedException;
 import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.client.impl.AccumuloServerException;
 import org.apache.accumulo.core.client.impl.ClientExec;
 import org.apache.accumulo.core.client.impl.ClientExecReturn;
 import org.apache.accumulo.core.client.impl.MasterClient;
+import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.ServerClient;
 import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
-import org.apache.accumulo.core.client.impl.TableNamespaces;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.TabletLocator;
 import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation;
@@ -129,7 +129,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   private Credentials credentials;
 
   public static final String CLONE_EXCLUDE_PREFIX = "!";
-  
+
   private static final Logger log = Logger.getLogger(TableOperations.class);
 
   /**
@@ -222,9 +222,9 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
     String namespace = Tables.extractNamespace(tableName);
     if (!namespaceExists(namespace)) {
-      String info = "Table namespace not found while trying to create table";
-      throw new IllegalArgumentException(new TableNamespaceNotFoundException(null, namespace, info));
-    } else if (namespace.equals(Constants.SYSTEM_TABLE_NAMESPACE)) {
+      String info = "Namespace not found while trying to create table";
+      throw new IllegalArgumentException(new NamespaceNotFoundException(null, namespace, info));
+    } else if (namespace.equals(Constants.SYSTEM_NAMESPACE)) {
       String info = "Can't create tables in the system namespace";
       throw new IllegalArgumentException(info);
     }
@@ -687,8 +687,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
     String namespace = Tables.extractNamespace(newTableName);
     if (!namespaceExists(namespace)) {
-      String info = "Table namespace not found while cloning table";
-      throw new IllegalArgumentException(new TableNamespaceNotFoundException(null, namespace, info));
+      String info = "Namespace not found while cloning table";
+      throw new IllegalArgumentException(new NamespaceNotFoundException(null, namespace, info));
     }
 
     String srcTableId = Tables.getTableId(instance, srcTableName);
@@ -725,13 +725,13 @@ public class TableOperationsImpl extends TableOperationsHelper {
     doTableOperation(TableOperation.CLONE, args, opts);
   }
 
-  // get the properties that are only in the table namespace so that we can exclude them when copying table properties.
+  // get the properties that are only in the namespace so that we can exclude them when copying table properties.
   // also, don't exclude properties that are going to be explicitly set.
   private HashSet<String> getUniqueNamespaceProperties(String namespace, String table, Map<String,String> propsToSet) throws TableNotFoundException,
       AccumuloException {
     HashSet<String> props = new HashSet<String>();
     try {
-      Iterable<Entry<String,String>> n = new TableNamespaceOperationsImpl(instance, credentials).getProperties(namespace);
+      Iterable<Entry<String,String>> n = new NamespaceOperationsImpl(instance, credentials).getProperties(namespace);
       Iterable<Entry<String,String>> t = getProperties(table);
       Map<String,String> tmap = new HashMap<String,String>();
       for (Entry<String,String> e : t) {
@@ -743,8 +743,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
           props.add(e.getKey());
         }
       }
-    } catch (TableNamespaceNotFoundException e) {
-      throw new IllegalStateException(new TableNamespaceNotFoundException(null, namespace, null));
+    } catch (NamespaceNotFoundException e) {
+      throw new IllegalStateException(new NamespaceNotFoundException(null, namespace, null));
     }
 
     for (Entry<String,String> e : propsToSet.entrySet()) {
@@ -775,8 +775,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
     String namespace = Tables.extractNamespace(newTableName);
     if (!namespaceExists(namespace)) {
-      String info = "Table namespace not found while renaming table";
-      throw new IllegalArgumentException(new TableNamespaceNotFoundException(null, namespace, info));
+      String info = "Namespace not found while renaming table";
+      throw new IllegalArgumentException(new NamespaceNotFoundException(null, namespace, info));
     }
 
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldTableName.getBytes()), ByteBuffer.wrap(newTableName.getBytes()));
@@ -1523,8 +1523,8 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
     String namespace = Tables.extractNamespace(tableName);
     if (!namespaceExists(namespace)) {
-      String info = "Table namespace not found while importing to table";
-      throw new RuntimeException(new TableNamespaceNotFoundException(null, namespace, info));
+      String info = "Namespace not found while importing to table";
+      throw new RuntimeException(new NamespaceNotFoundException(null, namespace, info));
     }
 
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes()), ByteBuffer.wrap(importDir.getBytes()));
@@ -1599,6 +1599,6 @@ public class TableOperationsImpl extends TableOperationsHelper {
   }
 
   private boolean namespaceExists(String namespace) {
-    return TableNamespaces.getNameToIdMap(instance).containsKey(namespace);
+    return Namespaces.getNameToIdMap(instance).containsKey(namespace);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
index bc3f39a..fcecb3a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
@@ -36,8 +36,8 @@ import org.apache.accumulo.core.client.admin.InstanceOperations;
 import org.apache.accumulo.core.client.admin.InstanceOperationsImpl;
 import org.apache.accumulo.core.client.admin.SecurityOperations;
 import org.apache.accumulo.core.client.admin.SecurityOperationsImpl;
-import org.apache.accumulo.core.client.admin.TableNamespaceOperations;
-import org.apache.accumulo.core.client.admin.TableNamespaceOperationsImpl;
+import org.apache.accumulo.core.client.admin.NamespaceOperations;
+import org.apache.accumulo.core.client.admin.NamespaceOperationsImpl;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.admin.TableOperationsImpl;
 import org.apache.accumulo.core.client.impl.thrift.ClientService;
@@ -53,7 +53,7 @@ public class ConnectorImpl extends Connector {
   private final Credentials credentials;
   private SecurityOperations secops = null;
   private TableOperations tableops = null;
-  private TableNamespaceOperations namespaceops = null;
+  private NamespaceOperations namespaceops = null;
   private InstanceOperations instanceops = null;
   
   public ConnectorImpl(final Instance instance, Credentials cred) throws AccumuloException, AccumuloSecurityException {
@@ -161,9 +161,9 @@ public class ConnectorImpl extends Connector {
   }
   
   @Override
-  public synchronized TableNamespaceOperations tableNamespaceOperations() {
+  public synchronized NamespaceOperations namespaceOperations() {
     if (namespaceops == null)
-      namespaceops = new TableNamespaceOperationsImpl(instance, credentials);
+      namespaceops = new NamespaceOperationsImpl(instance, credentials);
     return namespaceops;
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
new file mode 100644
index 0000000..0b5fff6
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
@@ -0,0 +1,107 @@
+/*
+ * 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.core.client.impl;
+
+import java.security.SecurityPermission;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.zookeeper.ZooCache;
+
+public class Namespaces {
+  private static SecurityPermission TABLES_PERMISSION = new SecurityPermission("tablesPermission");
+
+  private static ZooCache getZooCache(Instance instance) {
+    SecurityManager sm = System.getSecurityManager();
+    if (sm != null) {
+      sm.checkPermission(TABLES_PERMISSION);
+    }
+    return ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+  }
+
+  private static SortedMap<String,String> getMap(Instance instance, boolean nameAsKey) {
+    ZooCache zc = getZooCache(instance);
+
+    List<String> namespaceIds = zc.getChildren(ZooUtil.getRoot(instance) + Constants.ZNAMESPACES);
+
+    TreeMap<String,String> namespaceMap = new TreeMap<String,String>();
+
+    for (String id : namespaceIds) {
+      byte[] path = zc.get(ZooUtil.getRoot(instance) + Constants.ZNAMESPACES + "/" + id + Constants.ZNAMESPACE_NAME);
+      if (path != null) {
+        if (nameAsKey)
+          namespaceMap.put(new String(path, Constants.UTF8), id);
+        else
+          namespaceMap.put(id, new String(path, Constants.UTF8));
+      }
+    }
+    return namespaceMap;
+  }
+
+  public static String getNamespaceId(Instance instance, String namespace) throws NamespaceNotFoundException {
+    String id = getNameToIdMap(instance).get(namespace);
+    if (id == null)
+      throw new NamespaceNotFoundException(null, namespace, "getNamespaceId() failed to find namespace");
+    return id;
+  }
+
+  public static String getNamespaceName(Instance instance, String namespaceId) throws NamespaceNotFoundException {
+    String namespaceName = getIdToNameMap(instance).get(namespaceId);
+    if (namespaceName == null)
+      throw new NamespaceNotFoundException(namespaceId, null, "getNamespaceName() failed to find namespace");
+    return namespaceName;
+  }
+
+  public static SortedMap<String,String> getNameToIdMap(Instance instance) {
+    return getMap(instance, true);
+  }
+
+  public static SortedMap<String,String> getIdToNameMap(Instance instance) {
+    return getMap(instance, false);
+  }
+
+  public static List<String> getTableIds(Instance instance, String namespaceId) throws NamespaceNotFoundException {
+    List<String> l = new LinkedList<String>();
+    for (String id : Tables.getIdToNameMap(instance).keySet()) {
+      if (Tables.getNamespace(instance, id).equals(namespaceId)) {
+        l.add(id);
+      }
+    }
+    return l;
+  }
+
+  public static List<String> getTableNames(Instance instance, String namespaceId) throws NamespaceNotFoundException {
+    ZooCache zc = getZooCache(instance);
+    List<String> ids = getTableIds(instance, namespaceId);
+    List<String> names = new LinkedList<String>();
+    String namespace = getNamespaceName(instance, namespaceId) + ".";
+    if (namespaceId.equals(Constants.DEFAULT_NAMESPACE_ID) || namespaceId.equals(Constants.SYSTEM_NAMESPACE_ID)) {
+      // default and system namespaces aren't displayed for backwards compatibility
+      namespace = "";
+    }
+    for (String id : ids) {
+      names.add(namespace + new String(zc.get(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + id + Constants.ZTABLE_NAME), Constants.UTF8));
+    }
+    return names;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/impl/TableNamespaces.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TableNamespaces.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TableNamespaces.java
deleted file mode 100644
index 935cf16..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TableNamespaces.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.core.client.impl;
-
-import java.security.SecurityPermission;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.SortedMap;
-import java.util.TreeMap;
-
-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.zookeeper.ZooUtil;
-import org.apache.accumulo.fate.zookeeper.ZooCache;
-
-public class TableNamespaces {
-  private static SecurityPermission TABLES_PERMISSION = new SecurityPermission("tablesPermission");
-
-  private static ZooCache getZooCache(Instance instance) {
-    SecurityManager sm = System.getSecurityManager();
-    if (sm != null) {
-      sm.checkPermission(TABLES_PERMISSION);
-    }
-    return ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
-  }
-
-  private static SortedMap<String,String> getMap(Instance instance, boolean nameAsKey) {
-    ZooCache zc = getZooCache(instance);
-
-    List<String> namespaceIds = zc.getChildren(ZooUtil.getRoot(instance) + Constants.ZNAMESPACES);
-
-    TreeMap<String,String> namespaceMap = new TreeMap<String,String>();
-
-    for (String id : namespaceIds) {
-      byte[] path = zc.get(ZooUtil.getRoot(instance) + Constants.ZNAMESPACES + "/" + id + Constants.ZNAMESPACE_NAME);
-      if (path != null) {
-        if (nameAsKey)
-          namespaceMap.put(new String(path, Constants.UTF8), id);
-        else
-          namespaceMap.put(id, new String(path, Constants.UTF8));
-      }
-    }
-    return namespaceMap;
-  }
-
-  public static String getNamespaceId(Instance instance, String namespace) throws TableNamespaceNotFoundException {
-    String id = getNameToIdMap(instance).get(namespace);
-    if (id == null)
-      throw new TableNamespaceNotFoundException(null, namespace, "getNamespaceId() failed to find namespace");
-    return id;
-  }
-
-  public static String getNamespaceName(Instance instance, String namespaceId) throws TableNamespaceNotFoundException {
-    String namespaceName = getIdToNameMap(instance).get(namespaceId);
-    if (namespaceName == null)
-      throw new TableNamespaceNotFoundException(namespaceId, null, "getNamespaceName() failed to find namespace");
-    return namespaceName;
-  }
-
-  public static SortedMap<String,String> getNameToIdMap(Instance instance) {
-    return getMap(instance, true);
-  }
-
-  public static SortedMap<String,String> getIdToNameMap(Instance instance) {
-    return getMap(instance, false);
-  }
-
-  public static List<String> getTableIds(Instance instance, String namespaceId) throws TableNamespaceNotFoundException {
-    List<String> l = new LinkedList<String>();
-    for (String id : Tables.getIdToNameMap(instance).keySet()) {
-      if (Tables.getNamespace(instance, id).equals(namespaceId)) {
-        l.add(id);
-      }
-    }
-    return l;
-  }
-
-  public static List<String> getTableNames(Instance instance, String namespaceId) throws TableNamespaceNotFoundException {
-    ZooCache zc = getZooCache(instance);
-    List<String> ids = getTableIds(instance, namespaceId);
-    List<String> names = new LinkedList<String>();
-    String namespace = getNamespaceName(instance, namespaceId) + ".";
-    if (namespaceId.equals(Constants.DEFAULT_TABLE_NAMESPACE_ID) || namespaceId.equals(Constants.SYSTEM_TABLE_NAMESPACE_ID)) {
-      // default and system namespaces aren't displayed for backwards compatibility
-      namespace = "";
-    }
-    for (String id : ids) {
-      names.add(namespace + new String(zc.get(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + id + Constants.ZTABLE_NAME), Constants.UTF8));
-    }
-    return names;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
index f80f506..f017df4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
@@ -27,7 +27,7 @@ import jline.internal.Log;
 
 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.TableNotFoundException;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.metadata.MetadataTable;
@@ -61,10 +61,10 @@ public class Tables {
       // create fully qualified table name if it's in a namespace other than default or system.
       if (nId != null) {
         String namespaceId = new String(nId, Constants.UTF8);
-        if (!namespaceId.equals(Constants.DEFAULT_TABLE_NAMESPACE_ID) && !namespaceId.equals(Constants.SYSTEM_TABLE_NAMESPACE_ID)) {
+        if (!namespaceId.equals(Constants.DEFAULT_NAMESPACE_ID) && !namespaceId.equals(Constants.SYSTEM_NAMESPACE_ID)) {
           try {
-            name += TableNamespaces.getNamespaceName(instance, namespaceId) + ".";
-          } catch (TableNamespaceNotFoundException e) {
+            name += Namespaces.getNamespaceName(instance, namespaceId) + ".";
+          } catch (NamespaceNotFoundException e) {
             Log.error("Table (" + tableId + ") contains reference to namespace (" + namespaceId + ") that doesn't exist");
             continue;
           }
@@ -163,11 +163,11 @@ public class Tables {
   public static String extractNamespace(String tableName) {
     String[] s = tableName.split("\\.");
     if (tableName.equals(MetadataTable.NAME) || tableName.equals(RootTable.NAME)) {
-      return Constants.SYSTEM_TABLE_NAMESPACE;
+      return Constants.SYSTEM_NAMESPACE;
     } else if (s.length == 2 && !s[0].isEmpty()) {
       return s[0];
     } else {
-      return Constants.DEFAULT_TABLE_NAMESPACE;
+      return Constants.DEFAULT_NAMESPACE;
     }
   }
   


[16/50] [abbrv] git commit: ACCUMULO-802 added displaying namespace properties in the 'config' command

Posted by ct...@apache.org.
ACCUMULO-802 added displaying namespace properties in the 'config' command


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

Branch: refs/heads/master
Commit: 9db79fc70e9a78a9761d0b3458a6f7225f0da682
Parents: 5bcd3d2
Author: Sean Hickey <ta...@gmail.com>
Authored: Fri Aug 9 09:19:16 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:10 2013 -0500

----------------------------------------------------------------------
 .../core/client/admin/TableOperationsImpl.java  |  1 -
 .../core/util/shell/commands/ConfigCommand.java | 30 +++++++++++++++++---
 2 files changed, 26 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/9db79fc7/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
index b1ab058..cfaa657 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
@@ -218,7 +218,6 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes()), ByteBuffer.wrap(timeType.name().getBytes()));
 
-    // Map<String,String> opts = IteratorUtil.generateInitialTableProperties(limitVersion);
     Map<String,String> opts = new HashMap<String,String>();
 
     String namespace = Tables.extractNamespace(tableName);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9db79fc7/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ConfigCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ConfigCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ConfigCommand.java
index c2f93be..1e49c4d 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ConfigCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ConfigCommand.java
@@ -30,6 +30,8 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.impl.TableNamespaces;
+import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.security.ColumnVisibility;
@@ -46,7 +48,7 @@ import org.apache.commons.cli.Options;
 public class ConfigCommand extends Command {
   private Option tableOpt, deleteOpt, setOpt, filterOpt, disablePaginationOpt, outputFileOpt, tableNamespaceOpt;
   
-  private int COL1 = 8, COL2 = 7;
+  private int COL1 = 10, COL2 = 7;
   private ConsoleReader reader;
   
   @Override
@@ -72,7 +74,7 @@ public class ConfigCommand extends Command {
     }
     final String tableNamespace = cl.getOptionValue(tableNamespaceOpt.getOpt());
     if (tableNamespace != null && !shellState.getConnector().tableNamespaceOperations().exists(tableNamespace)) {
-      throw new TableNotFoundException(null, tableName, null);
+      throw new TableNamespaceNotFoundException(null, tableNamespace, null);
     }
     if (cl.hasOption(deleteOpt.getOpt())) {
       // delete property from table
@@ -149,6 +151,16 @@ public class ConfigCommand extends Command {
       for (Entry<String,String> defaultEntry : AccumuloConfiguration.getDefaultConfiguration()) {
         defaults.put(defaultEntry.getKey(), defaultEntry.getValue());
       }
+      
+      final TreeMap<String,String> namespaceConfig = new TreeMap<String,String>();
+      if (tableName != null) {
+        String n = TableNamespaces.getNamespaceName(shellState.getInstance(),
+            Tables.getNamespace(shellState.getInstance(), Tables.getTableId(shellState.getInstance(), tableName)));
+        for (Entry<String,String> e : shellState.getConnector().tableNamespaceOperations().getProperties(n)) {
+          namespaceConfig.put(e.getKey(), e.getValue());
+        }
+      }
+      
       Iterable<Entry<String,String>> acuconf = shellState.getConnector().instanceOperations().getSystemConfiguration().entrySet();
       if (tableName != null) {
         acuconf = shellState.getConnector().tableOperations().getProperties(tableName);
@@ -191,6 +203,7 @@ public class ConfigCommand extends Command {
         String sysVal = systemConfig.get(key);
         String curVal = propEntry.getValue();
         String dfault = defaults.get(key);
+        String nspVal = namespaceConfig.get(key);
         boolean printed = false;
         
         if (dfault != null && key.toLowerCase().contains("password")) {
@@ -209,11 +222,20 @@ public class ConfigCommand extends Command {
             printConfLine(output, "system", printed ? "   @override" : key, sysVal == null ? "" : sysVal);
             printed = true;
           }
+          
+        }
+        if (nspVal != null) {
+          if (!systemConfig.containsKey(key) || !sysVal.equals(nspVal)) {
+            printConfLine(output, "namespace", printed ? "   @override" : key, nspVal == null ? "" : nspVal);
+            printed = true;
+          }
         }
         
         // show per-table value only if it is different (overridden)
-        if ((tableName != null || tableNamespace != null) && !curVal.equals(sysVal)) {
+        if (tableName != null && !curVal.equals(nspVal)) {
           printConfLine(output, "table", printed ? "   @override" : key, curVal);
+        } else if (tableNamespace != null && !curVal.equals(sysVal)) {
+          printConfLine(output, "namespace", printed ? "   @override" : key, curVal);
         }
       }
       printConfFooter(output);
@@ -262,7 +284,7 @@ public class ConfigCommand extends Command {
     disablePaginationOpt = new Option("np", "no-pagination", false, "disables pagination of output");
     outputFileOpt = new Option("o", "output", true, "local file to write the scan output to");
     tableNamespaceOpt = new Option(Shell.tableNamespaceOption, "table-namespace", true, "table namespace to display/set/delete properties for");
-
+    
     tableOpt.setArgName("table");
     deleteOpt.setArgName("property");
     setOpt.setArgName("property=value");


[21/50] [abbrv] git commit: ACCUMULO-802 fixed deadlock problem with the table namespace locks during fate operations

Posted by ct...@apache.org.
ACCUMULO-802 fixed deadlock problem with the table namespace locks during fate operations


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

Branch: refs/heads/master
Commit: c2ed43e4b58cc2278b9955fc36ad9ba788a171c2
Parents: e1cf746
Author: Sean Hickey <ta...@gmail.com>
Authored: Tue Aug 13 15:10:51 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:10 2013 -0500

----------------------------------------------------------------------
 .../java/org/apache/accumulo/master/Master.java   | 13 +++++++++----
 .../accumulo/master/tableOps/CloneTable.java      | 18 ++++++++++--------
 .../accumulo/master/tableOps/CreateTable.java     |  5 +++--
 .../accumulo/master/tableOps/DeleteTable.java     | 10 ++++++----
 .../accumulo/master/tableOps/RenameTable.java     | 10 +++++-----
 .../test/randomwalk/concurrent/Setup.java         |  2 +-
 .../system/randomwalk/conf/modules/Concurrent.xml |  2 +-
 7 files changed, 35 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2ed43e4/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 0af32d4..338550a 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
@@ -42,6 +42,7 @@ import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.IteratorSetting;
 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;
@@ -927,7 +928,11 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
 
           TimeType timeType = TimeType.valueOf(ByteBufferUtil.toString(arguments.get(1)));
 
-          fate.seedTransaction(opid, new TraceRepo<Master>(new CreateTable(c.getPrincipal(), tableName, timeType, options)), autoCleanup);
+          try {
+            fate.seedTransaction(opid, new TraceRepo<Master>(new CreateTable(c.getPrincipal(), tableName, timeType, options, getInstance())), autoCleanup);
+          } catch (TableNamespaceNotFoundException e) {
+            throw new ThriftTableOperationException(null, tableName, TableOperation.CREATE, TableOperationExceptionType.NOTFOUND, e.getMessage());
+          }
           break;
         }
         case RENAME: {
@@ -941,7 +946,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
           if (!security.canRenameTable(c, tableId, oldTableName, newTableName))
             throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-          fate.seedTransaction(opid, new TraceRepo<Master>(new RenameTable(tableId, oldTableName, newTableName)), autoCleanup);
+          fate.seedTransaction(opid, new TraceRepo<Master>(new RenameTable(tableId, oldTableName, newTableName, getInstance())), autoCleanup);
           
           break;
         }
@@ -970,7 +975,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
             propertiesToSet.put(entry.getKey(), entry.getValue());
           }
 
-          fate.seedTransaction(opid, new TraceRepo<Master>(new CloneTable(c.getPrincipal(), srcTableId, tableName, propertiesToSet, propertiesToExclude)),
+          fate.seedTransaction(opid, new TraceRepo<Master>(new CloneTable(c.getPrincipal(), srcTableId, tableName, propertiesToSet, propertiesToExclude, getInstance())),
               autoCleanup);
 
           break;
@@ -981,7 +986,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
           checkNotMetadataTable(tableName, TableOperation.DELETE);
           if (!security.canDeleteTable(c, tableId))
             throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-          fate.seedTransaction(opid, new TraceRepo<Master>(new DeleteTable(tableId)), autoCleanup);
+          fate.seedTransaction(opid, new TraceRepo<Master>(new DeleteTable(tableId, getInstance())), autoCleanup);
           break;
         }
         case ONLINE: {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2ed43e4/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 31e5e3e..46edc25 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
@@ -139,9 +139,10 @@ class CloneZookeeper extends MasterRepo {
   @Override
   public long isReady(long tid, Master environment) throws Exception {
     cloneInfo.namespaceId = TableNamespaces.getNamespaceId(environment.getInstance(), Tables.extractNamespace(cloneInfo.tableName));
-    long val = Utils.reserveTable(cloneInfo.tableId, tid, true, false, TableOperation.CLONE);
-    if (!cloneInfo.srcNamespaceId.equals(cloneInfo.namespaceId)) 
+    long val = 0;
+    if (!cloneInfo.srcNamespaceId.equals(cloneInfo.namespaceId))
       val += Utils.reserveTableNamespace(cloneInfo.namespaceId, tid, false, true, TableOperation.CLONE);
+    val += Utils.reserveTable(cloneInfo.tableId, tid, true, false, TableOperation.CLONE);
     return val;
   }
   
@@ -157,7 +158,7 @@ class CloneZookeeper extends MasterRepo {
       TableManager.getInstance().cloneTable(cloneInfo.srcTableId, cloneInfo.tableId, cloneInfo.tableName, cloneInfo.propertiesToSet,
           cloneInfo.propertiesToExclude, NodeExistsPolicy.OVERWRITE);
       Tables.clearCache(instance);
-
+      
       TableManager.getInstance().addNamespaceToTable(cloneInfo.tableId, cloneInfo.namespaceId);
       
       return new CloneMetadata(cloneInfo);
@@ -171,7 +172,7 @@ class CloneZookeeper extends MasterRepo {
     Instance instance = HdfsZooInstance.getInstance();
     TableManager.getInstance().removeTable(cloneInfo.tableId);
     Utils.unreserveTable(cloneInfo.tableId, tid, true);
-    if (!cloneInfo.namespaceId.equals(cloneInfo.srcNamespaceId)) 
+    if (!cloneInfo.namespaceId.equals(cloneInfo.srcNamespaceId))
       Utils.unreserveTableNamespace(cloneInfo.namespaceId, tid, false);
     Tables.clearCache(instance);
   }
@@ -223,20 +224,21 @@ public class CloneTable extends MasterRepo {
   private static final long serialVersionUID = 1L;
   private CloneInfo cloneInfo;
   
-  public CloneTable(String user, String srcTableId, String tableName, Map<String,String> propertiesToSet, Set<String> propertiesToExclude) {
+  public CloneTable(String user, String srcTableId, String tableName, Map<String,String> propertiesToSet, Set<String> propertiesToExclude, Instance inst) {
     cloneInfo = new CloneInfo();
     cloneInfo.user = user;
     cloneInfo.srcTableId = srcTableId;
     cloneInfo.tableName = tableName;
     cloneInfo.propertiesToExclude = propertiesToExclude;
     cloneInfo.propertiesToSet = propertiesToSet;
+    cloneInfo.srcNamespaceId = Tables.getNamespace(inst, cloneInfo.srcTableId);
   }
   
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    cloneInfo.srcNamespaceId = Tables.getNamespace(environment.getInstance(), cloneInfo.srcTableId);
-    long val = Utils.reserveTable(cloneInfo.srcTableId, tid, false, true, TableOperation.CLONE);
-    val += Utils.reserveTableNamespace(cloneInfo.srcNamespaceId, tid, false, true, TableOperation.CLONE);
+    
+    long val = Utils.reserveTableNamespace(cloneInfo.srcNamespaceId, tid, false, true, TableOperation.CLONE);
+    val += Utils.reserveTable(cloneInfo.srcTableId, tid, false, true, TableOperation.CLONE);
     return val;
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2ed43e4/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 217bfda..f657a1b 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,6 +22,7 @@ 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.admin.TimeType;
 import org.apache.accumulo.core.client.impl.TableNamespaces;
 import org.apache.accumulo.core.client.impl.Tables;
@@ -279,18 +280,18 @@ public class CreateTable extends MasterRepo {
   
   private TableInfo tableInfo;
   
-  public CreateTable(String user, String tableName, TimeType timeType, Map<String,String> props) {
+  public CreateTable(String user, String tableName, TimeType timeType, Map<String,String> props, Instance inst) throws TableNamespaceNotFoundException {
     tableInfo = new TableInfo();
     tableInfo.tableName = tableName;
     tableInfo.timeType = TabletTime.getTimeID(timeType);
     tableInfo.user = user;
     tableInfo.props = props;
+    tableInfo.namespaceId = TableNamespaces.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
-    tableInfo.namespaceId = TableNamespaces.getNamespaceId(environment.getInstance(), Tables.extractNamespace(tableInfo.tableName));
     return Utils.reserveTableNamespace(tableInfo.namespaceId, tid, false, true, TableOperation.CREATE);
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2ed43e4/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 439d0c2..98d1ae9 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
@@ -22,6 +22,7 @@ import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.impl.Tables;
@@ -220,15 +221,16 @@ public class DeleteTable extends MasterRepo {
   
   private String tableId, namespaceId;
   
-  public DeleteTable(String tableId) {
+  public DeleteTable(String tableId, Instance inst) {
     this.tableId = tableId;
+    this.namespaceId = Tables.getNamespace(inst, tableId);
   }
   
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    this.namespaceId = Tables.getNamespace(environment.getInstance(), tableId);
-    return Utils.reserveTable(tableId, tid, true, true, TableOperation.DELETE)
-        + Utils.reserveTableNamespace(namespaceId, tid, false, false, TableOperation.CREATE);
+    
+    return Utils.reserveTableNamespace(namespaceId, tid, false, false, TableOperation.DELETE)
+        + Utils.reserveTable(tableId, tid, true, true, TableOperation.DELETE);
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2ed43e4/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
index 8b82593..871c0a0 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
@@ -42,15 +42,15 @@ public class RenameTable extends MasterRepo {
 
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    this.namespaceId = Tables.getNamespace(environment.getInstance(), tableId);
-    return Utils.reserveTable(tableId, tid, true, true, TableOperation.RENAME)
-        + Utils.reserveTableNamespace(namespaceId, tid, false, true, TableOperation.RENAME);
+    return Utils.reserveTableNamespace(namespaceId, tid, false, true, TableOperation.RENAME)
+        + Utils.reserveTable(tableId, tid, true, true, TableOperation.RENAME);
   }
 
-  public RenameTable(String tableId, String oldTableName, String newTableName) {
+  public RenameTable(String tableId, String oldTableName, String newTableName, Instance inst) {
     this.tableId = tableId;
     this.oldTableName = oldTableName;
     this.newTableName = newTableName;
+    this.namespaceId = Tables.getNamespace(inst, tableId);
   }
 
   @Override
@@ -96,7 +96,7 @@ public class RenameTable extends MasterRepo {
     } finally {
       Utils.tableNameLock.unlock();
       Utils.unreserveTable(tableId, tid, true);
-      Utils.unreserveTableNamespace(namespaceId, tid, false);
+      Utils.unreserveTableNamespace(this.namespaceId, tid, false);
     }
 
     Logger.getLogger(RenameTable.class).debug("Renamed table " + tableId + " " + oldTableName + " " + newTableName);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2ed43e4/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Setup.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Setup.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Setup.java
index 7b077ee..edab2b5 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Setup.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Setup.java
@@ -31,7 +31,7 @@ public class Setup extends Test {
     Random rand = new Random();
     state.set("rand", rand);
     
-    int numTables = Integer.parseInt(props.getProperty("numTables", "15"));
+    int numTables = Integer.parseInt(props.getProperty("numTables", "9"));
     int numNamespaces = Integer.parseInt(props.getProperty("numNamespaces", "2"));
     log.debug("numTables = " + numTables);
     log.debug("numNamespaces = " + numNamespaces);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c2ed43e4/test/system/randomwalk/conf/modules/Concurrent.xml
----------------------------------------------------------------------
diff --git a/test/system/randomwalk/conf/modules/Concurrent.xml b/test/system/randomwalk/conf/modules/Concurrent.xml
index cce8675..cb057c2 100644
--- a/test/system/randomwalk/conf/modules/Concurrent.xml
+++ b/test/system/randomwalk/conf/modules/Concurrent.xml
@@ -59,7 +59,7 @@
 </node>
 
 <node id="ct.Setup">
-  <property key="numTables" value="15"/>
+  <property key="numTables" value="9"/>
   <property key="numNamespaces" value="2"/>
   <edge id="ct.CreateTable" weight="1"/>
 </node>


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

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterClientService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterClientService.java b/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterClientService.java
index f5d18d7..a43fb03 100644
--- a/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterClientService.java
+++ b/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterClientService.java
@@ -58,9 +58,9 @@ import org.slf4j.LoggerFactory;
 
     public void removeTableProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String tableName, String property) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException;
 
-    public void setTableNamespaceProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property, String value) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException;
+    public void setNamespaceProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property, String value) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException;
 
-    public void removeTableNamespaceProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException;
+    public void removeNamespaceProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException;
 
     public void setMasterGoalState(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, MasterGoalState state) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
 
@@ -86,13 +86,13 @@ import org.slf4j.LoggerFactory;
 
     public void finishTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
 
-    public long beginTableNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+    public long beginNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
 
-    public void executeTableNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException;
+    public void executeNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException;
 
-    public String waitForTableNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException;
+    public String waitForNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException;
 
-    public void finishTableNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
+    public void finishNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
 
   }
 
@@ -106,9 +106,9 @@ import org.slf4j.LoggerFactory;
 
     public void removeTableProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String tableName, String property, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.removeTableProperty_call> resultHandler) throws org.apache.thrift.TException;
 
-    public void setTableNamespaceProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property, String value, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.setTableNamespaceProperty_call> resultHandler) throws org.apache.thrift.TException;
+    public void setNamespaceProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property, String value, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.setNamespaceProperty_call> resultHandler) throws org.apache.thrift.TException;
 
-    public void removeTableNamespaceProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.removeTableNamespaceProperty_call> resultHandler) throws org.apache.thrift.TException;
+    public void removeNamespaceProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.removeNamespaceProperty_call> resultHandler) throws org.apache.thrift.TException;
 
     public void setMasterGoalState(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, MasterGoalState state, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.setMasterGoalState_call> resultHandler) throws org.apache.thrift.TException;
 
@@ -134,13 +134,13 @@ import org.slf4j.LoggerFactory;
 
     public void finishTableOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.finishTableOperation_call> resultHandler) throws org.apache.thrift.TException;
 
-    public void beginTableNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.beginTableNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException;
+    public void beginNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.beginNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException;
 
-    public void executeTableNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.executeTableNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException;
+    public void executeNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.executeNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException;
 
-    public void waitForTableNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.waitForTableNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException;
+    public void waitForNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.waitForNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException;
 
-    public void finishTableNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.finishTableNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException;
+    public void finishNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.finishNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException;
 
   }
 
@@ -286,27 +286,27 @@ import org.slf4j.LoggerFactory;
       return;
     }
 
-    public void setTableNamespaceProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property, String value) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
+    public void setNamespaceProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property, String value) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
     {
-      send_setTableNamespaceProperty(tinfo, credentials, ns, property, value);
-      recv_setTableNamespaceProperty();
+      send_setNamespaceProperty(tinfo, credentials, ns, property, value);
+      recv_setNamespaceProperty();
     }
 
-    public void send_setTableNamespaceProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property, String value) throws org.apache.thrift.TException
+    public void send_setNamespaceProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property, String value) throws org.apache.thrift.TException
     {
-      setTableNamespaceProperty_args args = new setTableNamespaceProperty_args();
+      setNamespaceProperty_args args = new setNamespaceProperty_args();
       args.setTinfo(tinfo);
       args.setCredentials(credentials);
       args.setNs(ns);
       args.setProperty(property);
       args.setValue(value);
-      sendBase("setTableNamespaceProperty", args);
+      sendBase("setNamespaceProperty", args);
     }
 
-    public void recv_setTableNamespaceProperty() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
+    public void recv_setNamespaceProperty() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
     {
-      setTableNamespaceProperty_result result = new setTableNamespaceProperty_result();
-      receiveBase(result, "setTableNamespaceProperty");
+      setNamespaceProperty_result result = new setNamespaceProperty_result();
+      receiveBase(result, "setNamespaceProperty");
       if (result.sec != null) {
         throw result.sec;
       }
@@ -316,26 +316,26 @@ import org.slf4j.LoggerFactory;
       return;
     }
 
-    public void removeTableNamespaceProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
+    public void removeNamespaceProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
     {
-      send_removeTableNamespaceProperty(tinfo, credentials, ns, property);
-      recv_removeTableNamespaceProperty();
+      send_removeNamespaceProperty(tinfo, credentials, ns, property);
+      recv_removeNamespaceProperty();
     }
 
-    public void send_removeTableNamespaceProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property) throws org.apache.thrift.TException
+    public void send_removeNamespaceProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property) throws org.apache.thrift.TException
     {
-      removeTableNamespaceProperty_args args = new removeTableNamespaceProperty_args();
+      removeNamespaceProperty_args args = new removeNamespaceProperty_args();
       args.setTinfo(tinfo);
       args.setCredentials(credentials);
       args.setNs(ns);
       args.setProperty(property);
-      sendBase("removeTableNamespaceProperty", args);
+      sendBase("removeNamespaceProperty", args);
     }
 
-    public void recv_removeTableNamespaceProperty() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
+    public void recv_removeNamespaceProperty() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
     {
-      removeTableNamespaceProperty_result result = new removeTableNamespaceProperty_result();
-      receiveBase(result, "removeTableNamespaceProperty");
+      removeNamespaceProperty_result result = new removeNamespaceProperty_result();
+      receiveBase(result, "removeNamespaceProperty");
       if (result.sec != null) {
         throw result.sec;
       }
@@ -645,42 +645,42 @@ import org.slf4j.LoggerFactory;
       return;
     }
 
-    public long beginTableNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    public long beginNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
     {
-      send_beginTableNamespaceOperation(tinfo, credentials);
-      return recv_beginTableNamespaceOperation();
+      send_beginNamespaceOperation(tinfo, credentials);
+      return recv_beginNamespaceOperation();
     }
 
-    public void send_beginTableNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.thrift.TException
+    public void send_beginNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.thrift.TException
     {
-      beginTableNamespaceOperation_args args = new beginTableNamespaceOperation_args();
+      beginNamespaceOperation_args args = new beginNamespaceOperation_args();
       args.setTinfo(tinfo);
       args.setCredentials(credentials);
-      sendBase("beginTableNamespaceOperation", args);
+      sendBase("beginNamespaceOperation", args);
     }
 
-    public long recv_beginTableNamespaceOperation() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    public long recv_beginNamespaceOperation() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
     {
-      beginTableNamespaceOperation_result result = new beginTableNamespaceOperation_result();
-      receiveBase(result, "beginTableNamespaceOperation");
+      beginNamespaceOperation_result result = new beginNamespaceOperation_result();
+      receiveBase(result, "beginNamespaceOperation");
       if (result.isSetSuccess()) {
         return result.success;
       }
       if (result.sec != null) {
         throw result.sec;
       }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "beginTableNamespaceOperation failed: unknown result");
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "beginNamespaceOperation failed: unknown result");
     }
 
-    public void executeTableNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
+    public void executeNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
     {
-      send_executeTableNamespaceOperation(tinfo, credentials, opid, op, arguments, options, autoClean);
-      recv_executeTableNamespaceOperation();
+      send_executeNamespaceOperation(tinfo, credentials, opid, op, arguments, options, autoClean);
+      recv_executeNamespaceOperation();
     }
 
-    public void send_executeTableNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean) throws org.apache.thrift.TException
+    public void send_executeNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean) throws org.apache.thrift.TException
     {
-      executeTableNamespaceOperation_args args = new executeTableNamespaceOperation_args();
+      executeNamespaceOperation_args args = new executeNamespaceOperation_args();
       args.setTinfo(tinfo);
       args.setCredentials(credentials);
       args.setOpid(opid);
@@ -688,13 +688,13 @@ import org.slf4j.LoggerFactory;
       args.setArguments(arguments);
       args.setOptions(options);
       args.setAutoClean(autoClean);
-      sendBase("executeTableNamespaceOperation", args);
+      sendBase("executeNamespaceOperation", args);
     }
 
-    public void recv_executeTableNamespaceOperation() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
+    public void recv_executeNamespaceOperation() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
     {
-      executeTableNamespaceOperation_result result = new executeTableNamespaceOperation_result();
-      receiveBase(result, "executeTableNamespaceOperation");
+      executeNamespaceOperation_result result = new executeNamespaceOperation_result();
+      receiveBase(result, "executeNamespaceOperation");
       if (result.sec != null) {
         throw result.sec;
       }
@@ -704,25 +704,25 @@ import org.slf4j.LoggerFactory;
       return;
     }
 
-    public String waitForTableNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
+    public String waitForNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
     {
-      send_waitForTableNamespaceOperation(tinfo, credentials, opid);
-      return recv_waitForTableNamespaceOperation();
+      send_waitForNamespaceOperation(tinfo, credentials, opid);
+      return recv_waitForNamespaceOperation();
     }
 
-    public void send_waitForTableNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
+    public void send_waitForNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
     {
-      waitForTableNamespaceOperation_args args = new waitForTableNamespaceOperation_args();
+      waitForNamespaceOperation_args args = new waitForNamespaceOperation_args();
       args.setTinfo(tinfo);
       args.setCredentials(credentials);
       args.setOpid(opid);
-      sendBase("waitForTableNamespaceOperation", args);
+      sendBase("waitForNamespaceOperation", args);
     }
 
-    public String recv_waitForTableNamespaceOperation() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
+    public String recv_waitForNamespaceOperation() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException, org.apache.thrift.TException
     {
-      waitForTableNamespaceOperation_result result = new waitForTableNamespaceOperation_result();
-      receiveBase(result, "waitForTableNamespaceOperation");
+      waitForNamespaceOperation_result result = new waitForNamespaceOperation_result();
+      receiveBase(result, "waitForNamespaceOperation");
       if (result.isSetSuccess()) {
         return result.success;
       }
@@ -732,28 +732,28 @@ import org.slf4j.LoggerFactory;
       if (result.tope != null) {
         throw result.tope;
       }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "waitForTableNamespaceOperation failed: unknown result");
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "waitForNamespaceOperation failed: unknown result");
     }
 
-    public void finishTableNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    public void finishNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
     {
-      send_finishTableNamespaceOperation(tinfo, credentials, opid);
-      recv_finishTableNamespaceOperation();
+      send_finishNamespaceOperation(tinfo, credentials, opid);
+      recv_finishNamespaceOperation();
     }
 
-    public void send_finishTableNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
+    public void send_finishNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid) throws org.apache.thrift.TException
     {
-      finishTableNamespaceOperation_args args = new finishTableNamespaceOperation_args();
+      finishNamespaceOperation_args args = new finishNamespaceOperation_args();
       args.setTinfo(tinfo);
       args.setCredentials(credentials);
       args.setOpid(opid);
-      sendBase("finishTableNamespaceOperation", args);
+      sendBase("finishNamespaceOperation", args);
     }
 
-    public void recv_finishTableNamespaceOperation() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
+    public void recv_finishNamespaceOperation() throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException
     {
-      finishTableNamespaceOperation_result result = new finishTableNamespaceOperation_result();
-      receiveBase(result, "finishTableNamespaceOperation");
+      finishNamespaceOperation_result result = new finishNamespaceOperation_result();
+      receiveBase(result, "finishNamespaceOperation");
       if (result.sec != null) {
         throw result.sec;
       }
@@ -951,20 +951,20 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public void setTableNamespaceProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property, String value, org.apache.thrift.async.AsyncMethodCallback<setTableNamespaceProperty_call> resultHandler) throws org.apache.thrift.TException {
+    public void setNamespaceProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property, String value, org.apache.thrift.async.AsyncMethodCallback<setNamespaceProperty_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      setTableNamespaceProperty_call method_call = new setTableNamespaceProperty_call(tinfo, credentials, ns, property, value, resultHandler, this, ___protocolFactory, ___transport);
+      setNamespaceProperty_call method_call = new setNamespaceProperty_call(tinfo, credentials, ns, property, value, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class setTableNamespaceProperty_call extends org.apache.thrift.async.TAsyncMethodCall {
+    public static class setNamespaceProperty_call extends org.apache.thrift.async.TAsyncMethodCall {
       private org.apache.accumulo.trace.thrift.TInfo tinfo;
       private org.apache.accumulo.core.security.thrift.TCredentials credentials;
       private String ns;
       private String property;
       private String value;
-      public setTableNamespaceProperty_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property, String value, org.apache.thrift.async.AsyncMethodCallback<setTableNamespaceProperty_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public setNamespaceProperty_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property, String value, org.apache.thrift.async.AsyncMethodCallback<setNamespaceProperty_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
@@ -974,8 +974,8 @@ import org.slf4j.LoggerFactory;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("setTableNamespaceProperty", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        setTableNamespaceProperty_args args = new setTableNamespaceProperty_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("setNamespaceProperty", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        setNamespaceProperty_args args = new setNamespaceProperty_args();
         args.setTinfo(tinfo);
         args.setCredentials(credentials);
         args.setNs(ns);
@@ -991,23 +991,23 @@ import org.slf4j.LoggerFactory;
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        (new Client(prot)).recv_setTableNamespaceProperty();
+        (new Client(prot)).recv_setNamespaceProperty();
       }
     }
 
-    public void removeTableNamespaceProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property, org.apache.thrift.async.AsyncMethodCallback<removeTableNamespaceProperty_call> resultHandler) throws org.apache.thrift.TException {
+    public void removeNamespaceProperty(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property, org.apache.thrift.async.AsyncMethodCallback<removeNamespaceProperty_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      removeTableNamespaceProperty_call method_call = new removeTableNamespaceProperty_call(tinfo, credentials, ns, property, resultHandler, this, ___protocolFactory, ___transport);
+      removeNamespaceProperty_call method_call = new removeNamespaceProperty_call(tinfo, credentials, ns, property, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class removeTableNamespaceProperty_call extends org.apache.thrift.async.TAsyncMethodCall {
+    public static class removeNamespaceProperty_call extends org.apache.thrift.async.TAsyncMethodCall {
       private org.apache.accumulo.trace.thrift.TInfo tinfo;
       private org.apache.accumulo.core.security.thrift.TCredentials credentials;
       private String ns;
       private String property;
-      public removeTableNamespaceProperty_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property, org.apache.thrift.async.AsyncMethodCallback<removeTableNamespaceProperty_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public removeNamespaceProperty_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String ns, String property, org.apache.thrift.async.AsyncMethodCallback<removeNamespaceProperty_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
@@ -1016,8 +1016,8 @@ import org.slf4j.LoggerFactory;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("removeTableNamespaceProperty", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        removeTableNamespaceProperty_args args = new removeTableNamespaceProperty_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("removeNamespaceProperty", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        removeNamespaceProperty_args args = new removeNamespaceProperty_args();
         args.setTinfo(tinfo);
         args.setCredentials(credentials);
         args.setNs(ns);
@@ -1032,7 +1032,7 @@ import org.slf4j.LoggerFactory;
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        (new Client(prot)).recv_removeTableNamespaceProperty();
+        (new Client(prot)).recv_removeNamespaceProperty();
       }
     }
 
@@ -1511,25 +1511,25 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public void beginTableNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<beginTableNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException {
+    public void beginNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<beginNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      beginTableNamespaceOperation_call method_call = new beginTableNamespaceOperation_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
+      beginNamespaceOperation_call method_call = new beginNamespaceOperation_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class beginTableNamespaceOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
+    public static class beginNamespaceOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
       private org.apache.accumulo.trace.thrift.TInfo tinfo;
       private org.apache.accumulo.core.security.thrift.TCredentials credentials;
-      public beginTableNamespaceOperation_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<beginTableNamespaceOperation_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public beginNamespaceOperation_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback<beginNamespaceOperation_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("beginTableNamespaceOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        beginTableNamespaceOperation_args args = new beginTableNamespaceOperation_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("beginNamespaceOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        beginNamespaceOperation_args args = new beginNamespaceOperation_args();
         args.setTinfo(tinfo);
         args.setCredentials(credentials);
         args.write(prot);
@@ -1542,18 +1542,18 @@ import org.slf4j.LoggerFactory;
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_beginTableNamespaceOperation();
+        return (new Client(prot)).recv_beginNamespaceOperation();
       }
     }
 
-    public void executeTableNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<executeTableNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException {
+    public void executeNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<executeNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      executeTableNamespaceOperation_call method_call = new executeTableNamespaceOperation_call(tinfo, credentials, opid, op, arguments, options, autoClean, resultHandler, this, ___protocolFactory, ___transport);
+      executeNamespaceOperation_call method_call = new executeNamespaceOperation_call(tinfo, credentials, opid, op, arguments, options, autoClean, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class executeTableNamespaceOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
+    public static class executeNamespaceOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
       private org.apache.accumulo.trace.thrift.TInfo tinfo;
       private org.apache.accumulo.core.security.thrift.TCredentials credentials;
       private long opid;
@@ -1561,7 +1561,7 @@ import org.slf4j.LoggerFactory;
       private List<ByteBuffer> arguments;
       private Map<String,String> options;
       private boolean autoClean;
-      public executeTableNamespaceOperation_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<executeTableNamespaceOperation_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public executeNamespaceOperation_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, TableOperation op, List<ByteBuffer> arguments, Map<String,String> options, boolean autoClean, org.apache.thrift.async.AsyncMethodCallback<executeNamespaceOperation_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
@@ -1573,8 +1573,8 @@ import org.slf4j.LoggerFactory;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("executeTableNamespaceOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        executeTableNamespaceOperation_args args = new executeTableNamespaceOperation_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("executeNamespaceOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        executeNamespaceOperation_args args = new executeNamespaceOperation_args();
         args.setTinfo(tinfo);
         args.setCredentials(credentials);
         args.setOpid(opid);
@@ -1592,22 +1592,22 @@ import org.slf4j.LoggerFactory;
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        (new Client(prot)).recv_executeTableNamespaceOperation();
+        (new Client(prot)).recv_executeNamespaceOperation();
       }
     }
 
-    public void waitForTableNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<waitForTableNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException {
+    public void waitForNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<waitForNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      waitForTableNamespaceOperation_call method_call = new waitForTableNamespaceOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport);
+      waitForNamespaceOperation_call method_call = new waitForNamespaceOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class waitForTableNamespaceOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
+    public static class waitForNamespaceOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
       private org.apache.accumulo.trace.thrift.TInfo tinfo;
       private org.apache.accumulo.core.security.thrift.TCredentials credentials;
       private long opid;
-      public waitForTableNamespaceOperation_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<waitForTableNamespaceOperation_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public waitForNamespaceOperation_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<waitForNamespaceOperation_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
@@ -1615,8 +1615,8 @@ import org.slf4j.LoggerFactory;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("waitForTableNamespaceOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        waitForTableNamespaceOperation_args args = new waitForTableNamespaceOperation_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("waitForNamespaceOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        waitForNamespaceOperation_args args = new waitForNamespaceOperation_args();
         args.setTinfo(tinfo);
         args.setCredentials(credentials);
         args.setOpid(opid);
@@ -1630,22 +1630,22 @@ import org.slf4j.LoggerFactory;
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_waitForTableNamespaceOperation();
+        return (new Client(prot)).recv_waitForNamespaceOperation();
       }
     }
 
-    public void finishTableNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<finishTableNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException {
+    public void finishNamespaceOperation(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<finishNamespaceOperation_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      finishTableNamespaceOperation_call method_call = new finishTableNamespaceOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport);
+      finishNamespaceOperation_call method_call = new finishNamespaceOperation_call(tinfo, credentials, opid, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
-    public static class finishTableNamespaceOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
+    public static class finishNamespaceOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
       private org.apache.accumulo.trace.thrift.TInfo tinfo;
       private org.apache.accumulo.core.security.thrift.TCredentials credentials;
       private long opid;
-      public finishTableNamespaceOperation_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<finishTableNamespaceOperation_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+      public finishNamespaceOperation_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, long opid, org.apache.thrift.async.AsyncMethodCallback<finishNamespaceOperation_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
         super(client, protocolFactory, transport, resultHandler, false);
         this.tinfo = tinfo;
         this.credentials = credentials;
@@ -1653,8 +1653,8 @@ import org.slf4j.LoggerFactory;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("finishTableNamespaceOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        finishTableNamespaceOperation_args args = new finishTableNamespaceOperation_args();
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("finishNamespaceOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        finishNamespaceOperation_args args = new finishNamespaceOperation_args();
         args.setTinfo(tinfo);
         args.setCredentials(credentials);
         args.setOpid(opid);
@@ -1668,7 +1668,7 @@ import org.slf4j.LoggerFactory;
         }
         org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
         org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        (new Client(prot)).recv_finishTableNamespaceOperation();
+        (new Client(prot)).recv_finishNamespaceOperation();
       }
     }
 
@@ -1689,8 +1689,8 @@ import org.slf4j.LoggerFactory;
       processMap.put("waitForFlush", new waitForFlush());
       processMap.put("setTableProperty", new setTableProperty());
       processMap.put("removeTableProperty", new removeTableProperty());
-      processMap.put("setTableNamespaceProperty", new setTableNamespaceProperty());
-      processMap.put("removeTableNamespaceProperty", new removeTableNamespaceProperty());
+      processMap.put("setNamespaceProperty", new setNamespaceProperty());
+      processMap.put("removeNamespaceProperty", new removeNamespaceProperty());
       processMap.put("setMasterGoalState", new setMasterGoalState());
       processMap.put("shutdown", new shutdown());
       processMap.put("shutdownTabletServer", new shutdownTabletServer());
@@ -1703,10 +1703,10 @@ import org.slf4j.LoggerFactory;
       processMap.put("executeTableOperation", new executeTableOperation());
       processMap.put("waitForTableOperation", new waitForTableOperation());
       processMap.put("finishTableOperation", new finishTableOperation());
-      processMap.put("beginTableNamespaceOperation", new beginTableNamespaceOperation());
-      processMap.put("executeTableNamespaceOperation", new executeTableNamespaceOperation());
-      processMap.put("waitForTableNamespaceOperation", new waitForTableNamespaceOperation());
-      processMap.put("finishTableNamespaceOperation", new finishTableNamespaceOperation());
+      processMap.put("beginNamespaceOperation", new beginNamespaceOperation());
+      processMap.put("executeNamespaceOperation", new executeNamespaceOperation());
+      processMap.put("waitForNamespaceOperation", new waitForNamespaceOperation());
+      processMap.put("finishNamespaceOperation", new finishNamespaceOperation());
       return processMap;
     }
 
@@ -1815,23 +1815,23 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public static class setTableNamespaceProperty<I extends Iface> extends org.apache.thrift.ProcessFunction<I, setTableNamespaceProperty_args> {
-      public setTableNamespaceProperty() {
-        super("setTableNamespaceProperty");
+    public static class setNamespaceProperty<I extends Iface> extends org.apache.thrift.ProcessFunction<I, setNamespaceProperty_args> {
+      public setNamespaceProperty() {
+        super("setNamespaceProperty");
       }
 
-      public setTableNamespaceProperty_args getEmptyArgsInstance() {
-        return new setTableNamespaceProperty_args();
+      public setNamespaceProperty_args getEmptyArgsInstance() {
+        return new setNamespaceProperty_args();
       }
 
       protected boolean isOneway() {
         return false;
       }
 
-      public setTableNamespaceProperty_result getResult(I iface, setTableNamespaceProperty_args args) throws org.apache.thrift.TException {
-        setTableNamespaceProperty_result result = new setTableNamespaceProperty_result();
+      public setNamespaceProperty_result getResult(I iface, setNamespaceProperty_args args) throws org.apache.thrift.TException {
+        setNamespaceProperty_result result = new setNamespaceProperty_result();
         try {
-          iface.setTableNamespaceProperty(args.tinfo, args.credentials, args.ns, args.property, args.value);
+          iface.setNamespaceProperty(args.tinfo, args.credentials, args.ns, args.property, args.value);
         } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
           result.sec = sec;
         } catch (org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException tope) {
@@ -1841,23 +1841,23 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public static class removeTableNamespaceProperty<I extends Iface> extends org.apache.thrift.ProcessFunction<I, removeTableNamespaceProperty_args> {
-      public removeTableNamespaceProperty() {
-        super("removeTableNamespaceProperty");
+    public static class removeNamespaceProperty<I extends Iface> extends org.apache.thrift.ProcessFunction<I, removeNamespaceProperty_args> {
+      public removeNamespaceProperty() {
+        super("removeNamespaceProperty");
       }
 
-      public removeTableNamespaceProperty_args getEmptyArgsInstance() {
-        return new removeTableNamespaceProperty_args();
+      public removeNamespaceProperty_args getEmptyArgsInstance() {
+        return new removeNamespaceProperty_args();
       }
 
       protected boolean isOneway() {
         return false;
       }
 
-      public removeTableNamespaceProperty_result getResult(I iface, removeTableNamespaceProperty_args args) throws org.apache.thrift.TException {
-        removeTableNamespaceProperty_result result = new removeTableNamespaceProperty_result();
+      public removeNamespaceProperty_result getResult(I iface, removeNamespaceProperty_args args) throws org.apache.thrift.TException {
+        removeNamespaceProperty_result result = new removeNamespaceProperty_result();
         try {
-          iface.removeTableNamespaceProperty(args.tinfo, args.credentials, args.ns, args.property);
+          iface.removeNamespaceProperty(args.tinfo, args.credentials, args.ns, args.property);
         } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
           result.sec = sec;
         } catch (org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException tope) {
@@ -2150,23 +2150,23 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public static class beginTableNamespaceOperation<I extends Iface> extends org.apache.thrift.ProcessFunction<I, beginTableNamespaceOperation_args> {
-      public beginTableNamespaceOperation() {
-        super("beginTableNamespaceOperation");
+    public static class beginNamespaceOperation<I extends Iface> extends org.apache.thrift.ProcessFunction<I, beginNamespaceOperation_args> {
+      public beginNamespaceOperation() {
+        super("beginNamespaceOperation");
       }
 
-      public beginTableNamespaceOperation_args getEmptyArgsInstance() {
-        return new beginTableNamespaceOperation_args();
+      public beginNamespaceOperation_args getEmptyArgsInstance() {
+        return new beginNamespaceOperation_args();
       }
 
       protected boolean isOneway() {
         return false;
       }
 
-      public beginTableNamespaceOperation_result getResult(I iface, beginTableNamespaceOperation_args args) throws org.apache.thrift.TException {
-        beginTableNamespaceOperation_result result = new beginTableNamespaceOperation_result();
+      public beginNamespaceOperation_result getResult(I iface, beginNamespaceOperation_args args) throws org.apache.thrift.TException {
+        beginNamespaceOperation_result result = new beginNamespaceOperation_result();
         try {
-          result.success = iface.beginTableNamespaceOperation(args.tinfo, args.credentials);
+          result.success = iface.beginNamespaceOperation(args.tinfo, args.credentials);
           result.setSuccessIsSet(true);
         } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
           result.sec = sec;
@@ -2175,23 +2175,23 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public static class executeTableNamespaceOperation<I extends Iface> extends org.apache.thrift.ProcessFunction<I, executeTableNamespaceOperation_args> {
-      public executeTableNamespaceOperation() {
-        super("executeTableNamespaceOperation");
+    public static class executeNamespaceOperation<I extends Iface> extends org.apache.thrift.ProcessFunction<I, executeNamespaceOperation_args> {
+      public executeNamespaceOperation() {
+        super("executeNamespaceOperation");
       }
 
-      public executeTableNamespaceOperation_args getEmptyArgsInstance() {
-        return new executeTableNamespaceOperation_args();
+      public executeNamespaceOperation_args getEmptyArgsInstance() {
+        return new executeNamespaceOperation_args();
       }
 
       protected boolean isOneway() {
         return false;
       }
 
-      public executeTableNamespaceOperation_result getResult(I iface, executeTableNamespaceOperation_args args) throws org.apache.thrift.TException {
-        executeTableNamespaceOperation_result result = new executeTableNamespaceOperation_result();
+      public executeNamespaceOperation_result getResult(I iface, executeNamespaceOperation_args args) throws org.apache.thrift.TException {
+        executeNamespaceOperation_result result = new executeNamespaceOperation_result();
         try {
-          iface.executeTableNamespaceOperation(args.tinfo, args.credentials, args.opid, args.op, args.arguments, args.options, args.autoClean);
+          iface.executeNamespaceOperation(args.tinfo, args.credentials, args.opid, args.op, args.arguments, args.options, args.autoClean);
         } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
           result.sec = sec;
         } catch (org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException tope) {
@@ -2201,23 +2201,23 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public static class waitForTableNamespaceOperation<I extends Iface> extends org.apache.thrift.ProcessFunction<I, waitForTableNamespaceOperation_args> {
-      public waitForTableNamespaceOperation() {
-        super("waitForTableNamespaceOperation");
+    public static class waitForNamespaceOperation<I extends Iface> extends org.apache.thrift.ProcessFunction<I, waitForNamespaceOperation_args> {
+      public waitForNamespaceOperation() {
+        super("waitForNamespaceOperation");
       }
 
-      public waitForTableNamespaceOperation_args getEmptyArgsInstance() {
-        return new waitForTableNamespaceOperation_args();
+      public waitForNamespaceOperation_args getEmptyArgsInstance() {
+        return new waitForNamespaceOperation_args();
       }
 
       protected boolean isOneway() {
         return false;
       }
 
-      public waitForTableNamespaceOperation_result getResult(I iface, waitForTableNamespaceOperation_args args) throws org.apache.thrift.TException {
-        waitForTableNamespaceOperation_result result = new waitForTableNamespaceOperation_result();
+      public waitForNamespaceOperation_result getResult(I iface, waitForNamespaceOperation_args args) throws org.apache.thrift.TException {
+        waitForNamespaceOperation_result result = new waitForNamespaceOperation_result();
         try {
-          result.success = iface.waitForTableNamespaceOperation(args.tinfo, args.credentials, args.opid);
+          result.success = iface.waitForNamespaceOperation(args.tinfo, args.credentials, args.opid);
         } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
           result.sec = sec;
         } catch (org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException tope) {
@@ -2227,23 +2227,23 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public static class finishTableNamespaceOperation<I extends Iface> extends org.apache.thrift.ProcessFunction<I, finishTableNamespaceOperation_args> {
-      public finishTableNamespaceOperation() {
-        super("finishTableNamespaceOperation");
+    public static class finishNamespaceOperation<I extends Iface> extends org.apache.thrift.ProcessFunction<I, finishNamespaceOperation_args> {
+      public finishNamespaceOperation() {
+        super("finishNamespaceOperation");
       }
 
-      public finishTableNamespaceOperation_args getEmptyArgsInstance() {
-        return new finishTableNamespaceOperation_args();
+      public finishNamespaceOperation_args getEmptyArgsInstance() {
+        return new finishNamespaceOperation_args();
       }
 
       protected boolean isOneway() {
         return false;
       }
 
-      public finishTableNamespaceOperation_result getResult(I iface, finishTableNamespaceOperation_args args) throws org.apache.thrift.TException {
-        finishTableNamespaceOperation_result result = new finishTableNamespaceOperation_result();
+      public finishNamespaceOperation_result getResult(I iface, finishNamespaceOperation_args args) throws org.apache.thrift.TException {
+        finishNamespaceOperation_result result = new finishNamespaceOperation_result();
         try {
-          iface.finishTableNamespaceOperation(args.tinfo, args.credentials, args.opid);
+          iface.finishNamespaceOperation(args.tinfo, args.credentials, args.opid);
         } catch (org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
           result.sec = sec;
         }
@@ -7155,8 +7155,8 @@ import org.slf4j.LoggerFactory;
 
   }
 
-  public static class setTableNamespaceProperty_args implements org.apache.thrift.TBase<setTableNamespaceProperty_args, setTableNamespaceProperty_args._Fields>, java.io.Serializable, Cloneable   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("setTableNamespaceProperty_args");
+  public static class setNamespaceProperty_args implements org.apache.thrift.TBase<setNamespaceProperty_args, setNamespaceProperty_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("setNamespaceProperty_args");
 
     private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)5);
     private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)1);
@@ -7166,8 +7166,8 @@ import org.slf4j.LoggerFactory;
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new setTableNamespaceProperty_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new setTableNamespaceProperty_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new setNamespaceProperty_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new setNamespaceProperty_argsTupleSchemeFactory());
     }
 
     public org.apache.accumulo.trace.thrift.TInfo tinfo; // required
@@ -7261,13 +7261,13 @@ import org.slf4j.LoggerFactory;
       tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(setTableNamespaceProperty_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(setNamespaceProperty_args.class, metaDataMap);
     }
 
-    public setTableNamespaceProperty_args() {
+    public setNamespaceProperty_args() {
     }
 
-    public setTableNamespaceProperty_args(
+    public setNamespaceProperty_args(
       org.apache.accumulo.trace.thrift.TInfo tinfo,
       org.apache.accumulo.core.security.thrift.TCredentials credentials,
       String ns,
@@ -7285,7 +7285,7 @@ import org.slf4j.LoggerFactory;
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public setTableNamespaceProperty_args(setTableNamespaceProperty_args other) {
+    public setNamespaceProperty_args(setNamespaceProperty_args other) {
       if (other.isSetTinfo()) {
         this.tinfo = new org.apache.accumulo.trace.thrift.TInfo(other.tinfo);
       }
@@ -7303,8 +7303,8 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public setTableNamespaceProperty_args deepCopy() {
-      return new setTableNamespaceProperty_args(this);
+    public setNamespaceProperty_args deepCopy() {
+      return new setNamespaceProperty_args(this);
     }
 
     @Override
@@ -7320,7 +7320,7 @@ import org.slf4j.LoggerFactory;
       return this.tinfo;
     }
 
-    public setTableNamespaceProperty_args setTinfo(org.apache.accumulo.trace.thrift.TInfo tinfo) {
+    public setNamespaceProperty_args setTinfo(org.apache.accumulo.trace.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -7344,7 +7344,7 @@ import org.slf4j.LoggerFactory;
       return this.credentials;
     }
 
-    public setTableNamespaceProperty_args setCredentials(org.apache.accumulo.core.security.thrift.TCredentials credentials) {
+    public setNamespaceProperty_args setCredentials(org.apache.accumulo.core.security.thrift.TCredentials credentials) {
       this.credentials = credentials;
       return this;
     }
@@ -7368,7 +7368,7 @@ import org.slf4j.LoggerFactory;
       return this.ns;
     }
 
-    public setTableNamespaceProperty_args setNs(String ns) {
+    public setNamespaceProperty_args setNs(String ns) {
       this.ns = ns;
       return this;
     }
@@ -7392,7 +7392,7 @@ import org.slf4j.LoggerFactory;
       return this.property;
     }
 
-    public setTableNamespaceProperty_args setProperty(String property) {
+    public setNamespaceProperty_args setProperty(String property) {
       this.property = property;
       return this;
     }
@@ -7416,7 +7416,7 @@ import org.slf4j.LoggerFactory;
       return this.value;
     }
 
-    public setTableNamespaceProperty_args setValue(String value) {
+    public setNamespaceProperty_args setValue(String value) {
       this.value = value;
       return this;
     }
@@ -7527,12 +7527,12 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof setTableNamespaceProperty_args)
-        return this.equals((setTableNamespaceProperty_args)that);
+      if (that instanceof setNamespaceProperty_args)
+        return this.equals((setNamespaceProperty_args)that);
       return false;
     }
 
-    public boolean equals(setTableNamespaceProperty_args that) {
+    public boolean equals(setNamespaceProperty_args that) {
       if (that == null)
         return false;
 
@@ -7589,13 +7589,13 @@ import org.slf4j.LoggerFactory;
       return 0;
     }
 
-    public int compareTo(setTableNamespaceProperty_args other) {
+    public int compareTo(setNamespaceProperty_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
-      setTableNamespaceProperty_args typedOther = (setTableNamespaceProperty_args)other;
+      setNamespaceProperty_args typedOther = (setNamespaceProperty_args)other;
 
       lastComparison = Boolean.valueOf(isSetTinfo()).compareTo(typedOther.isSetTinfo());
       if (lastComparison != 0) {
@@ -7664,7 +7664,7 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("setTableNamespaceProperty_args(");
+      StringBuilder sb = new StringBuilder("setNamespaceProperty_args(");
       boolean first = true;
 
       sb.append("tinfo:");
@@ -7737,15 +7737,15 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    private static class setTableNamespaceProperty_argsStandardSchemeFactory implements SchemeFactory {
-      public setTableNamespaceProperty_argsStandardScheme getScheme() {
-        return new setTableNamespaceProperty_argsStandardScheme();
+    private static class setNamespaceProperty_argsStandardSchemeFactory implements SchemeFactory {
+      public setNamespaceProperty_argsStandardScheme getScheme() {
+        return new setNamespaceProperty_argsStandardScheme();
       }
     }
 
-    private static class setTableNamespaceProperty_argsStandardScheme extends StandardScheme<setTableNamespaceProperty_args> {
+    private static class setNamespaceProperty_argsStandardScheme extends StandardScheme<setNamespaceProperty_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, setTableNamespaceProperty_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, setNamespaceProperty_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -7808,7 +7808,7 @@ import org.slf4j.LoggerFactory;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, setTableNamespaceProperty_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, setNamespaceProperty_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -7843,16 +7843,16 @@ import org.slf4j.LoggerFactory;
 
     }
 
-    private static class setTableNamespaceProperty_argsTupleSchemeFactory implements SchemeFactory {
-      public setTableNamespaceProperty_argsTupleScheme getScheme() {
-        return new setTableNamespaceProperty_argsTupleScheme();
+    private static class setNamespaceProperty_argsTupleSchemeFactory implements SchemeFactory {
+      public setNamespaceProperty_argsTupleScheme getScheme() {
+        return new setNamespaceProperty_argsTupleScheme();
       }
     }
 
-    private static class setTableNamespaceProperty_argsTupleScheme extends TupleScheme<setTableNamespaceProperty_args> {
+    private static class setNamespaceProperty_argsTupleScheme extends TupleScheme<setNamespaceProperty_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, setTableNamespaceProperty_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, setNamespaceProperty_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetTinfo()) {
@@ -7889,7 +7889,7 @@ import org.slf4j.LoggerFactory;
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, setTableNamespaceProperty_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, setNamespaceProperty_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
@@ -7919,16 +7919,16 @@ import org.slf4j.LoggerFactory;
 
   }
 
-  public static class setTableNamespaceProperty_result implements org.apache.thrift.TBase<setTableNamespaceProperty_result, setTableNamespaceProperty_result._Fields>, java.io.Serializable, Cloneable   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("setTableNamespaceProperty_result");
+  public static class setNamespaceProperty_result implements org.apache.thrift.TBase<setNamespaceProperty_result, setNamespaceProperty_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("setNamespaceProperty_result");
 
     private static final org.apache.thrift.protocol.TField SEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sec", org.apache.thrift.protocol.TType.STRUCT, (short)1);
     private static final org.apache.thrift.protocol.TField TOPE_FIELD_DESC = new org.apache.thrift.protocol.TField("tope", org.apache.thrift.protocol.TType.STRUCT, (short)2);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new setTableNamespaceProperty_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new setTableNamespaceProperty_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new setNamespaceProperty_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new setNamespaceProperty_resultTupleSchemeFactory());
     }
 
     public org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec; // required
@@ -8004,13 +8004,13 @@ import org.slf4j.LoggerFactory;
       tmpMap.put(_Fields.TOPE, new org.apache.thrift.meta_data.FieldMetaData("tope", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(setTableNamespaceProperty_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(setNamespaceProperty_result.class, metaDataMap);
     }
 
-    public setTableNamespaceProperty_result() {
+    public setNamespaceProperty_result() {
     }
 
-    public setTableNamespaceProperty_result(
+    public setNamespaceProperty_result(
       org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec,
       org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException tope)
     {
@@ -8022,7 +8022,7 @@ import org.slf4j.LoggerFactory;
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public setTableNamespaceProperty_result(setTableNamespaceProperty_result other) {
+    public setNamespaceProperty_result(setNamespaceProperty_result other) {
       if (other.isSetSec()) {
         this.sec = new org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException(other.sec);
       }
@@ -8031,8 +8031,8 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public setTableNamespaceProperty_result deepCopy() {
-      return new setTableNamespaceProperty_result(this);
+    public setNamespaceProperty_result deepCopy() {
+      return new setNamespaceProperty_result(this);
     }
 
     @Override
@@ -8045,7 +8045,7 @@ import org.slf4j.LoggerFactory;
       return this.sec;
     }
 
-    public setTableNamespaceProperty_result setSec(org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
+    public setNamespaceProperty_result setSec(org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException sec) {
       this.sec = sec;
       return this;
     }
@@ -8069,7 +8069,7 @@ import org.slf4j.LoggerFactory;
       return this.tope;
     }
 
-    public setTableNamespaceProperty_result setTope(org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException tope) {
+    public setNamespaceProperty_result setTope(org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException tope) {
       this.tope = tope;
       return this;
     }
@@ -8141,12 +8141,12 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof setTableNamespaceProperty_result)
-        return this.equals((setTableNamespaceProperty_result)that);
+      if (that instanceof setNamespaceProperty_result)
+        return this.equals((setNamespaceProperty_result)that);
       return false;
     }
 
-    public boolean equals(setTableNamespaceProperty_result that) {
+    public boolean equals(setNamespaceProperty_result that) {
       if (that == null)
         return false;
 
@@ -8176,13 +8176,13 @@ import org.slf4j.LoggerFactory;
       return 0;
     }
 
-    public int compareTo(setTableNamespaceProperty_result other) {
+    public int compareTo(setNamespaceProperty_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
-      setTableNamespaceProperty_result typedOther = (setTableNamespaceProperty_result)other;
+      setNamespaceProperty_result typedOther = (setNamespaceProperty_result)other;
 
       lastComparison = Boolean.valueOf(isSetSec()).compareTo(typedOther.isSetSec());
       if (lastComparison != 0) {
@@ -8221,7 +8221,7 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("setTableNamespaceProperty_result(");
+      StringBuilder sb = new StringBuilder("setNamespaceProperty_result(");
       boolean first = true;
 
       sb.append("sec:");
@@ -8264,15 +8264,15 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    private static class setTableNamespaceProperty_resultStandardSchemeFactory implements SchemeFactory {
-      public setTableNamespaceProperty_resultStandardScheme getScheme() {
-        return new setTableNamespaceProperty_resultStandardScheme();
+    private static class setNamespaceProperty_resultStandardSchemeFactory implements SchemeFactory {
+      public setNamespaceProperty_resultStandardScheme getScheme() {
+        return new setNamespaceProperty_resultStandardScheme();
       }
     }
 
-    private static class setTableNamespaceProperty_resultStandardScheme extends StandardScheme<setTableNamespaceProperty_result> {
+    private static class setNamespaceProperty_resultStandardScheme extends StandardScheme<setNamespaceProperty_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, setTableNamespaceProperty_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, setNamespaceProperty_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -8311,7 +8311,7 @@ import org.slf4j.LoggerFactory;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, setTableNamespaceProperty_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, setNamespaceProperty_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -8331,16 +8331,16 @@ import org.slf4j.LoggerFactory;
 
     }
 
-    private static class setTableNamespaceProperty_resultTupleSchemeFactory implements SchemeFactory {
-      public setTableNamespaceProperty_resultTupleScheme getScheme() {
-        return new setTableNamespaceProperty_resultTupleScheme();
+    private static class setNamespaceProperty_resultTupleSchemeFactory implements SchemeFactory {
+      public setNamespaceProperty_resultTupleScheme getScheme() {
+        return new setNamespaceProperty_resultTupleScheme();
       }
     }
 
-    private static class setTableNamespaceProperty_resultTupleScheme extends TupleScheme<setTableNamespaceProperty_result> {
+    private static class setNamespaceProperty_resultTupleScheme extends TupleScheme<setNamespaceProperty_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, setTableNamespaceProperty_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, setNamespaceProperty_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetSec()) {
@@ -8359,7 +8359,7 @@ import org.slf4j.LoggerFactory;
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, setTableNamespaceProperty_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, setNamespaceProperty_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
@@ -8377,8 +8377,8 @@ import org.slf4j.LoggerFactory;
 
   }
 
-  public static class removeTableNamespaceProperty_args implements org.apache.thrift.TBase<removeTableNamespaceProperty_args, removeTableNamespaceProperty_args._Fields>, java.io.Serializable, Cloneable   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("removeTableNamespaceProperty_args");
+  public static class removeNamespaceProperty_args implements org.apache.thrift.TBase<removeNamespaceProperty_args, removeNamespaceProperty_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("removeNamespaceProperty_args");
 
     private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)4);
     private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)1);
@@ -8387,8 +8387,8 @@ import org.slf4j.LoggerFactory;
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new removeTableNamespaceProperty_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new removeTableNamespaceProperty_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new removeNamespaceProperty_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new removeNamespaceProperty_argsTupleSchemeFactory());
     }
 
     public org.apache.accumulo.trace.thrift.TInfo tinfo; // required
@@ -8476,13 +8476,13 @@ import org.slf4j.LoggerFactory;
       tmpMap.put(_Fields.PROPERTY, new org.apache.thrift.meta_data.FieldMetaData("property", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(removeTableNamespaceProperty_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(removeNamespaceProperty_args.class, metaDataMap);
     }
 
-    public removeTableNamespaceProperty_args() {
+    public removeNamespaceProperty_args() {
     }
 
-    public removeTableNamespaceProperty_args(
+    public removeNamespaceProperty_args(
       org.apache.accumulo.trace.thrift.TInfo tinfo,
       org.apache.accumulo.core.security.thrift.TCredentials credentials,
       String ns,
@@ -8498,7 +8498,7 @@ import org.slf4j.LoggerFactory;
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public removeTableNamespaceProperty_args(removeTableNamespaceProperty_args other) {
+    public removeNamespaceProperty_args(removeNamespaceProperty_args other) {
       if (other.isSetTinfo()) {
         this.tinfo = new org.apache.accumulo.trace.thrift.TInfo(other.tinfo);
       }
@@ -8513,8 +8513,8 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public removeTableNamespaceProperty_args deepCopy() {
-      return new removeTableNamespaceProperty_args(this);
+    public removeNamespaceProperty_args deepCopy() {
+      return new removeNamespaceProperty_args(this);
     }
 
     @Override
@@ -8529,7 +8529,7 @@ import org.slf4j.LoggerFactory;
       return this.tinfo;
     }
 
-    public removeTableNamespaceProperty_args setTinfo(org.apache.accumulo.trace.thrift.TInfo tinfo) {
+    public removeNamespaceProperty_args setTinfo(org.apache.accumulo.trace.thrift.TInfo tinfo) {
       this.tinfo = tinfo;
       return this;
     }
@@ -8553,7 +8553,7 @@ import org.slf4j.LoggerFactory;
       return this.credentials;
     }
 
-    public removeTableNamespaceProperty_args setCredentials(org.apache.accumulo.core.security.thrift.TCredentials credentials) {
+    public removeNamespaceProperty_args setCredentials(org.apache.accumulo.core.security.thrift.TCredentials credentials) {
       this.credentials = credentials;
       return this;
     }
@@ -8577,7 +8577,7 @@ import org.slf4j.LoggerFactory;
       return this.ns;
     }
 
-    public removeTableNamespaceProperty_args setNs(String ns) {
+    public removeNamespaceProperty_args setNs(String ns) {
       this.ns = ns;
       return this;
     }
@@ -8601,7 +8601,7 @@ import org.slf4j.LoggerFactory;
       return this.property;
     }
 
-    public removeTableNamespaceProperty_args setProperty(String property) {
+    public removeNamespaceProperty_args setProperty(String property) {
       this.property = property;
       return this;
     }
@@ -8699,12 +8699,12 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof removeTableNamespaceProperty_args)
-        return this.equals((removeTableNamespaceProperty_args)that);
+      if (that instanceof removeNamespaceProperty_args)
+        return this.equals((removeNamespaceProperty_args)that);
       return false;
     }
 
-    public boolean equals(removeTableNamespaceProperty_args that) {
+    public boolean equals(removeNamespaceProperty_args that) {
       if (that == null)
         return false;
 
@@ -8752,13 +8752,13 @@ import org.slf4j.LoggerFactory;
       return 0;
     }
 
-    public int compareTo(removeTableNamespaceProperty_args other) {
+    public int compareTo(removeNamespaceProperty_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
-      removeTableNamespaceProperty_args typedOther = (removeTableNamespaceProperty_args)other;
+      removeNamespaceProperty_args typedOther = (removeNamespaceProperty_args)other;
 
       lastComparison = Boolean.valueOf(isSetTinfo()).compareTo(typedOther.isSetTinfo());
       if (lastComparison != 0) {
@@ -8817,7 +8817,7 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("removeTableNamespaceProperty_args(");
+      StringBuilder sb = new StringBuilder("removeNamespaceProperty_args(");
       boolean first = true;
 
       sb.append("tinfo:");
@@ -8882,15 +8882,15 @@ import org.slf4j.Log

<TRUNCATED>

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

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java
new file mode 100644
index 0000000..30adef1
--- /dev/null
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java
@@ -0,0 +1,92 @@
+/*
+ * 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.Constants;
+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.TableOperationExceptionType;
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.Repo;
+import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
+import org.apache.accumulo.fate.zookeeper.IZooReaderWriter.Mutator;
+import org.apache.accumulo.master.Master;
+import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
+import org.apache.log4j.Logger;
+
+public class RenameNamespace extends MasterRepo {
+
+  private static final long serialVersionUID = 1L;
+  private String namespaceId;
+  private String oldName;
+  private String newName;
+
+  @Override
+  public long isReady(long id, Master environment) throws Exception {
+    return Utils.reserveNamespace(namespaceId, id, true, true, TableOperation.RENAME);
+  }
+
+  public RenameNamespace(String namespaceId, String oldName, String newName) {
+    this.namespaceId = namespaceId;
+    this.oldName = oldName;
+    this.newName = newName;
+  }
+
+  @Override
+  public Repo<Master> call(long id, Master master) throws Exception {
+
+    Instance instance = master.getInstance();
+
+    IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
+
+    Utils.tableNameLock.lock();
+    try {
+      Utils.checkNamespaceDoesNotExist(instance, newName, namespaceId, TableOperation.RENAME);
+
+      final String tap = ZooUtil.getRoot(instance) + Constants.ZNAMESPACES + "/" + namespaceId + Constants.ZNAMESPACE_NAME;
+
+      zoo.mutate(tap, null, null, new Mutator() {
+        @Override
+        public byte[] mutate(byte[] current) throws Exception {
+          final String currentName = new String(current);
+          if (currentName.equals(newName))
+            return null; // assume in this case the operation is running again, so we are done
+          if (!currentName.equals(oldName)) {
+            throw new ThriftTableOperationException(null, oldName, TableOperation.RENAME, TableOperationExceptionType.NOTFOUND, "Name changed while processing");
+          }
+          return newName.getBytes();
+        }
+      });
+      Tables.clearCache(instance);
+    } finally {
+      Utils.tableNameLock.unlock();
+      Utils.unreserveNamespace(namespaceId, id, true);
+    }
+
+    Logger.getLogger(RenameNamespace.class).debug("Renamed namespace " + namespaceId + " " + oldName + " " + newName);
+
+    return null;
+  }
+
+  @Override
+  public void undo(long tid, Master env) throws Exception {
+    Utils.unreserveNamespace(namespaceId, tid, true);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
index 9044247..a0ddb8d 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
@@ -18,8 +18,8 @@ package org.apache.accumulo.master.tableOps;
 
 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.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.TableOperationExceptionType;
@@ -45,18 +45,18 @@ public class RenameTable extends MasterRepo {
 
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    return Utils.reserveTableNamespace(oldNamespaceId, tid, false, true, TableOperation.RENAME)
-        + Utils.reserveTableNamespace(newNamespaceId, tid, false, true, TableOperation.RENAME)
+    return Utils.reserveNamespace(oldNamespaceId, tid, false, true, TableOperation.RENAME)
+        + Utils.reserveNamespace(newNamespaceId, tid, false, true, TableOperation.RENAME)
         + Utils.reserveTable(tableId, tid, true, true, TableOperation.RENAME);
   }
 
-  public RenameTable(String tableId, String oldTableName, String newTableName) throws TableNamespaceNotFoundException {
+  public RenameTable(String tableId, String oldTableName, String newTableName) throws NamespaceNotFoundException {
     this.tableId = tableId;
     this.oldTableName = oldTableName;
     this.newTableName = newTableName;
     Instance inst = HdfsZooInstance.getInstance();
     this.oldNamespaceId = Tables.getNamespace(inst, tableId);
-    this.newNamespaceId = TableNamespaces.getNamespaceId(inst, Tables.extractNamespace(newTableName));
+    this.newNamespaceId = Namespaces.getNamespaceId(inst, Tables.extractNamespace(newTableName));
   }
 
   @Override
@@ -97,8 +97,8 @@ public class RenameTable extends MasterRepo {
     } finally {
       Utils.tableNameLock.unlock();
       Utils.unreserveTable(tableId, tid, true);
-      Utils.unreserveTableNamespace(this.oldNamespaceId, tid, false);
-      Utils.unreserveTableNamespace(this.newNamespaceId, tid, false);
+      Utils.unreserveNamespace(this.oldNamespaceId, tid, false);
+      Utils.unreserveNamespace(this.newNamespaceId, tid, false);
     }
 
     Logger.getLogger(RenameTable.class).debug("Renamed table " + tableId + " " + oldTableName + " " + newTableName);
@@ -108,8 +108,8 @@ public class RenameTable extends MasterRepo {
 
   @Override
   public void undo(long tid, Master env) throws Exception {
-    Utils.unreserveTableNamespace(newNamespaceId, tid, false);
-    Utils.unreserveTableNamespace(oldNamespaceId, tid, false);
+    Utils.unreserveNamespace(newNamespaceId, tid, false);
+    Utils.unreserveNamespace(oldNamespaceId, 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/RenameTableNamespace.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTableNamespace.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTableNamespace.java
deleted file mode 100644
index 9662fbb..0000000
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTableNamespace.java
+++ /dev/null
@@ -1,91 +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.Constants;
-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.TableOperationExceptionType;
-import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
-import org.apache.accumulo.core.zookeeper.ZooUtil;
-import org.apache.accumulo.fate.Repo;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter.Mutator;
-import org.apache.accumulo.master.Master;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
-import org.apache.log4j.Logger;
-
-public class RenameTableNamespace extends MasterRepo {
-
-  private static final long serialVersionUID = 1L;
-  private String namespaceId;
-  private String oldName;
-  private String newName;
-
-  @Override
-  public long isReady(long id, Master environment) throws Exception {
-    return Utils.reserveTableNamespace(namespaceId, id, true, true, TableOperation.RENAME);
-  }
-
-  public RenameTableNamespace(String namespaceId, String oldName, String newName) {
-    this.namespaceId = namespaceId;
-    this.oldName = oldName;
-    this.newName = newName;
-  }
-
-  @Override
-  public Repo<Master> call(long id, Master master) throws Exception {
-
-    Instance instance = master.getInstance();
-
-    IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
-
-    Utils.tableNameLock.lock();
-    try {
-      Utils.checkTableNamespaceDoesNotExist(instance, newName, namespaceId, TableOperation.RENAME);
-
-      final String tap = ZooUtil.getRoot(instance) + Constants.ZNAMESPACES + "/" + namespaceId + Constants.ZNAMESPACE_NAME;
-
-      zoo.mutate(tap, null, null, new Mutator() {
-        public byte[] mutate(byte[] current) throws Exception {
-          final String currentName = new String(current);
-          if (currentName.equals(newName))
-            return null; // assume in this case the operation is running again, so we are done
-          if (!currentName.equals(oldName)) {
-            throw new ThriftTableOperationException(null, oldName, TableOperation.RENAME, TableOperationExceptionType.NOTFOUND, "Name changed while processing");
-          }
-          return newName.getBytes();
-        }
-      });
-      Tables.clearCache(instance);
-    } finally {
-      Utils.tableNameLock.unlock();
-      Utils.unreserveTableNamespace(namespaceId, id, true);
-    }
-
-    Logger.getLogger(RenameTableNamespace.class).debug("Renamed table namespace " + namespaceId + " " + oldName + " " + newName);
-
-    return null;
-  }
-
-  @Override
-  public void undo(long tid, Master env) throws Exception {
-    Utils.unreserveTableNamespace(namespaceId, tid, true);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableRangeOp.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableRangeOp.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableRangeOp.java
index 77a6606..a0f0af4 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableRangeOp.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableRangeOp.java
@@ -72,7 +72,7 @@ class TableRangeOpWait extends MasterRepo {
     MergeInfo mergeInfo = master.getMergeInfo(tableIdText);
     log.info("removing merge information " + mergeInfo);
     master.clearMergeState(tableIdText);
-    Utils.unreserveTableNamespace(namespaceId, tid, false);
+    Utils.unreserveNamespace(namespaceId, tid, false);
     Utils.unreserveTable(tableId, tid, true);
     return null;
   }
@@ -91,7 +91,7 @@ public class TableRangeOp extends MasterRepo {
 
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    return Utils.reserveTableNamespace(namespaceId, tid, false, true, TableOperation.MERGE)
+    return Utils.reserveNamespace(namespaceId, tid, false, true, TableOperation.MERGE)
         + Utils.reserveTable(tableId, tid, true, true, TableOperation.MERGE);
   }
 
@@ -141,7 +141,7 @@ public class TableRangeOp extends MasterRepo {
     if (mergeInfo.getState() != MergeState.NONE)
       log.info("removing merge information " + mergeInfo);
     env.clearMergeState(tableIdText);
-    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/Utils.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java
index 58524f3..9ff1b6a 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java
@@ -22,7 +22,7 @@ import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
-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;
@@ -40,17 +40,17 @@ import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
 
 public class Utils {
-  
+
   static void checkTableDoesNotExist(Instance instance, String tableName, String tableId, TableOperation operation) throws ThriftTableOperationException {
-    
+
     String id = Tables.getNameToIdMap(instance).get(tableName);
-    
+
     if (id != null && !id.equals(tableId))
       throw new ThriftTableOperationException(null, tableName, operation, TableOperationExceptionType.EXISTS, null);
   }
-  
+
   static String getNextTableId(String tableName, Instance instance) throws ThriftTableOperationException {
-    
+
     String tableId = null;
     try {
       IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
@@ -69,11 +69,11 @@ public class Utils {
       throw new ThriftTableOperationException(tableId, tableName, TableOperation.CREATE, TableOperationExceptionType.OTHER, e1.getMessage());
     }
   }
-  
+
   static final Lock tableNameLock = new ReentrantLock();
   static final Lock idLock = new ReentrantLock();
   private static final Logger log = Logger.getLogger(Utils.class);
-  
+
   public static long reserveTable(String tableId, long tid, boolean writeLock, boolean tableMustExist, TableOperation op) throws Exception {
     if (getLock(tableId, tid, writeLock).tryLock()) {
       if (tableMustExist) {
@@ -87,50 +87,50 @@ public class Utils {
     } else
       return 100;
   }
-  
+
   public static void unreserveTable(String tableId, long tid, boolean writeLock) throws Exception {
     getLock(tableId, tid, writeLock).unlock();
     log.info("table " + tableId + " (" + Long.toHexString(tid) + ") unlocked for " + (writeLock ? "write" : "read"));
   }
-  
-  public static void unreserveTableNamespace(String namespaceId, long id, boolean writeLock) throws Exception {
+
+  public static void unreserveNamespace(String namespaceId, long id, boolean writeLock) throws Exception {
     getLock(namespaceId, id, writeLock).unlock();
-    log.info("table namespace " + namespaceId + " (" + Long.toHexString(id) + ") unlocked for " + (writeLock ? "write" : "read"));
+    log.info("namespace " + namespaceId + " (" + Long.toHexString(id) + ") unlocked for " + (writeLock ? "write" : "read"));
   }
-  
-  public static long reserveTableNamespace(String namespaceId, long id, boolean writeLock, boolean mustExist, TableOperation op) throws Exception {
+
+  public static long reserveNamespace(String namespaceId, long id, boolean writeLock, boolean mustExist, TableOperation op) throws Exception {
     if (getLock(namespaceId, id, writeLock).tryLock()) {
       if (mustExist) {
         Instance instance = HdfsZooInstance.getInstance();
         IZooReaderWriter zk = ZooReaderWriter.getRetryingInstance();
         if (!zk.exists(ZooUtil.getRoot(instance) + Constants.ZNAMESPACES + "/" + namespaceId))
-          throw new ThriftTableOperationException(namespaceId, "", op, TableOperationExceptionType.NOTFOUND, "Table namespace does not exist");
+          throw new ThriftTableOperationException(namespaceId, "", op, TableOperationExceptionType.NOTFOUND, "Namespace does not exist");
       }
-      log.info("table namespace " + namespaceId + " (" + Long.toHexString(id) + ") locked for " + (writeLock ? "write" : "read") + " operation: " + op);
+      log.info("namespace " + namespaceId + " (" + Long.toHexString(id) + ") locked for " + (writeLock ? "write" : "read") + " operation: " + op);
       return 0;
     } else
       return 100;
   }
-  
+
   public static long reserveHdfsDirectory(String directory, long tid) throws KeeperException, InterruptedException {
     Instance instance = HdfsZooInstance.getInstance();
-    
+
     String resvPath = ZooUtil.getRoot(instance) + Constants.ZHDFS_RESERVATIONS + "/" + new String(Base64.encodeBase64(directory.getBytes()));
-    
+
     IZooReaderWriter zk = ZooReaderWriter.getRetryingInstance();
-    
+
     if (ZooReservation.attempt(zk, resvPath, String.format("%016x", tid), "")) {
       return 0;
     } else
       return 50;
   }
-  
+
   public static void unreserveHdfsDirectory(String directory, long tid) throws KeeperException, InterruptedException {
     Instance instance = HdfsZooInstance.getInstance();
     String resvPath = ZooUtil.getRoot(instance) + Constants.ZHDFS_RESERVATIONS + "/" + new String(Base64.encodeBase64(directory.getBytes()));
     ZooReservation.release(ZooReaderWriter.getRetryingInstance(), resvPath, String.format("%016x", tid));
   }
-  
+
   private static Lock getLock(String tableId, long tid, boolean writeLock) throws Exception {
     byte[] lockData = String.format("%016x", tid).getBytes();
     ZooQueueLock qlock = new ZooQueueLock(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZTABLE_LOCKS + "/" + tableId, false);
@@ -144,16 +144,16 @@ public class Utils {
     }
     return lock;
   }
-  
+
   public static Lock getReadLock(String tableId, long tid) throws Exception {
     return Utils.getLock(tableId, tid, false);
   }
-  
-  static void checkTableNamespaceDoesNotExist(Instance instance, String namespace, String namespaceId, TableOperation operation)
+
+  static void checkNamespaceDoesNotExist(Instance instance, String namespace, String namespaceId, TableOperation operation)
       throws ThriftTableOperationException {
-    
-    String n = TableNamespaces.getNameToIdMap(instance).get(namespace);
-    
+
+    String n = Namespaces.getNameToIdMap(instance).get(namespace);
+
     if (n != null && !n.equals(namespaceId))
       throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.EXISTS, null);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/ChangePermissions.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/ChangePermissions.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/ChangePermissions.java
index a63391c..d3c7c70 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/ChangePermissions.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/ChangePermissions.java
@@ -26,7 +26,7 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
 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.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
@@ -48,8 +48,8 @@ public class ChangePermissions extends Test {
     String tableName = tableNames.get(rand.nextInt(tableNames.size()));
     
     @SuppressWarnings("unchecked")
-    List<String> tableNamespaces = (List<String>) state.get("namespaces");
-    String tableNamespace = tableNamespaces.get(rand.nextInt(tableNamespaces.size()));
+    List<String> namespaces = (List<String>) state.get("namespaces");
+    String namespace = namespaces.get(rand.nextInt(namespaces.size()));
     
     try {
       int dice = rand.nextInt(2);
@@ -58,7 +58,7 @@ public class ChangePermissions extends Test {
       else if (dice == 1)
         changeTablePermission(conn, rand, userName, tableName);
       else if (dice == 2)
-        changeTableNamespacePermission(conn, rand, userName, tableNamespace);
+        changeNamespacePermission(conn, rand, userName, namespace);
     } catch (AccumuloSecurityException ex) {
       log.debug("Unable to change user permissions: " + ex.getCause());
     }
@@ -116,28 +116,28 @@ public class ChangePermissions extends Test {
     }
   }
   
-  private void changeTableNamespacePermission(Connector conn, Random rand, String userName, String tableNamespace) throws AccumuloException, AccumuloSecurityException {
+  private void changeNamespacePermission(Connector conn, Random rand, String userName, String namespace) throws AccumuloException, AccumuloSecurityException {
     
-    EnumSet<TableNamespacePermission> perms = EnumSet.noneOf(TableNamespacePermission.class);
-    for (TableNamespacePermission p : TableNamespacePermission.values()) {
-      if (conn.securityOperations().hasTableNamespacePermission(userName, tableNamespace, p))
+    EnumSet<NamespacePermission> perms = EnumSet.noneOf(NamespacePermission.class);
+    for (NamespacePermission p : NamespacePermission.values()) {
+      if (conn.securityOperations().hasNamespacePermission(userName, namespace, p))
         perms.add(p);
     }
     
-    EnumSet<TableNamespacePermission> more = EnumSet.allOf(TableNamespacePermission.class);
+    EnumSet<NamespacePermission> more = EnumSet.allOf(NamespacePermission.class);
     more.removeAll(perms);
     
     if (rand.nextBoolean() && more.size() > 0) {
-      List<TableNamespacePermission> moreList = new ArrayList<TableNamespacePermission>(more);
-      TableNamespacePermission choice = moreList.get(rand.nextInt(moreList.size()));
+      List<NamespacePermission> moreList = new ArrayList<NamespacePermission>(more);
+      NamespacePermission choice = moreList.get(rand.nextInt(moreList.size()));
       log.debug("adding permission " + choice);
-      conn.securityOperations().grantTableNamespacePermission(userName, tableNamespace, choice);
+      conn.securityOperations().grantNamespacePermission(userName, namespace, choice);
     } else {
       if (perms.size() > 0) {
-        List<TableNamespacePermission> permList = new ArrayList<TableNamespacePermission>(perms);
-        TableNamespacePermission choice = permList.get(rand.nextInt(permList.size()));
+        List<NamespacePermission> permList = new ArrayList<NamespacePermission>(perms);
+        NamespacePermission choice = permList.get(rand.nextInt(permList.size()));
         log.debug("removing permission " + choice);
-        conn.securityOperations().revokeTableNamespacePermission(userName, tableNamespace, choice);
+        conn.securityOperations().revokeNamespacePermission(userName, namespace, choice);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CheckPermission.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CheckPermission.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CheckPermission.java
index 544ce96..50948b3 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CheckPermission.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CheckPermission.java
@@ -22,32 +22,32 @@ import java.util.Random;
 
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.security.TableNamespacePermission;
 import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
 public class CheckPermission extends Test {
-  
+
   @Override
   public void visit(State state, Properties props) throws Exception {
     Connector conn = state.getConnector();
-    
+
     Random rand = (Random) state.get("rand");
-    
+
     @SuppressWarnings("unchecked")
     List<String> userNames = (List<String>) state.get("users");
     String userName = userNames.get(rand.nextInt(userNames.size()));
-    
+
     @SuppressWarnings("unchecked")
     List<String> tableNames = (List<String>) state.get("tables");
     String tableName = tableNames.get(rand.nextInt(tableNames.size()));
-    
+
     @SuppressWarnings("unchecked")
-    List<String> tableNamespaces = (List<String>) state.get("namespaces");
-    String tableNamespace = tableNamespaces.get(rand.nextInt(tableNamespaces.size()));
-    
+    List<String> namespaces = (List<String>) state.get("namespaces");
+    String namespace = namespaces.get(rand.nextInt(namespaces.size()));
+
     try {
       int dice = rand.nextInt(2);
       if (dice == 0) {
@@ -57,13 +57,13 @@ public class CheckPermission extends Test {
         log.debug("Checking table permission " + userName + " " + tableName);
         conn.securityOperations().hasTablePermission(userName, tableName, TablePermission.values()[rand.nextInt(TablePermission.values().length)]);
       } else if (dice == 2) {
-        log.debug("Checking table namespace permission " + userName + " " + tableNamespace);
-        conn.securityOperations().hasTableNamespacePermission(userName, tableNamespace, TableNamespacePermission.values()[rand.nextInt(TableNamespacePermission.values().length)]);
+        log.debug("Checking namespace permission " + userName + " " + namespace);
+        conn.securityOperations().hasNamespacePermission(userName, namespace, NamespacePermission.values()[rand.nextInt(NamespacePermission.values().length)]);
       }
-      
+
     } catch (AccumuloSecurityException ex) {
       log.debug("Unable to check permissions: " + ex.getCause());
     }
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
index 5a2008f..74526fa 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Config.java
@@ -140,10 +140,10 @@ public class Config extends Test {
       String namespace = parts[0];
       int choice = Integer.parseInt(parts[1]);
       Property property = tableSettings[choice].property;
-      if (state.getConnector().tableNamespaceOperations().exists(namespace)) {
+      if (state.getConnector().namespaceOperations().exists(namespace)) {
         log.debug("Setting " + property.getKey() + " on " + namespace + " back to " + property.getDefaultValue());
         try {
-          state.getConnector().tableNamespaceOperations().setProperty(namespace, property.getKey(), property.getDefaultValue());
+          state.getConnector().namespaceOperations().setProperty(namespace, property.getKey(), property.getDefaultValue());
         } catch (AccumuloException ex) {
           if (ex.getCause() instanceof ThriftTableOperationException) {
             ThriftTableOperationException ttoe = (ThriftTableOperationException) ex.getCause();
@@ -162,7 +162,7 @@ public class Config extends Test {
     if (dice == 0) {
       changeTableSetting(random, state, props);
     } else if (dice == 1) {
-      changeTableNamespaceSetting(random, state, props);
+      changeNamespaceSetting(random, state, props);
     } else {
       changeSetting(random, state, props);
     }
@@ -195,13 +195,13 @@ public class Config extends Test {
     }
   }
 
-  private void changeTableNamespaceSetting(RandomData random, State state, Properties props) throws Exception {
+  private void changeNamespaceSetting(RandomData random, State state, Properties props) throws Exception {
     // pick a random property
     int choice = random.nextInt(0, tableSettings.length - 1);
     Setting setting = tableSettings[choice];
 
     // pick a random table
-    SortedSet<String> namespaces = state.getConnector().tableNamespaceOperations().list();
+    SortedSet<String> namespaces = state.getConnector().namespaceOperations().list();
     if (namespaces.isEmpty())
       return;
     String namespace = random.nextSample(namespaces, 1)[0].toString();
@@ -209,9 +209,9 @@ public class Config extends Test {
     // generate a random value
     long newValue = random.nextLong(setting.min, setting.max);
     state.getMap().put(LAST_NAMESPACE_SETTING, namespace + "," + choice);
-    log.debug("Setting " + setting.property.getKey() + " on table namespace " + namespace + " to " + newValue);
+    log.debug("Setting " + setting.property.getKey() + " on namespace " + namespace + " to " + newValue);
     try {
-      state.getConnector().tableNamespaceOperations().setProperty(namespace, setting.property.getKey(), "" + newValue);
+      state.getConnector().namespaceOperations().setProperty(namespace, setting.property.getKey(), "" + newValue);
     } catch (AccumuloException ex) {
       if (ex.getCause() instanceof ThriftTableOperationException) {
         ThriftTableOperationException ttoe = (ThriftTableOperationException) ex.getCause();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateNamespace.java
new file mode 100644
index 0000000..43e6c06
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateNamespace.java
@@ -0,0 +1,48 @@
+/*
+ * 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.test.randomwalk.concurrent;
+
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.NamespaceExistsException;
+import org.apache.accumulo.test.randomwalk.State;
+import org.apache.accumulo.test.randomwalk.Test;
+
+public class CreateNamespace extends Test {
+
+  @Override
+  public void visit(State state, Properties props) throws Exception {
+    Connector conn = state.getConnector();
+
+    Random rand = (Random) state.get("rand");
+
+    @SuppressWarnings("unchecked")
+    List<String> namespaces = (List<String>) state.get("namespaces");
+
+    String namespace = namespaces.get(rand.nextInt(namespaces.size()));
+
+    try {
+      conn.namespaceOperations().create(namespace);
+      log.debug("Created namespace " + namespace);
+    } catch (NamespaceExistsException e) {
+      log.debug("Create namespace " + namespace + " failed, it exists");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTableNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTableNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTableNamespace.java
deleted file mode 100644
index a65043d..0000000
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTableNamespace.java
+++ /dev/null
@@ -1,48 +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.test.randomwalk.concurrent;
-
-import java.util.List;
-import java.util.Properties;
-import java.util.Random;
-
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.TableNamespaceExistsException;
-import org.apache.accumulo.test.randomwalk.State;
-import org.apache.accumulo.test.randomwalk.Test;
-
-public class CreateTableNamespace extends Test {
-
-  @Override
-  public void visit(State state, Properties props) throws Exception {
-    Connector conn = state.getConnector();
-
-    Random rand = (Random) state.get("rand");
-
-    @SuppressWarnings("unchecked")
-    List<String> namespaces = (List<String>) state.get("namespaces");
-
-    String namespace = namespaces.get(rand.nextInt(namespaces.size()));
-
-    try {
-      conn.tableNamespaceOperations().create(namespace);
-      log.debug("Created namespace " + namespace);
-    } catch (TableNamespaceExistsException e) {
-      log.debug("Create namespace " + namespace + " failed, it exists");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteNamespace.java
new file mode 100644
index 0000000..f7c813a
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteNamespace.java
@@ -0,0 +1,48 @@
+/*
+ * 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.test.randomwalk.concurrent;
+
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.test.randomwalk.State;
+import org.apache.accumulo.test.randomwalk.Test;
+
+public class DeleteNamespace extends Test {
+
+  @Override
+  public void visit(State state, Properties props) throws Exception {
+    Connector conn = state.getConnector();
+
+    Random rand = (Random) state.get("rand");
+
+    @SuppressWarnings("unchecked")
+    List<String> namespaces = (List<String>) state.get("namespaces");
+
+    String namespace = namespaces.get(rand.nextInt(namespaces.size()));
+
+    try {
+      conn.namespaceOperations().delete(namespace, true);
+      log.debug("Deleted namespace " + namespace);
+    } catch (NamespaceNotFoundException e) {
+      log.debug("Delete namespace " + namespace + " failed, doesnt exist");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteTableNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteTableNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteTableNamespace.java
deleted file mode 100644
index ca6e39c..0000000
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteTableNamespace.java
+++ /dev/null
@@ -1,48 +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.test.randomwalk.concurrent;
-
-import java.util.List;
-import java.util.Properties;
-import java.util.Random;
-
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
-import org.apache.accumulo.test.randomwalk.State;
-import org.apache.accumulo.test.randomwalk.Test;
-
-public class DeleteTableNamespace extends Test {
-
-  @Override
-  public void visit(State state, Properties props) throws Exception {
-    Connector conn = state.getConnector();
-
-    Random rand = (Random) state.get("rand");
-
-    @SuppressWarnings("unchecked")
-    List<String> namespaces = (List<String>) state.get("namespaces");
-
-    String namespace = namespaces.get(rand.nextInt(namespaces.size()));
-
-    try {
-      conn.tableNamespaceOperations().delete(namespace, true);
-      log.debug("Deleted table namespace " + namespace);
-    } catch (TableNamespaceNotFoundException e) {
-      log.debug("Delete namespace " + namespace + " failed, doesnt exist");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineNamespace.java
new file mode 100644
index 0000000..e689921
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineNamespace.java
@@ -0,0 +1,53 @@
+/*
+ * 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.test.randomwalk.concurrent;
+
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.test.randomwalk.State;
+import org.apache.accumulo.test.randomwalk.Test;
+
+public class OfflineNamespace extends Test {
+
+  @Override
+  public void visit(State state, Properties props) throws Exception {
+    Connector conn = state.getConnector();
+
+    Random rand = (Random) state.get("rand");
+
+    @SuppressWarnings("unchecked")
+    List<String> namespaces = (List<String>) state.get("namespaces");
+
+    String namespace = namespaces.get(rand.nextInt(namespaces.size()));
+
+    try {
+      conn.namespaceOperations().offline(namespace);
+      log.debug("Offlined namespace " + namespace);
+      UtilWaitThread.sleep(rand.nextInt(200));
+      conn.namespaceOperations().online(namespace);
+      log.debug("Onlined namespace " + namespace);
+    } catch (NamespaceNotFoundException tne) {
+      log.debug("offline or online failed " + namespace + ", doesnt exist");
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTableNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTableNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTableNamespace.java
deleted file mode 100644
index 4ee2417..0000000
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTableNamespace.java
+++ /dev/null
@@ -1,53 +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.test.randomwalk.concurrent;
-
-import java.util.List;
-import java.util.Properties;
-import java.util.Random;
-
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
-import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.test.randomwalk.State;
-import org.apache.accumulo.test.randomwalk.Test;
-
-public class OfflineTableNamespace extends Test {
-
-  @Override
-  public void visit(State state, Properties props) throws Exception {
-    Connector conn = state.getConnector();
-
-    Random rand = (Random) state.get("rand");
-
-    @SuppressWarnings("unchecked")
-    List<String> namespaces = (List<String>) state.get("namespaces");
-
-    String namespace = namespaces.get(rand.nextInt(namespaces.size()));
-
-    try {
-      conn.tableNamespaceOperations().offline(namespace);
-      log.debug("Offlined namespace " + namespace);
-      UtilWaitThread.sleep(rand.nextInt(200));
-      conn.tableNamespaceOperations().online(namespace);
-      log.debug("Onlined namespace " + namespace);
-    } catch (TableNamespaceNotFoundException tne) {
-      log.debug("offline or online failed " + namespace + ", doesnt exist");
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameNamespace.java
new file mode 100644
index 0000000..bf22ad6
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameNamespace.java
@@ -0,0 +1,52 @@
+/*
+ * 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.test.randomwalk.concurrent;
+
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.NamespaceExistsException;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.test.randomwalk.State;
+import org.apache.accumulo.test.randomwalk.Test;
+
+public class RenameNamespace extends Test {
+
+  @Override
+  public void visit(State state, Properties props) throws Exception {
+    Connector conn = state.getConnector();
+
+    Random rand = (Random) state.get("rand");
+
+    @SuppressWarnings("unchecked")
+    List<String> namespaces = (List<String>) state.get("namespaces");
+
+    String srcName = namespaces.get(rand.nextInt(namespaces.size()));
+    String newName = namespaces.get(rand.nextInt(namespaces.size()));
+
+    try {
+      conn.namespaceOperations().rename(srcName, newName);
+      log.debug("Renamed namespace " + srcName + " " + newName);
+    } catch (NamespaceExistsException e) {
+      log.debug("Rename namespace " + srcName + " failed, " + newName + " exists");
+    } catch (NamespaceNotFoundException e) {
+      log.debug("Rename namespace " + srcName + " failed, doesnt exist");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java
deleted file mode 100644
index 15c7c00..0000000
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java
+++ /dev/null
@@ -1,52 +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.test.randomwalk.concurrent;
-
-import java.util.List;
-import java.util.Properties;
-import java.util.Random;
-
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.TableNamespaceExistsException;
-import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
-import org.apache.accumulo.test.randomwalk.State;
-import org.apache.accumulo.test.randomwalk.Test;
-
-public class RenameTableNamespace extends Test {
-
-  @Override
-  public void visit(State state, Properties props) throws Exception {
-    Connector conn = state.getConnector();
-
-    Random rand = (Random) state.get("rand");
-
-    @SuppressWarnings("unchecked")
-    List<String> namespaces = (List<String>) state.get("namespaces");
-
-    String srcName = namespaces.get(rand.nextInt(namespaces.size()));
-    String newName = namespaces.get(rand.nextInt(namespaces.size()));
-
-    try {
-      conn.tableNamespaceOperations().rename(srcName, newName);
-      log.debug("Renamed table namespace " + srcName + " " + newName);
-    } catch (TableNamespaceExistsException e) {
-      log.debug("Rename namespace " + srcName + " failed, " + newName + " exists");
-    } catch (TableNamespaceNotFoundException e) {
-      log.debug("Rename namespace " + srcName + " failed, doesnt exist");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/test/src/main/java/org/apache/accumulo/test/randomwalk/security/WalkingSecurity.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/security/WalkingSecurity.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/security/WalkingSecurity.java
index 08fd395..4bc072c 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/security/WalkingSecurity.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/security/WalkingSecurity.java
@@ -27,7 +27,7 @@ import java.util.Set;
 import java.util.TreeSet;
 
 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.client.impl.thrift.ThriftSecurityException;
@@ -35,8 +35,8 @@ import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 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.core.util.CachedConfiguration;
@@ -54,27 +54,27 @@ import org.apache.log4j.Logger;
 public class WalkingSecurity extends SecurityOperation implements Authorizor, Authenticator, PermissionHandler {
   State state = null;
   protected final static Logger log = Logger.getLogger(WalkingSecurity.class);
-  
+
   private static final String tableName = "SecurityTableName";
   private static final String userName = "UserName";
-  
+
   private static final String userPass = "UserPass";
   private static final String userExists = "UserExists";
   private static final String tableExists = "TableExists";
-  private static final String tableNamespaceExists = "TableNamespaceExists";
-  
+  private static final String namespaceExists = "NamespaceExists";
+
   private static final String connector = "UserConnection";
-  
+
   private static final String authsMap = "authorizationsCountMap";
   private static final String lastKey = "lastMutationKey";
   private static final String filesystem = "securityFileSystem";
-  
+
   private static WalkingSecurity instance = null;
-  
+
   public WalkingSecurity(Authorizor author, Authenticator authent, PermissionHandler pm, String instanceId) {
     super(author, authent, pm, instanceId);
   }
-  
+
   public WalkingSecurity(State state2) {
     super(state2.getInstance().getInstanceID());
     this.state = state2;
@@ -82,53 +82,53 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
     authenticator = this;
     permHandle = this;
   }
-  
+
   public static WalkingSecurity get(State state) {
     if (instance == null || instance.state != state) {
       instance = new WalkingSecurity(state);
       state.set(tableExists, Boolean.toString(false));
       state.set(authsMap, new HashMap<String,Integer>());
     }
-    
+
     return instance;
   }
-  
+
   @Override
   public void initialize(String instanceId, boolean initialize) {
     throw new UnsupportedOperationException("nope");
   }
-  
+
   @Override
   public boolean validSecurityHandlers(Authenticator one, PermissionHandler two) {
     return this.getClass().equals(one.getClass()) && this.getClass().equals(two.getClass());
   }
-  
+
   @Override
   public boolean validSecurityHandlers(Authenticator one, Authorizor two) {
     return this.getClass().equals(one.getClass()) && this.getClass().equals(two.getClass());
   }
-  
+
   @Override
   public boolean validSecurityHandlers(Authorizor one, PermissionHandler two) {
     return this.getClass().equals(one.getClass()) && this.getClass().equals(two.getClass());
   }
-  
+
   @Override
   public void initializeSecurity(TCredentials rootuser, String token) throws ThriftSecurityException {
     throw new UnsupportedOperationException("nope");
   }
-  
+
   @Override
   public void changeAuthorizations(String user, Authorizations authorizations) throws AccumuloSecurityException {
     state.set(user + "_auths", authorizations);
     state.set("Auths-" + user + '-' + "time", System.currentTimeMillis());
   }
-  
+
   @Override
   public Authorizations getCachedUserAuthorizations(String user) throws AccumuloSecurityException {
     return (Authorizations) state.get(user + "_auths");
   }
-  
+
   public boolean ambiguousAuthorizations(String userName) {
     Long setTime = state.getLong("Auths-" + userName + '-' + "time");
     if (setTime == null)
@@ -137,12 +137,12 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
       return true;
     return false;
   }
-  
+
   @Override
   public void initUser(String user) throws AccumuloSecurityException {
     changeAuthorizations(user, new Authorizations());
   }
-  
+
   @Override
   public Set<String> listUsers() throws AccumuloSecurityException {
     Set<String> userList = new TreeSet<String>();
@@ -152,21 +152,21 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
     }
     return userList;
   }
-  
+
   @Override
   public boolean authenticateUser(String principal, AuthenticationToken token) {
     PasswordToken pass = (PasswordToken) state.get(principal + userPass);
     boolean ret = pass.equals(token);
     return ret;
   }
-  
+
   @Override
   public void createUser(String principal, AuthenticationToken token) throws AccumuloSecurityException {
     state.set(principal + userExists, Boolean.toString(true));
     changePassword(principal, token);
     cleanUser(principal);
   }
-  
+
   @Override
   public void dropUser(String user) throws AccumuloSecurityException {
     state.set(user + userExists, Boolean.toString(false));
@@ -174,69 +174,71 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
     if (user.equals(getTabUserName()))
       state.set("table" + connector, null);
   }
-  
+
   @Override
   public void changePassword(String principal, AuthenticationToken token) throws AccumuloSecurityException {
     state.set(principal + userPass, token);
     state.set(principal + userPass + "time", System.currentTimeMillis());
   }
-  
+
   @Override
   public boolean userExists(String user) {
     return Boolean.parseBoolean(state.getString(user + userExists));
   }
-  
+
   @Override
   public boolean hasSystemPermission(String user, SystemPermission permission) throws AccumuloSecurityException {
     boolean res = Boolean.parseBoolean(state.getString("Sys-" + user + '-' + permission.name()));
     return res;
   }
-  
+
   @Override
   public boolean hasCachedSystemPermission(String user, SystemPermission permission) throws AccumuloSecurityException {
     return hasSystemPermission(user, permission);
   }
-  
+
   @Override
   public boolean hasTablePermission(String user, String table, TablePermission permission) throws AccumuloSecurityException, TableNotFoundException {
     return Boolean.parseBoolean(state.getString("Tab-" + user + '-' + permission.name()));
   }
-  
+
   @Override
   public boolean hasCachedTablePermission(String user, String table, TablePermission permission) throws AccumuloSecurityException, TableNotFoundException {
     return hasTablePermission(user, table, permission);
   }
-  
+
   @Override
-  public boolean hasTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws AccumuloSecurityException, TableNamespaceNotFoundException {
+  public boolean hasNamespacePermission(String user, String namespace, NamespacePermission permission) throws AccumuloSecurityException,
+      NamespaceNotFoundException {
     return Boolean.parseBoolean(state.getString("Nsp-" + user + '-' + permission.name()));
   }
-  
+
   @Override
-  public boolean hasCachedTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws AccumuloSecurityException, TableNamespaceNotFoundException {
-    return hasTableNamespacePermission(user, tableNamespace, permission);
+  public boolean hasCachedNamespacePermission(String user, String namespace, NamespacePermission permission) throws AccumuloSecurityException,
+      NamespaceNotFoundException {
+    return hasNamespacePermission(user, namespace, permission);
   }
-  
+
   @Override
   public void grantSystemPermission(String user, SystemPermission permission) throws AccumuloSecurityException {
     setSysPerm(state, user, permission, true);
   }
-  
+
   @Override
   public void revokeSystemPermission(String user, SystemPermission permission) throws AccumuloSecurityException {
     setSysPerm(state, user, permission, false);
   }
-  
+
   @Override
   public void grantTablePermission(String user, String table, TablePermission permission) throws AccumuloSecurityException, TableNotFoundException {
     setTabPerm(state, user, permission, table, true);
   }
-  
+
   private static void setSysPerm(State state, String userName, SystemPermission tp, boolean value) {
     log.debug((value ? "Gave" : "Took") + " the system permission " + tp.name() + (value ? " to" : " from") + " user " + userName);
     state.set("Sys-" + userName + '-' + tp.name(), Boolean.toString(value));
   }
-  
+
   private void setTabPerm(State state, String userName, TablePermission tp, String table, boolean value) {
     if (table.equals(userName))
       throw new RuntimeException("This is also fucked up");
@@ -245,31 +247,33 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
     if (tp.equals(TablePermission.READ) || tp.equals(TablePermission.WRITE))
       state.set("Tab-" + userName + '-' + tp.name() + '-' + "time", System.currentTimeMillis());
   }
-  
+
   @Override
   public void revokeTablePermission(String user, String table, TablePermission permission) throws AccumuloSecurityException, TableNotFoundException {
     setTabPerm(state, user, permission, table, false);
   }
-  
+
   @Override
-  public void grantTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws AccumuloSecurityException, TableNamespaceNotFoundException {
-    setNspPerm(state, user, permission, tableNamespace, true);
+  public void grantNamespacePermission(String user, String namespace, NamespacePermission permission) throws AccumuloSecurityException,
+      NamespaceNotFoundException {
+    setNspPerm(state, user, permission, namespace, true);
   }
-  
-  private void setNspPerm(State state, String userName, TableNamespacePermission tnp, String tableNamespace, boolean value) {
-    if (tableNamespace.equals(userName))
+
+  private void setNspPerm(State state, String userName, NamespacePermission tnp, String namespace, boolean value) {
+    if (namespace.equals(userName))
       throw new RuntimeException("I don't even know");
     log.debug((value ? "Gave" : "Took") + " the table permission " + tnp.name() + (value ? " to" : " from") + " user " + userName);
     state.set("Nsp-" + userName + '-' + tnp.name(), Boolean.toString(value));
-    if (tnp.equals(TableNamespacePermission.READ) || tnp.equals(TableNamespacePermission.WRITE))
+    if (tnp.equals(NamespacePermission.READ) || tnp.equals(NamespacePermission.WRITE))
       state.set("Nsp-" + userName + '-' + tnp.name() + '-' + "time", System.currentTimeMillis());
   }
-  
+
   @Override
-  public void revokeTableNamespacePermission(String user, String tableNamespace, TableNamespacePermission permission) throws AccumuloSecurityException, TableNamespaceNotFoundException {
-    setNspPerm(state, user, permission, tableNamespace, false);
+  public void revokeNamespacePermission(String user, String namespace, NamespacePermission permission) throws AccumuloSecurityException,
+      NamespaceNotFoundException {
+    setNspPerm(state, user, permission, namespace, false);
   }
-  
+
   @Override
   public void cleanTablePermissions(String table) throws AccumuloSecurityException, TableNotFoundException {
     for (String user : new String[] {getSysUserName(), getTabUserName()}) {
@@ -279,17 +283,17 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
     }
     state.set(tableExists, Boolean.toString(false));
   }
-  
+
   @Override
-  public void cleanTableNamespacePermissions(String tableNamespace) throws AccumuloSecurityException, TableNamespaceNotFoundException {
+  public void cleanNamespacePermissions(String namespace) throws AccumuloSecurityException, NamespaceNotFoundException {
     for (String user : new String[] {getSysUserName(), getNspUserName()}) {
-      for (TableNamespacePermission tnp : TableNamespacePermission.values()) {
-        revokeTableNamespacePermission(user, tableNamespace, tnp);
+      for (NamespacePermission tnp : NamespacePermission.values()) {
+        revokeNamespacePermission(user, namespace, tnp);
       }
     }
-    state.set(tableNamespaceExists, Boolean.toString(false));
+    state.set(namespaceExists, Boolean.toString(false));
   }
-  
+
   @Override
   public void cleanUser(String user) throws AccumuloSecurityException {
     if (getTableExists())
@@ -300,57 +304,57 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
     for (SystemPermission sp : SystemPermission.values())
       revokeSystemPermission(user, sp);
   }
-  
+
   public String getTabUserName() {
     return state.getString("table" + userName);
   }
-  
+
   public String getSysUserName() {
     return state.getString("system" + userName);
   }
-  
+
   public String getNspUserName() {
     return state.getString("namespace" + userName);
   }
-  
+
   public void setTabUserName(String name) {
     state.set("table" + userName, name);
     state.set(name + userExists, Boolean.toString(false));
   }
-  
+
   public void setNspUserName(String name) {
     state.set("namespace" + userName, name);
     state.set(name + userExists, Boolean.toString(false));
   }
-  
+
   public void setSysUserName(String name) {
     state.set("system" + userName, name);
   }
-  
+
   public String getTableName() {
     return state.getString(tableName);
   }
-  
+
   public boolean getTableExists() {
     return Boolean.parseBoolean(state.getString(tableExists));
   }
-  
+
   public TCredentials getSysCredentials() {
     return new Credentials(getSysUserName(), getSysToken()).toThrift(this.state.getInstance());
   }
-  
+
   public TCredentials getTabCredentials() {
     return new Credentials(getTabUserName(), getTabToken()).toThrift(this.state.getInstance());
   }
-  
+
   public AuthenticationToken getSysToken() {
     return new PasswordToken(getSysPassword());
   }
-  
+
   public AuthenticationToken getTabToken() {
     return new PasswordToken(getTabPassword());
   }
-  
+
   public byte[] getUserPassword(String user) {
     Object obj = state.get(user + userPass);
     if (obj instanceof PasswordToken) {
@@ -358,7 +362,7 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
     }
     return null;
   }
-  
+
   public byte[] getSysPassword() {
     Object obj = state.get(getSysUserName() + userPass);
     if (obj instanceof PasswordToken) {
@@ -366,7 +370,7 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
     }
     return null;
   }
-  
+
   public byte[] getTabPassword() {
     Object obj = state.get(getTabUserName() + userPass);
     if (obj instanceof PasswordToken) {
@@ -374,25 +378,25 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
     }
     return null;
   }
-  
+
   public boolean userPassTransient(String user) {
     return System.currentTimeMillis() - state.getLong(user + userPass + "time") < 1000;
   }
-  
+
   public void setTableName(String tName) {
     state.set(tableName, tName);
   }
-  
+
   @Override
   public void initTable(String table) throws AccumuloSecurityException {
     state.set(tableExists, Boolean.toString(true));
     state.set(tableName, table);
   }
-  
+
   public String[] getAuthsArray() {
     return new String[] {"Fishsticks", "PotatoSkins", "Ribs", "Asparagus", "Paper", "Towels", "Lint", "Brush", "Celery"};
   }
-  
+
   public boolean inAmbiguousZone(String userName, TablePermission tp) {
     if (tp.equals(TablePermission.READ) || tp.equals(TablePermission.WRITE)) {
       Long setTime = state.getLong("Tab-" + userName + '-' + tp.name() + '-' + "time");
@@ -403,16 +407,16 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
     }
     return false;
   }
-  
+
   @SuppressWarnings("unchecked")
   public Map<String,Integer> getAuthsMap() {
     return (Map<String,Integer>) state.get(authsMap);
   }
-  
+
   public String getLastKey() {
     return state.getString(lastKey);
   }
-  
+
   public void increaseAuthMap(String s, int increment) {
     Integer curVal = getAuthsMap().get(s);
     if (curVal == null) {
@@ -421,7 +425,7 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
     }
     curVal += increment;
   }
-  
+
   public FileSystem getFs() {
     FileSystem fs = null;
     try {
@@ -438,7 +442,7 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
     }
     return fs;
   }
-  
+
   @Override
   public boolean canAskAboutUser(TCredentials credentials, String user) throws ThriftSecurityException {
     try {
@@ -449,23 +453,23 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
       throw tse;
     }
   }
-  
+
   @Override
   public boolean validTokenClass(String tokenClass) {
     return tokenClass.equals(PasswordToken.class.getName());
   }
-  
+
   public static void clearInstance() {
     instance = null;
   }
-  
+
   @Override
   public Set<Class<? extends AuthenticationToken>> getSupportedTokenTypes() {
     Set<Class<? extends AuthenticationToken>> cs = new HashSet<Class<? extends AuthenticationToken>>();
     cs.add(PasswordToken.class);
     return cs;
   }
-  
+
   @Override
   public boolean isValidAuthorizations(String user, List<ByteBuffer> auths) throws AccumuloSecurityException {
     Collection<ByteBuffer> userauths = getCachedUserAuthorizations(user).getAuthorizationsBB();
@@ -474,5 +478,5 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
         return false;
     return true;
   }
-  
+
 }


[46/50] [abbrv] git commit: ACCUMULO-802 Address some issues from ReviewBoard

Posted by ct...@apache.org.
ACCUMULO-802 Address some issues from ReviewBoard


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

Branch: refs/heads/master
Commit: 725d82150c1933a2948822f018f8d8767ad13a11
Parents: bbba3f5
Author: Christopher Tubbs <ct...@apache.org>
Authored: Thu Nov 7 20:10:39 2013 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:11 2013 -0500

----------------------------------------------------------------------
 .../admin/TableNamespaceOperationsImpl.java     |   5 +-
 .../client/mock/MockSecurityOperations.java     |  67 ++++++------
 .../core/client/mock/MockTableOperations.java   | 106 +++++++++----------
 3 files changed, 89 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/725d8215/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
index 35fa8ae..602110b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
@@ -49,6 +49,7 @@ import org.apache.accumulo.core.client.impl.ServerClient;
 import org.apache.accumulo.core.client.impl.TableNamespaces;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.ClientService;
+import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.core.constraints.Constraint;
@@ -331,8 +332,8 @@ public class TableNamespaceOperationsImpl extends TableNamespaceOperationsHelper
     String namespaceId = TableNamespaces.getNamespaceId(instance, namespace);
 
     if (namespaceId.equals(Constants.SYSTEM_TABLE_NAMESPACE_ID) || namespaceId.equals(Constants.DEFAULT_TABLE_NAMESPACE_ID)) {
-      String why = "Can't delete the system or default table namespace";
-      throw new RuntimeException(why);
+      log.debug(credentials.getPrincipal() + " attempted to delete the " + namespaceId + " table namespace");
+      throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.UNSUPPORTED_OPERATION);
     }
 
     if (TableNamespaces.getTableIds(instance, namespaceId).size() > 0) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/725d8215/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
index dd48b52..9d85f9f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
@@ -31,42 +31,42 @@ import org.apache.accumulo.core.security.TableNamespacePermission;
 import org.apache.accumulo.core.security.TablePermission;
 
 public class MockSecurityOperations implements SecurityOperations {
-  
+
   final private MockAccumulo acu;
-  
+
   MockSecurityOperations(MockAccumulo acu) {
     this.acu = acu;
   }
-  
+
   @Deprecated
   @Override
   public void createUser(String user, byte[] password, Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
     createLocalUser(user, new PasswordToken(password));
     changeUserAuthorizations(user, authorizations);
   }
-  
+
   @Override
   public void createLocalUser(String principal, PasswordToken password) throws AccumuloException, AccumuloSecurityException {
     this.acu.users.put(principal, new MockUser(principal, password, new Authorizations()));
   }
-  
+
   @Deprecated
   @Override
   public void dropUser(String user) throws AccumuloException, AccumuloSecurityException {
     dropLocalUser(user);
   }
-  
+
   @Override
   public void dropLocalUser(String principal) throws AccumuloException, AccumuloSecurityException {
     this.acu.users.remove(principal);
   }
-  
+
   @Deprecated
   @Override
   public boolean authenticateUser(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
     return authenticateUser(user, new PasswordToken(password));
   }
-  
+
   @Override
   public boolean authenticateUser(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
     MockUser user = acu.users.get(principal);
@@ -74,13 +74,13 @@ public class MockSecurityOperations implements SecurityOperations {
       return false;
     return user.token.equals(token);
   }
-  
+
   @Deprecated
   @Override
   public void changeUserPassword(String user, byte[] password) throws AccumuloException, AccumuloSecurityException {
     changeLocalUserPassword(user, new PasswordToken(password));
   }
-  
+
   @Override
   public void changeLocalUserPassword(String principal, PasswordToken token) throws AccumuloException, AccumuloSecurityException {
     MockUser user = acu.users.get(principal);
@@ -89,7 +89,7 @@ public class MockSecurityOperations implements SecurityOperations {
     else
       throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
   }
-  
+
   @Override
   public void changeUserAuthorizations(String principal, Authorizations authorizations) throws AccumuloException, AccumuloSecurityException {
     MockUser user = acu.users.get(principal);
@@ -98,7 +98,7 @@ public class MockSecurityOperations implements SecurityOperations {
     else
       throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
   }
-  
+
   @Override
   public Authorizations getUserAuthorizations(String principal) throws AccumuloException, AccumuloSecurityException {
     MockUser user = acu.users.get(principal);
@@ -107,7 +107,7 @@ public class MockSecurityOperations implements SecurityOperations {
     else
       throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
   }
-  
+
   @Override
   public boolean hasSystemPermission(String principal, SystemPermission perm) throws AccumuloException, AccumuloSecurityException {
     MockUser user = acu.users.get(principal);
@@ -116,7 +116,7 @@ public class MockSecurityOperations implements SecurityOperations {
     else
       throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
   }
-  
+
   @Override
   public boolean hasTablePermission(String principal, String tableName, TablePermission perm) throws AccumuloException, AccumuloSecurityException {
     MockTable table = acu.tables.get(tableName);
@@ -127,18 +127,19 @@ public class MockSecurityOperations implements SecurityOperations {
       return false;
     return perms.contains(perm);
   }
-  
+
   @Override
-  public boolean hasTableNamespacePermission(String principal, String tableNamespace, TableNamespacePermission perm) throws AccumuloException, AccumuloSecurityException {
+  public boolean hasTableNamespacePermission(String principal, String tableNamespace, TableNamespacePermission perm) throws AccumuloException,
+      AccumuloSecurityException {
     MockTableNamespace namespace = acu.namespaces.get(tableNamespace);
     if (namespace == null)
-      throw new AccumuloSecurityException(tableNamespace, SecurityErrorCode.TABLE_DOESNT_EXIST);
+      throw new AccumuloSecurityException(tableNamespace, SecurityErrorCode.TABLE_NAMESPACE_DOESNT_EXIST);
     EnumSet<TableNamespacePermission> perms = namespace.userPermissions.get(principal);
     if (perms == null)
       return false;
     return perms.contains(perm);
   }
-  
+
   @Override
   public void grantSystemPermission(String principal, SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
     MockUser user = acu.users.get(principal);
@@ -147,7 +148,7 @@ public class MockSecurityOperations implements SecurityOperations {
     else
       throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
   }
-  
+
   @Override
   public void grantTablePermission(String principal, String tableName, TablePermission permission) throws AccumuloException, AccumuloSecurityException {
     if (acu.users.get(principal) == null)
@@ -161,21 +162,22 @@ public class MockSecurityOperations implements SecurityOperations {
     else
       perms.add(permission);
   }
-  
+
   @Override
-  public void grantTableNamespacePermission(String principal, String tableNamespace, TableNamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
+  public void grantTableNamespacePermission(String principal, String tableNamespace, TableNamespacePermission permission) throws AccumuloException,
+      AccumuloSecurityException {
     if (acu.users.get(principal) == null)
       throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
     MockTableNamespace namespace = acu.namespaces.get(tableNamespace);
     if (namespace == null)
-      throw new AccumuloSecurityException(tableNamespace, SecurityErrorCode.TABLE_DOESNT_EXIST);
+      throw new AccumuloSecurityException(tableNamespace, SecurityErrorCode.TABLE_NAMESPACE_DOESNT_EXIST);
     EnumSet<TableNamespacePermission> perms = namespace.userPermissions.get(principal);
     if (perms == null)
       namespace.userPermissions.put(principal, EnumSet.of(permission));
     else
       perms.add(permission);
   }
-  
+
   @Override
   public void revokeSystemPermission(String principal, SystemPermission permission) throws AccumuloException, AccumuloSecurityException {
     MockUser user = acu.users.get(principal);
@@ -184,7 +186,7 @@ public class MockSecurityOperations implements SecurityOperations {
     else
       throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
   }
-  
+
   @Override
   public void revokeTablePermission(String principal, String tableName, TablePermission permission) throws AccumuloException, AccumuloSecurityException {
     if (acu.users.get(principal) == null)
@@ -195,31 +197,32 @@ public class MockSecurityOperations implements SecurityOperations {
     EnumSet<TablePermission> perms = table.userPermissions.get(principal);
     if (perms != null)
       perms.remove(permission);
-    
+
   }
-  
+
   @Override
-  public void revokeTableNamespacePermission(String principal, String tableNamespace, TableNamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
+  public void revokeTableNamespacePermission(String principal, String tableNamespace, TableNamespacePermission permission) throws AccumuloException,
+      AccumuloSecurityException {
     if (acu.users.get(principal) == null)
       throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
     MockTableNamespace namespace = acu.namespaces.get(tableNamespace);
     if (namespace == null)
-      throw new AccumuloSecurityException(tableNamespace, SecurityErrorCode.TABLE_DOESNT_EXIST);
+      throw new AccumuloSecurityException(tableNamespace, SecurityErrorCode.TABLE_NAMESPACE_DOESNT_EXIST);
     EnumSet<TableNamespacePermission> perms = namespace.userPermissions.get(principal);
     if (perms != null)
       perms.remove(permission);
-    
+
   }
-  
+
   @Deprecated
   @Override
   public Set<String> listUsers() throws AccumuloException, AccumuloSecurityException {
     return listLocalUsers();
   }
-  
+
   @Override
   public Set<String> listLocalUsers() throws AccumuloException, AccumuloSecurityException {
     return acu.users.keySet();
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/725d8215/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
index 78b7826..e70a545 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
@@ -61,43 +61,39 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 
 public class MockTableOperations extends TableOperationsHelper {
-  
+
   final private MockAccumulo acu;
   final private String username;
-  
+
   MockTableOperations(MockAccumulo acu, String username) {
     this.acu = acu;
     this.username = username;
   }
-  
+
   @Override
   public SortedSet<String> list() {
     return new TreeSet<String>(acu.tables.keySet());
   }
-  
-  /*
-   * @Override public SortedSet<String> listNamespaces() { return new TreeSet<String>(acu.namespaces.keySet()); }
-   */
-  
+
   @Override
   public boolean exists(String tableName) {
     return acu.tables.containsKey(tableName);
   }
-  
+
   private boolean namespaceExists(String namespace) {
     return acu.namespaces.containsKey(namespace);
   }
-  
+
   @Override
   public void create(String tableName) throws AccumuloException, AccumuloSecurityException, TableExistsException {
     create(tableName, true, TimeType.MILLIS);
   }
-  
+
   @Override
   public void create(String tableName, boolean versioningIter) throws AccumuloException, AccumuloSecurityException, TableExistsException {
     create(tableName, versioningIter, TimeType.MILLIS);
   }
-  
+
   @Override
   public void create(String tableName, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException {
     String namespace = Tables.extractNamespace(tableName);
@@ -106,51 +102,51 @@ public class MockTableOperations extends TableOperationsHelper {
     }
     if (exists(tableName))
       throw new TableExistsException(tableName, tableName, "");
-    
+
     if (!namespaceExists(namespace)) {
       throw new IllegalArgumentException("Table namespace (" + namespace + ") does not exist, create it first");
     }
     acu.createTable(username, tableName, versioningIter, timeType);
   }
-  
+
   @Override
   public void addSplits(String tableName, SortedSet<Text> partitionKeys) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
     if (!exists(tableName))
       throw new TableNotFoundException(tableName, tableName, "");
     acu.addSplits(tableName, partitionKeys);
   }
-  
+
   @Deprecated
   @Override
   public Collection<Text> getSplits(String tableName) throws TableNotFoundException {
     return listSplits(tableName);
   }
-  
+
   @Deprecated
   @Override
   public Collection<Text> getSplits(String tableName, int maxSplits) throws TableNotFoundException {
     return listSplits(tableName);
   }
-  
+
   @Override
   public Collection<Text> listSplits(String tableName) throws TableNotFoundException {
     if (!exists(tableName))
       throw new TableNotFoundException(tableName, tableName, "");
     return acu.getSplits(tableName);
   }
-  
+
   @Override
   public Collection<Text> listSplits(String tableName, int maxSplits) throws TableNotFoundException {
     return listSplits(tableName);
   }
-  
+
   @Override
   public void delete(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     if (!exists(tableName))
       throw new TableNotFoundException(tableName, tableName, "");
     acu.tables.remove(tableName);
   }
-  
+
   @Override
   public void rename(String oldTableName, String newTableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException,
       TableExistsException {
@@ -169,35 +165,35 @@ public class MockTableOperations extends TableOperationsHelper {
     acu.namespaces.put(namespace, n);
     acu.tables.put(newTableName, t);
   }
-  
+
   @Deprecated
   @Override
   public void flush(String tableName) throws AccumuloException, AccumuloSecurityException {}
-  
+
   @Override
   public void setProperty(String tableName, String property, String value) throws AccumuloException, AccumuloSecurityException {
     acu.tables.get(tableName).settings.put(property, value);
   }
-  
+
   @Override
   public void removeProperty(String tableName, String property) throws AccumuloException, AccumuloSecurityException {
     acu.tables.get(tableName).settings.remove(property);
   }
-  
+
   @Override
   public Iterable<Entry<String,String>> getProperties(String tableName) throws TableNotFoundException {
     String namespace = Tables.extractNamespace(tableName);
-    
+
     if (!namespaceExists(namespace)) {
       throw new IllegalArgumentException("Table namespace (" + namespace + ") does not exist");
     }
-    
+
     Set<Entry<String,String>> props = new HashSet<Entry<String,String>>(acu.namespaces.get(namespace).settings.entrySet());
-    
+
     if (!exists(tableName)) {
       throw new TableNotFoundException(tableName, tableName, "");
     }
-    
+
     Set<Entry<String,String>> tableProps = acu.tables.get(tableName).settings.entrySet();
     for (Entry<String,String> e : tableProps) {
       if (props.contains(e)) {
@@ -207,21 +203,21 @@ public class MockTableOperations extends TableOperationsHelper {
     }
     return props;
   }
-  
+
   @Override
   public void setLocalityGroups(String tableName, Map<String,Set<Text>> groups) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     if (!exists(tableName))
       throw new TableNotFoundException(tableName, tableName, "");
     acu.tables.get(tableName).setLocalityGroups(groups);
   }
-  
+
   @Override
   public Map<String,Set<Text>> getLocalityGroups(String tableName) throws AccumuloException, TableNotFoundException {
     if (!exists(tableName))
       throw new TableNotFoundException(tableName, tableName, "");
     return acu.tables.get(tableName).getLocalityGroups();
   }
-  
+
   @Override
   public Set<Range> splitRangeByTablets(String tableName, Range range, int maxSplits) throws AccumuloException, AccumuloSecurityException,
       TableNotFoundException {
@@ -229,7 +225,7 @@ public class MockTableOperations extends TableOperationsHelper {
       throw new TableNotFoundException(tableName, tableName, "");
     return Collections.singleton(range);
   }
-  
+
   @Override
   public void importDirectory(String tableName, String dir, String failureDir, boolean setTime) throws IOException, AccumuloException,
       AccumuloSecurityException, TableNotFoundException {
@@ -240,7 +236,7 @@ public class MockTableOperations extends TableOperationsHelper {
     }
     Path importPath = new Path(dir);
     Path failurePath = new Path(failureDir);
-    
+
     FileSystem fs = acu.getFileSystem();
     /*
      * check preconditions
@@ -315,35 +311,35 @@ public class MockTableOperations extends TableOperationsHelper {
       fs.delete(importStatus.getPath(), true);
     }
   }
-  
+
   @Override
   public void offline(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
     offline(tableName, false);
   }
-  
+
   @Override
   public void offline(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
     if (!exists(tableName))
       throw new TableNotFoundException(tableName, tableName, "");
   }
-  
+
   @Override
   public void online(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
     online(tableName, false);
   }
-  
+
   @Override
   public void online(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
     if (!exists(tableName))
       throw new TableNotFoundException(tableName, tableName, "");
   }
-  
+
   @Override
   public void clearLocatorCache(String tableName) throws TableNotFoundException {
     if (!exists(tableName))
       throw new TableNotFoundException(tableName, tableName, "");
   }
-  
+
   @Override
   public Map<String,String> tableIdMap() {
     Map<String,String> result = new HashMap<String,String>();
@@ -357,23 +353,23 @@ public class MockTableOperations extends TableOperationsHelper {
     }
     return result;
   }
-  
+
   @Override
   public List<DiskUsage> getDiskUsage(Set<String> tables) throws AccumuloException, AccumuloSecurityException {
-    
+
     List<DiskUsage> diskUsages = new ArrayList<DiskUsage>();
     diskUsages.add(new DiskUsage(new TreeSet<String>(tables), 0l));
-    
+
     return diskUsages;
   }
-  
+
   @Override
   public void merge(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     if (!exists(tableName))
       throw new TableNotFoundException(tableName, tableName, "");
     acu.merge(tableName, start, end);
   }
-  
+
   @Override
   public void deleteRows(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     if (!exists(tableName))
@@ -382,63 +378,63 @@ public class MockTableOperations extends TableOperationsHelper {
     Set<Key> keep = new TreeSet<Key>(t.table.tailMap(new Key(start)).headMap(new Key(end)).keySet());
     t.table.keySet().removeAll(keep);
   }
-  
+
   @Override
   public void compact(String tableName, Text start, Text end, boolean flush, boolean wait) throws AccumuloSecurityException, TableNotFoundException,
       AccumuloException {
     if (!exists(tableName))
       throw new TableNotFoundException(tableName, tableName, "");
   }
-  
+
   @Override
   public void compact(String tableName, Text start, Text end, List<IteratorSetting> iterators, boolean flush, boolean wait) throws AccumuloSecurityException,
       TableNotFoundException, AccumuloException {
     if (!exists(tableName))
       throw new TableNotFoundException(tableName, tableName, "");
   }
-  
+
   @Override
   public void cancelCompaction(String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException {
     if (!exists(tableName))
       throw new TableNotFoundException(tableName, tableName, "");
   }
-  
+
   @Override
   public void clone(String srcTableName, String newTableName, boolean flush, Map<String,String> propertiesToSet, Set<String> propertiesToExclude)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException, TableExistsException {
     throw new NotImplementedException();
   }
-  
+
   @Override
   public void flush(String tableName, Text start, Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     if (!exists(tableName))
       throw new TableNotFoundException(tableName, tableName, "");
   }
-  
+
   @Override
   public Text getMaxRow(String tableName, Authorizations auths, Text startRow, boolean startInclusive, Text endRow, boolean endInclusive)
       throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
     MockTable table = acu.tables.get(tableName);
     if (table == null)
       throw new TableNotFoundException(tableName, tableName, "no such table");
-    
+
     return FindMax.findMax(new MockScanner(table, auths), startRow, startInclusive, endRow, endInclusive);
   }
-  
+
   @Override
   public void importTable(String tableName, String exportDir) throws TableExistsException, AccumuloException, AccumuloSecurityException {
     throw new NotImplementedException();
   }
-  
+
   @Override
   public void exportTable(String tableName, String exportDir) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
     throw new NotImplementedException();
   }
-  
+
   @Override
   public boolean testClassLoad(String tableName, String className, String asTypeName) throws AccumuloException, AccumuloSecurityException,
       TableNotFoundException {
-    
+
     try {
       AccumuloVFSClassLoader.loadClass(className, Class.forName(asTypeName));
     } catch (ClassNotFoundException e) {


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

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/SecurityErrorCode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/SecurityErrorCode.java b/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/SecurityErrorCode.java
index 9bf554f..adeff6f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/SecurityErrorCode.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/SecurityErrorCode.java
@@ -46,7 +46,7 @@ import org.apache.thrift.TEnum;
   TOKEN_EXPIRED(15),
   SERIALIZATION_ERROR(16),
   INSUFFICIENT_PROPERTIES(17),
-  TABLE_NAMESPACE_DOESNT_EXIST(18);
+  NAMESPACE_DOESNT_EXIST(18);
 
   private final int value;
 
@@ -104,7 +104,7 @@ import org.apache.thrift.TEnum;
       case 17:
         return INSUFFICIENT_PROPERTIES;
       case 18:
-        return TABLE_NAMESPACE_DOESNT_EXIST;
+        return NAMESPACE_DOESNT_EXIST;
       default:
         return null;
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
index f96abad..497716f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
@@ -32,14 +32,14 @@ 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.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.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.Text;
 
 public class MockAccumulo {
   final Map<String,MockTable> tables = new HashMap<String,MockTable>();
-  final Map<String,MockTableNamespace> namespaces = new HashMap<String,MockTableNamespace>();
+  final Map<String,MockNamespace> namespaces = new HashMap<String,MockNamespace>();
   final Map<String,String> systemProperties = new HashMap<String,String>();
   Map<String,MockUser> users = new HashMap<String,MockUser>();
   final FileSystem fs;
@@ -52,8 +52,8 @@ public class MockAccumulo {
     MockUser root = new MockUser("root", new PasswordToken(new byte[0]), Authorizations.EMPTY);
     root.permissions.add(SystemPermission.SYSTEM);
     users.put(root.name, root);
-    namespaces.put(Constants.DEFAULT_TABLE_NAMESPACE, new MockTableNamespace());
-    namespaces.put(Constants.SYSTEM_TABLE_NAMESPACE, new MockTableNamespace());
+    namespaces.put(Constants.DEFAULT_NAMESPACE, new MockNamespace());
+    namespaces.put(Constants.SYSTEM_NAMESPACE, new MockNamespace());
     createTable("root", RootTable.NAME, true, TimeType.LOGICAL);
     createTable("root", MetadataTable.NAME, true, TimeType.LOGICAL);
   }
@@ -90,7 +90,7 @@ public class MockAccumulo {
       return;
     }
     
-    MockTableNamespace n = namespaces.get(namespace);
+    MockNamespace n = namespaces.get(namespace);
     MockTable t = new MockTable(n, useVersions, timeType);
     t.userPermissions.put(username, EnumSet.allOf(TablePermission.class));
     t.setNamespaceName(namespace);
@@ -100,8 +100,8 @@ public class MockAccumulo {
   
   public void createNamespace(String username, String namespace) {
     if (!namespaceExists(namespace)) {
-      MockTableNamespace n = new MockTableNamespace();
-      n.userPermissions.put(username, EnumSet.allOf(TableNamespacePermission.class));
+      MockNamespace n = new MockNamespace();
+      n.userPermissions.put(username, EnumSet.allOf(NamespacePermission.class));
       namespaces.put(namespace, n);
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
index ecca271..996198c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
@@ -32,7 +32,7 @@ import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.InstanceOperations;
 import org.apache.accumulo.core.client.admin.SecurityOperations;
-import org.apache.accumulo.core.client.admin.TableNamespaceOperations;
+import org.apache.accumulo.core.client.admin.NamespaceOperations;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.NullToken;
@@ -138,8 +138,8 @@ public class MockConnector extends Connector {
   }
 
   @Override
-  public TableNamespaceOperations tableNamespaceOperations() {
-    return new MockTableNamespaceOperations(acu, username);
+  public NamespaceOperations namespaceOperations() {
+    return new MockNamespaceOperations(acu, username);
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespace.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespace.java
new file mode 100644
index 0000000..2ab9b09
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespace.java
@@ -0,0 +1,54 @@
+/*
+ * 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.core.client.mock;
+
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.security.NamespacePermission;
+
+public class MockNamespace {
+
+  final Map<String,String> settings;
+  Map<String,EnumSet<NamespacePermission>> userPermissions = new HashMap<String,EnumSet<NamespacePermission>>();
+
+  public MockNamespace() {
+    settings = new HashMap<String,String>();
+    for (Entry<String,String> entry : AccumuloConfiguration.getDefaultConfiguration()) {
+      String key = entry.getKey();
+      if (key.startsWith(Property.TABLE_PREFIX.getKey())) {
+        settings.put(key, entry.getValue());
+      }
+    }
+  }
+
+  public List<String> getTables(MockAccumulo acu) {
+    List<String> l = new LinkedList<String>();
+    for (String t : acu.tables.keySet()) {
+      if (acu.tables.get(t).getNamespace().equals(this)) {
+        l.add(t);
+      }
+    }
+    return l;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
new file mode 100644
index 0000000..fc76daa
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
@@ -0,0 +1,188 @@
+/*
+ * 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.core.client.mock;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.NamespaceExistsException;
+import org.apache.accumulo.core.client.NamespaceNotEmptyException;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.DiskUsage;
+import org.apache.accumulo.core.client.admin.NamespaceOperationsHelper;
+import org.apache.accumulo.core.client.admin.TimeType;
+import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
+
+public class MockNamespaceOperations extends NamespaceOperationsHelper {
+
+  final private MockAccumulo acu;
+  final private String username;
+
+  MockNamespaceOperations(MockAccumulo acu, String username) {
+    this.acu = acu;
+    this.username = username;
+  }
+
+  @Override
+  public SortedSet<String> list() {
+    return new TreeSet<String>(acu.namespaces.keySet());
+  }
+
+  @Override
+  public boolean exists(String tableName) {
+    return acu.namespaces.containsKey(tableName);
+  }
+
+  @Override
+  public void create(String tableName) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException {
+    create(tableName, true, TimeType.MILLIS);
+  }
+
+  @Override
+  public void create(String tableName, boolean versioningIter) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException {
+    create(tableName, versioningIter, TimeType.MILLIS);
+  }
+
+  @Override
+  public void create(String namespace, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException,
+      NamespaceExistsException {
+    if (!namespace.matches(Constants.VALID_NAMESPACE_REGEX)) {
+      throw new IllegalArgumentException();
+    }
+
+    if (exists(namespace))
+      throw new NamespaceExistsException(namespace, namespace, "");
+
+    if (!exists(namespace)) {
+      acu.createNamespace(username, namespace);
+    }
+    acu.createTable(username, namespace, versioningIter, timeType);
+  }
+
+  @Override
+  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException, NamespaceNotEmptyException,
+      TableNotFoundException {
+    delete(namespace, false);
+  }
+
+  @Override
+  public void delete(String namespace, boolean deleteTables) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException,
+      NamespaceNotEmptyException, TableNotFoundException {
+    if (!exists(namespace))
+      throw new NamespaceNotFoundException(namespace, namespace, "");
+
+    MockNamespace n = acu.namespaces.get(namespace);
+    if (!deleteTables) {
+      if (n.getTables(acu).size() > 0) {
+        throw new NamespaceNotEmptyException(null, namespace, null);
+      }
+    } else {
+      for (String t : n.getTables(acu)) {
+        new MockTableOperations(acu, username).delete(t);
+      }
+    }
+    acu.namespaces.remove(namespace);
+  }
+
+  @Override
+  public void rename(String oldNamespaceName, String newNamespaceName) throws AccumuloSecurityException, NamespaceNotFoundException, AccumuloException,
+      NamespaceExistsException {
+    if (!exists(oldNamespaceName))
+      throw new NamespaceNotFoundException(oldNamespaceName, oldNamespaceName, "");
+    if (exists(newNamespaceName))
+      throw new NamespaceExistsException(newNamespaceName, newNamespaceName, "");
+
+    MockNamespace n = acu.namespaces.get(oldNamespaceName);
+    for (String t : n.getTables(acu)) {
+      String tt = newNamespaceName + "." + Tables.extractTableName(t);
+      acu.tables.put(tt, acu.tables.remove(t));
+    }
+    acu.namespaces.put(newNamespaceName, acu.namespaces.remove(oldNamespaceName));
+  }
+
+  @Override
+  public void setProperty(String namespace, String property, String value) throws AccumuloException, AccumuloSecurityException {
+    acu.namespaces.get(namespace).settings.put(property, value);
+  }
+
+  @Override
+  public void removeProperty(String namespace, String property) throws AccumuloException, AccumuloSecurityException {
+    acu.namespaces.get(namespace).settings.remove(property);
+  }
+
+  @Override
+  public Iterable<Entry<String,String>> getProperties(String namespace) throws NamespaceNotFoundException {
+    if (!exists(namespace)) {
+      throw new NamespaceNotFoundException(namespace, namespace, "");
+    }
+
+    return acu.namespaces.get(namespace).settings.entrySet();
+  }
+
+  @Override
+  public void offline(String namespace) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException {
+    if (!exists(namespace))
+      throw new NamespaceNotFoundException(namespace, namespace, "");
+  }
+
+  @Override
+  public void online(String namespace) throws AccumuloSecurityException, AccumuloException, NamespaceNotFoundException {
+    if (!exists(namespace))
+      throw new NamespaceNotFoundException(namespace, namespace, "");
+  }
+
+  @Override
+  public Map<String,String> namespaceIdMap() {
+    Map<String,String> result = new HashMap<String,String>();
+    for (String table : acu.tables.keySet()) {
+      result.put(table, table);
+    }
+    return result;
+  }
+
+  @Override
+  public List<DiskUsage> getDiskUsage(String namespace) throws AccumuloException, AccumuloSecurityException {
+
+    List<DiskUsage> diskUsages = new ArrayList<DiskUsage>();
+    diskUsages.add(new DiskUsage(new TreeSet<String>(acu.namespaces.get(namespace).getTables(acu)), 0l));
+
+    return diskUsages;
+  }
+
+  @Override
+  public boolean testClassLoad(String namespace, String className, String asTypeName) throws AccumuloException, AccumuloSecurityException,
+      NamespaceNotFoundException {
+
+    try {
+      AccumuloVFSClassLoader.loadClass(className, Class.forName(asTypeName));
+    } catch (ClassNotFoundException e) {
+      e.printStackTrace();
+      return false;
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
index 9d85f9f..16a8e02 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
@@ -26,8 +26,8 @@ import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.security.Authorizations;
+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;
 
 public class MockSecurityOperations implements SecurityOperations {
@@ -129,15 +129,14 @@ public class MockSecurityOperations implements SecurityOperations {
   }
 
   @Override
-  public boolean hasTableNamespacePermission(String principal, String tableNamespace, TableNamespacePermission perm) throws AccumuloException,
-      AccumuloSecurityException {
-    MockTableNamespace namespace = acu.namespaces.get(tableNamespace);
-    if (namespace == null)
-      throw new AccumuloSecurityException(tableNamespace, SecurityErrorCode.TABLE_NAMESPACE_DOESNT_EXIST);
-    EnumSet<TableNamespacePermission> perms = namespace.userPermissions.get(principal);
+  public boolean hasNamespacePermission(String principal, String namespace, NamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
+    MockNamespace mockNamespace = acu.namespaces.get(namespace);
+    if (mockNamespace == null)
+      throw new AccumuloSecurityException(namespace, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
+    EnumSet<NamespacePermission> perms = mockNamespace.userPermissions.get(principal);
     if (perms == null)
       return false;
-    return perms.contains(perm);
+    return perms.contains(permission);
   }
 
   @Override
@@ -164,16 +163,15 @@ public class MockSecurityOperations implements SecurityOperations {
   }
 
   @Override
-  public void grantTableNamespacePermission(String principal, String tableNamespace, TableNamespacePermission permission) throws AccumuloException,
-      AccumuloSecurityException {
+  public void grantNamespacePermission(String principal, String namespace, NamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
     if (acu.users.get(principal) == null)
       throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-    MockTableNamespace namespace = acu.namespaces.get(tableNamespace);
-    if (namespace == null)
-      throw new AccumuloSecurityException(tableNamespace, SecurityErrorCode.TABLE_NAMESPACE_DOESNT_EXIST);
-    EnumSet<TableNamespacePermission> perms = namespace.userPermissions.get(principal);
+    MockNamespace mockNamespace = acu.namespaces.get(namespace);
+    if (mockNamespace == null)
+      throw new AccumuloSecurityException(namespace, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
+    EnumSet<NamespacePermission> perms = mockNamespace.userPermissions.get(principal);
     if (perms == null)
-      namespace.userPermissions.put(principal, EnumSet.of(permission));
+      mockNamespace.userPermissions.put(principal, EnumSet.of(permission));
     else
       perms.add(permission);
   }
@@ -201,14 +199,13 @@ public class MockSecurityOperations implements SecurityOperations {
   }
 
   @Override
-  public void revokeTableNamespacePermission(String principal, String tableNamespace, TableNamespacePermission permission) throws AccumuloException,
-      AccumuloSecurityException {
+  public void revokeNamespacePermission(String principal, String namespace, NamespacePermission permission) throws AccumuloException, AccumuloSecurityException {
     if (acu.users.get(principal) == null)
       throw new AccumuloSecurityException(principal, SecurityErrorCode.USER_DOESNT_EXIST);
-    MockTableNamespace namespace = acu.namespaces.get(tableNamespace);
-    if (namespace == null)
-      throw new AccumuloSecurityException(tableNamespace, SecurityErrorCode.TABLE_NAMESPACE_DOESNT_EXIST);
-    EnumSet<TableNamespacePermission> perms = namespace.userPermissions.get(principal);
+    MockNamespace mockNamespace = acu.namespaces.get(namespace);
+    if (mockNamespace == null)
+      throw new AccumuloSecurityException(namespace, SecurityErrorCode.NAMESPACE_DOESNT_EXIST);
+    EnumSet<NamespacePermission> perms = mockNamespace.userPermissions.get(principal);
     if (perms != null)
       perms.remove(permission);
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
index 5c4cb36..cb50761 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
@@ -90,7 +90,7 @@ public class MockTable {
   private TimeType timeType;
   SortedSet<Text> splits = new ConcurrentSkipListSet<Text>();
   Map<String,Set<Text>> localityGroups = new TreeMap<String,Set<Text>>();
-  private MockTableNamespace namespace;
+  private MockNamespace namespace;
   private String namespaceName;
   
   MockTable(boolean limitVersion, TimeType timeType) {
@@ -103,7 +103,7 @@ public class MockTable {
     }
   }
   
-  MockTable(MockTableNamespace namespace, boolean limitVersion, TimeType timeType) {
+  MockTable(MockNamespace namespace, boolean limitVersion, TimeType timeType) {
     this(limitVersion, timeType);
     Set<Entry<String,String>> set = namespace.settings.entrySet();
     Iterator<Entry<String,String>> entries = set.iterator();
@@ -164,7 +164,7 @@ public class MockTable {
     this.namespaceName = n;
   }
   
-  public void setNamespace(MockTableNamespace n) {
+  public void setNamespace(MockNamespace n) {
     this.namespace = n;
   }
   
@@ -172,7 +172,7 @@ public class MockTable {
     return this.namespaceName;
   }
   
-  public MockTableNamespace getNamespace() {
+  public MockNamespace getNamespace() {
     return this.namespace;
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java
deleted file mode 100644
index 8523145..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java
+++ /dev/null
@@ -1,54 +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.core.client.mock;
-
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.security.TableNamespacePermission;
-
-public class MockTableNamespace {
-
-  final Map<String,String> settings;
-  Map<String,EnumSet<TableNamespacePermission>> userPermissions = new HashMap<String,EnumSet<TableNamespacePermission>>();
-
-  public MockTableNamespace() {
-    settings = new HashMap<String,String>();
-    for (Entry<String,String> entry : AccumuloConfiguration.getDefaultConfiguration()) {
-      String key = entry.getKey();
-      if (key.startsWith(Property.TABLE_PREFIX.getKey())) {
-        settings.put(key, entry.getValue());
-      }
-    }
-  }
-
-  public List<String> getTables(MockAccumulo acu) {
-    List<String> l = new LinkedList<String>();
-    for (String t : acu.tables.keySet()) {
-      if (acu.tables.get(t).getNamespace().equals(this)) {
-        l.add(t);
-      }
-    }
-    return l;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java
deleted file mode 100644
index c04c522..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java
+++ /dev/null
@@ -1,188 +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.core.client.mock;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.TableNamespaceExistsException;
-import org.apache.accumulo.core.client.TableNamespaceNotEmptyException;
-import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.admin.DiskUsage;
-import org.apache.accumulo.core.client.admin.TableNamespaceOperationsHelper;
-import org.apache.accumulo.core.client.admin.TimeType;
-import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
-
-public class MockTableNamespaceOperations extends TableNamespaceOperationsHelper {
-
-  final private MockAccumulo acu;
-  final private String username;
-
-  MockTableNamespaceOperations(MockAccumulo acu, String username) {
-    this.acu = acu;
-    this.username = username;
-  }
-
-  @Override
-  public SortedSet<String> list() {
-    return new TreeSet<String>(acu.namespaces.keySet());
-  }
-
-  @Override
-  public boolean exists(String tableName) {
-    return acu.namespaces.containsKey(tableName);
-  }
-
-  @Override
-  public void create(String tableName) throws AccumuloException, AccumuloSecurityException, TableNamespaceExistsException {
-    create(tableName, true, TimeType.MILLIS);
-  }
-
-  @Override
-  public void create(String tableName, boolean versioningIter) throws AccumuloException, AccumuloSecurityException, TableNamespaceExistsException {
-    create(tableName, versioningIter, TimeType.MILLIS);
-  }
-
-  @Override
-  public void create(String namespace, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException,
-      TableNamespaceExistsException {
-    if (!namespace.matches(Constants.VALID_TABLE_NAMESPACE_REGEX)) {
-      throw new IllegalArgumentException();
-    }
-
-    if (exists(namespace))
-      throw new TableNamespaceExistsException(namespace, namespace, "");
-
-    if (!exists(namespace)) {
-      acu.createNamespace(username, namespace);
-    }
-    acu.createTable(username, namespace, versioningIter, timeType);
-  }
-
-  @Override
-  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException, TableNamespaceNotEmptyException,
-      TableNotFoundException {
-    delete(namespace, false);
-  }
-
-  @Override
-  public void delete(String namespace, boolean deleteTables) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException,
-      TableNamespaceNotEmptyException, TableNotFoundException {
-    if (!exists(namespace))
-      throw new TableNamespaceNotFoundException(namespace, namespace, "");
-
-    MockTableNamespace n = acu.namespaces.get(namespace);
-    if (!deleteTables) {
-      if (n.getTables(acu).size() > 0) {
-        throw new TableNamespaceNotEmptyException(null, namespace, null);
-      }
-    } else {
-      for (String t : n.getTables(acu)) {
-        new MockTableOperations(acu, username).delete(t);
-      }
-    }
-    acu.namespaces.remove(namespace);
-  }
-
-  @Override
-  public void rename(String oldNamespaceName, String newNamespaceName) throws AccumuloSecurityException, TableNamespaceNotFoundException, AccumuloException,
-      TableNamespaceExistsException {
-    if (!exists(oldNamespaceName))
-      throw new TableNamespaceNotFoundException(oldNamespaceName, oldNamespaceName, "");
-    if (exists(newNamespaceName))
-      throw new TableNamespaceExistsException(newNamespaceName, newNamespaceName, "");
-
-    MockTableNamespace n = acu.namespaces.get(oldNamespaceName);
-    for (String t : n.getTables(acu)) {
-      String tt = newNamespaceName + "." + Tables.extractTableName(t);
-      acu.tables.put(tt, acu.tables.remove(t));
-    }
-    acu.namespaces.put(newNamespaceName, acu.namespaces.remove(oldNamespaceName));
-  }
-
-  @Override
-  public void setProperty(String namespace, String property, String value) throws AccumuloException, AccumuloSecurityException {
-    acu.namespaces.get(namespace).settings.put(property, value);
-  }
-
-  @Override
-  public void removeProperty(String namespace, String property) throws AccumuloException, AccumuloSecurityException {
-    acu.namespaces.get(namespace).settings.remove(property);
-  }
-
-  @Override
-  public Iterable<Entry<String,String>> getProperties(String namespace) throws TableNamespaceNotFoundException {
-    if (!exists(namespace)) {
-      throw new TableNamespaceNotFoundException(namespace, namespace, "");
-    }
-
-    return acu.namespaces.get(namespace).settings.entrySet();
-  }
-
-  @Override
-  public void offline(String namespace) throws AccumuloSecurityException, AccumuloException, TableNamespaceNotFoundException {
-    if (!exists(namespace))
-      throw new TableNamespaceNotFoundException(namespace, namespace, "");
-  }
-
-  @Override
-  public void online(String namespace) throws AccumuloSecurityException, AccumuloException, TableNamespaceNotFoundException {
-    if (!exists(namespace))
-      throw new TableNamespaceNotFoundException(namespace, namespace, "");
-  }
-
-  @Override
-  public Map<String,String> namespaceIdMap() {
-    Map<String,String> result = new HashMap<String,String>();
-    for (String table : acu.tables.keySet()) {
-      result.put(table, table);
-    }
-    return result;
-  }
-
-  @Override
-  public List<DiskUsage> getDiskUsage(String namespace) throws AccumuloException, AccumuloSecurityException {
-
-    List<DiskUsage> diskUsages = new ArrayList<DiskUsage>();
-    diskUsages.add(new DiskUsage(new TreeSet<String>(acu.namespaces.get(namespace).getTables(acu)), 0l));
-
-    return diskUsages;
-  }
-
-  @Override
-  public boolean testClassLoad(String namespace, String className, String asTypeName) throws AccumuloException, AccumuloSecurityException,
-      TableNamespaceNotFoundException {
-
-    try {
-      AccumuloVFSClassLoader.loadClass(className, Class.forName(asTypeName));
-    } catch (ClassNotFoundException e) {
-      e.printStackTrace();
-      return false;
-    }
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
index e70a545..83748d4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
@@ -104,7 +104,7 @@ public class MockTableOperations extends TableOperationsHelper {
       throw new TableExistsException(tableName, tableName, "");
 
     if (!namespaceExists(namespace)) {
-      throw new IllegalArgumentException("Table namespace (" + namespace + ") does not exist, create it first");
+      throw new IllegalArgumentException("Namespace (" + namespace + ") does not exist, create it first");
     }
     acu.createTable(username, tableName, versioningIter, timeType);
   }
@@ -156,9 +156,9 @@ public class MockTableOperations extends TableOperationsHelper {
       throw new TableExistsException(newTableName, newTableName, "");
     MockTable t = acu.tables.remove(oldTableName);
     String namespace = Tables.extractNamespace(newTableName);
-    MockTableNamespace n = acu.namespaces.get(namespace);
+    MockNamespace n = acu.namespaces.get(namespace);
     if (n == null) {
-      n = new MockTableNamespace();
+      n = new MockNamespace();
     }
     t.setNamespaceName(namespace);
     t.setNamespace(n);
@@ -185,7 +185,7 @@ public class MockTableOperations extends TableOperationsHelper {
     String namespace = Tables.extractNamespace(tableName);
 
     if (!namespaceExists(namespace)) {
-      throw new IllegalArgumentException("Table namespace (" + namespace + ") does not exist");
+      throw new IllegalArgumentException("Namespace (" + namespace + ") does not exist");
     }
 
     Set<Entry<String,String>> props = new HashSet<Entry<String,String>>(acu.namespaces.get(namespace).settings.entrySet());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/client/security/SecurityErrorCode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/security/SecurityErrorCode.java b/core/src/main/java/org/apache/accumulo/core/client/security/SecurityErrorCode.java
index f1ea539..30c60d5 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/security/SecurityErrorCode.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/security/SecurityErrorCode.java
@@ -38,5 +38,5 @@ public enum SecurityErrorCode {
   TOKEN_EXPIRED,
   SERIALIZATION_ERROR,
   INSUFFICIENT_PROPERTIES,
-  TABLE_NAMESPACE_DOESNT_EXIST;
+  NAMESPACE_DOESNT_EXIST;
 }


[48/50] [abbrv] git commit: ACCUMULO-324 Don't inherit iters/constraints in system namespace

Posted by ct...@apache.org.
ACCUMULO-324 Don't inherit iters/constraints in system namespace


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

Branch: refs/heads/master
Commit: 4a7c614ef8cd58aea02d55f0fd60b8654c388c2a
Parents: 5b791b1
Author: Christopher Tubbs <ct...@apache.org>
Authored: Thu Nov 14 15:05:18 2013 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:11 2013 -0500

----------------------------------------------------------------------
 .../accumulo/server/conf/TableNamespaceConfiguration.java    | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/4a7c614e/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
index 74621a3..888c5ed 100644
--- 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
@@ -57,8 +57,10 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
     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 (!isIterConst(property.getKey()))
+      if (!(namespaceId.equals(Constants.SYSTEM_TABLE_NAMESPACE_ID) && isIteratorOrConstraint(property.getKey()))) {
+        // ignore iterators from parent if system namespace
         value = parent.get(property);
+      }
     }
     return value;
   }
@@ -92,7 +94,7 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
 
     @Override
     public boolean accept(String key) {
-      if (isIterConst(key))
+      if (isIteratorOrConstraint(key))
         return false;
       return userFilter.accept(key);
     }
@@ -166,7 +168,7 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
       co.propertiesChanged();
   }
 
-  protected boolean isIterConst(String key) {
+  protected boolean isIteratorOrConstraint(String key) {
     return key.startsWith(Property.TABLE_ITERATOR_PREFIX.getKey()) || key.startsWith(Property.TABLE_CONSTRAINT_PREFIX.getKey());
   }
 }


[09/50] [abbrv] git commit: ACCUMULO-802 fixed problem where moving a table to a new namespace would still inherit its properties from the old namespace

Posted by ct...@apache.org.
ACCUMULO-802 fixed problem where moving a table to a new namespace would still inherit its properties from the old namespace


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

Branch: refs/heads/master
Commit: 88d44bc6f512ed7a1226fcad628557fee44d1b88
Parents: 08729d2
Author: Sean Hickey <ta...@gmail.com>
Authored: Wed Aug 7 11:28:26 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:10 2013 -0500

----------------------------------------------------------------------
 .../server/conf/ServerConfiguration.java        | 10 ++---
 .../server/conf/TableConfiguration.java         | 10 +++++
 .../conf/TableNamespaceConfiguration.java       |  6 +--
 .../accumulo/master/tableOps/RenameTable.java   |  3 --
 .../server/conf/TableParentConfiguration.java   | 41 ++++++++++++++++++++
 5 files changed, 59 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/88d44bc6/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java
index fe78ca4..5093025 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java
@@ -33,6 +33,7 @@ public class ServerConfiguration {
 
   private static final Map<String,TableConfiguration> tableInstances = new HashMap<String,TableConfiguration>(1);
   private static final Map<String,TableNamespaceConfiguration> tableNamespaceInstances = new HashMap<String,TableNamespaceConfiguration>(1);
+  private static final Map<String,TableNamespaceConfiguration> tableParentInstances = new HashMap<String,TableNamespaceConfiguration>(1);
   private static SecurityPermission CONFIGURATION_PERMISSION = new SecurityPermission("configurationPermission");
 
   public static synchronized SiteConfiguration getSiteConfiguration() {
@@ -63,13 +64,12 @@ public class ServerConfiguration {
 
   public static TableNamespaceConfiguration getTableNamespaceConfigurationForTable(Instance instance, String tableId) {
     checkPermissions();
-    String namespaceId = Tables.getNamespace(instance, tableId);
-    synchronized (tableNamespaceInstances) {
-      TableNamespaceConfiguration conf = tableNamespaceInstances.get(namespaceId);
+    synchronized (tableParentInstances) {
+      TableNamespaceConfiguration conf = tableParentInstances.get(tableId);
       if (conf == null) {
-        conf = new TableNamespaceConfiguration(namespaceId, getSystemConfiguration(instance));
+        conf = new TableParentConfiguration(tableId, getSystemConfiguration(instance));
         ConfigSanityCheck.validate(conf);
-        tableNamespaceInstances.put(namespaceId, conf);
+        tableParentInstances.put(tableId, conf);
       }
       return conf;
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/88d44bc6/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 2e595a9..530b2c2 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
@@ -142,7 +142,17 @@ public class TableConfiguration extends AccumuloConfiguration {
     return table;
   }
   
+  /** 
+   * returns the actual TableNamespaceConfiguration that corresponds to the current parent namespace.
+   */
   public TableNamespaceConfiguration getNamespaceConfiguration() {
+    return ServerConfiguration.getTableNamespaceConfiguration(parent.inst, parent.namespaceId);
+  }
+  
+  /**
+   * returns the parent, which is actually a TableParentConfiguration that can change which namespace it references
+   */
+  public TableNamespaceConfiguration getParentConfiguration() {
     return parent;
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/88d44bc6/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
index 60da78a..6c75e25 100644
--- 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
@@ -40,8 +40,8 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
 
   private final AccumuloConfiguration parent;
   private static ZooCache propCache = null;
-  private String namespaceId = null;
-  private Instance inst = null;
+  protected String namespaceId = null;
+  protected Instance inst = null;
   private Set<ConfigurationObserver> observers;
 
   public TableNamespaceConfiguration(String namespaceId, AccumuloConfiguration parent) {
@@ -103,7 +103,7 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
     return entries.entrySet().iterator();
   }
 
-  private String getNamespaceId() {
+  protected String getNamespaceId() {
     return namespaceId;
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/88d44bc6/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
index a58b847..18b7532 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
@@ -71,9 +71,6 @@ public class RenameTable extends MasterRepo {
     newTableName = Tables.extractTableName(newTableName);
     oldTableName = Tables.extractTableName(oldTableName);
 
-    // TODO ACCUMULO-802 renaming a table to a new namespace does not change it's parent configuration to be the new namespace
-    // ...it should...somehow...
-
     IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
 
     Utils.tableNameLock.lock();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/88d44bc6/server/src/main/java/org/apache/accumulo/server/conf/TableParentConfiguration.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/conf/TableParentConfiguration.java b/server/src/main/java/org/apache/accumulo/server/conf/TableParentConfiguration.java
new file mode 100644
index 0000000..7590d76
--- /dev/null
+++ b/server/src/main/java/org/apache/accumulo/server/conf/TableParentConfiguration.java
@@ -0,0 +1,41 @@
+/*
+ * 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.client.impl.Tables;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+
+
+/**
+ * Used by TableConfiguration to dynamically get the TableNamespaceConfiguration if the namespace changes
+ */
+public class TableParentConfiguration extends TableNamespaceConfiguration {
+
+  private String tableId;
+  
+  public TableParentConfiguration(String tableId, AccumuloConfiguration parent) {
+    super(null, parent);
+    this.tableId = tableId;
+    this.namespaceId = getNamespaceId();
+  }
+  
+  @Override
+  protected String getNamespaceId() {
+    this.namespaceId = Tables.getNamespace(inst, tableId); 
+    return this.namespaceId;
+  }
+}


[40/50] [abbrv] git commit: ACCUMULO-802 Fixup formatting

Posted by ct...@apache.org.
ACCUMULO-802 Fixup formatting


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

Branch: refs/heads/master
Commit: bbba3f5811c4774904c8cee88e0d1caec760044e
Parents: bf7c1db
Author: Christopher Tubbs <ct...@apache.org>
Authored: Thu Oct 31 21:50:40 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:11 2013 -0500

----------------------------------------------------------------------
 .../client/TableNamespaceExistsException.java   |  11 +-
 .../client/TableNamespaceNotEmptyException.java |  15 +--
 .../client/TableNamespaceNotFoundException.java |  15 +--
 .../client/admin/TableNamespaceOperations.java  |  52 +++++-----
 .../admin/TableNamespaceOperationsHelper.java   |  20 ++--
 .../core/client/impl/TableNamespaces.java       |  22 ++--
 .../core/client/mock/MockTableNamespace.java    |   7 +-
 .../mock/MockTableNamespaceOperations.java      |   9 +-
 .../core/security/TableNamespacePermission.java |  20 ++--
 .../shell/commands/CloneNamespaceCommand.java   |  31 +++---
 .../shell/commands/CreateNamespaceCommand.java  |  31 +++---
 .../shell/commands/DeleteNamespaceCommand.java  |  19 ++--
 .../commands/NamespacePermissionsCommand.java   |   4 +-
 .../util/shell/commands/NamespacesCommand.java  |  21 ++--
 .../shell/commands/RenameNamespaceCommand.java  |  14 +--
 .../client/mock/MockTableNamespacesTest.java    | 101 ++++++++++---------
 .../server/conf/TableNamespaceConfWatcher.java  |  16 +--
 .../conf/TableNamespaceConfiguration.java       |   4 +-
 .../server/conf/TableParentConfiguration.java   |   7 +-
 .../accumulo/server/util/NamespacePropUtil.java |  14 +--
 .../concurrent/CloneTableNamespace.java         |  10 +-
 .../concurrent/CreateTableNamespace.java        |  10 +-
 .../concurrent/DeleteTableNamespace.java        |  10 +-
 .../concurrent/OfflineTableNamespace.java       |  12 +--
 .../concurrent/RenameTableNamespace.java        |  10 +-
 25 files changed, 245 insertions(+), 240 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceExistsException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceExistsException.java b/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceExistsException.java
index ddddc4f..c6a8591 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceExistsException.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceExistsException.java
@@ -27,7 +27,7 @@ public class TableNamespaceExistsException extends Exception {
    * 
    */
   private static final long serialVersionUID = 1L;
-  
+
   /**
    * @param namespaceId
    *          the internal id of the table namespace that exists
@@ -37,10 +37,11 @@ public class TableNamespaceExistsException extends Exception {
    *          the specific reason why it failed
    */
   public TableNamespaceExistsException(String namespaceId, String namespaceName, String description) {
-    super("Table namespace" + (namespaceName != null && !namespaceName.isEmpty() ? " " + namespaceName : "") + (namespaceId != null && !namespaceId.isEmpty() ? " (Id=" + namespaceId + ")" : "")
-        + " exists" + (description != null && !description.isEmpty() ? " (" + description + ")" : ""));
+    super("Table namespace" + (namespaceName != null && !namespaceName.isEmpty() ? " " + namespaceName : "")
+        + (namespaceId != null && !namespaceId.isEmpty() ? " (Id=" + namespaceId + ")" : "") + " exists"
+        + (description != null && !description.isEmpty() ? " (" + description + ")" : ""));
   }
-  
+
   /**
    * @param namespaceId
    *          the internal id of the table namespace that exists
@@ -55,7 +56,7 @@ public class TableNamespaceExistsException extends Exception {
     this(namespaceId, namespaceName, description);
     super.initCause(cause);
   }
-  
+
   /**
    * @param e
    *          constructs an exception from a thrift exception

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotEmptyException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotEmptyException.java b/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotEmptyException.java
index e333693..04195e3 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotEmptyException.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotEmptyException.java
@@ -24,9 +24,9 @@ import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException
 public class TableNamespaceNotEmptyException extends Exception {
 
   private static final long serialVersionUID = 1L;
-  
+
   private String namespace;
-  
+
   /**
    * @param namespaceId
    *          the internal id of the table namespace
@@ -36,11 +36,12 @@ public class TableNamespaceNotEmptyException extends Exception {
    *          the specific reason why it failed
    */
   public TableNamespaceNotEmptyException(String namespaceId, String namespaceName, String description) {
-    super("Table namespace" + (namespaceName != null && !namespaceName.isEmpty() ? " " + namespaceName : "") + (namespaceId != null && !namespaceId.isEmpty() ? " (Id=" + namespaceId + ")" : "")
-        + " it not empty, contains at least one table" + (description != null && !description.isEmpty() ? " (" + description + ")" : ""));
+    super("Table namespace" + (namespaceName != null && !namespaceName.isEmpty() ? " " + namespaceName : "")
+        + (namespaceId != null && !namespaceId.isEmpty() ? " (Id=" + namespaceId + ")" : "") + " it not empty, contains at least one table"
+        + (description != null && !description.isEmpty() ? " (" + description + ")" : ""));
     this.namespace = namespaceName;
   }
-  
+
   /**
    * @param namespaceId
    *          the internal id of the table namespace
@@ -55,7 +56,7 @@ public class TableNamespaceNotEmptyException extends Exception {
     this(namespaceId, namespaceName, description);
     super.initCause(cause);
   }
-  
+
   /**
    * @param e
    *          constructs an exception from a thrift exception
@@ -63,7 +64,7 @@ public class TableNamespaceNotEmptyException extends Exception {
   public TableNamespaceNotEmptyException(ThriftTableOperationException e) {
     this(e.getTableId(), e.getTableName(), e.getDescription(), e);
   }
-  
+
   /**
    * @return the name of the table namespace
    */

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotFoundException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotFoundException.java b/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotFoundException.java
index 30e63c3..c4db89b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotFoundException.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/TableNamespaceNotFoundException.java
@@ -27,9 +27,9 @@ public class TableNamespaceNotFoundException extends Exception {
    * 
    */
   private static final long serialVersionUID = 1L;
-  
+
   private String namespace;
-  
+
   /**
    * @param namespaceId
    *          the internal id of the table namespace that was sought
@@ -39,11 +39,12 @@ public class TableNamespaceNotFoundException extends Exception {
    *          the specific reason why it failed
    */
   public TableNamespaceNotFoundException(String namespaceId, String namespaceName, String description) {
-    super("Table namespace" + (namespaceName != null && !namespaceName.isEmpty() ? " " + namespaceName : "") + (namespaceId != null && !namespaceId.isEmpty() ? " (Id=" + namespaceId + ")" : "")
-        + " does not exist" + (description != null && !description.isEmpty() ? " (" + description + ")" : ""));
+    super("Table namespace" + (namespaceName != null && !namespaceName.isEmpty() ? " " + namespaceName : "")
+        + (namespaceId != null && !namespaceId.isEmpty() ? " (Id=" + namespaceId + ")" : "") + " does not exist"
+        + (description != null && !description.isEmpty() ? " (" + description + ")" : ""));
     this.namespace = namespaceName;
   }
-  
+
   /**
    * @param namespaceId
    *          the internal id of the table namespace that was sought
@@ -58,7 +59,7 @@ public class TableNamespaceNotFoundException extends Exception {
     this(namespaceId, namespaceName, description);
     super.initCause(cause);
   }
-  
+
   /**
    * @param e
    *          constructs an exception from a thrift exception
@@ -66,7 +67,7 @@ public class TableNamespaceNotFoundException extends Exception {
   public TableNamespaceNotFoundException(ThriftTableOperationException e) {
     this(e.getTableId(), e.getTableName(), e.getDescription(), e);
   }
-  
+
   /**
    * @return the name of the table namespace sought
    */

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
index 136f79b..313a899 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
@@ -38,14 +38,14 @@ import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
  */
 
 public interface TableNamespaceOperations {
-  
+
   /**
    * Retrieve a list of table namespaces in Accumulo.
    * 
    * @return List of table namespaces in accumulo
    */
   public SortedSet<String> list();
-  
+
   /**
    * A method to check if a table namespace exists in Accumulo.
    * 
@@ -54,7 +54,7 @@ public interface TableNamespaceOperations {
    * @return true if the table namespace exists
    */
   public boolean exists(String namespace);
-  
+
   /**
    * Create a table namespace with no special configuration
    * 
@@ -68,7 +68,7 @@ public interface TableNamespaceOperations {
    *           if the table namespace already exists
    */
   public void create(String namespace) throws AccumuloException, AccumuloSecurityException, TableNamespaceExistsException;
-  
+
   /**
    * @param namespace
    *          the name of the table namespace
@@ -82,7 +82,7 @@ public interface TableNamespaceOperations {
    *           if the table namespace already exists
    */
   public void create(String namespace, boolean limitVersion) throws AccumuloException, AccumuloSecurityException, TableNamespaceExistsException;
-  
+
   /**
    * @param namespace
    *          the name of the table namespace
@@ -99,7 +99,7 @@ public interface TableNamespaceOperations {
    */
   public void create(String namespace, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException,
       TableNamespaceExistsException;
-  
+
   /**
    * Delete a table namespace if it is empty
    * 
@@ -118,7 +118,7 @@ public interface TableNamespaceOperations {
    */
   public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException, TableNamespaceNotEmptyException,
       TableNotFoundException;
-  
+
   /**
    * Delete a table namespace
    * 
@@ -139,7 +139,7 @@ public interface TableNamespaceOperations {
    */
   public void delete(String namespace, boolean deleteTables) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException,
       TableNamespaceNotEmptyException, TableNotFoundException;
-  
+
   /**
    * Rename a table namespace
    * 
@@ -158,7 +158,7 @@ public interface TableNamespaceOperations {
    */
   public void rename(String oldNamespaceName, String newNamespaceName) throws AccumuloSecurityException, TableNamespaceNotFoundException, AccumuloException,
       TableNamespaceExistsException;
-  
+
   /**
    * Sets a property on a table namespace which applies to all tables in the namespace. Note that it may take a short period of time (a second) to propagate the
    * change everywhere.
@@ -175,7 +175,7 @@ public interface TableNamespaceOperations {
    *           if the user does not have permission
    */
   public void setProperty(String namespace, String property, String value) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * Removes a property from a table namespace. Note that it may take a short period of time (a second) to propagate the change everywhere.
    * 
@@ -189,7 +189,7 @@ public interface TableNamespaceOperations {
    *           if the user does not have permission
    */
   public void removeProperty(String namespace, String property) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * Gets properties of a table namespace. Note that recently changed properties may not be available immediately.
    * 
@@ -200,7 +200,7 @@ public interface TableNamespaceOperations {
    *           if the table namespace does not exist
    */
   public Iterable<Entry<String,String>> getProperties(String namespace) throws AccumuloException, TableNamespaceNotFoundException;
-  
+
   /**
    * 
    * @param namespace
@@ -213,7 +213,7 @@ public interface TableNamespaceOperations {
    *           if the table namespace does not exist
    */
   public void offline(String namespace) throws AccumuloSecurityException, AccumuloException, TableNamespaceNotFoundException;
-  
+
   /**
    * 
    * @param namespace
@@ -226,14 +226,14 @@ public interface TableNamespaceOperations {
    *           if the table namespace does not exist
    */
   public void online(String namespace) throws AccumuloSecurityException, AccumuloException, TableNamespaceNotFoundException;
-  
+
   /**
    * Get a mapping of table namespace name to internal table namespace id.
    * 
    * @return the map from table namespace name to internal table namespace id
    */
   public Map<String,String> namespaceIdMap();
-  
+
   /**
    * Gets the number of bytes being used in the files for the set of tables in this namespace
    * 
@@ -247,7 +247,7 @@ public interface TableNamespaceOperations {
    *           when the user does not have the proper permissions
    */
   public List<DiskUsage> getDiskUsage(String namespace) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException;
-  
+
   /**
    * Clone a all the tables in a table namespace to a new table namespace. Optionally copy all their properties as well.
    * 
@@ -274,7 +274,7 @@ public interface TableNamespaceOperations {
    */
   public void clone(String srcName, String newName, boolean flush, Map<String,String> propertiesToSet, Set<String> propertiesToExclude, Boolean copyTableProps)
       throws AccumuloSecurityException, AccumuloException, TableNamespaceNotFoundException, TableNamespaceExistsException;
-  
+
   /**
    * Add an iterator to a table namespace on all scopes.
    * 
@@ -292,7 +292,7 @@ public interface TableNamespaceOperations {
    */
   public void attachIterator(String tableNamespace, IteratorSetting setting) throws AccumuloSecurityException, AccumuloException,
       TableNamespaceNotFoundException;
-  
+
   /**
    * Add an iterator to a table namespace on the given scopes.
    * 
@@ -310,7 +310,7 @@ public interface TableNamespaceOperations {
    */
   public void attachIterator(String tableNamespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException,
       AccumuloException, TableNamespaceNotFoundException;
-  
+
   /**
    * Remove an iterator from a table namespace by name.
    * 
@@ -328,7 +328,7 @@ public interface TableNamespaceOperations {
    */
   public void removeIterator(String tableNamespace, String name, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
       TableNamespaceNotFoundException;
-  
+
   /**
    * Get the settings for an iterator.
    * 
@@ -347,7 +347,7 @@ public interface TableNamespaceOperations {
    */
   public IteratorSetting getIteratorSetting(String tableNamespace, String name, IteratorScope scope) throws AccumuloSecurityException, AccumuloException,
       NumberFormatException, TableNamespaceNotFoundException;
-  
+
   /**
    * Get a list of iterators for this table namespace.
    * 
@@ -362,7 +362,7 @@ public interface TableNamespaceOperations {
    */
   public Map<String,EnumSet<IteratorScope>> listIterators(String tableNamespace) throws AccumuloSecurityException, AccumuloException,
       TableNamespaceNotFoundException;
-  
+
   /**
    * Check whether a given iterator configuration conflicts with existing configuration; in particular, determine if the name or priority are already in use for
    * the specified scopes.
@@ -379,7 +379,7 @@ public interface TableNamespaceOperations {
    */
   public void checkIteratorConflicts(String tableNamespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException,
       TableNamespaceNotFoundException;
-  
+
   /**
    * Add a new constraint to a table namespace.
    * 
@@ -397,7 +397,7 @@ public interface TableNamespaceOperations {
    */
   public int addConstraint(String tableNamespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException,
       TableNamespaceNotFoundException;
-  
+
   /**
    * Remove a constraint from a table namespace.
    * 
@@ -410,7 +410,7 @@ public interface TableNamespaceOperations {
    *           thrown if the user doesn't have permission to remove the constraint
    */
   public void removeConstraint(String tableNamespace, int number) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * List constraints on a table namespace with their assigned numbers.
    * 
@@ -423,7 +423,7 @@ public interface TableNamespaceOperations {
    *           thrown if the table namespace no longer exists
    */
   public Map<String,Integer> listConstraints(String tableNamespace) throws AccumuloException, TableNamespaceNotFoundException;
-  
+
   /**
    * Test to see if the instance can load the given class as the given type. This check uses the table classpath property if it is set.
    * 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsHelper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsHelper.java
index 7ba1497..c73d8e9 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsHelper.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsHelper.java
@@ -31,13 +31,13 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 
 public abstract class TableNamespaceOperationsHelper implements TableNamespaceOperations {
-  
+
   @Override
   public void attachIterator(String tableNamespace, IteratorSetting setting) throws AccumuloSecurityException, AccumuloException,
       TableNamespaceNotFoundException {
     attachIterator(tableNamespace, setting, EnumSet.allOf(IteratorScope.class));
   }
-  
+
   @Override
   public void attachIterator(String tableNamespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException,
       AccumuloException, TableNamespaceNotFoundException {
@@ -50,7 +50,7 @@ public abstract class TableNamespaceOperationsHelper implements TableNamespaceOp
       this.setProperty(tableNamespace, root, setting.getPriority() + "," + setting.getIteratorClass());
     }
   }
-  
+
   @Override
   public void removeIterator(String tableNamespace, String name, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
       TableNamespaceNotFoundException {
@@ -68,7 +68,7 @@ public abstract class TableNamespaceOperationsHelper implements TableNamespaceOp
       }
     }
   }
-  
+
   @Override
   public IteratorSetting getIteratorSetting(String tableNamespace, String name, IteratorScope scope) throws AccumuloSecurityException, AccumuloException,
       TableNamespaceNotFoundException {
@@ -77,7 +77,7 @@ public abstract class TableNamespaceOperationsHelper implements TableNamespaceOp
     int priority = -1;
     String classname = null;
     Map<String,String> settings = new HashMap<String,String>();
-    
+
     String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), name);
     String opt = root + ".opt.";
     for (Entry<String,String> property : this.getProperties(tableNamespace)) {
@@ -97,7 +97,7 @@ public abstract class TableNamespaceOperationsHelper implements TableNamespaceOp
     }
     return new IteratorSetting(priority, name, classname, settings);
   }
-  
+
   @Override
   public Map<String,EnumSet<IteratorScope>> listIterators(String tableNamespace) throws AccumuloSecurityException, AccumuloException,
       TableNamespaceNotFoundException {
@@ -118,7 +118,7 @@ public abstract class TableNamespaceOperationsHelper implements TableNamespaceOp
     }
     return result;
   }
-  
+
   @Override
   public void checkIteratorConflicts(String tableNamespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloException,
       TableNamespaceNotFoundException {
@@ -153,7 +153,7 @@ public abstract class TableNamespaceOperationsHelper implements TableNamespaceOp
         throw new AccumuloException(new IllegalArgumentException("iterator options conflict for " + setting.getName() + ": " + optionConflicts));
     }
   }
-  
+
   @Override
   public int addConstraint(String tableNamespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException,
       TableNamespaceNotFoundException {
@@ -180,12 +180,12 @@ public abstract class TableNamespaceOperationsHelper implements TableNamespaceOp
     this.setProperty(tableNamespace, Property.TABLE_CONSTRAINT_PREFIX.toString() + i, constraintClassName);
     return i;
   }
-  
+
   @Override
   public void removeConstraint(String tableNamespace, int number) throws AccumuloException, AccumuloSecurityException {
     this.removeProperty(tableNamespace, Property.TABLE_CONSTRAINT_PREFIX.toString() + number);
   }
-  
+
   @Override
   public Map<String,Integer> listConstraints(String tableNamespace) throws AccumuloException, TableNamespaceNotFoundException {
     Map<String,Integer> constraints = new TreeMap<String,Integer>();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/core/src/main/java/org/apache/accumulo/core/client/impl/TableNamespaces.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TableNamespaces.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TableNamespaces.java
index e3de9db..935cf16 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TableNamespaces.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TableNamespaces.java
@@ -30,7 +30,7 @@ import org.apache.accumulo.fate.zookeeper.ZooCache;
 
 public class TableNamespaces {
   private static SecurityPermission TABLES_PERMISSION = new SecurityPermission("tablesPermission");
-  
+
   private static ZooCache getZooCache(Instance instance) {
     SecurityManager sm = System.getSecurityManager();
     if (sm != null) {
@@ -38,14 +38,14 @@ public class TableNamespaces {
     }
     return ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
   }
-  
+
   private static SortedMap<String,String> getMap(Instance instance, boolean nameAsKey) {
     ZooCache zc = getZooCache(instance);
-    
+
     List<String> namespaceIds = zc.getChildren(ZooUtil.getRoot(instance) + Constants.ZNAMESPACES);
-    
+
     TreeMap<String,String> namespaceMap = new TreeMap<String,String>();
-    
+
     for (String id : namespaceIds) {
       byte[] path = zc.get(ZooUtil.getRoot(instance) + Constants.ZNAMESPACES + "/" + id + Constants.ZNAMESPACE_NAME);
       if (path != null) {
@@ -57,29 +57,29 @@ public class TableNamespaces {
     }
     return namespaceMap;
   }
-  
+
   public static String getNamespaceId(Instance instance, String namespace) throws TableNamespaceNotFoundException {
     String id = getNameToIdMap(instance).get(namespace);
     if (id == null)
       throw new TableNamespaceNotFoundException(null, namespace, "getNamespaceId() failed to find namespace");
     return id;
   }
-  
+
   public static String getNamespaceName(Instance instance, String namespaceId) throws TableNamespaceNotFoundException {
     String namespaceName = getIdToNameMap(instance).get(namespaceId);
     if (namespaceName == null)
       throw new TableNamespaceNotFoundException(namespaceId, null, "getNamespaceName() failed to find namespace");
     return namespaceName;
   }
-  
+
   public static SortedMap<String,String> getNameToIdMap(Instance instance) {
     return getMap(instance, true);
   }
-  
+
   public static SortedMap<String,String> getIdToNameMap(Instance instance) {
     return getMap(instance, false);
   }
-  
+
   public static List<String> getTableIds(Instance instance, String namespaceId) throws TableNamespaceNotFoundException {
     List<String> l = new LinkedList<String>();
     for (String id : Tables.getIdToNameMap(instance).keySet()) {
@@ -89,7 +89,7 @@ public class TableNamespaces {
     }
     return l;
   }
-  
+
   public static List<String> getTableNames(Instance instance, String namespaceId) throws TableNamespaceNotFoundException {
     ZooCache zc = getZooCache(instance);
     List<String> ids = getTableIds(instance, namespaceId);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java
index 1798dd1..8523145 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java
@@ -27,12 +27,11 @@ import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.security.TableNamespacePermission;
 
-
 public class MockTableNamespace {
-  
+
   final Map<String,String> settings;
   Map<String,EnumSet<TableNamespacePermission>> userPermissions = new HashMap<String,EnumSet<TableNamespacePermission>>();
-  
+
   public MockTableNamespace() {
     settings = new HashMap<String,String>();
     for (Entry<String,String> entry : AccumuloConfiguration.getDefaultConfiguration()) {
@@ -42,7 +41,7 @@ public class MockTableNamespace {
       }
     }
   }
-  
+
   public List<String> getTables(MockAccumulo acu) {
     List<String> l = new LinkedList<String>();
     for (String t : acu.tables.keySet()) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java
index 5b53fbd..28fff21 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java
@@ -39,8 +39,6 @@ import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
 import org.apache.commons.lang.NotImplementedException;
 
-
-
 public class MockTableNamespaceOperations extends TableNamespaceOperationsHelper {
 
   final private MockAccumulo acu;
@@ -88,7 +86,8 @@ public class MockTableNamespaceOperations extends TableNamespaceOperationsHelper
   }
 
   @Override
-  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException, TableNamespaceNotEmptyException, TableNotFoundException {
+  public void delete(String namespace) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException, TableNamespaceNotEmptyException,
+      TableNotFoundException {
     delete(namespace, false);
   }
 
@@ -182,11 +181,11 @@ public class MockTableNamespaceOperations extends TableNamespaceOperationsHelper
     // TODO Implement clone in Mock
     throw new NotImplementedException();
   }
-  
+
   @Override
   public boolean testClassLoad(String namespace, String className, String asTypeName) throws AccumuloException, AccumuloSecurityException,
       TableNamespaceNotFoundException {
-    
+
     try {
       AccumuloVFSClassLoader.loadClass(className, Class.forName(asTypeName));
     } catch (ClassNotFoundException e) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/core/src/main/java/org/apache/accumulo/core/security/TableNamespacePermission.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/security/TableNamespacePermission.java b/core/src/main/java/org/apache/accumulo/core/security/TableNamespacePermission.java
index 9354b02..2cf57c7 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/TableNamespacePermission.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/TableNamespacePermission.java
@@ -28,39 +28,39 @@ public enum TableNamespacePermission {
   ALTER_TABLE((byte) 4),
   CREATE_TABLE((byte) 5),
   DROP_TABLE((byte) 6);
-  
+
   final private byte permID;
-  
+
   final private static TableNamespacePermission mapping[] = new TableNamespacePermission[8];
   static {
     for (TableNamespacePermission perm : TableNamespacePermission.values())
       mapping[perm.permID] = perm;
   }
-  
+
   private TableNamespacePermission(byte id) {
     this.permID = id;
   }
-  
+
   public byte getId() {
     return this.permID;
   }
-  
+
   public static List<String> printableValues() {
     TableNamespacePermission[] a = TableNamespacePermission.values();
-    
+
     List<String> list = new ArrayList<String>(a.length);
-    
+
     for (TableNamespacePermission p : a)
       list.add("Namespace." + p);
-    
+
     return list;
   }
-  
+
   public static TableNamespacePermission getPermissionById(byte id) {
     TableNamespacePermission result = mapping[id];
     if (result != null)
       return result;
     throw new IndexOutOfBoundsException("No such permission");
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CloneNamespaceCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CloneNamespaceCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CloneNamespaceCommand.java
index 21f5a9f..dade389 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CloneNamespaceCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CloneNamespaceCommand.java
@@ -35,21 +35,21 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
 
 public class CloneNamespaceCommand extends Command {
-  
+
   private Option setPropsOption;
   private Option excludePropsOption;
   private Option noFlushOption;
   private Option copyTablePropsOption;
-  
+
   @Override
-  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException, TableNotFoundException,
-      TableExistsException, TableNamespaceNotFoundException, TableNamespaceExistsException {
-    
+  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException,
+      TableNotFoundException, TableExistsException, TableNamespaceNotFoundException, TableNamespaceExistsException {
+
     final HashMap<String,String> props = new HashMap<String,String>();
     final HashSet<String> exclude = new HashSet<String>();
     boolean flush = true;
     boolean copyTableProps = false;
-    
+
     if (cl.hasOption(setPropsOption.getOpt())) {
       String[] keyVals = cl.getOptionValue(setPropsOption.getOpt()).split(",");
       for (String keyVal : keyVals) {
@@ -57,40 +57,41 @@ public class CloneNamespaceCommand extends Command {
         props.put(sa[0], sa[1]);
       }
     }
-    
+
     if (cl.hasOption(excludePropsOption.getOpt())) {
       String[] keys = cl.getOptionValue(excludePropsOption.getOpt()).split(",");
       for (String key : keys) {
         exclude.add(key);
       }
     }
-    
+
     if (cl.hasOption(noFlushOption.getOpt())) {
       flush = false;
     }
-    
+
     if (cl.hasOption(noFlushOption.getOpt())) {
       copyTableProps = true;
     }
-    
+
     shellState.getConnector().tableNamespaceOperations().clone(cl.getArgs()[0], cl.getArgs()[1], flush, props, exclude, copyTableProps);
     return 0;
   }
-  
+
   @Override
   public String usage() {
     return getName() + " <current name> <new name>";
   }
-  
+
   @Override
   public String description() {
     return "clones a table namespace";
   }
-  
+
+  @Override
   public void registerCompletion(final Token root, final Map<Command.CompletionSet,Set<String>> completionSet) {
     registerCompletionForTableNamespaces(root, completionSet);
   }
-  
+
   @Override
   public Options getOptions() {
     final Options o = new Options();
@@ -104,7 +105,7 @@ public class CloneNamespaceCommand extends Command {
     o.addOption(copyTablePropsOption);
     return o;
   }
-  
+
   @Override
   public int numArgs() {
     return 2;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateNamespaceCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateNamespaceCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateNamespaceCommand.java
index a869070..7434767 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateNamespaceCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateNamespaceCommand.java
@@ -36,18 +36,19 @@ import org.apache.commons.cli.Options;
 public class CreateNamespaceCommand extends Command {
   private Option createTableOptCopyConfig, createTableNamespaceOptCopyConfig;
   private Option base64Opt;
-  
+
+  @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException,
       TableExistsException, TableNotFoundException, IOException, ClassNotFoundException, TableNamespaceExistsException, TableNamespaceNotFoundException {
-    
+
     if (createTableOptCopyConfig == null) {
       getOptions();
     }
-    
+
     String namespace = cl.getArgs()[0];
-    
+
     shellState.getConnector().tableNamespaceOperations().create(namespace);
-    
+
     // Copy options if flag was set
     Iterable<Entry<String,String>> configuration = null;
     if (cl.hasOption(createTableNamespaceOptCopyConfig.getOpt())) {
@@ -68,41 +69,41 @@ public class CreateNamespaceCommand extends Command {
         }
       }
     }
-    
+
     return 0;
   }
-  
+
   @Override
   public String description() {
     return "creates a new table namespace";
   }
-  
+
   @Override
   public String usage() {
     return getName() + " <namespaceName>";
   }
-  
+
   @Override
   public Options getOptions() {
     final Options o = new Options();
-    
+
     createTableNamespaceOptCopyConfig = new Option("cc", "copy-config", true, "table namespace to copy configuration from");
     createTableNamespaceOptCopyConfig.setArgName("tableNamespace");
-    
+
     createTableOptCopyConfig = new Option("ctc", "copy-table-config", true, "table to copy configuration from");
     createTableOptCopyConfig.setArgName("tableName");
-    
+
     base64Opt = new Option("b64", "base64encoded", false, "decode encoded split points");
     o.addOption(base64Opt);
     OptionGroup ogp = new OptionGroup();
     ogp.addOption(createTableOptCopyConfig);
     ogp.addOption(createTableNamespaceOptCopyConfig);
-    
+
     o.addOptionGroup(ogp);
-    
+
     return o;
   }
-  
+
   @Override
   public int numArgs() {
     return 1;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteNamespaceCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteNamespaceCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteNamespaceCommand.java
index 27d669b..e1b41cd 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteNamespaceCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteNamespaceCommand.java
@@ -24,15 +24,16 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
 import org.apache.accumulo.core.client.impl.TableNamespaces;
 import org.apache.accumulo.core.util.shell.Shell;
-import org.apache.accumulo.core.util.shell.Token;
 import org.apache.accumulo.core.util.shell.Shell.Command;
+import org.apache.accumulo.core.util.shell.Token;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
 
 public class DeleteNamespaceCommand extends Command {
   private Option forceOpt;
-  
+
+  @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws Exception {
     boolean force = false;
     boolean operate = true;
@@ -40,7 +41,7 @@ public class DeleteNamespaceCommand extends Command {
       force = true;
     }
     String namespace = cl.getArgs()[0];
-    
+
     if (!force) {
       shellState.getReader().flush();
       String line = shellState.getReader().readLine(getName() + " { " + namespace + " } (yes|no)? ");
@@ -51,19 +52,19 @@ public class DeleteNamespaceCommand extends Command {
     }
     return 0;
   }
-  
+
   @Override
   public String description() {
     return "deletes a table namespace";
   }
-  
+
   protected void doTableOp(final Shell shellState, final String namespace, boolean force) throws Exception {
     boolean resetContext = false;
     String currentTable = shellState.getTableName();
     if (!TableNamespaces.getNameToIdMap(shellState.getInstance()).containsKey(namespace)) {
       throw new TableNamespaceNotFoundException(null, namespace, null);
     }
-    
+
     String namespaceId = TableNamespaces.getNamespaceId(shellState.getInstance(), namespace);
     List<String> tables = TableNamespaces.getTableNames(shellState.getInstance(), namespaceId);
     resetContext = tables.contains(currentTable);
@@ -82,12 +83,12 @@ public class DeleteNamespaceCommand extends Command {
       shellState.setTableName("");
     }
   }
-  
+
   @Override
   public Options getOptions() {
     forceOpt = new Option("f", "force", false, "force deletion without prompting");
     final Options opts = super.getOptions();
-    
+
     opts.addOption(forceOpt);
     return opts;
   }
@@ -96,7 +97,7 @@ public class DeleteNamespaceCommand extends Command {
   public int numArgs() {
     return 1;
   }
-  
+
   @Override
   public void registerCompletion(final Token root, final Map<Command.CompletionSet,Set<String>> special) {
     registerCompletionForTableNamespaces(root, special);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacePermissionsCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacePermissionsCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacePermissionsCommand.java
index 822522e..f3e0e6f 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacePermissionsCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacePermissionsCommand.java
@@ -31,12 +31,12 @@ public class NamespacePermissionsCommand extends Command {
     }
     return 0;
   }
-  
+
   @Override
   public String description() {
     return "displays a list of valid table namespace permissions";
   }
-  
+
   @Override
   public int numArgs() {
     return 0;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java
index ad30699..2056afd 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java
@@ -18,8 +18,8 @@ package org.apache.accumulo.core.util.shell.commands;
 
 import java.io.IOException;
 import java.util.Iterator;
-import java.util.TreeMap;
 import java.util.Map.Entry;
+import java.util.TreeMap;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -32,13 +32,14 @@ import org.apache.commons.collections.iterators.AbstractIteratorDecorator;
 
 public class NamespacesCommand extends Command {
   private Option disablePaginationOpt, namespaceIdOption;
-  
+
   @SuppressWarnings("unchecked")
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException, IOException {
-      Iterator<String> names = shellState.getConnector().tableNamespaceOperations().list().iterator();
-      Iterator<String> ids = new NamespaceIdIterator(new TreeMap<String,String>(shellState.getConnector().tableNamespaceOperations().namespaceIdMap()).entrySet().iterator());
-    
+    Iterator<String> names = shellState.getConnector().tableNamespaceOperations().list().iterator();
+    Iterator<String> ids = new NamespaceIdIterator(new TreeMap<String,String>(shellState.getConnector().tableNamespaceOperations().namespaceIdMap()).entrySet()
+        .iterator());
+
     if (cl.hasOption(namespaceIdOption.getOpt())) {
       shellState.printLines(ids, !cl.hasOption(disablePaginationOpt.getOpt()));
     } else {
@@ -46,7 +47,7 @@ public class NamespacesCommand extends Command {
     }
     return 0;
   }
-  
+
   /**
    * Decorator that formats the id and name for display.
    */
@@ -54,7 +55,7 @@ public class NamespacesCommand extends Command {
     public NamespaceIdIterator(Iterator<Entry<String,String>> iterator) {
       super(iterator);
     }
-    
+
     @SuppressWarnings("rawtypes")
     @Override
     public Object next() {
@@ -62,12 +63,12 @@ public class NamespacesCommand extends Command {
       return String.format("%-15s => %10s%n", entry.getKey(), entry.getValue());
     }
   }
-  
+
   @Override
   public String description() {
     return "displays a list of all existing table namespaces";
   }
-  
+
   @Override
   public Options getOptions() {
     final Options o = new Options();
@@ -77,7 +78,7 @@ public class NamespacesCommand extends Command {
     o.addOption(disablePaginationOpt);
     return o;
   }
-  
+
   @Override
   public int numArgs() {
     return 0;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameNamespaceCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameNamespaceCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameNamespaceCommand.java
index 9f4596a..34fd9fa 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameNamespaceCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameNamespaceCommand.java
@@ -47,31 +47,31 @@ public class RenameNamespaceCommand extends Command {
       currentTableId = Tables.getTableId(shellState.getInstance(), shellState.getTableName());
       resetContext = tableIds.contains(currentTableId);
     }
-    
+
     shellState.getConnector().tableNamespaceOperations().rename(old, newer);
-    
+
     if (resetContext) {
       shellState.setTableName(Tables.getTableName(shellState.getInstance(), currentTableId));
     }
-    
+
     return 0;
   }
-  
+
   @Override
   public String usage() {
     return getName() + " <current table namespace> <new table namespace>";
   }
-  
+
   @Override
   public String description() {
     return "renames a table namespace";
   }
-  
+
   @Override
   public void registerCompletion(final Token root, final Map<Command.CompletionSet,Set<String>> special) {
     registerCompletionForTableNamespaces(root, special);
   }
-  
+
   @Override
   public int numArgs() {
     return 2;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableNamespacesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableNamespacesTest.java b/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableNamespacesTest.java
index 2ba1006..2cf0541 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableNamespacesTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableNamespacesTest.java
@@ -20,9 +20,9 @@ package org.apache.accumulo.core.client.mock;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import java.util.Map.Entry;
 import java.util.EnumSet;
 import java.util.HashSet;
+import java.util.Map.Entry;
 import java.util.Random;
 
 import org.apache.accumulo.core.Constants;
@@ -48,10 +48,10 @@ import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 
 public class MockTableNamespacesTest {
-  
+
   Random random = new Random();
   public static TemporaryFolder folder = new TemporaryFolder();
-  
+
   /**
    * This test creates a table without specifying a namespace. In this case, it puts the table into the default namespace.
    * 
@@ -62,12 +62,12 @@ public class MockTableNamespacesTest {
     String tableName = "test";
     Instance instance = new MockInstance("default");
     Connector c = instance.getConnector("user", new PasswordToken("pass"));
-    
+
     assertTrue(c.tableNamespaceOperations().exists(Constants.DEFAULT_TABLE_NAMESPACE));
     c.tableOperations().create(tableName);
     assertTrue(c.tableOperations().exists(tableName));
   }
-  
+
   /**
    * This test creates a new namespace "testing" and a table "testing.table1" which puts "table1" into the "testing" namespace. Then we create "testing.table2"
    * which creates "table2" and puts it into "testing" as well. Then we make sure that you can't delete a namespace with tables in it, and then we delete the
@@ -80,19 +80,19 @@ public class MockTableNamespacesTest {
     String namespace = "testing";
     String tableName1 = namespace + ".table1";
     String tableName2 = namespace + ".table2";
-    
+
     Instance instance = new MockInstance("createdelete");
     Connector c = instance.getConnector("user", new PasswordToken("pass"));
-    
+
     c.tableNamespaceOperations().create(namespace);
     assertTrue(c.tableNamespaceOperations().exists(namespace));
-    
+
     c.tableOperations().create(tableName1);
     assertTrue(c.tableOperations().exists(tableName1));
-    
+
     c.tableOperations().create(tableName2);
     assertTrue(c.tableOperations().exists(tableName2));
-    
+
     // deleting
     try {
       // can't delete a namespace with tables in it
@@ -104,17 +104,17 @@ public class MockTableNamespacesTest {
     assertTrue(c.tableNamespaceOperations().exists(namespace));
     assertTrue(c.tableOperations().exists(tableName1));
     assertTrue(c.tableOperations().exists(tableName2));
-    
+
     c.tableOperations().delete(tableName2);
     assertTrue(!c.tableOperations().exists(tableName2));
     assertTrue(c.tableNamespaceOperations().exists(namespace));
-    
+
     c.tableOperations().delete(tableName1);
     assertTrue(!c.tableOperations().exists(tableName1));
     c.tableNamespaceOperations().delete(namespace);
     assertTrue(!c.tableNamespaceOperations().exists(namespace));
   }
-  
+
   /**
    * This test creates a namespace, modifies it's properties, and checks to make sure that those properties are applied to its tables. To do something on a
    * namespace-wide level, use TableNamespaceOperations.
@@ -125,72 +125,72 @@ public class MockTableNamespacesTest {
    * 
    * @throws Exception
    */
-  
+
   @Test
   public void testNamespaceProperties() throws Exception {
     String namespace = "propchange";
     String tableName1 = namespace + ".table1";
     String tableName2 = namespace + ".table2";
-    
+
     String propKey = Property.TABLE_SCAN_MAXMEM.getKey();
     String propVal = "42K";
-    
+
     Instance instance = new MockInstance("props");
     Connector c = instance.getConnector("user", new PasswordToken("pass"));
-    
+
     c.tableNamespaceOperations().create(namespace);
     c.tableOperations().create(tableName1);
     c.tableNamespaceOperations().setProperty(namespace, propKey, propVal);
-    
+
     // check the namespace has the property
     assertTrue(checkTableNamespaceHasProp(c, namespace, propKey, propVal));
-    
+
     // check that the table gets it from the namespace
     assertTrue(checkTableHasProp(c, tableName1, propKey, propVal));
-    
+
     // test a second table to be sure the first wasn't magical
     // (also, changed the order, the namespace has the property already)
     c.tableOperations().create(tableName2);
     assertTrue(checkTableHasProp(c, tableName2, propKey, propVal));
-    
+
     // test that table properties override namespace properties
     String propKey2 = Property.TABLE_FILE_MAX.getKey();
     String propVal2 = "42";
     String tablePropVal = "13";
-    
+
     c.tableOperations().setProperty(tableName2, propKey2, tablePropVal);
     c.tableNamespaceOperations().setProperty("propchange", propKey2, propVal2);
-    
+
     assertTrue(checkTableHasProp(c, tableName2, propKey2, tablePropVal));
-    
+
     // now check that you can change the default namespace's properties
     propVal = "13K";
     String tableName = "some_table";
     c.tableOperations().create(tableName);
     c.tableNamespaceOperations().setProperty(Constants.DEFAULT_TABLE_NAMESPACE, propKey, propVal);
-    
+
     assertTrue(checkTableHasProp(c, tableName, propKey, propVal));
-    
+
     // test the properties server-side by configuring an iterator.
     // should not show anything with column-family = 'a'
     String tableName3 = namespace + ".table3";
     c.tableOperations().create(tableName3);
-    
+
     IteratorSetting setting = new IteratorSetting(250, "thing", SimpleFilter.class.getName());
     c.tableNamespaceOperations().attachIterator(namespace, setting);
-    
+
     BatchWriter bw = c.createBatchWriter(tableName3, new BatchWriterConfig());
     Mutation m = new Mutation("r");
     m.put("a", "b", new Value("abcde".getBytes()));
     bw.addMutation(m);
     bw.flush();
     bw.close();
-    
+
     // Scanner s = c.createScanner(tableName3, Authorizations.EMPTY);
     // do scanners work correctly in mock?
     // assertTrue(!s.iterator().hasNext());
   }
-  
+
   /**
    * This test renames and clones two separate table into different namespaces. different namespace.
    * 
@@ -203,19 +203,19 @@ public class MockTableNamespacesTest {
     String tableName = "table";
     String tableName1 = "renamed.table1";
     // String tableName2 = "cloned.table2";
-    
+
     Instance instance = new MockInstance("renameclone");
     Connector c = instance.getConnector("user", new PasswordToken("pass"));
-    
+
     c.tableOperations().create(tableName);
     c.tableNamespaceOperations().create(namespace1);
     c.tableNamespaceOperations().create(namespace2);
-    
+
     c.tableOperations().rename(tableName, tableName1);
-    
+
     assertTrue(c.tableOperations().exists(tableName1));
     assertTrue(!c.tableOperations().exists(tableName));
-    
+
     // TODO implement clone in mock
     /*
      * c.tableOperations().clone(tableName1, tableName2, false, null, null);
@@ -224,7 +224,7 @@ public class MockTableNamespacesTest {
      */
     return;
   }
-  
+
   /**
    * This test renames a table namespace and ensures that its tables are still correct
    */
@@ -233,21 +233,21 @@ public class MockTableNamespacesTest {
     String namespace1 = "n1";
     String namespace2 = "n2";
     String table = "t";
-    
+
     Instance instance = new MockInstance("rename");
     Connector c = instance.getConnector("user", new PasswordToken("pass"));
-    
+
     c.tableNamespaceOperations().create(namespace1);
     c.tableOperations().create(namespace1 + "." + table);
-    
+
     c.tableNamespaceOperations().rename(namespace1, namespace2);
-    
+
     assertTrue(!c.tableNamespaceOperations().exists(namespace1));
     assertTrue(c.tableNamespaceOperations().exists(namespace2));
     assertTrue(!c.tableOperations().exists(namespace1 + "." + table));
     assertTrue(c.tableOperations().exists(namespace2 + "." + table));
   }
-  
+
   /**
    * This tests adding iterators to a namespace, listing them, and removing them
    */
@@ -255,34 +255,34 @@ public class MockTableNamespacesTest {
   public void testNamespaceIterators() throws Exception {
     Instance instance = new MockInstance("Iterators");
     Connector c = instance.getConnector("user", new PasswordToken("pass"));
-    
+
     String namespace = "iterator";
     String tableName = namespace + ".table";
     String iter = "thing";
-    
+
     c.tableNamespaceOperations().create(namespace);
     c.tableOperations().create(tableName);
-    
+
     IteratorSetting setting = new IteratorSetting(250, iter, SimpleFilter.class.getName());
     HashSet<IteratorScope> scope = new HashSet<IteratorScope>();
     scope.add(IteratorScope.scan);
     c.tableNamespaceOperations().attachIterator(namespace, setting, EnumSet.copyOf(scope));
-    
+
     BatchWriter bw = c.createBatchWriter(tableName, new BatchWriterConfig());
     Mutation m = new Mutation("r");
     m.put("a", "b", new Value("abcde".getBytes(Constants.UTF8)));
     bw.addMutation(m);
     bw.flush();
-    
+
     Scanner s = c.createScanner(tableName, Authorizations.EMPTY);
     System.out.println(s.iterator().next());
     // do scanners work correctly in mock?
     // assertTrue(!s.iterator().hasNext());
-    
+
     assertTrue(c.tableNamespaceOperations().listIterators(namespace).containsKey(iter));
     c.tableNamespaceOperations().removeIterator(namespace, iter, EnumSet.copyOf(scope));
   }
-  
+
   private boolean checkTableHasProp(Connector c, String t, String propKey, String propVal) throws AccumuloException, TableNotFoundException {
     for (Entry<String,String> e : c.tableOperations().getProperties(t)) {
       if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) {
@@ -291,7 +291,7 @@ public class MockTableNamespacesTest {
     }
     return false;
   }
-  
+
   private boolean checkTableNamespaceHasProp(Connector c, String n, String propKey, String propVal) throws AccumuloException, TableNamespaceNotFoundException {
     for (Entry<String,String> e : c.tableNamespaceOperations().getProperties(n)) {
       if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) {
@@ -300,8 +300,9 @@ public class MockTableNamespacesTest {
     }
     return false;
   }
-  
+
   public static class SimpleFilter extends Filter {
+    @Override
     public boolean accept(Key k, Value v) {
       if (k.getColumnFamily().toString().equals("a"))
         return false;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/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
index 8bb33b8..b331940 100644
--- 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
@@ -29,25 +29,25 @@ class TableNamespaceConfWatcher implements Watcher {
     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());
@@ -57,13 +57,13 @@ class TableNamespaceConfWatcher implements Watcher {
             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())

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/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
index 2ebe338..b4e9598 100644
--- 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
@@ -43,7 +43,7 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
   protected String namespaceId = null;
   protected Instance inst = null;
   private Set<ConfigurationObserver> observers;
-  
+
   public TableNamespaceConfiguration(String namespaceId, AccumuloConfiguration parent) {
     inst = HdfsZooInstance.getInstance();
     this.parent = parent;
@@ -152,7 +152,7 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
     for (ConfigurationObserver co : copy)
       co.propertiesChanged();
   }
-  
+
   protected boolean isIterConst(Entry<String,String> e) {
     if (e.getKey().startsWith(Property.TABLE_ITERATOR_PREFIX.getKey()) || e.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.getKey())) {
       return true;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/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 f8d8411..8608a3e 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
@@ -19,22 +19,21 @@ package org.apache.accumulo.server.conf;
 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
  */
 public class TableParentConfiguration extends TableNamespaceConfiguration {
 
   private String tableId;
-  
+
   public TableParentConfiguration(String tableId, AccumuloConfiguration parent) {
     super(null, parent);
     this.tableId = tableId;
     this.namespaceId = getNamespaceId();
   }
-  
+
   @Override
   protected String getNamespaceId() {
-    return Tables.getNamespace(inst, tableId); 
+    return Tables.getNamespace(inst, tableId);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/server/base/src/main/java/org/apache/accumulo/server/util/NamespacePropUtil.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/NamespacePropUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/NamespacePropUtil.java
index 61ba133..4e5df9e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/NamespacePropUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/NamespacePropUtil.java
@@ -29,31 +29,31 @@ public class NamespacePropUtil {
   public static boolean setNamespaceProperty(String namespaceId, String property, String value) throws KeeperException, InterruptedException {
     if (!isPropertyValid(property, value))
       return false;
-    
+
     // create the zk node for per-namespace properties for this namespace if it doesn't already exist
     String zkNamespacePath = getPath(namespaceId);
     ZooReaderWriter.getInstance().putPersistentData(zkNamespacePath, new byte[0], NodeExistsPolicy.SKIP);
-    
+
     // create the zk node for this property and set it's data to the specified value
     String zPath = zkNamespacePath + "/" + property;
     ZooReaderWriter.getInstance().putPersistentData(zPath, value.getBytes(Constants.UTF8), NodeExistsPolicy.OVERWRITE);
-    
+
     return true;
   }
-  
+
   public static boolean isPropertyValid(String property, String value) {
     Property p = Property.getPropertyByKey(property);
     if ((p != null && !p.getType().isValidFormat(value)) || !Property.isValidTablePropertyKey(property))
       return false;
-    
+
     return true;
   }
-  
+
   public static void removeNamespaceProperty(String namespaceId, String property) throws InterruptedException, KeeperException {
     String zPath = getPath(namespaceId) + "/" + property;
     ZooReaderWriter.getInstance().recursiveDelete(zPath, NodeMissingPolicy.SKIP);
   }
-  
+
   private static String getPath(String namespaceId) {
     return ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZNAMESPACES + "/" + namespaceId + Constants.ZNAMESPACE_CONF;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CloneTableNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CloneTableNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CloneTableNamespace.java
index c6a923f..2c35173 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CloneTableNamespace.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CloneTableNamespace.java
@@ -29,20 +29,20 @@ import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
 public class CloneTableNamespace extends Test {
-  
+
   @Override
   public void visit(State state, Properties props) throws Exception {
     Connector conn = state.getConnector();
-    
+
     Random rand = (Random) state.get("rand");
-    
+
     @SuppressWarnings("unchecked")
     List<String> namespaces = (List<String>) state.get("namespaces");
-    
+
     String srcName = namespaces.get(rand.nextInt(namespaces.size()));
     String newName = namespaces.get(rand.nextInt(namespaces.size()));
     boolean flush = rand.nextBoolean();
-    
+
     try {
       log.debug("Cloning table namespace " + srcName + " " + newName + " " + flush);
       conn.tableNamespaceOperations().clone(srcName, newName, flush, new HashMap<String,String>(), new HashSet<String>(), true);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTableNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTableNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTableNamespace.java
index 749fa0f..a65043d 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTableNamespace.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTableNamespace.java
@@ -26,18 +26,18 @@ import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
 public class CreateTableNamespace extends Test {
-  
+
   @Override
   public void visit(State state, Properties props) throws Exception {
     Connector conn = state.getConnector();
-    
+
     Random rand = (Random) state.get("rand");
-    
+
     @SuppressWarnings("unchecked")
     List<String> namespaces = (List<String>) state.get("namespaces");
-    
+
     String namespace = namespaces.get(rand.nextInt(namespaces.size()));
-    
+
     try {
       conn.tableNamespaceOperations().create(namespace);
       log.debug("Created namespace " + namespace);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteTableNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteTableNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteTableNamespace.java
index 16ad555..ca6e39c 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteTableNamespace.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteTableNamespace.java
@@ -26,18 +26,18 @@ import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
 public class DeleteTableNamespace extends Test {
-  
+
   @Override
   public void visit(State state, Properties props) throws Exception {
     Connector conn = state.getConnector();
-    
+
     Random rand = (Random) state.get("rand");
-    
+
     @SuppressWarnings("unchecked")
     List<String> namespaces = (List<String>) state.get("namespaces");
-    
+
     String namespace = namespaces.get(rand.nextInt(namespaces.size()));
-    
+
     try {
       conn.tableNamespaceOperations().delete(namespace, true);
       log.debug("Deleted table namespace " + namespace);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTableNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTableNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTableNamespace.java
index c895d04..4ee2417 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTableNamespace.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTableNamespace.java
@@ -27,18 +27,18 @@ import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
 public class OfflineTableNamespace extends Test {
-  
+
   @Override
   public void visit(State state, Properties props) throws Exception {
     Connector conn = state.getConnector();
-    
+
     Random rand = (Random) state.get("rand");
-    
+
     @SuppressWarnings("unchecked")
     List<String> namespaces = (List<String>) state.get("namespaces");
-    
+
     String namespace = namespaces.get(rand.nextInt(namespaces.size()));
-    
+
     try {
       conn.tableNamespaceOperations().offline(namespace);
       log.debug("Offlined namespace " + namespace);
@@ -48,6 +48,6 @@ public class OfflineTableNamespace extends Test {
     } catch (TableNamespaceNotFoundException tne) {
       log.debug("offline or online failed " + namespace + ", doesnt exist");
     }
-    
+
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bbba3f58/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java
index fdd4cef..15c7c00 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java
@@ -27,19 +27,19 @@ import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
 public class RenameTableNamespace extends Test {
-  
+
   @Override
   public void visit(State state, Properties props) throws Exception {
     Connector conn = state.getConnector();
-    
+
     Random rand = (Random) state.get("rand");
-    
+
     @SuppressWarnings("unchecked")
     List<String> namespaces = (List<String>) state.get("namespaces");
-    
+
     String srcName = namespaces.get(rand.nextInt(namespaces.size()));
     String newName = namespaces.get(rand.nextInt(namespaces.size()));
-    
+
     try {
       conn.tableNamespaceOperations().rename(srcName, newName);
       log.debug("Renamed table namespace " + srcName + " " + newName);


[45/50] [abbrv] git commit: ACCUMULO-1906 Drop clone namespace feature

Posted by ct...@apache.org.
ACCUMULO-1906 Drop clone namespace feature


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

Branch: refs/heads/master
Commit: 71bed4df6bd76578449b27b52c65fa4fc8222f63
Parents: 4a7c614
Author: Christopher Tubbs <ct...@apache.org>
Authored: Thu Nov 14 16:03:01 2013 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:11 2013 -0500

----------------------------------------------------------------------
 .../client/admin/TableNamespaceOperations.java  |  28 --
 .../admin/TableNamespaceOperationsImpl.java     |  67 ----
 .../mock/MockTableNamespaceOperations.java      |   9 -
 .../apache/accumulo/core/util/shell/Shell.java  | 302 +++++++++----------
 .../shell/commands/CloneNamespaceCommand.java   | 113 -------
 .../java/org/apache/accumulo/master/Master.java |  31 +-
 .../master/tableOps/CloneTableNamespace.java    | 201 ------------
 .../concurrent/CloneTableNamespace.java         |  55 ----
 .../apache/accumulo/test/TableNamespacesIT.java | 253 +++++++---------
 .../randomwalk/conf/modules/Concurrent.xml      |   5 -
 10 files changed, 256 insertions(+), 808 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/71bed4df/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
index 313a899..d966f3c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
@@ -20,7 +20,6 @@ import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Set;
 import java.util.SortedSet;
 
 import org.apache.accumulo.core.client.AccumuloException;
@@ -249,33 +248,6 @@ public interface TableNamespaceOperations {
   public List<DiskUsage> getDiskUsage(String namespace) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException;
 
   /**
-   * Clone a all the tables in a table namespace to a new table namespace. Optionally copy all their properties as well.
-   * 
-   * @param srcName
-   *          The table namespace to clone
-   * @param newName
-   *          The new table namespace to clone to
-   * @param flush
-   *          Whether to flush each table before cloning
-   * @param propertiesToSet
-   *          Which table namespace properties to set
-   * @param propertiesToExclude
-   *          Which table namespace properties to exclude
-   * @param copyTableProps
-   *          Whether to copy each table's properties
-   * @throws AccumuloSecurityException
-   *           when the user does not have the proper permissions
-   * @throws AccumuloException
-   *           when there is a general accumulo error
-   * @throws TableNamespaceNotFoundException
-   *           If the old table namespace doesn't exist
-   * @throws TableNamespaceExistsException
-   *           If the new table namespace already exists
-   */
-  public void clone(String srcName, String newName, boolean flush, Map<String,String> propertiesToSet, Set<String> propertiesToExclude, Boolean copyTableProps)
-      throws AccumuloSecurityException, AccumuloException, TableNamespaceNotFoundException, TableNamespaceExistsException;
-
-  /**
    * Add an iterator to a table namespace on all scopes.
    * 
    * @param tableNamespace

http://git-wip-us.apache.org/repos/asf/accumulo/blob/71bed4df/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
index 602110b..a7d0c34 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
@@ -18,7 +18,6 @@ package org.apache.accumulo.core.client.admin;
 
 import java.nio.ByteBuffer;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -36,7 +35,6 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNamespaceExistsException;
 import org.apache.accumulo.core.client.TableNamespaceNotEmptyException;
 import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
@@ -362,71 +360,6 @@ public class TableNamespaceOperationsImpl extends TableNamespaceOperationsHelper
   }
 
   /**
-   * Clone a all the tables in a table namespace to a new table namespace. Optionally copy all their properties as well.
-   * 
-   * @param srcName
-   *          The table namespace to clone
-   * @param newName
-   *          The new table namespace to clone to
-   * @param flush
-   *          Whether to flush each table before cloning
-   * @param propertiesToSet
-   *          Which table namespace properties to set
-   * @param propertiesToExclude
-   *          Which table namespace properties to exclude
-   * @param copyTableProps
-   *          Whether to copy each table's properties
-   * @throws AccumuloSecurityException
-   *           when the user does not have the proper permissions
-   * @throws AccumuloException
-   *           when there is a general accumulo error
-   * @throws TableNamespaceNotFoundException
-   *           If the old table namespace doesn't exist
-   * @throws TableNamespaceExistsException
-   *           If the new table namespace already exists
-   */
-  @Override
-  public void clone(String srcName, String newName, boolean flush, Map<String,String> propertiesToSet, Set<String> propertiesToExclude, Boolean copyTableProps)
-      throws AccumuloSecurityException, AccumuloException, TableNamespaceNotFoundException, TableNamespaceExistsException {
-
-    ArgumentChecker.notNull(srcName, newName);
-
-    String namespaceId = TableNamespaces.getNamespaceId(instance, srcName);
-
-    if (propertiesToExclude == null)
-      propertiesToExclude = Collections.emptySet();
-
-    if (propertiesToSet == null)
-      propertiesToSet = Collections.emptyMap();
-
-    if (!Collections.disjoint(propertiesToExclude, propertiesToSet.keySet()))
-      throw new IllegalArgumentException("propertiesToSet and propertiesToExclude not disjoint");
-
-    String srcNamespaceId = TableNamespaces.getNamespaceId(instance, srcName);
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(srcNamespaceId.getBytes()), ByteBuffer.wrap(newName.getBytes()));
-    Map<String,String> opts = new HashMap<String,String>();
-    opts.putAll(propertiesToSet);
-    for (String prop : propertiesToExclude)
-      opts.put(prop, "");
-    doTableNamespaceOperation(TableOperation.CLONE, args, opts);
-
-    for (String tableId : TableNamespaces.getTableIds(instance, namespaceId)) {
-      try {
-        String tableName = Tables.getTableName(instance, tableId);
-
-        String newTableName = newName + "." + Tables.extractTableName(tableName);
-        getTableOperations().clone(tableName, newTableName, flush, null, null);
-      } catch (TableNotFoundException e) {
-        String why = "Table (" + tableId + ") dissappeared while cloning namespace (" + srcName + ")";
-        throw new IllegalStateException(why);
-      } catch (TableExistsException e) {
-        String why = "Table somehow already existed in the newly created namespace (" + newName + ")";
-        throw new IllegalStateException(why);
-      }
-    }
-  }
-
-  /**
    * Rename a table namespace
    * 
    * @param oldNamespaceName

http://git-wip-us.apache.org/repos/asf/accumulo/blob/71bed4df/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java
index 28fff21..c04c522 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespaceOperations.java
@@ -21,7 +21,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
@@ -37,7 +36,6 @@ import org.apache.accumulo.core.client.admin.TableNamespaceOperationsHelper;
 import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
-import org.apache.commons.lang.NotImplementedException;
 
 public class MockTableNamespaceOperations extends TableNamespaceOperationsHelper {
 
@@ -176,13 +174,6 @@ public class MockTableNamespaceOperations extends TableNamespaceOperationsHelper
   }
 
   @Override
-  public void clone(String srcName, String newName, boolean flush, Map<String,String> propertiesToSet, Set<String> propertiesToExclude, Boolean copyTableProps)
-      throws AccumuloSecurityException, AccumuloException, TableNamespaceNotFoundException, TableNamespaceExistsException {
-    // TODO Implement clone in Mock
-    throw new NotImplementedException();
-  }
-
-  @Override
   public boolean testClassLoad(String namespace, String className, String asTypeName) throws AccumuloException, AccumuloSecurityException,
       TableNamespaceNotFoundException {
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/71bed4df/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java b/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
index c19e84a..7d13d5f 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
@@ -73,7 +73,6 @@ import org.apache.accumulo.core.util.shell.commands.AuthenticateCommand;
 import org.apache.accumulo.core.util.shell.commands.ByeCommand;
 import org.apache.accumulo.core.util.shell.commands.ClasspathCommand;
 import org.apache.accumulo.core.util.shell.commands.ClearCommand;
-import org.apache.accumulo.core.util.shell.commands.CloneNamespaceCommand;
 import org.apache.accumulo.core.util.shell.commands.CloneTableCommand;
 import org.apache.accumulo.core.util.shell.commands.ClsCommand;
 import org.apache.accumulo.core.util.shell.commands.CompactCommand;
@@ -174,14 +173,14 @@ import com.beust.jcommander.ParameterException;
 public class Shell extends ShellOptions {
   public static final Logger log = Logger.getLogger(Shell.class);
   private static final Logger audit = Logger.getLogger(Shell.class.getName() + ".audit");
-  
+
   public static final String CHARSET = "ISO-8859-1";
   public static final int NO_FIXED_ARG_LENGTH_CHECK = -1;
   public static final String COMMENT_PREFIX = "#";
   public static final String HISTORY_DIR_NAME = ".accumulo";
   public static final String HISTORY_FILE_NAME = "shell_history.txt";
   private static final String SHELL_DESCRIPTION = "Shell - Apache Accumulo Interactive Shell";
-  
+
   protected int exitCode = 0;
   private String tableName;
   protected Instance instance;
@@ -193,21 +192,21 @@ public class Shell extends ShellOptions {
   private final Class<? extends Formatter> binaryFormatterClass = BinaryFormatter.class;
   public Map<String,List<IteratorSetting>> scanIteratorOptions = new HashMap<String,List<IteratorSetting>>();
   public Map<String,List<IteratorSetting>> iteratorProfiles = new HashMap<String,List<IteratorSetting>>();
-  
+
   private Token rootToken;
   public final Map<String,Command> commandFactory = new TreeMap<String,Command>();
   public final Map<String,Command[]> commandGrouping = new TreeMap<String,Command[]>();
   protected boolean configError = false;
-  
+
   // exit if true
   private boolean exit = false;
-  
+
   // file to execute commands from
   protected File execFile = null;
   // single command to execute from the command line
   protected String execCommand = null;
   protected boolean verbose = true;
-  
+
   private boolean tabCompletion;
   private boolean disableAuthTimeout;
   private long authTimeout;
@@ -215,26 +214,26 @@ public class Shell extends ShellOptions {
   private boolean logErrorsToConsole = false;
   private PrintWriter writer = null;
   private boolean masking = false;
-  
+
   public Shell() throws IOException {
     this(new ConsoleReader());
   }
-  
+
   public Shell(ConsoleReader reader) {
     super();
     this.reader = reader;
   }
-  
+
   public Shell(ConsoleReader reader, PrintWriter writer) {
     this(reader);
     this.writer = writer;
   }
-  
+
   // Not for client use
   public boolean config(String... args) {
     ShellOptionsJC options = new ShellOptionsJC();
     JCommander jc = new JCommander();
-    
+
     jc.setProgramName("accumulo shell");
     jc.addObject(options);
     try {
@@ -242,47 +241,47 @@ public class Shell extends ShellOptions {
     } catch (ParameterException e) {
       configError = true;
     }
-    
+
     if (options.isHelpEnabled()) {
       configError = true;
     }
-    
+
     if (!configError && options.getUnrecognizedOptions() != null) {
       configError = true;
       logError("Unrecognized Options: " + options.getUnrecognizedOptions().toString());
     }
-    
+
     if (configError) {
       jc.usage();
       return true;
     }
-    
+
     setDebugging(options.isDebugEnabled());
     authTimeout = options.getAuthTimeout() * 60 * 1000; // convert minutes to milliseconds
     disableAuthTimeout = options.isAuthTimeoutDisabled();
-    
+
     // get the options that were parsed
     String user = options.getUsername();
     String password = options.getPassword();
-    
+
     tabCompletion = !options.isTabCompletionDisabled();
-    
+
     // Use a fake (Mock), ZK, or HdfsZK Accumulo instance
     setInstance(options);
-    
+
     // AuthenticationToken options
     token = options.getAuthenticationToken();
     Map<String,String> loginOptions = options.getTokenProperties();
-    
+
     // process default parameters if unspecified
     try {
       boolean hasToken = (token != null);
       boolean hasTokenOptions = !loginOptions.isEmpty();
-      
+
       if (hasToken && password != null) {
         throw new ParameterException("Can not supply '--pass' option with '--tokenClass' option");
       }
-      
+
       Runtime.getRuntime().addShutdownHook(new Thread() {
         @Override
         public void run() {
@@ -298,7 +297,7 @@ public class Shell extends ShellOptions {
         org.apache.accumulo.core.client.security.tokens.AuthenticationToken.Properties props;
         // and line wrap it because the package name is so long
         props = new org.apache.accumulo.core.client.security.tokens.AuthenticationToken.Properties();
-        
+
         props.putAllStrings(loginOptions);
         token.init(props);
       } else {
@@ -306,7 +305,7 @@ public class Shell extends ShellOptions {
         if ("stdin".equals(password) || password == null) {
           password = reader.readLine("Password: ", '*');
         }
-        
+
         if (password == null) {
           // User cancel, e.g. Ctrl-D pressed
           throw new ParameterException("No password or token option supplied");
@@ -314,21 +313,21 @@ public class Shell extends ShellOptions {
           this.token = new PasswordToken(password);
         }
       }
-      
+
       if (!options.isFake()) {
         ZooReader zr = new ZooReader(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
         DistributedTrace.enable(instance, zr, "shell", InetAddress.getLocalHost().getHostName());
       }
-      
+
       this.setTableName("");
       this.principal = user;
       connector = instance.getConnector(this.principal, token);
-      
+
     } catch (Exception e) {
       printException(e);
       configError = true;
     }
-    
+
     // decide whether to execute commands from a file and quit
     if (options.getExecFile() != null) {
       execFile = options.getExecFile();
@@ -341,9 +340,9 @@ public class Shell extends ShellOptions {
     if (execCommand != null) {
       verbose = false;
     }
-    
+
     rootToken = new Token();
-    
+
     Command[] dataCommands = {new DeleteCommand(), new DeleteManyCommand(), new DeleteRowsCommand(), new EGrepCommand(), new FormatterCommand(),
         new InterpreterCommand(), new GrepCommand(), new ImportDirectoryCommand(), new InsertCommand(), new MaxRowCommand(), new ScanCommand()};
     Command[] debuggingCommands = {new ClasspathCommand(), new DebugCommand(), new ListScansCommand(), new ListCompactionsCommand(), new TraceCommand(),
@@ -360,7 +359,7 @@ public class Shell extends ShellOptions {
         new TableCommand(), new UserCommand(), new WhoAmICommand()};
     Command[] tableCommands = {new CloneTableCommand(), new ConfigCommand(), new CreateTableCommand(), new DeleteTableCommand(), new DropTableCommand(),
         new DUCommand(), new ExportTableCommand(), new ImportTableCommand(), new OfflineCommand(), new OnlineCommand(), new RenameTableCommand(),
-        new TablesCommand(), new NamespacesCommand(), new CreateNamespaceCommand(), new DeleteNamespaceCommand(), new RenameNamespaceCommand(), new CloneNamespaceCommand()};
+        new TablesCommand(), new NamespacesCommand(), new CreateNamespaceCommand(), new DeleteNamespaceCommand(), new RenameNamespaceCommand()};
     Command[] tableControlCommands = {new AddSplitsCommand(), new CompactCommand(), new ConstraintCommand(), new FlushCommand(), new GetGroupsCommand(),
         new GetSplitsCommand(), new MergeCommand(), new SetGroupsCommand()};
     Command[] userCommands = {new AddAuthsCommand(), new CreateUserCommand(), new DeleteUserCommand(), new DropUserCommand(), new GetAuthsCommand(),
@@ -376,7 +375,7 @@ public class Shell extends ShellOptions {
     commandGrouping.put("-- Table Administration Commands --------", tableCommands);
     commandGrouping.put("-- Table Control Commands ---------------", tableControlCommands);
     commandGrouping.put("-- User Administration Commands ---------", userCommands);
-    
+
     for (Command[] cmds : commandGrouping.values()) {
       for (Command cmd : cmds)
         commandFactory.put(cmd.getName(), cmd);
@@ -386,11 +385,12 @@ public class Shell extends ShellOptions {
     }
     return configError;
   }
-  
+
   /**
    * Sets the instance used by the shell based on the given options.
-   *
-   * @param options shell options
+   * 
+   * @param options
+   *          shell options
    */
   protected void setInstance(ShellOptionsJC options) {
     // should only be one set of instance options set
@@ -416,7 +416,7 @@ public class Shell extends ShellOptions {
       }
     }
   }
-  
+
   /*
    * Takes instanceName and keepers as separate arguments, rather than just packaged into the clientConfig,
    * so that we can fail over to accumulo-site.xml or HDFS config if they're unspecified.
@@ -443,30 +443,30 @@ public class Shell extends ShellOptions {
       return new ZooKeeperInstance(clientConfig.withInstance(instanceName).withZkHosts(keepers));
     }
   }
-  
+
   public Connector getConnector() {
     return connector;
   }
-  
+
   public Instance getInstance() {
     return instance;
   }
-  
+
   public static void main(String args[]) throws IOException {
     Shell shell = new Shell();
     shell.config(args);
-    
+
     System.exit(shell.start());
   }
-  
+
   public int start() throws IOException {
     if (configError)
       return 1;
-    
+
     String input;
     if (isVerbose())
       printInfo();
-    
+
     String home = System.getProperty("HOME");
     if (home == null)
       home = System.getenv("HOME");
@@ -492,15 +492,15 @@ public class Shell extends ShellOptions {
     } catch (IOException e) {
       log.warn("Unable to load history file at " + historyPath);
     }
-    
+
     // This would be a nice feature but !METADATA screws it up
     reader.setExpandEvents(false);
-    
+
     // Turn Ctrl+C into Exception instead of JVM exit
     reader.setHandleUserInterrupt(true);
-    
+
     ShellCompletor userCompletor = null;
-    
+
     if (execFile != null) {
       java.util.Scanner scanner = new java.util.Scanner(execFile);
       try {
@@ -516,33 +516,33 @@ public class Shell extends ShellOptions {
       }
       return exitCode;
     }
-    
+
     while (true) {
       try {
         if (hasExited())
           return exitCode;
-      
+
         // If tab completion is true we need to reset
         if (tabCompletion) {
           if (userCompletor != null)
             reader.removeCompleter(userCompletor);
-          
+
           userCompletor = setupCompletion();
           reader.addCompleter(userCompletor);
         }
-      
+
         reader.setPrompt(getDefaultPrompt());
         input = reader.readLine();
         if (input == null) {
           reader.println();
           return exitCode;
         } // User Canceled (Ctrl+D)
-      
+
         execCommand(input, disableAuthTimeout, false);
       } catch (UserInterruptException uie) {
         // User Cancelled (Ctrl+C)
         reader.println();
-        
+
         String partialLine = uie.getPartialLine();
         if (partialLine == null || "".equals(uie.getPartialLine().trim())) {
           // No content, actually exit
@@ -553,14 +553,14 @@ public class Shell extends ShellOptions {
       }
     }
   }
-  
+
   public void printInfo() throws IOException {
     reader.print("\n" + SHELL_DESCRIPTION + "\n" + "- \n" + "- version: " + Constants.VERSION + "\n" + "- instance name: "
         + connector.getInstance().getInstanceName() + "\n" + "- instance id: " + connector.getInstance().getInstanceID() + "\n" + "- \n"
         + "- type 'help' for a list of available commands\n" + "- \n");
     reader.flush();
   }
-  
+
   public void printVerboseInfo() throws IOException {
     StringBuilder sb = new StringBuilder("-\n");
     sb.append("- Current user: ").append(connector.whoami()).append("\n");
@@ -587,22 +587,22 @@ public class Shell extends ShellOptions {
     sb.append("-\n");
     reader.print(sb.toString());
   }
-  
+
   public String getDefaultPrompt() {
     return connector.whoami() + "@" + connector.getInstance().getInstanceName() + (getTableName().isEmpty() ? "" : " ") + getTableName() + "> ";
   }
-  
+
   public void execCommand(String input, boolean ignoreAuthTimeout, boolean echoPrompt) throws IOException {
     audit.log(Level.INFO, getDefaultPrompt() + input);
     if (echoPrompt) {
       reader.print(getDefaultPrompt());
       reader.println(input);
     }
-    
+
     if (input.startsWith(COMMENT_PREFIX)) {
       return;
     }
-    
+
     String fields[];
     try {
       fields = new QuotedStringTokenizer(input).getTokens();
@@ -613,10 +613,10 @@ public class Shell extends ShellOptions {
     }
     if (fields.length == 0)
       return;
-    
+
     String command = fields[0];
     fields = fields.length > 1 ? Arrays.copyOfRange(fields, 1, fields.length) : new String[] {};
-    
+
     Command sc = null;
     if (command.length() > 0) {
       try {
@@ -627,7 +627,7 @@ public class Shell extends ShellOptions {
           reader.flush();
           return;
         }
-        
+
         if (!(sc instanceof ExitCommand) && !ignoreAuthTimeout && System.currentTimeMillis() - lastUserActivity > authTimeout) {
           reader.println("Shell has been idle for too long. Please re-authenticate.");
           boolean authFailed = true;
@@ -637,26 +637,26 @@ public class Shell extends ShellOptions {
               reader.println();
               return;
             } // user canceled
-            
+
             try {
               authFailed = !connector.securityOperations().authenticateUser(connector.whoami(), new PasswordToken(pwd));
             } catch (Exception e) {
               ++exitCode;
               printException(e);
             }
-            
+
             if (authFailed)
               reader.print("Invalid password. ");
           } while (authFailed);
           lastUserActivity = System.currentTimeMillis();
         }
-        
+
         // Get the options from the command on how to parse the string
         Options parseOpts = sc.getOptionsWithHelp();
-        
+
         // Parse the string using the given options
         CommandLine cl = new BasicParser().parse(parseOpts, fields);
-        
+
         int actualArgLen = cl.getArgs().length;
         int expectedArgLen = sc.numArgs();
         if (cl.hasOption(helpOption)) {
@@ -675,7 +675,7 @@ public class Shell extends ShellOptions {
           exitCode += tmpCode;
           reader.flush();
         }
-        
+
       } catch (ConstraintViolationException e) {
         ++exitCode;
         printConstraintViolationException(e);
@@ -703,14 +703,14 @@ public class Shell extends ShellOptions {
     }
     reader.flush();
   }
-  
+
   /**
    * The command tree is built in reverse so that the references are more easily linked up. There is some code in token to allow forward building of the command
    * tree.
    */
   private ShellCompletor setupCompletion() {
     rootToken = new Token();
-    
+
     Set<String> tableNames = null;
     try {
       tableNames = connector.tableOperations().list();
@@ -718,7 +718,7 @@ public class Shell extends ShellOptions {
       log.debug("Unable to obtain list of tables", e);
       tableNames = Collections.emptySet();
     }
-    
+
     Set<String> userlist = null;
     try {
       userlist = connector.securityOperations().listLocalUsers();
@@ -726,7 +726,7 @@ public class Shell extends ShellOptions {
       log.debug("Unable to obtain list of users", e);
       userlist = Collections.emptySet();
     }
-    
+
     Set<String> tableNamespaces = null;
     try {
       tableNamespaces = connector.tableNamespaceOperations().list();
@@ -734,29 +734,29 @@ public class Shell extends ShellOptions {
       log.debug("Unable to obtain list of table namespaces", e);
       tableNamespaces = Collections.emptySet();
     }
-    
+
     Map<Command.CompletionSet,Set<String>> options = new HashMap<Command.CompletionSet,Set<String>>();
-    
+
     Set<String> commands = new HashSet<String>();
     for (String a : commandFactory.keySet())
       commands.add(a);
-    
+
     Set<String> modifiedUserlist = new HashSet<String>();
     Set<String> modifiedTablenames = new HashSet<String>();
     Set<String> modifiedTableNamespaces = new HashSet<String>();
-    
+
     for (String a : tableNames)
       modifiedTablenames.add(a.replaceAll("([\\s'\"])", "\\\\$1"));
     for (String a : userlist)
       modifiedUserlist.add(a.replaceAll("([\\s'\"])", "\\\\$1"));
     for (String a : tableNamespaces)
       modifiedTableNamespaces.add(a.replaceAll("([\\s'\"])", "\\\\$1"));
-    
+
     options.put(Command.CompletionSet.USERNAMES, modifiedUserlist);
     options.put(Command.CompletionSet.TABLENAMES, modifiedTablenames);
     options.put(Command.CompletionSet.TABLENAMESPACES, modifiedTableNamespaces);
     options.put(Command.CompletionSet.COMMANDS, commands);
-    
+
     for (Command[] cmdGroup : commandGrouping.values()) {
       for (Command c : cmdGroup) {
         c.getOptionsWithHelp(); // prep the options for the command
@@ -767,7 +767,7 @@ public class Shell extends ShellOptions {
     }
     return new ShellCompletor(rootToken, options);
   }
-  
+
   /**
    * The Command class represents a command to be run in the shell. It contains the methods to execute along with some methods to help tab completion, and
    * return the command name, help, and usage.
@@ -777,62 +777,62 @@ public class Shell extends ShellOptions {
     public enum CompletionSet {
       TABLENAMES, USERNAMES, COMMANDS, TABLENAMESPACES
     }
-    
+
     static Set<String> getCommandNames(Map<CompletionSet,Set<String>> objects) {
       return objects.get(CompletionSet.COMMANDS);
     }
-    
+
     static Set<String> getTableNames(Map<CompletionSet,Set<String>> objects) {
       return objects.get(CompletionSet.TABLENAMES);
     }
-    
+
     static Set<String> getUserNames(Map<CompletionSet,Set<String>> objects) {
       return objects.get(CompletionSet.USERNAMES);
     }
-    
+
     static Set<String> getTableNamespaces(Map<CompletionSet,Set<String>> objects) {
       return objects.get(CompletionSet.TABLENAMESPACES);
     }
-    
+
     public void registerCompletionGeneral(Token root, Set<String> args, boolean caseSens) {
       Token t = new Token(args);
       t.setCaseSensitive(caseSens);
-      
+
       Token command = new Token(getName());
       command.addSubcommand(t);
-      
+
       root.addSubcommand(command);
     }
-    
+
     public void registerCompletionForTables(Token root, Map<CompletionSet,Set<String>> completionSet) {
       registerCompletionGeneral(root, completionSet.get(CompletionSet.TABLENAMES), true);
     }
-    
+
     public void registerCompletionForUsers(Token root, Map<CompletionSet,Set<String>> completionSet) {
       registerCompletionGeneral(root, completionSet.get(CompletionSet.USERNAMES), true);
     }
-    
+
     public void registerCompletionForCommands(Token root, Map<CompletionSet,Set<String>> completionSet) {
       registerCompletionGeneral(root, completionSet.get(CompletionSet.COMMANDS), false);
     }
-    
+
     public void registerCompletionForTableNamespaces(Token root, Map<CompletionSet,Set<String>> completionSet) {
       registerCompletionGeneral(root, completionSet.get(CompletionSet.TABLENAMESPACES), true);
     }
-    
+
     // abstract methods to override
     public abstract int execute(String fullCommand, CommandLine cl, Shell shellState) throws Exception;
-    
+
     public abstract String description();
-    
+
     /**
      * If the number of arguments is not always zero (not including those arguments handled through Options), make sure to override the {@link #usage()} method.
      * Otherwise, {@link #usage()} does need to be overridden.
      */
     public abstract int numArgs();
-    
+
     // OPTIONAL methods to override:
-    
+
     // the general version of getname uses reflection to get the class name
     // and then cuts off the suffix -Command to get the name of the command
     public String getName() {
@@ -841,54 +841,54 @@ public class Shell extends ShellOptions {
       int i = s.indexOf("Command");
       return i > 0 ? s.substring(st + 1, i).toLowerCase(Locale.ENGLISH) : null;
     }
-    
+
     // The general version of this method adds the name
     // of the command to the completion tree
     public void registerCompletion(Token root, Map<CompletionSet,Set<String>> completion_set) {
       root.addSubcommand(new Token(getName()));
     }
-    
+
     // The general version of this method uses the HelpFormatter
     // that comes with the apache Options package to print out the help
     public final void printHelp(Shell shellState) {
       shellState.printHelp(usage(), "description: " + this.description(), getOptionsWithHelp());
     }
-    
+
     public final void printHelp(Shell shellState, int width) {
       shellState.printHelp(usage(), "description: " + this.description(), getOptionsWithHelp(), width);
     }
-    
+
     // Get options with help
     public final Options getOptionsWithHelp() {
       Options opts = getOptions();
       opts.addOption(new Option(helpOption, helpLongOption, false, "display this help"));
       return opts;
     }
-    
+
     // General usage is just the command
     public String usage() {
       return getName();
     }
-    
+
     // General Options are empty
     public Options getOptions() {
       return new Options();
     }
   }
-  
+
   public interface PrintLine {
     public void print(String s);
-    
+
     public void close();
   }
-  
+
   public static class PrintShell implements PrintLine {
     ConsoleReader reader;
-    
+
     public PrintShell(ConsoleReader reader) {
       this.reader = reader;
     }
-    
+
     @Override
     public void print(String s) {
       try {
@@ -897,40 +897,40 @@ public class Shell extends ShellOptions {
         throw new RuntimeException(ex);
       }
     }
-    
+
     @Override
     public void close() {}
   };
-  
+
   public static class PrintFile implements PrintLine {
     PrintWriter writer;
-    
+
     public PrintFile(String filename) throws FileNotFoundException {
       writer = new PrintWriter(filename);
     }
-    
+
     @Override
     public void print(String s) {
       writer.println(s);
     }
-    
+
     @Override
     public void close() {
       writer.close();
     }
   };
-  
+
   public final void printLines(Iterator<String> lines, boolean paginate) throws IOException {
     printLines(lines, paginate, null);
   }
-  
+
   public final void printLines(Iterator<String> lines, boolean paginate, PrintLine out) throws IOException {
     int linesPrinted = 0;
     String prompt = "-- hit any key to continue or 'q' to quit --";
     int lastPromptLength = prompt.length();
     int termWidth = reader.getTerminal().getWidth();
     int maxLines = reader.getTerminal().getHeight();
-    
+
     String peek = null;
     while (lines.hasNext()) {
       String nextLine = lines.next();
@@ -942,7 +942,7 @@ public class Shell extends ShellOptions {
             reader.println(peek);
             if (paginate) {
               linesPrinted += peek.length() == 0 ? 0 : Math.ceil(peek.length() * 1.0 / termWidth);
-              
+
               // check if displaying the next line would result in
               // scrolling off the screen
               if (linesPrinted + Math.ceil(lastPromptLength * 1.0 / termWidth) + Math.ceil(prompt.length() * 1.0 / termWidth)
@@ -953,7 +953,7 @@ public class Shell extends ShellOptions {
                 lastPromptLength = nextPrompt.length();
                 reader.print(nextPrompt);
                 reader.flush();
-                
+
                 if (Character.toUpperCase((char) reader.readCharacter()) == 'Q') {
                   reader.println();
                   return;
@@ -974,38 +974,38 @@ public class Shell extends ShellOptions {
       reader.println(peek);
     }
   }
-  
+
   public final void printRecords(Iterable<Entry<Key,Value>> scanner, boolean printTimestamps, boolean paginate, Class<? extends Formatter> formatterClass,
       PrintLine outFile) throws IOException {
     printLines(FormatterFactory.getFormatter(formatterClass, scanner, printTimestamps), paginate, outFile);
   }
-  
+
   public final void printRecords(Iterable<Entry<Key,Value>> scanner, boolean printTimestamps, boolean paginate, Class<? extends Formatter> formatterClass)
       throws IOException {
     printLines(FormatterFactory.getFormatter(formatterClass, scanner, printTimestamps), paginate);
   }
-  
+
   public final void printBinaryRecords(Iterable<Entry<Key,Value>> scanner, boolean printTimestamps, boolean paginate, PrintLine outFile) throws IOException {
     printLines(FormatterFactory.getFormatter(binaryFormatterClass, scanner, printTimestamps), paginate, outFile);
   }
-  
+
   public final void printBinaryRecords(Iterable<Entry<Key,Value>> scanner, boolean printTimestamps, boolean paginate) throws IOException {
     printLines(FormatterFactory.getFormatter(binaryFormatterClass, scanner, printTimestamps), paginate);
   }
-  
+
   public static String repeat(String s, int c) {
     StringBuilder sb = new StringBuilder();
     for (int i = 0; i < c; i++)
       sb.append(s);
     return sb.toString();
   }
-  
+
   public void checkTableState() {
     if (getTableName().isEmpty())
       throw new IllegalStateException(
           "Not in a table context. Please use 'table <tableName>' to switch to a table, or use '-t' to specify a table if option is available.");
   }
-  
+
   private final void printConstraintViolationException(ConstraintViolationException cve) {
     printException(cve, "");
     int COL1 = 50, COL2 = 14;
@@ -1017,28 +1017,28 @@ public class Shell extends ShellOptions {
       logError(String.format("%-" + COL1 + "s | %" + COL2 + "d | %-" + col3 + "s%n", cvs.constrainClass, cvs.violationCode, cvs.violationDescription));
     logError(String.format("%" + COL1 + "s-+-%" + COL2 + "s-+-%" + col3 + "s%n", repeat("-", COL1), repeat("-", COL2), repeat("-", col3)));
   }
-  
+
   public final void printException(Exception e) {
     printException(e, e.getMessage());
   }
-  
+
   private final void printException(Exception e, String msg) {
     logError(e.getClass().getName() + (msg != null ? ": " + msg : ""));
     log.debug(e.getClass().getName() + (msg != null ? ": " + msg : ""), e);
   }
-  
+
   public static final void setDebugging(boolean debuggingEnabled) {
     Logger.getLogger(Constants.CORE_PACKAGE_NAME).setLevel(debuggingEnabled ? Level.TRACE : Level.INFO);
   }
-  
+
   public static final boolean isDebuggingEnabled() {
     return Logger.getLogger(Constants.CORE_PACKAGE_NAME).isTraceEnabled();
   }
-  
+
   private final void printHelp(String usage, String description, Options opts) {
     printHelp(usage, description, opts, Integer.MAX_VALUE);
   }
-  
+
   private final void printHelp(String usage, String description, Options opts, int width) {
     PrintWriter pw = new PrintWriter(System.err);
     new HelpFormatter().printHelp(pw, width, usage, description, opts, 2, 5, null, true);
@@ -1048,53 +1048,53 @@ public class Shell extends ShellOptions {
       writer.flush();
     }
   }
-  
+
   public int getExitCode() {
     return exitCode;
   }
-  
+
   public void resetExitCode() {
     exitCode = 0;
   }
-  
+
   public void setExit(boolean exit) {
     this.exit = exit;
   }
-  
+
   public boolean getExit() {
     return this.exit;
   }
-  
+
   public boolean isVerbose() {
     return verbose;
   }
-  
+
   public void setTableName(String tableName) {
     this.tableName = tableName;
   }
-  
+
   public String getTableName() {
     return tableName;
   }
-  
+
   public ConsoleReader getReader() {
     return reader;
   }
-  
+
   public void updateUser(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
     connector = instance.getConnector(principal, token);
     this.principal = principal;
     this.token = token;
   }
-  
+
   public String getPrincipal() {
     return principal;
   }
-  
+
   public AuthenticationToken getToken() {
     return token;
   }
-  
+
   /**
    * Return the formatter for the current table.
    * 
@@ -1103,7 +1103,7 @@ public class Shell extends ShellOptions {
   public Class<? extends Formatter> getFormatter() {
     return getFormatter(this.tableName);
   }
-  
+
   /**
    * Return the formatter for the given table.
    * 
@@ -1113,7 +1113,7 @@ public class Shell extends ShellOptions {
    */
   public Class<? extends Formatter> getFormatter(String tableName) {
     Class<? extends Formatter> formatter = FormatterCommand.getCurrentFormatter(tableName, this);
-    
+
     if (null == formatter) {
       logError("Could not load the specified formatter. Using the DefaultFormatter");
       return this.defaultFormatterClass;
@@ -1121,11 +1121,11 @@ public class Shell extends ShellOptions {
       return formatter;
     }
   }
-  
+
   public void setLogErrorsToConsole() {
     this.logErrorsToConsole = true;
   }
-  
+
   private void logError(String s) {
     log.error(s);
     if (logErrorsToConsole) {
@@ -1135,24 +1135,24 @@ public class Shell extends ShellOptions {
       } catch (IOException e) {}
     }
   }
-  
+
   public String readMaskedLine(String prompt, Character mask) throws IOException {
     this.masking = true;
     String s = reader.readLine(prompt, mask);
     this.masking = false;
     return s;
   }
-  
+
   public boolean isMasking() {
     return masking;
   }
-  
+
   public boolean hasExited() {
     return exit;
   }
-  
+
   public boolean isTabCompletion() {
     return tabCompletion;
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/71bed4df/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CloneNamespaceCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CloneNamespaceCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CloneNamespaceCommand.java
deleted file mode 100644
index dade389..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CloneNamespaceCommand.java
+++ /dev/null
@@ -1,113 +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.core.util.shell.commands;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.TableNamespaceExistsException;
-import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.util.shell.Shell;
-import org.apache.accumulo.core.util.shell.Shell.Command;
-import org.apache.accumulo.core.util.shell.Token;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.Options;
-
-public class CloneNamespaceCommand extends Command {
-
-  private Option setPropsOption;
-  private Option excludePropsOption;
-  private Option noFlushOption;
-  private Option copyTablePropsOption;
-
-  @Override
-  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException,
-      TableNotFoundException, TableExistsException, TableNamespaceNotFoundException, TableNamespaceExistsException {
-
-    final HashMap<String,String> props = new HashMap<String,String>();
-    final HashSet<String> exclude = new HashSet<String>();
-    boolean flush = true;
-    boolean copyTableProps = false;
-
-    if (cl.hasOption(setPropsOption.getOpt())) {
-      String[] keyVals = cl.getOptionValue(setPropsOption.getOpt()).split(",");
-      for (String keyVal : keyVals) {
-        String[] sa = keyVal.split("=");
-        props.put(sa[0], sa[1]);
-      }
-    }
-
-    if (cl.hasOption(excludePropsOption.getOpt())) {
-      String[] keys = cl.getOptionValue(excludePropsOption.getOpt()).split(",");
-      for (String key : keys) {
-        exclude.add(key);
-      }
-    }
-
-    if (cl.hasOption(noFlushOption.getOpt())) {
-      flush = false;
-    }
-
-    if (cl.hasOption(noFlushOption.getOpt())) {
-      copyTableProps = true;
-    }
-
-    shellState.getConnector().tableNamespaceOperations().clone(cl.getArgs()[0], cl.getArgs()[1], flush, props, exclude, copyTableProps);
-    return 0;
-  }
-
-  @Override
-  public String usage() {
-    return getName() + " <current name> <new name>";
-  }
-
-  @Override
-  public String description() {
-    return "clones a table namespace";
-  }
-
-  @Override
-  public void registerCompletion(final Token root, final Map<Command.CompletionSet,Set<String>> completionSet) {
-    registerCompletionForTableNamespaces(root, completionSet);
-  }
-
-  @Override
-  public Options getOptions() {
-    final Options o = new Options();
-    setPropsOption = new Option("s", "set", true, "set initial properties. Expects <prop>=<value>{,<prop>=<value>}");
-    o.addOption(setPropsOption);
-    excludePropsOption = new Option("e", "exclude", true, "exclude properties that should not be copied from source. Expects <prop>{,<prop>}");
-    o.addOption(excludePropsOption);
-    noFlushOption = new Option("nf", "noFlush", false, "do not flush table data in memory before cloning.");
-    o.addOption(noFlushOption);
-    copyTablePropsOption = new Option("tp", "copyTableProps", false, "copy each table's properties to the cloned table in the new namespace.");
-    o.addOption(copyTablePropsOption);
-    return o;
-  }
-
-  @Override
-  public int numArgs() {
-    return 2;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/71bed4df/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 e0c1398..64c6340 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
@@ -99,7 +99,6 @@ import org.apache.accumulo.master.tableOps.BulkImport;
 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.CloneTableNamespace;
 import org.apache.accumulo.master.tableOps.CompactRange;
 import org.apache.accumulo.master.tableOps.CreateTable;
 import org.apache.accumulo.master.tableOps.CreateTableNamespace;
@@ -951,7 +950,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
           } catch (TableNamespaceNotFoundException e) {
             throw new TException(e.getMessage(), e);
           }
-          
+
           break;
         }
         case CLONE: {
@@ -1190,34 +1189,6 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
           fate.seedTransaction(opid, new TraceRepo<Master>(new DeleteTableNamespace(namespaceId)), autoCleanup);
           break;
         }
-        case CLONE: {
-          String namespaceId = ByteBufferUtil.toString(arguments.get(0));
-          String namespace = ByteBufferUtil.toString(arguments.get(1));
-          checkNotSystemNamespace(namespace, TableOperation.CLONE);
-          checkTableNamespaceName(namespace, TableOperation.CLONE);
-          if (!security.canCloneNamespace(c, namespaceId, namespace))
-            throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-
-          Map<String,String> propertiesToSet = new HashMap<String,String>();
-          Set<String> propertiesToExclude = new HashSet<String>();
-
-          for (Entry<String,String> entry : options.entrySet()) {
-            if (entry.getValue() == null || entry.getValue().isEmpty()) {
-              propertiesToExclude.add(entry.getKey());
-              continue;
-            }
-            if (!TablePropUtil.isPropertyValid(entry.getKey(), entry.getValue())) {
-              throw new ThriftTableOperationException(null, namespace, TableOperation.CLONE, TableOperationExceptionType.OTHER, "Property or value not valid "
-                  + entry.getKey() + "=" + entry.getValue());
-            }
-            propertiesToSet.put(entry.getKey(), entry.getValue());
-          }
-
-          fate.seedTransaction(opid, new TraceRepo<Master>(new CloneTableNamespace(c.getPrincipal(), namespaceId, namespace, propertiesToSet,
-              propertiesToExclude)), autoCleanup);
-
-          break;
-        }
         default:
           throw new UnsupportedOperationException();
       }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/71bed4df/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTableNamespace.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTableNamespace.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTableNamespace.java
deleted file mode 100644
index ffa1448..0000000
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTableNamespace.java
+++ /dev/null
@@ -1,201 +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.Set;
-
-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.client.HdfsZooInstance;
-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 CloneNamespaceInfo implements Serializable {
-
-  private static final long serialVersionUID = 1L;
-
-  String srcId;
-  String namespace;
-  String newId;
-  Map<String,String> propertiesToSet;
-  Set<String> propertiesToExclude;
-
-  public String user;
-}
-
-class FinishCloneTableNamespace extends MasterRepo {
-
-  private static final long serialVersionUID = 1L;
-  private CloneNamespaceInfo cloneInfo;
-
-  public FinishCloneTableNamespace(CloneNamespaceInfo cloneInfo) {
-    this.cloneInfo = cloneInfo;
-  }
-
-  @Override
-  public long isReady(long tid, Master environment) throws Exception {
-    return 0;
-  }
-
-  @Override
-  public Repo<Master> call(long tid, Master environment) throws Exception {
-    Utils.unreserveTableNamespace(cloneInfo.srcId, tid, false);
-    Utils.unreserveTableNamespace(cloneInfo.newId, tid, true);
-
-    environment.getEventCoordinator().event("Cloned table namespace %s from %s", cloneInfo.namespace, cloneInfo.srcId);
-
-    Logger.getLogger(FinishCloneTableNamespace.class).debug("Cloned table namespace " + cloneInfo.srcId + " " + cloneInfo.newId + " " + cloneInfo.namespace);
-
-    return null;
-  }
-
-  @Override
-  public void undo(long tid, Master environment) throws Exception {}
-}
-
-class CloneNamespaceZookeeper extends MasterRepo {
-
-  private static final long serialVersionUID = 1L;
-
-  private CloneNamespaceInfo cloneInfo;
-
-  public CloneNamespaceZookeeper(CloneNamespaceInfo cloneInfo) {
-    this.cloneInfo = cloneInfo;
-  }
-
-  @Override
-  public long isReady(long tid, Master environment) throws Exception {
-    return Utils.reserveTableNamespace(cloneInfo.newId, tid, true, false, TableOperation.CLONE);
-  }
-
-  @Override
-  public Repo<Master> call(long tid, Master environment) throws Exception {
-    Utils.tableNameLock.lock();
-    try {
-      // write namespace to zookeeper
-      Instance instance = HdfsZooInstance.getInstance();
-
-      Utils.checkTableNamespaceDoesNotExist(instance, cloneInfo.namespace, cloneInfo.newId, TableOperation.CLONE);
-
-      TableManager.getInstance().addNamespace(cloneInfo.newId, cloneInfo.namespace, NodeExistsPolicy.FAIL);
-      TableManager.getInstance().cloneNamespace(cloneInfo.srcId, cloneInfo.newId, cloneInfo.namespace, cloneInfo.propertiesToSet,
-          cloneInfo.propertiesToExclude, NodeExistsPolicy.OVERWRITE);
-      Tables.clearCache(instance);
-
-      return new FinishCloneTableNamespace(cloneInfo);
-    } finally {
-      Utils.tableNameLock.unlock();
-    }
-  }
-
-  @Override
-  public void undo(long tid, Master environment) throws Exception {
-    Instance instance = HdfsZooInstance.getInstance();
-    TableManager.getInstance().removeNamespace(cloneInfo.newId);
-    Utils.unreserveTableNamespace(cloneInfo.newId, tid, true);
-    Tables.clearCache(instance);
-  }
-}
-
-class CloneNamespacePermissions extends MasterRepo {
-
-  private static final long serialVersionUID = 1L;
-
-  private CloneNamespaceInfo cloneInfo;
-
-  public CloneNamespacePermissions(CloneNamespaceInfo cloneInfo) {
-    this.cloneInfo = cloneInfo;
-  }
-
-  @Override
-  public long isReady(long tid, Master environment) throws Exception {
-    return 0;
-  }
-
-  @Override
-  public Repo<Master> call(long tid, Master environment) throws Exception {
-    // give all table namespace permissions to the creator
-    for (TableNamespacePermission permission : TableNamespacePermission.values()) {
-      try {
-        AuditedSecurityOperation.getInstance().grantTableNamespacePermission(SystemCredentials.get().toThrift(environment.getInstance()), cloneInfo.user,
-            cloneInfo.newId, permission);
-      } catch (ThriftSecurityException e) {
-        Logger.getLogger(FinishCloneTableNamespace.class).error(e.getMessage(), e);
-        throw e;
-      }
-    }
-
-    // setup permissions in zookeeper before table namespace info in zookeeper
-    // this way concurrent users will not get a spurious pemission denied
-    // error
-    return new CloneNamespaceZookeeper(cloneInfo);
-  }
-
-  @Override
-  public void undo(long tid, Master environment) throws Exception {
-
-  }
-}
-
-public class CloneTableNamespace extends MasterRepo {
-
-  private static final long serialVersionUID = 1L;
-  private CloneNamespaceInfo cloneInfo;
-
-  public CloneTableNamespace(String user, String srcId, String namespace, Map<String,String> propertiesToSet, Set<String> propertiesToExclude) {
-    cloneInfo = new CloneNamespaceInfo();
-    cloneInfo.user = user;
-    cloneInfo.srcId = srcId;
-    cloneInfo.namespace = namespace;
-    cloneInfo.propertiesToExclude = propertiesToExclude;
-    cloneInfo.propertiesToSet = propertiesToSet;
-  }
-
-  @Override
-  public long isReady(long tid, Master environment) throws Exception {
-    return Utils.reserveTableNamespace(cloneInfo.srcId, tid, false, true, TableOperation.CLONE);
-  }
-
-  @Override
-  public Repo<Master> call(long tid, Master environment) throws Exception {
-
-    Utils.idLock.lock();
-    try {
-      Instance instance = HdfsZooInstance.getInstance();
-      cloneInfo.newId = Utils.getNextTableId(cloneInfo.namespace, instance);
-      return new CloneNamespacePermissions(cloneInfo);
-    } finally {
-      Utils.idLock.unlock();
-    }
-  }
-
-  @Override
-  public void undo(long tid, Master environment) throws Exception {
-    Utils.unreserveTableNamespace(cloneInfo.srcId, tid, false);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/71bed4df/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CloneTableNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CloneTableNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CloneTableNamespace.java
deleted file mode 100644
index 2c35173..0000000
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CloneTableNamespace.java
+++ /dev/null
@@ -1,55 +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.test.randomwalk.concurrent;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Properties;
-import java.util.Random;
-
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.TableNamespaceExistsException;
-import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
-import org.apache.accumulo.test.randomwalk.State;
-import org.apache.accumulo.test.randomwalk.Test;
-
-public class CloneTableNamespace extends Test {
-
-  @Override
-  public void visit(State state, Properties props) throws Exception {
-    Connector conn = state.getConnector();
-
-    Random rand = (Random) state.get("rand");
-
-    @SuppressWarnings("unchecked")
-    List<String> namespaces = (List<String>) state.get("namespaces");
-
-    String srcName = namespaces.get(rand.nextInt(namespaces.size()));
-    String newName = namespaces.get(rand.nextInt(namespaces.size()));
-    boolean flush = rand.nextBoolean();
-
-    try {
-      log.debug("Cloning table namespace " + srcName + " " + newName + " " + flush);
-      conn.tableNamespaceOperations().clone(srcName, newName, flush, new HashMap<String,String>(), new HashSet<String>(), true);
-    } catch (TableNamespaceExistsException e) {
-      log.debug("Clone namespace " + srcName + " failed, " + newName + " exists");
-    } catch (TableNamespaceNotFoundException e) {
-      log.debug("Clone namespace " + srcName + " failed, doesnt exist");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/71bed4df/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java b/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
index 5705044..6cb2568 100644
--- a/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
@@ -21,13 +21,10 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
-import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Random;
-import java.util.Set;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
@@ -63,25 +60,25 @@ import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 
 public class TableNamespacesIT {
-  
+
   Random random = new Random();
   public static TemporaryFolder folder = new TemporaryFolder();
   static private MiniAccumuloCluster accumulo;
   static private String secret = "secret";
-  
+
   @BeforeClass
   static public void setUp() throws Exception {
     folder.create();
     accumulo = new MiniAccumuloCluster(folder.getRoot(), secret);
     accumulo.start();
   }
-  
+
   @AfterClass
   static public void tearDown() throws Exception {
     accumulo.stop();
     folder.delete();
   }
-  
+
   /**
    * This test creates a table without specifying a namespace. In this case, it puts the table into the default namespace.
    */
@@ -89,12 +86,12 @@ public class TableNamespacesIT {
   public void testDefaultNamespace() throws Exception {
     String tableName = "test";
     Connector c = accumulo.getConnector("root", secret);
-    
+
     assertTrue(c.tableNamespaceOperations().exists(Constants.DEFAULT_TABLE_NAMESPACE));
     c.tableOperations().create(tableName);
     assertTrue(c.tableOperations().exists(tableName));
   }
-  
+
   /**
    * This test creates a new namespace "testing" and a table "testing.table1" which puts "table1" into the "testing" namespace. Then we create "testing.table2"
    * which creates "table2" and puts it into "testing" as well. Then we make sure that you can't delete a namespace with tables in it, and then we delete the
@@ -105,18 +102,18 @@ public class TableNamespacesIT {
     String namespace = "testing";
     String tableName1 = namespace + ".table1";
     String tableName2 = namespace + ".table2";
-    
+
     Connector c = accumulo.getConnector("root", secret);
-    
+
     c.tableNamespaceOperations().create(namespace);
     assertTrue(c.tableNamespaceOperations().exists(namespace));
-    
+
     c.tableOperations().create(tableName1);
     assertTrue(c.tableOperations().exists(tableName1));
-    
+
     c.tableOperations().create(tableName2);
     assertTrue(c.tableOperations().exists(tableName2));
-    
+
     // deleting
     try {
       // can't delete a namespace with tables in it
@@ -128,17 +125,17 @@ public class TableNamespacesIT {
     assertTrue(c.tableNamespaceOperations().exists(namespace));
     assertTrue(c.tableOperations().exists(tableName1));
     assertTrue(c.tableOperations().exists(tableName2));
-    
+
     c.tableOperations().delete(tableName2);
     assertTrue(!c.tableOperations().exists(tableName2));
     assertTrue(c.tableNamespaceOperations().exists(namespace));
-    
+
     c.tableOperations().delete(tableName1);
     assertTrue(!c.tableOperations().exists(tableName1));
     c.tableNamespaceOperations().delete(namespace);
     assertTrue(!c.tableNamespaceOperations().exists(namespace));
   }
-  
+
   /**
    * This test creates a namespace, modifies it's properties, and checks to make sure that those properties are applied to its tables. To do something on a
    * namespace-wide level, use TableNamespaceOperations.
@@ -147,70 +144,70 @@ public class TableNamespacesIT {
    * 
    * Checks to see if the default namespace's properties work as well.
    */
-  
+
   @Test
   public void testNamespaceProperties() throws Exception {
     String namespace = "propchange";
     String tableName1 = namespace + ".table1";
     String tableName2 = namespace + ".table2";
-    
+
     String propKey = Property.TABLE_SCAN_MAXMEM.getKey();
     String propVal = "42K";
-    
+
     Connector c = accumulo.getConnector("root", secret);
-    
+
     c.tableNamespaceOperations().create(namespace);
     c.tableOperations().create(tableName1);
     c.tableNamespaceOperations().setProperty(namespace, propKey, propVal);
-    
+
     // check the namespace has the property
     assertTrue(checkTableNamespaceHasProp(c, namespace, propKey, propVal));
-    
+
     // check that the table gets it from the namespace
     assertTrue(checkTableHasProp(c, tableName1, propKey, propVal));
-    
+
     // test a second table to be sure the first wasn't magical
     // (also, changed the order, the namespace has the property already)
     c.tableOperations().create(tableName2);
     assertTrue(checkTableHasProp(c, tableName2, propKey, propVal));
-    
+
     // test that table properties override namespace properties
     String propKey2 = Property.TABLE_FILE_MAX.getKey();
     String propVal2 = "42";
     String tablePropVal = "13";
-    
+
     c.tableOperations().setProperty(tableName2, propKey2, tablePropVal);
     c.tableNamespaceOperations().setProperty("propchange", propKey2, propVal2);
-    
+
     assertTrue(checkTableHasProp(c, tableName2, propKey2, tablePropVal));
-    
+
     // now check that you can change the default namespace's properties
     propVal = "13K";
     String tableName = "some_table";
     c.tableOperations().create(tableName);
     c.tableNamespaceOperations().setProperty(Constants.DEFAULT_TABLE_NAMESPACE, propKey, propVal);
-    
+
     assertTrue(checkTableHasProp(c, tableName, propKey, propVal));
-    
+
     // test the properties server-side by configuring an iterator.
     // should not show anything with column-family = 'a'
     String tableName3 = namespace + ".table3";
     c.tableOperations().create(tableName3);
-    
+
     IteratorSetting setting = new IteratorSetting(250, "thing", SimpleFilter.class.getName());
     c.tableNamespaceOperations().attachIterator(namespace, setting);
-    
+
     BatchWriter bw = c.createBatchWriter(tableName3, new BatchWriterConfig());
     Mutation m = new Mutation("r");
     m.put("a", "b", new Value("abcde".getBytes()));
     bw.addMutation(m);
     bw.flush();
     bw.close();
-    
+
     Scanner s = c.createScanner(tableName3, Authorizations.EMPTY);
     assertTrue(!s.iterator().hasNext());
   }
-  
+
   /**
    * This test renames and clones two separate table into different namespaces. different namespace.
    * 
@@ -222,25 +219,25 @@ public class TableNamespacesIT {
     String tableName = "table";
     String tableName1 = "renamed.table1";
     String tableName2 = "cloned.table2";
-    
+
     Connector c = accumulo.getConnector("root", secret);
-    
+
     c.tableOperations().create(tableName);
     c.tableNamespaceOperations().create(namespace1);
     c.tableNamespaceOperations().create(namespace2);
-    
+
     c.tableOperations().rename(tableName, tableName1);
-    
+
     assertTrue(c.tableOperations().exists(tableName1));
     assertTrue(!c.tableOperations().exists(tableName));
-    
+
     c.tableOperations().clone(tableName1, tableName2, false, null, null);
-    
+
     assertTrue(c.tableOperations().exists(tableName1));
     assertTrue(c.tableOperations().exists(tableName2));
     return;
   }
-  
+
   /**
    * This test renames a table namespace and ensures that its tables are still correct
    */
@@ -249,15 +246,15 @@ public class TableNamespacesIT {
     String namespace1 = "n1";
     String namespace2 = "n2";
     String table = "t";
-    
+
     Connector c = accumulo.getConnector("root", secret);
     Instance instance = c.getInstance();
-    
+
     c.tableNamespaceOperations().create(namespace1);
     c.tableOperations().create(namespace1 + "." + table);
-    
+
     c.tableNamespaceOperations().rename(namespace1, namespace2);
-    
+
     assertTrue(!c.tableNamespaceOperations().exists(namespace1));
     assertTrue(c.tableNamespaceOperations().exists(namespace2));
     assertTrue(c.tableOperations().exists(namespace2 + "." + table));
@@ -266,7 +263,7 @@ public class TableNamespacesIT {
     String tnamespace = TableNamespaces.getNamespaceName(instance, tnid);
     assertTrue(namespace2.equals(tnamespace));
   }
-  
+
   /**
    * This test clones a table to a different namespace and ensures it's properties are correct
    */
@@ -276,110 +273,67 @@ public class TableNamespacesIT {
     String n2 = "namespace2";
     String t1 = n1 + ".table";
     String t2 = n2 + ".table";
-    
+
     String propKey = Property.TABLE_FILE_MAX.getKey();
     String propVal1 = "55";
     String propVal2 = "66";
-    
+
     Connector c = accumulo.getConnector("root", secret);
-    
+
     c.tableNamespaceOperations().create(n1);
     c.tableOperations().create(t1);
-    
+
     c.tableOperations().removeProperty(t1, Property.TABLE_FILE_MAX.getKey());
     c.tableNamespaceOperations().setProperty(n1, propKey, propVal1);
-    
+
     assertTrue(checkTableHasProp(c, t1, propKey, propVal1));
-    
+
     c.tableNamespaceOperations().create(n2);
     c.tableNamespaceOperations().setProperty(n2, propKey, propVal2);
     c.tableOperations().clone(t1, t2, true, null, null);
     c.tableOperations().removeProperty(t2, propKey);
-    
+
     assertTrue(checkTableHasProp(c, t2, propKey, propVal2));
-    
+
     c.tableNamespaceOperations().delete(n1, true);
     c.tableNamespaceOperations().delete(n2, true);
   }
-  
-  /**
-   * This test clones namespaces. First checks to see that the properties were correctly copied over, then checks to see that the correct properties were set
-   * when given that option and that table properties copy successfully.
-   */
-  @Test
-  public void testCloneNamespace() throws Exception {
-    String n1 = "nspace1";
-    String n2 = "nspace2";
-    String n3 = "nspace3";
-    String t = ".table";
-    
-    String propKey1 = Property.TABLE_FILE_MAX.getKey();
-    String propKey2 = Property.TABLE_SCAN_MAXMEM.getKey();
-    String propVal1 = "55";
-    String propVal2 = "66";
-    String propVal3 = "77K";
-    
-    Connector c = accumulo.getConnector("root", secret);
-    c.tableNamespaceOperations().create(n1);
-    c.tableOperations().create(n1 + t);
-    
-    c.tableNamespaceOperations().setProperty(n1, propKey1, propVal1);
-    c.tableOperations().setProperty(n1 + t, propKey1, propVal2);
-    c.tableNamespaceOperations().setProperty(n1, propKey2, propVal3);
-    
-    c.tableNamespaceOperations().clone(n1, n2, false, null, null, false);
-    assertTrue(c.tableNamespaceOperations().exists(n2));
-    assertTrue(checkTableNamespaceHasProp(c, n2, propKey1, propVal1));
-    assertTrue(checkTableHasProp(c, n2 + t, propKey1, propVal2));
-    assertTrue(checkTableNamespaceHasProp(c, n2, propKey2, propVal3));
-    
-    Map<String,String> propsToSet = new HashMap<String,String>();
-    propsToSet.put(propKey1, propVal1);
-    Set<String> propsToExclude = new HashSet<String>();
-    propsToExclude.add(propKey2);
-    c.tableNamespaceOperations().clone(n1, n3, true, propsToSet, propsToExclude, true);
-    
-    assertTrue(checkTableNamespaceHasProp(c, n3, propKey1, propVal1));
-    assertTrue(checkTableHasProp(c, n3 + t, propKey1, propVal2));
-    assertTrue(!checkTableNamespaceHasProp(c, n3, propKey2, propVal3));
-    assertTrue(!checkTableHasProp(c, n3 + t, propKey2, propVal3));
-  }
-  
+
   /**
    * This tests adding iterators to a namespace, listing them, and removing them as well as adding and removing constraints
    */
   @Test
   public void testNamespaceIteratorsAndConstraints() throws Exception {
     Connector c = accumulo.getConnector("root", secret);
-    
+
     String namespace = "iterator";
     String tableName = namespace + ".table";
     String iter = "thing";
-    
+
     c.tableNamespaceOperations().create(namespace);
     c.tableOperations().create(tableName);
-    
+
     IteratorSetting setting = new IteratorSetting(250, iter, SimpleFilter.class.getName());
     HashSet<IteratorScope> scope = new HashSet<IteratorScope>();
     scope.add(IteratorScope.scan);
     c.tableNamespaceOperations().attachIterator(namespace, setting, EnumSet.copyOf(scope));
-    
+
     BatchWriter bw = c.createBatchWriter(tableName, new BatchWriterConfig());
     Mutation m = new Mutation("r");
     m.put("a", "b", new Value("abcde".getBytes(Constants.UTF8)));
     bw.addMutation(m);
     bw.flush();
-    
+
     Scanner s = c.createScanner(tableName, Authorizations.EMPTY);
     assertTrue(!s.iterator().hasNext());
-    
+
     assertTrue(c.tableNamespaceOperations().listIterators(namespace).containsKey(iter));
     c.tableNamespaceOperations().removeIterator(namespace, iter, EnumSet.copyOf(scope));
-    
+
     c.tableNamespaceOperations().addConstraint(namespace, NumericValueConstraint.class.getName());
     // doesn't take effect immediately, needs time to propagate
     UtilWaitThread.sleep(250);
-    
+
     m = new Mutation("rowy");
     m.put("a", "b", new Value("abcde".getBytes(Constants.UTF8)));
     try {
@@ -393,26 +347,26 @@ public class TableNamespacesIT {
     int num = c.tableNamespaceOperations().listConstraints(namespace).get(NumericValueConstraint.class.getName());
     c.tableNamespaceOperations().removeConstraint(namespace, num);
   }
-  
+
   /**
    * Tests that when a table moves to a new namespace that it's properties inherit from the new namespace and not the old one
    */
   @Test
   public void testRenameToNewNamespaceProperties() throws Exception {
     Connector c = accumulo.getConnector("root", secret);
-    
+
     String namespace1 = "moveToNewNamespace1";
     String namespace2 = "moveToNewNamespace2";
     String tableName1 = namespace1 + ".table";
     String tableName2 = namespace2 + ".table";
-    
+
     String propKey = Property.TABLE_FILE_MAX.getKey();
     String propVal = "42";
-    
+
     c.tableNamespaceOperations().create(namespace1);
     c.tableNamespaceOperations().create(namespace2);
     c.tableOperations().create(tableName1);
-    
+
     c.tableNamespaceOperations().setProperty(namespace1, propKey, propVal);
     boolean hasProp = false;
     for (Entry<String,String> p : c.tableOperations().getProperties(tableName1)) {
@@ -421,9 +375,9 @@ public class TableNamespacesIT {
       }
     }
     assertTrue(hasProp);
-    
+
     c.tableOperations().rename(tableName1, tableName2);
-    
+
     hasProp = false;
     for (Entry<String,String> p : c.tableOperations().getProperties(tableName2)) {
       if (p.getKey().equals(propKey) && p.getValue().equals(propVal)) {
@@ -432,7 +386,7 @@ public class TableNamespacesIT {
     }
     assertTrue(!hasProp);
   }
-  
+
   /**
    * Tests new Namespace permissions as well as modifications to Table permissions because of namespaces. Checks each permission to first make sure the user
    * doesn't have permission to perform the action, then root grants them the permission and we check to make sure they could perform the action.
@@ -440,51 +394,51 @@ public class TableNamespacesIT {
   @Test
   public void testPermissions() throws Exception {
     Connector c = accumulo.getConnector("root", secret);
-    
+
     PasswordToken pass = new PasswordToken(secret);
-    
+
     String n1 = "spaceOfTheName";
-    
+
     String user1 = "dude";
-    
+
     c.tableNamespaceOperations().create(n1);
     c.tableOperations().create(n1 + ".table1");
-    
+
     c.securityOperations().createLocalUser(user1, pass);
-    
+
     Connector user1Con = accumulo.getConnector(user1, secret);
-    
+
     try {
       user1Con.tableOperations().create(n1 + ".table2");
       fail();
     } catch (AccumuloSecurityException e) {
       // supposed to happen
     }
-    
+
     c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.CREATE_TABLE);
     user1Con.tableOperations().create(n1 + ".table2");
     assertTrue(c.tableOperations().list().contains(n1 + ".table2"));
     c.securityOperations().revokeTableNamespacePermission(user1, n1, TableNamespacePermission.CREATE_TABLE);
-    
+
     try {
       user1Con.tableOperations().delete(n1 + ".table1");
       fail();
     } catch (AccumuloSecurityException e) {
       // should happen
     }
-    
+
     c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.DROP_TABLE);
     user1Con.tableOperations().delete(n1 + ".table1");
     assertTrue(!c.tableOperations().list().contains(n1 + ".table1"));
     c.securityOperations().revokeTableNamespacePermission(user1, n1, TableNamespacePermission.DROP_TABLE);
-    
+
     c.tableOperations().create(n1 + ".t");
     BatchWriter bw = c.createBatchWriter(n1 + ".t", null);
     Mutation m = new Mutation("row");
     m.put("cf", "cq", "value");
     bw.addMutation(m);
     bw.close();
-    
+
     Iterator<Entry<Key,Value>> i = user1Con.createScanner(n1 + ".t", new Authorizations()).iterator();
     try {
       i.next();
@@ -492,7 +446,7 @@ public class TableNamespacesIT {
     } catch (RuntimeException e) {
       // yup
     }
-    
+
     m = new Mutation("user1");
     m.put("cf", "cq", "turtles");
     bw = user1Con.createBatchWriter(n1 + ".t", null);
@@ -503,12 +457,12 @@ public class TableNamespacesIT {
     } catch (MutationsRejectedException e) {
       // good
     }
-    
+
     c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.READ);
     i = user1Con.createScanner(n1 + ".t", new Authorizations()).iterator();
     assertTrue(i.hasNext());
     c.securityOperations().revokeTableNamespacePermission(user1, n1, TableNamespacePermission.READ);
-    
+
     c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.WRITE);
     m = new Mutation("user1");
     m.put("cf", "cq", "turtles");
@@ -516,91 +470,91 @@ public class TableNamespacesIT {
     bw.addMutation(m);
     bw.close();
     c.securityOperations().revokeTableNamespacePermission(user1, n1, TableNamespacePermission.WRITE);
-    
+
     try {
       user1Con.tableOperations().setProperty(n1 + ".t", Property.TABLE_FILE_MAX.getKey(), "42");
       fail();
     } catch (AccumuloSecurityException e) {}
-    
+
     c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.ALTER_TABLE);
     user1Con.tableOperations().setProperty(n1 + ".t", Property.TABLE_FILE_MAX.getKey(), "42");
     user1Con.tableOperations().removeProperty(n1 + ".t", Property.TABLE_FILE_MAX.getKey());
     c.securityOperations().revokeTableNamespacePermission(user1, n1, TableNamespacePermission.ALTER_TABLE);
-    
+
     try {
       user1Con.tableNamespaceOperations().setProperty(n1, Property.TABLE_FILE_MAX.getKey(), "55");
       fail();
     } catch (AccumuloSecurityException e) {}
-    
+
     c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.ALTER_NAMESPACE);
     user1Con.tableNamespaceOperations().setProperty(n1, Property.TABLE_FILE_MAX.getKey(), "42");
     user1Con.tableNamespaceOperations().removeProperty(n1, Property.TABLE_FILE_MAX.getKey());
     c.securityOperations().revokeTableNamespacePermission(user1, n1, TableNamespacePermission.ALTER_NAMESPACE);
-    
+
     String user2 = "guy";
     c.securityOperations().createLocalUser(user2, pass);
     try {
       user1Con.securityOperations().grantTableNamespacePermission(user2, n1, TableNamespacePermission.ALTER_NAMESPACE);
       fail();
     } catch (AccumuloSecurityException e) {}
-    
+
     c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.GRANT);
     user1Con.securityOperations().grantTableNamespacePermission(user2, n1, TableNamespacePermission.ALTER_NAMESPACE);
     user1Con.securityOperations().revokeTableNamespacePermission(user2, n1, TableNamespacePermission.ALTER_NAMESPACE);
     c.securityOperations().revokeTableNamespacePermission(user1, n1, TableNamespacePermission.GRANT);
-    
+
     String n2 = "namespace2";
     try {
       user1Con.tableNamespaceOperations().create(n2);
       fail();
     } catch (AccumuloSecurityException e) {}
-    
+
     c.securityOperations().grantSystemPermission(user1, SystemPermission.CREATE_NAMESPACE);
     user1Con.tableNamespaceOperations().create(n2);
     c.securityOperations().revokeSystemPermission(user1, SystemPermission.CREATE_NAMESPACE);
-    
+
     try {
       user1Con.tableNamespaceOperations().delete(n2);
       fail();
     } catch (AccumuloSecurityException e) {}
-    
+
     c.securityOperations().grantSystemPermission(user1, SystemPermission.DROP_NAMESPACE);
     user1Con.tableNamespaceOperations().delete(n2);
     c.securityOperations().revokeSystemPermission(user1, SystemPermission.DROP_NAMESPACE);
-    
+
     try {
       user1Con.tableNamespaceOperations().setProperty(n1, Property.TABLE_FILE_MAX.getKey(), "33");
       fail();
     } catch (AccumuloSecurityException e) {}
-    
+
     c.securityOperations().grantSystemPermission(user1, SystemPermission.ALTER_NAMESPACE);
     user1Con.tableNamespaceOperations().setProperty(n1, Property.TABLE_FILE_MAX.getKey(), "33");
     user1Con.tableNamespaceOperations().removeProperty(n1, Property.TABLE_FILE_MAX.getKey());
     c.securityOperations().revokeSystemPermission(user1, SystemPermission.ALTER_NAMESPACE);
   }
-  
+
   /**
    * This test makes sure that system-level iterators and constraints are ignored by the system namespace so that the metadata and root tables aren't affected
    */
   @Test
   public void excludeSystemIterConst() throws Exception {
     Connector c = accumulo.getConnector("root", secret);
-    
+
     c.instanceOperations().setProperty("table.iterator.scan.sum", "20," + SimpleFilter.class.getName());
     assertTrue(c.instanceOperations().getSystemConfiguration().containsValue("20," + SimpleFilter.class.getName()));
-    
+
     assertTrue(checkTableNamespaceHasProp(c, Constants.DEFAULT_TABLE_NAMESPACE, "table.iterator.scan.sum", "20," + SimpleFilter.class.getName()));
     assertTrue(!checkTableNamespaceHasProp(c, Constants.SYSTEM_TABLE_NAMESPACE, "table.iterator.scan.sum", "20," + SimpleFilter.class.getName()));
     c.instanceOperations().removeProperty("table.iterator.scan.sum");
-    
+
     c.instanceOperations().setProperty("table.constraint.42", NumericValueConstraint.class.getName());
     assertTrue(c.instanceOperations().getSystemConfiguration().containsValue(NumericValueConstraint.class.getName()));
-    
+
     assertTrue(checkTableNamespaceHasProp(c, Constants.DEFAULT_TABLE_NAMESPACE, "table.constraint.42", NumericValueConstraint.class.getName()));
     assertTrue(!checkTableNamespaceHasProp(c, Constants.SYSTEM_TABLE_NAMESPACE, "table.constraint.42", NumericValueConstraint.class.getName()));
     c.instanceOperations().removeProperty("table.constraint.42");
   }
-  
+
   private boolean checkTableHasProp(Connector c, String t, String propKey, String propVal) throws AccumuloException, TableNotFoundException {
     for (Entry<String,String> e : c.tableOperations().getProperties(t)) {
       if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) {
@@ -609,7 +563,7 @@ public class TableNamespacesIT {
     }
     return false;
   }
-  
+
   private boolean checkTableNamespaceHasProp(Connector c, String n, String propKey, String propVal) throws AccumuloException, TableNamespaceNotFoundException {
     for (Entry<String,String> e : c.tableNamespaceOperations().getProperties(n)) {
       if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) {
@@ -618,8 +572,9 @@ public class TableNamespacesIT {
     }
     return false;
   }
-  
+
   public static class SimpleFilter extends Filter {
+    @Override
     public boolean accept(Key k, Value v) {
       if (k.getColumnFamily().toString().equals("a"))
         return false;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/71bed4df/test/system/randomwalk/conf/modules/Concurrent.xml
----------------------------------------------------------------------
diff --git a/test/system/randomwalk/conf/modules/Concurrent.xml b/test/system/randomwalk/conf/modules/Concurrent.xml
index cb057c2..3768878 100644
--- a/test/system/randomwalk/conf/modules/Concurrent.xml
+++ b/test/system/randomwalk/conf/modules/Concurrent.xml
@@ -52,7 +52,6 @@
   <edge id="ct.CreateTableNamespace" weight="1000"/>
   <edge id="ct.DeleteTableNamespace" weight="100"/>
   <edge id="ct.RenameTableNamespace" weight="100"/>
-  <edge id="ct.CloneTableNamespace" weight="100"/>
   <edge id="ct.OfflineTableNamespace" weight="100"/>
   <edge id="ct.Apocalypse" weight="10"/>
   <edge id="END" weight="1"/>
@@ -180,10 +179,6 @@
   <edge id="ct.StartAll" weight="1"/>
 </node>
 
-<node id="ct.CloneTableNamespace">
-  <edge id="ct.StartAll" weight="1"/>
-</node>
-
 <node id="ct.OfflineTableNamespace">
   <edge id="ct.StartAll" weight="1"/>
 </node>


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

Posted by ct...@apache.org.
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);
   }
 }


[07/50] [abbrv] ACCUMULO-1479 implemented most of Table Namespace Permissions, doesnt entirely work, not well tested

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfdf5113/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/ClientService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/ClientService.java b/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/ClientService.java
index b857c27..3ccb234 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/ClientService.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/ClientService.java
@@ -84,6 +84,8 @@ import org.slf4j.LoggerFactory;
 
     public boolean hasTablePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableName, byte tblPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
 
+    public boolean hasTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte tblNspcPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+
     public void grantSystemPermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, byte permission) throws ThriftSecurityException, org.apache.thrift.TException;
 
     public void revokeSystemPermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, byte permission) throws ThriftSecurityException, org.apache.thrift.TException;
@@ -92,6 +94,10 @@ import org.slf4j.LoggerFactory;
 
     public void revokeTablePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableName, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
 
+    public void grantTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+
+    public void revokeTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException;
+
     public Map<String,String> getConfiguration(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, ConfigurationType type) throws org.apache.thrift.TException;
 
     public Map<String,String> getTableConfiguration(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String tableName) throws ThriftTableOperationException, org.apache.thrift.TException;
@@ -142,6 +148,8 @@ import org.slf4j.LoggerFactory;
 
     public void hasTablePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableName, byte tblPerm, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.hasTablePermission_call> resultHandler) throws org.apache.thrift.TException;
 
+    public void hasTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte tblNspcPerm, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.hasTableNamespacePermission_call> resultHandler) throws org.apache.thrift.TException;
+
     public void grantSystemPermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.grantSystemPermission_call> resultHandler) throws org.apache.thrift.TException;
 
     public void revokeSystemPermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.revokeSystemPermission_call> resultHandler) throws org.apache.thrift.TException;
@@ -150,6 +158,10 @@ import org.slf4j.LoggerFactory;
 
     public void revokeTablePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableName, byte permission, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.revokeTablePermission_call> resultHandler) throws org.apache.thrift.TException;
 
+    public void grantTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.grantTableNamespacePermission_call> resultHandler) throws org.apache.thrift.TException;
+
+    public void revokeTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.revokeTableNamespacePermission_call> resultHandler) throws org.apache.thrift.TException;
+
     public void getConfiguration(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, ConfigurationType type, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.getConfiguration_call> resultHandler) throws org.apache.thrift.TException;
 
     public void getTableConfiguration(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String tableName, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.getTableConfiguration_call> resultHandler) throws org.apache.thrift.TException;
@@ -637,6 +649,39 @@ import org.slf4j.LoggerFactory;
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "hasTablePermission failed: unknown result");
     }
 
+    public boolean hasTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte tblNspcPerm) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    {
+      send_hasTableNamespacePermission(tinfo, credentials, principal, tableNamespace, tblNspcPerm);
+      return recv_hasTableNamespacePermission();
+    }
+
+    public void send_hasTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte tblNspcPerm) throws org.apache.thrift.TException
+    {
+      hasTableNamespacePermission_args args = new hasTableNamespacePermission_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setPrincipal(principal);
+      args.setTableNamespace(tableNamespace);
+      args.setTblNspcPerm(tblNspcPerm);
+      sendBase("hasTableNamespacePermission", args);
+    }
+
+    public boolean recv_hasTableNamespacePermission() throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    {
+      hasTableNamespacePermission_result result = new hasTableNamespacePermission_result();
+      receiveBase(result, "hasTableNamespacePermission");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.sec != null) {
+        throw result.sec;
+      }
+      if (result.tope != null) {
+        throw result.tope;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "hasTableNamespacePermission failed: unknown result");
+    }
+
     public void grantSystemPermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, byte permission) throws ThriftSecurityException, org.apache.thrift.TException
     {
       send_grantSystemPermission(tinfo, credentials, principal, permission);
@@ -749,6 +794,66 @@ import org.slf4j.LoggerFactory;
       return;
     }
 
+    public void grantTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    {
+      send_grantTableNamespacePermission(tinfo, credentials, principal, tableNamespace, permission);
+      recv_grantTableNamespacePermission();
+    }
+
+    public void send_grantTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission) throws org.apache.thrift.TException
+    {
+      grantTableNamespacePermission_args args = new grantTableNamespacePermission_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setPrincipal(principal);
+      args.setTableNamespace(tableNamespace);
+      args.setPermission(permission);
+      sendBase("grantTableNamespacePermission", args);
+    }
+
+    public void recv_grantTableNamespacePermission() throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    {
+      grantTableNamespacePermission_result result = new grantTableNamespacePermission_result();
+      receiveBase(result, "grantTableNamespacePermission");
+      if (result.sec != null) {
+        throw result.sec;
+      }
+      if (result.tope != null) {
+        throw result.tope;
+      }
+      return;
+    }
+
+    public void revokeTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission) throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    {
+      send_revokeTableNamespacePermission(tinfo, credentials, principal, tableNamespace, permission);
+      recv_revokeTableNamespacePermission();
+    }
+
+    public void send_revokeTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission) throws org.apache.thrift.TException
+    {
+      revokeTableNamespacePermission_args args = new revokeTableNamespacePermission_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setPrincipal(principal);
+      args.setTableNamespace(tableNamespace);
+      args.setPermission(permission);
+      sendBase("revokeTableNamespacePermission", args);
+    }
+
+    public void recv_revokeTableNamespacePermission() throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException
+    {
+      revokeTableNamespacePermission_result result = new revokeTableNamespacePermission_result();
+      receiveBase(result, "revokeTableNamespacePermission");
+      if (result.sec != null) {
+        throw result.sec;
+      }
+      if (result.tope != null) {
+        throw result.tope;
+      }
+      return;
+    }
+
     public Map<String,String> getConfiguration(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, ConfigurationType type) throws org.apache.thrift.TException
     {
       send_getConfiguration(tinfo, credentials, type);
@@ -1571,6 +1676,50 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public void hasTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte tblNspcPerm, org.apache.thrift.async.AsyncMethodCallback<hasTableNamespacePermission_call> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      hasTableNamespacePermission_call method_call = new hasTableNamespacePermission_call(tinfo, credentials, principal, tableNamespace, tblNspcPerm, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class hasTableNamespacePermission_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private org.apache.accumulo.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
+      private String principal;
+      private String tableNamespace;
+      private byte tblNspcPerm;
+      public hasTableNamespacePermission_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte tblNspcPerm, org.apache.thrift.async.AsyncMethodCallback<hasTableNamespacePermission_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.principal = principal;
+        this.tableNamespace = tableNamespace;
+        this.tblNspcPerm = tblNspcPerm;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("hasTableNamespacePermission", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        hasTableNamespacePermission_args args = new hasTableNamespacePermission_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setPrincipal(principal);
+        args.setTableNamespace(tableNamespace);
+        args.setTblNspcPerm(tblNspcPerm);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public boolean getResult() throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_hasTableNamespacePermission();
+      }
+    }
+
     public void grantSystemPermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, byte permission, org.apache.thrift.async.AsyncMethodCallback<grantSystemPermission_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       grantSystemPermission_call method_call = new grantSystemPermission_call(tinfo, credentials, principal, permission, resultHandler, this, ___protocolFactory, ___transport);
@@ -1741,6 +1890,94 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public void grantTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission, org.apache.thrift.async.AsyncMethodCallback<grantTableNamespacePermission_call> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      grantTableNamespacePermission_call method_call = new grantTableNamespacePermission_call(tinfo, credentials, principal, tableNamespace, permission, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class grantTableNamespacePermission_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private org.apache.accumulo.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
+      private String principal;
+      private String tableNamespace;
+      private byte permission;
+      public grantTableNamespacePermission_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission, org.apache.thrift.async.AsyncMethodCallback<grantTableNamespacePermission_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.principal = principal;
+        this.tableNamespace = tableNamespace;
+        this.permission = permission;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("grantTableNamespacePermission", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        grantTableNamespacePermission_args args = new grantTableNamespacePermission_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setPrincipal(principal);
+        args.setTableNamespace(tableNamespace);
+        args.setPermission(permission);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_grantTableNamespacePermission();
+      }
+    }
+
+    public void revokeTableNamespacePermission(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission, org.apache.thrift.async.AsyncMethodCallback<revokeTableNamespacePermission_call> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      revokeTableNamespacePermission_call method_call = new revokeTableNamespacePermission_call(tinfo, credentials, principal, tableNamespace, permission, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class revokeTableNamespacePermission_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private org.apache.accumulo.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
+      private String principal;
+      private String tableNamespace;
+      private byte permission;
+      public revokeTableNamespacePermission_call(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, String principal, String tableNamespace, byte permission, org.apache.thrift.async.AsyncMethodCallback<revokeTableNamespacePermission_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.principal = principal;
+        this.tableNamespace = tableNamespace;
+        this.permission = permission;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("revokeTableNamespacePermission", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        revokeTableNamespacePermission_args args = new revokeTableNamespacePermission_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setPrincipal(principal);
+        args.setTableNamespace(tableNamespace);
+        args.setPermission(permission);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws ThriftSecurityException, ThriftTableOperationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_revokeTableNamespacePermission();
+      }
+    }
+
     public void getConfiguration(org.apache.accumulo.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, ConfigurationType type, org.apache.thrift.async.AsyncMethodCallback<getConfiguration_call> resultHandler) throws org.apache.thrift.TException {
       checkReady();
       getConfiguration_call method_call = new getConfiguration_call(tinfo, credentials, type, resultHandler, this, ___protocolFactory, ___transport);
@@ -2014,10 +2251,13 @@ import org.slf4j.LoggerFactory;
       processMap.put("getUserAuthorizations", new getUserAuthorizations());
       processMap.put("hasSystemPermission", new hasSystemPermission());
       processMap.put("hasTablePermission", new hasTablePermission());
+      processMap.put("hasTableNamespacePermission", new hasTableNamespacePermission());
       processMap.put("grantSystemPermission", new grantSystemPermission());
       processMap.put("revokeSystemPermission", new revokeSystemPermission());
       processMap.put("grantTablePermission", new grantTablePermission());
       processMap.put("revokeTablePermission", new revokeTablePermission());
+      processMap.put("grantTableNamespacePermission", new grantTableNamespacePermission());
+      processMap.put("revokeTableNamespacePermission", new revokeTableNamespacePermission());
       processMap.put("getConfiguration", new getConfiguration());
       processMap.put("getTableConfiguration", new getTableConfiguration());
       processMap.put("getTableNamespaceConfiguration", new getTableNamespaceConfiguration());
@@ -2430,6 +2670,33 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public static class hasTableNamespacePermission<I extends Iface> extends org.apache.thrift.ProcessFunction<I, hasTableNamespacePermission_args> {
+      public hasTableNamespacePermission() {
+        super("hasTableNamespacePermission");
+      }
+
+      public hasTableNamespacePermission_args getEmptyArgsInstance() {
+        return new hasTableNamespacePermission_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public hasTableNamespacePermission_result getResult(I iface, hasTableNamespacePermission_args args) throws org.apache.thrift.TException {
+        hasTableNamespacePermission_result result = new hasTableNamespacePermission_result();
+        try {
+          result.success = iface.hasTableNamespacePermission(args.tinfo, args.credentials, args.principal, args.tableNamespace, args.tblNspcPerm);
+          result.setSuccessIsSet(true);
+        } catch (ThriftSecurityException sec) {
+          result.sec = sec;
+        } catch (ThriftTableOperationException tope) {
+          result.tope = tope;
+        }
+        return result;
+      }
+    }
+
     public static class grantSystemPermission<I extends Iface> extends org.apache.thrift.ProcessFunction<I, grantSystemPermission_args> {
       public grantSystemPermission() {
         super("grantSystemPermission");
@@ -2530,6 +2797,58 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public static class grantTableNamespacePermission<I extends Iface> extends org.apache.thrift.ProcessFunction<I, grantTableNamespacePermission_args> {
+      public grantTableNamespacePermission() {
+        super("grantTableNamespacePermission");
+      }
+
+      public grantTableNamespacePermission_args getEmptyArgsInstance() {
+        return new grantTableNamespacePermission_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public grantTableNamespacePermission_result getResult(I iface, grantTableNamespacePermission_args args) throws org.apache.thrift.TException {
+        grantTableNamespacePermission_result result = new grantTableNamespacePermission_result();
+        try {
+          iface.grantTableNamespacePermission(args.tinfo, args.credentials, args.principal, args.tableNamespace, args.permission);
+        } catch (ThriftSecurityException sec) {
+          result.sec = sec;
+        } catch (ThriftTableOperationException tope) {
+          result.tope = tope;
+        }
+        return result;
+      }
+    }
+
+    public static class revokeTableNamespacePermission<I extends Iface> extends org.apache.thrift.ProcessFunction<I, revokeTableNamespacePermission_args> {
+      public revokeTableNamespacePermission() {
+        super("revokeTableNamespacePermission");
+      }
+
+      public revokeTableNamespacePermission_args getEmptyArgsInstance() {
+        return new revokeTableNamespacePermission_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public revokeTableNamespacePermission_result getResult(I iface, revokeTableNamespacePermission_args args) throws org.apache.thrift.TException {
+        revokeTableNamespacePermission_result result = new revokeTableNamespacePermission_result();
+        try {
+          iface.revokeTableNamespacePermission(args.tinfo, args.credentials, args.principal, args.tableNamespace, args.permission);
+        } catch (ThriftSecurityException sec) {
+          result.sec = sec;
+        } catch (ThriftTableOperationException tope) {
+          result.tope = tope;
+        }
+        return result;
+      }
+    }
+
     public static class getConfiguration<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getConfiguration_args> {
       public getConfiguration() {
         super("getConfiguration");
@@ -19227,31 +19546,34 @@ import org.slf4j.LoggerFactory;
 
   }
 
-  public static class grantSystemPermission_args implements org.apache.thrift.TBase<grantSystemPermission_args, grantSystemPermission_args._Fields>, java.io.Serializable, Cloneable   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("grantSystemPermission_args");
+  public static class hasTableNamespacePermission_args implements org.apache.thrift.TBase<hasTableNamespacePermission_args, hasTableNamespacePermission_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("hasTableNamespacePermission_args");
 
-    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)4);
-    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)5);
-    private static final org.apache.thrift.protocol.TField PRINCIPAL_FIELD_DESC = new org.apache.thrift.protocol.TField("principal", org.apache.thrift.protocol.TType.STRING, (short)2);
-    private static final org.apache.thrift.protocol.TField PERMISSION_FIELD_DESC = new org.apache.thrift.protocol.TField("permission", org.apache.thrift.protocol.TType.BYTE, (short)3);
+    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+    private static final org.apache.thrift.protocol.TField PRINCIPAL_FIELD_DESC = new org.apache.thrift.protocol.TField("principal", org.apache.thrift.protocol.TType.STRING, (short)3);
+    private static final org.apache.thrift.protocol.TField TABLE_NAMESPACE_FIELD_DESC = new org.apache.thrift.protocol.TField("tableNamespace", org.apache.thrift.protocol.TType.STRING, (short)4);
+    private static final org.apache.thrift.protocol.TField TBL_NSPC_PERM_FIELD_DESC = new org.apache.thrift.protocol.TField("tblNspcPerm", org.apache.thrift.protocol.TType.BYTE, (short)5);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new grantSystemPermission_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new grantSystemPermission_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new hasTableNamespacePermission_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new hasTableNamespacePermission_argsTupleSchemeFactory());
     }
 
     public org.apache.accumulo.trace.thrift.TInfo tinfo; // required
     public org.apache.accumulo.core.security.thrift.TCredentials credentials; // required
     public String principal; // required
-    public byte permission; // required
+    public String tableNamespace; // required
+    public byte tblNspcPerm; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      TINFO((short)4, "tinfo"),
-      CREDENTIALS((short)5, "credentials"),
-      PRINCIPAL((short)2, "principal"),
-      PERMISSION((short)3, "permission");
+      TINFO((short)1, "tinfo"),
+      CREDENTIALS((short)2, "credentials"),
+      PRINCIPAL((short)3, "principal"),
+      TABLE_NAMESPACE((short)4, "tableNamespace"),
+      TBL_NSPC_PERM((short)5, "tblNspcPerm");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -19266,14 +19588,16 @@ import org.slf4j.LoggerFactory;
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 4: // TINFO
+          case 1: // TINFO
             return TINFO;
-          case 5: // CREDENTIALS
+          case 2: // CREDENTIALS
             return CREDENTIALS;
-          case 2: // PRINCIPAL
+          case 3: // PRINCIPAL
             return PRINCIPAL;
-          case 3: // PERMISSION
-            return PERMISSION;
+          case 4: // TABLE_NAMESPACE
+            return TABLE_NAMESPACE;
+          case 5: // TBL_NSPC_PERM
+            return TBL_NSPC_PERM;
           default:
             return null;
         }
@@ -19314,7 +19638,7 @@ import org.slf4j.LoggerFactory;
     }
 
     // isset id assignments
-    private static final int __PERMISSION_ISSET_ID = 0;
+    private static final int __TBLNSPCPERM_ISSET_ID = 0;
     private byte __isset_bitfield = 0;
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
@@ -19325,33 +19649,37 @@ import org.slf4j.LoggerFactory;
           new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.security.thrift.TCredentials.class)));
       tmpMap.put(_Fields.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.PERMISSION, new org.apache.thrift.meta_data.FieldMetaData("permission", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+      tmpMap.put(_Fields.TABLE_NAMESPACE, new org.apache.thrift.meta_data.FieldMetaData("tableNamespace", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.TBL_NSPC_PERM, new org.apache.thrift.meta_data.FieldMetaData("tblNspcPerm", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BYTE)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(grantSystemPermission_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(hasTableNamespacePermission_args.class, metaDataMap);
     }
 
-    public grantSystemPermission_args() {
+    public hasTableNamespacePermission_args() {
     }
 
-    public grantSystemPermission_args(
+    public hasTableNamespacePermission_args(
       org.apache.accumulo.trace.thrift.TInfo tinfo,
       org.apache.accumulo.core.security.thrift.TCredentials credentials,
       String principal,
-      byte permission)
+      String tableNamespace,
+      byte tblNspcPerm)
     {
       this();
       this.tinfo = tinfo;
       this.credentials = credentials;
       this.principal = principal;
-      this.permission = permission;
-      setPermissionIsSet(true);
+      this.tableNamespace = tableNamespace;
+      this.tblNspcPerm = tblNspcPerm;
+      setTblNspcPermIsSet(true);
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public grantSystemPermission_args(grantSystemPermission_args other) {
+    public hasTableNamespacePermission_args(hasTableNamespacePermission_args other) {
       __isset_bitfield = other.__isset_bitfield;
       if (other.isSetTinfo()) {
         this.tinfo = new org.apache.accumulo.trace.thrift.TInfo(other.tinfo);
@@ -19362,11 +19690,3370 @@ import org.slf4j.LoggerFactory;
       if (other.isSetPrincipal()) {
         this.principal = other.principal;
       }
-      this.permission = other.permission;
+      if (other.isSetTableNamespace()) {
+        this.tableNamespace = other.tableNamespace;
+      }
+      this.tblNspcPerm = other.tblNspcPerm;
+    }
+
+    public hasTableNamespacePermission_args deepCopy() {
+      return new hasTableNamespacePermission_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.principal = null;
+      this.tableNamespace = null;
+      setTblNspcPermIsSet(false);
+      this.tblNspcPerm = 0;
+    }
+
+    public org.apache.accumulo.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public hasTableNamespacePermission_args setTinfo(org.apache.accumulo.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    public org.apache.accumulo.core.security.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public hasTableNamespacePermission_args setCredentials(org.apache.accumulo.core.security.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    public String getPrincipal() {
+      return this.principal;
+    }
+
+    public hasTableNamespacePermission_args setPrincipal(String principal) {
+      this.principal = principal;
+      return this;
+    }
+
+    public void unsetPrincipal() {
+      this.principal = null;
+    }
+
+    /** Returns true if field principal is set (has been assigned a value) and false otherwise */
+    public boolean isSetPrincipal() {
+      return this.principal != null;
+    }
+
+    public void setPrincipalIsSet(boolean value) {
+      if (!value) {
+        this.principal = null;
+      }
+    }
+
+    public String getTableNamespace() {
+      return this.tableNamespace;
+    }
+
+    public hasTableNamespacePermission_args setTableNamespace(String tableNamespace) {
+      this.tableNamespace = tableNamespace;
+      return this;
+    }
+
+    public void unsetTableNamespace() {
+      this.tableNamespace = null;
+    }
+
+    /** Returns true if field tableNamespace is set (has been assigned a value) and false otherwise */
+    public boolean isSetTableNamespace() {
+      return this.tableNamespace != null;
+    }
+
+    public void setTableNamespaceIsSet(boolean value) {
+      if (!value) {
+        this.tableNamespace = null;
+      }
+    }
+
+    public byte getTblNspcPerm() {
+      return this.tblNspcPerm;
+    }
+
+    public hasTableNamespacePermission_args setTblNspcPerm(byte tblNspcPerm) {
+      this.tblNspcPerm = tblNspcPerm;
+      setTblNspcPermIsSet(true);
+      return this;
+    }
+
+    public void unsetTblNspcPerm() {
+      __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TBLNSPCPERM_ISSET_ID);
+    }
+
+    /** Returns true if field tblNspcPerm is set (has been assigned a value) and false otherwise */
+    public boolean isSetTblNspcPerm() {
+      return EncodingUtils.testBit(__isset_bitfield, __TBLNSPCPERM_ISSET_ID);
+    }
+
+    public void setTblNspcPermIsSet(boolean value) {
+      __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TBLNSPCPERM_ISSET_ID, value);
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.security.thrift.TCredentials)value);
+        }
+        break;
+
+      case PRINCIPAL:
+        if (value == null) {
+          unsetPrincipal();
+        } else {
+          setPrincipal((String)value);
+        }
+        break;
+
+      case TABLE_NAMESPACE:
+        if (value == null) {
+          unsetTableNamespace();
+        } else {
+          setTableNamespace((String)value);
+        }
+        break;
+
+      case TBL_NSPC_PERM:
+        if (value == null) {
+          unsetTblNspcPerm();
+        } else {
+          setTblNspcPerm((Byte)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      case PRINCIPAL:
+        return getPrincipal();
+
+      case TABLE_NAMESPACE:
+        return getTableNamespace();
+
+      case TBL_NSPC_PERM:
+        return Byte.valueOf(getTblNspcPerm());
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case PRINCIPAL:
+        return isSetPrincipal();
+      case TABLE_NAMESPACE:
+        return isSetTableNamespace();
+      case TBL_NSPC_PERM:
+        return isSetTblNspcPerm();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof hasTableNamespacePermission_args)
+        return this.equals((hasTableNamespacePermission_args)that);
+      return false;
+    }
+
+    public boolean equals(hasTableNamespacePermission_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      boolean this_present_principal = true && this.isSetPrincipal();
+      boolean that_present_principal = true && that.isSetPrincipal();
+      if (this_present_principal || that_present_principal) {
+        if (!(this_present_principal && that_present_principal))
+          return false;
+        if (!this.principal.equals(that.principal))
+          return false;
+      }
+
+      boolean this_present_tableNamespace = true && this.isSetTableNamespace();
+      boolean that_present_tableNamespace = true && that.isSetTableNamespace();
+      if (this_present_tableNamespace || that_present_tableNamespace) {
+        if (!(this_present_tableNamespace && that_present_tableNamespace))
+          return false;
+        if (!this.tableNamespace.equals(that.tableNamespace))
+          return false;
+      }
+
+      boolean this_present_tblNspcPerm = true;
+      boolean that_present_tblNspcPerm = true;
+      if (this_present_tblNspcPerm || that_present_tblNspcPerm) {
+        if (!(this_present_tblNspcPerm && that_present_tblNspcPerm))
+          return false;
+        if (this.tblNspcPerm != that.tblNspcPerm)
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return 0;
+    }
+
+    public int compareTo(hasTableNamespacePermission_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      hasTableNamespacePermission_args typedOther = (hasTableNamespacePermission_args)other;
+
+      lastComparison = Boolean.valueOf(isSetTinfo()).compareTo(typedOther.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, typedOther.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetCredentials()).compareTo(typedOther.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, typedOther.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetPrincipal()).compareTo(typedOther.isSetPrincipal());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetPrincipal()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.principal, typedOther.principal);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetTableNamespace()).compareTo(typedOther.isSetTableNamespace());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTableNamespace()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableNamespace, typedOther.tableNamespace);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetTblNspcPerm()).compareTo(typedOther.isSetTblNspcPerm());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTblNspcPerm()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tblNspcPerm, typedOther.tblNspcPerm);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("hasTableNamespacePermission_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("principal:");
+      if (this.principal == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.principal);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("tableNamespace:");
+      if (this.tableNamespace == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tableNamespace);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("tblNspcPerm:");
+      sb.append(this.tblNspcPerm);
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bitfield = 0;
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class hasTableNamespacePermission_argsStandardSchemeFactory implements SchemeFactory {
+      public hasTableNamespacePermission_argsStandardScheme getScheme() {
+        return new hasTableNamespacePermission_argsStandardScheme();
+      }
+    }
+
+    private static class hasTableNamespacePermission_argsStandardScheme extends StandardScheme<hasTableNamespacePermission_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, hasTableNamespacePermission_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.trace.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // CREDENTIALS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.credentials = new org.apache.accumulo.core.security.thrift.TCredentials();
+                struct.credentials.read(iprot);
+                struct.setCredentialsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // PRINCIPAL
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.principal = iprot.readString();
+                struct.setPrincipalIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 4: // TABLE_NAMESPACE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.tableNamespace = iprot.readString();
+                struct.setTableNamespaceIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 5: // TBL_NSPC_PERM
+              if (schemeField.type == org.apache.thrift.protocol.TType.BYTE) {
+                struct.tblNspcPerm = iprot.readByte();
+                struct.setTblNspcPermIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, hasTableNamespacePermission_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.principal != null) {
+          oprot.writeFieldBegin(PRINCIPAL_FIELD_DESC);
+          oprot.writeString(struct.principal);
+          oprot.writeFieldEnd();
+        }
+        if (struct.tableNamespace != null) {
+          oprot.writeFieldBegin(TABLE_NAMESPACE_FIELD_DESC);
+          oprot.writeString(struct.tableNamespace);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldBegin(TBL_NSPC_PERM_FIELD_DESC);
+        oprot.writeByte(struct.tblNspcPerm);
+        oprot.writeFieldEnd();
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class hasTableNamespacePermission_argsTupleSchemeFactory implements SchemeFactory {
+      public hasTableNamespacePermission_argsTupleScheme getScheme() {
+        return new hasTableNamespacePermission_argsTupleScheme();
+      }
+    }
+
+    private static class hasTableNamespacePermission_argsTupleScheme extends TupleScheme<hasTableNamespacePermission_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, hasTableNamespacePermission_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        if (struct.isSetPrincipal()) {
+          optionals.set(2);
+        }
+        if (struct.isSetTableNamespace()) {
+          optionals.set(3);
+        }
+        if (struct.isSetTblNspcPerm()) {
+          optionals.set(4);
+        }
+        oprot.writeBitSet(optionals, 5);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetPrincipal()) {
+          oprot.writeString(struct.principal);
+        }
+        if (struct.isSetTableNamespace()) {
+          oprot.writeString(struct.tableNamespace);
+        }
+        if (struct.isSetTblNspcPerm()) {
+          oprot.writeByte(struct.tblNspcPerm);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, hasTableNamespacePermission_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(5);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.trace.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.credentials = new org.apache.accumulo.core.security.thrift.TCredentials();
+          struct.credentials.read(iprot);
+          struct.setCredentialsIsSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.principal = iprot.readString();
+          struct.setPrincipalIsSet(true);
+        }
+        if (incoming.get(3)) {
+          struct.tableNamespace = iprot.readString();
+          struct.setTableNamespaceIsSet(true);
+        }
+        if (incoming.get(4)) {
+          struct.tblNspcPerm = iprot.readByte();
+          struct.setTblNspcPermIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class hasTableNamespacePermission_result implements org.apache.thrift.TBase<hasTableNamespacePermission_result, hasTableNamespacePermission_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("hasTableNamespacePermission_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.BOOL, (short)0);
+    private static final org.apache.thrift.protocol.TField SEC_FIELD_DESC = new org.apache.thrift.protocol.TField("sec", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField TOPE_FIELD_DESC = new org.apache.thrift.protocol.TField("tope", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new hasTableNamespacePermission_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new hasTableNamespacePermission_resultTupleSchemeFactory());
+    }
+
+    public boolean success; // required
+    public ThriftSecurityException sec; // required
+    public ThriftTableOperationException tope; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success"),
+      SEC((short)1, "sec"),
+      TOPE((short)2, "tope");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          case 1: // SEC
+            return SEC;
+          case 2: // TOPE
+            return TOPE;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    private static final int __SUCCESS_ISSET_ID = 0;
+    private byte __isset_bitfield = 0;
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+      tmpMap.put(_Fields.SEC, new org.apache.thrift.meta_data.FieldMetaData("sec", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      tmpMap.put(_Fields.TOPE, new org.apache.thrift.meta_data.FieldMetaData("tope", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(hasTableNamespacePermission_result.class, metaDataMap);
+    }
+
+    public hasTableNamespacePermission_result() {
+    }
+
+    public hasTableNamespacePermission_result(
+      boolean success,
+      ThriftSecurityException sec,
+      ThriftTableOperationException tope)
+    {
+      this();
+      this.success = success;
+      setSuccessIsSet(true);
+      this.sec = sec;
+      this.tope = tope;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public hasTableNamespacePermission_result(hasTableNamespacePermission_result other) {
+      __isset_bitfield = other.__isset_bitfield;
+      this.success = other.success;
+      if (other.isSetSec()) {
+        this.sec = new ThriftSecurityException(other.sec);
+      }
+      if (other.isSetTope()) {
+        this.tope = new ThriftTableOperationException(other.tope);
+      }
+    }
+
+    public hasTableNamespacePermission_result deepCopy() {
+      return new hasTableNamespacePermission_result(this);
+    }
+
+    @Override
+    public void clear() {
+      setSuccessIsSet(false);
+      this.success = false;
+      this.sec = null;
+      this.tope = null;
+    }
+
+    public boolean isSuccess() {
+      return this.success;
+    }
+
+    public hasTableNamespacePermission_result setSuccess(boolean success) {
+      this.success = success;
+      setSuccessIsSet(true);
+      return this;
+    }
+
+    public void unsetSuccess() {
+      __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return EncodingUtils.testBit(__isset_bitfield, __SUCCESS_ISSET_ID);
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUCCESS_ISSET_ID, value);
+    }
+
+    public ThriftSecurityException getSec() {
+      return this.sec;
+    }
+
+    public hasTableNamespacePermission_result setSec(ThriftSecurityException sec) {
+      this.sec = sec;
+      return this;
+    }
+
+    public void unsetSec() {
+      this.sec = null;
+    }
+
+    /** Returns true if field sec is set (has been assigned a value) and false otherwise */
+    public boolean isSetSec() {
+      return this.sec != null;
+    }
+
+    public void setSecIsSet(boolean value) {
+      if (!value) {
+        this.sec = null;
+      }
+    }
+
+    public ThriftTableOperationException getTope() {
+      return this.tope;
+    }
+
+    public hasTableNamespacePermission_result setTope(ThriftTableOperationException tope) {
+      this.tope = tope;
+      return this;
+    }
+
+    public void unsetTope() {
+      this.tope = null;
+    }
+
+    /** Returns true if field tope is set (has been assigned a value) and false otherwise */
+    public boolean isSetTope() {
+      return this.tope != null;
+    }
+
+    public void setTopeIsSet(boolean value) {
+      if (!value) {
+        this.tope = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((Boolean)value);
+        }
+        break;
+
+      case SEC:
+        if (value == null) {
+          unsetSec();
+        } else {
+          setSec((ThriftSecurityException)value);
+        }
+        break;
+
+      case TOPE:
+        if (value == null) {
+          unsetTope();
+        } else {
+          setTope((ThriftTableOperationException)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return Boolean.valueOf(isSuccess());
+
+      case SEC:
+        return getSec();
+
+      case TOPE:
+        return getTope();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      case SEC:
+        return isSetSec();
+      case TOPE:
+        return isSetTope();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof hasTableNamespacePermission_result)
+        return this.equals((hasTableNamespacePermission_result)that);
+      return false;
+    }
+
+    public boolean equals(hasTableNamespacePermission_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true;
+      boolean that_present_success = true;
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (this.success != that.success)
+          return false;
+      }
+
+      boolean this_present_sec = true && this.isSetSec();
+      boolean that_present_sec = true && that.isSetSec();
+      if (this_present_sec || that_present_sec) {
+        if (!(this_present_sec && that_present_sec))
+          return false;
+        if (!this.sec.equals(that.sec))
+          return false;
+      }
+
+      boolean this_present_tope = true && this.isSetTope();
+      boolean that_present_tope = true && that.isSetTope();
+      if (this_present_tope || that_present_tope) {
+        if (!(this_present_tope && that_present_tope))
+          return false;
+        if (!this.tope.equals(that.tope))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return 0;
+    }
+
+    public int compareTo(hasTableNamespacePermission_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      hasTableNamespacePermission_result typedOther = (hasTableNamespacePermission_result)other;
+
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetSec()).compareTo(typedOther.isSetSec());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSec()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sec, typedOther.sec);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetTope()).compareTo(typedOther.isSetTope());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTope()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tope, typedOther.tope);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("hasTableNamespacePermission_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      sb.append(this.success);
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("sec:");
+      if (this.sec == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.sec);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("tope:");
+      if (this.tope == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tope);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bitfield = 0;
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class hasTableNamespacePermission_resultStandardSchemeFactory implements SchemeFactory {
+      public hasTableNamespacePermission_resultStandardScheme getScheme() {
+        return new hasTableNamespacePermission_resultStandardScheme();
+      }
+    }
+
+    private static class hasTableNamespacePermission_resultStandardScheme extends StandardScheme<hasTableNamespacePermission_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, hasTableNamespacePermission_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+                struct.success = iprot.readBool();
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // SEC
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.sec = new ThriftSecurityException();
+                struct.sec.read(iprot);
+                struct.setSecIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // TOPE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tope = new ThriftTableOperationException();
+                struct.tope.read(iprot);
+                struct.setTopeIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, hasTableNamespacePermission_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.isSetSuccess()) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          oprot.writeBool(struct.success);
+          oprot.writeFieldEnd();
+        }
+        if (struct.sec != null) {
+          oprot.writeFieldBegin(SEC_FIELD_DESC);
+          struct.sec.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.tope != null) {
+          oprot.writeFieldBegin(TOPE_FIELD_DESC);
+          struct.tope.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class hasTableNamespacePermission_resultTupleSchemeFactory implements SchemeFactory {
+      public hasTableNamespacePermission_resultTupleScheme getScheme() {
+        return new hasTableNamespacePermission_resultTupleScheme();
+      }
+    }
+
+    private static class hasTableNamespacePermission_resultTupleScheme extends TupleScheme<hasTableNamespacePermission_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, hasTableNamespacePermission_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        if (struct.isSetSec()) {
+          optionals.set(1);
+        }
+        if (struct.isSetTope()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.isSetSuccess()) {
+          oprot.writeBool(struct.success);
+        }
+        if (struct.isSetSec()) {
+          struct.sec.write(oprot);
+        }
+        if (struct.isSetTope()) {
+          struct.tope.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, hasTableNamespacePermission_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(3);
+        if (incoming.get(0)) {
+          struct.success = iprot.readBool();
+          struct.setSuccessIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.sec = new ThriftSecurityException();
+          struct.sec.read(iprot);
+          struct.setSecIsSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.tope = new ThriftTableOperationException();
+          struct.tope.read(iprot);
+          struct.setTopeIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class grantSystemPermission_args implements org.apache.thrift.TBase<grantSystemPermission_args, grantSystemPermission_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("grantSystemPermission_args");
+
+    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)4);
+    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)5);
+    private static final org.apache.thrift.protocol.TField PRINCIPAL_FIELD_DESC = new org.apache.thrift.protocol.TField("principal", org.apache.thrift.protocol.TType.STRING, (short)2);
+    private static final org.apache.thrift.protocol.TField PERMISSION_FIELD_DESC = new org.apache.thrift.protocol.TField("permission", org.apache.thrift.protocol.TType.BYTE, (short)3);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new grantSystemPermission_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new grantSystemPermission_argsTupleSchemeFactory());
+    }
+
+    public org.apache.accumulo.trace.thrift.TInfo tinfo; // required
+    public org.apache.accumulo.core.security.thrift.TCredentials credentials; // required
+    public String principal; // required
+    public byte permission; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      TINFO((short)4, "tinfo"),
+      CREDENTIALS((short)5, "credentials"),
+      PRINCIPAL((short)2, "principal"),
+      PERMISSION((short)3, "permission");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 4: // TINFO
+            return TINFO;
+          case 5: // CREDENTIALS
+            return CREDENTIALS;
+          case 2: // PRINCIPAL
+            return PRINCIPAL;
+          case 3: // PERMISSION
+            return PERMISSION;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    private static final int __PERMISSION_ISSET_ID = 0;
+    private byte __isset_bitfield = 0;
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.trace.thrift.TInfo.class)));
+      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.security.thrift.TCredentials.class)));
+      tmpMap.put(_Fields.PRINCIPAL, new org.apache.thrift.meta_data.FieldMetaData("principal", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.PERMISSION, new org.apache.thrift.meta_data.FieldMetaData("permission", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BYTE)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(grantSystemPermission_args.class, metaDataMap);
+    }
+
+    public grantSystemPermission_args() {
+    }
+
+    public grantSystemPermission_args(
+      org.apache.accumulo.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.security.thrift.TCredentials credentials,
+      String principal,
+      byte permission)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.principal = principal;
+      this.permission = permission;
+      setPermissionIsSet(true);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public grantSystemPermission_args(grantSystemPermission_args other) {
+      __isset_bitfield = other.__isset_bitfield;
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.trace.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.security.thrift.TCredentials(other.credentials);
+      }
+      if (other.isSetPrincipal()) {
+        this.principal = other.principal;
+      }
+      this.permission = other.permission;
+    }
+
+    public grantSystemPermission_args deepCopy() {
+      return new grantSystemPermission_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.principal = null;
+      setPermissionIsSet(false);
+      this.permission = 0;
+    }
+
+    public org.apache.accumulo.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public grantSystemPermission_args setTinfo(org.apache.accumulo.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    public org.apache.accumulo.core.security.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public grantSystemPermission_args setCredentials(org.apache.accumulo.core.security.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    public String getPrincipal() {
+      return this.principal;
+    }
+
+    public grantSystemPermission_args setPrincipal(String principal) {
+      this.principal = principal;
+      return this;
+    }
+
+    public void unsetPrincipal() {
+      this.principal = null;
+    }
+
+    /** Returns true if field principal is set (has been assigned a value) and false otherwise */
+    public boolean isSetPrincipal() {
+      return this.principal != null;
+    }
+
+    public void setPrincipalIsSet(boolean value) {
+      if (!value) {
+        this.principal = null;
+      }
+    }
+
+    public byte getPermission() {
+      return this.permission;
+    }
+
+    public grantSystemPermission_args setPermission(byte permission) {
+      this.permission = permission;
+      setPermissionIsSet(true);
+      return this;
+    }
+
+    public void unsetPermission() {
+      __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PERMISSION_ISSET_ID);
+    }
+
+    /** Returns true if field permission is set (has been assigned a value) and false otherwise */
+    public boolean isSetPermission() {
+      return EncodingUtils.testBit(__isset_bitfield, __PERMISSION_ISSET_ID);
+    }
+
+    public void setPermissionIsSet(boolean value) {
+      __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PERMISSION_ISSET_ID, value);
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.security.thrift.TCredentials)value);
+        }
+        break;
+
+      case PRINCIPAL:
+        if (value == null) {
+          unsetPrincipal();
+        } else {
+          setPrincipal((String)value);
+        }
+        break;
+
+      case PERMISSION:
+        if (value == null) {
+          unsetPermission();
+        } else {
+          setPermission((Byte)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      case PRINCIPAL:
+        return getPrincipal();
+
+      case PERMISSION:
+        return Byte.valueOf(getPermission());
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case PRINCIPAL:
+        return isSetPrincipal();
+      case PERMISSION:
+        return isSetPermission();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof grantSystemPermission_args)
+        return this.equals((grantSystemPermission_args)that);
+      return false;
+    }
+
+    public boolean equals(grantSystemPermission_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      boolean this_present_principal = true && this.isSetPrincipal();
+      boolean that_present_principal = true && that.isSetPrincipal();
+      if (this_present_principal || that_present_principal) {
+        if (!(this_present_principal && that_present_principal))
+          return false;
+        if (!this.principal.equals(that.principal))
+          return false;
+      }
+
+      boolean this_present_permission = true;
+      boolean that_present_permission = true;
+      if (this_present_permission || that_present_permission) {
+        if (!(this_present_permission && that_present_permission))
+          return false;
+        if (this.permission != that.permission)
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return 0;
+    }
+
+    public int compareTo(grantSystemPermission_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      grantSystemPermission_args typedOther = (grantSystemPermission_args)other;
+
+      lastComparison = Boolean.valueOf(isSetTinfo()).compareTo(typedOther.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, typedOther.tinfo);
+

<TRUNCATED>

[37/50] [abbrv] ACCUMULO-1712 Rename system tables and namespaces

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index fa29135..fa127fb 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -1195,7 +1195,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       // processing the write from the dead client is still in progress,
       // the restarted client may not see the write unless we wait here.
       // this behavior is very important when the client is reading the
-      // !METADATA table
+      // metadata
       if (waitForWrites)
         writeTracker.waitForWrites(TabletType.type(extent));
 
@@ -2896,7 +2896,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
          * 
          * 1) The log recovery code does not handle data written to the tablet on multiple tablet servers. 2) The log recovery code does not block if memory is
          * full. Therefore recovering lots of tablets that use a lot of memory could run out of memory. 3) The minor compaction finish event did not make it to
-         * the logs (the file will be in !METADATA, preventing replay of compacted data)... but do not want a majc to wipe the file out from !METADATA and then
+         * the logs (the file will be in metadata, preventing replay of compacted data)... but do not want a majc to wipe the file out from metadata and then
          * have another process failure... this could cause duplicate data to replay
          */
         if (tablet.getNumEntriesInMemory() > 0 && !tablet.minorCompactNow(MinorCompactionReason.SYSTEM)) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/test/scale/deleteLargeTable.txt
----------------------------------------------------------------------
diff --git a/test/scale/deleteLargeTable.txt b/test/scale/deleteLargeTable.txt
index 97ba831..59dd648 100644
--- a/test/scale/deleteLargeTable.txt
+++ b/test/scale/deleteLargeTable.txt
@@ -13,4 +13,4 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-Create and delete a large table (do this with !METADATA split alot) see #2381
+Create and delete a large table (do this with accumulo.metadata split alot) see #2381

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
index 09ed966..a52651e 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
@@ -134,7 +134,7 @@ public class ZombieTServer {
     if (zlock.tryLock(lw, lockContent)) {
       log.debug("Obtained tablet server lock " + zlock.getLockPath());
     }
-    // modify !METADATA
+    // modify metadata
     synchronized (tch) {
       while (!tch.halted) {
         tch.wait();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/test/src/main/java/org/apache/accumulo/test/performance/metadata/MetadataBatchScanTest.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/metadata/MetadataBatchScanTest.java b/test/src/main/java/org/apache/accumulo/test/performance/metadata/MetadataBatchScanTest.java
index 1a314bf..449d732 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/metadata/MetadataBatchScanTest.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/metadata/MetadataBatchScanTest.java
@@ -48,7 +48,7 @@ import org.apache.hadoop.io.Text;
 import com.google.common.net.HostAndPort;
 
 /**
- * This little program can be used to write a lot of entries to the !METADATA table and measure the performance of varying numbers of threads doing !METADATA
+ * This little program can be used to write a lot of metadata entries and measure the performance of varying numbers of threads doing metadata
  * lookups using the batch scanner.
  * 
  * 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
index a26e69e..ce0370b 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
@@ -77,7 +77,7 @@ import com.beust.jcommander.Parameter;
 import com.google.common.net.HostAndPort;
 
 /**
- * The purpose of this class is to server as fake tserver that is a data sink like /dev/null. NullTserver modifies the !METADATA location entries for a table to
+ * The purpose of this class is to server as fake tserver that is a data sink like /dev/null. NullTserver modifies the metadata location entries for a table to
  * point to it. This allows thrift performance to be measured by running any client code that writes to a table.
  * 
  */
@@ -247,7 +247,7 @@ public class NullTserver {
     
     HostAndPort addr = HostAndPort.fromParts(InetAddress.getLocalHost().getHostName(), opts.port);
     
-    // modify !METADATA
+    // modify metadata
     ZooKeeperInstance zki = new ZooKeeperInstance(new ClientConfiguration().withInstance(opts.iname).withZkHosts(opts.keepers));
     String tableId = Tables.getTableId(zki, opts.tableName);
     

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/test/src/test/java/org/apache/accumulo/test/DumpConfigIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/DumpConfigIT.java b/test/src/test/java/org/apache/accumulo/test/DumpConfigIT.java
index 349f709..39df878 100644
--- a/test/src/test/java/org/apache/accumulo/test/DumpConfigIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/DumpConfigIT.java
@@ -57,8 +57,8 @@ public class DumpConfigIT extends ConfigurableMacIT {
     assertTrue(meta.contains(Property.TABLE_FILE_REPLICATION.getKey()));
     String systemPerm = FunctionalTestUtils.readAll(new FileInputStream(new File(folder.getRoot(), "system_perm.cfg")));
     assertTrue(systemPerm.contains("grant System.ALTER_USER -s -u root"));
-    String metaPerm = FunctionalTestUtils.readAll(new FileInputStream(new File(folder.getRoot(), "!METADATA_perm.cfg")));
-    assertTrue(metaPerm.contains("grant Table.READ -t !METADATA -u root"));
-    assertFalse(metaPerm.contains("grant Table.DROP -t !METADATA -u root"));
+    String metaPerm = FunctionalTestUtils.readAll(new FileInputStream(new File(folder.getRoot(), MetadataTable.NAME + "_perm.cfg")));
+    assertTrue(metaPerm.contains("grant Table.READ -t " + MetadataTable.NAME + " -u root"));
+    assertFalse(metaPerm.contains("grant Table.DROP -t " + MetadataTable.NAME + " -u root"));
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java b/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
index 8341762..f38c944 100644
--- a/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
@@ -514,14 +514,14 @@ public class NamespacesIT extends SimpleMacIT {
     assertTrue(c.instanceOperations().getSystemConfiguration().containsValue("20," + SimpleFilter.class.getName()));
 
     assertTrue(checkNamespaceHasProp(c, Constants.DEFAULT_NAMESPACE, "table.iterator.scan.sum", "20," + SimpleFilter.class.getName()));
-    assertTrue(!checkNamespaceHasProp(c, Constants.SYSTEM_NAMESPACE, "table.iterator.scan.sum", "20," + SimpleFilter.class.getName()));
+    assertTrue(!checkNamespaceHasProp(c, Constants.ACCUMULO_NAMESPACE, "table.iterator.scan.sum", "20," + SimpleFilter.class.getName()));
     c.instanceOperations().removeProperty("table.iterator.scan.sum");
 
     c.instanceOperations().setProperty("table.constraint.42", NumericValueConstraint.class.getName());
     assertTrue(c.instanceOperations().getSystemConfiguration().containsValue(NumericValueConstraint.class.getName()));
 
     assertTrue(checkNamespaceHasProp(c, Constants.DEFAULT_NAMESPACE, "table.constraint.42", NumericValueConstraint.class.getName()));
-    assertTrue(!checkNamespaceHasProp(c, Constants.SYSTEM_NAMESPACE, "table.constraint.42", NumericValueConstraint.class.getName()));
+    assertTrue(!checkNamespaceHasProp(c, Constants.ACCUMULO_NAMESPACE, "table.constraint.42", NumericValueConstraint.class.getName()));
     c.instanceOperations().removeProperty("table.constraint.42");
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
index 7951c80..4f0c14c 100644
--- a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
@@ -295,14 +295,14 @@ public class ShellServerIT extends SimpleMacIT {
     exec("createuser xyzzy", true);
     exec("users", true, "xyzzy", true);
     String perms = exec("userpermissions -u xyzzy", true);
-    assertTrue(perms.contains("Table permissions (!METADATA): Table.READ"));
+    assertTrue(perms.contains("Table permissions (" + MetadataTable.NAME + "): Table.READ"));
     exec("grant -u xyzzy -s System.CREATE_TABLE", true);
     perms = exec("userpermissions -u xyzzy", true);
     assertTrue(perms.contains(""));
-    exec("grant -u root -t !METADATA Table.WRITE", true);
-    exec("grant -u root -t !METADATA Table.GOOFY", false);
+    exec("grant -u root -t " + MetadataTable.NAME + " Table.WRITE", true);
+    exec("grant -u root -t " + MetadataTable.NAME + " Table.GOOFY", false);
     exec("grant -u root -s foo", false);
-    exec("grant -u xyzzy -t !METADATA foo", false);
+    exec("grant -u xyzzy -t " + MetadataTable.NAME + " foo", false);
     input.set("secret\nsecret\n");
     exec("user xyzzy", true);
     exec("createtable t", true, "xyzzy@", true);
@@ -315,9 +315,9 @@ public class ShellServerIT extends SimpleMacIT {
     exec("revoke -u xyzzy -s System.CREATE_TABLE", true);
     exec("revoke -u xyzzy -s System.GOOFY", false);
     exec("revoke -u xyzzy -s foo", false);
-    exec("revoke -u xyzzy -t !METADATA Table.WRITE", true);
-    exec("revoke -u xyzzy -t !METADATA Table.GOOFY", false);
-    exec("revoke -u xyzzy -t !METADATA foo", false);
+    exec("revoke -u xyzzy -t " + MetadataTable.NAME + " Table.WRITE", true);
+    exec("revoke -u xyzzy -t " + MetadataTable.NAME + " Table.GOOFY", false);
+    exec("revoke -u xyzzy -t " + MetadataTable.NAME + " foo", false);
     exec("deleteuser xyzzy", true);
     exec("users", true, "xyzzy", false);
   }
@@ -486,7 +486,7 @@ public class ShellServerIT extends SimpleMacIT {
   @Test(timeout = 30 * 1000)
   public void constraint() throws Exception {
     // constraint
-    exec("constraint -l -t !METADATA", true, "MetadataConstraints=1", true);
+    exec("constraint -l -t " + MetadataTable.NAME + "", true, "MetadataConstraints=1", true);
     exec("createtable c -evc");
     exec("constraint -l -t c", true, "VisibilityConstraint=2", true);
     exec("constraint -t c -d 2", true, "Removed constraint 2 from table c");
@@ -672,12 +672,12 @@ public class ShellServerIT extends SimpleMacIT {
     exec("merge --all", true);
     exec("getsplits", true, "z", false);
     exec("deletetable -f t");
-    exec("getsplits -t !METADATA", true);
+    exec("getsplits -t " + MetadataTable.NAME + "", true);
     assertEquals(2, output.get().split("\n").length);
-    exec("getsplits -t !!ROOT", true);
+    exec("getsplits -t accumulo.root", true);
     assertEquals(1, output.get().split("\n").length);
-    exec("merge --all -t !METADATA");
-    exec("getsplits -t !METADATA", true);
+    exec("merge --all -t " + MetadataTable.NAME + "");
+    exec("getsplits -t " + MetadataTable.NAME + "", true);
     assertEquals(1, output.get().split("\n").length);
   }
 
@@ -949,7 +949,7 @@ public class ShellServerIT extends SimpleMacIT {
   }
 
   private int countFiles() throws IOException {
-    exec("scan -t !METADATA -np -c file");
+    exec("scan -t " + MetadataTable.NAME + " -np -c file");
     return output.get().split("\n").length - 1;
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/test/src/test/java/org/apache/accumulo/test/SplitRecoveryIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/SplitRecoveryIT.java b/test/src/test/java/org/apache/accumulo/test/SplitRecoveryIT.java
index 3482a1d..b53a724 100644
--- a/test/src/test/java/org/apache/accumulo/test/SplitRecoveryIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/SplitRecoveryIT.java
@@ -80,7 +80,7 @@ public class SplitRecoveryIT extends SimpleMacIT {
       while (!isOffline(tableName, connector))
         UtilWaitThread.sleep(200);
 
-      // poke a partial split into the !METADATA table
+      // poke a partial split into the metadata table
       connector.securityOperations().grantTablePermission("root", MetadataTable.NAME, TablePermission.WRITE);
       String tableId = connector.tableOperations().tableIdMap().get(tableName);
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/test/src/test/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java b/test/src/test/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
index 02f294f..df1a3f1 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
@@ -92,7 +92,7 @@ public class GarbageCollectorIT extends ConfigurableMacIT {
 
   @Test(timeout = 4 * 60 * 1000)
   public void gcLotsOfCandidatesIT() throws Exception {
-    log.info("Filling !METADATA table with bogus delete flags");
+    log.info("Filling metadata table with bogus delete flags");
     Connector c = getConnector();
     addEntries(c, new BatchWriterOpts());
     cluster.getConfig().setDefaultMemory(10, MemoryUnit.MEGABYTE);
@@ -105,7 +105,7 @@ public class GarbageCollectorIT extends ConfigurableMacIT {
 
   @Test(timeout = 20 * 60 * 1000)
   public void dontGCRootLog() throws Exception {
-    // dirty !METADATA
+    // dirty metadata
     Connector c = getConnector();
     String table = getTableNames(1)[0];
     c.tableOperations().create(table);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/test/system/bench/cloudstone1/cloudstone1.py
----------------------------------------------------------------------
diff --git a/test/system/bench/cloudstone1/cloudstone1.py b/test/system/bench/cloudstone1/cloudstone1.py
index cfd8f61..349bbdb 100755
--- a/test/system/bench/cloudstone1/cloudstone1.py
+++ b/test/system/bench/cloudstone1/cloudstone1.py
@@ -29,8 +29,8 @@ class CloudStone1(Benchmark):
                'and we can reach all the slaves. Lower is better.'
 
     def runTest(self):
-        code, out, err = cloudshell.run(self.username, self.password, 'table !METADATA\nscan\n')
-        self.assertEqual(code, 0, "Could not scan the !METADATA table. %s %s" % (out, err))
+        code, out, err = cloudshell.run(self.username, self.password, 'table accumulo.metadata\nscan\n')
+        self.assertEqual(code, 0, "Could not scan the metadata table. %s %s" % (out, err))
         results = runAll('echo help | %s shell' %
                          accumulo('bin', 'accumulo'))
                          


[20/50] [abbrv] git commit: ACCUMULO-802 fixed up some places in the code where a namespace might be null

Posted by ct...@apache.org.
ACCUMULO-802 fixed up some places in the code where a namespace might be null


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

Branch: refs/heads/master
Commit: 7b4f068e81b282469c106ef3c9e239d8f3d0e412
Parents: 5262bd5
Author: Sean Hickey <ta...@gmail.com>
Authored: Mon Aug 12 15:06:22 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:10 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/server/init/Initialize.java     |  4 ++--
 .../org/apache/accumulo/server/tables/TableManager.java |  1 +
 .../apache/accumulo/master/tableOps/RenameTable.java    | 12 ++++++------
 3 files changed, 9 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/7b4f068e/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 042811a..0e51b6c 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
@@ -420,9 +420,9 @@ public class Initialize {
     createInitialTableNamespace(zoo, zkInstanceRoot, Constants.SYSTEM_TABLE_NAMESPACE_ID, Constants.SYSTEM_TABLE_NAMESPACE, false);
     
     zoo.putPersistentData(zkInstanceRoot + Constants.ZTABLES + "/" + MetadataTable.ID + Constants.ZTABLE_NAMESPACE,
-        Constants.SYSTEM_TABLE_NAMESPACE_ID.getBytes(Constants.UTF8), NodeExistsPolicy.FAIL);
+        Constants.SYSTEM_TABLE_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.FAIL);
+        Constants.SYSTEM_TABLE_NAMESPACE_ID.getBytes(Constants.UTF8), NodeExistsPolicy.OVERWRITE);
   }
   
   private static void createInitialTableNamespace(IZooReaderWriter zoo, String root, String id, String namespace, boolean defaultOpts) throws KeeperException,

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7b4f068e/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 911459c..09bf92f 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,6 +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);
   }
   
   public synchronized static TableManager getInstance() {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7b4f068e/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
index 18b7532..8b82593 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
@@ -68,28 +68,28 @@ public class RenameTable extends MasterRepo {
       tm.addNamespaceToTable(tableId, namespaceId);
     }
 
-    newTableName = Tables.extractTableName(newTableName);
-    oldTableName = Tables.extractTableName(oldTableName);
-
     IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
 
     Utils.tableNameLock.lock();
     try {
       Utils.checkTableDoesNotExist(instance, newTableName, tableId, TableOperation.RENAME);
 
+      final String newName = Tables.extractTableName(newTableName);
+      final String oldName = Tables.extractTableName(oldTableName);
+
       final String tap = ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_NAME;
 
       zoo.mutate(tap, null, null, new Mutator() {
         @Override
         public byte[] mutate(byte[] current) throws Exception {
           final String currentName = new String(current);
-          if (currentName.equals(newTableName))
+          if (currentName.equals(newName))
             return null; // assume in this case the operation is running again, so we are done
-          if (!currentName.equals(oldTableName)) {
+          if (!currentName.equals(oldName)) {
             throw new ThriftTableOperationException(null, oldTableName, TableOperation.RENAME, TableOperationExceptionType.NOTFOUND,
                 "Name changed while processing");
           }
-          return newTableName.getBytes();
+          return newName.getBytes();
         }
       });
       Tables.clearCache(instance);


[50/50] [abbrv] git commit: Merge branch '1.6.0-SNAPSHOT'

Posted by ct...@apache.org.
Merge branch '1.6.0-SNAPSHOT'


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

Branch: refs/heads/master
Commit: 7b8922c70d0b30d0030052bcdd7db63ab57ccd02
Parents: c29df8e 859cf20
Author: Christopher Tubbs <ct...@apache.org>
Authored: Wed Dec 4 18:58:09 2013 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:58:09 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/core/Constants.java     |    62 +-
 .../apache/accumulo/core/client/Connector.java  |    40 +-
 .../core/client/NamespaceExistsException.java   |    67 +
 .../core/client/NamespaceNotEmptyException.java |    74 +
 .../core/client/NamespaceNotFoundException.java |    77 +
 .../core/client/admin/NamespaceOperations.java  |   370 +
 .../client/admin/NamespaceOperationsHelper.java |   203 +
 .../client/admin/NamespaceOperationsImpl.java   |   347 +
 .../core/client/admin/SecurityOperations.java   |    88 +-
 .../client/admin/SecurityOperationsImpl.java    |    83 +-
 .../core/client/admin/TableOperations.java      |   107 +-
 .../core/client/admin/TableOperationsImpl.java  |   450 +-
 .../core/client/impl/ConnectorImpl.java         |    10 +
 .../accumulo/core/client/impl/Namespaces.java   |   107 +
 .../accumulo/core/client/impl/Tables.java       |   112 +-
 .../client/impl/TabletServerBatchWriter.java    |     2 +-
 .../core/client/impl/thrift/ClientService.java  |  9196 +++++++++++--
 .../client/impl/thrift/SecurityErrorCode.java   |     5 +-
 .../accumulo/core/client/mock/MockAccumulo.java |    53 +-
 .../core/client/mock/MockConnector.java         |     6 +
 .../core/client/mock/MockNamespace.java         |    54 +
 .../client/mock/MockNamespaceOperations.java    |   130 +
 .../client/mock/MockSecurityOperations.java     |    83 +-
 .../accumulo/core/client/mock/MockTable.java    |    36 +-
 .../core/client/mock/MockTableOperations.java   |   133 +-
 .../core/client/security/SecurityErrorCode.java |     3 +-
 .../core/master/thrift/MasterClientService.java | 11700 ++++++++++++++---
 .../accumulo/core/metadata/MetadataTable.java   |     9 +-
 .../accumulo/core/metadata/RootTable.java       |    15 +-
 .../core/security/NamespacePermission.java      |    66 +
 .../core/security/SystemPermission.java         |     5 +-
 .../org/apache/accumulo/core/util/Merge.java    |     2 +-
 .../org/apache/accumulo/core/util/Pair.java     |    57 +-
 .../apache/accumulo/core/util/shell/Shell.java  |   340 +-
 .../core/util/shell/ShellCompletor.java         |    58 +-
 .../accumulo/core/util/shell/ShellOptions.java  |     3 +-
 .../core/util/shell/commands/ConfigCommand.java |   113 +-
 .../util/shell/commands/ConstraintCommand.java  |    84 +-
 .../shell/commands/CreateNamespaceCommand.java  |   108 +
 .../util/shell/commands/CreateTableCommand.java |    69 +-
 .../core/util/shell/commands/DUCommand.java     |    37 +-
 .../util/shell/commands/DeleteIterCommand.java  |    61 +-
 .../shell/commands/DeleteNamespaceCommand.java  |   100 +
 .../util/shell/commands/DeleteTableCommand.java |    12 +-
 .../core/util/shell/commands/GrantCommand.java  |    47 +-
 .../util/shell/commands/ListIterCommand.java    |    59 +-
 .../commands/NamespacePermissionsCommand.java   |    44 +
 .../util/shell/commands/NamespacesCommand.java  |    83 +
 .../core/util/shell/commands/OptUtil.java       |    56 +-
 .../shell/commands/RenameNamespaceCommand.java  |    79 +
 .../util/shell/commands/RenameTableCommand.java |    17 +-
 .../core/util/shell/commands/RevokeCommand.java |    45 +-
 .../util/shell/commands/SetIterCommand.java     |   149 +-
 .../util/shell/commands/TableOperation.java     |    56 +-
 .../core/util/shell/commands/TablesCommand.java |    88 +-
 .../shell/commands/UserPermissionsCommand.java  |    34 +-
 core/src/main/thrift/client.thrift              |    12 +-
 core/src/main/thrift/master.thrift              |    12 +-
 .../core/client/impl/TabletLocatorImplTest.java |    10 +-
 .../core/client/mock/MockNamespacesTest.java    |   315 +
 .../accumulo/core/util/shell/ShellTest.java     |    54 +-
 .../chapters/administration.tex                 |     2 +-
 .../accumulo_user_manual/chapters/shell.tex     |     8 +-
 .../chapters/table_configuration.tex            |     2 +-
 .../chapters/troubleshooting.tex                |    30 +-
 .../apache/accumulo/proxy/SimpleProxyIT.java    |     2 +-
 .../apache/accumulo/server/ServerConstants.java |     9 +-
 .../server/client/ClientServiceHandler.java     |   170 +-
 .../server/conf/NamespaceConfWatcher.java       |   107 +
 .../server/conf/NamespaceConfiguration.java     |   174 +
 .../server/conf/ServerConfiguration.java        |    75 +-
 .../server/conf/TableConfiguration.java         |    46 +-
 .../server/conf/TableParentConfiguration.java   |    39 +
 .../org/apache/accumulo/server/fs/FileRef.java  |     5 +-
 .../accumulo/server/fs/VolumeManager.java       |     2 +-
 .../apache/accumulo/server/init/Initialize.java |    26 +
 .../master/balancer/TableLoadBalancer.java      |    30 +-
 .../master/state/MetaDataTableScanner.java      |     2 +-
 .../security/AuditedSecurityOperation.java      |     2 +-
 .../server/security/SecurityOperation.java      |   411 +-
 .../security/handler/InsecurePermHandler.java   |    31 +
 .../security/handler/PermissionHandler.java     |    61 +-
 .../server/security/handler/ZKAuthorizor.java   |    41 +-
 .../server/security/handler/ZKPermHandler.java  |   209 +-
 .../server/security/handler/ZKSecurityTool.java |    21 +
 .../accumulo/server/tables/TableManager.java    |   121 +-
 .../accumulo/server/util/MetadataTableUtil.java |     4 +-
 .../accumulo/server/util/NamespacePropUtil.java |    60 +
 .../accumulo/server/util/TablePropUtil.java     |     2 +-
 .../accumulo/gc/SimpleGarbageCollector.java     |     8 +-
 .../java/org/apache/accumulo/master/Master.java |   307 +-
 .../accumulo/master/tableOps/BulkImport.java    |     2 +-
 .../master/tableOps/CancelCompactions.java      |    17 +-
 .../master/tableOps/ChangeTableState.java       |    26 +-
 .../accumulo/master/tableOps/CloneTable.java    |   116 +-
 .../accumulo/master/tableOps/CompactRange.java  |    95 +-
 .../master/tableOps/CreateNamespace.java        |   196 +
 .../accumulo/master/tableOps/CreateTable.java   |   134 +-
 .../master/tableOps/DeleteNamespace.java        |   104 +
 .../accumulo/master/tableOps/DeleteTable.java   |    19 +-
 .../accumulo/master/tableOps/ExportTable.java   |    10 +-
 .../accumulo/master/tableOps/ImportTable.java   |   268 +-
 .../master/tableOps/RenameNamespace.java        |    92 +
 .../accumulo/master/tableOps/RenameTable.java   |    54 +-
 .../accumulo/master/tableOps/TableRangeOp.java  |    56 +-
 .../apache/accumulo/master/tableOps/Utils.java  |    62 +-
 .../apache/accumulo/master/TestMergeState.java  |    52 +-
 .../src/main/resources/docs/bulkIngest.html     |     2 +-
 .../main/resources/docs/examples/README.bloom   |     9 +-
 .../main/resources/docs/examples/README.export  |     9 +-
 .../resources/docs/examples/README.visibility   |     4 +-
 .../org/apache/accumulo/tserver/Tablet.java     |    24 +-
 .../apache/accumulo/tserver/TabletServer.java   |     4 +-
 test/scale/deleteLargeTable.txt                 |     2 +-
 .../accumulo/test/functional/ZombieTServer.java |     2 +-
 .../metadata/MetadataBatchScanTest.java         |     2 +-
 .../test/performance/thrift/NullTserver.java    |     4 +-
 .../concurrent/ChangePermissions.java           |    37 +-
 .../randomwalk/concurrent/CheckPermission.java  |    27 +-
 .../test/randomwalk/concurrent/CloneTable.java  |     4 +-
 .../test/randomwalk/concurrent/Config.java      |   111 +-
 .../randomwalk/concurrent/CreateNamespace.java  |    48 +
 .../test/randomwalk/concurrent/CreateTable.java |     4 +-
 .../randomwalk/concurrent/DeleteNamespace.java  |    48 +
 .../test/randomwalk/concurrent/Merge.java       |     2 +-
 .../randomwalk/concurrent/RenameNamespace.java  |    52 +
 .../test/randomwalk/concurrent/RenameTable.java |     4 +-
 .../test/randomwalk/concurrent/Setup.java       |    23 +-
 .../randomwalk/security/WalkingSecurity.java    |   183 +-
 .../org/apache/accumulo/test/DumpConfigIT.java  |     6 +-
 .../org/apache/accumulo/test/NamespacesIT.java  |   555 +
 .../org/apache/accumulo/test/ShellServerIT.java |   111 +-
 .../apache/accumulo/test/SplitRecoveryIT.java   |     2 +-
 .../test/functional/GarbageCollectorIT.java     |     4 +-
 .../accumulo/test/functional/PermissionsIT.java |    93 +-
 .../accumulo/test/functional/RestartIT.java     |    18 +-
 test/system/bench/cloudstone1/cloudstone1.py    |     4 +-
 .../randomwalk/conf/modules/Concurrent.xml      |    23 +-
 138 files changed, 25343 insertions(+), 5285 deletions(-)
----------------------------------------------------------------------



[22/50] [abbrv] git commit: ACCUMULO-324 made the system namespace ignore system-level (and site and default-level) iterators and constraints

Posted by ct...@apache.org.
ACCUMULO-324 made the system namespace ignore system-level (and site and default-level) iterators and constraints


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

Branch: refs/heads/master
Commit: e1cf746acf8ce72692a2e6d9f4f20acc721814a1
Parents: 7b4f068
Author: Sean Hickey <ta...@gmail.com>
Authored: Tue Aug 13 10:05:37 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:10 2013 -0500

----------------------------------------------------------------------
 .../conf/TableNamespaceConfiguration.java       | 22 ++++++++++++++++---
 .../apache/accumulo/test/TableNamespacesIT.java | 23 ++++++++++++++++++++
 2 files changed, 42 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/e1cf746a/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
index 6c75e25..2ebe338 100644
--- 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
@@ -43,7 +43,7 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
   protected String namespaceId = null;
   protected Instance inst = null;
   private Set<ConfigurationObserver> observers;
-
+  
   public TableNamespaceConfiguration(String namespaceId, AccumuloConfiguration parent) {
     inst = HdfsZooInstance.getInstance();
     this.parent = parent;
@@ -87,8 +87,17 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
   public Iterator<Entry<String,String>> iterator() {
     TreeMap<String,String> entries = new TreeMap<String,String>();
 
-    for (Entry<String,String> parentEntry : parent)
-      entries.put(parentEntry.getKey(), parentEntry.getValue());
+    for (Entry<String,String> parentEntry : parent) {
+      if (this.namespaceId.equals(Constants.SYSTEM_TABLE_NAMESPACE_ID)) {
+        // exclude system iterators/constraints from the system namespace
+        // so they don't affect the metadata or root tables.
+        if (!isIterConst(parentEntry)) {
+          entries.put(parentEntry.getKey(), parentEntry.getValue());
+        }
+      } else {
+        entries.put(parentEntry.getKey(), parentEntry.getValue());
+      }
+    }
 
     List<String> children = getPropCache().getChildren(
         ZooUtil.getRoot(inst.getInstanceID()) + Constants.ZNAMESPACES + "/" + getNamespaceId() + Constants.ZNAMESPACE_CONF);
@@ -143,4 +152,11 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
     for (ConfigurationObserver co : copy)
       co.propertiesChanged();
   }
+  
+  protected boolean isIterConst(Entry<String,String> e) {
+    if (e.getKey().startsWith(Property.TABLE_ITERATOR_PREFIX.getKey()) || e.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.getKey())) {
+      return true;
+    }
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e1cf746a/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java b/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
index 38955e3..5705044 100644
--- a/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
@@ -432,6 +432,7 @@ public class TableNamespacesIT {
     }
     assertTrue(!hasProp);
   }
+  
   /**
    * Tests new Namespace permissions as well as modifications to Table permissions because of namespaces. Checks each permission to first make sure the user
    * doesn't have permission to perform the action, then root grants them the permission and we check to make sure they could perform the action.
@@ -578,6 +579,28 @@ public class TableNamespacesIT {
     c.securityOperations().revokeSystemPermission(user1, SystemPermission.ALTER_NAMESPACE);
   }
   
+  /**
+   * This test makes sure that system-level iterators and constraints are ignored by the system namespace so that the metadata and root tables aren't affected
+   */
+  @Test
+  public void excludeSystemIterConst() throws Exception {
+    Connector c = accumulo.getConnector("root", secret);
+    
+    c.instanceOperations().setProperty("table.iterator.scan.sum", "20," + SimpleFilter.class.getName());
+    assertTrue(c.instanceOperations().getSystemConfiguration().containsValue("20," + SimpleFilter.class.getName()));
+    
+    assertTrue(checkTableNamespaceHasProp(c, Constants.DEFAULT_TABLE_NAMESPACE, "table.iterator.scan.sum", "20," + SimpleFilter.class.getName()));
+    assertTrue(!checkTableNamespaceHasProp(c, Constants.SYSTEM_TABLE_NAMESPACE, "table.iterator.scan.sum", "20," + SimpleFilter.class.getName()));
+    c.instanceOperations().removeProperty("table.iterator.scan.sum");
+    
+    c.instanceOperations().setProperty("table.constraint.42", NumericValueConstraint.class.getName());
+    assertTrue(c.instanceOperations().getSystemConfiguration().containsValue(NumericValueConstraint.class.getName()));
+    
+    assertTrue(checkTableNamespaceHasProp(c, Constants.DEFAULT_TABLE_NAMESPACE, "table.constraint.42", NumericValueConstraint.class.getName()));
+    assertTrue(!checkTableNamespaceHasProp(c, Constants.SYSTEM_TABLE_NAMESPACE, "table.constraint.42", NumericValueConstraint.class.getName()));
+    c.instanceOperations().removeProperty("table.constraint.42");
+  }
+  
   private boolean checkTableHasProp(Connector c, String t, String propKey, String propVal) throws AccumuloException, TableNotFoundException {
     for (Entry<String,String> e : c.tableOperations().getProperties(t)) {
       if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) {


[11/50] [abbrv] git commit: ACCUMULO-1479 finished initial implementation of table namespace permissions, including tests

Posted by ct...@apache.org.
ACCUMULO-1479 finished initial implementation of table namespace permissions, including tests


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

Branch: refs/heads/master
Commit: 14fb257126810965c637ae234b1ba9623c2b5855
Parents: ad9abf4
Author: Sean Hickey <ta...@gmail.com>
Authored: Fri Aug 2 13:32:42 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:10 2013 -0500

----------------------------------------------------------------------
 .../accumulo/core/client/mock/MockTable.java    |   2 +-
 .../core/client/mock/MockTableNamespace.java    |   4 -
 .../core/security/TableNamespacePermission.java |   5 +-
 .../apache/accumulo/core/util/shell/Shell.java  |   3 +-
 .../commands/NamespacePermissionsCommand.java   |  44 +++++
 .../shell/commands/UserPermissionsCommand.java  |  22 +--
 .../client/mock/MockTableNamespacesTest.java    | 180 ++++++++++++-------
 .../server/conf/TableParentConfiguration.java   |  41 +++++
 .../security/AuditedSecurityOperation.java      |   2 +-
 .../server/security/SecurityOperation.java      |  48 ++++-
 .../server/security/handler/ZKPermHandler.java  |  19 ++
 .../java/org/apache/accumulo/master/Master.java |  30 +++-
 .../master/tableOps/CloneTableNamespace.java    |  25 ++-
 .../master/tableOps/CreateTableNamespace.java   |  24 ++-
 .../master/tableOps/DeleteTableNamespace.java   |  14 +-
 .../server/conf/TableParentConfiguration.java   |  41 -----
 .../org/apache/accumulo/test/ShellServerIT.java |   5 +
 .../apache/accumulo/test/TableNamespacesIT.java | 144 ++++++++++++---
 18 files changed, 466 insertions(+), 187 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/14fb2571/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
index 0e71414..5c4cb36 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
@@ -105,7 +105,7 @@ public class MockTable {
   
   MockTable(MockTableNamespace namespace, boolean limitVersion, TimeType timeType) {
     this(limitVersion, timeType);
-    Set<Entry<String,String>> set = namespace.getSettings().entrySet();
+    Set<Entry<String,String>> set = namespace.settings.entrySet();
     Iterator<Entry<String,String>> entries = set.iterator();
     while (entries.hasNext()) {
       Entry<String,String> entry = entries.next();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/14fb2571/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java
index beec4db..1798dd1 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableNamespace.java
@@ -43,10 +43,6 @@ public class MockTableNamespace {
     }
   }
   
-  public Map<String,String> getSettings() {
-    return settings;
-  }
-  
   public List<String> getTables(MockAccumulo acu) {
     List<String> l = new LinkedList<String>();
     for (String t : acu.tables.keySet()) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/14fb2571/core/src/main/java/org/apache/accumulo/core/security/TableNamespacePermission.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/security/TableNamespacePermission.java b/core/src/main/java/org/apache/accumulo/core/security/TableNamespacePermission.java
index 565a81a..9354b02 100644
--- a/core/src/main/java/org/apache/accumulo/core/security/TableNamespacePermission.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/TableNamespacePermission.java
@@ -23,12 +23,11 @@ public enum TableNamespacePermission {
   // One can add new permissions, with new numbers, but please don't change or use numbers previously assigned
   READ((byte) 0),
   WRITE((byte) 1),
-  ALTER_TABLE_NAMESPACE((byte) 2),
+  ALTER_NAMESPACE((byte) 2),
   GRANT((byte) 3),
   ALTER_TABLE((byte) 4),
   CREATE_TABLE((byte) 5),
-  DROP_TABLE((byte) 6),
-  BULK_IMPORT((byte) 7);
+  DROP_TABLE((byte) 6);
   
   final private byte permID;
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/14fb2571/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java b/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
index debca0b..c19e84a 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
@@ -122,6 +122,7 @@ import org.apache.accumulo.core.util.shell.commands.ListScansCommand;
 import org.apache.accumulo.core.util.shell.commands.ListShellIterCommand;
 import org.apache.accumulo.core.util.shell.commands.MaxRowCommand;
 import org.apache.accumulo.core.util.shell.commands.MergeCommand;
+import org.apache.accumulo.core.util.shell.commands.NamespacePermissionsCommand;
 import org.apache.accumulo.core.util.shell.commands.NamespacesCommand;
 import org.apache.accumulo.core.util.shell.commands.NoTableCommand;
 import org.apache.accumulo.core.util.shell.commands.OfflineCommand;
@@ -354,7 +355,7 @@ public class Shell extends ShellOptions {
         new SetShellIterCommand(), new ListShellIterCommand(), new DeleteShellIterCommand()};
     Command[] otherCommands = {new HiddenCommand()};
     Command[] permissionsCommands = {new GrantCommand(), new RevokeCommand(), new SystemPermissionsCommand(), new TablePermissionsCommand(),
-        new UserPermissionsCommand()};
+        new UserPermissionsCommand(), new NamespacePermissionsCommand()};
     Command[] stateCommands = {new AuthenticateCommand(), new ClsCommand(), new ClearCommand(), new FateCommand(), new NoTableCommand(), new SleepCommand(),
         new TableCommand(), new UserCommand(), new WhoAmICommand()};
     Command[] tableCommands = {new CloneTableCommand(), new ConfigCommand(), new CreateTableCommand(), new DeleteTableCommand(), new DropTableCommand(),

http://git-wip-us.apache.org/repos/asf/accumulo/blob/14fb2571/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacePermissionsCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacePermissionsCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacePermissionsCommand.java
new file mode 100644
index 0000000..822522e
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacePermissionsCommand.java
@@ -0,0 +1,44 @@
+/*
+ * 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.core.util.shell.commands;
+
+import java.io.IOException;
+
+import org.apache.accumulo.core.security.TableNamespacePermission;
+import org.apache.accumulo.core.util.shell.Shell;
+import org.apache.accumulo.core.util.shell.Shell.Command;
+import org.apache.commons.cli.CommandLine;
+
+public class NamespacePermissionsCommand extends Command {
+  @Override
+  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws IOException {
+    for (String p : TableNamespacePermission.printableValues()) {
+      shellState.getReader().println(p);
+    }
+    return 0;
+  }
+  
+  @Override
+  public String description() {
+    return "displays a list of valid table namespace permissions";
+  }
+  
+  @Override
+  public int numArgs() {
+    return 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/14fb2571/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserPermissionsCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserPermissionsCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserPermissionsCommand.java
index 25d9d54..1b6377a 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserPermissionsCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserPermissionsCommand.java
@@ -47,34 +47,34 @@ public class UserPermissionsCommand extends Command {
     }
     shellState.getReader().println();
     
-    for (String t : shellState.getConnector().tableOperations().list()) {
+    for (String n : shellState.getConnector().tableNamespaceOperations().list()) {
       delim = "";
-      for (TablePermission p : TablePermission.values()) {
-        if (shellState.getConnector().securityOperations().hasTablePermission(user, t, p) && p != null) {
+      for (TableNamespacePermission p : TableNamespacePermission.values()) {
+        if (p != null && shellState.getConnector().securityOperations().hasTableNamespacePermission(user, n, p)) {
           if (runOnce == 0) {
-            shellState.getReader().print("\nTable permissions (" + t + "): ");
+            shellState.getReader().print("\nTable Namespace permissions (" + n + "): ");
             runOnce++;
           }
-          shellState.getReader().print(delim + "Table." + p.name());
+          shellState.getReader().print(delim + "Namespace." + p.name());
           delim = ", ";
         }
-        
       }
       runOnce = 0;
     }
     shellState.getReader().println();
     
-    for (String n : shellState.getConnector().tableNamespaceOperations().list()) {
+    for (String t : shellState.getConnector().tableOperations().list()) {
       delim = "";
-      for (TableNamespacePermission p : TableNamespacePermission.values()) {
-        if (p != null && shellState.getConnector().securityOperations().hasTableNamespacePermission(user, n, p)) {
+      for (TablePermission p : TablePermission.values()) {
+        if (shellState.getConnector().securityOperations().hasTablePermission(user, t, p) && p != null) {
           if (runOnce == 0) {
-            shellState.getReader().print("\nTable Namespace permissions (" + n + "): ");
+            shellState.getReader().print("\nTable permissions (" + t + "): ");
             runOnce++;
           }
-          shellState.getReader().print(delim + "Namespace." + p.name());
+          shellState.getReader().print(delim + "Table." + p.name());
           delim = ", ";
         }
+        
       }
       runOnce = 0;
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/14fb2571/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableNamespacesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableNamespacesTest.java b/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableNamespacesTest.java
index d15b3e5..2ba1006 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableNamespacesTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableNamespacesTest.java
@@ -21,14 +21,29 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.util.Map.Entry;
+import java.util.EnumSet;
+import java.util.HashSet;
 import java.util.Random;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNamespaceNotEmptyException;
+import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
+import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.Filter;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.security.Authorizations;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 
@@ -54,9 +69,9 @@ public class MockTableNamespacesTest {
   }
   
   /**
-   * This test creates a new namespace "testing" and a table "testing.table1" which puts "table1" into the "testing" namespace.
-   * Then we create "testing.table2" which creates "table2" and puts it into "testing" as well. 
-   * Then we make sure that you can't delete a namespace with tables in it, and then we delete the tables and delete the namespace.
+   * This test creates a new namespace "testing" and a table "testing.table1" which puts "table1" into the "testing" namespace. Then we create "testing.table2"
+   * which creates "table2" and puts it into "testing" as well. Then we make sure that you can't delete a namespace with tables in it, and then we delete the
+   * tables and delete the namespace.
    * 
    * @throws Exception
    */
@@ -93,7 +108,7 @@ public class MockTableNamespacesTest {
     c.tableOperations().delete(tableName2);
     assertTrue(!c.tableOperations().exists(tableName2));
     assertTrue(c.tableNamespaceOperations().exists(namespace));
-
+    
     c.tableOperations().delete(tableName1);
     assertTrue(!c.tableOperations().exists(tableName1));
     c.tableNamespaceOperations().delete(namespace);
@@ -128,37 +143,15 @@ public class MockTableNamespacesTest {
     c.tableNamespaceOperations().setProperty(namespace, propKey, propVal);
     
     // check the namespace has the property
-    boolean itWorked = false;
-    for (Entry<String,String> prop : c.tableNamespaceOperations().getProperties(namespace)) {
-      if (prop.getKey().equals(propKey) && prop.getValue().equals(propVal)) {
-        itWorked = true;
-        break;
-      }
-    }
-    
-    assertTrue(itWorked);
+    assertTrue(checkTableNamespaceHasProp(c, namespace, propKey, propVal));
     
     // check that the table gets it from the namespace
-    itWorked = false;
-    for (Entry<String,String> prop : c.tableOperations().getProperties(tableName1)) {
-      if (prop.getKey().equals(propKey) && prop.getValue().equals(propVal)) {
-        itWorked = true;
-        break;
-      }
-    }
-    assertTrue(itWorked);
+    assertTrue(checkTableHasProp(c, tableName1, propKey, propVal));
     
     // test a second table to be sure the first wasn't magical
-    // (also, changed the order, the namespace already exists with the property)
-    itWorked = false;
+    // (also, changed the order, the namespace has the property already)
     c.tableOperations().create(tableName2);
-    for (Entry<String,String> prop : c.tableOperations().getProperties(tableName2)) {
-      if (prop.getKey().equals(propKey) && prop.getValue().equals(propVal)) {
-        itWorked = true;
-        break;
-      }
-    }
-    assertTrue(itWorked);
+    assertTrue(checkTableHasProp(c, tableName2, propKey, propVal));
     
     // test that table properties override namespace properties
     String propKey2 = Property.TABLE_FILE_MAX.getKey();
@@ -168,47 +161,38 @@ public class MockTableNamespacesTest {
     c.tableOperations().setProperty(tableName2, propKey2, tablePropVal);
     c.tableNamespaceOperations().setProperty("propchange", propKey2, propVal2);
     
-    itWorked = false;
-    for (Entry<String,String> prop : c.tableOperations().getProperties(tableName2)) {
-      if (prop.getKey().equals(propKey2) && prop.getValue().equals(tablePropVal)) {
-        itWorked = true;
-        break;
-      }
-    }
-    assertTrue(itWorked);
+    assertTrue(checkTableHasProp(c, tableName2, propKey2, tablePropVal));
     
     // now check that you can change the default namespace's properties
     propVal = "13K";
-    propVal2 = "44";
     String tableName = "some_table";
     c.tableOperations().create(tableName);
     c.tableNamespaceOperations().setProperty(Constants.DEFAULT_TABLE_NAMESPACE, propKey, propVal);
     
-    itWorked = false;
-    for (Entry<String,String> prop : c.tableOperations().getProperties(tableName)) {
-      if (prop.getKey().equals(propKey) && prop.getValue().equals(propVal)) {
-        itWorked = true;
-        break;
-      }
-    }
-    assertTrue(itWorked);
-  }
-  
-  /**
-   * This test creates a new user and a namespace. It checks to make sure the user can't modify anything in the namespace at first, then it grants the user
-   * permissions and makes sure that they can modify the namespace. Then it also checks if the user has the correct permissions on tables both already existing
-   * in the namespace and ones they create.
-   * 
-   * @throws Exception
-   */
-  @Test
-  public void testNamespacePermissions() throws Exception {
-    // TODO make the test once namespace-level permissions are implemented. (ACCUMULO-1479)
+    assertTrue(checkTableHasProp(c, tableName, propKey, propVal));
+    
+    // test the properties server-side by configuring an iterator.
+    // should not show anything with column-family = 'a'
+    String tableName3 = namespace + ".table3";
+    c.tableOperations().create(tableName3);
+    
+    IteratorSetting setting = new IteratorSetting(250, "thing", SimpleFilter.class.getName());
+    c.tableNamespaceOperations().attachIterator(namespace, setting);
+    
+    BatchWriter bw = c.createBatchWriter(tableName3, new BatchWriterConfig());
+    Mutation m = new Mutation("r");
+    m.put("a", "b", new Value("abcde".getBytes()));
+    bw.addMutation(m);
+    bw.flush();
+    bw.close();
+    
+    // Scanner s = c.createScanner(tableName3, Authorizations.EMPTY);
+    // do scanners work correctly in mock?
+    // assertTrue(!s.iterator().hasNext());
   }
   
   /**
-   * This test renames and clones two separate table into different namespaces.
-   * different namespace.
+   * This test renames and clones two separate table into different namespaces. different namespace.
    * 
    * @throws Exception
    */
@@ -218,11 +202,11 @@ public class MockTableNamespacesTest {
     String namespace2 = "cloned";
     String tableName = "table";
     String tableName1 = "renamed.table1";
-    //String tableName2 = "cloned.table2";
+    // String tableName2 = "cloned.table2";
     
     Instance instance = new MockInstance("renameclone");
     Connector c = instance.getConnector("user", new PasswordToken("pass"));
-
+    
     c.tableOperations().create(tableName);
     c.tableNamespaceOperations().create(namespace1);
     c.tableNamespaceOperations().create(namespace2);
@@ -233,10 +217,11 @@ public class MockTableNamespacesTest {
     assertTrue(!c.tableOperations().exists(tableName));
     
     // TODO implement clone in mock
-    /*c.tableOperations().clone(tableName1, tableName2, false, null, null);
-    
-    assertTrue(c.tableOperations().exists(tableName1));
-    assertTrue(c.tableOperations().exists(tableName2));*/
+    /*
+     * c.tableOperations().clone(tableName1, tableName2, false, null, null);
+     * 
+     * assertTrue(c.tableOperations().exists(tableName1)); assertTrue(c.tableOperations().exists(tableName2));
+     */
     return;
   }
   
@@ -262,4 +247,65 @@ public class MockTableNamespacesTest {
     assertTrue(!c.tableOperations().exists(namespace1 + "." + table));
     assertTrue(c.tableOperations().exists(namespace2 + "." + table));
   }
+  
+  /**
+   * This tests adding iterators to a namespace, listing them, and removing them
+   */
+  @Test
+  public void testNamespaceIterators() throws Exception {
+    Instance instance = new MockInstance("Iterators");
+    Connector c = instance.getConnector("user", new PasswordToken("pass"));
+    
+    String namespace = "iterator";
+    String tableName = namespace + ".table";
+    String iter = "thing";
+    
+    c.tableNamespaceOperations().create(namespace);
+    c.tableOperations().create(tableName);
+    
+    IteratorSetting setting = new IteratorSetting(250, iter, SimpleFilter.class.getName());
+    HashSet<IteratorScope> scope = new HashSet<IteratorScope>();
+    scope.add(IteratorScope.scan);
+    c.tableNamespaceOperations().attachIterator(namespace, setting, EnumSet.copyOf(scope));
+    
+    BatchWriter bw = c.createBatchWriter(tableName, new BatchWriterConfig());
+    Mutation m = new Mutation("r");
+    m.put("a", "b", new Value("abcde".getBytes(Constants.UTF8)));
+    bw.addMutation(m);
+    bw.flush();
+    
+    Scanner s = c.createScanner(tableName, Authorizations.EMPTY);
+    System.out.println(s.iterator().next());
+    // do scanners work correctly in mock?
+    // assertTrue(!s.iterator().hasNext());
+    
+    assertTrue(c.tableNamespaceOperations().listIterators(namespace).containsKey(iter));
+    c.tableNamespaceOperations().removeIterator(namespace, iter, EnumSet.copyOf(scope));
+  }
+  
+  private boolean checkTableHasProp(Connector c, String t, String propKey, String propVal) throws AccumuloException, TableNotFoundException {
+    for (Entry<String,String> e : c.tableOperations().getProperties(t)) {
+      if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) {
+        return true;
+      }
+    }
+    return false;
+  }
+  
+  private boolean checkTableNamespaceHasProp(Connector c, String n, String propKey, String propVal) throws AccumuloException, TableNamespaceNotFoundException {
+    for (Entry<String,String> e : c.tableNamespaceOperations().getProperties(n)) {
+      if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) {
+        return true;
+      }
+    }
+    return false;
+  }
+  
+  public static class SimpleFilter extends Filter {
+    public boolean accept(Key k, Value v) {
+      if (k.getColumnFamily().toString().equals("a"))
+        return false;
+      return true;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/14fb2571/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
new file mode 100644
index 0000000..7590d76
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/TableParentConfiguration.java
@@ -0,0 +1,41 @@
+/*
+ * 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.client.impl.Tables;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+
+
+/**
+ * Used by TableConfiguration to dynamically get the TableNamespaceConfiguration if the namespace changes
+ */
+public class TableParentConfiguration extends TableNamespaceConfiguration {
+
+  private String tableId;
+  
+  public TableParentConfiguration(String tableId, AccumuloConfiguration parent) {
+    super(null, parent);
+    this.tableId = tableId;
+    this.namespaceId = getNamespaceId();
+  }
+  
+  @Override
+  protected String getNamespaceId() {
+    this.namespaceId = Tables.getNamespace(inst, tableId); 
+    return this.namespaceId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/14fb2571/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java b/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
index 7148556..7ec9fd2 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/AuditedSecurityOperation.java
@@ -235,7 +235,7 @@ public class AuditedSecurityOperation extends SecurityOperation {
   @Override
   public boolean canCreateTable(TCredentials c, String tableName) throws ThriftSecurityException {
     try {
-      boolean result = super.canCreateTable(c);
+      boolean result = super.canCreateTable(c, tableName);
       audit(c, result, CAN_CREATE_TABLE_AUDIT_TEMPLATE, tableName);
       return result;
     } catch (ThriftSecurityException ex) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/14fb2571/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 6dcaf9d..7e7dde9 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
@@ -343,7 +343,7 @@ public class SecurityOperation {
       String namespace = TableNamespaces.getNamespaceId(HdfsZooInstance.getInstance(), tableNamespace);
       return hasTableNamespacePermission(credentials, namespace, permission, useCached);
     } catch (TableNamespaceNotFoundException e) {
-      return false;
+      throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.TABLE_NAMESPACE_DOESNT_EXIST);
     }
   }
   
@@ -451,7 +451,8 @@ public class SecurityOperation {
   }
   
   public boolean canCreateTable(TCredentials c, String tableName) throws ThriftSecurityException {
-    return canCreateTable(c) || hasTableNamespacePermissionForTableName(c, tableName, TableNamespacePermission.CREATE_TABLE, false);
+    authenticate(c);
+    return hasTableNamespacePermissionForTableName(c, tableName, TableNamespacePermission.CREATE_TABLE, false) || canCreateTable(c);
   }
   
   public boolean canCreateTable(TCredentials c) throws ThriftSecurityException {
@@ -469,7 +470,8 @@ public class SecurityOperation {
     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));
+        && (hasTablePermission(c, tableId, TablePermission.READ, false) || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.READ,
+            false));
   }
   
   public boolean canDeleteTable(TCredentials c, String tableId) throws ThriftSecurityException {
@@ -504,8 +506,7 @@ public class SecurityOperation {
   
   public boolean canBulkImport(TCredentials c, String tableId) throws ThriftSecurityException {
     authenticate(c);
-    return hasTablePermission(c, tableId, TablePermission.BULK_IMPORT, false)
-        || hasTableNamespacePermissionForTableId(c, tableId, TableNamespacePermission.BULK_IMPORT, false);
+    return hasTablePermission(c, tableId, TablePermission.BULK_IMPORT, false);
   }
   
   public boolean canCompact(TCredentials c, String tableId) throws ThriftSecurityException {
@@ -579,7 +580,7 @@ public class SecurityOperation {
   
   public boolean canRevokeTableNamespace(TCredentials c, String user, String tableNamespace) throws ThriftSecurityException {
     authenticate(c);
-    return hasSystemPermission(c, SystemPermission.ALTER_NAMESPACE, false) || hasTablePermission(c, tableNamespace, TablePermission.GRANT, false);
+    return hasSystemPermission(c, SystemPermission.ALTER_NAMESPACE, false) || hasTableNamespacePermission(c, tableNamespace, TableNamespacePermission.GRANT, false);
   }
   
   public void changeAuthorizations(TCredentials credentials, String user, Authorizations authorizations) throws ThriftSecurityException {
@@ -778,7 +779,7 @@ public class SecurityOperation {
   }
   
   public void deleteTableNamespace(TCredentials credentials, String tableNamespace) throws ThriftSecurityException {
-    if (!canDeleteTable(credentials, tableNamespace))
+    if (!canDeleteNamespace(credentials, tableNamespace))
       throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     try {
       permHandle.cleanTableNamespacePermissions(tableNamespace);
@@ -801,4 +802,37 @@ public class SecurityOperation {
     return hasSystemPermission(credentials, SystemPermission.CREATE_TABLE, false)
         || hasTableNamespacePermissionForTableName(credentials, tableName, TableNamespacePermission.CREATE_TABLE, false);
   }
+  
+  public boolean canAlterNamespace(TCredentials credentials, String namespaceId) throws ThriftSecurityException {
+    authenticate(credentials);
+    return hasTableNamespacePermission(credentials, namespaceId, TableNamespacePermission.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)
+        || 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)
+        && hasSystemPermission(credentials, SystemPermission.CREATE_NAMESPACE, false);
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/14fb2571/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 78b79a1..6f6304a 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
@@ -23,6 +23,7 @@ import java.util.Map.Entry;
 import java.util.Set;
 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.TableNotFoundException;
@@ -382,6 +383,10 @@ public class ZKPermHandler implements PermissionHandler {
     // Allow the root user to flush the system tables
     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));
     
     try {
       // prep parent node of users with root username
@@ -392,6 +397,8 @@ public class ZKPermHandler implements PermissionHandler {
       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());
     } catch (KeeperException e) {
       log.error(e, e);
       throw new RuntimeException(e);
@@ -432,6 +439,17 @@ public class ZKPermHandler implements PermissionHandler {
     }
   }
   
+  /**
+   * 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.
+   */
+  private void createTableNamespacePerm(String user, String namespace, Set<TableNamespacePermission> perms) throws KeeperException, InterruptedException {
+    synchronized (zooCache) {
+      zooCache.clear();
+      ZooReaderWriter.getRetryingInstance().putPersistentData(ZKUserPath + "/" + user + ZKUserNamespacePerms + "/" + namespace,
+          ZKSecurityTool.convertTableNamespacePermissions(perms), NodeExistsPolicy.FAIL);
+    }
+  }
+  
   @Override
   public void cleanUser(String user) throws AccumuloSecurityException {
     try {
@@ -439,6 +457,7 @@ public class ZKPermHandler implements PermissionHandler {
         IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
         zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserSysPerms, NodeMissingPolicy.SKIP);
         zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserTablePerms, NodeMissingPolicy.SKIP);
+        zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserNamespacePerms, NodeMissingPolicy.SKIP);
         zooCache.clear(ZKUserPath + "/" + user);
       }
     } catch (InterruptedException e) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/14fb2571/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 d3f22e6..293dbc9 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
@@ -43,9 +43,9 @@ import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.RowIterator;
 import org.apache.accumulo.core.client.Scanner;
 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.admin.TableOperationsImpl;
 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;
@@ -743,14 +743,22 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
         throws ThriftSecurityException, ThriftTableOperationException {
 
       String namespaceId = null;
+      namespaceId = checkNamespaceId(namespace, op);
+
+      if (!security.canAlterNamespace(c, namespaceId))
+        throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
       try {
-        namespaceId = TableNamespaces.getNamespaceId(instance, namespace);
-        // TODO insert a permission check here once namespace-level permissions exist. (ACCUMULO-1479)
         if (value == null) {
           NamespacePropUtil.removeNamespaceProperty(namespaceId, property);
         } else {
           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:
+        checkNamespaceId(namespaceId, op);
+        log.info("Error altering table namespace property", e);
+        throw new ThriftTableOperationException(namespaceId, namespace, op, TableOperationExceptionType.OTHER, "Problem altering table 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");
@@ -1125,7 +1133,9 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       switch (op) {
         case CREATE: {
           String namespace = ByteBufferUtil.toString(arguments.get(0));
-          // TODO security check once namespace permissions exist (ACCUMULO-1479)
+          if (!security.canCreateNamespace(c, namespace))
+            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);
@@ -1135,11 +1145,14 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
 
           String oldName = ByteBufferUtil.toString(arguments.get(0));
           String newName = ByteBufferUtil.toString(arguments.get(1));
-          // TODO security check (ACCUMULO-1479)
           String namespaceId = checkNamespaceId(oldName, TableOperation.RENAME);
+
           checkNotSystemNamespace(oldName, TableOperation.RENAME);
           checkNotSystemNamespace(newName, TableOperation.RENAME);
           checkTableNamespaceName(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);
           break;
         }
@@ -1147,7 +1160,9 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
           String namespace = ByteBufferUtil.toString(arguments.get(0));
           checkNotSystemNamespace(namespace, TableOperation.DELETE);
           String namespaceId = checkNamespaceId(namespace, TableOperation.DELETE);
-          // TODO security check (ACCUMULO-1479)
+          if (!security.canDeleteNamespace(c, namespaceId))
+            throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+
           fate.seedTransaction(opid, new TraceRepo<Master>(new DeleteTableNamespace(namespaceId)), autoCleanup);
           break;
         }
@@ -1156,7 +1171,8 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
           String namespace = ByteBufferUtil.toString(arguments.get(1));
           checkNotSystemNamespace(namespace, TableOperation.CLONE);
           checkTableNamespaceName(namespace, TableOperation.CLONE);
-          // TODO security check (ACCUMULO-1479)
+          if (!security.canCloneNamespace(c, namespaceId, namespace))
+            throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
           Map<String,String> propertiesToSet = new HashMap<String,String>();
           Set<String> propertiesToExclude = new HashSet<String>();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/14fb2571/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTableNamespace.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTableNamespace.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTableNamespace.java
index 9b661f2..ffa1448 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTableNamespace.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTableNamespace.java
@@ -23,10 +23,14 @@ import java.util.Set;
 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.client.HdfsZooInstance;
+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;
 
@@ -134,15 +138,18 @@ class CloneNamespacePermissions extends MasterRepo {
 
   @Override
   public Repo<Master> call(long tid, Master environment) throws Exception {
-    // TODO add table namespace permissions (ACCUMULO-1479)
-    // give all table permissions to the creator
-    /*
-     * for (TablePermission permission : TablePermission.values()) { try {
-     * AuditedSecurityOperation.getInstance().grantTablePermission(SecurityConstants.getSystemCredentials(), cloneInfo.user, cloneInfo.newId, permission); }
-     * catch (ThriftSecurityException e) { Logger.getLogger(FinishCloneTableNamespace.class).error(e.getMessage(), e); throw e; } }
-     */
-
-    // setup permissions in zookeeper before table info in zookeeper
+    // give all table namespace permissions to the creator
+    for (TableNamespacePermission permission : TableNamespacePermission.values()) {
+      try {
+        AuditedSecurityOperation.getInstance().grantTableNamespacePermission(SystemCredentials.get().toThrift(environment.getInstance()), cloneInfo.user,
+            cloneInfo.newId, permission);
+      } catch (ThriftSecurityException e) {
+        Logger.getLogger(FinishCloneTableNamespace.class).error(e.getMessage(), e);
+        throw e;
+      }
+    }
+
+    // setup permissions in zookeeper before table namespace info in zookeeper
     // this way concurrent users will not get a spurious pemission denied
     // error
     return new CloneNamespaceZookeeper(cloneInfo);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/14fb2571/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
index d6c6fc4..247ae56 100644
--- 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
@@ -23,9 +23,14 @@ 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;
@@ -135,14 +140,17 @@ class SetupNamespacePermissions extends MasterRepo {
 
   @Override
   public Repo<Master> call(long tid, Master env) throws Exception {
-    // TODO implement once namespace permissions exist (ACCUMULO-1479)
-
-    // give all table permissions to the creator
-    /*
-     * SecurityOperation security = AuditedSecurityOperation.getInstance(); for (TableNamespacePermission permission : TableNamespacePermission.values()) { try
-     * { security.grantTableNamespacePermission(SecurityConstants.getSystemCredentials(), tableNamespaceInfo.user, tableNamespaceInfo.tableId, permission); }
-     * catch (ThriftSecurityException e) { Logger.getLogger(FinishCreateTableNamespace.class).error(e.getMessage(), e); throw e; } }
-     */
+    // 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

http://git-wip-us.apache.org/repos/asf/accumulo/blob/14fb2571/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
index 5013a2f..bd8e5c3 100644
--- 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
@@ -18,8 +18,11 @@ 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;
 
@@ -51,11 +54,12 @@ class NamespaceCleanUp extends MasterRepo {
     }
     Tables.clearCache(master.getInstance());
 
-    // TODO remove any permissions associated with this once they exist (ACCUMULO-1479)
-    /*
-     * try { AuditedSecurityOperation.getInstance().deleteTable(SecurityConstants.getSystemCredentials(), namespaceName); } catch (ThriftSecurityException e) {
-     * log.error(e.getMessage(), e); }
-     */
+    // 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);
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/14fb2571/server/src/main/java/org/apache/accumulo/server/conf/TableParentConfiguration.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/conf/TableParentConfiguration.java b/server/src/main/java/org/apache/accumulo/server/conf/TableParentConfiguration.java
deleted file mode 100644
index 7590d76..0000000
--- a/server/src/main/java/org/apache/accumulo/server/conf/TableParentConfiguration.java
+++ /dev/null
@@ -1,41 +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.client.impl.Tables;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-
-
-/**
- * Used by TableConfiguration to dynamically get the TableNamespaceConfiguration if the namespace changes
- */
-public class TableParentConfiguration extends TableNamespaceConfiguration {
-
-  private String tableId;
-  
-  public TableParentConfiguration(String tableId, AccumuloConfiguration parent) {
-    super(null, parent);
-    this.tableId = tableId;
-    this.namespaceId = getNamespaceId();
-  }
-  
-  @Override
-  protected String getNamespaceId() {
-    this.namespaceId = Tables.getNamespace(inst, tableId); 
-    return this.namespaceId;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/14fb2571/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
index 663fa44..792ba3e 100644
--- a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
@@ -873,6 +873,11 @@ public class ShellServerIT extends SimpleMacIT {
     exec("setiter -tn thing2 -scan -class org.apache.accumulo.core.iterators.user.SummingCombiner -p 10 -n name", true);
     exec("listiter -tn thing2 -scan", true, "Summing", true);
     exec("deleteiter -tn thing2 -n name -scan", true);
+    exec("createuser dude");
+    exec("pass");
+    exec("pass");
+    exec("grant Namespace.CREATE_TABLE -tn thing2 -u dude", true);
+    exec("revoke Namespace.CREATE_TABLE -tn thing2 -u dude", true);
     
     // properties override and such
     exec("config -tn thing2 -s table.file.max=44444", true);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/14fb2571/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java b/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
index b779152..1a52f72 100644
--- a/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.fail;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Random;
@@ -51,6 +52,7 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.Filter;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 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.util.UtilWaitThread;
 import org.apache.accumulo.examples.simple.constraints.NumericValueConstraint;
@@ -82,8 +84,6 @@ public class TableNamespacesIT {
   
   /**
    * This test creates a table without specifying a namespace. In this case, it puts the table into the default namespace.
-   * 
-   * @throws Exception
    */
   @Test
   public void testDefaultNamespace() throws Exception {
@@ -99,8 +99,6 @@ public class TableNamespacesIT {
    * This test creates a new namespace "testing" and a table "testing.table1" which puts "table1" into the "testing" namespace. Then we create "testing.table2"
    * which creates "table2" and puts it into "testing" as well. Then we make sure that you can't delete a namespace with tables in it, and then we delete the
    * tables and delete the namespace.
-   * 
-   * @throws Exception
    */
   @Test
   public void testCreateAndDeleteNamespace() throws Exception {
@@ -148,8 +146,6 @@ public class TableNamespacesIT {
    * Checks to make sure namespace-level properties are overridden by table-level properties.
    * 
    * Checks to see if the default namespace's properties work as well.
-   * 
-   * @throws Exception
    */
   
   @Test
@@ -216,21 +212,8 @@ public class TableNamespacesIT {
   }
   
   /**
-   * This test creates a new user and a namespace. It checks to make sure the user can't modify anything in the namespace at first, then it grants the user
-   * permissions and makes sure that they can modify the namespace. Then it also checks if the user has the correct permissions on tables both already existing
-   * in the namespace and ones they create.
-   * 
-   * @throws Exception
-   */
-  @Test
-  public void testNamespacePermissions() throws Exception {
-    // TODO make the test once namespace-level permissions are implemented. (ACCUMULO-1479)
-  }
-  
-  /**
    * This test renames and clones two separate table into different namespaces. different namespace.
    * 
-   * @throws Exception
    */
   @Test
   public void testRenameAndCloneTableToNewNamespace() throws Exception {
@@ -413,6 +396,7 @@ public class TableNamespacesIT {
   
   /**
 <<<<<<< HEAD
+<<<<<<< HEAD
    * Tests that when a table moves to a new namespace that it's properties inherit from the new namespace and not the old one
    */
   @Test
@@ -452,6 +436,10 @@ public class TableNamespacesIT {
   }
   /**
    *  Tests new Namespace permissions as well as modifications to Table permissions because of namespaces 
+=======
+   * Tests new Namespace permissions as well as modifications to Table permissions because of namespaces. Checks each permission to first make sure the user
+   * doesn't have permission to perform the action, then root grants them the permission and we check to make sure they could perform the action.
+>>>>>>> ACCUMULO-1479 finished initial implementation of table namespace permissions, including tests
    */
   @Test
   public void testPermissions() throws Exception {
@@ -460,9 +448,9 @@ public class TableNamespacesIT {
     PasswordToken pass = new PasswordToken(secret);
     
     String n1 = "namespace1";
-
+    
     String user1 = "dude";
-
+    
     c.tableNamespaceOperations().create(n1);
     c.tableOperations().create(n1 + ".table1");
     
@@ -478,9 +466,121 @@ public class TableNamespacesIT {
     }
     
     c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.CREATE_TABLE);
-    
     user1Con.tableOperations().create(n1 + ".table2");
     assertTrue(c.tableOperations().list().contains(n1 + ".table2"));
+    c.securityOperations().revokeTableNamespacePermission(user1, n1, TableNamespacePermission.CREATE_TABLE);
+    
+    try {
+      user1Con.tableOperations().delete(n1 + ".table1");
+      fail();
+    } catch (AccumuloSecurityException e) {
+      // should happen
+    }
+    
+    c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.DROP_TABLE);
+    user1Con.tableOperations().delete(n1 + ".table1");
+    assertTrue(!c.tableOperations().list().contains(n1 + ".table1"));
+    c.securityOperations().revokeTableNamespacePermission(user1, n1, TableNamespacePermission.DROP_TABLE);
+    
+    c.tableOperations().create(n1 + ".t");
+    BatchWriter bw = c.createBatchWriter(n1 + ".t", null);
+    Mutation m = new Mutation("row");
+    m.put("cf", "cq", "value");
+    bw.addMutation(m);
+    bw.close();
+    
+    Iterator<Entry<Key,Value>> i = user1Con.createScanner(n1 + ".t", new Authorizations()).iterator();
+    try {
+      i.next();
+      fail();
+    } catch (RuntimeException e) {
+      // yup
+    }
+    
+    m = new Mutation("user1");
+    m.put("cf", "cq", "turtles");
+    bw = user1Con.createBatchWriter(n1 + ".t", null);
+    try {
+      bw.addMutation(m);
+      bw.close();
+      fail();
+    } catch (MutationsRejectedException e) {
+      // good
+    }
+    
+    c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.READ);
+    i = user1Con.createScanner(n1 + ".t", new Authorizations()).iterator();
+    assertTrue(i.hasNext());
+    c.securityOperations().revokeTableNamespacePermission(user1, n1, TableNamespacePermission.READ);
+    
+    c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.WRITE);
+    m = new Mutation("user1");
+    m.put("cf", "cq", "turtles");
+    bw = user1Con.createBatchWriter(n1 + ".t", null);
+    bw.addMutation(m);
+    bw.close();
+    c.securityOperations().revokeTableNamespacePermission(user1, n1, TableNamespacePermission.WRITE);
+    
+    try {
+      user1Con.tableOperations().setProperty(n1 + ".t", Property.TABLE_FILE_MAX.getKey(), "42");
+      fail();
+    } catch (AccumuloSecurityException e) {}
+    
+    c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.ALTER_TABLE);
+    user1Con.tableOperations().setProperty(n1 + ".t", Property.TABLE_FILE_MAX.getKey(), "42");
+    user1Con.tableOperations().removeProperty(n1 + ".t", Property.TABLE_FILE_MAX.getKey());
+    c.securityOperations().revokeTableNamespacePermission(user1, n1, TableNamespacePermission.ALTER_TABLE);
+    
+    try {
+      user1Con.tableNamespaceOperations().setProperty(n1, Property.TABLE_FILE_MAX.getKey(), "55");
+      fail();
+    } catch (AccumuloSecurityException e) {}
+    
+    c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.ALTER_NAMESPACE);
+    user1Con.tableNamespaceOperations().setProperty(n1, Property.TABLE_FILE_MAX.getKey(), "42");
+    user1Con.tableNamespaceOperations().removeProperty(n1, Property.TABLE_FILE_MAX.getKey());
+    c.securityOperations().revokeTableNamespacePermission(user1, n1, TableNamespacePermission.ALTER_NAMESPACE);
+    
+    String user2 = "guy";
+    c.securityOperations().createLocalUser(user2, pass);
+    try {
+      user1Con.securityOperations().grantTableNamespacePermission(user2, n1, TableNamespacePermission.ALTER_NAMESPACE);
+      fail();
+    } catch (AccumuloSecurityException e) {}
+    
+    c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.GRANT);
+    user1Con.securityOperations().grantTableNamespacePermission(user2, n1, TableNamespacePermission.ALTER_NAMESPACE);
+    user1Con.securityOperations().revokeTableNamespacePermission(user2, n1, TableNamespacePermission.ALTER_NAMESPACE);
+    c.securityOperations().revokeTableNamespacePermission(user1, n1, TableNamespacePermission.GRANT);
+    
+    String n2 = "namespace2";
+    try {
+      user1Con.tableNamespaceOperations().create(n2);
+      fail();
+    } catch (AccumuloSecurityException e) {}
+    
+    c.securityOperations().grantSystemPermission(user1, SystemPermission.CREATE_NAMESPACE);
+    user1Con.tableNamespaceOperations().create(n2);
+    c.securityOperations().revokeSystemPermission(user1, SystemPermission.CREATE_NAMESPACE);
+    
+    try {
+      user1Con.tableNamespaceOperations().delete(n2);
+      fail();
+    } catch (AccumuloSecurityException e) {}
+    
+    c.securityOperations().grantSystemPermission(user1, SystemPermission.DROP_NAMESPACE);
+    user1Con.tableNamespaceOperations().delete(n2);
+    c.securityOperations().revokeSystemPermission(user1, SystemPermission.DROP_NAMESPACE);
+    
+    try {
+      user1Con.tableNamespaceOperations().setProperty(n1, Property.TABLE_FILE_MAX.getKey(), "33");
+      fail();
+    } catch (AccumuloSecurityException e) {}
+    
+    c.securityOperations().grantSystemPermission(user1, SystemPermission.ALTER_NAMESPACE);
+    user1Con.tableNamespaceOperations().setProperty(n1, Property.TABLE_FILE_MAX.getKey(), "33");
+    user1Con.tableNamespaceOperations().removeProperty(n1, Property.TABLE_FILE_MAX.getKey());
+    c.securityOperations().revokeSystemPermission(user1, SystemPermission.ALTER_NAMESPACE);
   }
   
   private boolean checkTableHasProp(Connector c, String t, String propKey, String propVal) throws AccumuloException, TableNotFoundException {


[36/50] [abbrv] git commit: ACCUMULO-802 changed error type given to thrift for TableNamespaceNotFoundExceptions so they don't cause a RuntimException when caught on the client side

Posted by ct...@apache.org.
ACCUMULO-802 changed error type given to thrift for TableNamespaceNotFoundExceptions so they don't cause a RuntimException when caught on the client side


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

Branch: refs/heads/master
Commit: 7d8d688fa00e42179ae357efc5f711729bbad012
Parents: 3d707c8
Author: Sean Hickey <ta...@gmail.com>
Authored: Wed Aug 14 12:38:16 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:11 2013 -0500

----------------------------------------------------------------------
 server/master/src/main/java/org/apache/accumulo/master/Master.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/7d8d688f/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 5cfd3c3..aa0d17a 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
@@ -931,7 +931,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) {
-            throw new ThriftTableOperationException(null, tableName, TableOperation.CREATE, TableOperationExceptionType.NOTFOUND, e.getMessage());
+            throw new TException(e.getMessage(), e);
           }
           break;
         }


[14/50] [abbrv] git commit: ACCUMULO-1479 added table namespaces permission check to RandomWalk

Posted by ct...@apache.org.
ACCUMULO-1479 added table namespaces permission check to RandomWalk


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

Branch: refs/heads/master
Commit: 5f4b864b072dc82ba17294d49f3b0a364c79b8fd
Parents: 14fb257
Author: Sean Hickey <ta...@gmail.com>
Authored: Tue Aug 6 10:45:20 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:10 2013 -0500

----------------------------------------------------------------------
 .../concurrent/ChangePermissions.java           | 37 ++++++++++++++++++--
 .../randomwalk/concurrent/CheckPermission.java  | 13 +++++--
 2 files changed, 46 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/5f4b864b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/ChangePermissions.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/ChangePermissions.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/ChangePermissions.java
index be2de2a..a63391c 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/ChangePermissions.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/ChangePermissions.java
@@ -26,6 +26,7 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
 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.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
@@ -46,11 +47,18 @@ public class ChangePermissions extends Test {
     List<String> tableNames = (List<String>) state.get("tables");
     String tableName = tableNames.get(rand.nextInt(tableNames.size()));
     
+    @SuppressWarnings("unchecked")
+    List<String> tableNamespaces = (List<String>) state.get("namespaces");
+    String tableNamespace = tableNamespaces.get(rand.nextInt(tableNamespaces.size()));
+    
     try {
-      if (rand.nextBoolean())
+      int dice = rand.nextInt(2);
+      if (dice == 0)
         changeSystemPermission(conn, rand, userName);
-      else
+      else if (dice == 1)
         changeTablePermission(conn, rand, userName, tableName);
+      else if (dice == 2)
+        changeTableNamespacePermission(conn, rand, userName, tableNamespace);
     } catch (AccumuloSecurityException ex) {
       log.debug("Unable to change user permissions: " + ex.getCause());
     }
@@ -108,4 +116,29 @@ public class ChangePermissions extends Test {
     }
   }
   
+  private void changeTableNamespacePermission(Connector conn, Random rand, String userName, String tableNamespace) throws AccumuloException, AccumuloSecurityException {
+    
+    EnumSet<TableNamespacePermission> perms = EnumSet.noneOf(TableNamespacePermission.class);
+    for (TableNamespacePermission p : TableNamespacePermission.values()) {
+      if (conn.securityOperations().hasTableNamespacePermission(userName, tableNamespace, p))
+        perms.add(p);
+    }
+    
+    EnumSet<TableNamespacePermission> more = EnumSet.allOf(TableNamespacePermission.class);
+    more.removeAll(perms);
+    
+    if (rand.nextBoolean() && more.size() > 0) {
+      List<TableNamespacePermission> moreList = new ArrayList<TableNamespacePermission>(more);
+      TableNamespacePermission choice = moreList.get(rand.nextInt(moreList.size()));
+      log.debug("adding permission " + choice);
+      conn.securityOperations().grantTableNamespacePermission(userName, tableNamespace, choice);
+    } else {
+      if (perms.size() > 0) {
+        List<TableNamespacePermission> permList = new ArrayList<TableNamespacePermission>(perms);
+        TableNamespacePermission choice = permList.get(rand.nextInt(permList.size()));
+        log.debug("removing permission " + choice);
+        conn.securityOperations().revokeTableNamespacePermission(userName, tableNamespace, choice);
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5f4b864b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CheckPermission.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CheckPermission.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CheckPermission.java
index 5fa9bc4..544ce96 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CheckPermission.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CheckPermission.java
@@ -24,6 +24,7 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.core.security.TableNamespacePermission;
 import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
@@ -43,13 +44,21 @@ public class CheckPermission extends Test {
     List<String> tableNames = (List<String>) state.get("tables");
     String tableName = tableNames.get(rand.nextInt(tableNames.size()));
     
+    @SuppressWarnings("unchecked")
+    List<String> tableNamespaces = (List<String>) state.get("namespaces");
+    String tableNamespace = tableNamespaces.get(rand.nextInt(tableNamespaces.size()));
+    
     try {
-      if (rand.nextBoolean()) {
+      int dice = rand.nextInt(2);
+      if (dice == 0) {
         log.debug("Checking systerm permission " + userName);
         conn.securityOperations().hasSystemPermission(userName, SystemPermission.values()[rand.nextInt(SystemPermission.values().length)]);
-      } else {
+      } else if (dice == 1) {
         log.debug("Checking table permission " + userName + " " + tableName);
         conn.securityOperations().hasTablePermission(userName, tableName, TablePermission.values()[rand.nextInt(TablePermission.values().length)]);
+      } else if (dice == 2) {
+        log.debug("Checking table namespace permission " + userName + " " + tableNamespace);
+        conn.securityOperations().hasTableNamespacePermission(userName, tableNamespace, TableNamespacePermission.values()[rand.nextInt(TableNamespacePermission.values().length)]);
       }
       
     } catch (AccumuloSecurityException ex) {


[47/50] [abbrv] git commit: ACCUMULO-1859 update namepace config w/ new getPropertiesMethod

Posted by ct...@apache.org.
ACCUMULO-1859 update namepace config w/ new getPropertiesMethod


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

Branch: refs/heads/master
Commit: 5b791b166b9190285bfb3ba3d45c3de0dbb93af1
Parents: 725d821
Author: Keith Turner <kt...@apache.org>
Authored: Thu Nov 14 14:20:33 2013 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:11 2013 -0500

----------------------------------------------------------------------
 .../conf/TableNamespaceConfiguration.java       | 76 +++++++++++---------
 1 file changed, 43 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/5b791b16/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
index b4e9598..74621a3 100644
--- 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
@@ -19,11 +19,9 @@ package org.apache.accumulo.server.conf;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
-import java.util.Map.Entry;
+import java.util.Map;
 import java.util.Set;
-import java.util.TreeMap;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
@@ -54,19 +52,20 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
   @Override
   public String get(Property property) {
     String key = property.getKey();
-    String value = get(key);
+    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);
-      value = parent.get(property);
+      if (!isIterConst(property.getKey()))
+        value = parent.get(property);
     }
     return value;
   }
 
-  private String get(String key) {
+  private String get(ZooCache zc, String key) {
     String zPath = ZooUtil.getRoot(inst.getInstanceID()) + Constants.ZNAMESPACES + "/" + getNamespaceId() + Constants.ZNAMESPACE_CONF + "/" + key;
-    byte[] v = getPropCache().get(zPath);
+    byte[] v = zc.get(zPath);
     String value = null;
     if (v != null)
       value = new String(v, Constants.UTF8);
@@ -83,33 +82,47 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
     return propCache;
   }
 
-  @Override
-  public Iterator<Entry<String,String>> iterator() {
-    TreeMap<String,String> entries = new TreeMap<String,String>();
-
-    for (Entry<String,String> parentEntry : parent) {
-      if (this.namespaceId.equals(Constants.SYSTEM_TABLE_NAMESPACE_ID)) {
-        // exclude system iterators/constraints from the system namespace
-        // so they don't affect the metadata or root tables.
-        if (!isIterConst(parentEntry)) {
-          entries.put(parentEntry.getKey(), parentEntry.getValue());
-        }
-      } else {
-        entries.put(parentEntry.getKey(), parentEntry.getValue());
-      }
+  private class SystemNamespaceFilter implements PropertyFilter {
+
+    private PropertyFilter userFilter;
+
+    SystemNamespaceFilter(PropertyFilter userFilter) {
+      this.userFilter = userFilter;
     }
 
-    List<String> children = getPropCache().getChildren(
-        ZooUtil.getRoot(inst.getInstanceID()) + Constants.ZNAMESPACES + "/" + getNamespaceId() + Constants.ZNAMESPACE_CONF);
+    @Override
+    public boolean accept(String key) {
+      if (isIterConst(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) {
-        String value = get(child);
-        if (child != null && value != null)
-          entries.put(child, value);
+        if (child != null && filter.accept(child)) {
+          String value = get(zc, child);
+          if (value != null)
+            props.put(child, value);
+        }
       }
     }
-
-    return entries.entrySet().iterator();
   }
 
   protected String getNamespaceId() {
@@ -153,10 +166,7 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
       co.propertiesChanged();
   }
 
-  protected boolean isIterConst(Entry<String,String> e) {
-    if (e.getKey().startsWith(Property.TABLE_ITERATOR_PREFIX.getKey()) || e.getKey().startsWith(Property.TABLE_CONSTRAINT_PREFIX.getKey())) {
-      return true;
-    }
-    return false;
+  protected boolean isIterConst(String key) {
+    return key.startsWith(Property.TABLE_ITERATOR_PREFIX.getKey()) || key.startsWith(Property.TABLE_CONSTRAINT_PREFIX.getKey());
   }
 }


[03/50] [abbrv] git commit: ACCUMULO-802 updated more shell commands to include the tableNamespaces option

Posted by ct...@apache.org.
ACCUMULO-802 updated more shell commands to include the tableNamespaces option


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

Branch: refs/heads/master
Commit: 5acd6a480a93b1c13639db8e8ba64a81c32651bb
Parents: 431f4e2
Author: Sean Hickey <ta...@gmail.com>
Authored: Tue Aug 6 08:54:05 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:09 2013 -0500

----------------------------------------------------------------------
 .../client/admin/TableNamespaceOperations.java  |    9 +
 .../admin/TableNamespaceOperationsImpl.java     |   46 +
 .../core/client/admin/TableOperationsImpl.java  |  411 ++---
 .../core/client/impl/thrift/ClientService.java  | 1431 ++++++++++++++++++
 .../mock/MockTableNamespaceOperations.java      |   17 +-
 .../util/shell/commands/ConstraintCommand.java  |   18 +-
 .../util/shell/commands/DeleteIterCommand.java  |   37 +-
 .../util/shell/commands/ListIterCommand.java    |   32 +-
 .../core/util/shell/commands/OptUtil.java       |   25 +
 .../util/shell/commands/SetIterCommand.java     |   98 +-
 .../core/util/shell/commands/TablesCommand.java |    8 +-
 core/src/main/thrift/client.thrift              |    1 +
 .../server/client/ClientServiceHandler.java     |   46 +
 .../server/conf/TableNamespaceConfWatcher.java  |  107 ++
 .../server/conf/TableNamespaceConfWatcher.java  |  107 --
 .../org/apache/accumulo/test/ShellServerIT.java |    4 +
 16 files changed, 2048 insertions(+), 349 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/5acd6a48/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
index f572104..314d007 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperations.java
@@ -414,4 +414,13 @@ public interface TableNamespaceOperations {
    *           thrown if the table namespace no longer exists
    */
   public Map<String,Integer> listConstraints(String tableNamespace) throws AccumuloException, TableNamespaceNotFoundException;
+
+  
+  /**
+   * Test to see if the instance can load the given class as the given type. This check uses the table classpath property if it is set.
+   * 
+   * @return true if the instance can load the given class as the given type, false otherwise
+   */
+  boolean testClassLoad(String namespace, String className, String asTypeName) throws TableNamespaceNotFoundException, AccumuloException,
+      AccumuloSecurityException;
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5acd6a48/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
index 0d54b51..90d59af 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableNamespaceOperationsImpl.java
@@ -19,6 +19,7 @@ package org.apache.accumulo.core.client.admin;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -34,6 +35,7 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNamespaceExistsException;
 import org.apache.accumulo.core.client.TableNamespaceNotEmptyException;
@@ -49,7 +51,10 @@ import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.ClientService;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+import org.apache.accumulo.core.constraints.Constraint;
 import org.apache.accumulo.core.iterators.IteratorUtil;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
 import org.apache.accumulo.core.master.thrift.TableOperation;
 import org.apache.accumulo.core.security.Credentials;
@@ -599,4 +604,45 @@ public class TableNamespaceOperationsImpl extends TableNamespaceOperationsHelper
     return new TableOperationsImpl(instance, credentials);
   }
 
+  @Override
+  public void attachIterator(String namespace, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
+      TableNamespaceNotFoundException {
+    testClassLoad(namespace, setting.getIteratorClass(), SortedKeyValueIterator.class.getName());
+    super.attachIterator(namespace, setting, scopes);
+  }
+
+  @Override
+  public int addConstraint(String namespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException, TableNamespaceNotFoundException {
+    testClassLoad(namespace, constraintClassName, Constraint.class.getName());
+    return super.addConstraint(namespace, constraintClassName);
+  }
+
+  @Override
+  public boolean testClassLoad(final String namespace, final String className, final String asTypeName) throws TableNamespaceNotFoundException,
+      AccumuloException, AccumuloSecurityException {
+    ArgumentChecker.notNull(namespace, className, asTypeName);
+
+    try {
+      return ServerClient.executeRaw(instance, new ClientExecReturn<Boolean,ClientService.Client>() {
+        @Override
+        public Boolean execute(ClientService.Client client) throws Exception {
+          return client.checkTableNamespaceClass(Tracer.traceInfo(), credentials.toThrift(instance), namespace, className, asTypeName);
+        }
+      });
+    } catch (ThriftTableOperationException e) {
+      switch (e.getType()) {
+        case NOTFOUND:
+          throw new TableNamespaceNotFoundException(e);
+        case OTHER:
+        default:
+          throw new AccumuloException(e.description, e);
+      }
+    } catch (ThriftSecurityException e) {
+      throw new AccumuloSecurityException(e.user, e.code, e);
+    } catch (AccumuloException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new AccumuloException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5acd6a48/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
index 18d2e40..be1a783 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
@@ -127,11 +127,11 @@ import org.apache.thrift.transport.TTransportException;
 public class TableOperationsImpl extends TableOperationsHelper {
   private Instance instance;
   private Credentials credentials;
-  
+
   public static final String CLONE_EXCLUDE_PREFIX = "!";
 
   private static final Logger log = Logger.getLogger(TableOperations.class);
-  
+
   /**
    * @param instance
    *          the connection information for this instance
@@ -143,7 +143,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
     this.instance = instance;
     this.credentials = credentials;
   }
-  
+
   /**
    * Retrieve a list of tables in Accumulo.
    * 
@@ -156,7 +156,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
     opTimer.stop("Fetched " + tableNames.size() + " table names in %DURATION%");
     return tableNames;
   }
-  
+
   /**
    * A method to check if a table exists in Accumulo.
    * 
@@ -169,13 +169,13 @@ public class TableOperationsImpl extends TableOperationsHelper {
     ArgumentChecker.notNull(tableName);
     if (tableName.equals(MetadataTable.NAME) || tableName.equals(RootTable.NAME))
       return true;
-    
+
     OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Checking if table " + tableName + " exists...");
     boolean exists = Tables.getNameToIdMap(instance).containsKey(tableName);
     opTimer.stop("Checked existance of " + exists + " in %DURATION%");
     return exists;
   }
-  
+
   /**
    * Create a table with no special configuration
    * 
@@ -192,7 +192,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   public void create(String tableName) throws AccumuloException, AccumuloSecurityException, TableExistsException {
     create(tableName, true, TimeType.MILLIS);
   }
-  
+
   /**
    * @param tableName
    *          the name of the table
@@ -203,7 +203,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   public void create(String tableName, boolean limitVersion) throws AccumuloException, AccumuloSecurityException, TableExistsException {
     create(tableName, limitVersion, TimeType.MILLIS);
   }
-  
+
   /**
    * @param tableName
    *          the name of the table
@@ -215,12 +215,12 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void create(String tableName, boolean limitVersion, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException {
     ArgumentChecker.notNull(tableName, timeType);
-    
+
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes()), ByteBuffer.wrap(timeType.name().getBytes()));
-    
+
     // Map<String,String> opts = IteratorUtil.generateInitialTableProperties(limitVersion);
     Map<String,String> opts = new HashMap<String,String>();
-    
+
     String namespace = Tables.extractNamespace(tableName);
     if (!namespaceExists(namespace)) {
       String info = "Table namespace not found while trying to create table";
@@ -229,7 +229,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       String info = "Can't create tables in the system namespace";
       throw new IllegalArgumentException(info);
     }
-    
+
     try {
       doTableOperation(TableOperation.CREATE, args, opts);
     } catch (TableNotFoundException e1) {
@@ -237,7 +237,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       throw new RuntimeException(e1);
     }
   }
-  
+
   private long beginTableOperation() throws ThriftSecurityException, TException {
     while (true) {
       MasterClientService.Iface client = null;
@@ -252,7 +252,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       }
     }
   }
-  
+
   private void executeTableOperation(long opid, TableOperation op, List<ByteBuffer> args, Map<String,String> opts, boolean autoCleanUp)
       throws ThriftSecurityException, TException, ThriftTableOperationException {
     while (true) {
@@ -269,7 +269,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       }
     }
   }
-  
+
   private String waitForTableOperation(long opid) throws ThriftSecurityException, TException, ThriftTableOperationException {
     while (true) {
       MasterClientService.Iface client = null;
@@ -284,7 +284,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       }
     }
   }
-  
+
   private void finishTableOperation(long opid) throws ThriftSecurityException, TException {
     while (true) {
       MasterClientService.Iface client = null;
@@ -300,16 +300,16 @@ public class TableOperationsImpl extends TableOperationsHelper {
       }
     }
   }
-  
+
   private String doTableOperation(TableOperation op, List<ByteBuffer> args, Map<String,String> opts) throws AccumuloSecurityException, TableExistsException,
       TableNotFoundException, AccumuloException {
     return doTableOperation(op, args, opts, true);
   }
-  
+
   private String doTableOperation(TableOperation op, List<ByteBuffer> args, Map<String,String> opts, boolean wait) throws AccumuloSecurityException,
       TableExistsException, TableNotFoundException, AccumuloException {
     Long opid = null;
-    
+
     try {
       opid = beginTableOperation();
       executeTableOperation(opid, op, args, opts, !wait);
@@ -348,14 +348,14 @@ public class TableOperationsImpl extends TableOperationsHelper {
         }
     }
   }
-  
+
   private static class SplitEnv {
     private String tableName;
     private String tableId;
     private ExecutorService executor;
     private CountDownLatch latch;
     private AtomicReference<Exception> exception;
-    
+
     SplitEnv(String tableName, String tableId, ExecutorService executor, CountDownLatch latch, AtomicReference<Exception> exception) {
       this.tableName = tableName;
       this.tableId = tableId;
@@ -364,47 +364,47 @@ public class TableOperationsImpl extends TableOperationsHelper {
       this.exception = exception;
     }
   }
-  
+
   private class SplitTask implements Runnable {
-    
+
     private List<Text> splits;
     private SplitEnv env;
-    
+
     SplitTask(SplitEnv env, List<Text> splits) {
       this.env = env;
       this.splits = splits;
     }
-    
+
     @Override
     public void run() {
       try {
         if (env.exception.get() != null)
           return;
-        
+
         if (splits.size() <= 2) {
           addSplits(env.tableName, new TreeSet<Text>(splits), env.tableId);
           for (int i = 0; i < splits.size(); i++)
             env.latch.countDown();
           return;
         }
-        
+
         int mid = splits.size() / 2;
-        
+
         // split the middle split point to ensure that child task split different tablets and can therefore
         // run in parallel
         addSplits(env.tableName, new TreeSet<Text>(splits.subList(mid, mid + 1)), env.tableId);
         env.latch.countDown();
-        
+
         env.executor.submit(new SplitTask(env, splits.subList(0, mid)));
         env.executor.submit(new SplitTask(env, splits.subList(mid + 1, splits.size())));
-        
+
       } catch (Exception e) {
         env.exception.compareAndSet(null, e);
       }
     }
-    
+
   }
-  
+
   /**
    * @param tableName
    *          the name of the table
@@ -420,19 +420,19 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void addSplits(String tableName, SortedSet<Text> partitionKeys) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
     String tableId = Tables.getTableId(instance, tableName);
-    
+
     List<Text> splits = new ArrayList<Text>(partitionKeys);
     // should be sorted because we copied from a sorted set, but that makes assumptions about
     // how the copy was done so resort to be sure.
     Collections.sort(splits);
-    
+
     CountDownLatch latch = new CountDownLatch(splits.size());
     AtomicReference<Exception> exception = new AtomicReference<Exception>(null);
-    
+
     ExecutorService executor = Executors.newFixedThreadPool(16, new NamingThreadFactory("addSplits"));
     try {
       executor.submit(new SplitTask(new SplitEnv(tableName, tableId, executor, latch, exception), splits));
-      
+
       while (!latch.await(100, TimeUnit.MILLISECONDS)) {
         if (exception.get() != null) {
           executor.shutdownNow();
@@ -455,24 +455,24 @@ public class TableOperationsImpl extends TableOperationsHelper {
       executor.shutdown();
     }
   }
-  
+
   private void addSplits(String tableName, SortedSet<Text> partitionKeys, String tableId) throws AccumuloException, AccumuloSecurityException,
       TableNotFoundException, AccumuloServerException {
     TabletLocator tabLocator = TabletLocator.getLocator(instance, new Text(tableId));
-    
+
     for (Text split : partitionKeys) {
       boolean successful = false;
       int attempt = 0;
-      
+
       while (!successful) {
-        
+
         if (attempt > 0)
           UtilWaitThread.sleep(100);
-        
+
         attempt++;
-        
+
         TabletLocation tl = tabLocator.locateTablet(credentials, split, false, false);
-        
+
         if (tl == null) {
           if (!Tables.exists(instance, tableId))
             throw new TableNotFoundException(tableId, tableName, null);
@@ -480,25 +480,25 @@ public class TableOperationsImpl extends TableOperationsHelper {
             throw new TableOfflineException(instance, tableId);
           continue;
         }
-        
+
         try {
           TabletClientService.Client client = ThriftUtil.getTServerClient(tl.tablet_location, ServerConfigurationUtil.getConfiguration(instance));
           try {
             OpTimer opTimer = null;
             if (log.isTraceEnabled())
               opTimer = new OpTimer(log, Level.TRACE).start("Splitting tablet " + tl.tablet_extent + " on " + tl.tablet_location + " at " + split);
-            
+
             client.splitTablet(Tracer.traceInfo(), credentials.toThrift(instance), tl.tablet_extent.toThrift(), TextUtil.getByteBuffer(split));
-            
+
             // just split it, might as well invalidate it in the cache
             tabLocator.invalidateCache(tl.tablet_extent);
-            
+
             if (opTimer != null)
               opTimer.stop("Split tablet in %DURATION%");
           } finally {
             ThriftUtil.returnClient(client);
           }
-          
+
         } catch (TApplicationException tae) {
           throw new AccumuloServerException(tl.tablet_location, tae);
         } catch (TTransportException e) {
@@ -516,15 +516,15 @@ public class TableOperationsImpl extends TableOperationsHelper {
           tabLocator.invalidateCache(tl.tablet_location);
           continue;
         }
-        
+
         successful = true;
       }
     }
   }
-  
+
   @Override
   public void merge(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    
+
     ArgumentChecker.notNull(tableName);
     ByteBuffer EMPTY = ByteBuffer.allocate(0);
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes()), start == null ? EMPTY : TextUtil.getByteBuffer(start), end == null ? EMPTY
@@ -537,10 +537,10 @@ public class TableOperationsImpl extends TableOperationsHelper {
       throw new RuntimeException(e);
     }
   }
-  
+
   @Override
   public void deleteRows(String tableName, Text start, Text end) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    
+
     ArgumentChecker.notNull(tableName);
     ByteBuffer EMPTY = ByteBuffer.allocate(0);
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes()), start == null ? EMPTY : TextUtil.getByteBuffer(start), end == null ? EMPTY
@@ -553,7 +553,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       throw new RuntimeException(e);
     }
   }
-  
+
   /**
    * @param tableName
    *          the name of the table
@@ -561,13 +561,13 @@ public class TableOperationsImpl extends TableOperationsHelper {
    */
   @Override
   public Collection<Text> listSplits(String tableName) throws TableNotFoundException, AccumuloSecurityException {
-    
+
     ArgumentChecker.notNull(tableName);
-    
+
     String tableId = Tables.getTableId(instance, tableName);
-    
+
     TreeMap<KeyExtent,String> tabletLocations = new TreeMap<KeyExtent,String>();
-    
+
     while (true) {
       try {
         tabletLocations.clear();
@@ -580,25 +580,25 @@ public class TableOperationsImpl extends TableOperationsHelper {
         if (!Tables.exists(instance, tableId)) {
           throw new TableNotFoundException(tableId, tableName, null);
         }
-        
+
         if (e instanceof RuntimeException && e.getCause() instanceof AccumuloSecurityException) {
           throw (AccumuloSecurityException) e.getCause();
         }
-        
+
         log.info(e.getMessage() + " ... retrying ...");
         UtilWaitThread.sleep(3000);
       }
     }
-    
+
     ArrayList<Text> endRows = new ArrayList<Text>(tabletLocations.size());
-    
+
     for (KeyExtent ke : tabletLocations.keySet())
       if (ke.getEndRow() != null)
         endRows.add(ke.getEndRow());
-    
+
     return endRows;
   }
-  
+
   @Deprecated
   @Override
   public Collection<Text> getSplits(String tableName) throws TableNotFoundException {
@@ -608,7 +608,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       throw new RuntimeException(e);
     }
   }
-  
+
   /**
    * @param tableName
    *          the name of the table
@@ -620,15 +620,15 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public Collection<Text> listSplits(String tableName, int maxSplits) throws TableNotFoundException, AccumuloSecurityException {
     Collection<Text> endRows = listSplits(tableName);
-    
+
     if (endRows.size() <= maxSplits)
       return endRows;
-    
+
     double r = (maxSplits + 1) / (double) (endRows.size());
     double pos = 0;
-    
+
     ArrayList<Text> subset = new ArrayList<Text>(maxSplits);
-    
+
     int j = 0;
     for (int i = 0; i < endRows.size() && j < maxSplits; i++) {
       pos += r;
@@ -638,10 +638,10 @@ public class TableOperationsImpl extends TableOperationsHelper {
         pos -= 1;
       }
     }
-    
+
     return subset;
   }
-  
+
   @Deprecated
   @Override
   public Collection<Text> getSplits(String tableName, int maxSplits) throws TableNotFoundException {
@@ -651,7 +651,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       throw new RuntimeException(e);
     }
   }
-  
+
   /**
    * Delete a table
    * 
@@ -667,51 +667,51 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void delete(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     ArgumentChecker.notNull(tableName);
-    
+
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes()));
     Map<String,String> opts = new HashMap<String,String>();
-    
+
     try {
       doTableOperation(TableOperation.DELETE, args, opts);
     } catch (TableExistsException e) {
       // should not happen
       throw new RuntimeException(e);
     }
-    
+
   }
-  
+
   @Override
   public void clone(String srcTableName, String newTableName, boolean flush, Map<String,String> propertiesToSet, Set<String> propertiesToExclude)
       throws AccumuloSecurityException, TableNotFoundException, AccumuloException, TableExistsException {
-    
+
     ArgumentChecker.notNull(srcTableName, newTableName);
-    
+
     String namespace = Tables.extractNamespace(newTableName);
     if (!namespaceExists(namespace)) {
       String info = "Table namespace not found while cloning table";
       throw new IllegalArgumentException(new TableNamespaceNotFoundException(null, namespace, info));
     }
-    
+
     String srcTableId = Tables.getTableId(instance, srcTableName);
-    
+
     if (flush)
       _flush(srcTableId, null, null, true);
-    
+
     if (propertiesToExclude == null)
       propertiesToExclude = Collections.emptySet();
-    
+
     if (propertiesToSet == null)
       propertiesToSet = Collections.emptyMap();
-    
+
     // TODO ACCUMULO-1565 needs to be fixed before the commented-out code below will work.
-    HashSet<String> excludeProps = new HashSet<String>();//getUniqueNamespaceProperties(namespace, srcTableName);
+    HashSet<String> excludeProps = new HashSet<String>();// getUniqueNamespaceProperties(namespace, srcTableName);
     for (String p : propertiesToExclude) {
       excludeProps.add(p);
     }
-    
+
     if (!Collections.disjoint(excludeProps, propertiesToSet.keySet()))
       throw new IllegalArgumentException("propertiesToSet and propertiesToExclude not disjoint");
-    
+
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(srcTableId.getBytes()), ByteBuffer.wrap(newTableName.getBytes()));
     Map<String,String> opts = new HashMap<String,String>();
     for (Entry<String,String> entry : propertiesToSet.entrySet()) {
@@ -719,14 +719,14 @@ public class TableOperationsImpl extends TableOperationsHelper {
         throw new IllegalArgumentException("Property can not start with " + CLONE_EXCLUDE_PREFIX);
       opts.put(entry.getKey(), entry.getValue());
     }
-    
+
     for (String prop : propertiesToExclude) {
       opts.put(CLONE_EXCLUDE_PREFIX + prop, "");
     }
-    
+
     doTableOperation(TableOperation.CLONE, args, opts);
   }
-  
+
   // get the properties that are only in the table namespace so that we can exclude them when copying table properties
   // TODO ACCUMULO-1565 needs fixed first
   private HashSet<String> getUniqueNamespaceProperties(String namespace, String table) throws TableNotFoundException, AccumuloException {
@@ -749,7 +749,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
     }
     return props;
   }
-  
+
   /**
    * Rename a table
    * 
@@ -769,18 +769,18 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void rename(String oldTableName, String newTableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException,
       TableExistsException {
-    
+
     String namespace = Tables.extractNamespace(newTableName);
     if (!namespaceExists(namespace)) {
       String info = "Table namespace not found while renaming table";
       throw new IllegalArgumentException(new TableNamespaceNotFoundException(null, namespace, info));
     }
-    
+
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldTableName.getBytes()), ByteBuffer.wrap(newTableName.getBytes()));
     Map<String,String> opts = new HashMap<String,String>();
     doTableOperation(TableOperation.RENAME, args, opts);
   }
-  
+
   /**
    * @deprecated since 1.4 {@link #flush(String, Text, Text, boolean)}
    */
@@ -793,7 +793,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       throw new AccumuloException(e.getMessage(), e);
     }
   }
-  
+
   /**
    * Flush a table
    * 
@@ -808,31 +808,31 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @Override
   public void flush(String tableName, Text start, Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     ArgumentChecker.notNull(tableName);
-    
+
     String tableId = Tables.getTableId(instance, tableName);
     _flush(tableId, start, end, wait);
   }
-  
+
   @Override
   public void compact(String tableName, Text start, Text end, boolean flush, boolean wait) throws AccumuloSecurityException, TableNotFoundException,
       AccumuloException {
     compact(tableName, start, end, new ArrayList<IteratorSetting>(), flush, wait);
   }
-  
+
   @Override
   public void compact(String tableName, Text start, Text end, List<IteratorSetting> iterators, boolean flush, boolean wait) throws AccumuloSecurityException,
       TableNotFoundException, AccumuloException {
     ArgumentChecker.notNull(tableName);
     ByteBuffer EMPTY = ByteBuffer.allocate(0);
-    
+
     String tableId = Tables.getTableId(instance, tableName);
-    
+
     if (flush)
       _flush(tableId, start, end, true);
-    
+
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes()), start == null ? EMPTY : TextUtil.getByteBuffer(start), end == null ? EMPTY
         : TextUtil.getByteBuffer(end), ByteBuffer.wrap(IteratorUtil.encodeIteratorSettings(iterators)));
-    
+
     Map<String,String> opts = new HashMap<String,String>();
     try {
       doTableOperation(TableOperation.COMPACT, args, opts, wait);
@@ -841,13 +841,13 @@ public class TableOperationsImpl extends TableOperationsHelper {
       throw new RuntimeException(e);
     }
   }
-  
+
   @Override
   public void cancelCompaction(String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException {
     String tableId = Tables.getTableId(instance, tableName);
-    
+
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes()));
-    
+
     Map<String,String> opts = new HashMap<String,String>();
     try {
       doTableOperation(TableOperation.COMPACT_CANCEL, args, opts, true);
@@ -855,17 +855,17 @@ public class TableOperationsImpl extends TableOperationsHelper {
       // should not happen
       throw new RuntimeException(e);
     }
-    
+
   }
-  
+
   private void _flush(String tableId, Text start, Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    
+
     try {
       long flushID;
-      
+
       // used to pass the table name. but the tableid associated with a table name could change between calls.
       // so pass the tableid to both calls
-      
+
       while (true) {
         MasterClientService.Iface client = null;
         try {
@@ -879,7 +879,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
           MasterClient.close(client);
         }
       }
-      
+
       while (true) {
         MasterClientService.Iface client = null;
         try {
@@ -912,7 +912,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       throw new AccumuloException(e);
     }
   }
-  
+
   /**
    * Sets a property on a table
    * 
@@ -937,7 +937,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       }
     });
   }
-  
+
   /**
    * Removes a property from a table
    * 
@@ -960,7 +960,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       }
     });
   }
-  
+
   /**
    * Gets properties of a table
    * 
@@ -993,9 +993,9 @@ public class TableOperationsImpl extends TableOperationsHelper {
     } catch (Exception e) {
       throw new AccumuloException(e);
     }
-    
+
   }
-  
+
   /**
    * Sets a tables locality groups. A tables locality groups can be changed at any time.
    * 
@@ -1015,22 +1015,22 @@ public class TableOperationsImpl extends TableOperationsHelper {
     // ensure locality groups do not overlap
     HashSet<Text> all = new HashSet<Text>();
     for (Entry<String,Set<Text>> entry : groups.entrySet()) {
-      
+
       if (!Collections.disjoint(all, entry.getValue())) {
         throw new IllegalArgumentException("Group " + entry.getKey() + " overlaps with another group");
       }
-      
+
       all.addAll(entry.getValue());
     }
-    
+
     for (Entry<String,Set<Text>> entry : groups.entrySet()) {
       Set<Text> colFams = entry.getValue();
       String value = LocalityGroupUtil.encodeColumnFamilies(colFams);
       setProperty(tableName, Property.TABLE_LOCALITY_GROUP_PREFIX + entry.getKey(), value);
     }
-    
+
     setProperty(tableName, Property.TABLE_LOCALITY_GROUPS.getKey(), StringUtil.join(groups.keySet(), ","));
-    
+
     // remove anything extraneous
     String prefix = Property.TABLE_LOCALITY_GROUP_PREFIX.getKey();
     for (Entry<String,String> entry : getProperties(tableName)) {
@@ -1040,14 +1040,14 @@ public class TableOperationsImpl extends TableOperationsHelper {
         // one:
         String[] parts = property.split("\\.");
         String group = parts[parts.length - 1];
-        
+
         if (!groups.containsKey(group)) {
           removeProperty(tableName, property);
         }
       }
     }
   }
-  
+
   /**
    * 
    * Gets the locality groups currently set for a table.
@@ -1064,22 +1064,22 @@ public class TableOperationsImpl extends TableOperationsHelper {
   public Map<String,Set<Text>> getLocalityGroups(String tableName) throws AccumuloException, TableNotFoundException {
     AccumuloConfiguration conf = new ConfigurationCopy(this.getProperties(tableName));
     Map<String,Set<ByteSequence>> groups = LocalityGroupUtil.getLocalityGroups(conf);
-    
+
     Map<String,Set<Text>> groups2 = new HashMap<String,Set<Text>>();
     for (Entry<String,Set<ByteSequence>> entry : groups.entrySet()) {
-      
+
       HashSet<Text> colFams = new HashSet<Text>();
-      
+
       for (ByteSequence bs : entry.getValue()) {
         colFams.add(new Text(bs.toArray()));
       }
-      
+
       groups2.put(entry.getKey(), colFams);
     }
-    
+
     return groups2;
   }
-  
+
   /**
    * @param tableName
    *          the name of the table
@@ -1103,7 +1103,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       throw new IllegalArgumentException("maximum splits must be >= 1");
     if (maxSplits == 1)
       return Collections.singleton(range);
-    
+
     Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
     String tableId = Tables.getTableId(instance, tableName);
     TabletLocator tl = TabletLocator.getLocator(instance, new Text(tableId));
@@ -1114,24 +1114,24 @@ public class TableOperationsImpl extends TableOperationsHelper {
         throw new TableDeletedException(tableId);
       if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
         throw new TableOfflineException(instance, tableId);
-      
+
       log.warn("Unable to locate bins for specified range. Retrying.");
       // sleep randomly between 100 and 200ms
       UtilWaitThread.sleep(100 + (int) (Math.random() * 100));
       binnedRanges.clear();
       tl.invalidateCache();
     }
-    
+
     // group key extents to get <= maxSplits
     LinkedList<KeyExtent> unmergedExtents = new LinkedList<KeyExtent>();
     List<KeyExtent> mergedExtents = new ArrayList<KeyExtent>();
-    
+
     for (Map<KeyExtent,List<Range>> map : binnedRanges.values())
       unmergedExtents.addAll(map.keySet());
-    
+
     // the sort method is efficient for linked list
     Collections.sort(unmergedExtents);
-    
+
     while (unmergedExtents.size() + mergedExtents.size() > maxSplits) {
       if (unmergedExtents.size() >= 2) {
         KeyExtent first = unmergedExtents.removeFirst();
@@ -1144,15 +1144,15 @@ public class TableOperationsImpl extends TableOperationsHelper {
         unmergedExtents.addAll(mergedExtents);
         mergedExtents.clear();
       }
-      
+
     }
-    
+
     mergedExtents.addAll(unmergedExtents);
-    
+
     Set<Range> ranges = new HashSet<Range>();
     for (KeyExtent k : mergedExtents)
       ranges.add(k.toDataRange().clip(range));
-    
+
     return ranges;
   }
 
@@ -1191,14 +1191,14 @@ public class TableOperationsImpl extends TableOperationsHelper {
   public void importDirectory(String tableName, String dir, String failureDir, boolean setTime) throws IOException, AccumuloSecurityException,
       TableNotFoundException, AccumuloException {
     ArgumentChecker.notNull(tableName, dir, failureDir);
-    
+
     Path dirPath = checkPath(dir, "Bulk", "");
     Path failPath = checkPath(failureDir, "Bulk", "failure");
 
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes()), ByteBuffer.wrap(dirPath.toString().getBytes()),
         ByteBuffer.wrap(failPath.toString().getBytes()), ByteBuffer.wrap((setTime + "").getBytes()));
     Map<String,String> opts = new HashMap<String,String>();
-    
+
     try {
       doTableOperation(TableOperation.BULK_IMPORT, args, opts);
     } catch (TableExistsException e) {
@@ -1208,10 +1208,10 @@ public class TableOperationsImpl extends TableOperationsHelper {
     // return new BulkImportHelper(instance, credentials, tableName).importDirectory(new Path(dir), new Path(failureDir), numThreads, numAssignThreads,
     // disableGC);
   }
-  
+
   private void waitForTableStateTransition(String tableId, TableState expectedState) throws AccumuloException, TableNotFoundException,
       AccumuloSecurityException {
-    
+
     Text startRow = null;
     Text lastRow = null;
 
@@ -1225,12 +1225,13 @@ public class TableOperationsImpl extends TableOperationsHelper {
           throw new AccumuloException("Unexpected table state " + tableId + " " + Tables.getTableState(instance, tableId) + " != " + expectedState);
         }
       }
-      
+
       Range range = new KeyExtent(new Text(tableId), null, null).toMetadataRange();
-      if (startRow == null || lastRow == null) 
+      if (startRow == null || lastRow == null)
         range = new KeyExtent(new Text(tableId), null, null).toMetadataRange();
       else
         range = new Range(startRow, lastRow);
+
       String metaTable = MetadataTable.NAME;
       if (tableId.equals(MetadataTable.ID))
         metaTable = RootTable.NAME;
@@ -1239,83 +1240,84 @@ public class TableOperationsImpl extends TableOperationsHelper {
       TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
       scanner.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
       scanner.setRange(range);
-      
+
       RowIterator rowIter = new RowIterator(scanner);
-      
+
       KeyExtent lastExtent = null;
-      
+
       int total = 0;
       int waitFor = 0;
       int holes = 0;
       Text continueRow = null;
       MapCounter<String> serverCounts = new MapCounter<String>();
-      
+
       while (rowIter.hasNext()) {
         Iterator<Entry<Key,Value>> row = rowIter.next();
-        
+
         total++;
 
         KeyExtent extent = null;
         String future = null;
         String current = null;
-        
+
         while (row.hasNext()) {
           Entry<Key,Value> entry = row.next();
           Key key = entry.getKey();
-          
+
           if (key.getColumnFamily().equals(TabletsSection.FutureLocationColumnFamily.NAME))
             future = entry.getValue().toString();
-          
+
           if (key.getColumnFamily().equals(TabletsSection.CurrentLocationColumnFamily.NAME))
             current = entry.getValue().toString();
-          
+
           if (TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key))
             extent = new KeyExtent(key.getRow(), entry.getValue());
         }
-        
+
         if ((expectedState == TableState.ONLINE && current == null) || (expectedState == TableState.OFFLINE && (future != null || current != null))) {
           if (continueRow == null)
             continueRow = extent.getMetadataEntry();
           waitFor++;
           lastRow = extent.getMetadataEntry();
-          
-          if(current != null)
+
+          if (current != null)
             serverCounts.increment(current, 1);
-          if(future != null)
+          if (future != null)
             serverCounts.increment(future, 1);
         }
-        
+
         if (!extent.getTableId().toString().equals(tableId)) {
           throw new AccumuloException("Saw unexpected table Id " + tableId + " " + extent);
         }
-        
+
         if (lastExtent != null && !extent.isPreviousExtent(lastExtent)) {
           holes++;
         }
-        
+
         lastExtent = extent;
       }
-      
+
       if (continueRow != null) {
         startRow = continueRow;
       }
-      
+
       if (holes > 0 || total == 0) {
         startRow = null;
         lastRow = null;
       }
-      
+
       if (waitFor > 0 || holes > 0 || total == 0) {
         long waitTime;
         long maxPerServer = 0;
-        if(serverCounts.size() > 0){
+        if (serverCounts.size() > 0) {
           maxPerServer = Collections.max(serverCounts.values());
           waitTime = maxPerServer * 10;
-        }else
+        } else
           waitTime = waitFor * 10;
         waitTime = Math.max(100, waitTime);
         waitTime = Math.min(5000, waitTime);
-        log.trace("Waiting for " + waitFor + "("+maxPerServer+") tablets, startRow = " + startRow + " lastRow = "+lastRow+", holes=" + holes+" sleeping:"+waitTime+"ms");
+        log.trace("Waiting for " + waitFor + "(" + maxPerServer + ") tablets, startRow = " + startRow + " lastRow = " + lastRow + ", holes=" + holes
+            + " sleeping:" + waitTime + "ms");
         UtilWaitThread.sleep(waitTime);
       } else {
         break;
@@ -1323,11 +1325,12 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
     }
   }
+
   @Override
   public void offline(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
     offline(tableName, false);
   }
-  
+
   /**
    * 
    * @param tableName
@@ -1340,28 +1343,28 @@ public class TableOperationsImpl extends TableOperationsHelper {
    */
   @Override
   public void offline(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
-    
+
     ArgumentChecker.notNull(tableName);
     String tableId = Tables.getTableId(instance, tableName);
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes()));
     Map<String,String> opts = new HashMap<String,String>();
-    
+
     try {
       doTableOperation(TableOperation.OFFLINE, args, opts);
     } catch (TableExistsException e) {
       // should not happen
       throw new RuntimeException(e);
     }
-    
-    if(wait)
+
+    if (wait)
       waitForTableStateTransition(tableId, TableState.OFFLINE);
   }
-  
+
   @Override
   public void online(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
     online(tableName, false);
   }
-  
+
   /**
    * 
    * @param tableName
@@ -1378,18 +1381,18 @@ public class TableOperationsImpl extends TableOperationsHelper {
     String tableId = Tables.getTableId(instance, tableName);
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes()));
     Map<String,String> opts = new HashMap<String,String>();
-    
+
     try {
       doTableOperation(TableOperation.ONLINE, args, opts);
     } catch (TableExistsException e) {
       // should not happen
       throw new RuntimeException(e);
     }
-    
-    if(wait)
+
+    if (wait)
       waitForTableStateTransition(tableId, TableState.ONLINE);
   }
-  
+
   /**
    * Clears the tablet locator cache for a specified table
    * 
@@ -1404,7 +1407,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
     TabletLocator tabLocator = TabletLocator.getLocator(instance, new Text(Tables.getTableId(instance, tableName)));
     tabLocator.invalidateCache();
   }
-  
+
   /**
    * Get a mapping of table name to internal table id.
    * 
@@ -1414,7 +1417,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
   public Map<String,String> tableIdMap() {
     return Tables.getNameToIdMap(instance);
   }
-  
+
   @Override
   public Text getMaxRow(String tableName, Authorizations auths, Text startRow, boolean startInclusive, Text endRow, boolean endInclusive)
       throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
@@ -1422,10 +1425,10 @@ public class TableOperationsImpl extends TableOperationsHelper {
     Scanner scanner = instance.getConnector(credentials.getPrincipal(), credentials.getToken()).createScanner(tableName, auths);
     return FindMax.findMax(scanner, startRow, startInclusive, endRow, endInclusive);
   }
-  
+
   @Override
   public List<DiskUsage> getDiskUsage(Set<String> tableNames) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    
+
     List<TDiskUsage> diskUsages = null;
     while (diskUsages == null) {
       Pair<String,Client> pair = null;
@@ -1457,18 +1460,18 @@ public class TableOperationsImpl extends TableOperationsHelper {
           ServerClient.close(pair.getSecond());
       }
     }
-    
+
     List<DiskUsage> finalUsages = new ArrayList<DiskUsage>();
     for (TDiskUsage diskUsage : diskUsages) {
       finalUsages.add(new DiskUsage(new TreeSet<String>(diskUsage.getTables()), diskUsage.getUsage()));
     }
-    
+
     return finalUsages;
   }
-  
+
   public static Map<String,String> getExportedProps(FileSystem fs, Path path) throws IOException {
     HashMap<String,String> props = new HashMap<String,String>();
-    
+
     ZipInputStream zis = new ZipInputStream(fs.open(path));
     try {
       ZipEntry zipEntry;
@@ -1480,7 +1483,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
             String sa[] = line.split("=", 2);
             props.put(sa[0], sa[1]);
           }
-          
+
           break;
         }
       }
@@ -1489,11 +1492,11 @@ public class TableOperationsImpl extends TableOperationsHelper {
     }
     return props;
   }
-  
+
   @Override
   public void importTable(String tableName, String importDir) throws TableExistsException, AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(tableName, importDir);
-    
+
     try {
       importDir = checkPath(importDir, "Table", "").toString();
     } catch (IOException e) {
@@ -1503,45 +1506,45 @@ public class TableOperationsImpl extends TableOperationsHelper {
     try {
       FileSystem fs = new Path(importDir).getFileSystem(CachedConfiguration.getInstance());
       Map<String,String> props = getExportedProps(fs, new Path(importDir, Constants.EXPORT_FILE));
-      
+
       for (String propKey : props.keySet()) {
         if (Property.isClassProperty(propKey) && !props.get(propKey).contains(Constants.CORE_PACKAGE_NAME)) {
           Logger.getLogger(this.getClass()).info(
               "Imported table sets '" + propKey + "' to '" + props.get(propKey) + "'.  Ensure this class is on Accumulo classpath.");
         }
       }
-      
+
     } catch (IOException ioe) {
       Logger.getLogger(this.getClass()).warn("Failed to check if imported table references external java classes : " + ioe.getMessage());
     }
-    
+
     String namespace = Tables.extractNamespace(tableName);
     if (!namespaceExists(namespace)) {
       String info = "Table namespace not found while importing to table";
       throw new RuntimeException(new TableNamespaceNotFoundException(null, namespace, info));
     }
-    
+
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes()), ByteBuffer.wrap(importDir.getBytes()));
-    
+
     Map<String,String> opts = Collections.emptyMap();
-    
+
     try {
       doTableOperation(TableOperation.IMPORT, args, opts);
     } catch (TableNotFoundException e1) {
       // should not happen
       throw new RuntimeException(e1);
     }
-    
+
   }
-  
+
   @Override
   public void exportTable(String tableName, String exportDir) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
     ArgumentChecker.notNull(tableName, exportDir);
-    
+
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes()), ByteBuffer.wrap(exportDir.getBytes()));
-    
+
     Map<String,String> opts = Collections.emptyMap();
-    
+
     try {
       doTableOperation(TableOperation.EXPORT, args, opts);
     } catch (TableExistsException e1) {
@@ -1549,12 +1552,12 @@ public class TableOperationsImpl extends TableOperationsHelper {
       throw new RuntimeException(e1);
     }
   }
-  
+
   @Override
   public boolean testClassLoad(final String tableName, final String className, final String asTypeName) throws TableNotFoundException, AccumuloException,
       AccumuloSecurityException {
     ArgumentChecker.notNull(tableName, className, asTypeName);
-    
+
     try {
       return ServerClient.executeRaw(instance, new ClientExecReturn<Boolean,ClientService.Client>() {
         @Override
@@ -1578,20 +1581,20 @@ public class TableOperationsImpl extends TableOperationsHelper {
       throw new AccumuloException(e);
     }
   }
-  
+
   @Override
   public void attachIterator(String tableName, IteratorSetting setting, EnumSet<IteratorScope> scopes) throws AccumuloSecurityException, AccumuloException,
       TableNotFoundException {
     testClassLoad(tableName, setting.getIteratorClass(), SortedKeyValueIterator.class.getName());
     super.attachIterator(tableName, setting, scopes);
   }
-  
+
   @Override
   public int addConstraint(String tableName, String constraintClassName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     testClassLoad(tableName, constraintClassName, Constraint.class.getName());
     return super.addConstraint(tableName, constraintClassName);
   }
-  
+
   private boolean namespaceExists(String namespace) {
     return TableNamespaces.getNameToIdMap(instance).containsKey(namespace);
   }


[05/50] [abbrv] git commit: ACCUMULO-1479 added namespace permissions to the zookeeper upgrade part of Master

Posted by ct...@apache.org.
ACCUMULO-1479 added namespace permissions to the zookeeper upgrade part of Master


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

Branch: refs/heads/master
Commit: 02374a0e34c053709a76e7eacae5e9979aede737
Parents: 5f4b864
Author: Sean Hickey <ta...@gmail.com>
Authored: Wed Aug 7 13:59:50 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:10 2013 -0500

----------------------------------------------------------------------
 .../main/java/org/apache/accumulo/master/Master.java   | 13 +++++++++++++
 1 file changed, 13 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/02374a0e/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 293dbc9..0d508dd 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
@@ -78,6 +78,7 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.Lo
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.Credentials;
 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;
@@ -139,6 +140,7 @@ import org.apache.accumulo.server.master.state.ZooTabletStateStore;
 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.security.handler.ZKPermHandler;
 import org.apache.accumulo.server.tables.TableManager;
 import org.apache.accumulo.server.tables.TableObserver;
 import org.apache.accumulo.server.util.DefaultMap;
@@ -363,6 +365,17 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
                 NodeExistsPolicy.SKIP);
           }
         }
+
+        // add namespace permissions to existing users
+        ZKPermHandler perm = new ZKPermHandler();
+        perm.initialize(instance.getInstanceID(), true);
+        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.grantTableNamespacePermission("root", Constants.SYSTEM_TABLE_NAMESPACE_ID, TableNamespacePermission.ALTER_TABLE);
+
       } catch (Exception ex) {
         log.fatal("Error performing upgrade", ex);
         System.exit(1);


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

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/security/NamespacePermission.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/security/NamespacePermission.java b/core/src/main/java/org/apache/accumulo/core/security/NamespacePermission.java
new file mode 100644
index 0000000..fb6847b
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/security/NamespacePermission.java
@@ -0,0 +1,66 @@
+/*
+ * 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.core.security;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public enum NamespacePermission {
+  // One can add new permissions, with new numbers, but please don't change or use numbers previously assigned
+  READ((byte) 0),
+  WRITE((byte) 1),
+  ALTER_NAMESPACE((byte) 2),
+  GRANT((byte) 3),
+  ALTER_TABLE((byte) 4),
+  CREATE_TABLE((byte) 5),
+  DROP_TABLE((byte) 6);
+
+  final private byte permID;
+
+  final private static NamespacePermission mapping[] = new NamespacePermission[8];
+  static {
+    for (NamespacePermission perm : NamespacePermission.values())
+      mapping[perm.permID] = perm;
+  }
+
+  private NamespacePermission(byte id) {
+    this.permID = id;
+  }
+
+  public byte getId() {
+    return this.permID;
+  }
+
+  public static List<String> printableValues() {
+    NamespacePermission[] a = NamespacePermission.values();
+
+    List<String> list = new ArrayList<String>(a.length);
+
+    for (NamespacePermission p : a)
+      list.add("Namespace." + p);
+
+    return list;
+  }
+
+  public static NamespacePermission getPermissionById(byte id) {
+    NamespacePermission result = mapping[id];
+    if (result != null)
+      return result;
+    throw new IndexOutOfBoundsException("No such permission");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/security/TableNamespacePermission.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/security/TableNamespacePermission.java b/core/src/main/java/org/apache/accumulo/core/security/TableNamespacePermission.java
deleted file mode 100644
index 2cf57c7..0000000
--- a/core/src/main/java/org/apache/accumulo/core/security/TableNamespacePermission.java
+++ /dev/null
@@ -1,66 +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.core.security;
-
-import java.util.ArrayList;
-import java.util.List;
-
-public enum TableNamespacePermission {
-  // One can add new permissions, with new numbers, but please don't change or use numbers previously assigned
-  READ((byte) 0),
-  WRITE((byte) 1),
-  ALTER_NAMESPACE((byte) 2),
-  GRANT((byte) 3),
-  ALTER_TABLE((byte) 4),
-  CREATE_TABLE((byte) 5),
-  DROP_TABLE((byte) 6);
-
-  final private byte permID;
-
-  final private static TableNamespacePermission mapping[] = new TableNamespacePermission[8];
-  static {
-    for (TableNamespacePermission perm : TableNamespacePermission.values())
-      mapping[perm.permID] = perm;
-  }
-
-  private TableNamespacePermission(byte id) {
-    this.permID = id;
-  }
-
-  public byte getId() {
-    return this.permID;
-  }
-
-  public static List<String> printableValues() {
-    TableNamespacePermission[] a = TableNamespacePermission.values();
-
-    List<String> list = new ArrayList<String>(a.length);
-
-    for (TableNamespacePermission p : a)
-      list.add("Namespace." + p);
-
-    return list;
-  }
-
-  public static TableNamespacePermission getPermissionById(byte id) {
-    TableNamespacePermission result = mapping[id];
-    if (result != null)
-      return result;
-    throw new IndexOutOfBoundsException("No such permission");
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java b/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
index 7d13d5f..3a32d8e 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
@@ -727,12 +727,12 @@ public class Shell extends ShellOptions {
       userlist = Collections.emptySet();
     }
 
-    Set<String> tableNamespaces = null;
+    Set<String> namespaces = null;
     try {
-      tableNamespaces = connector.tableNamespaceOperations().list();
+      namespaces = connector.namespaceOperations().list();
     } catch (Exception e) {
-      log.debug("Unable to obtain list of table namespaces", e);
-      tableNamespaces = Collections.emptySet();
+      log.debug("Unable to obtain list of namespaces", e);
+      namespaces = Collections.emptySet();
     }
 
     Map<Command.CompletionSet,Set<String>> options = new HashMap<Command.CompletionSet,Set<String>>();
@@ -743,18 +743,18 @@ public class Shell extends ShellOptions {
 
     Set<String> modifiedUserlist = new HashSet<String>();
     Set<String> modifiedTablenames = new HashSet<String>();
-    Set<String> modifiedTableNamespaces = new HashSet<String>();
+    Set<String> modifiedNamespaces = new HashSet<String>();
 
     for (String a : tableNames)
       modifiedTablenames.add(a.replaceAll("([\\s'\"])", "\\\\$1"));
     for (String a : userlist)
       modifiedUserlist.add(a.replaceAll("([\\s'\"])", "\\\\$1"));
-    for (String a : tableNamespaces)
-      modifiedTableNamespaces.add(a.replaceAll("([\\s'\"])", "\\\\$1"));
+    for (String a : namespaces)
+      modifiedNamespaces.add(a.replaceAll("([\\s'\"])", "\\\\$1"));
 
     options.put(Command.CompletionSet.USERNAMES, modifiedUserlist);
     options.put(Command.CompletionSet.TABLENAMES, modifiedTablenames);
-    options.put(Command.CompletionSet.TABLENAMESPACES, modifiedTableNamespaces);
+    options.put(Command.CompletionSet.NAMESPACES, modifiedNamespaces);
     options.put(Command.CompletionSet.COMMANDS, commands);
 
     for (Command[] cmdGroup : commandGrouping.values()) {
@@ -775,7 +775,7 @@ public class Shell extends ShellOptions {
   public static abstract class Command {
     // Helper methods for completion
     public enum CompletionSet {
-      TABLENAMES, USERNAMES, COMMANDS, TABLENAMESPACES
+      TABLENAMES, USERNAMES, COMMANDS, NAMESPACES
     }
 
     static Set<String> getCommandNames(Map<CompletionSet,Set<String>> objects) {
@@ -790,8 +790,8 @@ public class Shell extends ShellOptions {
       return objects.get(CompletionSet.USERNAMES);
     }
 
-    static Set<String> getTableNamespaces(Map<CompletionSet,Set<String>> objects) {
-      return objects.get(CompletionSet.TABLENAMESPACES);
+    static Set<String> getNamespaces(Map<CompletionSet,Set<String>> objects) {
+      return objects.get(CompletionSet.NAMESPACES);
     }
 
     public void registerCompletionGeneral(Token root, Set<String> args, boolean caseSens) {
@@ -816,8 +816,8 @@ public class Shell extends ShellOptions {
       registerCompletionGeneral(root, completionSet.get(CompletionSet.COMMANDS), false);
     }
 
-    public void registerCompletionForTableNamespaces(Token root, Map<CompletionSet,Set<String>> completionSet) {
-      registerCompletionGeneral(root, completionSet.get(CompletionSet.TABLENAMESPACES), true);
+    public void registerCompletionForNamespaces(Token root, Map<CompletionSet,Set<String>> completionSet) {
+      registerCompletionGeneral(root, completionSet.get(CompletionSet.NAMESPACES), true);
     }
 
     // abstract methods to override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/ShellCompletor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/ShellCompletor.java b/core/src/main/java/org/apache/accumulo/core/util/shell/ShellCompletor.java
index 4046714..c64e0c7 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/ShellCompletor.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/ShellCompletor.java
@@ -28,23 +28,24 @@ import org.apache.accumulo.core.util.shell.Shell.Command.CompletionSet;
 import org.apache.accumulo.core.util.shell.commands.QuotedStringTokenizer;
 
 public class ShellCompletor implements Completer {
-  
+
   // private static final Logger log = Logger.getLogger(ShellCompletor.class);
-  
+
   Map<CompletionSet,Set<String>> options;
   Token root = null;
-  
+
   public ShellCompletor() {}
-  
+
   public ShellCompletor(Token root) {
     this.root = root;
   }
-  
+
   public ShellCompletor(Token rootToken, Map<CompletionSet,Set<String>> options) {
     this.root = rootToken;
     this.options = options;
   }
-  
+
+  @Override
   @SuppressWarnings({"unchecked", "rawtypes"})
   public int complete(String buffer, int cursor, List candidates) {
     try {
@@ -55,33 +56,33 @@ public class ShellCompletor implements Completer {
       return cursor;
     }
   }
-  
+
   private int _complete(String fullBuffer, int cursor, List<String> candidates) {
     boolean inTableFlag = false, inUserFlag = false, inNamespaceFlag = false;
     // Only want to grab the buffer up to the cursor because
     // the user could be trying to tab complete in the middle
     // of the line
     String buffer = fullBuffer.substring(0, cursor);
-    
+
     Token current_command_token = root;
     String current_string_token = null;
     boolean end_space = buffer.endsWith(" ");
-    
+
     // tabbing with no text
     if (buffer.length() == 0) {
       candidates.addAll(root.getSubcommandNames());
       return 0;
     }
-    
+
     String prefix = "";
-    
+
     QuotedStringTokenizer qst = new QuotedStringTokenizer(buffer);
-    
+
     Iterator<String> iter = qst.iterator();
     while (iter.hasNext()) {
       current_string_token = iter.next();
       current_string_token = current_string_token.replaceAll("([\\s'\"])", "\\\\$1");
-      
+
       if (!iter.hasNext()) {
         // if we end in a space and that space isn't part of the last token
         // (which would be the case at the start of a quote) OR the buffer
@@ -89,7 +90,7 @@ public class ShellCompletor implements Completer {
         // and not complete the current command.
         if (end_space && !current_string_token.endsWith(" ") || buffer.endsWith("\"")) {
           // match subcommands
-          
+
           // we're in a subcommand so try to match the universal
           // option flags if we're there
           if (current_string_token.trim().equals("-" + Shell.tableOption)) {
@@ -98,19 +99,19 @@ public class ShellCompletor implements Completer {
           } else if (current_string_token.trim().equals("-" + Shell.userOption)) {
             candidates.addAll(options.get(Shell.Command.CompletionSet.USERNAMES));
             prefix += "-" + Shell.userOption + " ";
-          } else if (current_string_token.trim().equals("-" + Shell.tableNamespaceOption)) {
-            candidates.addAll(options.get(Shell.Command.CompletionSet.TABLENAMESPACES));
-            prefix += "-" + Shell.tableNamespaceOption + " ";
-          }else if (current_command_token != null) {
+          } else if (current_string_token.trim().equals("-" + Shell.namespaceOption)) {
+            candidates.addAll(options.get(Shell.Command.CompletionSet.NAMESPACES));
+            prefix += "-" + Shell.namespaceOption + " ";
+          } else if (current_command_token != null) {
             Token next = current_command_token.getSubcommand(current_string_token);
             if (next != null) {
               current_command_token = next;
-              
+
               if (current_command_token.getCaseSensitive())
                 prefix += current_string_token + " ";
               else
                 prefix += current_string_token.toUpperCase() + " ";
-              
+
               candidates.addAll(current_command_token.getSubcommandNames());
             }
           }
@@ -118,7 +119,7 @@ public class ShellCompletor implements Completer {
           return (prefix.length());
         }
         // need to match current command
-        // if we're in -t <table>, -u <user>, or -tn <tableNamespace> complete those
+        // if we're in -t <table>, -u <user>, or -tn <namespace> complete those
         if (inTableFlag) {
           for (String a : options.get(Shell.Command.CompletionSet.TABLENAMES))
             if (a.startsWith(current_string_token))
@@ -128,33 +129,33 @@ public class ShellCompletor implements Completer {
             if (a.startsWith(current_string_token))
               candidates.add(a);
         } else if (inNamespaceFlag) {
-          for (String a : options.get(Shell.Command.CompletionSet.TABLENAMESPACES))
+          for (String a : options.get(Shell.Command.CompletionSet.NAMESPACES))
             if (a.startsWith(current_string_token))
               candidates.add(a);
         } else if (current_command_token != null)
           candidates.addAll(current_command_token.getSubcommandNames(current_string_token));
-        
+
         Collections.sort(candidates);
         return (prefix.length());
       }
-      
+
       if (current_string_token.trim().equals("-" + Shell.tableOption))
         inTableFlag = true;
       else if (current_string_token.trim().equals("-" + Shell.userOption))
         inUserFlag = true;
-      else if (current_string_token.trim().equals("-" + Shell.tableNamespaceOption))
+      else if (current_string_token.trim().equals("-" + Shell.namespaceOption))
         inNamespaceFlag = true;
       else
         inUserFlag = inTableFlag = inNamespaceFlag = false;
-      
+
       if (current_command_token != null && current_command_token.getCaseSensitive())
         prefix += current_string_token + " ";
       else
         prefix += current_string_token.toUpperCase() + " ";
-      
+
       if (current_command_token != null && current_command_token.getSubcommandNames().contains(current_string_token))
         current_command_token = current_command_token.getSubcommand(current_string_token);
-      
+
     }
     return 0;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/ShellOptions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/ShellOptions.java b/core/src/main/java/org/apache/accumulo/core/util/shell/ShellOptions.java
index af55d8d..4e573ed 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/ShellOptions.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/ShellOptions.java
@@ -25,9 +25,9 @@ public abstract class ShellOptions {
   // Global options flags
   public static final String userOption = "u";
   public static final String tableOption = "t";
-  public static final String tableNamespaceOption = "tn";
+  public static final String namespaceOption = "ns";
   public static final String helpOption = "?";
   public static final String helpLongOption = "help";
-  
+
   final Option helpOpt = new Option(helpOption, helpLongOption, false, "display this help");
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ConfigCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ConfigCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ConfigCommand.java
index 1e49c4d..90d02de 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ConfigCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ConfigCommand.java
@@ -28,9 +28,9 @@ import jline.console.ConsoleReader;
 
 import org.apache.accumulo.core.client.AccumuloException;
 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.TableNamespaces;
+import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -46,11 +46,11 @@ import org.apache.commons.cli.OptionGroup;
 import org.apache.commons.cli.Options;
 
 public class ConfigCommand extends Command {
-  private Option tableOpt, deleteOpt, setOpt, filterOpt, disablePaginationOpt, outputFileOpt, tableNamespaceOpt;
-  
+  private Option tableOpt, deleteOpt, setOpt, filterOpt, disablePaginationOpt, outputFileOpt, namespaceOpt;
+
   private int COL1 = 10, COL2 = 7;
   private ConsoleReader reader;
-  
+
   @Override
   public void registerCompletion(final Token root, final Map<Command.CompletionSet,Set<String>> completionSet) {
     final Token cmd = new Token(getName());
@@ -63,18 +63,19 @@ public class ConfigCommand extends Command {
     cmd.addSubcommand(sub);
     root.addSubcommand(cmd);
   }
-  
+
+  @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException,
-      TableNotFoundException, IOException, ClassNotFoundException, TableNamespaceNotFoundException {
+      TableNotFoundException, IOException, ClassNotFoundException, NamespaceNotFoundException {
     reader = shellState.getReader();
-    
+
     final String tableName = cl.getOptionValue(tableOpt.getOpt());
     if (tableName != null && !shellState.getConnector().tableOperations().exists(tableName)) {
       throw new TableNotFoundException(null, tableName, null);
     }
-    final String tableNamespace = cl.getOptionValue(tableNamespaceOpt.getOpt());
-    if (tableNamespace != null && !shellState.getConnector().tableNamespaceOperations().exists(tableNamespace)) {
-      throw new TableNamespaceNotFoundException(null, tableNamespace, null);
+    final String namespace = cl.getOptionValue(namespaceOpt.getOpt());
+    if (namespace != null && !shellState.getConnector().namespaceOperations().exists(namespace)) {
+      throw new NamespaceNotFoundException(null, namespace, null);
     }
     if (cl.hasOption(deleteOpt.getOpt())) {
       // delete property from table
@@ -88,12 +89,12 @@ public class ConfigCommand extends Command {
         }
         shellState.getConnector().tableOperations().removeProperty(tableName, property);
         Shell.log.debug("Successfully deleted table configuration option.");
-      } else if (tableNamespace != null) {
+      } else if (namespace != null) {
         if (!Property.isValidTablePropertyKey(property)) {
           Shell.log.warn("Invalid per-table property : " + property + ", still removing from zookeeper if it's there.");
         }
-        shellState.getConnector().tableNamespaceOperations().removeProperty(tableNamespace, property);
-        Shell.log.debug("Successfully deleted table namespace configuration option.");
+        shellState.getConnector().namespaceOperations().removeProperty(namespace, property);
+        Shell.log.debug("Successfully deleted namespace configuration option.");
       } else {
         if (!Property.isValidZooPropertyKey(property)) {
           Shell.log.warn("Invalid per-table property : " + property + ", still removing from zookeeper if it's there.");
@@ -110,7 +111,7 @@ public class ConfigCommand extends Command {
       final String pair[] = property.split("=", 2);
       property = pair[0];
       value = pair[1];
-      
+
       if (tableName != null) {
         if (!Property.isValidTablePropertyKey(property)) {
           throw new BadArgumentException("Invalid per-table property.", fullCommand, fullCommand.indexOf(property));
@@ -120,14 +121,14 @@ public class ConfigCommand extends Command {
         }
         shellState.getConnector().tableOperations().setProperty(tableName, property, value);
         Shell.log.debug("Successfully set table configuration option.");
-      } else if (tableNamespace != null) {
+      } else if (namespace != null) {
         if (!Property.isValidTablePropertyKey(property)) {
           throw new BadArgumentException("Invalid per-table property.", fullCommand, fullCommand.indexOf(property));
         }
         if (property.equals(Property.TABLE_DEFAULT_SCANTIME_VISIBILITY.getKey())) {
           new ColumnVisibility(value); // validate that it is a valid expression
         }
-        shellState.getConnector().tableNamespaceOperations().setProperty(tableNamespace, property, value);
+        shellState.getConnector().namespaceOperations().setProperty(namespace, property, value);
         Shell.log.debug("Successfully set table configuration option.");
       } else {
         if (!Property.isValidZooPropertyKey(property)) {
@@ -140,38 +141,38 @@ public class ConfigCommand extends Command {
       // display properties
       final TreeMap<String,String> systemConfig = new TreeMap<String,String>();
       systemConfig.putAll(shellState.getConnector().instanceOperations().getSystemConfiguration());
-      
+
       final String outputFile = cl.getOptionValue(outputFileOpt.getOpt());
       final PrintFile printFile = outputFile == null ? null : new PrintFile(outputFile);
-      
+
       final TreeMap<String,String> siteConfig = new TreeMap<String,String>();
       siteConfig.putAll(shellState.getConnector().instanceOperations().getSiteConfiguration());
-      
+
       final TreeMap<String,String> defaults = new TreeMap<String,String>();
       for (Entry<String,String> defaultEntry : AccumuloConfiguration.getDefaultConfiguration()) {
         defaults.put(defaultEntry.getKey(), defaultEntry.getValue());
       }
-      
+
       final TreeMap<String,String> namespaceConfig = new TreeMap<String,String>();
       if (tableName != null) {
-        String n = TableNamespaces.getNamespaceName(shellState.getInstance(),
+        String n = Namespaces.getNamespaceName(shellState.getInstance(),
             Tables.getNamespace(shellState.getInstance(), Tables.getTableId(shellState.getInstance(), tableName)));
-        for (Entry<String,String> e : shellState.getConnector().tableNamespaceOperations().getProperties(n)) {
+        for (Entry<String,String> e : shellState.getConnector().namespaceOperations().getProperties(n)) {
           namespaceConfig.put(e.getKey(), e.getValue());
         }
       }
-      
+
       Iterable<Entry<String,String>> acuconf = shellState.getConnector().instanceOperations().getSystemConfiguration().entrySet();
       if (tableName != null) {
         acuconf = shellState.getConnector().tableOperations().getProperties(tableName);
-      } else if (tableNamespace != null) {
-        acuconf = shellState.getConnector().tableNamespaceOperations().getProperties(tableNamespace);
+      } else if (namespace != null) {
+        acuconf = shellState.getConnector().namespaceOperations().getProperties(namespace);
       }
       final TreeMap<String,String> sortedConf = new TreeMap<String,String>();
       for (Entry<String,String> propEntry : acuconf) {
         sortedConf.put(propEntry.getKey(), propEntry.getValue());
       }
-      
+
       for (Entry<String,String> propEntry : acuconf) {
         final String key = propEntry.getKey();
         // only show properties with similar names to that
@@ -179,24 +180,24 @@ public class ConfigCommand extends Command {
         if (cl.hasOption(filterOpt.getOpt()) && !key.contains(cl.getOptionValue(filterOpt.getOpt()))) {
           continue;
         }
-        if ((tableName != null || tableNamespace != null) && !Property.isValidTablePropertyKey(key)) {
+        if ((tableName != null || namespace != null) && !Property.isValidTablePropertyKey(key)) {
           continue;
         }
         COL2 = Math.max(COL2, propEntry.getKey().length() + 3);
       }
-      
+
       final ArrayList<String> output = new ArrayList<String>();
       printConfHeader(output);
-      
+
       for (Entry<String,String> propEntry : sortedConf.entrySet()) {
         final String key = propEntry.getKey();
-        
+
         // only show properties with similar names to that
         // specified, or all of them if none specified
         if (cl.hasOption(filterOpt.getOpt()) && !key.contains(cl.getOptionValue(filterOpt.getOpt()))) {
           continue;
         }
-        if ((tableName != null || tableNamespace != null) && !Property.isValidTablePropertyKey(key)) {
+        if ((tableName != null || namespace != null) && !Property.isValidTablePropertyKey(key)) {
           continue;
         }
         String siteVal = siteConfig.get(key);
@@ -205,7 +206,7 @@ public class ConfigCommand extends Command {
         String dfault = defaults.get(key);
         String nspVal = namespaceConfig.get(key);
         boolean printed = false;
-        
+
         if (dfault != null && key.toLowerCase().contains("password")) {
           siteVal = sysVal = dfault = curVal = curVal.replaceAll(".", "*");
         }
@@ -222,7 +223,7 @@ public class ConfigCommand extends Command {
             printConfLine(output, "system", printed ? "   @override" : key, sysVal == null ? "" : sysVal);
             printed = true;
           }
-          
+
         }
         if (nspVal != null) {
           if (!systemConfig.containsKey(key) || !sysVal.equals(nspVal)) {
@@ -230,11 +231,11 @@ public class ConfigCommand extends Command {
             printed = true;
           }
         }
-        
+
         // show per-table value only if it is different (overridden)
         if (tableName != null && !curVal.equals(nspVal)) {
           printConfLine(output, "table", printed ? "   @override" : key, curVal);
-        } else if (tableNamespace != null && !curVal.equals(sysVal)) {
+        } else if (namespace != null && !curVal.equals(sysVal)) {
           printConfLine(output, "namespace", printed ? "   @override" : key, curVal);
         }
       }
@@ -246,13 +247,13 @@ public class ConfigCommand extends Command {
     }
     return 0;
   }
-  
+
   private void printConfHeader(List<String> output) {
     printConfFooter(output);
     output.add(String.format("%-" + COL1 + "s | %-" + COL2 + "s | %s", "SCOPE", "NAME", "VALUE"));
     printConfFooter(output);
   }
-  
+
   private void printConfLine(List<String> output, String s1, String s2, String s3) {
     if (s2.length() < COL2) {
       s2 += " " + Shell.repeat(".", COL2 - s2.length() - 1);
@@ -260,53 +261,53 @@ public class ConfigCommand extends Command {
     output.add(String.format("%-" + COL1 + "s | %-" + COL2 + "s | %s", s1, s2,
         s3.replace("\n", "\n" + Shell.repeat(" ", COL1 + 1) + "|" + Shell.repeat(" ", COL2 + 2) + "|" + " ")));
   }
-  
+
   private void printConfFooter(List<String> output) {
     int col3 = Math.max(1, Math.min(Integer.MAX_VALUE, reader.getTerminal().getWidth() - COL1 - COL2 - 6));
     output.add(String.format("%" + COL1 + "s-+-%" + COL2 + "s-+-%-" + col3 + "s", Shell.repeat("-", COL1), Shell.repeat("-", COL2), Shell.repeat("-", col3)));
   }
-  
+
   @Override
   public String description() {
     return "prints system properties and table specific properties";
   }
-  
+
   @Override
   public Options getOptions() {
     final Options o = new Options();
     final OptionGroup og = new OptionGroup();
     final OptionGroup tgroup = new OptionGroup();
-    
+
     tableOpt = new Option(Shell.tableOption, "table", true, "table to display/set/delete properties for");
     deleteOpt = new Option("d", "delete", true, "delete a per-table property");
     setOpt = new Option("s", "set", true, "set a per-table property");
     filterOpt = new Option("f", "filter", true, "show only properties that contain this string");
     disablePaginationOpt = new Option("np", "no-pagination", false, "disables pagination of output");
     outputFileOpt = new Option("o", "output", true, "local file to write the scan output to");
-    tableNamespaceOpt = new Option(Shell.tableNamespaceOption, "table-namespace", true, "table namespace to display/set/delete properties for");
-    
+    namespaceOpt = new Option(Shell.namespaceOption, "namespace", true, "namespace to display/set/delete properties for");
+
     tableOpt.setArgName("table");
     deleteOpt.setArgName("property");
     setOpt.setArgName("property=value");
     filterOpt.setArgName("string");
     outputFileOpt.setArgName("file");
-    tableNamespaceOpt.setArgName("tableNamespace");
-    
+    namespaceOpt.setArgName("namespace");
+
     og.addOption(deleteOpt);
     og.addOption(setOpt);
     og.addOption(filterOpt);
-    
+
     tgroup.addOption(tableOpt);
-    tgroup.addOption(tableNamespaceOpt);
-    
+    tgroup.addOption(namespaceOpt);
+
     o.addOptionGroup(tgroup);
     o.addOptionGroup(og);
     o.addOption(disablePaginationOpt);
     o.addOption(outputFileOpt);
-    
+
     return o;
   }
-  
+
   @Override
   public int numArgs() {
     return 0;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ConstraintCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ConstraintCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ConstraintCommand.java
index c91f29e..208ac4a 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ConstraintCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ConstraintCommand.java
@@ -29,37 +29,37 @@ import org.apache.commons.cli.OptionGroup;
 import org.apache.commons.cli.Options;
 
 public class ConstraintCommand extends Command {
-  protected Option tableNamespaceOpt;
-  
+  protected Option namespaceOpt;
+
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws Exception {
     final String tableName;
     final String namespace;
-    
-    if (cl.hasOption(tableNamespaceOpt.getOpt())) {
-      namespace = cl.getOptionValue(tableNamespaceOpt.getOpt());
+
+    if (cl.hasOption(namespaceOpt.getOpt())) {
+      namespace = cl.getOptionValue(namespaceOpt.getOpt());
     } else {
       namespace = null;
     }
-    
+
     if (cl.hasOption(OptUtil.tableOpt().getOpt()) || !shellState.getTableName().isEmpty()) {
       tableName = OptUtil.getTableOpt(cl, shellState);
     } else {
       tableName = null;
     }
-      
+
     int i;
     switch (OptUtil.getAldOpt(cl)) {
       case ADD:
         for (String constraint : cl.getArgs()) {
           if (namespace != null) {
-            if (!shellState.getConnector().tableNamespaceOperations().testClassLoad(namespace, constraint, Constraint.class.getName())) {
+            if (!shellState.getConnector().namespaceOperations().testClassLoad(namespace, constraint, Constraint.class.getName())) {
               throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Servers are unable to load " + constraint + " as type "
                   + Constraint.class.getName());
             }
-            i = shellState.getConnector().tableNamespaceOperations().addConstraint(namespace, constraint);
-            shellState.getReader().println("Added constraint " + constraint + " to table namespace " + namespace + " with number " + i);
-          } else if (tableName != null && !tableName.isEmpty()){
+            i = shellState.getConnector().namespaceOperations().addConstraint(namespace, constraint);
+            shellState.getReader().println("Added constraint " + constraint + " to namespace " + namespace + " with number " + i);
+          } else if (tableName != null && !tableName.isEmpty()) {
             if (!shellState.getConnector().tableOperations().testClassLoad(tableName, constraint, Constraint.class.getName())) {
               throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, "Servers are unable to load " + constraint + " as type "
                   + Constraint.class.getName());
@@ -67,7 +67,7 @@ public class ConstraintCommand extends Command {
             i = shellState.getConnector().tableOperations().addConstraint(tableName, constraint);
             shellState.getReader().println("Added constraint " + constraint + " to table " + tableName + " with number " + i);
           } else {
-            throw new IllegalArgumentException("Please specify either a table or a table namespace");
+            throw new IllegalArgumentException("Please specify either a table or a namespace");
           }
         }
         break;
@@ -75,59 +75,59 @@ public class ConstraintCommand extends Command {
         for (String constraint : cl.getArgs()) {
           i = Integer.parseInt(constraint);
           if (namespace != null) {
-            shellState.getConnector().tableNamespaceOperations().removeConstraint(namespace, i);
-            shellState.getReader().println("Removed constraint " + i + " from table namespace " + namespace);
-          } else if (tableName != null){
+            shellState.getConnector().namespaceOperations().removeConstraint(namespace, i);
+            shellState.getReader().println("Removed constraint " + i + " from namespace " + namespace);
+          } else if (tableName != null) {
             shellState.getConnector().tableOperations().removeConstraint(tableName, i);
             shellState.getReader().println("Removed constraint " + i + " from table " + tableName);
           } else {
-            throw new IllegalArgumentException("Please specify either a table or a table namespace");
+            throw new IllegalArgumentException("Please specify either a table or a namespace");
           }
         }
         break;
       case LIST:
         if (namespace != null) {
-          for (Entry<String,Integer> property : shellState.getConnector().tableNamespaceOperations().listConstraints(namespace).entrySet()) {
+          for (Entry<String,Integer> property : shellState.getConnector().namespaceOperations().listConstraints(namespace).entrySet()) {
             shellState.getReader().println(property.toString());
           }
-        } else if (tableName != null){
+        } else if (tableName != null) {
           for (Entry<String,Integer> property : shellState.getConnector().tableOperations().listConstraints(tableName).entrySet()) {
             shellState.getReader().println(property.toString());
           }
         } else {
-          throw new IllegalArgumentException("Please specify either a table or a table namespace");
+          throw new IllegalArgumentException("Please specify either a table or a namespace");
         }
     }
-    
+
     return 0;
   }
-  
+
   @Override
   public String description() {
     return "adds, deletes, or lists constraints for a table";
   }
-  
+
   @Override
   public int numArgs() {
     return Shell.NO_FIXED_ARG_LENGTH_CHECK;
   }
-  
+
   @Override
   public String usage() {
     return getName() + " <constraint>{ <constraint>}";
   }
-  
+
   @Override
   public Options getOptions() {
     final Options o = new Options();
     o.addOptionGroup(OptUtil.addListDeleteGroup("constraint"));
-    
+
     OptionGroup grp = new OptionGroup();
     grp.addOption(OptUtil.tableOpt("table to add, delete, or list constraints for"));
-    tableNamespaceOpt = new Option(Shell.tableNamespaceOption, "table-namespace", true, "name of a table namespace to operate on");
-    tableNamespaceOpt.setArgName("tableNamespace");
-    grp.addOption(tableNamespaceOpt);
-    
+    namespaceOpt = new Option(Shell.namespaceOption, "namespace", true, "name of a namespace to operate on");
+    namespaceOpt.setArgName("namespace");
+    grp.addOption(namespaceOpt);
+
     o.addOptionGroup(grp);
     return o;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateNamespaceCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateNamespaceCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateNamespaceCommand.java
index 7434767..908a9cc 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateNamespaceCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateNamespaceCommand.java
@@ -21,9 +21,9 @@ import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.NamespaceExistsException;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.TableNamespaceExistsException;
-import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.util.shell.Shell;
@@ -34,12 +34,12 @@ import org.apache.commons.cli.OptionGroup;
 import org.apache.commons.cli.Options;
 
 public class CreateNamespaceCommand extends Command {
-  private Option createTableOptCopyConfig, createTableNamespaceOptCopyConfig;
+  private Option createTableOptCopyConfig, createNamespaceOptCopyConfig;
   private Option base64Opt;
 
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException,
-      TableExistsException, TableNotFoundException, IOException, ClassNotFoundException, TableNamespaceExistsException, TableNamespaceNotFoundException {
+      TableExistsException, TableNotFoundException, IOException, ClassNotFoundException, NamespaceExistsException, NamespaceNotFoundException {
 
     if (createTableOptCopyConfig == null) {
       getOptions();
@@ -47,25 +47,25 @@ public class CreateNamespaceCommand extends Command {
 
     String namespace = cl.getArgs()[0];
 
-    shellState.getConnector().tableNamespaceOperations().create(namespace);
+    shellState.getConnector().namespaceOperations().create(namespace);
 
     // Copy options if flag was set
     Iterable<Entry<String,String>> configuration = null;
-    if (cl.hasOption(createTableNamespaceOptCopyConfig.getOpt())) {
-      String copy = cl.getOptionValue(createTableNamespaceOptCopyConfig.getOpt());
-      if (shellState.getConnector().tableNamespaceOperations().exists(namespace)) {
-        configuration = shellState.getConnector().tableNamespaceOperations().getProperties(copy);
+    if (cl.hasOption(createNamespaceOptCopyConfig.getOpt())) {
+      String copy = cl.getOptionValue(createNamespaceOptCopyConfig.getOpt());
+      if (shellState.getConnector().namespaceOperations().exists(namespace)) {
+        configuration = shellState.getConnector().namespaceOperations().getProperties(copy);
       }
     } else if (cl.hasOption(createTableOptCopyConfig.getOpt())) {
       String copy = cl.getOptionValue(createTableOptCopyConfig.getOpt());
-      if (shellState.getConnector().tableNamespaceOperations().exists(namespace)) {
+      if (shellState.getConnector().namespaceOperations().exists(namespace)) {
         configuration = shellState.getConnector().tableOperations().getProperties(copy);
       }
     }
     if (configuration != null) {
       for (Entry<String,String> entry : configuration) {
         if (Property.isValidTablePropertyKey(entry.getKey())) {
-          shellState.getConnector().tableNamespaceOperations().setProperty(namespace, entry.getKey(), entry.getValue());
+          shellState.getConnector().namespaceOperations().setProperty(namespace, entry.getKey(), entry.getValue());
         }
       }
     }
@@ -75,7 +75,7 @@ public class CreateNamespaceCommand extends Command {
 
   @Override
   public String description() {
-    return "creates a new table namespace";
+    return "creates a new namespace";
   }
 
   @Override
@@ -87,8 +87,8 @@ public class CreateNamespaceCommand extends Command {
   public Options getOptions() {
     final Options o = new Options();
 
-    createTableNamespaceOptCopyConfig = new Option("cc", "copy-config", true, "table namespace to copy configuration from");
-    createTableNamespaceOptCopyConfig.setArgName("tableNamespace");
+    createNamespaceOptCopyConfig = new Option("cc", "copy-config", true, "namespace to copy configuration from");
+    createNamespaceOptCopyConfig.setArgName("namespace");
 
     createTableOptCopyConfig = new Option("ctc", "copy-table-config", true, "table to copy configuration from");
     createTableOptCopyConfig.setArgName("tableName");
@@ -97,7 +97,7 @@ public class CreateNamespaceCommand extends Command {
     o.addOption(base64Opt);
     OptionGroup ogp = new OptionGroup();
     ogp.addOption(createTableOptCopyConfig);
-    ogp.addOption(createTableNamespaceOptCopyConfig);
+    ogp.addOption(createNamespaceOptCopyConfig);
 
     o.addOptionGroup(ogp);
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java
index 8081d92..b017eaf 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java
@@ -114,7 +114,7 @@ public class CreateTableCommand extends Command {
 
     String n = Tables.extractNamespace(tableName);
     String table = tableName;
-    if (n.equals(Constants.DEFAULT_TABLE_NAMESPACE) || n.equals(Constants.SYSTEM_TABLE_NAMESPACE)) {
+    if (n.equals(Constants.DEFAULT_NAMESPACE) || n.equals(Constants.SYSTEM_NAMESPACE)) {
       table = Tables.extractTableName(tableName);
     }
     shellState.setTableName(table); // switch shell to new table context
@@ -216,6 +216,6 @@ public class CreateTableCommand extends Command {
 
   @Override
   public void registerCompletion(final Token root, final Map<Command.CompletionSet,Set<String>> special) {
-    registerCompletionForTableNamespaces(root, special);
+    registerCompletionForNamespaces(root, special);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DUCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DUCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DUCommand.java
index 817809d..3d57961 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DUCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DUCommand.java
@@ -22,10 +22,10 @@ import java.util.SortedSet;
 import java.util.TreeSet;
 
 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.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.DiskUsage;
-import org.apache.accumulo.core.client.impl.TableNamespaces;
+import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.util.NumUtil;
 import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
@@ -34,13 +34,15 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.cli.Options;
 
 public class DUCommand extends Command {
-  
-  private Option optTablePattern, optHumanReadble, optTableNamespace;
-  
-  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws IOException, TableNotFoundException, TableNamespaceNotFoundException {
-    
+
+  private Option optTablePattern, optHumanReadble, optNamespace;
+
+  @Override
+  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws IOException, TableNotFoundException,
+      NamespaceNotFoundException {
+
     final SortedSet<String> tables = new TreeSet<String>(Arrays.asList(cl.getArgs()));
-    
+
     if (cl.hasOption(Shell.tableOption)) {
       String tableName = cl.getOptionValue(Shell.tableOption);
       if (!shellState.getConnector().tableOperations().exists(tableName)) {
@@ -48,15 +50,15 @@ public class DUCommand extends Command {
       }
       tables.add(tableName);
     }
-    
-    if (cl.hasOption(optTableNamespace.getOpt())) {
+
+    if (cl.hasOption(optNamespace.getOpt())) {
       Instance instance = shellState.getInstance();
-      String namespaceId = TableNamespaces.getNamespaceId(instance, cl.getOptionValue(optTableNamespace.getOpt()));
-      tables.addAll(TableNamespaces.getTableNames(instance, namespaceId));
+      String namespaceId = Namespaces.getNamespaceId(instance, cl.getOptionValue(optNamespace.getOpt()));
+      tables.addAll(Namespaces.getTableNames(instance, namespaceId));
     }
-    
+
     boolean prettyPrint = cl.hasOption(optHumanReadble.getOpt()) ? true : false;
-    
+
     if (cl.hasOption(optTablePattern.getOpt())) {
       for (String table : shellState.getConnector().tableOperations().list()) {
         if (table.matches(cl.getOptionValue(optTablePattern.getOpt()))) {
@@ -80,39 +82,39 @@ public class DUCommand extends Command {
     }
     return 0;
   }
-  
+
   @Override
   public String description() {
     return "prints how much space, in bytes, is used by files referenced by a table.  When multiple tables are specified it prints how much space, in bytes, is used by files shared between tables, if any.";
   }
-  
+
   @Override
   public Options getOptions() {
     final Options o = new Options();
-    
+
     optTablePattern = new Option("p", "pattern", true, "regex pattern of table names");
     optTablePattern.setArgName("pattern");
-    
+
     optHumanReadble = new Option("h", "human-readable", false, "format large sizes to human readable units");
     optHumanReadble.setArgName("human readable output");
-    
-    optTableNamespace = new Option(Shell.tableNamespaceOption, "table-namespace", true, "name of a table namespace");
-    optTableNamespace.setArgName("table-namespace");
-    
+
+    optNamespace = new Option(Shell.namespaceOption, "namespace", true, "name of a namespace");
+    optNamespace.setArgName("namespace");
+
     o.addOption(OptUtil.tableOpt("table to examine"));
-    
+
     o.addOption(optTablePattern);
     o.addOption(optHumanReadble);
-    o.addOption(optTableNamespace);
-    
+    o.addOption(optNamespace);
+
     return o;
   }
-  
+
   @Override
   public String usage() {
     return getName() + " <table>{ <table>}";
   }
-  
+
   @Override
   public int numArgs() {
     return Shell.NO_FIXED_ARG_LENGTH_CHECK;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteIterCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteIterCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteIterCommand.java
index 4f2f46f..c100325 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteIterCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteIterCommand.java
@@ -28,16 +28,17 @@ import org.apache.commons.cli.Options;
 
 public class DeleteIterCommand extends Command {
   private Option allScopeOpt, mincScopeOpt, majcScopeOpt, scanScopeOpt, nameOpt;
-  
+
+  @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws Exception {
-    
+
     boolean tables = cl.hasOption(OptUtil.tableOpt().getOpt()) || !shellState.getTableName().isEmpty();
-    boolean namespaces = cl.hasOption(OptUtil.tableNamespaceOpt().getOpt());
-    
+    boolean namespaces = cl.hasOption(OptUtil.namespaceOpt().getOpt());
+
     final String name = cl.getOptionValue(nameOpt.getOpt());
-    
+
     if (namespaces) {
-      if (!shellState.getConnector().tableNamespaceOperations().listIterators(OptUtil.getTableNamespaceOpt(cl, shellState)).containsKey(name)) {
+      if (!shellState.getConnector().namespaceOperations().listIterators(OptUtil.getNamespaceOpt(cl, shellState)).containsKey(name)) {
         Shell.log.warn("no iterators found that match your criteria");
         return 0;
       }
@@ -47,9 +48,9 @@ public class DeleteIterCommand extends Command {
         return 0;
       }
     } else {
-      throw new IllegalArgumentException("No table or table namespace specified");
+      throw new IllegalArgumentException("No table or namespace specified");
     }
-    
+
     final EnumSet<IteratorScope> scopes = EnumSet.noneOf(IteratorScope.class);
     if (cl.hasOption(allScopeOpt.getOpt()) || cl.hasOption(mincScopeOpt.getOpt())) {
       scopes.add(IteratorScope.minc);
@@ -63,48 +64,49 @@ public class DeleteIterCommand extends Command {
     if (scopes.isEmpty()) {
       throw new IllegalArgumentException("You must select at least one scope to configure");
     }
-    
+
     if (namespaces) {
-      shellState.getConnector().tableNamespaceOperations().removeIterator(OptUtil.getTableNamespaceOpt(cl, shellState), name, scopes);
+      shellState.getConnector().namespaceOperations().removeIterator(OptUtil.getNamespaceOpt(cl, shellState), name, scopes);
     } else if (tables) {
       shellState.getConnector().tableOperations().removeIterator(OptUtil.getTableOpt(cl, shellState), name, scopes);
     } else {
-      throw new IllegalArgumentException("No table or table namespace specified");
+      throw new IllegalArgumentException("No table or namespace specified");
     }
     return 0;
   }
-  
+
   @Override
   public String description() {
-    return "deletes a table-specific or table-namespace-specific iterator";
+    return "deletes a table-specific or namespace-specific iterator";
   }
-  
+
+  @Override
   public Options getOptions() {
     final Options o = new Options();
-    
+
     nameOpt = new Option("n", "name", true, "iterator to delete");
     nameOpt.setArgName("itername");
     nameOpt.setRequired(true);
-    
+
     allScopeOpt = new Option("all", "all-scopes", false, "remove from all scopes");
     mincScopeOpt = new Option(IteratorScope.minc.name(), "minor-compaction", false, "remove from minor compaction scope");
     majcScopeOpt = new Option(IteratorScope.majc.name(), "major-compaction", false, "remove from major compaction scope");
     scanScopeOpt = new Option(IteratorScope.scan.name(), "scan-time", false, "remove from scan scope");
-    
+
     OptionGroup grp = new OptionGroup();
     grp.addOption(OptUtil.tableOpt("table to delete the iterator from"));
-    grp.addOption(OptUtil.tableNamespaceOpt("table namespace to delete the iterator from"));
+    grp.addOption(OptUtil.namespaceOpt("namespace to delete the iterator from"));
     o.addOptionGroup(grp);
     o.addOption(nameOpt);
-    
+
     o.addOption(allScopeOpt);
     o.addOption(mincScopeOpt);
     o.addOption(majcScopeOpt);
     o.addOption(scanScopeOpt);
-    
+
     return o;
   }
-  
+
   @Override
   public int numArgs() {
     return 0;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteNamespaceCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteNamespaceCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteNamespaceCommand.java
index e1b41cd..26473ed 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteNamespaceCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteNamespaceCommand.java
@@ -21,8 +21,8 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.accumulo.core.Constants;
-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.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
 import org.apache.accumulo.core.util.shell.Token;
@@ -55,29 +55,29 @@ public class DeleteNamespaceCommand extends Command {
 
   @Override
   public String description() {
-    return "deletes a table namespace";
+    return "deletes a namespace";
   }
 
   protected void doTableOp(final Shell shellState, final String namespace, boolean force) throws Exception {
     boolean resetContext = false;
     String currentTable = shellState.getTableName();
-    if (!TableNamespaces.getNameToIdMap(shellState.getInstance()).containsKey(namespace)) {
-      throw new TableNamespaceNotFoundException(null, namespace, null);
+    if (!Namespaces.getNameToIdMap(shellState.getInstance()).containsKey(namespace)) {
+      throw new NamespaceNotFoundException(null, namespace, null);
     }
 
-    String namespaceId = TableNamespaces.getNamespaceId(shellState.getInstance(), namespace);
-    List<String> tables = TableNamespaces.getTableNames(shellState.getInstance(), namespaceId);
+    String namespaceId = Namespaces.getNamespaceId(shellState.getInstance(), namespace);
+    List<String> tables = Namespaces.getTableNames(shellState.getInstance(), namespaceId);
     resetContext = tables.contains(currentTable);
 
     if (force) {
-      shellState.getConnector().tableNamespaceOperations().delete(namespace, true);
+      shellState.getConnector().namespaceOperations().delete(namespace, true);
     } else {
-      shellState.getConnector().tableNamespaceOperations().delete(namespace);
+      shellState.getConnector().namespaceOperations().delete(namespace);
     }
-    if (namespace.equals(Constants.SYSTEM_TABLE_NAMESPACE)) {
-      shellState.getReader().println("Table namespace: [" + namespace + "], can't delete system or default namespace.");
+    if (namespace.equals(Constants.SYSTEM_NAMESPACE)) {
+      shellState.getReader().println("Namespace: [" + namespace + "], can't delete system or default namespace.");
     } else {
-      shellState.getReader().println("Table namespace: [" + namespace + "] has been deleted.");
+      shellState.getReader().println("Namespace: [" + namespace + "] has been deleted.");
     }
     if (resetContext) {
       shellState.setTableName("");
@@ -100,6 +100,6 @@ public class DeleteNamespaceCommand extends Command {
 
   @Override
   public void registerCompletion(final Token root, final Map<Command.CompletionSet,Set<String>> special) {
-    registerCompletionForTableNamespaces(root, special);
+    registerCompletionForNamespaces(root, special);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteTableCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteTableCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteTableCommand.java
index 74d3f36..10a12d3 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteTableCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteTableCommand.java
@@ -47,7 +47,7 @@ public class DeleteTableCommand extends TableOperation {
     
     String n = Tables.extractNamespace(tableName);
     String checkTable = tableName;
-    if (n.equals(Constants.DEFAULT_TABLE_NAMESPACE) || n.equals(Constants.SYSTEM_TABLE_NAMESPACE)) {
+    if (n.equals(Constants.DEFAULT_NAMESPACE) || n.equals(Constants.SYSTEM_NAMESPACE)) {
       checkTable = Tables.extractTableName(tableName);
     }
     if (shellState.getTableName().equals(checkTable)) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GrantCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GrantCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GrantCommand.java
index 9f6e7e4..de719fd 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GrantCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GrantCommand.java
@@ -19,9 +19,9 @@ package org.apache.accumulo.core.util.shell.commands;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.security.TableNamespacePermission;
 import org.apache.accumulo.core.util.BadArgumentException;
 import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
@@ -35,15 +35,15 @@ public class GrantCommand extends TableOperation {
   {
     disableUnflaggedTableOptions();
   }
-  
+
   private Option systemOpt, userOpt;
   private String user;
   private String[] permission;
-  
+
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws Exception {
     user = cl.hasOption(userOpt.getOpt()) ? cl.getOptionValue(userOpt.getOpt()) : shellState.getConnector().whoami();
-    
+
     permission = cl.getArgs()[0].split("\\.", 2);
     if (cl.hasOption(systemOpt.getOpt()) && permission[0].equalsIgnoreCase("System")) {
       try {
@@ -55,22 +55,22 @@ public class GrantCommand extends TableOperation {
     } else if (permission[0].equalsIgnoreCase("Table")) {
       super.execute(fullCommand, cl, shellState);
     } else if (permission[0].equalsIgnoreCase("Namespace")) {
-      if (cl.hasOption(optTableNamespace.getOpt())) {
+      if (cl.hasOption(optNamespace.getOpt())) {
         try {
           shellState.getConnector().securityOperations()
-              .grantTableNamespacePermission(user, cl.getOptionValue(optTableNamespace.getOpt()), TableNamespacePermission.valueOf(permission[1]));
+              .grantNamespacePermission(user, cl.getOptionValue(optNamespace.getOpt()), NamespacePermission.valueOf(permission[1]));
         } catch (IllegalArgumentException e) {
-          throw new BadArgumentException("No such table namespace permission", fullCommand, fullCommand.indexOf(cl.getArgs()[0]));
+          throw new BadArgumentException("No such namespace permission", fullCommand, fullCommand.indexOf(cl.getArgs()[0]));
         }
       } else {
-        throw new BadArgumentException("No Table Namespace specified to apply permission to", fullCommand, fullCommand.indexOf(cl.getArgs()[0]));
+        throw new BadArgumentException("No namespace specified to apply permission to", fullCommand, fullCommand.indexOf(cl.getArgs()[0]));
       }
     } else {
       throw new BadArgumentException("Unrecognized permission", fullCommand, fullCommand.indexOf(cl.getArgs()[0]));
     }
     return 0;
   }
-  
+
   @Override
   protected void doTableOp(final Shell shellState, final String tableName) throws Exception {
     try {
@@ -80,52 +80,52 @@ public class GrantCommand extends TableOperation {
       throw new IllegalArgumentException("No such table permission", e);
     }
   }
-  
+
   @Override
   public String description() {
-    return "grants system, table, or table namespace permissions for a user";
+    return "grants system, table, or namespace permissions for a user";
   }
-  
+
   @Override
   public String usage() {
     return getName() + " <permission>";
   }
-  
+
   @Override
   public void registerCompletion(final Token root, final Map<Command.CompletionSet,Set<String>> completionSet) {
     final Token cmd = new Token(getName());
     cmd.addSubcommand(new Token(TablePermission.printableValues()));
     cmd.addSubcommand(new Token(SystemPermission.printableValues()));
-    cmd.addSubcommand(new Token(TableNamespacePermission.printableValues()));
+    cmd.addSubcommand(new Token(NamespacePermission.printableValues()));
     root.addSubcommand(cmd);
   }
-  
+
   @Override
   public Options getOptions() {
     super.getOptions();
     final Options o = new Options();
-    
+
     final OptionGroup group = new OptionGroup();
-    
+
     systemOpt = new Option("s", "system", false, "grant a system permission");
-    
-    optTableNamespace = new Option(Shell.tableNamespaceOption, "table-namespace", true, "name of a table namespace to operate on");
-    optTableNamespace.setArgName("tableNamespace");
-    
+
+    optNamespace = new Option(Shell.namespaceOption, "namespace", true, "name of a namespace to operate on");
+    optNamespace.setArgName("namespace");
+
     group.addOption(systemOpt);
     group.addOption(optTableName);
     group.addOption(optTablePattern);
-    group.addOption(optTableNamespace);
-    
+    group.addOption(optNamespace);
+
     o.addOptionGroup(group);
     userOpt = new Option(Shell.userOption, "user", true, "user to operate on");
     userOpt.setArgName("username");
     userOpt.setRequired(true);
     o.addOption(userOpt);
-    
+
     return o;
   }
-  
+
   @Override
   public int numArgs() {
     return 1;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ListIterCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ListIterCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ListIterCommand.java
index 5e5018f..e29c5e6 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ListIterCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ListIterCommand.java
@@ -33,22 +33,22 @@ import org.apache.commons.cli.Options;
 public class ListIterCommand extends Command {
   private Option nameOpt;
   private Map<IteratorScope,Option> scopeOpts;
-  
+
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws Exception {
-    
+
     boolean tables = cl.hasOption(OptUtil.tableOpt().getOpt()) || !shellState.getTableName().isEmpty();
-    boolean namespaces = cl.hasOption(OptUtil.tableNamespaceOpt().getOpt());
-    
+    boolean namespaces = cl.hasOption(OptUtil.namespaceOpt().getOpt());
+
     final Map<String,EnumSet<IteratorScope>> iterators;
     if (namespaces) {
-      iterators = shellState.getConnector().tableNamespaceOperations().listIterators(OptUtil.getTableNamespaceOpt(cl, shellState));
+      iterators = shellState.getConnector().namespaceOperations().listIterators(OptUtil.getNamespaceOpt(cl, shellState));
     } else if (tables) {
       iterators = shellState.getConnector().tableOperations().listIterators(OptUtil.getTableOpt(cl, shellState));
     } else {
-      throw new IllegalArgumentException("No table or table namespace specified");
+      throw new IllegalArgumentException("No table or namespace specified");
     }
-    
+
     if (cl.hasOption(nameOpt.getOpt())) {
       final String name = cl.getOptionValue(nameOpt.getOpt());
       if (!iterators.containsKey(name)) {
@@ -59,7 +59,7 @@ public class ListIterCommand extends Command {
       iterators.clear();
       iterators.put(name, scopes);
     }
-    
+
     boolean hasScope = false;
     for (IteratorScope scope : IteratorScope.values()) {
       if (cl.hasOption(scopeOpts.get(scope).getOpt()))
@@ -74,11 +74,11 @@ public class ListIterCommand extends Command {
         if (cl.hasOption(scopeOpts.get(scope).getOpt())) {
           IteratorSetting setting;
           if (namespaces) {
-            setting = shellState.getConnector().tableNamespaceOperations().getIteratorSetting(OptUtil.getTableNamespaceOpt(cl, shellState), name, scope);
+            setting = shellState.getConnector().namespaceOperations().getIteratorSetting(OptUtil.getNamespaceOpt(cl, shellState), name, scope);
           } else if (tables) {
             setting = shellState.getConnector().tableOperations().getIteratorSetting(OptUtil.getTableOpt(cl, shellState), name, scope);
           } else {
-            throw new IllegalArgumentException("No table or table namespace specified");
+            throw new IllegalArgumentException("No table or namespace specified");
           }
           sb.append("-    Iterator ").append(setting.getName()).append(", ").append(scope).append(" scope options:\n");
           sb.append("-        ").append("iteratorPriority").append(" = ").append(setting.getPriority()).append("\n");
@@ -91,41 +91,42 @@ public class ListIterCommand extends Command {
     }
     sb.append("-");
     shellState.getReader().println(sb.toString());
-    
+
     return 0;
   }
-  
+
+  @Override
   public String description() {
-    return "lists table-specific or table-namespace-specific iterators configured in this shell session";
+    return "lists table-specific or namespace-specific iterators configured in this shell session";
   }
-  
+
   @Override
   public int numArgs() {
     return 0;
   }
-  
+
   @Override
   public Options getOptions() {
     final Options o = new Options();
-    
+
     nameOpt = new Option("n", "name", true, "iterator to list");
     nameOpt.setArgName("itername");
-    
+
     scopeOpts = new EnumMap<IteratorScope,Option>(IteratorScope.class);
     scopeOpts.put(IteratorScope.minc, new Option(IteratorScope.minc.name(), "minor-compaction", false, "list iterator for minor compaction scope"));
     scopeOpts.put(IteratorScope.majc, new Option(IteratorScope.majc.name(), "major-compaction", false, "list iterator for major compaction scope"));
     scopeOpts.put(IteratorScope.scan, new Option(IteratorScope.scan.name(), "scan-time", false, "list iterator for scan scope"));
-    
+
     OptionGroup grp = new OptionGroup();
     grp.addOption(OptUtil.tableOpt("table to list the configured iterators on"));
-    grp.addOption(OptUtil.tableNamespaceOpt("table namespace to list the configured iterators on"));
+    grp.addOption(OptUtil.namespaceOpt("namespace to list the configured iterators on"));
     o.addOptionGroup(grp);
     o.addOption(nameOpt);
-    
+
     for (Option opt : scopeOpts.values()) {
       o.addOption(opt);
     }
-    
+
     return o;
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacePermissionsCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacePermissionsCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacePermissionsCommand.java
index f3e0e6f..2dcba55 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacePermissionsCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacePermissionsCommand.java
@@ -18,7 +18,7 @@ package org.apache.accumulo.core.util.shell.commands;
 
 import java.io.IOException;
 
-import org.apache.accumulo.core.security.TableNamespacePermission;
+import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
 import org.apache.commons.cli.CommandLine;
@@ -26,7 +26,7 @@ import org.apache.commons.cli.CommandLine;
 public class NamespacePermissionsCommand extends Command {
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws IOException {
-    for (String p : TableNamespacePermission.printableValues()) {
+    for (String p : NamespacePermission.printableValues()) {
       shellState.getReader().println(p);
     }
     return 0;
@@ -34,7 +34,7 @@ public class NamespacePermissionsCommand extends Command {
 
   @Override
   public String description() {
-    return "displays a list of valid table namespace permissions";
+    return "displays a list of valid namespace permissions";
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java
index 2056afd..38f7c77 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/NamespacesCommand.java
@@ -36,8 +36,8 @@ public class NamespacesCommand extends Command {
   @SuppressWarnings("unchecked")
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException, IOException {
-    Iterator<String> names = shellState.getConnector().tableNamespaceOperations().list().iterator();
-    Iterator<String> ids = new NamespaceIdIterator(new TreeMap<String,String>(shellState.getConnector().tableNamespaceOperations().namespaceIdMap()).entrySet()
+    Iterator<String> names = shellState.getConnector().namespaceOperations().list().iterator();
+    Iterator<String> ids = new NamespaceIdIterator(new TreeMap<String,String>(shellState.getConnector().namespaceOperations().namespaceIdMap()).entrySet()
         .iterator());
 
     if (cl.hasOption(namespaceIdOption.getOpt())) {
@@ -66,13 +66,13 @@ public class NamespacesCommand extends Command {
 
   @Override
   public String description() {
-    return "displays a list of all existing table namespaces";
+    return "displays a list of all existing namespaces";
   }
 
   @Override
   public Options getOptions() {
     final Options o = new Options();
-    namespaceIdOption = new Option("l", "list-ids", false, "display internal table namespace ids along with the name");
+    namespaceIdOption = new Option("l", "list-ids", false, "display internal namespace ids along with the name");
     o.addOption(namespaceIdOption);
     disablePaginationOpt = new Option("np", "no-pagination", false, "disable pagination of output");
     o.addOption(disablePaginationOpt);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OptUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OptUtil.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OptUtil.java
index e92be4a..24b6b9d 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OptUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OptUtil.java
@@ -18,7 +18,7 @@ package org.apache.accumulo.core.util.shell.commands;
 
 import java.io.UnsupportedEncodingException;
 
-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.util.shell.Shell;
 import org.apache.commons.cli.CommandLine;
@@ -29,10 +29,10 @@ import org.apache.hadoop.io.Text;
 public abstract class OptUtil {
   public static final String START_ROW_OPT = "b";
   public static final String END_ROW_OPT = "e";
-  
+
   public static String getTableOpt(final CommandLine cl, final Shell shellState) throws TableNotFoundException {
     String tableName;
-    
+
     if (cl.hasOption(Shell.tableOption)) {
       tableName = cl.getOptionValue(Shell.tableOption);
       if (!shellState.getConnector().tableOperations().exists(tableName)) {
@@ -42,55 +42,55 @@ public abstract class OptUtil {
       shellState.checkTableState();
       tableName = shellState.getTableName();
     }
-    
+
     return tableName;
   }
-  
-  public static String getTableNamespaceOpt(final CommandLine cl, final Shell shellState) throws TableNamespaceNotFoundException {
+
+  public static String getNamespaceOpt(final CommandLine cl, final Shell shellState) throws NamespaceNotFoundException {
     String namespace = null;
-    if (cl.hasOption(Shell.tableNamespaceOption)) {
-      namespace = cl.getOptionValue(Shell.tableNamespaceOption);
-      if (!shellState.getConnector().tableNamespaceOperations().exists(namespace)) {
-        throw new TableNamespaceNotFoundException(namespace, namespace, "specified table namespace that doesn't exist");
+    if (cl.hasOption(Shell.namespaceOption)) {
+      namespace = cl.getOptionValue(Shell.namespaceOption);
+      if (!shellState.getConnector().namespaceOperations().exists(namespace)) {
+        throw new NamespaceNotFoundException(namespace, namespace, "specified namespace that doesn't exist");
       }
     } else {
-      throw new TableNamespaceNotFoundException(null, null, "no table namespace specified");
+      throw new NamespaceNotFoundException(null, null, "no namespace specified");
     }
     return namespace;
   }
-  
+
   public static Option tableOpt() {
     return tableOpt("tableName");
   }
-  
+
   public static Option tableOpt(final String description) {
     final Option tableOpt = new Option(Shell.tableOption, "table", true, description);
     tableOpt.setArgName("table");
     tableOpt.setRequired(false);
     return tableOpt;
   }
-  
-  public static Option tableNamespaceOpt() {
-    return tableNamespaceOpt("tableNamespace");
+
+  public static Option namespaceOpt() {
+    return namespaceOpt("namespace");
   }
-  
-  public static Option tableNamespaceOpt(final String description) {
-    final Option tableNamespaceOpt = new Option(Shell.tableNamespaceOption, "tableNamespace", true, description);
-    tableNamespaceOpt.setArgName("tableNamespace");
-    tableNamespaceOpt.setRequired(false);
-    return tableNamespaceOpt;
+
+  public static Option namespaceOpt(final String description) {
+    final Option namespaceOpt = new Option(Shell.namespaceOption, "namespace", true, description);
+    namespaceOpt.setArgName("namespace");
+    namespaceOpt.setRequired(false);
+    return namespaceOpt;
   }
-  
+
   public static enum AdlOpt {
     ADD("a"), DELETE("d"), LIST("l");
-    
+
     public final String opt;
-    
+
     private AdlOpt(String opt) {
       this.opt = opt;
     }
   }
-  
+
   public static AdlOpt getAldOpt(final CommandLine cl) {
     if (cl.hasOption(AdlOpt.ADD.opt)) {
       return AdlOpt.ADD;
@@ -100,7 +100,7 @@ public abstract class OptUtil {
       return AdlOpt.LIST;
     }
   }
-  
+
   public static OptionGroup addListDeleteGroup(final String name) {
     final Option addOpt = new Option(AdlOpt.ADD.opt, "add", false, "add " + name);
     final Option deleteOpt = new Option(AdlOpt.DELETE.opt, "delete", false, "delete " + name);
@@ -112,19 +112,19 @@ public abstract class OptUtil {
     og.setRequired(true);
     return og;
   }
-  
+
   public static Option startRowOpt() {
     final Option o = new Option(START_ROW_OPT, "begin-row", true, "begin row (inclusive)");
     o.setArgName("begin-row");
     return o;
   }
-  
+
   public static Option endRowOpt() {
     final Option o = new Option(END_ROW_OPT, "end-row", true, "end row (inclusive)");
     o.setArgName("end-row");
     return o;
   }
-  
+
   public static Text getStartRow(final CommandLine cl) throws UnsupportedEncodingException {
     if (cl.hasOption(START_ROW_OPT)) {
       return new Text(cl.getOptionValue(START_ROW_OPT).getBytes(Shell.CHARSET));
@@ -132,7 +132,7 @@ public abstract class OptUtil {
       return null;
     }
   }
-  
+
   public static Text getEndRow(final CommandLine cl) throws UnsupportedEncodingException {
     if (cl.hasOption(END_ROW_OPT)) {
       return new Text(cl.getOptionValue(END_ROW_OPT).getBytes(Shell.CHARSET));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameNamespaceCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameNamespaceCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameNamespaceCommand.java
index 34fd9fa..d7ff7c3 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameNamespaceCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameNamespaceCommand.java
@@ -22,11 +22,11 @@ import java.util.Set;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.NamespaceExistsException;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.TableNamespaceExistsException;
-import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
-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.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
@@ -36,19 +36,19 @@ import org.apache.commons.cli.CommandLine;
 public class RenameNamespaceCommand extends Command {
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException,
-      TableNotFoundException, TableExistsException, TableNamespaceNotFoundException, TableNamespaceExistsException {
+      TableNotFoundException, TableExistsException, NamespaceNotFoundException, NamespaceExistsException {
     String old = cl.getArgs()[0];
     String newer = cl.getArgs()[1];
     boolean resetContext = false;
     String currentTableId = "";
     if (!(shellState.getTableName() == null) && !shellState.getTableName().isEmpty()) {
-      String namespaceId = TableNamespaces.getNamespaceId(shellState.getInstance(), old);
-      List<String> tableIds = TableNamespaces.getTableIds(shellState.getInstance(), namespaceId);
+      String namespaceId = Namespaces.getNamespaceId(shellState.getInstance(), old);
+      List<String> tableIds = Namespaces.getTableIds(shellState.getInstance(), namespaceId);
       currentTableId = Tables.getTableId(shellState.getInstance(), shellState.getTableName());
       resetContext = tableIds.contains(currentTableId);
     }
 
-    shellState.getConnector().tableNamespaceOperations().rename(old, newer);
+    shellState.getConnector().namespaceOperations().rename(old, newer);
 
     if (resetContext) {
       shellState.setTableName(Tables.getTableName(shellState.getInstance(), currentTableId));
@@ -59,17 +59,17 @@ public class RenameNamespaceCommand extends Command {
 
   @Override
   public String usage() {
-    return getName() + " <current table namespace> <new table namespace>";
+    return getName() + " <current namespace> <new namespace>";
   }
 
   @Override
   public String description() {
-    return "renames a table namespace";
+    return "renames a namespace";
   }
 
   @Override
   public void registerCompletion(final Token root, final Map<Command.CompletionSet,Set<String>> special) {
-    registerCompletionForTableNamespaces(root, special);
+    registerCompletionForNamespaces(root, special);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameTableCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameTableCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameTableCommand.java
index 4ba039d..fe86f35 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameTableCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameTableCommand.java
@@ -38,7 +38,7 @@ public class RenameTableCommand extends Command {
     if (shellState.getTableName().equals(cl.getArgs()[0])) {
       String tableName = cl.getArgs()[1];
       String n = Tables.extractNamespace(tableName);
-      if (n.equals(Constants.DEFAULT_TABLE_NAMESPACE) || n.equals(Constants.SYSTEM_TABLE_NAMESPACE)) {
+      if (n.equals(Constants.DEFAULT_NAMESPACE) || n.equals(Constants.SYSTEM_NAMESPACE)) {
         tableName = Tables.extractTableName(tableName);
       }
       shellState.setTableName(tableName);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RevokeCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RevokeCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RevokeCommand.java
index cc9cd93..dac864d 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RevokeCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RevokeCommand.java
@@ -19,8 +19,8 @@ package org.apache.accumulo.core.util.shell.commands;
 import java.util.Map;
 import java.util.Set;
 
+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.util.BadArgumentException;
 import org.apache.accumulo.core.util.shell.Shell;
@@ -35,15 +35,15 @@ public class RevokeCommand extends TableOperation {
   {
     disableUnflaggedTableOptions();
   }
-  
+
   private Option systemOpt, userOpt;
   private String user;
   private String[] permission;
-  
+
   @Override
   public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws Exception {
     user = cl.hasOption(userOpt.getOpt()) ? cl.getOptionValue(userOpt.getOpt()) : shellState.getConnector().whoami();
-    
+
     permission = cl.getArgs()[0].split("\\.", 2);
     if (cl.hasOption(systemOpt.getOpt()) && permission[0].equalsIgnoreCase("System")) {
       try {
@@ -55,22 +55,22 @@ public class RevokeCommand extends TableOperation {
     } else if (permission[0].equalsIgnoreCase("Table")) {
       super.execute(fullCommand, cl, shellState);
     } else if (permission[0].equalsIgnoreCase("Namespace")) {
-      if (cl.hasOption(optTableNamespace.getOpt())) {
+      if (cl.hasOption(optNamespace.getOpt())) {
         try {
           shellState.getConnector().securityOperations()
-              .revokeTableNamespacePermission(user, cl.getOptionValue(optTableNamespace.getOpt()), TableNamespacePermission.valueOf(permission[1]));
+              .revokeNamespacePermission(user, cl.getOptionValue(optNamespace.getOpt()), NamespacePermission.valueOf(permission[1]));
         } catch (IllegalArgumentException e) {
-          throw new BadArgumentException("No such table namespace permission", fullCommand, fullCommand.indexOf(cl.getArgs()[0]));
+          throw new BadArgumentException("No such namespace permission", fullCommand, fullCommand.indexOf(cl.getArgs()[0]));
         }
       } else {
-        throw new BadArgumentException("No Table Namespace specified to apply permission to", fullCommand, fullCommand.indexOf(cl.getArgs()[0]));
-      } 
-    }else {
+        throw new BadArgumentException("No namespace specified to apply permission to", fullCommand, fullCommand.indexOf(cl.getArgs()[0]));
+      }
+    } else {
       throw new BadArgumentException("Unrecognized permission", fullCommand, fullCommand.indexOf(cl.getArgs()[0]));
     }
     return 0;
   }
-  
+
   @Override
   protected void doTableOp(final Shell shellState, final String tableName) throws Exception {
     try {
@@ -80,52 +80,52 @@ public class RevokeCommand extends TableOperation {
       throw new IllegalArgumentException("No such table permission", e);
     }
   }
-  
+
   @Override
   public String description() {
     return "revokes system or table permissions from a user";
   }
-  
+
   @Override
   public String usage() {
     return getName() + " <permission>";
   }
-  
+
   @Override
   public void registerCompletion(final Token root, final Map<Command.CompletionSet,Set<String>> completionSet) {
     final Token cmd = new Token(getName());
     cmd.addSubcommand(new Token(TablePermission.printableValues()));
     cmd.addSubcommand(new Token(SystemPermission.printableValues()));
-    cmd.addSubcommand(new Token(TableNamespacePermission.printableValues()));
+    cmd.addSubcommand(new Token(NamespacePermission.printableValues()));
     root.addSubcommand(cmd);
   }
-  
+
   @Override
   public Options getOptions() {
     super.getOptions();
     final Options o = new Options();
-    
+
     final OptionGroup group = new OptionGroup();
-    
+
     systemOpt = new Option("s", "system", false, "revoke a system permission");
-    
-    optTableNamespace = new Option(Shell.tableNamespaceOption, "table-namespace", true, "name of a table namespace to operate on");
-    optTableNamespace.setArgName("tableNamespace");
-        
+
+    optNamespace = new Option(Shell.namespaceOption, "namespace", true, "name of a namespace to operate on");
+    optNamespace.setArgName("namespace");
+
     group.addOption(systemOpt);
     group.addOption(optTableName);
     group.addOption(optTablePattern);
-    group.addOption(optTableNamespace);
-    
+    group.addOption(optNamespace);
+
     o.addOptionGroup(group);
     userOpt = new Option(Shell.userOption, "user", true, "user to operate on");
     userOpt.setArgName("username");
     userOpt.setRequired(true);
     o.addOption(userOpt);
-    
+
     return o;
   }
-  
+
   @Override
   public int numArgs() {
     return 1;


[15/50] [abbrv] git commit: ACCUMULO-802 minor fixes of variables that got clobbered in rebase

Posted by ct...@apache.org.
ACCUMULO-802 minor fixes of variables that got clobbered in rebase


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

Branch: refs/heads/master
Commit: 5bcd3d2966f321847085bc1d62fe39e2e7234609
Parents: 02374a0
Author: Sean Hickey <ta...@gmail.com>
Authored: Thu Aug 8 15:05:16 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:10 2013 -0500

----------------------------------------------------------------------
 .../src/test/java/org/apache/accumulo/test/ShellServerIT.java | 2 +-
 .../test/java/org/apache/accumulo/test/TableNamespacesIT.java | 7 +------
 2 files changed, 2 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bcd3d29/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
index 792ba3e..e60e414 100644
--- a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
@@ -719,7 +719,7 @@ public class ShellServerIT extends SimpleMacIT {
   @Test(timeout = 30 * 1000)
   public void systempermission() throws Exception {
     exec("systempermissions");
-    assertEquals(8, output.get().split("\n").length - 1);
+    assertEquals(11, output.get().split("\n").length - 1);
     exec("tablepermissions", true);
     assertEquals(6, output.get().split("\n").length - 1);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bcd3d29/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java b/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
index 1a52f72..38955e3 100644
--- a/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
@@ -395,8 +395,6 @@ public class TableNamespacesIT {
   }
   
   /**
-<<<<<<< HEAD
-<<<<<<< HEAD
    * Tests that when a table moves to a new namespace that it's properties inherit from the new namespace and not the old one
    */
   @Test
@@ -435,11 +433,8 @@ public class TableNamespacesIT {
     assertTrue(!hasProp);
   }
   /**
-   *  Tests new Namespace permissions as well as modifications to Table permissions because of namespaces 
-=======
    * Tests new Namespace permissions as well as modifications to Table permissions because of namespaces. Checks each permission to first make sure the user
    * doesn't have permission to perform the action, then root grants them the permission and we check to make sure they could perform the action.
->>>>>>> ACCUMULO-1479 finished initial implementation of table namespace permissions, including tests
    */
   @Test
   public void testPermissions() throws Exception {
@@ -447,7 +442,7 @@ public class TableNamespacesIT {
     
     PasswordToken pass = new PasswordToken(secret);
     
-    String n1 = "namespace1";
+    String n1 = "spaceOfTheName";
     
     String user1 = "dude";
     


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

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java b/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
new file mode 100644
index 0000000..5b7d11d
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
@@ -0,0 +1,584 @@
+/*
+ * 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.test;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map.Entry;
+import java.util.Random;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.NamespaceNotEmptyException;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.impl.Namespaces;
+import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.Filter;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.NamespacePermission;
+import org.apache.accumulo.core.security.SystemPermission;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.examples.simple.constraints.NumericValueConstraint;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class NamespacesIT {
+
+  Random random = new Random();
+  public static TemporaryFolder folder = new TemporaryFolder();
+  static private MiniAccumuloCluster accumulo;
+  static private String secret = "secret";
+
+  @BeforeClass
+  static public void setUp() throws Exception {
+    folder.create();
+    accumulo = new MiniAccumuloCluster(folder.getRoot(), secret);
+    accumulo.start();
+  }
+
+  @AfterClass
+  static public void tearDown() throws Exception {
+    accumulo.stop();
+    folder.delete();
+  }
+
+  /**
+   * This test creates a table without specifying a namespace. In this case, it puts the table into the default namespace.
+   */
+  @Test
+  public void testDefaultNamespace() throws Exception {
+    String tableName = "test";
+    Connector c = accumulo.getConnector("root", secret);
+
+    assertTrue(c.namespaceOperations().exists(Constants.DEFAULT_NAMESPACE));
+    c.tableOperations().create(tableName);
+    assertTrue(c.tableOperations().exists(tableName));
+  }
+
+  /**
+   * This test creates a new namespace "testing" and a table "testing.table1" which puts "table1" into the "testing" namespace. Then we create "testing.table2"
+   * which creates "table2" and puts it into "testing" as well. Then we make sure that you can't delete a namespace with tables in it, and then we delete the
+   * tables and delete the namespace.
+   */
+  @Test
+  public void testCreateAndDeleteNamespace() throws Exception {
+    String namespace = "testing";
+    String tableName1 = namespace + ".table1";
+    String tableName2 = namespace + ".table2";
+
+    Connector c = accumulo.getConnector("root", secret);
+
+    c.namespaceOperations().create(namespace);
+    assertTrue(c.namespaceOperations().exists(namespace));
+
+    c.tableOperations().create(tableName1);
+    assertTrue(c.tableOperations().exists(tableName1));
+
+    c.tableOperations().create(tableName2);
+    assertTrue(c.tableOperations().exists(tableName2));
+
+    // deleting
+    try {
+      // can't delete a namespace with tables in it
+      c.namespaceOperations().delete(namespace);
+      fail();
+    } catch (NamespaceNotEmptyException e) {
+      // ignore, supposed to happen
+    }
+    assertTrue(c.namespaceOperations().exists(namespace));
+    assertTrue(c.tableOperations().exists(tableName1));
+    assertTrue(c.tableOperations().exists(tableName2));
+
+    c.tableOperations().delete(tableName2);
+    assertTrue(!c.tableOperations().exists(tableName2));
+    assertTrue(c.namespaceOperations().exists(namespace));
+
+    c.tableOperations().delete(tableName1);
+    assertTrue(!c.tableOperations().exists(tableName1));
+    c.namespaceOperations().delete(namespace);
+    assertTrue(!c.namespaceOperations().exists(namespace));
+  }
+
+  /**
+   * This test creates a namespace, modifies it's properties, and checks to make sure that those properties are applied to its tables. To do something on a
+   * namespace-wide level, use NamespaceOperations.
+   * 
+   * Checks to make sure namespace-level properties are overridden by table-level properties.
+   * 
+   * Checks to see if the default namespace's properties work as well.
+   */
+
+  @Test
+  public void testNamespaceProperties() throws Exception {
+    String namespace = "propchange";
+    String tableName1 = namespace + ".table1";
+    String tableName2 = namespace + ".table2";
+
+    String propKey = Property.TABLE_SCAN_MAXMEM.getKey();
+    String propVal = "42K";
+
+    Connector c = accumulo.getConnector("root", secret);
+
+    c.namespaceOperations().create(namespace);
+    c.tableOperations().create(tableName1);
+    c.namespaceOperations().setProperty(namespace, propKey, propVal);
+
+    // check the namespace has the property
+    assertTrue(checkNamespaceHasProp(c, namespace, propKey, propVal));
+
+    // check that the table gets it from the namespace
+    assertTrue(checkTableHasProp(c, tableName1, propKey, propVal));
+
+    // test a second table to be sure the first wasn't magical
+    // (also, changed the order, the namespace has the property already)
+    c.tableOperations().create(tableName2);
+    assertTrue(checkTableHasProp(c, tableName2, propKey, propVal));
+
+    // test that table properties override namespace properties
+    String propKey2 = Property.TABLE_FILE_MAX.getKey();
+    String propVal2 = "42";
+    String tablePropVal = "13";
+
+    c.tableOperations().setProperty(tableName2, propKey2, tablePropVal);
+    c.namespaceOperations().setProperty("propchange", propKey2, propVal2);
+
+    assertTrue(checkTableHasProp(c, tableName2, propKey2, tablePropVal));
+
+    // now check that you can change the default namespace's properties
+    propVal = "13K";
+    String tableName = "some_table";
+    c.tableOperations().create(tableName);
+    c.namespaceOperations().setProperty(Constants.DEFAULT_NAMESPACE, propKey, propVal);
+
+    assertTrue(checkTableHasProp(c, tableName, propKey, propVal));
+
+    // test the properties server-side by configuring an iterator.
+    // should not show anything with column-family = 'a'
+    String tableName3 = namespace + ".table3";
+    c.tableOperations().create(tableName3);
+
+    IteratorSetting setting = new IteratorSetting(250, "thing", SimpleFilter.class.getName());
+    c.namespaceOperations().attachIterator(namespace, setting);
+
+    BatchWriter bw = c.createBatchWriter(tableName3, new BatchWriterConfig());
+    Mutation m = new Mutation("r");
+    m.put("a", "b", new Value("abcde".getBytes()));
+    bw.addMutation(m);
+    bw.flush();
+    bw.close();
+
+    Scanner s = c.createScanner(tableName3, Authorizations.EMPTY);
+    assertTrue(!s.iterator().hasNext());
+  }
+
+  /**
+   * This test renames and clones two separate table into different namespaces. different namespace.
+   * 
+   */
+  @Test
+  public void testRenameAndCloneTableToNewNamespace() throws Exception {
+    String namespace1 = "renamed";
+    String namespace2 = "cloned";
+    String tableName = "table";
+    String tableName1 = "renamed.table1";
+    String tableName2 = "cloned.table2";
+
+    Connector c = accumulo.getConnector("root", secret);
+
+    c.tableOperations().create(tableName);
+    c.namespaceOperations().create(namespace1);
+    c.namespaceOperations().create(namespace2);
+
+    c.tableOperations().rename(tableName, tableName1);
+
+    assertTrue(c.tableOperations().exists(tableName1));
+    assertTrue(!c.tableOperations().exists(tableName));
+
+    c.tableOperations().clone(tableName1, tableName2, false, null, null);
+
+    assertTrue(c.tableOperations().exists(tableName1));
+    assertTrue(c.tableOperations().exists(tableName2));
+    return;
+  }
+
+  /**
+   * This test renames a namespace and ensures that its tables are still correct
+   */
+  @Test
+  public void testNamespaceRename() throws Exception {
+    String namespace1 = "n1";
+    String namespace2 = "n2";
+    String table = "t";
+
+    Connector c = accumulo.getConnector("root", secret);
+    Instance instance = c.getInstance();
+
+    c.namespaceOperations().create(namespace1);
+    c.tableOperations().create(namespace1 + "." + table);
+
+    c.namespaceOperations().rename(namespace1, namespace2);
+
+    assertTrue(!c.namespaceOperations().exists(namespace1));
+    assertTrue(c.namespaceOperations().exists(namespace2));
+    assertTrue(c.tableOperations().exists(namespace2 + "." + table));
+    String tid = Tables.getTableId(instance, namespace2 + "." + table);
+    String tnid = Tables.getNamespace(instance, tid);
+    String tnamespace = Namespaces.getNamespaceName(instance, tnid);
+    assertTrue(namespace2.equals(tnamespace));
+  }
+
+  /**
+   * This test clones a table to a different namespace and ensures it's properties are correct
+   */
+  @Test
+  public void testCloneTableProperties() throws Exception {
+    String n1 = "namespace1";
+    String n2 = "namespace2";
+    String t1 = n1 + ".table";
+    String t2 = n2 + ".table";
+
+    String propKey = Property.TABLE_FILE_MAX.getKey();
+    String propVal1 = "55";
+    String propVal2 = "66";
+
+    Connector c = accumulo.getConnector("root", secret);
+
+    c.namespaceOperations().create(n1);
+    c.tableOperations().create(t1);
+
+    c.tableOperations().removeProperty(t1, Property.TABLE_FILE_MAX.getKey());
+    c.namespaceOperations().setProperty(n1, propKey, propVal1);
+
+    assertTrue(checkTableHasProp(c, t1, propKey, propVal1));
+
+    c.namespaceOperations().create(n2);
+    c.namespaceOperations().setProperty(n2, propKey, propVal2);
+    c.tableOperations().clone(t1, t2, true, null, null);
+    c.tableOperations().removeProperty(t2, propKey);
+
+    assertTrue(checkTableHasProp(c, t2, propKey, propVal2));
+
+    c.namespaceOperations().delete(n1, true);
+    c.namespaceOperations().delete(n2, true);
+  }
+
+  /**
+   * This tests adding iterators to a namespace, listing them, and removing them as well as adding and removing constraints
+   */
+  @Test
+  public void testNamespaceIteratorsAndConstraints() throws Exception {
+    Connector c = accumulo.getConnector("root", secret);
+
+    String namespace = "iterator";
+    String tableName = namespace + ".table";
+    String iter = "thing";
+
+    c.namespaceOperations().create(namespace);
+    c.tableOperations().create(tableName);
+
+    IteratorSetting setting = new IteratorSetting(250, iter, SimpleFilter.class.getName());
+    HashSet<IteratorScope> scope = new HashSet<IteratorScope>();
+    scope.add(IteratorScope.scan);
+    c.namespaceOperations().attachIterator(namespace, setting, EnumSet.copyOf(scope));
+
+    BatchWriter bw = c.createBatchWriter(tableName, new BatchWriterConfig());
+    Mutation m = new Mutation("r");
+    m.put("a", "b", new Value("abcde".getBytes(Constants.UTF8)));
+    bw.addMutation(m);
+    bw.flush();
+
+    Scanner s = c.createScanner(tableName, Authorizations.EMPTY);
+    assertTrue(!s.iterator().hasNext());
+
+    assertTrue(c.namespaceOperations().listIterators(namespace).containsKey(iter));
+    c.namespaceOperations().removeIterator(namespace, iter, EnumSet.copyOf(scope));
+
+    c.namespaceOperations().addConstraint(namespace, NumericValueConstraint.class.getName());
+    // doesn't take effect immediately, needs time to propagate
+    UtilWaitThread.sleep(250);
+
+    m = new Mutation("rowy");
+    m.put("a", "b", new Value("abcde".getBytes(Constants.UTF8)));
+    try {
+      bw.addMutation(m);
+      bw.flush();
+      bw.close();
+      fail();
+    } catch (MutationsRejectedException e) {
+      // supposed to be thrown
+    }
+    int num = c.namespaceOperations().listConstraints(namespace).get(NumericValueConstraint.class.getName());
+    c.namespaceOperations().removeConstraint(namespace, num);
+  }
+
+  /**
+   * Tests that when a table moves to a new namespace that it's properties inherit from the new namespace and not the old one
+   */
+  @Test
+  public void testRenameToNewNamespaceProperties() throws Exception {
+    Connector c = accumulo.getConnector("root", secret);
+
+    String namespace1 = "moveToNewNamespace1";
+    String namespace2 = "moveToNewNamespace2";
+    String tableName1 = namespace1 + ".table";
+    String tableName2 = namespace2 + ".table";
+
+    String propKey = Property.TABLE_FILE_MAX.getKey();
+    String propVal = "42";
+
+    c.namespaceOperations().create(namespace1);
+    c.namespaceOperations().create(namespace2);
+    c.tableOperations().create(tableName1);
+
+    c.namespaceOperations().setProperty(namespace1, propKey, propVal);
+    boolean hasProp = false;
+    for (Entry<String,String> p : c.tableOperations().getProperties(tableName1)) {
+      if (p.getKey().equals(propKey) && p.getValue().equals(propVal)) {
+        hasProp = true;
+      }
+    }
+    assertTrue(hasProp);
+
+    c.tableOperations().rename(tableName1, tableName2);
+
+    hasProp = false;
+    for (Entry<String,String> p : c.tableOperations().getProperties(tableName2)) {
+      if (p.getKey().equals(propKey) && p.getValue().equals(propVal)) {
+        hasProp = true;
+      }
+    }
+    assertTrue(!hasProp);
+  }
+
+  /**
+   * Tests new Namespace permissions as well as modifications to Table permissions because of namespaces. Checks each permission to first make sure the user
+   * doesn't have permission to perform the action, then root grants them the permission and we check to make sure they could perform the action.
+   */
+  @Test
+  public void testPermissions() throws Exception {
+    Connector c = accumulo.getConnector("root", secret);
+
+    PasswordToken pass = new PasswordToken(secret);
+
+    String n1 = "spaceOfTheName";
+
+    String user1 = "dude";
+
+    c.namespaceOperations().create(n1);
+    c.tableOperations().create(n1 + ".table1");
+
+    c.securityOperations().createLocalUser(user1, pass);
+
+    Connector user1Con = accumulo.getConnector(user1, secret);
+
+    try {
+      user1Con.tableOperations().create(n1 + ".table2");
+      fail();
+    } catch (AccumuloSecurityException e) {
+      // supposed to happen
+    }
+
+    c.securityOperations().grantNamespacePermission(user1, n1, NamespacePermission.CREATE_TABLE);
+    user1Con.tableOperations().create(n1 + ".table2");
+    assertTrue(c.tableOperations().list().contains(n1 + ".table2"));
+    c.securityOperations().revokeNamespacePermission(user1, n1, NamespacePermission.CREATE_TABLE);
+
+    try {
+      user1Con.tableOperations().delete(n1 + ".table1");
+      fail();
+    } catch (AccumuloSecurityException e) {
+      // should happen
+    }
+
+    c.securityOperations().grantNamespacePermission(user1, n1, NamespacePermission.DROP_TABLE);
+    user1Con.tableOperations().delete(n1 + ".table1");
+    assertTrue(!c.tableOperations().list().contains(n1 + ".table1"));
+    c.securityOperations().revokeNamespacePermission(user1, n1, NamespacePermission.DROP_TABLE);
+
+    c.tableOperations().create(n1 + ".t");
+    BatchWriter bw = c.createBatchWriter(n1 + ".t", null);
+    Mutation m = new Mutation("row");
+    m.put("cf", "cq", "value");
+    bw.addMutation(m);
+    bw.close();
+
+    Iterator<Entry<Key,Value>> i = user1Con.createScanner(n1 + ".t", new Authorizations()).iterator();
+    try {
+      i.next();
+      fail();
+    } catch (RuntimeException e) {
+      // yup
+    }
+
+    m = new Mutation("user1");
+    m.put("cf", "cq", "turtles");
+    bw = user1Con.createBatchWriter(n1 + ".t", null);
+    try {
+      bw.addMutation(m);
+      bw.close();
+      fail();
+    } catch (MutationsRejectedException e) {
+      // good
+    }
+
+    c.securityOperations().grantNamespacePermission(user1, n1, NamespacePermission.READ);
+    i = user1Con.createScanner(n1 + ".t", new Authorizations()).iterator();
+    assertTrue(i.hasNext());
+    c.securityOperations().revokeNamespacePermission(user1, n1, NamespacePermission.READ);
+
+    c.securityOperations().grantNamespacePermission(user1, n1, NamespacePermission.WRITE);
+    m = new Mutation("user1");
+    m.put("cf", "cq", "turtles");
+    bw = user1Con.createBatchWriter(n1 + ".t", null);
+    bw.addMutation(m);
+    bw.close();
+    c.securityOperations().revokeNamespacePermission(user1, n1, NamespacePermission.WRITE);
+
+    try {
+      user1Con.tableOperations().setProperty(n1 + ".t", Property.TABLE_FILE_MAX.getKey(), "42");
+      fail();
+    } catch (AccumuloSecurityException e) {}
+
+    c.securityOperations().grantNamespacePermission(user1, n1, NamespacePermission.ALTER_TABLE);
+    user1Con.tableOperations().setProperty(n1 + ".t", Property.TABLE_FILE_MAX.getKey(), "42");
+    user1Con.tableOperations().removeProperty(n1 + ".t", Property.TABLE_FILE_MAX.getKey());
+    c.securityOperations().revokeNamespacePermission(user1, n1, NamespacePermission.ALTER_TABLE);
+
+    try {
+      user1Con.namespaceOperations().setProperty(n1, Property.TABLE_FILE_MAX.getKey(), "55");
+      fail();
+    } catch (AccumuloSecurityException e) {}
+
+    c.securityOperations().grantNamespacePermission(user1, n1, NamespacePermission.ALTER_NAMESPACE);
+    user1Con.namespaceOperations().setProperty(n1, Property.TABLE_FILE_MAX.getKey(), "42");
+    user1Con.namespaceOperations().removeProperty(n1, Property.TABLE_FILE_MAX.getKey());
+    c.securityOperations().revokeNamespacePermission(user1, n1, NamespacePermission.ALTER_NAMESPACE);
+
+    String user2 = "guy";
+    c.securityOperations().createLocalUser(user2, pass);
+    try {
+      user1Con.securityOperations().grantNamespacePermission(user2, n1, NamespacePermission.ALTER_NAMESPACE);
+      fail();
+    } catch (AccumuloSecurityException e) {}
+
+    c.securityOperations().grantNamespacePermission(user1, n1, NamespacePermission.GRANT);
+    user1Con.securityOperations().grantNamespacePermission(user2, n1, NamespacePermission.ALTER_NAMESPACE);
+    user1Con.securityOperations().revokeNamespacePermission(user2, n1, NamespacePermission.ALTER_NAMESPACE);
+    c.securityOperations().revokeNamespacePermission(user1, n1, NamespacePermission.GRANT);
+
+    String n2 = "namespace2";
+    try {
+      user1Con.namespaceOperations().create(n2);
+      fail();
+    } catch (AccumuloSecurityException e) {}
+
+    c.securityOperations().grantSystemPermission(user1, SystemPermission.CREATE_NAMESPACE);
+    user1Con.namespaceOperations().create(n2);
+    c.securityOperations().revokeSystemPermission(user1, SystemPermission.CREATE_NAMESPACE);
+
+    try {
+      user1Con.namespaceOperations().delete(n2);
+      fail();
+    } catch (AccumuloSecurityException e) {}
+
+    c.securityOperations().grantSystemPermission(user1, SystemPermission.DROP_NAMESPACE);
+    user1Con.namespaceOperations().delete(n2);
+    c.securityOperations().revokeSystemPermission(user1, SystemPermission.DROP_NAMESPACE);
+
+    try {
+      user1Con.namespaceOperations().setProperty(n1, Property.TABLE_FILE_MAX.getKey(), "33");
+      fail();
+    } catch (AccumuloSecurityException e) {}
+
+    c.securityOperations().grantSystemPermission(user1, SystemPermission.ALTER_NAMESPACE);
+    user1Con.namespaceOperations().setProperty(n1, Property.TABLE_FILE_MAX.getKey(), "33");
+    user1Con.namespaceOperations().removeProperty(n1, Property.TABLE_FILE_MAX.getKey());
+    c.securityOperations().revokeSystemPermission(user1, SystemPermission.ALTER_NAMESPACE);
+  }
+
+  /**
+   * This test makes sure that system-level iterators and constraints are ignored by the system namespace so that the metadata and root tables aren't affected
+   */
+  @Test
+  public void excludeSystemIterConst() throws Exception {
+    Connector c = accumulo.getConnector("root", secret);
+
+    c.instanceOperations().setProperty("table.iterator.scan.sum", "20," + SimpleFilter.class.getName());
+    assertTrue(c.instanceOperations().getSystemConfiguration().containsValue("20," + SimpleFilter.class.getName()));
+
+    assertTrue(checkNamespaceHasProp(c, Constants.DEFAULT_NAMESPACE, "table.iterator.scan.sum", "20," + SimpleFilter.class.getName()));
+    assertTrue(!checkNamespaceHasProp(c, Constants.SYSTEM_NAMESPACE, "table.iterator.scan.sum", "20," + SimpleFilter.class.getName()));
+    c.instanceOperations().removeProperty("table.iterator.scan.sum");
+
+    c.instanceOperations().setProperty("table.constraint.42", NumericValueConstraint.class.getName());
+    assertTrue(c.instanceOperations().getSystemConfiguration().containsValue(NumericValueConstraint.class.getName()));
+
+    assertTrue(checkNamespaceHasProp(c, Constants.DEFAULT_NAMESPACE, "table.constraint.42", NumericValueConstraint.class.getName()));
+    assertTrue(!checkNamespaceHasProp(c, Constants.SYSTEM_NAMESPACE, "table.constraint.42", NumericValueConstraint.class.getName()));
+    c.instanceOperations().removeProperty("table.constraint.42");
+  }
+
+  private boolean checkTableHasProp(Connector c, String t, String propKey, String propVal) throws AccumuloException, TableNotFoundException {
+    for (Entry<String,String> e : c.tableOperations().getProperties(t)) {
+      if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private boolean checkNamespaceHasProp(Connector c, String n, String propKey, String propVal) throws AccumuloException, NamespaceNotFoundException {
+    for (Entry<String,String> e : c.namespaceOperations().getProperties(n)) {
+      if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  public static class SimpleFilter extends Filter {
+    @Override
+    public boolean accept(Key k, Value v) {
+      if (k.getColumnFamily().toString().equals("a"))
+        return false;
+      return true;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
index e60e414..f21dfd9 100644
--- a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
@@ -704,7 +704,7 @@ public class ShellServerIT extends SimpleMacIT {
     exec("scan -t xyzzy", true, "value", true);
     exec("deletetable -f xyzzy", true);
   }
-  
+
   @Test(timeout = 30 * 1000)
   public void tables() throws Exception {
     exec("createtable zzzz");
@@ -831,33 +831,33 @@ public class ShellServerIT extends SimpleMacIT {
     String err = exec("user NoSuchUser", false);
     assertTrue(err.contains("BAD_CREDENTIALS for user NoSuchUser"));
   }
-  
+
   @Test(timeout = 30 * 1000)
-  public void tablenamespaces() throws Exception {
-    exec("namespaces", true, Constants.DEFAULT_TABLE_NAMESPACE, true);
+  public void namespaces() throws Exception {
+    exec("namespaces", true, Constants.DEFAULT_NAMESPACE, true);
     exec("createnamespace thing1", true);
     String namespaces = exec("namespaces");
     assertTrue(namespaces.contains("thing1"));
-    
+
     exec("renamenamespace thing1 thing2");
     namespaces = exec("namespaces");
     assertTrue(namespaces.contains("thing2"));
     assertTrue(!namespaces.contains("thing1"));
-    
+
     // can't delete a namespace that still contains tables, unless you do -f
     exec("createtable thing2.thingy", true);
     exec("deletenamespace thing2");
     exec("y");
     exec("namespaces", true, "thing2", true);
-    
+
     exec("clonenamespace thing2 testers -e table.file.max", true);
     exec("namespaces", true, "testers", true);
     exec("tables", true, "testers.thingy", true);
     exec("clonenamespace thing2 testers2 -s table.file.max=42", true);
-    
+
     exec("du -tn thing2", true, "thing2.thingy", true);
-    
-    // all "TableOperation" commands can take a table namespace
+
+    // all "TableOperation" commands can take a namespace
     exec("offline -tn thing2", true);
     exec("online -tn thing2", true);
     exec("flush -tn thing2", true);
@@ -878,25 +878,25 @@ public class ShellServerIT extends SimpleMacIT {
     exec("pass");
     exec("grant Namespace.CREATE_TABLE -tn thing2 -u dude", true);
     exec("revoke Namespace.CREATE_TABLE -tn thing2 -u dude", true);
-    
+
     // properties override and such
     exec("config -tn thing2 -s table.file.max=44444", true);
     exec("config -tn thing2", true, "44444", true);
     exec("config -t thing2.thingy", true, "44444", true);
     exec("config -t thing2.thingy -s table.file.max=55555", true);
     exec("config -t thing2.thingy", true, "55555", true);
-    
+
     // can copy properties when creating
     exec("createnamespace thing3 -cc thing2", true);
     exec("config -tn thing3", true, "44444", true);
     exec("createnamespace thing4 -ctc thing2.thingy", true);
     exec("config -tn thing4", true, "55555", true);
-    
+
     exec("deletenamespace -f thing2", true);
     exec("namespaces", true, "thing2", false);
     exec("tables", true, "thing2.thingy", false);
-    
-    // put constraints on a table namespace
+
+    // put constraints on a namespace
     exec("constraint -tn thing4 -a org.apache.accumulo.examples.simple.constraints.NumericValueConstraint", true);
     exec("createtable thing4.constrained", true);
     exec("table thing4.constrained", true);
@@ -907,7 +907,7 @@ public class ShellServerIT extends SimpleMacIT {
     exec("sleep 1");
     exec("insert r cf cq abc", true);
   }
-  
+
   private int countkeys(String table) throws IOException {
     exec("scan -np -t " + table);
     return output.get().split("\n").length - 1;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java b/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
deleted file mode 100644
index 6cb2568..0000000
--- a/test/src/test/java/org/apache/accumulo/test/TableNamespacesIT.java
+++ /dev/null
@@ -1,584 +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.test;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.util.EnumSet;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map.Entry;
-import java.util.Random;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableNamespaceNotEmptyException;
-import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.impl.TableNamespaces;
-import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.iterators.Filter;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-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.util.UtilWaitThread;
-import org.apache.accumulo.examples.simple.constraints.NumericValueConstraint;
-import org.apache.accumulo.minicluster.MiniAccumuloCluster;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-public class TableNamespacesIT {
-
-  Random random = new Random();
-  public static TemporaryFolder folder = new TemporaryFolder();
-  static private MiniAccumuloCluster accumulo;
-  static private String secret = "secret";
-
-  @BeforeClass
-  static public void setUp() throws Exception {
-    folder.create();
-    accumulo = new MiniAccumuloCluster(folder.getRoot(), secret);
-    accumulo.start();
-  }
-
-  @AfterClass
-  static public void tearDown() throws Exception {
-    accumulo.stop();
-    folder.delete();
-  }
-
-  /**
-   * This test creates a table without specifying a namespace. In this case, it puts the table into the default namespace.
-   */
-  @Test
-  public void testDefaultNamespace() throws Exception {
-    String tableName = "test";
-    Connector c = accumulo.getConnector("root", secret);
-
-    assertTrue(c.tableNamespaceOperations().exists(Constants.DEFAULT_TABLE_NAMESPACE));
-    c.tableOperations().create(tableName);
-    assertTrue(c.tableOperations().exists(tableName));
-  }
-
-  /**
-   * This test creates a new namespace "testing" and a table "testing.table1" which puts "table1" into the "testing" namespace. Then we create "testing.table2"
-   * which creates "table2" and puts it into "testing" as well. Then we make sure that you can't delete a namespace with tables in it, and then we delete the
-   * tables and delete the namespace.
-   */
-  @Test
-  public void testCreateAndDeleteNamespace() throws Exception {
-    String namespace = "testing";
-    String tableName1 = namespace + ".table1";
-    String tableName2 = namespace + ".table2";
-
-    Connector c = accumulo.getConnector("root", secret);
-
-    c.tableNamespaceOperations().create(namespace);
-    assertTrue(c.tableNamespaceOperations().exists(namespace));
-
-    c.tableOperations().create(tableName1);
-    assertTrue(c.tableOperations().exists(tableName1));
-
-    c.tableOperations().create(tableName2);
-    assertTrue(c.tableOperations().exists(tableName2));
-
-    // deleting
-    try {
-      // can't delete a namespace with tables in it
-      c.tableNamespaceOperations().delete(namespace);
-      fail();
-    } catch (TableNamespaceNotEmptyException e) {
-      // ignore, supposed to happen
-    }
-    assertTrue(c.tableNamespaceOperations().exists(namespace));
-    assertTrue(c.tableOperations().exists(tableName1));
-    assertTrue(c.tableOperations().exists(tableName2));
-
-    c.tableOperations().delete(tableName2);
-    assertTrue(!c.tableOperations().exists(tableName2));
-    assertTrue(c.tableNamespaceOperations().exists(namespace));
-
-    c.tableOperations().delete(tableName1);
-    assertTrue(!c.tableOperations().exists(tableName1));
-    c.tableNamespaceOperations().delete(namespace);
-    assertTrue(!c.tableNamespaceOperations().exists(namespace));
-  }
-
-  /**
-   * This test creates a namespace, modifies it's properties, and checks to make sure that those properties are applied to its tables. To do something on a
-   * namespace-wide level, use TableNamespaceOperations.
-   * 
-   * Checks to make sure namespace-level properties are overridden by table-level properties.
-   * 
-   * Checks to see if the default namespace's properties work as well.
-   */
-
-  @Test
-  public void testNamespaceProperties() throws Exception {
-    String namespace = "propchange";
-    String tableName1 = namespace + ".table1";
-    String tableName2 = namespace + ".table2";
-
-    String propKey = Property.TABLE_SCAN_MAXMEM.getKey();
-    String propVal = "42K";
-
-    Connector c = accumulo.getConnector("root", secret);
-
-    c.tableNamespaceOperations().create(namespace);
-    c.tableOperations().create(tableName1);
-    c.tableNamespaceOperations().setProperty(namespace, propKey, propVal);
-
-    // check the namespace has the property
-    assertTrue(checkTableNamespaceHasProp(c, namespace, propKey, propVal));
-
-    // check that the table gets it from the namespace
-    assertTrue(checkTableHasProp(c, tableName1, propKey, propVal));
-
-    // test a second table to be sure the first wasn't magical
-    // (also, changed the order, the namespace has the property already)
-    c.tableOperations().create(tableName2);
-    assertTrue(checkTableHasProp(c, tableName2, propKey, propVal));
-
-    // test that table properties override namespace properties
-    String propKey2 = Property.TABLE_FILE_MAX.getKey();
-    String propVal2 = "42";
-    String tablePropVal = "13";
-
-    c.tableOperations().setProperty(tableName2, propKey2, tablePropVal);
-    c.tableNamespaceOperations().setProperty("propchange", propKey2, propVal2);
-
-    assertTrue(checkTableHasProp(c, tableName2, propKey2, tablePropVal));
-
-    // now check that you can change the default namespace's properties
-    propVal = "13K";
-    String tableName = "some_table";
-    c.tableOperations().create(tableName);
-    c.tableNamespaceOperations().setProperty(Constants.DEFAULT_TABLE_NAMESPACE, propKey, propVal);
-
-    assertTrue(checkTableHasProp(c, tableName, propKey, propVal));
-
-    // test the properties server-side by configuring an iterator.
-    // should not show anything with column-family = 'a'
-    String tableName3 = namespace + ".table3";
-    c.tableOperations().create(tableName3);
-
-    IteratorSetting setting = new IteratorSetting(250, "thing", SimpleFilter.class.getName());
-    c.tableNamespaceOperations().attachIterator(namespace, setting);
-
-    BatchWriter bw = c.createBatchWriter(tableName3, new BatchWriterConfig());
-    Mutation m = new Mutation("r");
-    m.put("a", "b", new Value("abcde".getBytes()));
-    bw.addMutation(m);
-    bw.flush();
-    bw.close();
-
-    Scanner s = c.createScanner(tableName3, Authorizations.EMPTY);
-    assertTrue(!s.iterator().hasNext());
-  }
-
-  /**
-   * This test renames and clones two separate table into different namespaces. different namespace.
-   * 
-   */
-  @Test
-  public void testRenameAndCloneTableToNewNamespace() throws Exception {
-    String namespace1 = "renamed";
-    String namespace2 = "cloned";
-    String tableName = "table";
-    String tableName1 = "renamed.table1";
-    String tableName2 = "cloned.table2";
-
-    Connector c = accumulo.getConnector("root", secret);
-
-    c.tableOperations().create(tableName);
-    c.tableNamespaceOperations().create(namespace1);
-    c.tableNamespaceOperations().create(namespace2);
-
-    c.tableOperations().rename(tableName, tableName1);
-
-    assertTrue(c.tableOperations().exists(tableName1));
-    assertTrue(!c.tableOperations().exists(tableName));
-
-    c.tableOperations().clone(tableName1, tableName2, false, null, null);
-
-    assertTrue(c.tableOperations().exists(tableName1));
-    assertTrue(c.tableOperations().exists(tableName2));
-    return;
-  }
-
-  /**
-   * This test renames a table namespace and ensures that its tables are still correct
-   */
-  @Test
-  public void testNamespaceRename() throws Exception {
-    String namespace1 = "n1";
-    String namespace2 = "n2";
-    String table = "t";
-
-    Connector c = accumulo.getConnector("root", secret);
-    Instance instance = c.getInstance();
-
-    c.tableNamespaceOperations().create(namespace1);
-    c.tableOperations().create(namespace1 + "." + table);
-
-    c.tableNamespaceOperations().rename(namespace1, namespace2);
-
-    assertTrue(!c.tableNamespaceOperations().exists(namespace1));
-    assertTrue(c.tableNamespaceOperations().exists(namespace2));
-    assertTrue(c.tableOperations().exists(namespace2 + "." + table));
-    String tid = Tables.getTableId(instance, namespace2 + "." + table);
-    String tnid = Tables.getNamespace(instance, tid);
-    String tnamespace = TableNamespaces.getNamespaceName(instance, tnid);
-    assertTrue(namespace2.equals(tnamespace));
-  }
-
-  /**
-   * This test clones a table to a different namespace and ensures it's properties are correct
-   */
-  @Test
-  public void testCloneTableProperties() throws Exception {
-    String n1 = "namespace1";
-    String n2 = "namespace2";
-    String t1 = n1 + ".table";
-    String t2 = n2 + ".table";
-
-    String propKey = Property.TABLE_FILE_MAX.getKey();
-    String propVal1 = "55";
-    String propVal2 = "66";
-
-    Connector c = accumulo.getConnector("root", secret);
-
-    c.tableNamespaceOperations().create(n1);
-    c.tableOperations().create(t1);
-
-    c.tableOperations().removeProperty(t1, Property.TABLE_FILE_MAX.getKey());
-    c.tableNamespaceOperations().setProperty(n1, propKey, propVal1);
-
-    assertTrue(checkTableHasProp(c, t1, propKey, propVal1));
-
-    c.tableNamespaceOperations().create(n2);
-    c.tableNamespaceOperations().setProperty(n2, propKey, propVal2);
-    c.tableOperations().clone(t1, t2, true, null, null);
-    c.tableOperations().removeProperty(t2, propKey);
-
-    assertTrue(checkTableHasProp(c, t2, propKey, propVal2));
-
-    c.tableNamespaceOperations().delete(n1, true);
-    c.tableNamespaceOperations().delete(n2, true);
-  }
-
-  /**
-   * This tests adding iterators to a namespace, listing them, and removing them as well as adding and removing constraints
-   */
-  @Test
-  public void testNamespaceIteratorsAndConstraints() throws Exception {
-    Connector c = accumulo.getConnector("root", secret);
-
-    String namespace = "iterator";
-    String tableName = namespace + ".table";
-    String iter = "thing";
-
-    c.tableNamespaceOperations().create(namespace);
-    c.tableOperations().create(tableName);
-
-    IteratorSetting setting = new IteratorSetting(250, iter, SimpleFilter.class.getName());
-    HashSet<IteratorScope> scope = new HashSet<IteratorScope>();
-    scope.add(IteratorScope.scan);
-    c.tableNamespaceOperations().attachIterator(namespace, setting, EnumSet.copyOf(scope));
-
-    BatchWriter bw = c.createBatchWriter(tableName, new BatchWriterConfig());
-    Mutation m = new Mutation("r");
-    m.put("a", "b", new Value("abcde".getBytes(Constants.UTF8)));
-    bw.addMutation(m);
-    bw.flush();
-
-    Scanner s = c.createScanner(tableName, Authorizations.EMPTY);
-    assertTrue(!s.iterator().hasNext());
-
-    assertTrue(c.tableNamespaceOperations().listIterators(namespace).containsKey(iter));
-    c.tableNamespaceOperations().removeIterator(namespace, iter, EnumSet.copyOf(scope));
-
-    c.tableNamespaceOperations().addConstraint(namespace, NumericValueConstraint.class.getName());
-    // doesn't take effect immediately, needs time to propagate
-    UtilWaitThread.sleep(250);
-
-    m = new Mutation("rowy");
-    m.put("a", "b", new Value("abcde".getBytes(Constants.UTF8)));
-    try {
-      bw.addMutation(m);
-      bw.flush();
-      bw.close();
-      fail();
-    } catch (MutationsRejectedException e) {
-      // supposed to be thrown
-    }
-    int num = c.tableNamespaceOperations().listConstraints(namespace).get(NumericValueConstraint.class.getName());
-    c.tableNamespaceOperations().removeConstraint(namespace, num);
-  }
-
-  /**
-   * Tests that when a table moves to a new namespace that it's properties inherit from the new namespace and not the old one
-   */
-  @Test
-  public void testRenameToNewNamespaceProperties() throws Exception {
-    Connector c = accumulo.getConnector("root", secret);
-
-    String namespace1 = "moveToNewNamespace1";
-    String namespace2 = "moveToNewNamespace2";
-    String tableName1 = namespace1 + ".table";
-    String tableName2 = namespace2 + ".table";
-
-    String propKey = Property.TABLE_FILE_MAX.getKey();
-    String propVal = "42";
-
-    c.tableNamespaceOperations().create(namespace1);
-    c.tableNamespaceOperations().create(namespace2);
-    c.tableOperations().create(tableName1);
-
-    c.tableNamespaceOperations().setProperty(namespace1, propKey, propVal);
-    boolean hasProp = false;
-    for (Entry<String,String> p : c.tableOperations().getProperties(tableName1)) {
-      if (p.getKey().equals(propKey) && p.getValue().equals(propVal)) {
-        hasProp = true;
-      }
-    }
-    assertTrue(hasProp);
-
-    c.tableOperations().rename(tableName1, tableName2);
-
-    hasProp = false;
-    for (Entry<String,String> p : c.tableOperations().getProperties(tableName2)) {
-      if (p.getKey().equals(propKey) && p.getValue().equals(propVal)) {
-        hasProp = true;
-      }
-    }
-    assertTrue(!hasProp);
-  }
-
-  /**
-   * Tests new Namespace permissions as well as modifications to Table permissions because of namespaces. Checks each permission to first make sure the user
-   * doesn't have permission to perform the action, then root grants them the permission and we check to make sure they could perform the action.
-   */
-  @Test
-  public void testPermissions() throws Exception {
-    Connector c = accumulo.getConnector("root", secret);
-
-    PasswordToken pass = new PasswordToken(secret);
-
-    String n1 = "spaceOfTheName";
-
-    String user1 = "dude";
-
-    c.tableNamespaceOperations().create(n1);
-    c.tableOperations().create(n1 + ".table1");
-
-    c.securityOperations().createLocalUser(user1, pass);
-
-    Connector user1Con = accumulo.getConnector(user1, secret);
-
-    try {
-      user1Con.tableOperations().create(n1 + ".table2");
-      fail();
-    } catch (AccumuloSecurityException e) {
-      // supposed to happen
-    }
-
-    c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.CREATE_TABLE);
-    user1Con.tableOperations().create(n1 + ".table2");
-    assertTrue(c.tableOperations().list().contains(n1 + ".table2"));
-    c.securityOperations().revokeTableNamespacePermission(user1, n1, TableNamespacePermission.CREATE_TABLE);
-
-    try {
-      user1Con.tableOperations().delete(n1 + ".table1");
-      fail();
-    } catch (AccumuloSecurityException e) {
-      // should happen
-    }
-
-    c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.DROP_TABLE);
-    user1Con.tableOperations().delete(n1 + ".table1");
-    assertTrue(!c.tableOperations().list().contains(n1 + ".table1"));
-    c.securityOperations().revokeTableNamespacePermission(user1, n1, TableNamespacePermission.DROP_TABLE);
-
-    c.tableOperations().create(n1 + ".t");
-    BatchWriter bw = c.createBatchWriter(n1 + ".t", null);
-    Mutation m = new Mutation("row");
-    m.put("cf", "cq", "value");
-    bw.addMutation(m);
-    bw.close();
-
-    Iterator<Entry<Key,Value>> i = user1Con.createScanner(n1 + ".t", new Authorizations()).iterator();
-    try {
-      i.next();
-      fail();
-    } catch (RuntimeException e) {
-      // yup
-    }
-
-    m = new Mutation("user1");
-    m.put("cf", "cq", "turtles");
-    bw = user1Con.createBatchWriter(n1 + ".t", null);
-    try {
-      bw.addMutation(m);
-      bw.close();
-      fail();
-    } catch (MutationsRejectedException e) {
-      // good
-    }
-
-    c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.READ);
-    i = user1Con.createScanner(n1 + ".t", new Authorizations()).iterator();
-    assertTrue(i.hasNext());
-    c.securityOperations().revokeTableNamespacePermission(user1, n1, TableNamespacePermission.READ);
-
-    c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.WRITE);
-    m = new Mutation("user1");
-    m.put("cf", "cq", "turtles");
-    bw = user1Con.createBatchWriter(n1 + ".t", null);
-    bw.addMutation(m);
-    bw.close();
-    c.securityOperations().revokeTableNamespacePermission(user1, n1, TableNamespacePermission.WRITE);
-
-    try {
-      user1Con.tableOperations().setProperty(n1 + ".t", Property.TABLE_FILE_MAX.getKey(), "42");
-      fail();
-    } catch (AccumuloSecurityException e) {}
-
-    c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.ALTER_TABLE);
-    user1Con.tableOperations().setProperty(n1 + ".t", Property.TABLE_FILE_MAX.getKey(), "42");
-    user1Con.tableOperations().removeProperty(n1 + ".t", Property.TABLE_FILE_MAX.getKey());
-    c.securityOperations().revokeTableNamespacePermission(user1, n1, TableNamespacePermission.ALTER_TABLE);
-
-    try {
-      user1Con.tableNamespaceOperations().setProperty(n1, Property.TABLE_FILE_MAX.getKey(), "55");
-      fail();
-    } catch (AccumuloSecurityException e) {}
-
-    c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.ALTER_NAMESPACE);
-    user1Con.tableNamespaceOperations().setProperty(n1, Property.TABLE_FILE_MAX.getKey(), "42");
-    user1Con.tableNamespaceOperations().removeProperty(n1, Property.TABLE_FILE_MAX.getKey());
-    c.securityOperations().revokeTableNamespacePermission(user1, n1, TableNamespacePermission.ALTER_NAMESPACE);
-
-    String user2 = "guy";
-    c.securityOperations().createLocalUser(user2, pass);
-    try {
-      user1Con.securityOperations().grantTableNamespacePermission(user2, n1, TableNamespacePermission.ALTER_NAMESPACE);
-      fail();
-    } catch (AccumuloSecurityException e) {}
-
-    c.securityOperations().grantTableNamespacePermission(user1, n1, TableNamespacePermission.GRANT);
-    user1Con.securityOperations().grantTableNamespacePermission(user2, n1, TableNamespacePermission.ALTER_NAMESPACE);
-    user1Con.securityOperations().revokeTableNamespacePermission(user2, n1, TableNamespacePermission.ALTER_NAMESPACE);
-    c.securityOperations().revokeTableNamespacePermission(user1, n1, TableNamespacePermission.GRANT);
-
-    String n2 = "namespace2";
-    try {
-      user1Con.tableNamespaceOperations().create(n2);
-      fail();
-    } catch (AccumuloSecurityException e) {}
-
-    c.securityOperations().grantSystemPermission(user1, SystemPermission.CREATE_NAMESPACE);
-    user1Con.tableNamespaceOperations().create(n2);
-    c.securityOperations().revokeSystemPermission(user1, SystemPermission.CREATE_NAMESPACE);
-
-    try {
-      user1Con.tableNamespaceOperations().delete(n2);
-      fail();
-    } catch (AccumuloSecurityException e) {}
-
-    c.securityOperations().grantSystemPermission(user1, SystemPermission.DROP_NAMESPACE);
-    user1Con.tableNamespaceOperations().delete(n2);
-    c.securityOperations().revokeSystemPermission(user1, SystemPermission.DROP_NAMESPACE);
-
-    try {
-      user1Con.tableNamespaceOperations().setProperty(n1, Property.TABLE_FILE_MAX.getKey(), "33");
-      fail();
-    } catch (AccumuloSecurityException e) {}
-
-    c.securityOperations().grantSystemPermission(user1, SystemPermission.ALTER_NAMESPACE);
-    user1Con.tableNamespaceOperations().setProperty(n1, Property.TABLE_FILE_MAX.getKey(), "33");
-    user1Con.tableNamespaceOperations().removeProperty(n1, Property.TABLE_FILE_MAX.getKey());
-    c.securityOperations().revokeSystemPermission(user1, SystemPermission.ALTER_NAMESPACE);
-  }
-
-  /**
-   * This test makes sure that system-level iterators and constraints are ignored by the system namespace so that the metadata and root tables aren't affected
-   */
-  @Test
-  public void excludeSystemIterConst() throws Exception {
-    Connector c = accumulo.getConnector("root", secret);
-
-    c.instanceOperations().setProperty("table.iterator.scan.sum", "20," + SimpleFilter.class.getName());
-    assertTrue(c.instanceOperations().getSystemConfiguration().containsValue("20," + SimpleFilter.class.getName()));
-
-    assertTrue(checkTableNamespaceHasProp(c, Constants.DEFAULT_TABLE_NAMESPACE, "table.iterator.scan.sum", "20," + SimpleFilter.class.getName()));
-    assertTrue(!checkTableNamespaceHasProp(c, Constants.SYSTEM_TABLE_NAMESPACE, "table.iterator.scan.sum", "20," + SimpleFilter.class.getName()));
-    c.instanceOperations().removeProperty("table.iterator.scan.sum");
-
-    c.instanceOperations().setProperty("table.constraint.42", NumericValueConstraint.class.getName());
-    assertTrue(c.instanceOperations().getSystemConfiguration().containsValue(NumericValueConstraint.class.getName()));
-
-    assertTrue(checkTableNamespaceHasProp(c, Constants.DEFAULT_TABLE_NAMESPACE, "table.constraint.42", NumericValueConstraint.class.getName()));
-    assertTrue(!checkTableNamespaceHasProp(c, Constants.SYSTEM_TABLE_NAMESPACE, "table.constraint.42", NumericValueConstraint.class.getName()));
-    c.instanceOperations().removeProperty("table.constraint.42");
-  }
-
-  private boolean checkTableHasProp(Connector c, String t, String propKey, String propVal) throws AccumuloException, TableNotFoundException {
-    for (Entry<String,String> e : c.tableOperations().getProperties(t)) {
-      if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  private boolean checkTableNamespaceHasProp(Connector c, String n, String propKey, String propVal) throws AccumuloException, TableNamespaceNotFoundException {
-    for (Entry<String,String> e : c.tableNamespaceOperations().getProperties(n)) {
-      if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  public static class SimpleFilter extends Filter {
-    @Override
-    public boolean accept(Key k, Value v) {
-      if (k.getColumnFamily().toString().equals("a"))
-        return false;
-      return true;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/test/src/test/java/org/apache/accumulo/test/functional/PermissionsIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/PermissionsIT.java b/test/src/test/java/org/apache/accumulo/test/functional/PermissionsIT.java
index b05ad18..f835015 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/PermissionsIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/PermissionsIT.java
@@ -32,11 +32,11 @@ import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.NamespaceExistsException;
+import org.apache.accumulo.core.client.NamespaceNotEmptyException;
+import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.TableNamespaceExistsException;
-import org.apache.accumulo.core.client.TableNamespaceNotEmptyException;
-import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.security.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
@@ -108,9 +108,9 @@ public class PermissionsIT extends SimpleMacIT {
   }
 
   private static void testMissingSystemPermission(String tableNamePrefix, Connector root_conn, Connector test_user_conn, SystemPermission perm)
-      throws AccumuloException, TableExistsException, AccumuloSecurityException, TableNotFoundException, TableNamespaceExistsException,
-      TableNamespaceNotFoundException, TableNamespaceNotEmptyException {
-    String tableName, user, password = "password", tableNamespace;
+      throws AccumuloException, TableExistsException, AccumuloSecurityException, TableNotFoundException, NamespaceExistsException, NamespaceNotFoundException,
+      NamespaceNotEmptyException {
+    String tableName, user, password = "password", namespace;
     log.debug("Confirming that the lack of the " + perm + " permission properly restricts the user");
 
     // test permission prior to granting it
@@ -204,53 +204,53 @@ public class PermissionsIT extends SimpleMacIT {
         // test for system permission would go here
         break;
       case CREATE_NAMESPACE:
-        tableNamespace = "__CREATE_TABLE_NAMESPACE WITHOUT_PERM_TEST__";
+        namespace = "__CREATE_NAMESPACE_WITHOUT_PERM_TEST__";
         try {
-          test_user_conn.tableNamespaceOperations().create(tableNamespace);
-          throw new IllegalStateException("Should NOT be able to create a table namespace");
+          test_user_conn.namespaceOperations().create(namespace);
+          throw new IllegalStateException("Should NOT be able to create a namespace");
         } catch (AccumuloSecurityException e) {
-          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED || root_conn.tableNamespaceOperations().list().contains(tableNamespace))
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED || root_conn.namespaceOperations().list().contains(namespace))
             throw e;
         }
         break;
       case DROP_NAMESPACE:
-        tableNamespace = "__DROP_TABLE_NAMESPACE_WITHOUT_PERM_TEST__";
-        root_conn.tableNamespaceOperations().create(tableNamespace);
+        namespace = "__DROP_NAMESPACE_WITHOUT_PERM_TEST__";
+        root_conn.namespaceOperations().create(namespace);
         try {
-          test_user_conn.tableNamespaceOperations().delete(tableNamespace);
-          throw new IllegalStateException("Should NOT be able to delete a table namespace");
+          test_user_conn.namespaceOperations().delete(namespace);
+          throw new IllegalStateException("Should NOT be able to delete a namespace");
         } catch (AccumuloSecurityException e) {
-          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED || !root_conn.tableNamespaceOperations().list().contains(tableNamespace))
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED || !root_conn.namespaceOperations().list().contains(namespace))
             throw e;
         }
         break;
       case ALTER_NAMESPACE:
-        tableNamespace = "__ALTER_TABLE_NAMESPACE_WITHOUT_PERM_TEST__";
-        root_conn.tableNamespaceOperations().create(tableNamespace);
+        namespace = "__ALTER_NAMESPACE_WITHOUT_PERM_TEST__";
+        root_conn.namespaceOperations().create(namespace);
         try {
-          test_user_conn.tableNamespaceOperations().setProperty(tableNamespace, Property.TABLE_BLOOM_ERRORRATE.getKey(), "003.14159%");
-          throw new IllegalStateException("Should NOT be able to set a table namespace property");
+          test_user_conn.namespaceOperations().setProperty(namespace, Property.TABLE_BLOOM_ERRORRATE.getKey(), "003.14159%");
+          throw new IllegalStateException("Should NOT be able to set a namespace property");
         } catch (AccumuloSecurityException e) {
           if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED
-              || map(root_conn.tableNamespaceOperations().getProperties(tableNamespace)).get(Property.TABLE_BLOOM_ERRORRATE.getKey()).equals("003.14159%"))
+              || map(root_conn.namespaceOperations().getProperties(namespace)).get(Property.TABLE_BLOOM_ERRORRATE.getKey()).equals("003.14159%"))
             throw e;
         }
-        root_conn.tableNamespaceOperations().setProperty(tableNamespace, Property.TABLE_BLOOM_ERRORRATE.getKey(), "003.14159%");
+        root_conn.namespaceOperations().setProperty(namespace, Property.TABLE_BLOOM_ERRORRATE.getKey(), "003.14159%");
         try {
-          test_user_conn.tableNamespaceOperations().removeProperty(tableNamespace, Property.TABLE_BLOOM_ERRORRATE.getKey());
-          throw new IllegalStateException("Should NOT be able to remove a table namespace property");
+          test_user_conn.namespaceOperations().removeProperty(namespace, Property.TABLE_BLOOM_ERRORRATE.getKey());
+          throw new IllegalStateException("Should NOT be able to remove a namespace property");
         } catch (AccumuloSecurityException e) {
           if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED
-              || !map(root_conn.tableNamespaceOperations().getProperties(tableNamespace)).get(Property.TABLE_BLOOM_ERRORRATE.getKey()).equals("003.14159%"))
+              || !map(root_conn.namespaceOperations().getProperties(namespace)).get(Property.TABLE_BLOOM_ERRORRATE.getKey()).equals("003.14159%"))
             throw e;
         }
-        String tableNamespace2 = tableNamespace + "2";
+        String namespace2 = namespace + "2";
         try {
-          test_user_conn.tableNamespaceOperations().rename(tableNamespace, tableNamespace2);
-          throw new IllegalStateException("Should NOT be able to rename a table namespace");
+          test_user_conn.namespaceOperations().rename(namespace, namespace2);
+          throw new IllegalStateException("Should NOT be able to rename a namespace");
         } catch (AccumuloSecurityException e) {
-          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED || !root_conn.tableNamespaceOperations().list().contains(tableNamespace)
-              || root_conn.tableNamespaceOperations().list().contains(tableNamespace2))
+          if (e.getSecurityErrorCode() != SecurityErrorCode.PERMISSION_DENIED || !root_conn.namespaceOperations().list().contains(namespace)
+              || root_conn.namespaceOperations().list().contains(namespace2))
             throw e;
         }
         break;
@@ -260,9 +260,9 @@ public class PermissionsIT extends SimpleMacIT {
   }
 
   private static void testGrantedSystemPermission(String tableNamePrefix, Connector root_conn, Connector test_user_conn, SystemPermission perm)
-      throws AccumuloException, AccumuloSecurityException, TableNotFoundException, TableExistsException, TableNamespaceExistsException,
-      TableNamespaceNotFoundException, TableNamespaceNotEmptyException {
-    String tableName, user, password = "password", tableNamespace;
+      throws AccumuloException, AccumuloSecurityException, TableNotFoundException, TableExistsException, NamespaceExistsException, NamespaceNotFoundException,
+      NamespaceNotEmptyException {
+    String tableName, user, password = "password", namespace;
     log.debug("Confirming that the presence of the " + perm + " permission properly permits the user");
 
     // test permission after granting it
@@ -320,32 +320,32 @@ public class PermissionsIT extends SimpleMacIT {
         // test for system permission would go here
         break;
       case CREATE_NAMESPACE:
-        tableNamespace = "__CREATE_TABLE_NAMESPACE_WITH_PERM_TEST__";
-        test_user_conn.tableNamespaceOperations().create(tableNamespace);
-        if (!root_conn.tableNamespaceOperations().list().contains(tableNamespace))
-          throw new IllegalStateException("Should be able to create a table namespace");
+        namespace = "__CREATE_NAMESPACE_WITH_PERM_TEST__";
+        test_user_conn.namespaceOperations().create(namespace);
+        if (!root_conn.namespaceOperations().list().contains(namespace))
+          throw new IllegalStateException("Should be able to create a namespace");
         break;
       case DROP_NAMESPACE:
-        tableNamespace = "__DROP_TABLE_NAMESPACE_WITH_PERM_TEST__";
-        root_conn.tableNamespaceOperations().create(tableNamespace);
-        test_user_conn.tableNamespaceOperations().delete(tableNamespace);
-        if (root_conn.tableNamespaceOperations().list().contains(tableNamespace))
-          throw new IllegalStateException("Should be able to delete a table namespace");
+        namespace = "__DROP_NAMESPACE_WITH_PERM_TEST__";
+        root_conn.namespaceOperations().create(namespace);
+        test_user_conn.namespaceOperations().delete(namespace);
+        if (root_conn.namespaceOperations().list().contains(namespace))
+          throw new IllegalStateException("Should be able to delete a namespace");
         break;
       case ALTER_NAMESPACE:
-        tableNamespace = "__ALTER_TABLE_NAMESPACE_WITH_PERM_TEST__";
-        String tableNamespace2 = tableNamespace + "2";
-        root_conn.tableNamespaceOperations().create(tableNamespace);
-        test_user_conn.tableNamespaceOperations().setProperty(tableNamespace, Property.TABLE_BLOOM_ERRORRATE.getKey(), "003.14159%");
-        Map<String,String> propies = map(root_conn.tableNamespaceOperations().getProperties(tableNamespace));
+        namespace = "__ALTER_NAMESPACE_WITH_PERM_TEST__";
+        String namespace2 = namespace + "2";
+        root_conn.namespaceOperations().create(namespace);
+        test_user_conn.namespaceOperations().setProperty(namespace, Property.TABLE_BLOOM_ERRORRATE.getKey(), "003.14159%");
+        Map<String,String> propies = map(root_conn.namespaceOperations().getProperties(namespace));
         if (!propies.get(Property.TABLE_BLOOM_ERRORRATE.getKey()).equals("003.14159%"))
           throw new IllegalStateException("Should be able to set a table property");
-        test_user_conn.tableNamespaceOperations().removeProperty(tableNamespace, Property.TABLE_BLOOM_ERRORRATE.getKey());
-        propies = map(root_conn.tableNamespaceOperations().getProperties(tableNamespace));
+        test_user_conn.namespaceOperations().removeProperty(namespace, Property.TABLE_BLOOM_ERRORRATE.getKey());
+        propies = map(root_conn.namespaceOperations().getProperties(namespace));
         if (propies.get(Property.TABLE_BLOOM_ERRORRATE.getKey()).equals("003.14159%"))
           throw new IllegalStateException("Should be able to remove a table property");
-        test_user_conn.tableNamespaceOperations().rename(tableNamespace, tableNamespace2);
-        if (root_conn.tableNamespaceOperations().list().contains(tableNamespace) || !root_conn.tableNamespaceOperations().list().contains(tableNamespace2))
+        test_user_conn.namespaceOperations().rename(namespace, namespace2);
+        if (root_conn.namespaceOperations().list().contains(namespace) || !root_conn.namespaceOperations().list().contains(namespace2))
           throw new IllegalStateException("Should be able to rename a table");
         break;
       default:

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/test/system/randomwalk/conf/modules/Concurrent.xml
----------------------------------------------------------------------
diff --git a/test/system/randomwalk/conf/modules/Concurrent.xml b/test/system/randomwalk/conf/modules/Concurrent.xml
index 3768878..dbe3d70 100644
--- a/test/system/randomwalk/conf/modules/Concurrent.xml
+++ b/test/system/randomwalk/conf/modules/Concurrent.xml
@@ -49,10 +49,10 @@
   <edge id="ct.StartAll" weight="1000"/>
   <edge id="ct.Shutdown" weight="10"/>
   <edge id="ct.Config" weight="1000"/>
-  <edge id="ct.CreateTableNamespace" weight="1000"/>
-  <edge id="ct.DeleteTableNamespace" weight="100"/>
-  <edge id="ct.RenameTableNamespace" weight="100"/>
-  <edge id="ct.OfflineTableNamespace" weight="100"/>
+  <edge id="ct.CreateNamespace" weight="1000"/>
+  <edge id="ct.DeleteNamespace" weight="100"/>
+  <edge id="ct.RenameNamespace" weight="100"/>
+  <edge id="ct.OfflineNamespace" weight="100"/>
   <edge id="ct.Apocalypse" weight="10"/>
   <edge id="END" weight="1"/>
 </node>
@@ -167,19 +167,19 @@
   <edge id="ct.StartAll" weight="1"/>
 </node>
 
-<node id="ct.CreateTableNamespace">
+<node id="ct.CreateNamespace">
   <edge id="ct.StartAll" weight="1"/>
 </node>
 
-<node id="ct.DeleteTableNamespace">
+<node id="ct.DeleteNamespace">
   <edge id="ct.StartAll" weight="1"/>
 </node>
 
-<node id="ct.RenameTableNamespace">
+<node id="ct.RenameNamespace">
   <edge id="ct.StartAll" weight="1"/>
 </node>
 
-<node id="ct.OfflineTableNamespace">
+<node id="ct.OfflineNamespace">
   <edge id="ct.StartAll" weight="1"/>
 </node>