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

[05/50] [abbrv] git commit: ACCUMULO-802 added namespace-specific randomwalk classes and updated randomwalk to include namespace operations. Changed the default randomwalk settings to 15 tables and 2 namespaces as opposed to 5 tables to help with namespa

ACCUMULO-802 added namespace-specific randomwalk classes and updated randomwalk to include namespace operations. Changed the default randomwalk settings to 15 tables and 2 namespaces as opposed to 5 tables to help with namespace testing"


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

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: be1ff27320871c05a762c28afd39bc4581416595
Parents: b28d4b3
Author: Sean Hickey <ta...@gmail.com>
Authored: Wed Jul 31 11:06:33 2013 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Wed Dec 4 18:46:09 2013 -0500

----------------------------------------------------------------------
 .../core/client/admin/TableOperationsImpl.java  |  4 +-
 .../apache/accumulo/master/tableOps/Utils.java  | 16 +++---
 .../test/randomwalk/concurrent/CloneTable.java  |  9 ++--
 .../concurrent/CloneTableNamespace.java         | 55 ++++++++++++++++++++
 .../test/randomwalk/concurrent/CreateTable.java |  9 ++--
 .../concurrent/CreateTableNamespace.java        | 48 +++++++++++++++++
 .../concurrent/DeleteTableNamespace.java        | 48 +++++++++++++++++
 .../test/randomwalk/concurrent/RenameTable.java |  9 ++--
 .../concurrent/RenameTableNamespace.java        | 52 ++++++++++++++++++
 .../test/randomwalk/concurrent/Setup.java       | 26 +++++++--
 .../randomwalk/conf/modules/Concurrent.xml      | 23 +++++++-
 11 files changed, 265 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/be1ff273/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 8965217..18d2e40 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
@@ -689,7 +689,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
     String namespace = Tables.extractNamespace(newTableName);
     if (!namespaceExists(namespace)) {
       String info = "Table namespace not found while cloning table";
-      throw new RuntimeException(new TableNamespaceNotFoundException(null, namespace, info));
+      throw new IllegalArgumentException(new TableNamespaceNotFoundException(null, namespace, info));
     }
     
     String srcTableId = Tables.getTableId(instance, srcTableName);
@@ -773,7 +773,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
     String namespace = Tables.extractNamespace(newTableName);
     if (!namespaceExists(namespace)) {
       String info = "Table namespace not found while renaming table";
-      throw new RuntimeException(new TableNamespaceNotFoundException(null, namespace, info));
+      throw new IllegalArgumentException(new TableNamespaceNotFoundException(null, namespace, info));
     }
     
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(oldTableName.getBytes()), ByteBuffer.wrap(newTableName.getBytes()));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/be1ff273/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java
index c01b5f0..58524f3 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java
@@ -88,6 +88,11 @@ public class Utils {
       return 100;
   }
   
