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 2014/01/04 04:01:02 UTC

git commit: ACCUMULO-2079 Prevent table creation in system namespace

Updated Branches:
  refs/heads/1.6.0-SNAPSHOT c44fa4f37 -> 764884e75


ACCUMULO-2079 Prevent table creation in system namespace

  Consolidated code that checked for preconditions in fate operations,
  and made the checks more explicit and readable, which common exception
  handling. This does not address client side exceptions, but does
  ensure that descriptive messages are sent back to the client.


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

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: 764884e753ba4ff8bab071f622e0bfbb6c8fc6a4
Parents: c44fa4f
Author: Christopher Tubbs <ct...@apache.org>
Authored: Fri Jan 3 21:58:47 2014 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Fri Jan 3 21:58:47 2014 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/core/Constants.java     |   2 -
 .../core/client/admin/TableOperations.java      |   2 +-
 .../accumulo/core/client/impl/Namespaces.java   |  41 ++++
 .../accumulo/core/client/impl/Tables.java       |  74 ++++++-
 .../client/mock/MockNamespaceOperations.java    |   4 +-
 .../core/client/mock/MockTableOperations.java   |   3 +-
 .../accumulo/core/util/ArgumentChecker.java     |  33 +++
 .../util/shell/commands/CreateTableCommand.java |   4 +-
 .../server/client/ClientServiceHandler.java     |  10 +-
 .../accumulo/master/FateServiceHandler.java     | 215 +++++++++----------
 .../master/MasterClientServiceHandler.java      |   4 +-
 .../org/apache/accumulo/test/NamespacesIT.java  |  37 ++--
 12 files changed, 285 insertions(+), 144 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/764884e7/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 c151abb..7dee32c 100644
