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

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

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfWatcher.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfWatcher.java b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfWatcher.java
new file mode 100644
index 0000000..7f3d73e
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfWatcher.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.server.conf;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+
+class NamespaceConfWatcher implements Watcher {
+  static {
+    Logger.getLogger("org.apache.zookeeper").setLevel(Level.WARN);
+    Logger.getLogger("org.apache.hadoop.io.compress").setLevel(Level.WARN);
+  }
+
+  private static final Logger log = Logger.getLogger(NamespaceConfWatcher.class);
+  private Instance instance = null;
+
+  NamespaceConfWatcher(Instance instance) {
+    this.instance = instance;
+  }
+
+  @Override
+  public void process(WatchedEvent event) {
+    String path = event.getPath();
+    if (log.isTraceEnabled())
+      log.trace("WatchEvent : " + path + " " + event.getState() + " " + event.getType());
+
+    String namespacesPrefix = ZooUtil.getRoot(instance) + Constants.ZNAMESPACES + "/";
+
+    String namespaceId = null;
+    String key = null;
+
+    if (path != null) {
+      if (path.startsWith(namespacesPrefix)) {
+        namespaceId = path.substring(namespacesPrefix.length());
+        if (namespaceId.contains("/")) {
+          namespaceId = namespaceId.substring(0, namespaceId.indexOf('/'));
+          if (path.startsWith(namespacesPrefix + namespaceId + Constants.ZNAMESPACE_CONF + "/"))
+            key = path.substring((namespacesPrefix + namespaceId + Constants.ZNAMESPACE_CONF + "/").length());
+        }
+      }
+
+      if (namespaceId == null) {
+        log.warn("Zookeeper told me about a path I was not watching " + path + " state=" + event.getState() + " type=" + event.getType());
+        return;
+      }
+    }
+
+    switch (event.getType()) {
+      case NodeDataChanged:
+        if (log.isTraceEnabled())
+          log.trace("EventNodeDataChanged " + event.getPath());
+        if (key != null)
+          ServerConfiguration.getNamespaceConfiguration(instance, namespaceId).propertyChanged(key);
+        break;
+      case NodeChildrenChanged:
+        ServerConfiguration.getNamespaceConfiguration(instance, namespaceId).propertiesChanged(key);
+        break;
+      case NodeDeleted:
+        if (key == null) {
+          ServerConfiguration.removeNamespaceIdInstance(namespaceId);
+        }
+        break;
+      case None:
+        switch (event.getState()) {
+          case Expired:
+            ServerConfiguration.expireAllTableObservers();
+            break;
+          case SyncConnected:
+            break;
+          case Disconnected:
+            break;
+          default:
+            log.warn("EventNone event not handled path = " + event.getPath() + " state=" + event.getState());
+        }
+        break;
+      case NodeCreated:
+        switch (event.getState()) {
+          case SyncConnected:
+            break;
+          default:
+            log.warn("Event NodeCreated event not handled path = " + event.getPath() + " state=" + event.getState());
+        }
+        break;
+      default:
+        log.warn("Event not handled path = " + event.getPath() + " state=" + event.getState() + " type = " + event.getType());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
new file mode 100644
index 0000000..d300366
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.server.conf;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.ConfigurationObserver;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.server.client.HdfsZooInstance;
+import org.apache.log4j.Logger;
+
+public class NamespaceConfiguration extends AccumuloConfiguration {
+  private static final Logger log = Logger.getLogger(NamespaceConfiguration.class);
+
+  private final AccumuloConfiguration parent;
+  private static ZooCache propCache = null;
+  protected String namespaceId = null;
+  protected Instance inst = null;
+  private Set<ConfigurationObserver> observers;
+
+  public NamespaceConfiguration(String namespaceId, AccumuloConfiguration parent) {
+    inst = HdfsZooInstance.getInstance();
+    this.parent = parent;
+    this.namespaceId = namespaceId;
+    this.observers = Collections.synchronizedSet(new HashSet<ConfigurationObserver>());
+  }
+
+  @Override
+  public String get(Property property) {
+    String key = property.getKey();
+    String value = get(getPropCache(), key);
+
+    if (value == null || !property.getType().isValidFormat(value)) {
+      if (value != null)
+        log.error("Using default value for " + key + " due to improperly formatted " + property.getType() + ": " + value);
+      if (!(namespaceId.equals(Constants.SYSTEM_NAMESPACE_ID) && isIteratorOrConstraint(property.getKey()))) {
+        // ignore iterators from parent if system namespace
+        value = parent.get(property);
+      }
+    }
+    return value;
+  }
+
+  private String get(ZooCache zc, String key) {
+    String zPath = ZooUtil.getRoot(inst.getInstanceID()) + Constants.ZNAMESPACES + "/" + getNamespaceId() + Constants.ZNAMESPACE_CONF + "/" + key;
+    byte[] v = zc.get(zPath);
+    String value = null;
+    if (v != null)
+      value = new String(v, Constants.UTF8);
+    return value;
+  }
+
+  private static ZooCache getPropCache() {
+    Instance inst = HdfsZooInstance.getInstance();
+    if (propCache == null)
+      synchronized (NamespaceConfiguration.class) {
+        if (propCache == null)
+          propCache = new ZooCache(inst.getZooKeepers(), inst.getZooKeepersSessionTimeOut(), new NamespaceConfWatcher(inst));
+      }
+    return propCache;
+  }
+
+  private class SystemNamespaceFilter implements PropertyFilter {
+
+    private PropertyFilter userFilter;
+
+    SystemNamespaceFilter(PropertyFilter userFilter) {
+      this.userFilter = userFilter;
+    }
+
+    @Override
+    public boolean accept(String key) {
+      if (isIteratorOrConstraint(key))
+        return false;
+      return userFilter.accept(key);
+    }
+
+  }
+
+  @Override
+  public void getProperties(Map<String,String> props, PropertyFilter filter) {
+
+    PropertyFilter parentFilter = filter;
+
+    // exclude system iterators/constraints from the system namespace
+    // so they don't affect the metadata or root tables.
+    if (this.namespaceId.equals(Constants.SYSTEM_NAMESPACE_ID))
+      parentFilter = new SystemNamespaceFilter(filter);
+
+    parent.getProperties(props, parentFilter);
+
+    ZooCache zc = getPropCache();
+
+    List<String> children = zc.getChildren(ZooUtil.getRoot(inst.getInstanceID()) + Constants.ZNAMESPACES + "/" + getNamespaceId() + Constants.ZNAMESPACE_CONF);
+    if (children != null) {
+      for (String child : children) {
+        if (child != null && filter.accept(child)) {
+          String value = get(zc, child);
+          if (value != null)
+            props.put(child, value);
+        }
+      }
+    }
+  }
+
+  protected String getNamespaceId() {
+    return namespaceId;
+  }
+
+  public void addObserver(ConfigurationObserver co) {
+    if (namespaceId == null) {
+      String err = "Attempt to add observer for non-namespace configuration";
+      log.error(err);
+      throw new RuntimeException(err);
+    }
+    iterator();
+    observers.add(co);
+  }
+
+  public void removeObserver(ConfigurationObserver configObserver) {
+    if (namespaceId == null) {
+      String err = "Attempt to remove observer for non-namespace configuration";
+      log.error(err);
+      throw new RuntimeException(err);
+    }
+    observers.remove(configObserver);
+  }
+
+  public void expireAllObservers() {
+    Collection<ConfigurationObserver> copy = Collections.unmodifiableCollection(observers);
+    for (ConfigurationObserver co : copy)
+      co.sessionExpired();
+  }
+
+  public void propertyChanged(String key) {
+    Collection<ConfigurationObserver> copy = Collections.unmodifiableCollection(observers);
+    for (ConfigurationObserver co : copy)
+      co.propertyChanged(key);
+  }
+
+  public void propertiesChanged(String key) {
+    Collection<ConfigurationObserver> copy = Collections.unmodifiableCollection(observers);
+    for (ConfigurationObserver co : copy)
+      co.propertiesChanged();
+  }
+
+  protected boolean isIteratorOrConstraint(String key) {
+    return key.startsWith(Property.TABLE_ITERATOR_PREFIX.getKey()) || key.startsWith(Property.TABLE_CONSTRAINT_PREFIX.getKey());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ea8ec193/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java
index 5093025..2115f3f 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfiguration.java
@@ -32,8 +32,8 @@ import org.apache.accumulo.core.data.KeyExtent;
 public class ServerConfiguration {
 
   private static final Map<String,TableConfiguration> tableInstances = new HashMap<String,TableConfiguration>(1);
-  private static final Map<String,TableNamespaceConfiguration> tableNamespaceInstances = new HashMap<String,TableNamespaceConfiguration>(1);
-  private static final Map<String,TableNamespaceConfiguration> tableParentInstances = new HashMap<String,TableNamespaceConfiguration>(1);
+  private static final Map<String,NamespaceConfiguration> namespaceInstances = new HashMap<String,NamespaceConfiguration>(1);
+  private static final Map<String,NamespaceConfiguration> tableParentInstances = new HashMap<String,NamespaceConfiguration>(1);
   private static SecurityPermission CONFIGURATION_PERMISSION = new SecurityPermission("configurationPermission");
 
   public static synchronized SiteConfiguration getSiteConfiguration() {
@@ -62,10 +62,10 @@ public class ServerConfiguration {
     return getZooConfiguration(instance);
   }
 
-  public static TableNamespaceConfiguration getTableNamespaceConfigurationForTable(Instance instance, String tableId) {
+  public static NamespaceConfiguration getNamespaceConfigurationForTable(Instance instance, String tableId) {
     checkPermissions();
     synchronized (tableParentInstances) {
-      TableNamespaceConfiguration conf = tableParentInstances.get(tableId);
+      NamespaceConfiguration conf = tableParentInstances.get(tableId);
       if (conf == null) {
         conf = new TableParentConfiguration(tableId, getSystemConfiguration(instance));
         ConfigSanityCheck.validate(conf);
@@ -75,14 +75,14 @@ public class ServerConfiguration {
     }
   }
 
-  public static TableNamespaceConfiguration getTableNamespaceConfiguration(Instance instance, String namespaceId) {
+  public static NamespaceConfiguration getNamespaceConfiguration(Instance instance, String namespaceId) {
     checkPermissions();
-    synchronized (tableNamespaceInstances) {
-      TableNamespaceConfiguration conf = tableNamespaceInstances.get(namespaceId);
+    synchronized (namespaceInstances) {
+      NamespaceConfiguration conf = namespaceInstances.get(namespaceId);
       if (conf == null) {
-        conf = new TableNamespaceConfiguration(namespaceId, getSystemConfiguration(instance));
+        conf = new NamespaceConfiguration(namespaceId, getSystemConfiguration(instance));
         ConfigSanityCheck.validate(conf);
-        tableNamespaceInstances.put(namespaceId, conf);
+        namespaceInstances.put(namespaceId, conf);
       }
       return conf;
     }
@@ -93,7 +93,7 @@ public class ServerConfiguration {
     synchronized (tableInstances) {
       TableConfiguration conf = tableInstances.get(tableId);
       if (conf == null && Tables.exists(instance, tableId)) {
-        conf = new TableConfiguration(instance.getInstanceID(), tableId, getTableNamespaceConfigurationForTable(instance, tableId));
+        conf = new TableConfiguration(instance.getInstanceID(), tableId, getNamespaceConfigurationForTable(instance, tableId));
         ConfigSanityCheck.validate(conf);
         tableInstances.put(tableId, conf);
       }
@@ -108,8 +108,8 @@ public class ServerConfiguration {
   }
 
   static void removeNamespaceIdInstance(String namespaceId) {
-    synchronized (tableNamespaceInstances) {
-      tableNamespaceInstances.remove(namespaceId);
+    synchronized (namespaceInstances) {
+      namespaceInstances.remove(namespaceId);
     }
   }
 
@@ -135,8 +135,8 @@ public class ServerConfiguration {
     return getTableConfiguration(extent.getTableId().toString());
   }
 
-  public TableNamespaceConfiguration getTableNamespaceConfiguration(String namespaceId) {
-    return getTableNamespaceConfiguration(instance, namespaceId);
+  public NamespaceConfiguration getNamespaceConfiguration(String namespaceId) {
+    return getNamespaceConfiguration(instance, namespaceId);
   }
 
   public synchronized AccumuloConfiguration getConfiguration() {