+  public static void unreserveTable(String tableId, long tid, boolean writeLock) throws Exception {
+    getLock(tableId, tid, writeLock).unlock();
+    log.info("table " + tableId + " (" + Long.toHexString(tid) + ") unlocked for " + (writeLock ? "write" : "read"));
+  }
+  
   public static void unreserveTableNamespace(String namespaceId, long id, boolean writeLock) throws Exception {
     getLock(namespaceId, id, writeLock).unlock();
     log.info("table namespace " + namespaceId + " (" + Long.toHexString(id) + ") unlocked for " + (writeLock ? "write" : "read"));
@@ -98,7 +103,7 @@ public class Utils {
       if (mustExist) {
         Instance instance = HdfsZooInstance.getInstance();
         IZooReaderWriter zk = ZooReaderWriter.getRetryingInstance();
-        if (!zk.exists(ZooUtil.getRoot(instance) + Constants.ZNAMESPACES+ "/" + namespaceId))
+        if (!zk.exists(ZooUtil.getRoot(instance) + Constants.ZNAMESPACES + "/" + namespaceId))
           throw new ThriftTableOperationException(namespaceId, "", op, TableOperationExceptionType.NOTFOUND, "Table namespace does not exist");
       }
       log.info("table namespace " + namespaceId + " (" + Long.toHexString(id) + ") locked for " + (writeLock ? "write" : "read") + " operation: " + op);
@@ -107,11 +112,6 @@ public class Utils {
       return 100;
   }
   
-  public static void unreserveTable(String tableId, long tid, boolean writeLock) throws Exception {
-    getLock(tableId, tid, writeLock).unlock();
-    log.info("table " + tableId + " (" + Long.toHexString(tid) + ") unlocked for " + (writeLock ? "write" : "read"));
-  }
-  
   public static long reserveHdfsDirectory(String directory, long tid) throws KeeperException, InterruptedException {
     Instance instance = HdfsZooInstance.getInstance();
     
@@ -149,8 +149,8 @@ public class Utils {
     return Utils.getLock(tableId, tid, false);
   }
   
-  
-  static void checkTableNamespaceDoesNotExist(Instance instance, String namespace, String namespaceId, TableOperation operation) throws ThriftTableOperationException {
+  static void checkTableNamespaceDoesNotExist(Instance instance, String namespace, String namespaceId, TableOperation operation)
+      throws ThriftTableOperationException {
     
     String n = TableNamespaces.getNameToIdMap(instance).get(namespace);
     

http://git-wip-us.apache.org/repos/asf/accumulo/blob/be1ff273/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CloneTable.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CloneTable.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CloneTable.java
index fdbbc4e..f45c610 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CloneTable.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CloneTable.java
@@ -25,7 +25,6 @@ import java.util.Random;
 import org.apache.accumulo.core.client.Connector;
 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.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
@@ -45,16 +44,14 @@ public class CloneTable extends Test {
     boolean flush = rand.nextBoolean();
     
     try {
-      String n = Tables.extractNamespace(newTableName);
-      if (!conn.tableNamespaceOperations().exists(n)) {
-        conn.tableNamespaceOperations().create(n);
-      }
       log.debug("Cloning table " + srcTableName + " " + newTableName + " " + flush);
       conn.tableOperations().clone(srcTableName, newTableName, flush, new HashMap<String,String>(), new HashSet<String>());
     } catch (TableExistsException e) {
-      log.debug("Clone " + srcTableName + " failed, " + newTableName + " exist");
+      log.debug("Clone " + srcTableName + " failed, " + newTableName + " exists");
     } catch (TableNotFoundException e) {
       log.debug("Clone " + srcTableName + " failed, doesnt exist");
+    } catch (IllegalArgumentException e) {
+      log.debug("Clone: " + e.toString());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/be1ff273/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CloneTableNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CloneTableNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CloneTableNamespace.java
new file mode 100644
index 0000000..c6a923f
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CloneTableNamespace.java
@@ -0,0 +1,55 @@
+/*
+ * 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.test.randomwalk.concurrent;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.TableNamespaceExistsException;
+import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
+import org.apache.accumulo.test.randomwalk.State;
+import org.apache.accumulo.test.randomwalk.Test;
+
+public class CloneTableNamespace extends Test {
+  
+  @Override
+  public void visit(State state, Properties props) throws Exception {
+    Connector conn = state.getConnector();
+    
+    Random rand = (Random) state.get("rand");
+    
+    @SuppressWarnings("unchecked")
+    List<String> namespaces = (List<String>) state.get("namespaces");
+    
+    String srcName = namespaces.get(rand.nextInt(namespaces.size()));
+    String newName = namespaces.get(rand.nextInt(namespaces.size()));
+    boolean flush = rand.nextBoolean();
+    
+    try {
+      log.debug("Cloning table namespace " + srcName + " " + newName + " " + flush);
+      conn.tableNamespaceOperations().clone(srcName, newName, flush, new HashMap<String,String>(), new HashSet<String>(), true);
+    } catch (TableNamespaceExistsException e) {
+      log.debug("Clone namespace " + srcName + " failed, " + newName + " exists");
+    } catch (TableNamespaceNotFoundException e) {
+      log.debug("Clone namespace " + srcName + " failed, doesnt exist");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/be1ff273/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTable.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTable.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTable.java
index bbd18a9..b65fbaa 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTable.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTable.java
@@ -22,7 +22,6 @@ import java.util.Random;
 
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
@@ -40,14 +39,12 @@ public class CreateTable extends Test {
     String tableName = tableNames.get(rand.nextInt(tableNames.size()));
     
     try {
-      String n = Tables.extractNamespace(tableName);
-      if (!conn.tableNamespaceOperations().exists(n)) {
-        conn.tableNamespaceOperations().create(n);
-      }
       conn.tableOperations().create(tableName);
       log.debug("Created table " + tableName);
     } catch (TableExistsException e) {
-      log.debug("Create " + tableName + " failed, it exist");
+      log.debug("Create " + tableName + " failed, it exists");
+    } catch (IllegalArgumentException e) {
+      log.debug("Create: " + e.toString());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/be1ff273/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTableNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTableNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTableNamespace.java
new file mode 100644
index 0000000..749fa0f
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTableNamespace.java
@@ -0,0 +1,48 @@
+/*
+ * 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.test.randomwalk.concurrent;
+
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.TableNamespaceExistsException;
+import org.apache.accumulo.test.randomwalk.State;
+import org.apache.accumulo.test.randomwalk.Test;
+
+public class CreateTableNamespace extends Test {
+  
+  @Override
+  public void visit(State state, Properties props) throws Exception {
+    Connector conn = state.getConnector();
+    
+    Random rand = (Random) state.get("rand");
+    
+    @SuppressWarnings("unchecked")
+    List<String> namespaces = (List<String>) state.get("namespaces");
+    
+    String namespace = namespaces.get(rand.nextInt(namespaces.size()));
+    
+    try {
+      conn.tableNamespaceOperations().create(namespace);
+      log.debug("Created namespace " + namespace);
+    } catch (TableNamespaceExistsException e) {
+      log.debug("Create namespace " + namespace + " failed, it exists");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/be1ff273/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteTableNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteTableNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteTableNamespace.java
new file mode 100644
index 0000000..16ad555
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/DeleteTableNamespace.java
@@ -0,0 +1,48 @@
+/*
+ * 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.test.randomwalk.concurrent;
+
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
+import org.apache.accumulo.test.randomwalk.State;
+import org.apache.accumulo.test.randomwalk.Test;
+
+public class DeleteTableNamespace extends Test {
+  
+  @Override
+  public void visit(State state, Properties props) throws Exception {
+    Connector conn = state.getConnector();
+    
+    Random rand = (Random) state.get("rand");
+    
+    @SuppressWarnings("unchecked")
+    List<String> namespaces = (List<String>) state.get("namespaces");
+    
+    String namespace = namespaces.get(rand.nextInt(namespaces.size()));
+    
+    try {
+      conn.tableNamespaceOperations().delete(namespace, true);
+      log.debug("Deleted table namespace " + namespace);
+    } catch (TableNamespaceNotFoundException e) {
+      log.debug("Delete namespace " + namespace + " failed, doesnt exist");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/be1ff273/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTable.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTable.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTable.java
index 690d6cf..0e0b406 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTable.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTable.java
@@ -23,7 +23,6 @@ import java.util.Random;
 import org.apache.accumulo.core.client.Connector;
 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.test.randomwalk.State;
 import org.apache.accumulo.test.randomwalk.Test;
 
@@ -42,16 +41,14 @@ public class RenameTable extends Test {
     String newTableName = tableNames.get(rand.nextInt(tableNames.size()));
     
     try {
-      String n = Tables.extractNamespace(newTableName);
-      if (!conn.tableNamespaceOperations().exists(n)) {
-        conn.tableNamespaceOperations().create(n);
-      }
       conn.tableOperations().rename(srcTableName, newTableName);
       log.debug("Renamed table " + srcTableName + " " + newTableName);
     } catch (TableExistsException e) {
-      log.debug("Rename " + srcTableName + " failed, " + newTableName + " exist");
+      log.debug("Rename " + srcTableName + " failed, " + newTableName + " exists");
     } catch (TableNotFoundException e) {
       log.debug("Rename " + srcTableName + " failed, doesnt exist");
+    } catch (IllegalArgumentException e) {
+      log.debug("Rename: " + e.toString());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/be1ff273/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java
new file mode 100644
index 0000000..01329d0
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/RenameTableNamespace.java
@@ -0,0 +1,52 @@
+/*
+ * 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.test.randomwalk.concurrent;
+
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.TableNamespaceExistsException;
+import org.apache.accumulo.core.client.TableNamespaceNotFoundException;
+import org.apache.accumulo.test.randomwalk.State;
+import org.apache.accumulo.test.randomwalk.Test;
+
+public class RenameTableNamespace extends Test {
+  
+  @Override
+  public void visit(State state, Properties props) throws Exception {
+    Connector conn = state.getConnector();
+    
+    Random rand = (Random) state.get("rand");
+    
+    @SuppressWarnings("unchecked")
+    List<String> namespaces = (List<String>) state.get("tables");
+    
+    String srcName = namespaces.get(rand.nextInt(namespaces.size()));
+    String newName = namespaces.get(rand.nextInt(namespaces.size()));
+    
+    try {
+      conn.tableNamespaceOperations().rename(srcName, newName);
+      log.debug("Renamed table namespace " + srcName + " " + newName);
+    } catch (TableNamespaceExistsException e) {
+      log.debug("Rename namespace " + srcName + " failed, " + newName + " exists");
+    } catch (TableNamespaceNotFoundException e) {
+      log.debug("Rename namespace " + srcName + " failed, doesnt exist");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/be1ff273/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 976db87..7b077ee 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
@@ -31,15 +31,31 @@ public class Setup extends Test {
     Random rand = new Random();
     state.set("rand", rand);
     
-    int numTables = Integer.parseInt(props.getProperty("numTables", "5"));
+    int numTables = Integer.parseInt(props.getProperty("numTables", "15"));
+    int numNamespaces = Integer.parseInt(props.getProperty("numNamespaces", "2"));
     log.debug("numTables = " + numTables);
+    log.debug("numNamespaces = " + numNamespaces);
     List<String> tables = new ArrayList<String>();
-    for (int i = 0; i < numTables; i++) {
-      tables.add(String.format("ctt_%03d", i, i));
+    List<String> namespaces = new ArrayList<String>();
+    
+    for (int i = 0; i < numNamespaces; i++) {
+      namespaces.add(String.format("nspc_%03d", i));
     }
-    //tables.add(String.format("nspace.ctt_%03d", numTables - 1));
-
+    
+    double tableCeil = Math.ceil(numTables / (numNamespaces + 1));
+    for (int i = 0; i < tableCeil; i++) {
+      tables.add(String.format("ctt_%03d", i));
+    }
+    
+    double tableFloor = Math.floor(numTables / (numNamespaces + 1));
+    for (String n : namespaces) {
+      for (int i = 0; i < tableFloor; i++) {
+        tables.add(String.format(n + ".ctt_%03d", i));
+      }
+    }
+    
     state.set("tables", tables);
+    state.set("namespaces", namespaces);
     
     int numUsers = Integer.parseInt(props.getProperty("numUsers", "5"));
     log.debug("numUsers = " + numUsers);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/be1ff273/test/system/randomwalk/conf/modules/Concurrent.xml
----------------------------------------------------------------------
diff --git a/test/system/randomwalk/conf/modules/Concurrent.xml b/test/system/randomwalk/conf/modules/Concurrent.xml
index 03e5542..89b1a88 100644
--- a/test/system/randomwalk/conf/modules/Concurrent.xml
+++ b/test/system/randomwalk/conf/modules/Concurrent.xml
@@ -49,12 +49,17 @@
   <edge id="ct.StartAll" weight="1000"/>
   <edge id="ct.Shutdown" weight="10"/>
   <edge id="ct.Config" weight="1000"/>
+  <edge id="ct.CreateTableNamespace" weight="1000"/>
+  <edge id="ct.DeleteTableNamespace" weight="100"/>
+  <edge id="ct.RenameTableNamespace" weight="100"/>
+  <edge id="ct.CloneTableNamespace" weight="100"/>
   <edge id="ct.Apocalypse" weight="10"/>
   <edge id="END" weight="1"/>
 </node>
 
 <node id="ct.Setup">
-  <property key="numTables" value="5"/>
+  <property key="numTables" value="15"/>
+  <property key="numNamespaces" value="2"/>
   <edge id="ct.CreateTable" weight="1"/>
 </node>
 
@@ -162,4 +167,20 @@
   <edge id="ct.StartAll" weight="1"/>
 </node>
 
+<node id="ct.CreateTableNamespace">
+  <edge id="ct.StartAll" weight="1"/>
+</node>
+
+<node id="ct.DeleteTableNamespace">
+  <edge id="ct.StartAll" weight="1"/>
+</node>
+
+<node id="ct.RenameTableNamespace">
+  <edge id="ct.StartAll" weight="1"/>
+</node>
+
+<node id="ct.CloneTableNamespace">
+  <edge id="ct.StartAll" weight="1"/>
+</node>
+
 </module>