--- a/core/src/main/java/org/apache/accumulo/core/Constants.java
+++ b/core/src/main/java/org/apache/accumulo/core/Constants.java
@@ -101,8 +101,6 @@ public class Constants {
 
   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_NAMESPACE_REGEX = "^\\w+$";
   public static final String MAPFILE_EXTENSION = "map";
   public static final String GENERATED_TABLET_DIRECTORY_PREFIX = "t-";
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/764884e7/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 97066c4..4f6c315 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
@@ -344,7 +344,7 @@ public interface TableOperations {
    * @param oldTableName
    *          the old table name
    * @param newTableName
-   *          the new table name
+   *          the new table name, which must be either unqualified (no namespace) or in the same namespace as the oldTableName
    * @throws AccumuloException
    *           if a general error occurs
    * @throws AccumuloSecurityException

http://git-wip-us.apache.org/repos/asf/accumulo/blob/764884e7/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 1ca5dee..c134c04 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
@@ -26,10 +26,50 @@ 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.util.ArgumentChecker.Validator;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 
 public class Namespaces {
+  public static final String VALID_NAME_REGEX = "^\\w*$";
+  public static final Validator<String> VALID_NAME = new Validator<String>() {
+    @Override
+    public boolean isValid(String namespace) {
+      return namespace != null && namespace.matches(VALID_NAME_REGEX);
+    }
+
+    @Override
+    public String invalidMessage(String namespace) {
+      if (namespace == null)
+        return "Namespace cannot be null";
+      return "Namespaces must only contain word characters (letters, digits, and underscores): " + namespace;
+    }
+  };
+
+  public static final Validator<String> NOT_DEFAULT = new Validator<String>() {
+    @Override
+    public boolean isValid(String namespace) {
+      return !Namespaces.DEFAULT_NAMESPACE.equals(namespace);
+    }
+
+    @Override
+    public String invalidMessage(String namespace) {
+      return "Namespace cannot be the reserved empty namespace";
+    }
+  };
+
+  public static final Validator<String> NOT_ACCUMULO = new Validator<String>() {
+    @Override
+    public boolean isValid(String namespace) {
+      return !Namespaces.ACCUMULO_NAMESPACE.equals(namespace);
+    }
+
+    @Override
+    public String invalidMessage(String namespace) {
+      return "Namespace cannot be the reserved namespace, " + Namespaces.ACCUMULO_NAMESPACE;
+    }
+  };
+
   private static SecurityPermission TABLES_PERMISSION = new SecurityPermission("tablesPermission");
 
   public static final String DEFAULT_NAMESPACE_ID = "+default";
@@ -109,4 +149,5 @@ public class Namespaces {
         names.add(name);
     return names;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/764884e7/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 279453b..f3b5a8d 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
@@ -30,13 +30,84 @@ 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.ArgumentChecker;
+import org.apache.accumulo.core.util.ArgumentChecker.Validator;
 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;
 
 public class Tables {
+  public static final String VALID_NAME_REGEX = "^(\\w+\\.)?(\\w+)$";
+  public static final String VALID_ID_REGEX = "^([a-z0-9]+)$"; // BigDecimal base36
+  public static final Validator<String> VALID_NAME = new Validator<String>() {
+    @Override
+    public boolean isValid(String tableName) {
+      return tableName != null && tableName.matches(VALID_NAME_REGEX);
+    }
+
+    @Override
+    public String invalidMessage(String tableName) {
+      if (tableName == null)
+        return "Table name cannot be null";
+      return "Table names must only contain word characters (letters, digits, and underscores): " + tableName;
+    }
+  };
+
+  public static final Validator<String> VALID_ID = new Validator<String>() {
+    @Override
+    public boolean isValid(String tableId) {
+      return tableId != null && (RootTable.ID.equals(tableId) || MetadataTable.ID.equals(tableId) || tableId.matches(VALID_ID_REGEX));
+    }
+
+    @Override
+    public String invalidMessage(String tableId) {
+      if (tableId == null)
+        return "Table id cannot be null";
+      return "Table IDs are base-36 numbers, represented with lowercase alphanumeric digits: " + tableId;
+    }
+  };
+
+  public static final Validator<String> NOT_SYSTEM = new Validator<String>() {
+
+    @Override
+    public boolean isValid(String tableName) {
+      return !Namespaces.ACCUMULO_NAMESPACE.equals(Tables.qualify(tableName).getFirst());
+    }
+
+    @Override
+    public String invalidMessage(String tableName) {
+      return "Table cannot be in the " + Namespaces.ACCUMULO_NAMESPACE + " namespace";
+    }
+  };
+
+  public static final Validator<String> NOT_ROOT = new Validator<String>() {
+
+    @Override
+    public boolean isValid(String tableName) {
+      return !RootTable.NAME.equals(tableName);
+    }
+
+    @Override
+    public String invalidMessage(String tableName) {
+      return "Table cannot be the " + RootTable.NAME + "(Id: " + RootTable.ID + ") table";
+    }
+  };
+
+  public static final Validator<String> NOT_ROOT_ID = new Validator<String>() {
+
+    @Override
+    public boolean isValid(String tableId) {
+      return !RootTable.ID.equals(tableId);
+    }
+
+    @Override
+    public String invalidMessage(String tableId) {
+      return "Table cannot be the " + RootTable.NAME + "(Id: " + RootTable.ID + ") table";
+    }
+  };
+
   private static SecurityPermission TABLES_PERMISSION = new SecurityPermission("tablesPermission");
   private static AtomicLong cacheResetCount = new AtomicLong(0);
   private static final Logger log = Logger.getLogger(Tables.class);
@@ -203,7 +274,7 @@ public class Tables {
   }
 
   public static Pair<String,String> qualify(String tableName, String defaultNamespace) {
-    if (!tableName.matches(Constants.VALID_TABLE_NAME_REGEX))
+    if (!tableName.matches(Tables.VALID_NAME_REGEX))
       throw new IllegalArgumentException("Improper table name format");
     if (MetadataTable.OLD_NAME.equals(tableName))
       tableName = MetadataTable.NAME;
@@ -238,4 +309,5 @@ public class Tables {
 
     return new String(n, Constants.UTF8);
   }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/764884e7/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 eb45d2d..9f0594a 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
@@ -22,13 +22,13 @@ 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.admin.NamespaceOperationsHelper;
+import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
 
@@ -54,7 +54,7 @@ public class MockNamespaceOperations extends NamespaceOperationsHelper {
 
   @Override
   public void create(String namespace) throws AccumuloException, AccumuloSecurityException, NamespaceExistsException {
-    if (!namespace.matches(Constants.VALID_NAMESPACE_REGEX))
+    if (!namespace.matches(Namespaces.VALID_NAME_REGEX))
       throw new IllegalArgumentException();
 
     if (exists(namespace))

http://git-wip-us.apache.org/repos/asf/accumulo/blob/764884e7/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 8c73c0f..cc6bce7 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
@@ -30,7 +30,6 @@ import java.util.Set;
 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.IteratorSetting;
@@ -98,7 +97,7 @@ public class MockTableOperations extends TableOperationsHelper {
   @Override
   public void create(String tableName, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException {
     String namespace = Tables.qualify(tableName).getFirst();
-    if (!tableName.matches(Constants.VALID_TABLE_NAME_REGEX)) {
+    if (!tableName.matches(Tables.VALID_NAME_REGEX)) {
       throw new IllegalArgumentException();
     }
     if (exists(tableName))

http://git-wip-us.apache.org/repos/asf/accumulo/blob/764884e7/core/src/main/java/org/apache/accumulo/core/util/ArgumentChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/ArgumentChecker.java b/core/src/main/java/org/apache/accumulo/core/util/ArgumentChecker.java
index 20a1373..1ba133a 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/ArgumentChecker.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/ArgumentChecker.java
@@ -65,4 +65,37 @@ public class ArgumentChecker {
     if (!arg.iterator().hasNext())
       throw new IllegalArgumentException("Argument should not be empty");
   }
+
+  public static abstract class Validator<T> {
+
+    public final T validate(final T argument) throws IllegalArgumentException {
+      if (!isValid(argument))
+        throw new IllegalArgumentException(invalidMessage(argument));
+      return argument;
+    }
+
+    public abstract boolean isValid(final T argument);
+
+    public abstract String invalidMessage(final T argument);
+
+    public Validator<T> and(final Validator<T> other) {
+      if (other == null)
+        return this;
+      final Validator<T> mine = this;
+      return new Validator<T>() {
+
+        @Override
+        public boolean isValid(T argument) {
+          return mine.isValid(argument) && other.isValid(argument);
+        }
+
+        @Override
+        public String invalidMessage(T argument) {
+          return (mine.isValid(argument) ? other : mine).invalidMessage(argument);
+        }
+
+      };
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/764884e7/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 b0e7432..96d8b13 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
@@ -25,12 +25,12 @@ import java.util.Set;
 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.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;
@@ -62,7 +62,7 @@ public class CreateTableCommand extends Command {
 
     final String testTableName = cl.getArgs()[0];
 
-    if (!testTableName.matches(Constants.VALID_TABLE_NAME_REGEX)) {
+    if (!testTableName.matches(Tables.VALID_NAME_REGEX)) {
       shellState.getReader().println("Only letters, numbers and underscores are allowed for use in table names.");
       throw new IllegalArgumentException();
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/764884e7/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 4df7c88..ed8cbc6 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
@@ -92,7 +92,7 @@ public class ClientServiceHandler implements ClientService.Iface {
     throw new ThriftTableOperationException(null, tableName, operation, reason, null);
   }
 
-  protected String checkNamespaceId(String namespace, TableOperation operation) throws ThriftTableOperationException {
+  public static String checkNamespaceId(Instance instance, String namespace, TableOperation operation) throws ThriftTableOperationException {
     String namespaceId = Namespaces.getNameToIdMap(instance).get(namespace);
     if (namespaceId == null) {
       // maybe the namespace exists, but the cache was not updated yet... so try to clear the cache and check again
@@ -189,7 +189,7 @@ public class ClientServiceHandler implements ClientService.Iface {
   @Override
   public void grantNamespacePermission(TInfo tinfo, TCredentials credentials, String user, String ns, byte permission) throws ThriftSecurityException,
       ThriftTableOperationException {
-    String namespaceId = checkNamespaceId(ns, TableOperation.PERMISSION);
+    String namespaceId = checkNamespaceId(instance, ns, TableOperation.PERMISSION);
     security.grantNamespacePermission(credentials, user, namespaceId, NamespacePermission.getPermissionById(permission));
   }
 
@@ -220,14 +220,14 @@ public class ClientServiceHandler implements ClientService.Iface {
   @Override
   public boolean hasNamespacePermission(TInfo tinfo, TCredentials credentials, String user, String ns, byte perm) throws ThriftSecurityException,
       ThriftTableOperationException {
-    String namespaceId = checkNamespaceId(ns, TableOperation.PERMISSION);
+    String namespaceId = checkNamespaceId(instance, ns, TableOperation.PERMISSION);
     return security.hasNamespacePermission(credentials, user, namespaceId, NamespacePermission.getPermissionById(perm));
   }
 
   @Override
   public void revokeNamespacePermission(TInfo tinfo, TCredentials credentials, String user, String ns, byte permission) throws ThriftSecurityException,
       ThriftTableOperationException {
-    String namespaceId = checkNamespaceId(ns, TableOperation.PERMISSION);
+    String namespaceId = checkNamespaceId(instance, ns, TableOperation.PERMISSION);
     security.revokeNamespacePermission(credentials, user, namespaceId, NamespacePermission.getPermissionById(permission));
   }
 
@@ -361,7 +361,7 @@ public class ClientServiceHandler implements ClientService.Iface {
 
     security.authenticateUser(credentials, credentials);
 
-    String namespaceId = checkNamespaceId(ns, null);
+    String namespaceId = checkNamespaceId(instance, ns, null);
 
     ClassLoader loader = getClass().getClassLoader();
     Class<?> shouldMatch;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/764884e7/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java b/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
index e7f1dde..afcda86 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
@@ -24,7 +24,6 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.admin.TableOperationsImpl;
@@ -39,9 +38,8 @@ import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException
 import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.master.thrift.FateOperation;
 import org.apache.accumulo.core.master.thrift.FateService;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.core.util.ArgumentChecker.Validator;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.fate.TStore.TStatus;
 import org.apache.accumulo.master.tableOps.BulkImport;
@@ -91,24 +89,21 @@ class FateServiceHandler implements FateService.Iface {
 
     switch (op) {
       case NAMESPACE_CREATE: {
-        String namespace = ByteBufferUtil.toString(arguments.get(0));
+        TableOperation tableOp = TableOperation.CREATE;
+        String namespace = validateNamespaceArgument(arguments.get(0), tableOp, null);
+
         if (!master.security.canCreateNamespace(c, namespace))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-        checkNotSystemNamespace(namespace, TableOperation.CREATE);
-        checkNamespaceName(namespace, TableOperation.CREATE);
         master.fate.seedTransaction(opid, new TraceRepo<Master>(new CreateNamespace(c.getPrincipal(), namespace, options)), autoCleanup);
         break;
       }
       case NAMESPACE_RENAME: {
+        TableOperation tableOp = TableOperation.RENAME;
+        String oldName = validateNamespaceArgument(arguments.get(0), tableOp, Namespaces.NOT_DEFAULT.and(Namespaces.NOT_ACCUMULO));
+        String newName = validateNamespaceArgument(arguments.get(1), tableOp, null);
 
-        String oldName = ByteBufferUtil.toString(arguments.get(0));
-        String newName = ByteBufferUtil.toString(arguments.get(1));
-        String namespaceId = checkNamespaceId(oldName, TableOperation.RENAME);
-
-        checkNotSystemNamespace(oldName, TableOperation.RENAME);
-        checkNotSystemNamespace(newName, TableOperation.RENAME);
-        checkNamespaceName(newName, TableOperation.RENAME);
+        String namespaceId = ClientServiceHandler.checkNamespaceId(master.getInstance(), oldName, tableOp);
         if (!master.security.canRenameNamespace(c, namespaceId, oldName, newName))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
@@ -116,9 +111,10 @@ class FateServiceHandler implements FateService.Iface {
         break;
       }
       case NAMESPACE_DELETE: {
-        String namespace = ByteBufferUtil.toString(arguments.get(0));
-        checkNotSystemNamespace(namespace, TableOperation.DELETE);
-        String namespaceId = checkNamespaceId(namespace, TableOperation.DELETE);
+        TableOperation tableOp = TableOperation.DELETE;
+        String namespace = validateNamespaceArgument(arguments.get(0), tableOp, Namespaces.NOT_DEFAULT.and(Namespaces.NOT_ACCUMULO));
+
+        String namespaceId = ClientServiceHandler.checkNamespaceId(master.getInstance(), namespace, tableOp);
         if (!master.security.canDeleteNamespace(c, namespaceId))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
@@ -126,45 +122,57 @@ class FateServiceHandler implements FateService.Iface {
         break;
       }
       case TABLE_CREATE: {
-        String tableName = ByteBufferUtil.toString(arguments.get(0));
+        TableOperation tableOp = TableOperation.CREATE;
+        String tableName = validateTableNameArgument(arguments.get(0), tableOp, Tables.NOT_SYSTEM);
+        TimeType timeType = TimeType.valueOf(ByteBufferUtil.toString(arguments.get(1)));
+
         if (!master.security.canCreateTable(c, tableName))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-        checkNotMetadataTable(tableName, TableOperation.DELETE);
-        checkTableName(tableName, TableOperation.CREATE);
-
-        TimeType timeType = TimeType.valueOf(ByteBufferUtil.toString(arguments.get(1)));
 
         try {
           master.fate.seedTransaction(opid, new TraceRepo<Master>(new CreateTable(c.getPrincipal(), tableName, timeType, options)), autoCleanup);
         } catch (NamespaceNotFoundException e) {
-          throw new ThriftTableOperationException(null, tableName, TableOperation.CREATE, TableOperationExceptionType.NAMESPACE_NOTFOUND, "");
+          throw new ThriftTableOperationException(null, tableName, tableOp, TableOperationExceptionType.NAMESPACE_NOTFOUND, "");
         }
         break;
       }
       case TABLE_RENAME: {
-        String oldTableName = ByteBufferUtil.toString(arguments.get(0));
-        String newTableName = ByteBufferUtil.toString(arguments.get(1));
+        TableOperation tableOp = TableOperation.RENAME;
+        final String oldTableName = validateTableNameArgument(arguments.get(0), tableOp, Tables.NOT_SYSTEM);
+        String newTableName = validateTableNameArgument(arguments.get(1), tableOp, new Validator<String>() {
+
+          @Override
+          public boolean isValid(String argument) {
+            // verify they are in the same namespace
+            String oldNamespace = Tables.qualify(oldTableName).getFirst();
+            return oldNamespace.equals(Tables.qualify(argument, oldNamespace).getFirst());
+          }
+
+          @Override
+          public String invalidMessage(String argument) {
+            return "Cannot move tables to a new namespace by renaming. The namespace for " + oldTableName + " does not match " + argument;
+          }
+
+        });
+
+        String tableId = ClientServiceHandler.checkTableId(master.getInstance(), oldTableName, tableOp);
 
-        String tableId = ClientServiceHandler.checkTableId(master.getInstance(), oldTableName, TableOperation.RENAME);
-        checkNotMetadataTable(oldTableName, TableOperation.RENAME);
-        checkNotMetadataTable(newTableName, TableOperation.RENAME);
-        checkTableName(newTableName, TableOperation.RENAME);
         if (!master.security.canRenameTable(c, tableId, oldTableName, newTableName))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
         try {
           master.fate.seedTransaction(opid, new TraceRepo<Master>(new RenameTable(tableId, oldTableName, newTableName)), autoCleanup);
         } catch (NamespaceNotFoundException e) {
-          throw new ThriftTableOperationException(null, oldTableName, TableOperation.RENAME, TableOperationExceptionType.NAMESPACE_NOTFOUND, "");
+          throw new ThriftTableOperationException(null, oldTableName, tableOp, TableOperationExceptionType.NAMESPACE_NOTFOUND, "");
         }
 
         break;
       }
       case TABLE_CLONE: {
-        String srcTableId = ByteBufferUtil.toString(arguments.get(0));
-        String tableName = ByteBufferUtil.toString(arguments.get(1));
-        checkNotMetadataTable(tableName, TableOperation.CLONE);
-        checkTableName(tableName, TableOperation.CLONE);
+        TableOperation tableOp = TableOperation.CLONE;
+        String srcTableId = validateTableIdArgument(arguments.get(0), tableOp, null);
+        String tableName = validateTableNameArgument(arguments.get(1), tableOp, Tables.NOT_SYSTEM);
+
         if (!master.security.canCloneTable(c, srcTableId, tableName))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
@@ -178,7 +186,7 @@ class FateServiceHandler implements FateService.Iface {
           }
 
           if (!TablePropUtil.isPropertyValid(entry.getKey(), entry.getValue())) {
-            throw new ThriftTableOperationException(null, tableName, TableOperation.CLONE, TableOperationExceptionType.OTHER, "Property or value not valid "
+            throw new ThriftTableOperationException(null, tableName, tableOp, TableOperationExceptionType.OTHER, "Property or value not valid "
                 + entry.getKey() + "=" + entry.getValue());
           }
 
@@ -191,55 +199,56 @@ class FateServiceHandler implements FateService.Iface {
         break;
       }
       case TABLE_DELETE: {
-        String tableName = ByteBufferUtil.toString(arguments.get(0));
-        final String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, TableOperation.DELETE);
-        checkNotMetadataTable(tableName, TableOperation.DELETE);
+        TableOperation tableOp = TableOperation.DELETE;
+        String tableName = validateTableNameArgument(arguments.get(0), tableOp, Tables.NOT_SYSTEM);
+
+        final String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, tableOp);
         if (!master.security.canDeleteTable(c, tableId))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
         master.fate.seedTransaction(opid, new TraceRepo<Master>(new DeleteTable(tableId)), autoCleanup);
         break;
       }
       case TABLE_ONLINE: {
-        final String tableId = ByteBufferUtil.toString(arguments.get(0));
-        checkNotRootID(tableId, TableOperation.ONLINE);
+        TableOperation tableOp = TableOperation.ONLINE;
+        final String tableId = validateTableIdArgument(arguments.get(0), tableOp, Tables.NOT_ROOT_ID);
 
         if (!master.security.canOnlineOfflineTable(c, tableId, op))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-        master.fate.seedTransaction(opid, new TraceRepo<Master>(new ChangeTableState(tableId, TableOperation.ONLINE)), autoCleanup);
+        master.fate.seedTransaction(opid, new TraceRepo<Master>(new ChangeTableState(tableId, tableOp)), autoCleanup);
         break;
       }
       case TABLE_OFFLINE: {
-        final String tableId = ByteBufferUtil.toString(arguments.get(0));
-        checkNotRootID(tableId, TableOperation.OFFLINE);
+        TableOperation tableOp = TableOperation.OFFLINE;
+        final String tableId = validateTableIdArgument(arguments.get(0), tableOp, Tables.NOT_ROOT_ID);
 
         if (!master.security.canOnlineOfflineTable(c, tableId, op))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-        master.fate.seedTransaction(opid, new TraceRepo<Master>(new ChangeTableState(tableId, TableOperation.OFFLINE)), autoCleanup);
+        master.fate.seedTransaction(opid, new TraceRepo<Master>(new ChangeTableState(tableId, tableOp)), autoCleanup);
         break;
       }
       case TABLE_MERGE: {
-        String tableName = ByteBufferUtil.toString(arguments.get(0));
+        TableOperation tableOp = TableOperation.MERGE;
+        String tableName = validateTableNameArgument(arguments.get(0), tableOp, null);
         Text startRow = ByteBufferUtil.toText(arguments.get(1));
         Text endRow = ByteBufferUtil.toText(arguments.get(2));
-        final String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, TableOperation.MERGE);
-        Master.log.debug("Creating merge op: " + tableId + " " + startRow + " " + endRow);
 
+        final String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, tableOp);
         if (!master.security.canMerge(c, tableId))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
+        Master.log.debug("Creating merge op: " + tableId + " " + startRow + " " + endRow);
         master.fate.seedTransaction(opid, new TraceRepo<Master>(new TableRangeOp(MergeInfo.Operation.MERGE, tableId, startRow, endRow)), autoCleanup);
         break;
       }
       case TABLE_DELETE_RANGE: {
-        String tableName = ByteBufferUtil.toString(arguments.get(0));
+        TableOperation tableOp = TableOperation.DELETE_RANGE;
+        String tableName = validateTableNameArgument(arguments.get(0), tableOp, Tables.NOT_SYSTEM);
         Text startRow = ByteBufferUtil.toText(arguments.get(1));
         Text endRow = ByteBufferUtil.toText(arguments.get(2));
 
-        final String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, TableOperation.DELETE_RANGE);
-        checkNotMetadataTable(tableName, TableOperation.DELETE_RANGE);
-
+        final String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, tableOp);
         if (!master.security.canDeleteRange(c, tableId, tableName, startRow, endRow))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
@@ -247,14 +256,13 @@ class FateServiceHandler implements FateService.Iface {
         break;
       }
       case TABLE_BULK_IMPORT: {
-        String tableName = ByteBufferUtil.toString(arguments.get(0));
+        TableOperation tableOp = TableOperation.BULK_IMPORT;
+        String tableName = validateTableNameArgument(arguments.get(0), tableOp, Tables.NOT_SYSTEM);
         String dir = ByteBufferUtil.toString(arguments.get(1));
         String failDir = ByteBufferUtil.toString(arguments.get(2));
         boolean setTime = Boolean.parseBoolean(ByteBufferUtil.toString(arguments.get(3)));
 
-        final String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, TableOperation.BULK_IMPORT);
-        checkNotMetadataTable(tableName, TableOperation.BULK_IMPORT);
-
+        final String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, tableOp);
         if (!master.security.canBulkImport(c, tableId, tableName, dir, failDir))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
@@ -262,7 +270,8 @@ class FateServiceHandler implements FateService.Iface {
         break;
       }
       case TABLE_COMPACT: {
-        String tableId = ByteBufferUtil.toString(arguments.get(0));
+        TableOperation tableOp = TableOperation.COMPACT;
+        String tableId = validateTableIdArgument(arguments.get(0), tableOp, null);
         byte[] startRow = ByteBufferUtil.toBytes(arguments.get(1));
         byte[] endRow = ByteBufferUtil.toBytes(arguments.get(2));
         List<IteratorSetting> iterators = IteratorUtil.decodeIteratorSettings(ByteBufferUtil.toBytes(arguments.get(3)));
@@ -274,7 +283,8 @@ class FateServiceHandler implements FateService.Iface {
         break;
       }
       case TABLE_CANCEL_COMPACT: {
-        String tableId = ByteBufferUtil.toString(arguments.get(0));
+        TableOperation tableOp = TableOperation.COMPACT_CANCEL;
+        String tableId = validateTableIdArgument(arguments.get(0), tableOp, null);
 
         if (!master.security.canCompact(c, tableId))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
@@ -283,41 +293,35 @@ class FateServiceHandler implements FateService.Iface {
         break;
       }
       case TABLE_IMPORT: {
-        String tableName = ByteBufferUtil.toString(arguments.get(0));
+        TableOperation tableOp = TableOperation.IMPORT;
+        String tableName = validateTableNameArgument(arguments.get(0), tableOp, Tables.NOT_SYSTEM);
         String exportDir = ByteBufferUtil.toString(arguments.get(1));
 
         if (!master.security.canImport(c, tableName, exportDir))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-        checkNotMetadataTable(tableName, TableOperation.IMPORT);
-        checkTableName(tableName, TableOperation.CREATE);
-
         try {
           master.fate.seedTransaction(opid, new TraceRepo<Master>(new ImportTable(c.getPrincipal(), tableName, exportDir)), autoCleanup);
         } catch (NamespaceNotFoundException e) {
-          throw new ThriftTableOperationException(null, tableName, TableOperation.IMPORT, TableOperationExceptionType.NAMESPACE_NOTFOUND, "");
+          throw new ThriftTableOperationException(null, tableName, tableOp, TableOperationExceptionType.NAMESPACE_NOTFOUND, "");
         }
         break;
       }
       case TABLE_EXPORT: {
-        String tableName = ByteBufferUtil.toString(arguments.get(0));
+        TableOperation tableOp = TableOperation.EXPORT;
+        String tableName = validateTableNameArgument(arguments.get(0), tableOp, Tables.NOT_SYSTEM);
         String exportDir = ByteBufferUtil.toString(arguments.get(1));
 
-        String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, TableOperation.EXPORT);
-
+        String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, tableOp);
         if (!master.security.canExport(c, tableId, tableName, exportDir))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-        checkNotMetadataTable(tableName, TableOperation.EXPORT);
-
         master.fate.seedTransaction(opid, new TraceRepo<Master>(new ExportTable(tableName, tableId, exportDir)), autoCleanup);
         break;
       }
-
       default:
         throw new UnsupportedOperationException();
     }
-
   }
 
   @Override
@@ -349,64 +353,45 @@ class FateServiceHandler implements FateService.Iface {
     master.fate.delete(opid);
   }
 
-  protected void authenticate(TCredentials c) throws ThriftSecurityException {
-    if (!master.security.authenticateUser(c, c))
-      throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.BAD_CREDENTIALS);
-  }
-
-  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);
-      throw new ThriftTableOperationException(tableId, null, operation, TableOperationExceptionType.OTHER, why);
-    }
+  protected void authenticate(TCredentials credentials) throws ThriftSecurityException {
+    // this is a bit redundant, the credentials of the caller (the first arg) will throw an exception if it fails to authenticate
+    // before the second arg is checked (which would return true or false)
+    if (!master.security.authenticateUser(credentials, credentials))
+      throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.BAD_CREDENTIALS);
   }
 
-  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;
+  // Verify table name arguments are valid, and match any additional restrictions
+  private String validateTableIdArgument(ByteBuffer tableIdArg, TableOperation op, Validator<String> userValidator) throws ThriftTableOperationException {
+    String tableId = tableIdArg == null ? null : ByteBufferUtil.toString(tableIdArg);
+    try {
+      return Tables.VALID_ID.and(userValidator).validate(tableId);
+    } catch (IllegalArgumentException e) {
+      String why = e.getMessage();
       log.warn(why);
-      throw new ThriftTableOperationException(null, tableName, operation, TableOperationExceptionType.OTHER, why);
+      throw new ThriftTableOperationException(tableId, null, op, TableOperationExceptionType.INVALID_NAME, why);
     }
   }
 
-  private static void checkNotSystemNamespace(String namespace, TableOperation operation) throws ThriftTableOperationException {
-    if (Namespaces.ACCUMULO_NAMESPACE.equals(namespace)) {
-      String why = "Namespaces cannot be == " + Namespaces.ACCUMULO_NAMESPACE;
-      log.warn(why);
-      throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.OTHER, why);
-    }
+  // Verify table name arguments are valid, and match any additional restrictions
+  private String validateTableNameArgument(ByteBuffer tableNameArg, TableOperation op, Validator<String> userValidator) throws ThriftTableOperationException {
+    String tableName = tableNameArg == null ? null : ByteBufferUtil.toString(tableNameArg);
+    return _validateArgument(tableName, op, Tables.VALID_NAME.and(userValidator));
   }
 
-  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(master.getInstance()).containsKey(tableName)) {
-      String why = "Table name already exists: " + tableName;
-      throw new ThriftTableOperationException(null, tableName, operation, TableOperationExceptionType.EXISTS, why);
-    }
+  // Verify namespace arguments are valid, and match any additional restrictions
+  private String validateNamespaceArgument(ByteBuffer namespaceArg, TableOperation op, Validator<String> userValidator) throws ThriftTableOperationException {
+    String namespace = namespaceArg == null ? null : ByteBufferUtil.toString(namespaceArg);
+    return _validateArgument(namespace, op, Namespaces.VALID_NAME.and(userValidator));
   }
 
-  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;
+  // helper to handle the exception
+  private <T> T _validateArgument(T arg, TableOperation op, Validator<T> validator) throws ThriftTableOperationException {
+    try {
+      return validator.validate(arg);
+    } catch (IllegalArgumentException e) {
+      String why = e.getMessage();
       log.warn(why);
-      throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.INVALID_NAME, why);
+      throw new ThriftTableOperationException(null, String.valueOf(arg), op, TableOperationExceptionType.INVALID_NAME, why);
     }
-    if (Namespaces.getNameToIdMap(master.getInstance()).containsKey(namespace)) {
-      String why = "Namespace already exists: " + namespace;
-      throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.NAMESPACE_EXISTS, why);
-    }
-  }
-
-  protected String checkNamespaceId(String namespace, TableOperation operation) throws ThriftTableOperationException {
-    final String namespaceId = Namespaces.getNameToIdMap(master.getInstance()).get(namespace);
-    if (namespaceId == null)
-      throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.NAMESPACE_NOTFOUND, null);
-    return namespaceId;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/764884e7/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java b/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
index dc654ae..3a39f5c 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
@@ -405,7 +405,7 @@ class MasterClientServiceHandler extends FateServiceHandler implements MasterCli
       ThriftTableOperationException {
 
     String namespaceId = null;
-    namespaceId = checkNamespaceId(namespace, op);
+    namespaceId = ClientServiceHandler.checkNamespaceId(master.getInstance(), namespace, op);
 
     if (!master.security.canAlterNamespace(c, namespaceId))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
@@ -418,7 +418,7 @@ class MasterClientServiceHandler extends FateServiceHandler implements MasterCli
       }
     } catch (KeeperException.NoNodeException e) {
       // race condition... namespace no longer exists? This call will throw an exception if the namespace was deleted:
-      checkNamespaceId(namespaceId, op);
+      ClientServiceHandler.checkNamespaceId(master.getInstance(), namespaceId, op);
       log.info("Error altering namespace property", e);
       throw new ThriftTableOperationException(namespaceId, namespace, op, TableOperationExceptionType.OTHER, "Problem altering namespaceproperty");
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/764884e7/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 8354d56..addb377 100644
--- a/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/NamespacesIT.java
@@ -76,7 +76,6 @@ import org.apache.accumulo.test.functional.SimpleMacIT;
 import org.apache.hadoop.io.Text;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Test;
 
 public class NamespacesIT extends SimpleMacIT {
@@ -86,12 +85,14 @@ public class NamespacesIT extends SimpleMacIT {
 
   @Before
   public void setUpConnectorAndNamespace() throws Exception {
+    // prepare a unique namespace and get a new root connector for each test
     c = getConnector();
     namespace = "ns_" + getTableNames(1)[0];
   }
 
   @After
-  public void removeTablesAndNamespaces() throws Exception {
+  public void swingMjölnir() throws Exception {
+    // clean up any added tables, namespaces, and users, after each test
     for (String t : c.tableOperations().list())
       if (!Tables.qualify(t).getFirst().equals(Namespaces.ACCUMULO_NAMESPACE))
         c.tableOperations().delete(t);
@@ -114,16 +115,14 @@ public class NamespacesIT extends SimpleMacIT {
 
   @Test
   public void createTableInDefaultNamespace() throws Exception {
-    String tableName = getTableNames(1)[0];
+    String tableName = "1";
     c.tableOperations().create(tableName);
     assertTrue(c.tableOperations().exists(tableName));
   }
 
-  // TODO enable after ACCUMULO-2079 is fixed
-  @Ignore
-  @Test(expected = AccumuloSecurityException.class)
+  @Test(expected = AccumuloException.class)
   public void createTableInAccumuloNamespace() throws Exception {
-    String tableName = Namespaces.ACCUMULO_NAMESPACE + "." + getTableNames(1)[0];
+    String tableName = Namespaces.ACCUMULO_NAMESPACE + ".1";
     assertFalse(c.tableOperations().exists(tableName));
     c.tableOperations().create(tableName); // should fail
   }
@@ -1058,14 +1057,28 @@ public class NamespacesIT extends SimpleMacIT {
             ops.create(namespace + "0"); // should fail here
             fail();
           case 1:
-            ops.create(namespace + "1");
-            ops.create(namespace + "2");
-            ops.rename(namespace + "1", namespace + "2"); // should fail here
+            ops.create(namespace + i + "_1");
+            ops.create(namespace + i + "_2");
+            ops.rename(namespace + i + "_1", namespace + i + "_2"); // should fail here
+            fail();
+          case 2:
+            ops.create(Namespaces.DEFAULT_NAMESPACE);
+            fail();
+          case 3:
+            ops.create(Namespaces.ACCUMULO_NAMESPACE);
+            fail();
+          case 4:
+            ops.create(namespace + i + "_1");
+            ops.rename(namespace + i + "_1", Namespaces.DEFAULT_NAMESPACE); // should fail here
+            fail();
+          case 5:
+            ops.create(namespace + i + "_1");
+            ops.rename(namespace + i + "_1", Namespaces.ACCUMULO_NAMESPACE); // should fail here
             fail();
           default:
             // break out of infinite loop
-            assertEquals(2, i); // check test integrity
-            assertEquals(2, numRun); // check test integrity
+            assertEquals(6, i); // check test integrity
+            assertEquals(6, numRun); // check test integrity
             break NAMESPACEEXISTS;
         }
       } catch (Exception e) {