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/11/01 03:00:12 UTC

[11/54] git commit: ACCUMULO-802 Added global shell completion for table namespaces with -tn (just like -t for tables). Fixed problem with RenameTable to a new namespace copying over the old namespace configuration to the new table. Many other minor fixe

ACCUMULO-802 Added global shell completion for table namespaces with -tn (just like -t for tables). Fixed problem with RenameTable to a new namespace copying over the old namespace configuration to the new table. Many other minor fixes. A few tests fail due to CloneTable not working until ACCUMULO-1565 is fixed


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

Branch: refs/heads/ACCUMULO-802
Commit: fd5f11df840873205c33afb31c899fb11ad99a81
Parents: 77a3aa4
Author: Sean Hickey <ta...@gmail.com>
Authored: Fri Jul 12 13:44:29 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Thu Oct 31 21:24:10 2013 -0400

----------------------------------------------------------------------
 .../core/client/admin/TableOperationsImpl.java  | 28 ++++----
 .../core/util/shell/ShellCompletor.java         | 17 +++--
 .../accumulo/core/util/shell/ShellOptions.java  |  1 +
 .../util/shell/commands/CreateTableCommand.java | 75 ++++++++++++--------
 .../util/shell/commands/DeleteTableCommand.java | 10 ++-
 .../util/shell/commands/RenameTableCommand.java | 16 +++--
 .../server/conf/ServerConfiguration.java        | 21 +++++-
 .../conf/TableNamespaceConfiguration.java       | 19 ++++-
 .../apache/accumulo/server/init/Initialize.java |  8 +++
 .../java/org/apache/accumulo/master/Master.java |  8 +++
 .../test/randomwalk/concurrent/Setup.java       |  4 +-
 .../org/apache/accumulo/test/ShellServerIT.java | 18 +++++
 12 files changed, 168 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/fd5f11df/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
index 6fa4174..ab8ec7a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
@@ -217,12 +217,16 @@ public class TableOperationsImpl extends TableOperationsHelper {
     
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes()), ByteBuffer.wrap(timeType.name().getBytes()));
     
