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

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

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;
   }
-  
+
 }