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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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