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

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

ACCUMULO-1712 Rename system tables and namespaces

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


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

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

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


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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8b54ced2/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
index 4b19edb..1fb0b16 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
@@ -386,8 +386,8 @@ public class ZKPermHandler implements PermissionHandler {
     tablePerms.put(MetadataTable.ID, Collections.singleton(TablePermission.ALTER_TABLE));
     // essentially the same but on the system namespace, the ALTER_TABLE permission is now redundant
     Map<String,Set<NamespacePermission>> namespacePerms = new HashMap<String,Set<NamespacePermission>>();
-    namespacePerms.put(Constants.SYSTEM_NAMESPACE_ID, Collections.singleton(NamespacePermission.ALTER_NAMESPACE));
-    namespacePerms.put(Constants.SYSTEM_NAMESPACE_ID, Collections.singleton(NamespacePermission.ALTER_TABLE));
+    namespacePerms.put(Constants.ACCUMULO_NAMESPACE_ID, Collections.singleton(NamespacePermission.ALTER_NAMESPACE));
+    namespacePerms.put(Constants.ACCUMULO_NAMESPACE_ID, Collections.singleton(NamespacePermission.ALTER_TABLE));
 
     try {
       // prep parent node of users with root username