-    Map<String,String> opts = IteratorUtil.generateInitialTableProperties(limitVersion);
+    // Map<String,String> opts = IteratorUtil.generateInitialTableProperties(limitVersion);
+    Map<String,String> opts = new HashMap<String,String>();
     
     String namespace = Tables.extractNamespace(tableName);
     if (!namespaceExists(namespace)) {
       String info = "Table namespace not found while trying to create table";
-      throw new RuntimeException(new TableNamespaceNotFoundException(null, namespace, info));
+      throw new IllegalArgumentException(new TableNamespaceNotFoundException(null, namespace, info));
+    } else if (namespace.equals(Constants.SYSTEM_TABLE_NAMESPACE)) {
+      String info = "Can't create tables in the system namespace";
+      throw new IllegalArgumentException(info);
     }
     
     try {
@@ -693,18 +697,17 @@ public class TableOperationsImpl extends TableOperationsHelper {
       _flush(srcTableId, null, null, true);
     
     if (propertiesToExclude == null)
-      propertiesToExclude = new HashSet<String>();
+      propertiesToExclude = Collections.emptySet();
     
     if (propertiesToSet == null)
       propertiesToSet = Collections.emptyMap();
     
-    
-    Set<String> nProps = getUniqueNamespaceProperties(namespace, srcTableName);
-    for (String p : nProps) {
-      propertiesToExclude.add(p);
+    HashSet<String> excludeProps = getUniqueNamespaceProperties(namespace, srcTableName);
+    for (String p : propertiesToExclude) {
+      excludeProps.add(p);
     }
     
-    if (!Collections.disjoint(propertiesToExclude, propertiesToSet.keySet()))
+    if (!Collections.disjoint(excludeProps, propertiesToSet.keySet()))
       throw new IllegalArgumentException("propertiesToSet and propertiesToExclude not disjoint");
     
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(srcTableId.getBytes()), ByteBuffer.wrap(newTableName.getBytes()));
@@ -722,9 +725,10 @@ public class TableOperationsImpl extends TableOperationsHelper {
     doTableOperation(TableOperation.CLONE, args, opts);
   }
   
-  private Set<String> getUniqueNamespaceProperties(String namespace, String table) throws TableNotFoundException, AccumuloException {
-    Set<String> props = new HashSet<String>();
-    /*try {
+  // get the properties that are only in the table namespace so that we can exclude them when copying table properties
+  private HashSet<String> getUniqueNamespaceProperties(String namespace, String table) throws TableNotFoundException, AccumuloException {
+    HashSet<String> props = new HashSet<String>();
+    try {
       Iterable<Entry<String,String>> n = new TableNamespaceOperationsImpl(instance, credentials).getProperties(namespace);
       Iterable<Entry<String,String>> t = getProperties(table);
       Map<String,String> tmap = new HashMap<String,String>();
@@ -739,7 +743,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       }
     } catch (TableNamespaceNotFoundException e) {
       throw new IllegalStateException(new TableNamespaceNotFoundException(null, namespace, null));
-    }*/
+    }
     return props;
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fd5f11df/core/src/main/java/org/apache/accumulo/core/util/shell/ShellCompletor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/ShellCompletor.java b/core/src/main/java/org/apache/accumulo/core/util/shell/ShellCompletor.java
index 85f68ef..4046714 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/ShellCompletor.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/ShellCompletor.java
@@ -57,7 +57,7 @@ public class ShellCompletor implements Completer {
   }
   
   private int _complete(String fullBuffer, int cursor, List<String> candidates) {
-    boolean inTableFlag = false, inUserFlag = false;
+    boolean inTableFlag = false, inUserFlag = false, inNamespaceFlag = false;
     // Only want to grab the buffer up to the cursor because
     // the user could be trying to tab complete in the middle
     // of the line
@@ -98,7 +98,10 @@ public class ShellCompletor implements Completer {
           } else if (current_string_token.trim().equals("-" + Shell.userOption)) {
             candidates.addAll(options.get(Shell.Command.CompletionSet.USERNAMES));
             prefix += "-" + Shell.userOption + " ";
-          } else if (current_command_token != null) {
+          } else if (current_string_token.trim().equals("-" + Shell.tableNamespaceOption)) {
+            candidates.addAll(options.get(Shell.Command.CompletionSet.TABLENAMESPACES));
+            prefix += "-" + Shell.tableNamespaceOption + " ";
+          }else if (current_command_token != null) {
             Token next = current_command_token.getSubcommand(current_string_token);
             if (next != null) {
               current_command_token = next;
@@ -115,7 +118,7 @@ public class ShellCompletor implements Completer {
           return (prefix.length());
         }
         // need to match current command
-        // if we're in -t <table> or -u <user> complete those
+        // if we're in -t <table>, -u <user>, or -tn <tableNamespace> complete those
         if (inTableFlag) {
           for (String a : options.get(Shell.Command.CompletionSet.TABLENAMES))
             if (a.startsWith(current_string_token))
@@ -124,6 +127,10 @@ public class ShellCompletor implements Completer {
           for (String a : options.get(Shell.Command.CompletionSet.USERNAMES))
             if (a.startsWith(current_string_token))
               candidates.add(a);
+        } else if (inNamespaceFlag) {
+          for (String a : options.get(Shell.Command.CompletionSet.TABLENAMESPACES))
+            if (a.startsWith(current_string_token))
+              candidates.add(a);
         } else if (current_command_token != null)
           candidates.addAll(current_command_token.getSubcommandNames(current_string_token));
         
@@ -135,8 +142,10 @@ public class ShellCompletor implements Completer {
         inTableFlag = true;
       else if (current_string_token.trim().equals("-" + Shell.userOption))
         inUserFlag = true;
+      else if (current_string_token.trim().equals("-" + Shell.tableNamespaceOption))
+        inNamespaceFlag = true;
       else
-        inUserFlag = inTableFlag = false;
+        inUserFlag = inTableFlag = inNamespaceFlag = false;
       
       if (current_command_token != null && current_command_token.getCaseSensitive())
         prefix += current_string_token + " ";

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fd5f11df/core/src/main/java/org/apache/accumulo/core/util/shell/ShellOptions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/ShellOptions.java b/core/src/main/java/org/apache/accumulo/core/util/shell/ShellOptions.java
index 3152973..af55d8d 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/ShellOptions.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/ShellOptions.java
@@ -25,6 +25,7 @@ public abstract class ShellOptions {
   // Global options flags
   public static final String userOption = "u";
   public static final String tableOption = "t";
+  public static final String tableNamespaceOption = "tn";
   public static final String helpOption = "?";
   public static final String helpLongOption = "help";
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fd5f11df/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java
index 0ad787d..8081d92 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java
@@ -18,8 +18,10 @@ package org.apache.accumulo.core.util.shell.commands;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Scanner;
+import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
@@ -29,11 +31,13 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.TimeType;
+import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.security.VisibilityConstraint;
 import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
+import org.apache.accumulo.core.util.shell.Token;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionGroup;
@@ -52,27 +56,28 @@ public class CreateTableCommand extends Command {
   private Option base64Opt;
   private Option createTableOptFormatter;
   public static String testTable;
-  
-  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException, TableExistsException,
-      TableNotFoundException, IOException, ClassNotFoundException {
-    
+
+  @Override
+  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException,
+      TableExistsException, TableNotFoundException, IOException, ClassNotFoundException {
+
     final String testTableName = cl.getArgs()[0];
-    
+
     if (!testTableName.matches(Constants.VALID_TABLE_NAME_REGEX)) {
       shellState.getReader().println("Only letters, numbers and underscores are allowed for use in table names.");
       throw new IllegalArgumentException();
     }
-    
+
     final String tableName = cl.getArgs()[0];
     if (shellState.getConnector().tableOperations().exists(tableName)) {
       throw new TableExistsException(null, tableName, null);
     }
     final SortedSet<Text> partitions = new TreeSet<Text>();
     final boolean decode = cl.hasOption(base64Opt.getOpt());
-    
+
     if (cl.hasOption(createTableOptSplit.getOpt())) {
       final String f = cl.getOptionValue(createTableOptSplit.getOpt());
-      
+
       String line;
       Scanner file = new Scanner(new File(f));
       while (file.hasNextLine()) {
@@ -88,33 +93,38 @@ public class CreateTableCommand extends Command {
       }
       partitions.addAll(shellState.getConnector().tableOperations().listSplits(oldTable));
     }
-    
+
     if (cl.hasOption(createTableOptCopyConfig.getOpt())) {
       final String oldTable = cl.getOptionValue(createTableOptCopyConfig.getOpt());
       if (!shellState.getConnector().tableOperations().exists(oldTable)) {
         throw new TableNotFoundException(null, oldTable, null);
       }
     }
-    
+
     TimeType timeType = TimeType.MILLIS;
     if (cl.hasOption(createTableOptTimeLogical.getOpt())) {
       timeType = TimeType.LOGICAL;
     }
-    
+
     // create table
     shellState.getConnector().tableOperations().create(tableName, true, timeType);
     if (partitions.size() > 0) {
       shellState.getConnector().tableOperations().addSplits(tableName, partitions);
     }
-    shellState.setTableName(tableName); // switch shell to new table
-    // context
-    
+
+    String n = Tables.extractNamespace(tableName);
+    String table = tableName;
+    if (n.equals(Constants.DEFAULT_TABLE_NAMESPACE) || n.equals(Constants.SYSTEM_TABLE_NAMESPACE)) {
+      table = Tables.extractTableName(tableName);
+    }
+    shellState.setTableName(table); // switch shell to new table context
+
     if (cl.hasOption(createTableNoDefaultIters.getOpt())) {
       for (String key : IteratorUtil.generateInitialTableProperties(true).keySet()) {
         shellState.getConnector().tableOperations().removeProperty(tableName, key);
       }
     }
-    
+
     // Copy options if flag was set
     if (cl.hasOption(createTableOptCopyConfig.getOpt())) {
       if (shellState.getConnector().tableOperations().exists(tableName)) {
@@ -127,7 +137,7 @@ public class CreateTableCommand extends Command {
         }
       }
     }
-    
+
     if (cl.hasOption(createTableOptEVC.getOpt())) {
       try {
         shellState.getConnector().tableOperations().addConstraint(tableName, VisibilityConstraint.class.getName());
@@ -135,31 +145,31 @@ public class CreateTableCommand extends Command {
         Shell.log.warn(e.getMessage() + " while setting visibility constraint, but table was created");
       }
     }
-    
+
     // Load custom formatter if set
     if (cl.hasOption(createTableOptFormatter.getOpt())) {
       final String formatterClass = cl.getOptionValue(createTableOptFormatter.getOpt());
-      
+
       shellState.getConnector().tableOperations().setProperty(tableName, Property.TABLE_FORMATTER_CLASS.toString(), formatterClass);
     }
-    
+
     return 0;
   }
-  
+
   @Override
   public String description() {
     return "creates a new table, with optional aggregators and optionally pre-split";
   }
-  
+
   @Override
   public String usage() {
     return getName() + " <tableName>";
   }
-  
+
   @Override
   public Options getOptions() {
     final Options o = new Options();
-    
+
     createTableOptCopyConfig = new Option("cc", "copy-config", true, "table to copy configuration from");
     createTableOptCopySplits = new Option("cs", "copy-splits", true, "table to copy current splits from");
     createTableOptSplit = new Option("sf", "splits-file", true, "file with a newline-separated list of rows to split the table with");
@@ -169,25 +179,25 @@ public class CreateTableCommand extends Command {
     createTableOptEVC = new Option("evc", "enable-visibility-constraint", false,
         "prevent users from writing data they cannot read.  When enabling this, consider disabling bulk import and alter table.");
     createTableOptFormatter = new Option("f", "formatter", true, "default formatter to set");
-    
+
     createTableOptCopyConfig.setArgName("table");
     createTableOptCopySplits.setArgName("table");
     createTableOptSplit.setArgName("filename");
     createTableOptFormatter.setArgName("className");
-    
+
     // Splits and CopySplits are put in an optionsgroup to make them
     // mutually exclusive
     final OptionGroup splitOrCopySplit = new OptionGroup();
     splitOrCopySplit.addOption(createTableOptSplit);
     splitOrCopySplit.addOption(createTableOptCopySplits);
-    
+
     final OptionGroup timeGroup = new OptionGroup();
     timeGroup.addOption(createTableOptTimeLogical);
     timeGroup.addOption(createTableOptTimeMillis);
-    
+
     base64Opt = new Option("b64", "base64encoded", false, "decode encoded split points");
     o.addOption(base64Opt);
-    
+
     o.addOptionGroup(splitOrCopySplit);
     o.addOptionGroup(timeGroup);
     o.addOption(createTableOptSplit);
@@ -195,12 +205,17 @@ public class CreateTableCommand extends Command {
     o.addOption(createTableNoDefaultIters);
     o.addOption(createTableOptEVC);
     o.addOption(createTableOptFormatter);
-    
+
     return o;
   }
-  
+
   @Override
   public int numArgs() {
     return 1;
   }
+
+  @Override
+  public void registerCompletion(final Token root, final Map<Command.CompletionSet,Set<String>> special) {
+    registerCompletionForTableNamespaces(root, special);
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fd5f11df/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteTableCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteTableCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteTableCommand.java
index 35c534d..74d3f36 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteTableCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/DeleteTableCommand.java
@@ -16,6 +16,8 @@
  */
 package org.apache.accumulo.core.util.shell.commands;
 
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
@@ -42,7 +44,13 @@ public class DeleteTableCommand extends TableOperation {
   protected void doTableOp(final Shell shellState, final String tableName) throws Exception {
     shellState.getConnector().tableOperations().delete(tableName);
     shellState.getReader().println("Table: [" + tableName + "] has been deleted.");
-    if (shellState.getTableName().equals(tableName)) {
+    
+    String n = Tables.extractNamespace(tableName);
+    String checkTable = tableName;
+    if (n.equals(Constants.DEFAULT_TABLE_NAMESPACE) || n.equals(Constants.SYSTEM_TABLE_NAMESPACE)) {
+      checkTable = Tables.extractTableName(tableName);
+    }
+    if (shellState.getTableName().equals(checkTable)) {
       shellState.setTableName("");
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fd5f11df/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameTableCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameTableCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameTableCommand.java
index 057f2b4..4ba039d 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameTableCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/RenameTableCommand.java
@@ -19,10 +19,12 @@ package org.apache.accumulo.core.util.shell.commands;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.util.shell.Shell;
 import org.apache.accumulo.core.util.shell.Shell.Command;
 import org.apache.accumulo.core.util.shell.Token;
@@ -30,11 +32,17 @@ import org.apache.commons.cli.CommandLine;
 
 public class RenameTableCommand extends Command {
   @Override
-  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException, TableNotFoundException,
-      TableExistsException {
+  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException,
+      TableNotFoundException, TableExistsException {
     shellState.getConnector().tableOperations().rename(cl.getArgs()[0], cl.getArgs()[1]);
-    if (shellState.getTableName().equals(cl.getArgs()[0]))
-      shellState.setTableName(cl.getArgs()[1]);
+    if (shellState.getTableName().equals(cl.getArgs()[0])) {
+      String tableName = cl.getArgs()[1];
+      String n = Tables.extractNamespace(tableName);
+      if (n.equals(Constants.DEFAULT_TABLE_NAMESPACE) || n.equals(Constants.SYSTEM_TABLE_NAMESPACE)) {
+        tableName = Tables.extractTableName(tableName);
+      }
+      shellState.setTableName(tableName);
+    }
     return 0;
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fd5f11df/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 f1bd8e7..f3421cc 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
@@ -61,7 +61,7 @@ public class ServerConfiguration {
     return getZooConfiguration(instance);
   }
 
-  public static TableNamespaceConfiguration getTableNamespaceConfiguration(Instance instance, String tableId) {
+  public static TableNamespaceConfiguration getTableNamespaceConfigurationForTable(Instance instance, String tableId) {
     checkPermissions();
     synchronized (tableNamespaceInstances) {
       TableNamespaceConfiguration conf = tableNamespaceInstances.get(tableId);
@@ -74,12 +74,25 @@ public class ServerConfiguration {
     }
   }
 
+  public static TableNamespaceConfiguration getTableNamespaceConfiguration(Instance instance, String namespaceId) {
+    checkPermissions();
+    synchronized (tableNamespaceInstances) {
+      TableNamespaceConfiguration conf = tableNamespaceInstances.get(namespaceId);
+      if (conf == null) {
+        conf = new TableNamespaceConfiguration(namespaceId, getSystemConfiguration(instance), true);
+        ConfigSanityCheck.validate(conf);
+        tableNamespaceInstances.put(namespaceId, conf);
+      }
+      return conf;
+    }
+  }
+
   public static TableConfiguration getTableConfiguration(Instance instance, String tableId) {
     checkPermissions();
     synchronized (tableInstances) {
       TableConfiguration conf = tableInstances.get(tableId);
       if (conf == null && Tables.exists(instance, tableId)) {
-        conf = new TableConfiguration(instance.getInstanceID(), tableId, getTableNamespaceConfiguration(instance, tableId));
+        conf = new TableConfiguration(instance.getInstanceID(), tableId, getTableNamespaceConfigurationForTable(instance, tableId));
         ConfigSanityCheck.validate(conf);
         tableInstances.put(tableId, conf);
       }
@@ -115,6 +128,10 @@ public class ServerConfiguration {
     return getTableConfiguration(extent.getTableId().toString());
   }
 
+  public TableNamespaceConfiguration getTableNamespaceConfiguration(String namespaceId) {
+    return getTableNamespaceConfiguration(instance, namespaceId);
+  }
+
   public synchronized AccumuloConfiguration getConfiguration() {
     return getZooConfiguration(instance);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fd5f11df/server/base/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfiguration.java
index c8498a1..0454c29 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/TableNamespaceConfiguration.java
@@ -37,6 +37,7 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
   private final AccumuloConfiguration parent;
   private static ZooCache propCache = null;
   private String tableId = null;
+  private String namespaceId = null;
   private Instance inst = null;
   
   public TableNamespaceConfiguration(String tableId, AccumuloConfiguration parent) {
@@ -46,6 +47,13 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
     this.tableId = tableId;
   }
   
+  public TableNamespaceConfiguration(String namespaceId, AccumuloConfiguration parent, boolean notForSpecificTable) {
+    inst = HdfsZooInstance.getInstance();
+    propCache = new ZooCache(inst.getZooKeepers(), inst.getZooKeepersSessionTimeOut());
+    this.parent = parent;
+    this.namespaceId = namespaceId;
+  }
+  
   @Override
   public void invalidateCache() {
     if (propCache != null)
@@ -66,7 +74,7 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
   }
   
   private String get(String key) {
-    String zPath = ZooUtil.getRoot(inst.getInstanceID()) + Constants.ZNAMESPACES + "/" + Tables.getNamespace(inst, tableId) + Constants.ZNAMESPACE_CONF + "/"
+    String zPath = ZooUtil.getRoot(inst.getInstanceID()) + Constants.ZNAMESPACES + "/" + getNamespaceId() + Constants.ZNAMESPACE_CONF + "/"
         + key;
     byte[] v = getPropCache().get(zPath);
     String value = null;
@@ -93,7 +101,7 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
       entries.put(parentEntry.getKey(), parentEntry.getValue());
     
     List<String> children = getPropCache().getChildren(
-        ZooUtil.getRoot(inst.getInstanceID()) + Constants.ZNAMESPACES + "/" + Tables.getNamespace(inst, tableId) + Constants.ZNAMESPACE_CONF);
+        ZooUtil.getRoot(inst.getInstanceID()) + Constants.ZNAMESPACES + "/" + getNamespaceId() + Constants.ZNAMESPACE_CONF);
     if (children != null) {
       for (String child : children) {
         String value = get(child);
@@ -104,4 +112,11 @@ public class TableNamespaceConfiguration extends AccumuloConfiguration {
     
     return entries.entrySet().iterator();
   }
+  
+  private String getNamespaceId() {
+    if (tableId != null) {
+      return Tables.getNamespace(inst, tableId);
+    }
+    return namespaceId;
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fd5f11df/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
index 47179fc..06568cd 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Locale;
+import java.util.Map;
 import java.util.Map.Entry;
 import java.util.UUID;
 
@@ -38,6 +39,7 @@ import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVWriter;
+import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.iterators.user.VersioningIterator;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.master.thrift.MasterGoalState;
@@ -392,6 +394,12 @@ public class Initialize {
     zoo.putPersistentData(zPath, new byte[0], NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zPath + Constants.ZNAMESPACE_NAME, namespace.getBytes(Constants.UTF8), NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zPath + Constants.ZNAMESPACE_CONF, new byte[0], NodeExistsPolicy.FAIL);
+    
+    Map<String,String> opts = IteratorUtil.generateInitialTableProperties(true);
+    for (Entry<String,String> e : opts.entrySet()) {
+      zoo.putPersistentData(zPath + Constants.ZNAMESPACE_CONF + "/" + e.getKey(), e.getValue().getBytes(Constants.UTF8),
+          NodeExistsPolicy.SKIP);
+    }
   }
   
   private static String getInstanceNamePath(Opts opts) throws IOException, KeeperException, InterruptedException {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fd5f11df/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 3325778..e4c078f 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
@@ -346,6 +346,14 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
         zoo.putPersistentData(systemTableNamespace + Constants.ZNAMESPACE_NAME, Constants.SYSTEM_TABLE_NAMESPACE.getBytes(Constants.UTF8),
             NodeExistsPolicy.SKIP);
 
+        Map<String,String> opts = IteratorUtil.generateInitialTableProperties(true);
+        for (Entry<String,String> e : opts.entrySet()) {
+          zoo.putPersistentData(defaultTableNamespace + Constants.ZNAMESPACE_CONF + "/" + e.getKey(), e.getValue().getBytes(Constants.UTF8),
+              NodeExistsPolicy.SKIP);
+          zoo.putPersistentData(systemTableNamespace + Constants.ZNAMESPACE_CONF + "/" + e.getKey(), e.getValue().getBytes(Constants.UTF8),
+              NodeExistsPolicy.SKIP);
+        }
+
         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_TABLE_NAMESPACE_ID.getBytes(Constants.UTF8),

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fd5f11df/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Setup.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Setup.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Setup.java
index 5a9edca..d1b6d17 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Setup.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Setup.java
@@ -35,9 +35,9 @@ public class Setup extends Test {
     log.debug("numTables = " + numTables);
     List<String> tables = new ArrayList<String>();
     for (int i = 0; i < numTables - 1; i++) {
-      tables.add(String.format("nspace_%03d.ctt_%03d", i, i));
+      tables.add(String.format("ctt_%03d", i, i));
     }
-    tables.add(String.format("ctt_%03d", numTables - 1));
+    tables.add(String.format("nspace.ctt_%03d", numTables - 1));
 
     state.set("tables", tables);
     

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fd5f11df/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
index cc46d72..79230a5 100644
--- a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
@@ -838,22 +838,40 @@ public class ShellServerIT extends SimpleMacIT {
     assertTrue(namespaces.contains("thing2"));
     assertTrue(!namespaces.contains("thing1"));
     
+    // can't delete a namespace that still contains tables, unless you do -f
     exec("createtable thing2.thingy", true);
     exec("deletenamespace thing2");
     exec("y");
     exec("namespaces", true, "thing2", true);
     
+    /* doesn't work yet, waiting on ACCUMULO-1565
+    exec("clonenamespace thing2 testers -e table.file.max", true);
+    exec("namespaces", true, "testers", true);
+    exec("tables", true, "testers.thingy", true);
+    exec("clonenamespace thing2 testers2 -s table.file.max=42", true);*/
+    
     exec("du -tn thing2", true, "thing2.thingy", true);
     
+    // all "TableOperation" commands can take a table namespace
     exec("offline -tn thing2", true);
     exec("online -tn thing2", true);
+    exec("flush -tn thing2", true);
+    exec("compact -tn thing2", true);
+    exec("createtable testers.1", true);
+    exec("createtable testers.2", true);
+    exec("deletetable -tn testers -f", true);
+    exec("tables", true, "testers.1", false);
+    exec("namespaces", true, "testers", true);
+    exec("deletenamespace testers -f", true);
     
+    // properties override and such
     exec("config -tn thing2 -s table.file.max=44444", true);
     exec("config -tn thing2", true, "44444", true);
     exec("config -t thing2.thingy", true, "44444", true);
     exec("config -t thing2.thingy -s table.file.max=55555", true);
     exec("config -t thing2.thingy", true, "55555", true);
     
+    // can copy properties when creating
     exec("createnamespace thing3 -cc thing2", true);
     exec("config -tn thing3", true, "44444", true);
     exec("createnamespace thing4 -ctc thing2.thingy", true);