You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by mw...@apache.org on 2017/01/25 17:24:20 UTC

[1/5] accumulo-testing git commit: ACCUMULO-4510 Refactored Continous Ingest tests

Repository: accumulo-testing
Updated Branches:
  refs/heads/master b81229d78 -> fc3ddfc4f


http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/Validate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/Validate.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/Validate.java
index 8ad104a..32c815e 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/Validate.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/Validate.java
@@ -26,7 +26,7 @@ import org.apache.accumulo.core.client.security.SecurityErrorCode;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.slf4j.Logger;
@@ -34,12 +34,12 @@ import org.slf4j.Logger;
 public class Validate extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     validate(state, env, log);
   }
 
-  public static void validate(State state, Environment env, Logger log) throws Exception {
-    Connector conn = env.getConnector();
+  public static void validate(State state, RandWalkEnv env, Logger log) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     boolean tableExists = WalkingSecurity.get(state, env).getTableExists();
     boolean cloudTableExists = conn.tableOperations().list().contains(WalkingSecurity.get(state, env).getTableName());
@@ -53,9 +53,9 @@ public class Validate extends Test {
 
     Properties props = new Properties();
     props.setProperty("target", "system");
-    Authenticate.authenticate(env.getUserName(), env.getToken(), state, env, props);
+    Authenticate.authenticate(env.getAccumuloUserName(), env.getToken(), state, env, props);
     props.setProperty("target", "table");
-    Authenticate.authenticate(env.getUserName(), env.getToken(), state, env, props);
+    Authenticate.authenticate(env.getAccumuloUserName(), env.getToken(), state, env, props);
 
     for (String user : new String[] {WalkingSecurity.get(state, env).getSysUserName(), WalkingSecurity.get(state, env).getTabUserName()}) {
       for (SystemPermission sp : SystemPermission.values()) {

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/WalkingSecurity.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/WalkingSecurity.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/WalkingSecurity.java
index 302d6ec..7cf0ec5 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/WalkingSecurity.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/WalkingSecurity.java
@@ -47,7 +47,7 @@ import org.apache.accumulo.server.security.SecurityOperation;
 import org.apache.accumulo.server.security.handler.Authenticator;
 import org.apache.accumulo.server.security.handler.Authorizor;
 import org.apache.accumulo.server.security.handler.PermissionHandler;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.hadoop.fs.FileSystem;
 import org.slf4j.Logger;
@@ -58,7 +58,7 @@ import org.slf4j.LoggerFactory;
  */
 public class WalkingSecurity extends SecurityOperation implements Authorizor, Authenticator, PermissionHandler {
   State state = null;
-  Environment env = null;
+  RandWalkEnv env = null;
   private static final Logger log = LoggerFactory.getLogger(WalkingSecurity.class);
 
   private static final String tableName = "SecurityTableName";
@@ -82,7 +82,7 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
     super(context, author, authent, pm);
   }
 
-  public WalkingSecurity(State state2, Environment env2) {
+  public WalkingSecurity(State state2, RandWalkEnv env2) {
     super(new AccumuloServerContext(new ServerConfigurationFactory(HdfsZooInstance.getInstance())));
     this.state = state2;
     this.env = env2;
@@ -91,7 +91,7 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
     permHandle = this;
   }
 
-  public static WalkingSecurity get(State state, Environment env) {
+  public static WalkingSecurity get(State state, RandWalkEnv env) {
     if (instance == null || instance.state != state) {
       instance = new WalkingSecurity(state, env);
       state.set(tableExists, Boolean.toString(false));
@@ -359,11 +359,11 @@ public class WalkingSecurity extends SecurityOperation implements Authorizor, Au
   }
 
   public TCredentials getSysCredentials() {
-    return new Credentials(getSysUserName(), getSysToken()).toThrift(this.env.getInstance());
+    return new Credentials(getSysUserName(), getSysToken()).toThrift(this.env.getAccumuloInstance());
   }
 
   public TCredentials getTabCredentials() {
-    return new Credentials(getTabUserName(), getTabToken()).toThrift(this.env.getInstance());
+    return new Credentials(getTabUserName(), getTabToken()).toThrift(this.env.getAccumuloInstance());
   }
 
   public AuthenticationToken getSysToken() {

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/BatchVerify.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/BatchVerify.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/BatchVerify.java
index dc59b05..fc1f202 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/BatchVerify.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/BatchVerify.java
@@ -30,7 +30,7 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -38,7 +38,7 @@ import org.apache.hadoop.io.Text;
 public class BatchVerify extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
 
     Random rand = new Random();
 
@@ -50,7 +50,7 @@ public class BatchVerify extends Test {
       numVerify = numWrites / 4;
     }
 
-    Connector conn = env.getConnector();
+    Connector conn = env.getAccumuloConnector();
     BatchScanner scanner = conn.createBatchScanner(state.getString("seqTableName"), new Authorizations(), 2);
 
     try {

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/Commit.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/Commit.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/Commit.java
index 6865557..7ecd063 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/Commit.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/Commit.java
@@ -18,14 +18,14 @@ package org.apache.accumulo.testing.core.randomwalk.sequential;
 
 import java.util.Properties;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class Commit extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
 
     env.getMultiTableBatchWriter().flush();
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/MapRedVerify.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/MapRedVerify.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/MapRedVerify.java
index e17c98d..95b7d32 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/MapRedVerify.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/MapRedVerify.java
@@ -26,7 +26,7 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.testing.core.TestProps;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.util.ToolRunner;
@@ -34,16 +34,16 @@ import org.apache.hadoop.util.ToolRunner;
 public class MapRedVerify extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
 
     String[] args = new String[6];
-    args[0] = env.getUserName();
-    args[1] = env.getPassword();
+    args[0] = env.getAccumuloUserName();
+    args[1] = env.getAccumuloPassword();
     if (null == args[1]) {
-      args[1] = env.getKeytab();
+      args[1] = env.getAccumuloKeytab();
     }
     args[2] = state.getString("seqTableName");
-    args[3] = env.getInstance().getInstanceName();
+    args[3] = env.getAccumuloInstance().getInstanceName();
     args[4] = env.getConfigProperty(TestProps.ZOOKEEPERS);
     args[5] = args[2] + "_MR";
 
@@ -52,7 +52,7 @@ public class MapRedVerify extends Test {
       return;
     }
 
-    Scanner outputScanner = env.getConnector().createScanner(args[7], Authorizations.EMPTY);
+    Scanner outputScanner = env.getAccumuloConnector().createScanner(args[7], Authorizations.EMPTY);
     outputScanner.setRange(new Range());
 
     int count = 0;
@@ -71,7 +71,7 @@ public class MapRedVerify extends Test {
     }
 
     log.debug("Dropping table: " + args[5]);
-    Connector conn = env.getConnector();
+    Connector conn = env.getAccumuloConnector();
     conn.tableOperations().delete(args[5]);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/SequentialFixture.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/SequentialFixture.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/SequentialFixture.java
index de8af18..e6372d7 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/SequentialFixture.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/SequentialFixture.java
@@ -24,7 +24,7 @@ import org.apache.accumulo.core.client.MultiTableBatchWriter;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.Fixture;
 import org.apache.accumulo.testing.core.randomwalk.State;
 
@@ -33,10 +33,10 @@ public class SequentialFixture extends Fixture {
   String seqTableName;
 
   @Override
-  public void setUp(State state, Environment env) throws Exception {
+  public void setUp(State state, RandWalkEnv env) throws Exception {
 
-    Connector conn = env.getConnector();
-    Instance instance = env.getInstance();
+    Connector conn = env.getAccumuloConnector();
+    Instance instance = env.getAccumuloInstance();
 
     String hostname = InetAddress.getLocalHost().getHostName().replaceAll("[-.]", "_");
 
@@ -57,7 +57,7 @@ public class SequentialFixture extends Fixture {
   }
 
   @Override
-  public void tearDown(State state, Environment env) throws Exception {
+  public void tearDown(State state, RandWalkEnv env) throws Exception {
     // We have resources we need to clean up
     if (env.isMultiTableBatchWriterInitialized()) {
       MultiTableBatchWriter mtbw = env.getMultiTableBatchWriter();
@@ -73,7 +73,7 @@ public class SequentialFixture extends Fixture {
 
     log.debug("Dropping tables: " + seqTableName);
 
-    Connector conn = env.getConnector();
+    Connector conn = env.getAccumuloConnector();
 
     conn.tableOperations().delete(seqTableName);
   }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/Write.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/Write.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/Write.java
index 80e0e38..9394e28 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/Write.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/sequential/Write.java
@@ -23,7 +23,7 @@ import java.util.Properties;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -31,7 +31,7 @@ import org.apache.hadoop.io.Text;
 public class Write extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
 
     BatchWriter bw = env.getMultiTableBatchWriter().getBatchWriter(state.getString("seqTableName"));
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/BulkInsert.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/BulkInsert.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/BulkInsert.java
index 76b9ef6..f28bf8f 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/BulkInsert.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/BulkInsert.java
@@ -36,9 +36,8 @@ import org.apache.accumulo.core.data.ColumnUpdate;
 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.util.CachedConfiguration;
 import org.apache.accumulo.core.util.TextUtil;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.conf.Configuration;
@@ -96,7 +95,7 @@ public class BulkInsert extends Test {
   }
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
 
     String indexTableName = (String) state.get("indexTableName");
     String dataTableName = (String) state.get("docTableName");
@@ -139,14 +138,14 @@ public class BulkInsert extends Test {
     fs.delete(new Path(rootDir), true);
   }
 
-  private void bulkImport(FileSystem fs, State state, Environment env, String tableName, String rootDir, String prefix) throws Exception {
+  private void bulkImport(FileSystem fs, State state, RandWalkEnv env, String tableName, String rootDir, String prefix) throws Exception {
     while (true) {
       String bulkDir = rootDir + "/" + prefix + "_bulk";
       String failDir = rootDir + "/" + prefix + "_failure";
       Path failPath = new Path(failDir);
       fs.delete(failPath, true);
       fs.mkdirs(failPath);
-      env.getConnector().tableOperations().importDirectory(tableName, bulkDir, failDir, true);
+      env.getAccumuloConnector().tableOperations().importDirectory(tableName, bulkDir, failDir, true);
 
       FileStatus[] failures = fs.listStatus(failPath);
       if (failures != null && failures.length > 0) {
@@ -164,12 +163,12 @@ public class BulkInsert extends Test {
     }
   }
 
-  private void sort(State state, Environment env, FileSystem fs, String tableName, String seqFile, String outputDir, String workDir, int maxSplits)
+  private void sort(State state, RandWalkEnv env, FileSystem fs, String tableName, String seqFile, String outputDir, String workDir, int maxSplits)
       throws Exception {
 
     PrintStream out = new PrintStream(new BufferedOutputStream(fs.create(new Path(workDir + "/splits.txt"))), false, UTF_8.name());
 
-    Connector conn = env.getConnector();
+    Connector conn = env.getAccumuloConnector();
 
     Collection<Text> splits = conn.tableOperations().listSplits(tableName, maxSplits);
     for (Text split : splits)

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/CloneIndex.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/CloneIndex.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/CloneIndex.java
index c47d2a8..fb01d34 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/CloneIndex.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/CloneIndex.java
@@ -20,22 +20,22 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Properties;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class CloneIndex extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
 
     String indexTableName = (String) state.get("indexTableName");
     String tmpIndexTableName = indexTableName + "_tmp";
 
     long t1 = System.currentTimeMillis();
-    env.getConnector().tableOperations().flush(indexTableName, null, null, true);
+    env.getAccumuloConnector().tableOperations().flush(indexTableName, null, null, true);
     long t2 = System.currentTimeMillis();
-    env.getConnector().tableOperations().clone(indexTableName, tmpIndexTableName, false, new HashMap<String,String>(), new HashSet<String>());
+    env.getAccumuloConnector().tableOperations().clone(indexTableName, tmpIndexTableName, false, new HashMap<String,String>(), new HashSet<String>());
     long t3 = System.currentTimeMillis();
 
     log.debug("Cloned " + tmpIndexTableName + " from " + indexTableName + " flush: " + (t2 - t1) + "ms clone: " + (t3 - t2) + "ms");

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Commit.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Commit.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Commit.java
index 06e8b44..32bb6cf 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Commit.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Commit.java
@@ -18,14 +18,14 @@ package org.apache.accumulo.testing.core.randomwalk.shard;
 
 import java.util.Properties;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class Commit extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     env.getMultiTableBatchWriter().flush();
     log.debug("Committed inserts ");
   }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/CompactFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/CompactFilter.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/CompactFilter.java
index eacd36b..4fe6641 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/CompactFilter.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/CompactFilter.java
@@ -30,7 +30,7 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.RegExFilter;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -41,7 +41,7 @@ import org.apache.hadoop.io.Text;
 public class CompactFilter extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     String indexTableName = (String) state.get("indexTableName");
     String docTableName = (String) state.get("docTableName");
     Random rand = (Random) state.get("rand");
@@ -57,7 +57,7 @@ public class CompactFilter extends Test {
     documentFilters.add(is);
 
     long t1 = System.currentTimeMillis();
-    env.getConnector().tableOperations().compact(docTableName, null, null, documentFilters, true, true);
+    env.getAccumuloConnector().tableOperations().compact(docTableName, null, null, documentFilters, true, true);
     long t2 = System.currentTimeMillis();
     long t3 = t2 - t1;
 
@@ -69,12 +69,12 @@ public class CompactFilter extends Test {
     indexFilters.add(is);
 
     t1 = System.currentTimeMillis();
-    env.getConnector().tableOperations().compact(indexTableName, null, null, indexFilters, true, true);
+    env.getAccumuloConnector().tableOperations().compact(indexTableName, null, null, indexFilters, true, true);
     t2 = System.currentTimeMillis();
 
     log.debug("Filtered documents using compaction iterators " + regex + " " + (t3) + " " + (t2 - t1));
 
-    BatchScanner bscanner = env.getConnector().createBatchScanner(docTableName, new Authorizations(), 10);
+    BatchScanner bscanner = env.getAccumuloConnector().createBatchScanner(docTableName, new Authorizations(), 10);
 
     List<Range> ranges = new ArrayList<>();
     for (int i = 0; i < 16; i++) {

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Delete.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Delete.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Delete.java
index e2c8bea..6cd24ca 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Delete.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Delete.java
@@ -23,14 +23,14 @@ import java.util.Random;
 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.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class Delete extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     String indexTableName = (String) state.get("indexTableName");
     String dataTableName = (String) state.get("docTableName");
     int numPartitions = (Integer) state.get("numPartitions");

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/DeleteSomeDocs.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/DeleteSomeDocs.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/DeleteSomeDocs.java
index 2b790bd..4f6361e 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/DeleteSomeDocs.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/DeleteSomeDocs.java
@@ -27,7 +27,7 @@ import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.iterators.user.RegExFilter;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
@@ -35,7 +35,7 @@ import org.apache.accumulo.testing.core.randomwalk.Test;
 public class DeleteSomeDocs extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     // delete documents that where the document id matches a given pattern from doc and index table
     // using the batch deleter
 
@@ -51,7 +51,7 @@ public class DeleteSomeDocs extends Test {
 
     String pattern = patterns.get(rand.nextInt(patterns.size()));
     BatchWriterConfig bwc = new BatchWriterConfig();
-    BatchDeleter ibd = env.getConnector().createBatchDeleter(indexTableName, Authorizations.EMPTY, 8, bwc);
+    BatchDeleter ibd = env.getAccumuloConnector().createBatchDeleter(indexTableName, Authorizations.EMPTY, 8, bwc);
     ibd.setRanges(Collections.singletonList(new Range()));
 
     IteratorSetting iterSettings = new IteratorSetting(100, RegExFilter.class);
@@ -63,7 +63,7 @@ public class DeleteSomeDocs extends Test {
 
     ibd.close();
 
-    BatchDeleter dbd = env.getConnector().createBatchDeleter(dataTableName, Authorizations.EMPTY, 8, bwc);
+    BatchDeleter dbd = env.getAccumuloConnector().createBatchDeleter(dataTableName, Authorizations.EMPTY, 8, bwc);
     dbd.setRanges(Collections.singletonList(new Range()));
 
     iterSettings = new IteratorSetting(100, RegExFilter.class);

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/DeleteWord.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/DeleteWord.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/DeleteWord.java
index 544c35e..b380dde 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/DeleteWord.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/DeleteWord.java
@@ -29,7 +29,7 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -42,7 +42,7 @@ import org.apache.hadoop.io.Text;
 public class DeleteWord extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     String indexTableName = (String) state.get("indexTableName");
     String docTableName = (String) state.get("docTableName");
     int numPartitions = (Integer) state.get("numPartitions");
@@ -51,7 +51,7 @@ public class DeleteWord extends Test {
     String wordToDelete = Insert.generateRandomWord(rand);
 
     // use index to find all documents containing word
-    Scanner scanner = env.getConnector().createScanner(indexTableName, Authorizations.EMPTY);
+    Scanner scanner = env.getAccumuloConnector().createScanner(indexTableName, Authorizations.EMPTY);
     scanner.fetchColumnFamily(new Text(wordToDelete));
 
     ArrayList<Range> documentsToDelete = new ArrayList<>();
@@ -61,7 +61,7 @@ public class DeleteWord extends Test {
 
     if (documentsToDelete.size() > 0) {
       // use a batch scanner to fetch all documents
-      BatchScanner bscanner = env.getConnector().createBatchScanner(docTableName, Authorizations.EMPTY, 8);
+      BatchScanner bscanner = env.getAccumuloConnector().createBatchScanner(docTableName, Authorizations.EMPTY, 8);
       bscanner.setRanges(documentsToDelete);
 
       BatchWriter ibw = env.getMultiTableBatchWriter().getBatchWriter(indexTableName);

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/ExportIndex.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/ExportIndex.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/ExportIndex.java
index 0e4853d..5f521d3 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/ExportIndex.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/ExportIndex.java
@@ -26,8 +26,7 @@ import java.util.Map.Entry;
 import java.util.Properties;
 
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.util.CachedConfiguration;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.fs.FileSystem;
@@ -41,7 +40,7 @@ import org.apache.hadoop.io.Text;
 public class ExportIndex extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
 
     String indexTableName = (String) state.get("indexTableName");
     String tmpIndexTableName = indexTableName + "_tmp";
@@ -55,16 +54,16 @@ public class ExportIndex extends Test {
     fs.delete(new Path("/tmp/shard_export/" + tmpIndexTableName), true);
 
     // disable spits, so that splits can be compared later w/o worrying one table splitting and the other not
-    env.getConnector().tableOperations().setProperty(indexTableName, Property.TABLE_SPLIT_THRESHOLD.getKey(), "20G");
+    env.getAccumuloConnector().tableOperations().setProperty(indexTableName, Property.TABLE_SPLIT_THRESHOLD.getKey(), "20G");
 
     long t1 = System.currentTimeMillis();
 
-    env.getConnector().tableOperations().flush(indexTableName, null, null, true);
-    env.getConnector().tableOperations().offline(indexTableName);
+    env.getAccumuloConnector().tableOperations().flush(indexTableName, null, null, true);
+    env.getAccumuloConnector().tableOperations().offline(indexTableName);
 
     long t2 = System.currentTimeMillis();
 
-    env.getConnector().tableOperations().exportTable(indexTableName, exportDir);
+    env.getAccumuloConnector().tableOperations().exportTable(indexTableName, exportDir);
 
     long t3 = System.currentTimeMillis();
 
@@ -81,34 +80,34 @@ public class ExportIndex extends Test {
 
     long t4 = System.currentTimeMillis();
 
-    env.getConnector().tableOperations().online(indexTableName);
-    env.getConnector().tableOperations().importTable(tmpIndexTableName, copyDir);
+    env.getAccumuloConnector().tableOperations().online(indexTableName);
+    env.getAccumuloConnector().tableOperations().importTable(tmpIndexTableName, copyDir);
 
     long t5 = System.currentTimeMillis();
 
     fs.delete(new Path(exportDir), true);
     fs.delete(new Path(copyDir), true);
 
-    HashSet<Text> splits1 = new HashSet<>(env.getConnector().tableOperations().listSplits(indexTableName));
-    HashSet<Text> splits2 = new HashSet<>(env.getConnector().tableOperations().listSplits(tmpIndexTableName));
+    HashSet<Text> splits1 = new HashSet<>(env.getAccumuloConnector().tableOperations().listSplits(indexTableName));
+    HashSet<Text> splits2 = new HashSet<>(env.getAccumuloConnector().tableOperations().listSplits(tmpIndexTableName));
 
     if (!splits1.equals(splits2))
       throw new Exception("Splits not equals " + indexTableName + " " + tmpIndexTableName);
 
     HashMap<String,String> props1 = new HashMap<>();
-    for (Entry<String,String> entry : env.getConnector().tableOperations().getProperties(indexTableName))
+    for (Entry<String,String> entry : env.getAccumuloConnector().tableOperations().getProperties(indexTableName))
       props1.put(entry.getKey(), entry.getValue());
 
     HashMap<String,String> props2 = new HashMap<>();
-    for (Entry<String,String> entry : env.getConnector().tableOperations().getProperties(tmpIndexTableName))
+    for (Entry<String,String> entry : env.getAccumuloConnector().tableOperations().getProperties(tmpIndexTableName))
       props2.put(entry.getKey(), entry.getValue());
 
     if (!props1.equals(props2))
       throw new Exception("Props not equals " + indexTableName + " " + tmpIndexTableName);
 
     // unset the split threshold
-    env.getConnector().tableOperations().removeProperty(indexTableName, Property.TABLE_SPLIT_THRESHOLD.getKey());
-    env.getConnector().tableOperations().removeProperty(tmpIndexTableName, Property.TABLE_SPLIT_THRESHOLD.getKey());
+    env.getAccumuloConnector().tableOperations().removeProperty(indexTableName, Property.TABLE_SPLIT_THRESHOLD.getKey());
+    env.getAccumuloConnector().tableOperations().removeProperty(tmpIndexTableName, Property.TABLE_SPLIT_THRESHOLD.getKey());
 
     log.debug("Imported " + tmpIndexTableName + " from " + indexTableName + " flush: " + (t2 - t1) + "ms export: " + (t3 - t2) + "ms copy:" + (t4 - t3)
         + "ms import:" + (t5 - t4) + "ms");

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Flush.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Flush.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Flush.java
index e6ac574..f8d5183 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Flush.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Flush.java
@@ -19,14 +19,14 @@ package org.apache.accumulo.testing.core.randomwalk.shard;
 import java.util.Properties;
 import java.util.Random;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class Flush extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     String indexTableName = (String) state.get("indexTableName");
     String dataTableName = (String) state.get("docTableName");
     Random rand = (Random) state.get("rand");
@@ -38,7 +38,7 @@ public class Flush extends Test {
     else
       table = dataTableName;
 
-    env.getConnector().tableOperations().flush(table, null, null, true);
+    env.getAccumuloConnector().tableOperations().flush(table, null, null, true);
     log.debug("Flushed " + table);
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Grep.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Grep.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Grep.java
index 5892626..b8a79e5 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Grep.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Grep.java
@@ -31,7 +31,7 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.IntersectingIterator;
 import org.apache.accumulo.core.iterators.user.RegExFilter;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -39,7 +39,7 @@ import org.apache.hadoop.io.Text;
 public class Grep extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     // pick a few randoms words... grep for those words and search the index
     // ensure both return the same set of documents
 
@@ -53,7 +53,7 @@ public class Grep extends Test {
       words[i] = new Text(Insert.generateRandomWord(rand));
     }
 
-    BatchScanner bs = env.getConnector().createBatchScanner(indexTableName, Authorizations.EMPTY, 16);
+    BatchScanner bs = env.getAccumuloConnector().createBatchScanner(indexTableName, Authorizations.EMPTY, 16);
     IteratorSetting ii = new IteratorSetting(20, "ii", IntersectingIterator.class.getName());
     IntersectingIterator.setColumnFamilies(ii, words);
     bs.addScanIterator(ii);
@@ -67,7 +67,7 @@ public class Grep extends Test {
 
     bs.close();
 
-    bs = env.getConnector().createBatchScanner(dataTableName, Authorizations.EMPTY, 16);
+    bs = env.getAccumuloConnector().createBatchScanner(dataTableName, Authorizations.EMPTY, 16);
 
     for (int i = 0; i < words.length; i++) {
       IteratorSetting more = new IteratorSetting(20 + i, "ii" + i, RegExFilter.class);

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Insert.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Insert.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Insert.java
index 1b15323..9f480eb 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Insert.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Insert.java
@@ -26,7 +26,7 @@ import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
@@ -37,7 +37,7 @@ public class Insert extends Test {
   static final int MAX_WORDS_PER_DOC = 3000;
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     String indexTableName = (String) state.get("indexTableName");
     String dataTableName = (String) state.get("docTableName");
     int numPartitions = (Integer) state.get("numPartitions");

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Merge.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Merge.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Merge.java
index 106ab3b..afd36fd 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Merge.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Merge.java
@@ -21,7 +21,7 @@ import java.util.Properties;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -29,16 +29,16 @@ import org.apache.hadoop.io.Text;
 public class Merge extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     String indexTableName = (String) state.get("indexTableName");
 
-    Collection<Text> splits = env.getConnector().tableOperations().listSplits(indexTableName);
+    Collection<Text> splits = env.getAccumuloConnector().tableOperations().listSplits(indexTableName);
     SortedSet<Text> splitSet = new TreeSet<>(splits);
     log.debug("merging " + indexTableName);
-    env.getConnector().tableOperations().merge(indexTableName, null, null);
+    env.getAccumuloConnector().tableOperations().merge(indexTableName, null, null);
     org.apache.accumulo.core.util.Merge merge = new org.apache.accumulo.core.util.Merge();
-    merge.mergomatic(env.getConnector(), indexTableName, null, null, 256 * 1024 * 1024, true);
-    splits = env.getConnector().tableOperations().listSplits(indexTableName);
+    merge.mergomatic(env.getAccumuloConnector(), indexTableName, null, null, 256 * 1024 * 1024, true);
+    splits = env.getAccumuloConnector().tableOperations().listSplits(indexTableName);
     if (splits.size() > splitSet.size()) {
       // throw an excpetion so that test will die an no further changes to table will occur...
       // this way table is left as is for debugging.

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Reindex.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Reindex.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Reindex.java
index ac0c872..95fa6b8 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Reindex.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Reindex.java
@@ -26,14 +26,14 @@ import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class Reindex extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     String indexTableName = (String) state.get("indexTableName");
     String tmpIndexTableName = indexTableName + "_tmp";
     String docTableName = (String) state.get("docTableName");
@@ -43,8 +43,8 @@ public class Reindex extends Test {
 
     ShardFixture.createIndexTable(this.log, state, env, "_tmp", rand);
 
-    Scanner scanner = env.getConnector().createScanner(docTableName, Authorizations.EMPTY);
-    BatchWriter tbw = env.getConnector().createBatchWriter(tmpIndexTableName, new BatchWriterConfig());
+    Scanner scanner = env.getAccumuloConnector().createScanner(docTableName, Authorizations.EMPTY);
+    BatchWriter tbw = env.getAccumuloConnector().createBatchWriter(tmpIndexTableName, new BatchWriterConfig());
 
     int count = 0;
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Search.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Search.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Search.java
index c07397d..440469f 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Search.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Search.java
@@ -31,7 +31,7 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.IntersectingIterator;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -39,7 +39,7 @@ import org.apache.hadoop.io.Text;
 public class Search extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     String indexTableName = (String) state.get("indexTableName");
     String dataTableName = (String) state.get("docTableName");
 
@@ -69,7 +69,7 @@ public class Search extends Test {
 
     log.debug("Looking up terms " + searchTerms + " expect to find " + docID);
 
-    BatchScanner bs = env.getConnector().createBatchScanner(indexTableName, Authorizations.EMPTY, 10);
+    BatchScanner bs = env.getAccumuloConnector().createBatchScanner(indexTableName, Authorizations.EMPTY, 10);
     IteratorSetting ii = new IteratorSetting(20, "ii", IntersectingIterator.class);
     IntersectingIterator.setColumnFamilies(ii, columns);
     bs.addScanIterator(ii);
@@ -90,8 +90,8 @@ public class Search extends Test {
       throw new Exception("Did not see doc " + docID + " in index.  terms:" + searchTerms + " " + indexTableName + " " + dataTableName);
   }
 
-  static Entry<Key,Value> findRandomDocument(State state, Environment env, String dataTableName, Random rand) throws Exception {
-    Scanner scanner = env.getConnector().createScanner(dataTableName, Authorizations.EMPTY);
+  static Entry<Key,Value> findRandomDocument(State state, RandWalkEnv env, String dataTableName, Random rand) throws Exception {
+    Scanner scanner = env.getAccumuloConnector().createScanner(dataTableName, Authorizations.EMPTY);
     scanner.setBatchSize(1);
     scanner.setRange(new Range(Integer.toString(rand.nextInt(0xfffffff), 16), null));
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/ShardFixture.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/ShardFixture.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/ShardFixture.java
index 3462442..2d83326 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/ShardFixture.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/ShardFixture.java
@@ -25,7 +25,7 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.MultiTableBatchWriter;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.Fixture;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.hadoop.io.Text;
@@ -49,8 +49,8 @@ public class ShardFixture extends Fixture {
     return splits;
   }
 
-  static void createIndexTable(Logger log, State state, Environment env, String suffix, Random rand) throws Exception {
-    Connector conn = env.getConnector();
+  static void createIndexTable(Logger log, State state, RandWalkEnv env, String suffix, Random rand) throws Exception {
+    Connector conn = env.getAccumuloConnector();
     String name = (String) state.get("indexTableName") + suffix;
     int numPartitions = (Integer) state.get("numPartitions");
     boolean enableCache = (Boolean) state.get("cacheIndex");
@@ -73,7 +73,7 @@ public class ShardFixture extends Fixture {
   }
 
   @Override
-  public void setUp(State state, Environment env) throws Exception {
+  public void setUp(State state, RandWalkEnv env) throws Exception {
     String hostname = InetAddress.getLocalHost().getHostName().replaceAll("[-.]", "_");
     String pid = env.getPid();
 
@@ -88,7 +88,7 @@ public class ShardFixture extends Fixture {
     state.set("rand", rand);
     state.set("nextDocID", Long.valueOf(0));
 
-    Connector conn = env.getConnector();
+    Connector conn = env.getAccumuloConnector();
 
     createIndexTable(this.log, state, env, "", rand);
 
@@ -110,7 +110,7 @@ public class ShardFixture extends Fixture {
   }
 
   @Override
-  public void tearDown(State state, Environment env) throws Exception {
+  public void tearDown(State state, RandWalkEnv env) throws Exception {
     // We have resources we need to clean up
     if (env.isMultiTableBatchWriterInitialized()) {
       MultiTableBatchWriter mtbw = env.getMultiTableBatchWriter();
@@ -124,7 +124,7 @@ public class ShardFixture extends Fixture {
       env.resetMultiTableBatchWriter();
     }
 
-    Connector conn = env.getConnector();
+    Connector conn = env.getAccumuloConnector();
 
     log.info("Deleting index and doc tables");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Split.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Split.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Split.java
index bef5104..d150e99 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Split.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/Split.java
@@ -20,7 +20,7 @@ import java.util.Properties;
 import java.util.Random;
 import java.util.SortedSet;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -28,14 +28,14 @@ import org.apache.hadoop.io.Text;
 public class Split extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     String indexTableName = (String) state.get("indexTableName");
     int numPartitions = (Integer) state.get("numPartitions");
     Random rand = (Random) state.get("rand");
 
     SortedSet<Text> splitSet = ShardFixture.genSplits(numPartitions, rand.nextInt(numPartitions) + 1, "%06x");
     log.debug("adding splits " + indexTableName);
-    env.getConnector().tableOperations().addSplits(indexTableName, splitSet);
+    env.getAccumuloConnector().tableOperations().addSplits(indexTableName, splitSet);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/VerifyIndex.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/VerifyIndex.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/VerifyIndex.java
index caba1d7..cd1a1cd 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/VerifyIndex.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/shard/VerifyIndex.java
@@ -25,21 +25,21 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class VerifyIndex extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
 
     String indexTableName = (String) state.get("indexTableName");
     String tmpIndexTableName = indexTableName + "_tmp";
 
     // scan new and old index and verify identical
-    Scanner indexScanner1 = env.getConnector().createScanner(tmpIndexTableName, Authorizations.EMPTY);
-    Scanner indexScanner2 = env.getConnector().createScanner(indexTableName, Authorizations.EMPTY);
+    Scanner indexScanner1 = env.getAccumuloConnector().createScanner(tmpIndexTableName, Authorizations.EMPTY);
+    Scanner indexScanner2 = env.getAccumuloConnector().createScanner(indexTableName, Authorizations.EMPTY);
 
     Iterator<Entry<Key,Value>> iter = indexScanner2.iterator();
 
@@ -64,8 +64,8 @@ public class VerifyIndex extends Test {
 
     log.debug("Verified " + count + " index entries ");
 
-    env.getConnector().tableOperations().delete(indexTableName);
-    env.getConnector().tableOperations().rename(tmpIndexTableName, indexTableName);
+    env.getAccumuloConnector().tableOperations().delete(indexTableName);
+    env.getAccumuloConnector().tableOperations().rename(tmpIndexTableName, indexTableName);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/CreateTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/CreateTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/CreateTable.java
index df1df59..1282993 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/CreateTable.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/CreateTable.java
@@ -19,12 +19,12 @@ package org.apache.accumulo.testing.core.randomwalk.unit;
 
 import java.util.Properties;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class CreateTable extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {}
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {}
 }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/DeleteTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/DeleteTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/DeleteTable.java
index f7226cb..af90276 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/DeleteTable.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/DeleteTable.java
@@ -18,12 +18,12 @@ package org.apache.accumulo.testing.core.randomwalk.unit;
 
 import java.util.Properties;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class DeleteTable extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {}
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {}
 }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/Ingest.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/Ingest.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/Ingest.java
index 5681402..1b37baf 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/Ingest.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/Ingest.java
@@ -18,12 +18,12 @@ package org.apache.accumulo.testing.core.randomwalk.unit;
 
 import java.util.Properties;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class Ingest extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {}
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {}
 }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/Scan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/Scan.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/Scan.java
index c677cf9..8f6bdae 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/Scan.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/Scan.java
@@ -18,12 +18,12 @@ package org.apache.accumulo.testing.core.randomwalk.unit;
 
 import java.util.Properties;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class Scan extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {}
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {}
 }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/Verify.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/Verify.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/Verify.java
index 95acf4f..4dd5f29 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/Verify.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/unit/Verify.java
@@ -18,12 +18,12 @@ package org.apache.accumulo.testing.core.randomwalk.unit;
 
 import java.util.Properties;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class Verify extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {}
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {}
 }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/test/java/org/apache/accumulo/testing/core/randomwalk/ReplicationRandomWalkIT.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/testing/core/randomwalk/ReplicationRandomWalkIT.java b/core/src/test/java/org/apache/accumulo/testing/core/randomwalk/ReplicationRandomWalkIT.java
index c288bd7..6a6e713 100644
--- a/core/src/test/java/org/apache/accumulo/testing/core/randomwalk/ReplicationRandomWalkIT.java
+++ b/core/src/test/java/org/apache/accumulo/testing/core/randomwalk/ReplicationRandomWalkIT.java
@@ -43,19 +43,19 @@ public class ReplicationRandomWalkIT extends ConfigurableMacBase {
   public void runReplicationRandomWalkStep() throws Exception {
     Replication r = new Replication();
 
-    Environment env = new Environment(new Properties()) {
+    RandWalkEnv env = new RandWalkEnv(new Properties()) {
       @Override
-      public String getUserName() {
+      public String getAccumuloUserName() {
         return "root";
       }
 
       @Override
-      public String getPassword() {
+      public String getAccumuloPassword() {
         return ROOT_PASSWORD;
       }
 
       @Override
-      public Connector getConnector() throws AccumuloException, AccumuloSecurityException {
+      public Connector getAccumuloConnector() throws AccumuloException, AccumuloSecurityException {
         return ReplicationRandomWalkIT.this.getConnector();
       }
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/libexec/analyze-missing.pl
----------------------------------------------------------------------
diff --git a/libexec/analyze-missing.pl b/libexec/analyze-missing.pl
new file mode 100755
index 0000000..5cce1b1
--- /dev/null
+++ b/libexec/analyze-missing.pl
@@ -0,0 +1,127 @@
+#! /usr/bin/env perl
+
+# 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.
+
+
+use POSIX qw(strftime);
+
+if(scalar(@ARGV) != 4){
+	print "Usage : analyze-missing.pl <accumulo home> <continuous log dir> <user> <pass> \n";
+	exit(1);
+}
+
+$ACCUMULO_HOME=$ARGV[0];
+$CONTINUOUS_LOG_DIR=$ARGV[1];
+$USER=$ARGV[2];
+$PASS=$ARGV[3];
+
+
+@missing = `grep MIS $CONTINUOUS_LOG_DIR/*.err`;
+
+
+
+for $miss (@missing) {
+	chomp($miss);
+	($file, $type, $time, $row) = split(/[: ]/, $miss);
+
+	substr($file, -3, 3, "out");
+
+	$prevRowLine = `grep -B 1 $row $file | grep SRQ | grep -v $row`;
+
+	@prla = split(/\s+/, $prevRowLine);
+	$prevRow = $prla[2];
+#	print $prevRow."\n";
+
+	$aScript = `mktemp /tmp/miss_script.XXXXXXXXXX`;
+	chomp($aScript);
+	open(AS, ">$aScript") || die;
+
+	print AS "table ci\n";
+	print AS "scan -b $prevRow -e $prevRow\n";
+	print AS "scan -b $row -e $row\n";
+	print AS "quit\n";
+	close(AS);
+
+	$exist = 0;
+	$ingestIDSame = 0;
+	$ingestId = "";
+	$count = 0;
+
+	@entries = `$ACCUMULO_HOME/bin/accumulo shell -u $USER -p $PASS -f $aScript | grep $row`;
+	system("rm $aScript");
+
+	for $entry (@entries){
+		chomp($entry);
+		@entryA = split(/[: ]+/, $entry);
+		if($entryA[0] eq $row){
+			$exist = 1;
+
+			if($entryA[4] eq $ingestId){
+				$ingestIDSame = 1;
+			}
+		}else{
+			$ingestId = $entryA[4];
+			$count = hex($entryA[5]);
+		}
+	}
+
+
+	#look in ingest logs
+	@ingestLogs = `ls  $CONTINUOUS_LOG_DIR/*ingest*.out`;
+	@flushTimes = ();
+	chomp(@ingestLogs);
+	for $ingestLog (@ingestLogs){
+		open(IL, "<$ingestLog") || die;
+		
+
+		while($firstLine = <IL>){
+			chomp($firstLine);
+			if($firstLine =~ /UUID.*/){
+				last;
+			}
+		}
+
+		@iinfo = split(/\s+/,$firstLine);
+		if($iinfo[2] eq $ingestId){
+			while($line = <IL>){
+				if($line =~ /FLUSH (\d+) \d+ \d+ (\d+) \d+/){
+					push(@flushTimes, $1);
+					if(scalar(@flushTimes) > 3){
+						shift(@flushTimes);
+					}
+					if($count < $2){
+						last;
+					}
+				}
+			}
+		}
+		
+		
+
+		close(IL);
+	
+		if(scalar(@flushTimes) > 0){
+			last;
+		}
+	} 
+
+	$its0 = strftime "%m/%d/%Y_%H:%M:%S", gmtime($flushTimes[0]/1000);
+	$its1 = strftime "%m/%d/%Y_%H:%M:%S", gmtime($flushTimes[1]/1000);
+	$mts = strftime "%m/%d/%Y_%H:%M:%S", gmtime($time/1000);
+
+	print "$row $exist $ingestIDSame $prevRow $ingestId   $its0   $its1   $mts\n";
+}
+

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/libexec/datanode-agitator.pl
----------------------------------------------------------------------
diff --git a/libexec/datanode-agitator.pl b/libexec/datanode-agitator.pl
new file mode 100755
index 0000000..a98bb66
--- /dev/null
+++ b/libexec/datanode-agitator.pl
@@ -0,0 +1,140 @@
+#! /usr/bin/env perl
+
+# 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.
+
+
+use POSIX qw(strftime);
+use Cwd qw();
+
+if(scalar(@ARGV) != 5 && scalar(@ARGV) != 3){
+  print "Usage : datanode-agitator.pl <min sleep before kill in minutes>[:max sleep before kill in minutes] <min sleep before restart in minutes>[:max sleep before restart in minutes] HADOOP_PREFIX [<min kill> <max kill>]\n";
+  exit(1);
+}
+
+my $ACCUMULO_HOME;
+if( defined $ENV{'ACCUMULO_HOME'} ){
+  $ACCUMULO_HOME = $ENV{'ACCUMULO_HOME'};
+} else {
+  $cwd=Cwd::cwd();
+  $ACCUMULO_HOME=$cwd . '/../../..';
+}
+$HADOOP_PREFIX=$ARGV[2];
+
+print "ACCUMULO_HOME=$ACCUMULO_HOME\n";
+print "HADOOP_PREFIX=$HADOOP_PREFIX\n";
+
+@sleeprange1 = split(/:/, $ARGV[0]);
+$sleep1 = $sleeprange1[0];
+
+@sleeprange2 = split(/:/, $ARGV[1]);
+$sleep2 = $sleeprange2[0];
+
+if (scalar(@sleeprange1) > 1) {
+  $sleep1max = $sleeprange1[1] + 1;
+} else {
+  $sleep1max = $sleep1;
+}
+
+if ($sleep1 > $sleep1max) {
+  die("sleep1 > sleep1max $sleep1 > $sleep1max");
+}
+
+if (scalar(@sleeprange2) > 1) {
+  $sleep2max = $sleeprange2[1] + 1;
+} else {
+  $sleep2max = $sleep2;
+}
+
+if($sleep2 > $sleep2max){
+  die("sleep2 > sleep2max $sleep2 > $sleep2max");
+}
+
+if(defined $ENV{'ACCUMULO_CONF_DIR'}){
+  $ACCUMULO_CONF_DIR = $ENV{'ACCUMULO_CONF_DIR'};
+}else{
+  $ACCUMULO_CONF_DIR = $ACCUMULO_HOME . '/conf';
+}
+
+if(scalar(@ARGV) == 5){
+  $minKill = $ARGV[3];
+  $maxKill = $ARGV[4];
+}else{
+  $minKill = 1;
+  $maxKill = 1;
+}
+
+if($minKill > $maxKill){
+  die("minKill > maxKill $minKill > $maxKill");
+}
+
+@tserversRaw = `cat $ACCUMULO_CONF_DIR/tservers`;
+chomp(@tserversRaw);
+
+for $tserver (@tserversRaw){
+  if($tserver eq "" || substr($tserver,0,1) eq "#"){
+    next;
+  }
+
+  push(@tservers, $tserver);
+}
+
+
+if(scalar(@tservers) < $maxKill){
+  print STDERR "WARN setting maxKill to ".scalar(@tservers)."\n";
+  $maxKill = scalar(@tservers);
+}
+
+if ($minKill > $maxKill){
+  print STDERR "WARN setting minKill to equal maxKill\n";
+  $minKill = $maxKill;
+}
+
+while(1){
+
+  $numToKill = int(rand($maxKill - $minKill + 1)) + $minKill;
+  %killed = ();
+  $server = "";
+
+  for($i = 0; $i < $numToKill; $i++){
+    while($server eq "" || $killed{$server} != undef){
+      $index = int(rand(scalar(@tservers)));
+      $server = $tservers[$index];
+    }
+
+    $killed{$server} = 1;
+
+    $t = strftime "%Y%m%d %H:%M:%S", localtime;
+
+    print STDERR "$t Killing datanode on $server\n";
+    system("ssh $server \"pkill -9 -f '[p]roc_datanode'\"");
+  }
+
+  $nextsleep2 = int(rand($sleep2max - $sleep2)) + $sleep2;
+  sleep($nextsleep2 * 60);
+
+  foreach $restart (keys %killed) {
+
+    $t = strftime "%Y%m%d %H:%M:%S", localtime;
+
+    print STDERR "$t Starting datanode on $restart\n";
+    # We can just start as we're the HDFS user
+    system("ssh $restart '$HADOOP_PREFIX/sbin/hadoop-daemon.sh start datanode'");
+  }
+
+  $nextsleep1 = int(rand($sleep1max - $sleep1)) + $sleep1;
+  sleep($nextsleep1 * 60);
+}
+

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/libexec/hdfs-agitator.pl
----------------------------------------------------------------------
diff --git a/libexec/hdfs-agitator.pl b/libexec/hdfs-agitator.pl
new file mode 100755
index 0000000..85eab32
--- /dev/null
+++ b/libexec/hdfs-agitator.pl
@@ -0,0 +1,217 @@
+#! /usr/bin/env perl
+
+# 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.
+
+use strict;
+use warnings;
+use POSIX qw(strftime);
+use Getopt::Long;
+use Pod::Usage;
+
+my $help = 0;
+my $man = 0;
+my $sleep = 10;
+my $superuser = 'hdfs';
+my $hdfsCmd;
+if( defined $ENV{'HADOOP_PREFIX'} ){
+  $hdfsCmd = $ENV{'HADOOP_PREFIX'} . '/share/hadoop/hdfs/bin/hdfs';
+}
+my $sudo;
+my $nameservice;
+
+GetOptions('help|?' => \$help, 'man' => \$man, 'sleep=i' => \$sleep, 'nameservice=s' => \$nameservice, 'superuser=s' => \$superuser, 'hdfs-cmd=s' => \$hdfsCmd, 'sudo:s' => \$sudo) or pod2usage(2);
+pod2usage(-exitval => 0, -verbose => 1) if $help;
+pod2usage(-exitval => 0, -verbose => 2) if $man;
+pod2usage(-exitval => 1, -verbose => 1, -message => '$HADOOP_PREFIX not defined and no hdfs-cmd given. please use --hdfs-cmd to specify where your hdfs cli is.') if not defined $hdfsCmd;
+pod2usage(-exitval => 1, -verbose => 1, -message => "Your specified hdfs cli '$hdfsCmd' is not executable.") if not -x $hdfsCmd;
+if( defined $sudo and "" eq $sudo ){
+  $sudo = `which sudo`;
+  pod2usage(-exitval => 1, -verbose => 1, -message => "Error attempting to find the sudo command, please specify it with --sudo /path/to/sudo") if 0 != $?;
+  chomp($sudo);
+}
+if( defined $sudo ){
+  pod2usage(-exitval => 1, -verbose => 1, -message => "Your specified sudo command '$sudo' is not executable.") if not -x $sudo;
+}
+
+my $needsudo = defined $sudo;
+my $haadmin = "$hdfsCmd haadmin";
+if($needsudo) {
+  $haadmin = "$sudo -u $superuser $haadmin";
+  print STDERR "Starting HDFS agitator, configured to fail over every $sleep minutes. will run hdfs command '$hdfsCmd' as user '$superuser' via '$sudo'.\n";
+} else {
+  print STDERR "Starting HDFS agitator, configured to fail over every $sleep minutes. will run hdfs command '$hdfsCmd' as the current user.\n";
+}
+while(1){
+  sleep($sleep * 60);
+  my $t = strftime "%Y%m%d %H:%M:%S", localtime;
+  my @failServices;
+  if( defined $nameservice ){
+    @failServices = ($nameservice);
+  } else {
+    my $nameservicesRaw = `$hdfsCmd getconf -confKey dfs.nameservices`;
+    if(0 != $?) {
+      print STDERR "$t HDFS CLI failed. please see --help to set it correctly\n";
+      exit(1);
+    }
+    chomp($nameservicesRaw);
+    my @nameservices = split(/,/, $nameservicesRaw);
+    if(1 > scalar(@nameservices)) {
+      print STDERR "$t No HDFS NameServices found. Are you sure you're running in HA?\n";
+      exit(1);
+    }
+    if(rand(1) < .5){
+      my $serviceToFail = $nameservices[int(rand(scalar(@nameservices)))];
+      print STDERR "$t Failing over nameservice $serviceToFail\n";
+      @failServices = ($serviceToFail);
+    } else {
+      print STDERR "$t Failing over all nameservices\n";
+      @failServices = @nameservices;
+    }
+  }
+  for my $toFail (@failServices){
+    my $namenodesRaw = `$hdfsCmd getconf -confKey dfs.ha.namenodes.$toFail`;
+    if(0 != $?) {
+      print STDERR "$t HDFS CLI failed to look up namenodes in service $toFail.\n";
+      exit(1);
+    }
+    chomp($namenodesRaw);
+    my @namenodes = split(/,/, $namenodesRaw);
+    if(2 > scalar(@namenodes)) {
+      print STDERR "$t WARN NameService $toFail does not have at least 2 namenodes according to the HDFS configuration, skipping.\n";
+      next;
+    }
+    my $active;
+    for my $namenode (@namenodes){
+      my $status = `$haadmin -ns $toFail -getServiceState $namenode`;
+      if(0 != $?) {
+        if($needsudo) {
+          print STDERR "$t WARN Error while attempting to get the service state of $toFail :: $namenode\n";
+          $status = 'error';
+        } else {
+          print STDERR "$t WARN Current user may not run the HDFS haadmin utility, attempting to sudo to the $superuser user.\n";
+          $needsudo = 1;
+          if(not defined $sudo) {
+            $sudo = `which sudo`;
+            pod2usage(-exitval => 1, -verbose => 1, -message => "Error attempting to find the sudo command, please specify it with --sudo") if 0 != $?;
+            chomp($sudo);
+            pod2usage(-exitval => 1, -verbose => 1, -message => "The sudo command '$sudo' is not executable. please specify sudo with --sudo") if not -x $sudo;
+          }
+          $haadmin = "$sudo -u $superuser $haadmin";
+          redo;
+        }
+      }
+      chomp($status);
+      if( 'active' eq $status ){
+        $active = $namenode;
+        last;
+      }
+    }
+    if( defined $active ){
+      my @standby = grep { $_ ne $active } @namenodes;
+      my $newActive = $standby[int(rand(scalar(@standby)))];
+      print STDERR "$t Transitioning nameservice $toFail from $active to $newActive\n";
+      my $cmd = "$haadmin -ns $toFail -failover $active $newActive";
+      print "$t $cmd\n";
+      system($cmd);
+    } else {
+      my $newActive = $namenodes[int(rand(scalar(@namenodes)))];
+      print STDERR "$t WARN nameservice $toFail did not have an active namenode. Transitioning a random namenode to active. This will fail if HDFS is configured for automatic failover.\n";
+      my $cmd = "$haadmin -ns $toFail -transitionToActive $newActive";
+      print "$t $cmd\n";
+      system($cmd);
+    }
+  }
+}
+__END__
+
+=head1 NAME
+
+hdfs-agitator - causes HDFS to failover
+
+=head1 DESCRIPTION
+
+Sleeps for a configurable amount of time, then causes a NameNode failover in one
+or more HDFS NameServices. If a given NameService does not have an Active
+NameNode when it comes time to failover, a random standby is promoted.
+
+Only works on HDFS versions that support HA configurations and the haadmin
+command. In order to function, the user running this script must be able to
+use the haadmin command. This requires access to an HDFS superuser. By default,
+it will attempt to sudo to perform calls.
+
+=head1 SYNOPSIS
+
+hdfs-agitator [options]
+
+  Options:
+    --help         Brief help message
+    --man          Full documentation
+    --sleep        Time to sleep between failovers in minutes. Default 10
+    --superuser    HDFS superuser. Default 'hdfs'
+    --hdfs-cmd     hdfs command path. Default '$HADOOP_PREFIX/share/hadoop/hdfs/bin/hdfs'
+    --nameservice  Limit failovers to specified nameservice. Default all nameservices
+    --sudo         command to call to sudo to the HDFS superuser. Default 'sudo' if needed.
+
+=head1 OPTIONS
+
+=over 8
+
+=item B<--sleep>
+
+Sleep the given number of minutes between attempts to fail over nameservices.
+
+=item B<--nameservice>
+
+Limit failover attempts to the given nameservice. By default, we attempt ot list
+all known nameservices and choose either one or all of them to failover in a
+given cycle.
+
+=item B<--superuser>
+
+An HDFS superuser capable of running the haadmin command. Defaults to "hdfs".
+
+=item B<--hdfs-cmd>
+
+Path to the HDFS cli. Will be used both for non-administrative commands (e.g.
+listing the nameservices and serviceids in a given nameservice) and admin-only
+actions such as checking status and failing over.
+
+Defaults to using $HADOOP_PREFIX.
+
+=item B<--sudo>
+
+Command to allow us to act as the given HDFS superuser. By default we assume the current user
+can run HDFS administrative commands. When this argument is specified we will instead attempt
+to use the HDFS superuser instead. If given an argument, it will be called like
+sudo, i.e. "sudo -u $superuser $cmd". Defaults to "sudo" on the shell's path.
+
+=back
+
+=head1 SEE ALSO
+
+See the Apache Hadoop documentation on configuring HDFS HA
+
+=over 8
+
+=item B<HA with QJM>
+
+http://hadoop.apache.org/docs/r2.2.0/hadoop-yarn/hadoop-yarn-site/HDFSHighAvailabilityWithQJM.html#Administrative_commands
+
+=item B<HA with NFS>
+
+http://hadoop.apache.org/docs/r2.2.0/hadoop-yarn/hadoop-yarn-site/HDFSHighAvailabilityWithNFS.html#Administrative_commands
+
+=back

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/libexec/master-agitator.pl
----------------------------------------------------------------------
diff --git a/libexec/master-agitator.pl b/libexec/master-agitator.pl
new file mode 100755
index 0000000..d87f17e
--- /dev/null
+++ b/libexec/master-agitator.pl
@@ -0,0 +1,92 @@
+#! /usr/bin/env perl
+
+# 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.
+
+
+use POSIX qw(strftime);
+use Cwd qw();
+
+if(scalar(@ARGV) != 2){
+	print "Usage : master-agitator.pl <sleep before kill in minutes> <sleep before start in minutes>\n";
+	exit(1);
+}
+
+my $ACCUMULO_HOME;
+if( defined $ENV{'ACCUMULO_HOME'} ){
+  $ACCUMULO_HOME = $ENV{'ACCUMULO_HOME'};
+} else {
+  $cwd=Cwd::cwd();
+  $ACCUMULO_HOME=$cwd . '/../../..';
+}
+
+if(defined $ENV{'ACCUMULO_CONF_DIR'}){
+        $ACCUMULO_CONF_DIR = $ENV{'ACCUMULO_CONF_DIR'};
+}else{
+	$ACCUMULO_CONF_DIR = $ACCUMULO_HOME . '/conf';
+}
+
+$sleep1 = $ARGV[0];
+$sleep2 = $ARGV[1];
+
+@mastersRaw = `cat $ACCUMULO_CONF_DIR/masters`;
+chomp(@mastersRaw);
+
+for $master (@mastersRaw){
+	if($master eq "" || substr($master,0,1) eq "#"){
+		next;
+	}
+
+	push(@masters, $master);
+}
+
+
+while(1){
+	sleep($sleep1 * 60);
+	$t = strftime "%Y%m%d %H:%M:%S", localtime;
+	if(rand(1) < .5){
+		$masterNodeToWack = $masters[int(rand(scalar(@masters)))];
+		print STDERR "$t Killing master on $masterNodeToWack\n";
+		$cmd = "ssh $masterNodeToWack \"pkill -f '[ ]org.apache.accumulo.start.*master'\"";
+		print "$t $cmd\n";
+		system($cmd);
+	}else{
+		print STDERR "$t Killing all masters\n";
+		$cmd = "pssh -h $ACCUMULO_CONF_DIR/masters \"pkill -f '[ ]org.apache.accumulo.start.*master'\" < /dev/null";
+		print "$t $cmd\n";
+		system($cmd);
+
+		$file = '';
+		if (-e "$ACCUMULO_CONF_DIR/gc") {
+			$file = 'gc';
+		} else {
+			$file = 'masters';
+		}
+
+		$cmd = "pssh -h $ACCUMULO_CONF_DIR/$file \"pkill -f '[ ]org.apache.accumulo.start.*gc'\" < /dev/null";
+		print "$t $cmd\n";
+		system($cmd);
+	}
+
+	sleep($sleep2 * 60);
+	$t = strftime "%Y%m%d %H:%M:%S", localtime;
+	print STDERR "$t Running start-all\n";
+
+	$cmd = "pssh -h $ACCUMULO_CONF_DIR/masters \"$ACCUMULO_HOME/bin/accumulo-service master start\" < /dev/null";
+	print "$t $cmd\n";
+	system($cmd);
+}
+
+

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/libexec/tserver-agitator.pl
----------------------------------------------------------------------
diff --git a/libexec/tserver-agitator.pl b/libexec/tserver-agitator.pl
new file mode 100755
index 0000000..de29e3a
--- /dev/null
+++ b/libexec/tserver-agitator.pl
@@ -0,0 +1,134 @@
+#! /usr/bin/env perl
+
+# 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.
+
+
+use POSIX qw(strftime);
+use Cwd qw();
+
+if(scalar(@ARGV) != 4 && scalar(@ARGV) != 2){
+  print "Usage : tserver-agitator.pl <min sleep before kill in minutes>[:max sleep before kill in minutes] <min sleep before tup in minutes>[:max sleep before tup in minutes] [<min kill> <max kill>]\n";
+  exit(1);
+}
+
+my $ACCUMULO_HOME;
+if( defined $ENV{'ACCUMULO_HOME'} ){
+  $ACCUMULO_HOME = $ENV{'ACCUMULO_HOME'};
+} else {
+  $cwd=Cwd::cwd();
+  $ACCUMULO_HOME=$cwd . '/../../..';
+}
+
+print "ACCUMULO_HOME=$ACCUMULO_HOME\n";
+
+@sleeprange1 = split(/:/, $ARGV[0]);
+$sleep1 = $sleeprange1[0];
+
+@sleeprange2 = split(/:/, $ARGV[1]);
+$sleep2 = $sleeprange2[0];
+
+if (scalar(@sleeprange1) > 1) {
+  $sleep1max = $sleeprange1[1] + 1;
+} else {
+  $sleep1max = $sleep1;
+}
+
+if ($sleep1 > $sleep1max) {
+  die("sleep1 > sleep1max $sleep1 > $sleep1max");
+}
+
+if (scalar(@sleeprange2) > 1) {
+  $sleep2max = $sleeprange2[1] + 1;
+} else {
+  $sleep2max = $sleep2;
+}
+
+if($sleep2 > $sleep2max){
+  die("sleep2 > sleep2max $sleep2 > $sleep2max");
+}
+
+if(defined $ENV{'ACCUMULO_CONF_DIR'}){
+  $ACCUMULO_CONF_DIR = $ENV{'ACCUMULO_CONF_DIR'};
+}else{
+  $ACCUMULO_CONF_DIR = $ACCUMULO_HOME . '/conf';
+}
+
+if(scalar(@ARGV) == 4){
+  $minKill = $ARGV[2];
+  $maxKill = $ARGV[3];
+}else{
+  $minKill = 1;
+  $maxKill = 1;
+}
+
+if($minKill > $maxKill){
+  die("minKill > maxKill $minKill > $maxKill");
+}
+
+@tserversRaw = `cat $ACCUMULO_CONF_DIR/tservers`;
+chomp(@tserversRaw);
+
+for $tserver (@tserversRaw){
+  if($tserver eq "" || substr($tserver,0,1) eq "#"){
+    next;
+  }
+
+  push(@tservers, $tserver);
+}
+
+
+if(scalar(@tservers) < $maxKill){
+  print STDERR "WARN setting maxKill to ".scalar(@tservers)."\n";
+  $maxKill = scalar(@tservers);
+}
+
+if ($minKill > $maxKill){
+  print STDERR "WARN setting minKill to equal maxKill\n";
+  $minKill = $maxKill;
+}
+
+while(1){
+
+  $numToKill = int(rand($maxKill - $minKill + 1)) + $minKill;
+  %killed = {};
+  $server = "";
+
+  for($i = 0; $i < $numToKill; $i++){
+    while($server eq "" || $killed{$server} != undef){
+      $index = int(rand(scalar(@tservers)));
+      $server = $tservers[$index];
+    }
+
+    $killed{$server} = 1;
+
+    $t = strftime "%Y%m%d %H:%M:%S", localtime;
+
+    print STDERR "$t Killing tserver on $server\n";
+    # We're the accumulo user, just run the commandj
+    system("ssh $server '$ACCUMULO_HOME/bin/accumulo-service tserver stop'");
+  }
+
+  $nextsleep2 = int(rand($sleep2max - $sleep2)) + $sleep2;
+  sleep($nextsleep2 * 60);
+  $t = strftime "%Y%m%d %H:%M:%S", localtime;
+  print STDERR "$t Running tup\n";
+  # restart the as them as the accumulo user
+  system("$ACCUMULO_HOME/libexec/cluster.sh start-tservers");
+
+  $nextsleep1 = int(rand($sleep1max - $sleep1)) + $sleep1;
+  sleep($nextsleep1 * 60);
+}
+


[2/5] accumulo-testing git commit: ACCUMULO-4510 Refactored Continous Ingest tests

Posted by mw...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/DeleteTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/DeleteTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/DeleteTable.java
index 5937a29..d122229 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/DeleteTable.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/DeleteTable.java
@@ -22,15 +22,15 @@ import java.util.Random;
 
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class DeleteTable extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/DropUser.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/DropUser.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/DropUser.java
index 2034d3d..6f1342f 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/DropUser.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/DropUser.java
@@ -22,14 +22,14 @@ import java.util.Random;
 
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class DropUser extends Test {
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/IsolatedScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/IsolatedScan.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/IsolatedScan.java
index 8a5483e..d7ac842 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/IsolatedScan.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/IsolatedScan.java
@@ -31,15 +31,15 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.PeekingIterator;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class IsolatedScan extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ListSplits.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ListSplits.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ListSplits.java
index a84c4fd..03e0fcd 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ListSplits.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ListSplits.java
@@ -24,7 +24,7 @@ import java.util.Random;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -32,8 +32,8 @@ import org.apache.hadoop.io.Text;
 public class ListSplits extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Merge.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Merge.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Merge.java
index 87a48f9..fa01572 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Merge.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Merge.java
@@ -25,7 +25,7 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -33,8 +33,8 @@ import org.apache.hadoop.io.Text;
 public class Merge extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/OfflineTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/OfflineTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/OfflineTable.java
index f663802..d568cac 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/OfflineTable.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/OfflineTable.java
@@ -25,15 +25,15 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class OfflineTable extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/RenameNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/RenameNamespace.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/RenameNamespace.java
index dab41bf..e0946d8 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/RenameNamespace.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/RenameNamespace.java
@@ -23,15 +23,15 @@ import java.util.Random;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.NamespaceExistsException;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class RenameNamespace extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/RenameTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/RenameTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/RenameTable.java
index 4c5a52f..acd9367 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/RenameTable.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/RenameTable.java
@@ -25,15 +25,15 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class RenameTable extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Replication.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Replication.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Replication.java
index 713adaf..8033eda 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Replication.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Replication.java
@@ -49,7 +49,7 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.replication.ReplicationTable;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.accumulo.tserver.replication.AccumuloReplicaSystem;
@@ -61,8 +61,8 @@ public class Replication extends Test {
   final int COLS = 50;
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    final Connector c = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    final Connector c = env.getAccumuloConnector();
     final Instance inst = c.getInstance();
     final String instName = inst.getInstanceName();
     final InstanceOperations iOps = c.instanceOperations();
@@ -71,8 +71,8 @@ public class Replication extends Test {
     // Replicate to ourselves
     iOps.setProperty(REPLICATION_NAME.getKey(), instName);
     iOps.setProperty(REPLICATION_PEERS.getKey() + instName, getPeerConfigurationValue(AccumuloReplicaSystem.class, instName + "," + inst.getZooKeepers()));
-    iOps.setProperty(REPLICATION_PEER_USER.getKey() + instName, env.getUserName());
-    iOps.setProperty(REPLICATION_PEER_PASSWORD.getKey() + instName, env.getPassword());
+    iOps.setProperty(REPLICATION_PEER_USER.getKey() + instName, env.getAccumuloUserName());
+    iOps.setProperty(REPLICATION_PEER_PASSWORD.getKey() + instName, env.getAccumuloPassword());
     // Tweak some replication parameters to make the replication go faster
     iOps.setProperty(MASTER_REPLICATION_SCAN_INTERVAL.getKey(), "1s");
     iOps.setProperty(REPLICATION_WORK_ASSIGNMENT_SLEEP.getKey(), "1s");

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ScanTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ScanTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ScanTable.java
index ab89bea..f22d406 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ScanTable.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ScanTable.java
@@ -31,15 +31,15 @@ import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class ScanTable extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Setup.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Setup.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Setup.java
index 164fd4f..3902c97 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Setup.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Setup.java
@@ -21,14 +21,14 @@ import java.util.List;
 import java.util.Properties;
 import java.util.Random;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class Setup extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     Random rand = new Random();
     state.set("rand", rand);
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Compact.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Compact.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Compact.java
index b0aa7e1..0754af1 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Compact.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Compact.java
@@ -20,7 +20,7 @@ import java.util.Properties;
 import java.util.Random;
 
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -30,10 +30,10 @@ import org.apache.hadoop.io.Text;
  */
 public class Compact extends Test {
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     String table = state.getString("tableName");
     Random rand = (Random) state.get("rand");
-    Connector conn = env.getConnector();
+    Connector conn = env.getAccumuloConnector();
     Text row1 = new Text(Utils.getBank(rand.nextInt((Integer) state.get("numBanks"))));
     Text row2 = new Text(Utils.getBank(rand.nextInt((Integer) state.get("numBanks"))));
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Flush.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Flush.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Flush.java
index 2c5448d..af8b387 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Flush.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Flush.java
@@ -20,7 +20,7 @@ import java.util.Properties;
 import java.util.Random;
 
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -30,10 +30,10 @@ import org.apache.hadoop.io.Text;
  */
 public class Flush extends Test {
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     String table = state.getString("tableName");
     Random rand = (Random) state.get("rand");
-    Connector conn = env.getConnector();
+    Connector conn = env.getAccumuloConnector();
     Text row1 = new Text(Utils.getBank(rand.nextInt((Integer) state.get("numBanks"))));
     Text row2 = new Text(Utils.getBank(rand.nextInt((Integer) state.get("numBanks"))));
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Init.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Init.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Init.java
index 3656653..031639a 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Init.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Init.java
@@ -26,7 +26,7 @@ import org.apache.accumulo.core.client.ConditionalWriter;
 import org.apache.accumulo.core.client.ConditionalWriter.Status;
 import org.apache.accumulo.core.data.Condition;
 import org.apache.accumulo.core.data.ConditionalMutation;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -37,7 +37,7 @@ import org.apache.hadoop.io.Text;
 public class Init extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
 
     int numBanks = (Integer) state.get("numBanks");
     int numAccts = (Integer) state.get("numAccts");
@@ -46,7 +46,7 @@ public class Init extends Test {
     TreeSet<Text> splits = new TreeSet<>();
     for (int i = 1; i < 10; i++)
       splits.add(new Text(Utils.getBank((int) (numBanks * .1 * i))));
-    env.getConnector().tableOperations().addSplits((String) state.get("tableName"), splits);
+    env.getAccumuloConnector().tableOperations().addSplits((String) state.get("tableName"), splits);
     log.info("Added splits " + splits);
 
     ArrayList<Integer> banks = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Merge.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Merge.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Merge.java
index 2f5d52b..af2bc25 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Merge.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Merge.java
@@ -20,7 +20,7 @@ import java.util.Properties;
 import java.util.Random;
 
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -30,10 +30,10 @@ import org.apache.hadoop.io.Text;
  */
 public class Merge extends Test {
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     String table = state.getString("tableName");
     Random rand = (Random) state.get("rand");
-    Connector conn = env.getConnector();
+    Connector conn = env.getAccumuloConnector();
     Text row1 = new Text(Utils.getBank(rand.nextInt((Integer) state.get("numBanks"))));
     Text row2 = new Text(Utils.getBank(rand.nextInt((Integer) state.get("numBanks"))));
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Setup.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Setup.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Setup.java
index 1e4ad01..6973eb5 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Setup.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Setup.java
@@ -23,14 +23,14 @@ import org.apache.accumulo.core.client.ConditionalWriter;
 import org.apache.accumulo.core.client.ConditionalWriterConfig;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class Setup extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     Random rand = new Random();
     state.set("rand", rand);
 
@@ -46,14 +46,14 @@ public class Setup extends Test {
     state.set("tableName", tableName);
 
     try {
-      env.getConnector().tableOperations().create(tableName);
+      env.getAccumuloConnector().tableOperations().create(tableName);
       log.debug("created table " + tableName);
       boolean blockCache = rand.nextBoolean();
-      env.getConnector().tableOperations().setProperty(tableName, Property.TABLE_BLOCKCACHE_ENABLED.getKey(), blockCache + "");
+      env.getAccumuloConnector().tableOperations().setProperty(tableName, Property.TABLE_BLOCKCACHE_ENABLED.getKey(), blockCache + "");
       log.debug("set " + Property.TABLE_BLOCKCACHE_ENABLED.getKey() + " " + blockCache);
     } catch (TableExistsException tee) {}
 
-    ConditionalWriter cw = env.getConnector().createConditionalWriter(tableName, new ConditionalWriterConfig().setMaxWriteThreads(1));
+    ConditionalWriter cw = env.getAccumuloConnector().createConditionalWriter(tableName, new ConditionalWriterConfig().setMaxWriteThreads(1));
     state.set("cw", cw);
 
   }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Split.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Split.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Split.java
index 8ea9aab..e8c1732 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Split.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Split.java
@@ -22,7 +22,7 @@ import java.util.Random;
 import java.util.TreeSet;
 
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -32,10 +32,10 @@ import org.apache.hadoop.io.Text;
  */
 public class Split extends Test {
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     String table = state.getString("tableName");
     Random rand = (Random) state.get("rand");
-    Connector conn = env.getConnector();
+    Connector conn = env.getAccumuloConnector();
     String row = Utils.getBank(rand.nextInt((Integer) state.get("numBanks")));
 
     log.debug("adding split " + row);

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/TearDown.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/TearDown.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/TearDown.java
index cf72607..324d1a0 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/TearDown.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/TearDown.java
@@ -19,7 +19,7 @@ package org.apache.accumulo.testing.core.randomwalk.conditional;
 import java.util.Properties;
 
 import org.apache.accumulo.core.client.ConditionalWriter;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
@@ -28,7 +28,7 @@ import org.apache.accumulo.testing.core.randomwalk.Test;
  */
 public class TearDown extends Test {
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     ConditionalWriter cw = (ConditionalWriter) state.get("cw");
     cw.close();
   }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Transfer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Transfer.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Transfer.java
index 73a7d91..7fb888d 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Transfer.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Transfer.java
@@ -31,7 +31,7 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.commons.math3.distribution.ZipfDistribution;
@@ -61,10 +61,10 @@ public class Transfer extends Test {
   }
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     String table = state.getString("tableName");
     Random rand = (Random) state.get("rand");
-    Connector conn = env.getConnector();
+    Connector conn = env.getAccumuloConnector();
 
     int numAccts = (Integer) state.get("numAccts");
     // note: non integer exponents are slow

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Verify.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Verify.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Verify.java
index fa516f1..f35646c 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Verify.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/conditional/Verify.java
@@ -29,7 +29,7 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.ColumnSliceFilter;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
@@ -39,9 +39,9 @@ import org.apache.accumulo.testing.core.randomwalk.Test;
 public class Verify extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     String table = state.getString("tableName");
-    Connector conn = env.getConnector();
+    Connector conn = env.getAccumuloConnector();
 
     int numAccts = (Integer) state.get("numAccts");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Commit.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Commit.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Commit.java
index 09774ff..b7e5dae 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Commit.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Commit.java
@@ -18,14 +18,14 @@ package org.apache.accumulo.testing.core.randomwalk.image;
 
 import java.util.Properties;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class Commit extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     env.getMultiTableBatchWriter().flush();
 
     log.debug("Committed " + state.getLong("numWrites") + " writes.  Total writes: " + state.getLong("totalWrites"));

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/ImageFixture.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/ImageFixture.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/ImageFixture.java
index 687b2d1..a638126 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/ImageFixture.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/ImageFixture.java
@@ -31,7 +31,7 @@ import org.apache.accumulo.core.client.MultiTableBatchWriter;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.Fixture;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.hadoop.io.Text;
@@ -42,10 +42,10 @@ public class ImageFixture extends Fixture {
   String indexTableName;
 
   @Override
-  public void setUp(State state, Environment env) throws Exception {
+  public void setUp(State state, RandWalkEnv env) throws Exception {
 
-    Connector conn = env.getConnector();
-    Instance instance = env.getInstance();
+    Connector conn = env.getAccumuloConnector();
+    Instance instance = env.getAccumuloInstance();
 
     SortedSet<Text> splits = new TreeSet<>();
     for (int i = 1; i < 256; i++) {
@@ -107,7 +107,7 @@ public class ImageFixture extends Fixture {
   }
 
   @Override
-  public void tearDown(State state, Environment env) throws Exception {
+  public void tearDown(State state, RandWalkEnv env) throws Exception {
     // We have resources we need to clean up
     if (env.isMultiTableBatchWriterInitialized()) {
       MultiTableBatchWriter mtbw = env.getMultiTableBatchWriter();
@@ -124,7 +124,7 @@ public class ImageFixture extends Fixture {
     // Now we can safely delete the tables
     log.debug("Dropping tables: " + imageTableName + " " + indexTableName);
 
-    Connector conn = env.getConnector();
+    Connector conn = env.getAccumuloConnector();
 
     conn.tableOperations().delete(imageTableName);
     conn.tableOperations().delete(indexTableName);

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/ScanMeta.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/ScanMeta.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/ScanMeta.java
index dbd89e8..8fa1910 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/ScanMeta.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/ScanMeta.java
@@ -32,7 +32,7 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -40,7 +40,7 @@ import org.apache.hadoop.io.Text;
 public class ScanMeta extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
 
     // scan just the metadata of the images table to find N hashes... use the batch scanner to lookup those N hashes in the index table
     // this scan will test locality groups....
@@ -50,7 +50,7 @@ public class ScanMeta extends Test {
 
     String uuid = UUID.randomUUID().toString();
 
-    Connector conn = env.getConnector();
+    Connector conn = env.getAccumuloConnector();
 
     Scanner imageScanner = conn.createScanner(imageTableName, new Authorizations());
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/TableOp.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/TableOp.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/TableOp.java
index 1d14a90..e7ad6b0 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/TableOp.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/TableOp.java
@@ -24,7 +24,7 @@ import java.util.Set;
 
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -32,7 +32,7 @@ import org.apache.hadoop.io.Text;
 public class TableOp extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
 
     // choose a table
     Random rand = new Random();
@@ -44,7 +44,7 @@ public class TableOp extends Test {
     }
 
     // check if chosen table exists
-    Connector conn = env.getConnector();
+    Connector conn = env.getAccumuloConnector();
     TableOperations tableOps = conn.tableOperations();
     if (tableOps.exists(tableName) == false) {
       log.error("Table " + tableName + " does not exist!");

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Verify.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Verify.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Verify.java
index f3caf15..fb20a5b 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Verify.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Verify.java
@@ -31,7 +31,7 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -42,7 +42,7 @@ public class Verify extends Test {
   String imageTableName;
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
 
     Random rand = new Random();
 
@@ -52,7 +52,7 @@ public class Verify extends Test {
     indexTableName = state.getString("indexTableName");
     imageTableName = state.getString("imageTableName");
 
-    Connector conn = env.getConnector();
+    Connector conn = env.getAccumuloConnector();
 
     Scanner indexScanner = conn.createScanner(indexTableName, new Authorizations());
     Scanner imageScanner = conn.createScanner(imageTableName, new Authorizations());

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Write.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Write.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Write.java
index f7a2781..965ba60 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Write.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/image/Write.java
@@ -27,7 +27,7 @@ import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.MultiTableBatchWriter;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -42,7 +42,7 @@ public class Write extends Test {
   static final Text CONTENT_COLUMN_FAMILY = new Text("content");
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
 
     MultiTableBatchWriter mtbw = env.getMultiTableBatchWriter();
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/Commit.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/Commit.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/Commit.java
index 4ac6b47..72c3326 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/Commit.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/Commit.java
@@ -18,14 +18,14 @@ package org.apache.accumulo.testing.core.randomwalk.multitable;
 
 import java.util.Properties;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class Commit extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     env.getMultiTableBatchWriter().flush();
 
     Long numWrites = state.getLong("numWrites");

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CopyTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CopyTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CopyTable.java
index 8d1e451..a4cdbcf 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CopyTable.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CopyTable.java
@@ -23,7 +23,7 @@ import java.util.TreeSet;
 
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.testing.core.TestProps;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -41,7 +41,7 @@ public class CopyTable extends Test {
   }
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
 
     @SuppressWarnings("unchecked")
     List<String> tables = (List<String>) state.get("tableList");
@@ -55,33 +55,33 @@ public class CopyTable extends Test {
     String dstTableName = String.format("%s_%d", state.getString("tableNamePrefix"), nextId);
 
     String[] args = new String[6];
-    args[0] = env.getUserName();
-    args[1] = env.getPassword();
+    args[0] = env.getAccumuloUserName();
+    args[1] = env.getAccumuloPassword();
     if (null == args[1]) {
-      args[1] = env.getKeytab();
+      args[1] = env.getAccumuloKeytab();
     }
     args[2] = srcTableName;
-    args[3] = env.getInstance().getInstanceName();
+    args[3] = env.getAccumuloInstance().getInstanceName();
     args[4] = env.getConfigProperty(TestProps.ZOOKEEPERS);
     args[5] = dstTableName;
 
     log.debug("copying " + srcTableName + " to " + dstTableName);
 
-    env.getConnector().tableOperations().create(dstTableName);
+    env.getAccumuloConnector().tableOperations().create(dstTableName);
 
-    env.getConnector().tableOperations().addSplits(dstTableName, splits);
+    env.getAccumuloConnector().tableOperations().addSplits(dstTableName, splits);
 
     if (ToolRunner.run(env.getHadoopConfiguration(), new CopyTool(), args) != 0) {
       log.error("Failed to run map/red verify");
       return;
     }
 
-    String tableId = Tables.getNameToIdMap(env.getInstance()).get(dstTableName);
+    String tableId = Tables.getNameToIdMap(env.getAccumuloInstance()).get(dstTableName);
     log.debug("copied " + srcTableName + " to " + dstTableName + " (id - " + tableId + " )");
 
     tables.add(dstTableName);
 
-    env.getConnector().tableOperations().delete(srcTableName);
+    env.getAccumuloConnector().tableOperations().delete(srcTableName);
     log.debug("dropped " + srcTableName);
 
     nextId++;

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CreateTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CreateTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CreateTable.java
index 4250003..f258297 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CreateTable.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/CreateTable.java
@@ -23,7 +23,7 @@ import java.util.TreeSet;
 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.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -40,8 +40,8 @@ public class CreateTable extends Test {
   }
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     int nextId = ((Integer) state.get("nextId")).intValue();
     String tableName = String.format("%s_%d", state.getString("tableNamePrefix"), nextId);
@@ -49,7 +49,7 @@ public class CreateTable extends Test {
       conn.tableOperations().create(tableName);
       // Add some splits to make the server's life easier
       conn.tableOperations().addSplits(tableName, splits);
-      String tableId = Tables.getNameToIdMap(env.getInstance()).get(tableName);
+      String tableId = Tables.getNameToIdMap(env.getAccumuloInstance()).get(tableName);
       log.debug("created " + tableName + " (id:" + tableId + ")");
       // Add some splits to make the server's life easier
       conn.tableOperations().addSplits(tableName, splits);

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/DropTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/DropTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/DropTable.java
index 61904ca..a6069b8 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/DropTable.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/DropTable.java
@@ -21,14 +21,14 @@ import java.util.Properties;
 import java.util.Random;
 
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class DropTable extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
 
     @SuppressWarnings("unchecked")
     List<String> tables = (List<String>) state.get("tableList");
@@ -42,7 +42,7 @@ public class DropTable extends Test {
     String tableName = tables.remove(rand.nextInt(tables.size()));
 
     try {
-      env.getConnector().tableOperations().delete(tableName);
+      env.getAccumuloConnector().tableOperations().delete(tableName);
       log.debug("Dropped " + tableName);
     } catch (TableNotFoundException e) {
       log.error("Tried to drop table " + tableName + " but could not be found!");

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/MultiTableFixture.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/MultiTableFixture.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/MultiTableFixture.java
index 5a7a415..249acf1 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/MultiTableFixture.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/MultiTableFixture.java
@@ -24,14 +24,14 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.MultiTableBatchWriter;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.Fixture;
 import org.apache.accumulo.testing.core.randomwalk.State;
 
 public class MultiTableFixture extends Fixture {
 
   @Override
-  public void setUp(State state, Environment env) throws Exception {
+  public void setUp(State state, RandWalkEnv env) throws Exception {
 
     String hostname = InetAddress.getLocalHost().getHostName().replaceAll("[-.]", "_");
 
@@ -43,7 +43,7 @@ public class MultiTableFixture extends Fixture {
   }
 
   @Override
-  public void tearDown(State state, Environment env) throws Exception {
+  public void tearDown(State state, RandWalkEnv env) throws Exception {
     // We have resources we need to clean up
     if (env.isMultiTableBatchWriterInitialized()) {
       MultiTableBatchWriter mtbw = env.getMultiTableBatchWriter();
@@ -57,7 +57,7 @@ public class MultiTableFixture extends Fixture {
       env.resetMultiTableBatchWriter();
     }
 
-    Connector conn = env.getConnector();
+    Connector conn = env.getAccumuloConnector();
 
     @SuppressWarnings("unchecked")
     List<String> tables = (List<String>) state.get("tableList");

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/OfflineTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/OfflineTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/OfflineTable.java
index 70a6b21..d9e783f 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/OfflineTable.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/OfflineTable.java
@@ -20,14 +20,14 @@ import java.util.List;
 import java.util.Properties;
 import java.util.Random;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class OfflineTable extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
 
     @SuppressWarnings("unchecked")
     List<String> tables = (List<String>) state.get("tableList");
@@ -39,9 +39,9 @@ public class OfflineTable extends Test {
     Random rand = new Random();
     String tableName = tables.get(rand.nextInt(tables.size()));
 
-    env.getConnector().tableOperations().offline(tableName, rand.nextBoolean());
+    env.getAccumuloConnector().tableOperations().offline(tableName, rand.nextBoolean());
     log.debug("Table " + tableName + " offline ");
-    env.getConnector().tableOperations().online(tableName, rand.nextBoolean());
+    env.getAccumuloConnector().tableOperations().online(tableName, rand.nextBoolean());
     log.debug("Table " + tableName + " online ");
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/Write.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/Write.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/Write.java
index 3c0c792..80e9c0e 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/Write.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/multitable/Write.java
@@ -29,7 +29,7 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -37,7 +37,7 @@ import org.apache.hadoop.io.Text;
 public class Write extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
 
     @SuppressWarnings("unchecked")
     List<String> tables = (List<String>) state.get("tableList");

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterSystemPerm.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterSystemPerm.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterSystemPerm.java
index c075541..dd0cd54 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterSystemPerm.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterSystemPerm.java
@@ -23,15 +23,15 @@ 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.security.SystemPermission;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class AlterSystemPerm extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
     WalkingSecurity ws = new WalkingSecurity(state, env);
 
     String action = props.getProperty("task", "toggle");

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterTable.java
index 0e613d8..77ce3bf 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterTable.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterTable.java
@@ -25,15 +25,15 @@ 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.security.SecurityErrorCode;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class AlterTable extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getInstance().getConnector(WalkingSecurity.get(state, env).getSysUserName(), WalkingSecurity.get(state, env).getSysToken());
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloInstance().getConnector(WalkingSecurity.get(state, env).getSysUserName(), WalkingSecurity.get(state, env).getSysToken());
 
     String tableName = WalkingSecurity.get(state, env).getTableName();
     String namespaceName = WalkingSecurity.get(state, env).getNamespaceName();
@@ -46,7 +46,7 @@ public class AlterTable extends Test {
     renameTable(conn, state, env, tableName, newTableName, hasPermission, exists);
   }
 
-  public static void renameTable(Connector conn, State state, Environment env, String oldName, String newName, boolean hasPermission, boolean tableExists)
+  public static void renameTable(Connector conn, State state, RandWalkEnv env, String oldName, String newName, boolean hasPermission, boolean tableExists)
       throws AccumuloSecurityException, AccumuloException, TableExistsException {
     try {
       conn.tableOperations().rename(oldName, newName);

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterTablePerm.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterTablePerm.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterTablePerm.java
index 381a801..6d6bf64 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterTablePerm.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/AlterTablePerm.java
@@ -25,18 +25,18 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class AlterTablePerm extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     alter(state, env, props);
   }
 
-  public static void alter(State state, Environment env, Properties props) throws Exception {
+  public static void alter(State state, RandWalkEnv env, Properties props) throws Exception {
     String action = props.getProperty("task", "toggle");
     String perm = props.getProperty("perm", "random");
     String sourceUserProp = props.getProperty("source", "system");
@@ -71,19 +71,19 @@ public class AlterTablePerm extends Test {
       sourceUser = WalkingSecurity.get(state, env).getTabUserName();
       sourceToken = WalkingSecurity.get(state, env).getTabToken();
     } else {
-      sourceUser = env.getUserName();
+      sourceUser = env.getAccumuloUserName();
       sourceToken = env.getToken();
     }
-    Connector conn = env.getInstance().getConnector(sourceUser, sourceToken);
+    Connector conn = env.getAccumuloInstance().getConnector(sourceUser, sourceToken);
 
-    canGive = WalkingSecurity.get(state, env).canGrantTable(new Credentials(sourceUser, sourceToken).toThrift(env.getInstance()), target,
-        WalkingSecurity.get(state, env).getTableName(), WalkingSecurity.get(state, env).getNamespaceName());
+    canGive = WalkingSecurity.get(state, env).canGrantTable(new Credentials(sourceUser, sourceToken).toThrift(env.getAccumuloInstance()), target,
+                                                            WalkingSecurity.get(state, env).getTableName(), WalkingSecurity.get(state, env).getNamespaceName());
 
     // toggle
     if (!"take".equals(action) && !"give".equals(action)) {
       try {
         boolean res;
-        if (hasPerm != (res = env.getConnector().securityOperations().hasTablePermission(target, tableName, tabPerm)))
+        if (hasPerm != (res = env.getAccumuloConnector().securityOperations().hasTablePermission(target, tableName, tabPerm)))
           throw new AccumuloException("Test framework and accumulo are out of sync for user " + conn.whoami() + " for perm " + tabPerm.name()
               + " with local vs. accumulo being " + hasPerm + " " + res);
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/Authenticate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/Authenticate.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/Authenticate.java
index a9548c4..ba3c545 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/Authenticate.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/Authenticate.java
@@ -25,22 +25,22 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class Authenticate extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     authenticate(WalkingSecurity.get(state, env).getSysUserName(), WalkingSecurity.get(state, env).getSysToken(), state, env, props);
   }
 
-  public static void authenticate(String principal, AuthenticationToken token, State state, Environment env, Properties props) throws Exception {
+  public static void authenticate(String principal, AuthenticationToken token, State state, RandWalkEnv env, Properties props) throws Exception {
     String targetProp = props.getProperty("target");
     boolean success = Boolean.parseBoolean(props.getProperty("valid"));
 
-    Connector conn = env.getInstance().getConnector(principal, token);
+    Connector conn = env.getAccumuloInstance().getConnector(principal, token);
 
     String target;
 
@@ -52,7 +52,7 @@ public class Authenticate extends Test {
     boolean exists = WalkingSecurity.get(state, env).userExists(target);
     // Copy so if failed it doesn't mess with the password stored in state
     byte[] password = Arrays.copyOf(WalkingSecurity.get(state, env).getUserPassword(target), WalkingSecurity.get(state, env).getUserPassword(target).length);
-    boolean hasPermission = WalkingSecurity.get(state, env).canAskAboutUser(new Credentials(principal, token).toThrift(env.getInstance()), target);
+    boolean hasPermission = WalkingSecurity.get(state, env).canAskAboutUser(new Credentials(principal, token).toThrift(env.getAccumuloInstance()), target);
 
     if (!success)
       for (int i = 0; i < password.length; i++)

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/ChangePass.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/ChangePass.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/ChangePass.java
index e58db32..42ea317 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/ChangePass.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/ChangePass.java
@@ -25,14 +25,14 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class ChangePass extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     String target = props.getProperty("target");
     String source = props.getProperty("source");
 
@@ -45,7 +45,7 @@ public class ChangePass extends Test {
       principal = WalkingSecurity.get(state, env).getTabUserName();
       token = WalkingSecurity.get(state, env).getTabToken();
     }
-    Connector conn = env.getInstance().getConnector(principal, token);
+    Connector conn = env.getAccumuloInstance().getConnector(principal, token);
 
     boolean hasPerm;
     boolean targetExists;
@@ -56,7 +56,7 @@ public class ChangePass extends Test {
 
     targetExists = WalkingSecurity.get(state, env).userExists(target);
 
-    hasPerm = WalkingSecurity.get(state, env).canChangePassword(new Credentials(principal, token).toThrift(env.getInstance()), target);
+    hasPerm = WalkingSecurity.get(state, env).canChangePassword(new Credentials(principal, token).toThrift(env.getAccumuloInstance()), target);
 
     Random r = new Random();
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/CreateTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/CreateTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/CreateTable.java
index c0b1cd9..5e71410 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/CreateTable.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/CreateTable.java
@@ -24,15 +24,15 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.security.SecurityErrorCode;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class CreateTable extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getInstance().getConnector(WalkingSecurity.get(state, env).getSysUserName(), WalkingSecurity.get(state, env).getSysToken());
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloInstance().getConnector(WalkingSecurity.get(state, env).getSysUserName(), WalkingSecurity.get(state, env).getSysToken());
 
     String tableName = WalkingSecurity.get(state, env).getTableName();
 
@@ -48,7 +48,7 @@ public class CreateTable extends Test {
         else {
           // create table anyway for sake of state
           try {
-            env.getConnector().tableOperations().create(tableName);
+            env.getAccumuloConnector().tableOperations().create(tableName);
             WalkingSecurity.get(state, env).initTable(tableName);
           } catch (TableExistsException tee) {
             if (exists)

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/CreateUser.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/CreateUser.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/CreateUser.java
index f604928..91e8f8b 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/CreateUser.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/CreateUser.java
@@ -22,15 +22,15 @@ 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.security.tokens.PasswordToken;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class CreateUser extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getInstance().getConnector(WalkingSecurity.get(state, env).getSysUserName(), WalkingSecurity.get(state, env).getSysToken());
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloInstance().getConnector(WalkingSecurity.get(state, env).getSysUserName(), WalkingSecurity.get(state, env).getSysToken());
 
     String tableUserName = WalkingSecurity.get(state, env).getTabUserName();
 
@@ -47,7 +47,7 @@ public class CreateUser extends Test {
           else {
             // create user anyway for sake of state
             if (!exists) {
-              env.getConnector().securityOperations().createLocalUser(tableUserName, tabUserPass);
+              env.getAccumuloConnector().securityOperations().createLocalUser(tableUserName, tabUserPass);
               WalkingSecurity.get(state, env).createUser(tableUserName, tabUserPass);
               Thread.sleep(1000);
             }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/DropTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/DropTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/DropTable.java
index 235c9ba..a69b295 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/DropTable.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/DropTable.java
@@ -26,18 +26,18 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.security.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class DropTable extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     dropTable(state, env, props);
   }
 
-  public static void dropTable(State state, Environment env, Properties props) throws Exception {
+  public static void dropTable(State state, RandWalkEnv env, Properties props) throws Exception {
     String sourceUser = props.getProperty("source", "system");
     String principal;
     AuthenticationToken token;
@@ -48,14 +48,14 @@ public class DropTable extends Test {
       principal = WalkingSecurity.get(state, env).getSysUserName();
       token = WalkingSecurity.get(state, env).getSysToken();
     }
-    Connector conn = env.getInstance().getConnector(principal, token);
+    Connector conn = env.getAccumuloInstance().getConnector(principal, token);
 
     String tableName = WalkingSecurity.get(state, env).getTableName();
     String namespaceName = WalkingSecurity.get(state, env).getNamespaceName();
 
     boolean exists = WalkingSecurity.get(state, env).getTableExists();
-    boolean hasPermission = WalkingSecurity.get(state, env).canDeleteTable(new Credentials(principal, token).toThrift(env.getInstance()), tableName,
-        namespaceName);
+    boolean hasPermission = WalkingSecurity.get(state, env).canDeleteTable(new Credentials(principal, token).toThrift(env.getAccumuloInstance()), tableName,
+                                                                           namespaceName);
 
     try {
       conn.tableOperations().delete(tableName);
@@ -65,7 +65,7 @@ public class DropTable extends Test {
           throw new AccumuloException("Got a security exception when I should have had permission.", ae);
         else {
           // Drop anyway for sake of state
-          env.getConnector().tableOperations().delete(tableName);
+          env.getAccumuloConnector().tableOperations().delete(tableName);
           WalkingSecurity.get(state, env).cleanTablePermissions(tableName);
           return;
         }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/DropUser.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/DropUser.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/DropUser.java
index 8d6080b..7d1a9b6 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/DropUser.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/DropUser.java
@@ -21,15 +21,15 @@ import java.util.Properties;
 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.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class DropUser extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getInstance().getConnector(WalkingSecurity.get(state, env).getSysUserName(), WalkingSecurity.get(state, env).getSysToken());
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloInstance().getConnector(WalkingSecurity.get(state, env).getSysUserName(), WalkingSecurity.get(state, env).getSysToken());
 
     String tableUserName = WalkingSecurity.get(state, env).getTabUserName();
 
@@ -45,7 +45,7 @@ public class DropUser extends Test {
             throw new AccumuloException("Got a security exception when I should have had permission.", ae);
           else {
             if (exists) {
-              env.getConnector().securityOperations().dropLocalUser(tableUserName);
+              env.getAccumuloConnector().securityOperations().dropLocalUser(tableUserName);
               WalkingSecurity.get(state, env).dropUser(tableUserName);
             }
             return;

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/SecurityFixture.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/SecurityFixture.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/SecurityFixture.java
index edfd15f..bc8fdc1 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/SecurityFixture.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/SecurityFixture.java
@@ -26,14 +26,14 @@ import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.Fixture;
 import org.apache.accumulo.testing.core.randomwalk.State;
 
 public class SecurityFixture extends Fixture {
 
   @Override
-  public void setUp(State state, Environment env) throws Exception {
+  public void setUp(State state, RandWalkEnv env) throws Exception {
     String secTableName, systemUserName, tableUserName, secNamespaceName;
     // A best-effort sanity check to guard against not password-based auth
     ClientConfiguration clientConf = ClientConfiguration.loadDefault();
@@ -41,7 +41,7 @@ public class SecurityFixture extends Fixture {
       throw new IllegalStateException("Security module currently cannot support Kerberos/SASL instances");
     }
 
-    Connector conn = env.getConnector();
+    Connector conn = env.getAccumuloConnector();
 
     String hostname = InetAddress.getLocalHost().getHostName().replaceAll("[-.]", "_");
 
@@ -84,10 +84,10 @@ public class SecurityFixture extends Fixture {
   }
 
   @Override
-  public void tearDown(State state, Environment env) throws Exception {
+  public void tearDown(State state, RandWalkEnv env) throws Exception {
     log.debug("One last validate");
     Validate.validate(state, env, log);
-    Connector conn = env.getConnector();
+    Connector conn = env.getAccumuloConnector();
 
     if (WalkingSecurity.get(state, env).getTableExists()) {
       String secTableName = WalkingSecurity.get(state, env).getTableName();

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/SetAuths.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/SetAuths.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/SetAuths.java
index 54ab69f..d7a3cdd 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/SetAuths.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/SetAuths.java
@@ -25,14 +25,14 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class SetAuths extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     String authsString = props.getProperty("auths", "_random");
 
     String targetUser = props.getProperty("system");
@@ -45,13 +45,13 @@ public class SetAuths extends Test {
       authToken = WalkingSecurity.get(state, env).getSysToken();
     } else {
       target = WalkingSecurity.get(state, env).getSysUserName();
-      authPrincipal = env.getUserName();
+      authPrincipal = env.getAccumuloUserName();
       authToken = env.getToken();
     }
-    Connector conn = env.getInstance().getConnector(authPrincipal, authToken);
+    Connector conn = env.getAccumuloInstance().getConnector(authPrincipal, authToken);
 
     boolean exists = WalkingSecurity.get(state, env).userExists(target);
-    boolean hasPermission = WalkingSecurity.get(state, env).canChangeAuthorizations(new Credentials(authPrincipal, authToken).toThrift(env.getInstance()),
+    boolean hasPermission = WalkingSecurity.get(state, env).canChangeAuthorizations(new Credentials(authPrincipal, authToken).toThrift(env.getAccumuloInstance()),
         target);
 
     Authorizations auths;

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/TableOp.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/TableOp.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/TableOp.java
index d3335c4..7a40ac3 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/TableOp.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/security/TableOp.java
@@ -45,7 +45,7 @@ import org.apache.accumulo.core.file.rfile.RFile;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.fs.FileSystem;
@@ -55,8 +55,8 @@ import org.apache.hadoop.io.Text;
 public class TableOp extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getInstance().getConnector(WalkingSecurity.get(state, env).getTabUserName(), WalkingSecurity.get(state, env).getTabToken());
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloInstance().getConnector(WalkingSecurity.get(state, env).getTabUserName(), WalkingSecurity.get(state, env).getTabToken());
 
     String action = props.getProperty("action", "_random");
     TablePermission tp;


[3/5] accumulo-testing git commit: ACCUMULO-4510 Refactored Continous Ingest tests

Posted by mw...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/continuous/CreateTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/continuous/CreateTable.java b/core/src/main/java/org/apache/accumulo/testing/core/continuous/CreateTable.java
new file mode 100644
index 0000000..b8f2d8d
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/continuous/CreateTable.java
@@ -0,0 +1,74 @@
+/*
+ * 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.testing.core.continuous;
+
+import java.util.Properties;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.testing.core.TestProps;
+import org.apache.hadoop.io.Text;
+
+public class CreateTable {
+
+  public static void main(String[] args) throws Exception {
+
+    if (args.length != 1) {
+      System.err.println("Usage: CreateTable <propsPath>");
+      System.exit(-1);
+    }
+
+    Properties props = TestProps.loadFromFile(args[0]);
+    ContinuousEnv env = new ContinuousEnv(props);
+
+    Connector conn = env.getAccumuloConnector();
+    String tableName = env.getAccumuloTableName();
+    if (conn.tableOperations().exists(tableName)) {
+      System.err.println("ERROR: Accumulo table '"+ tableName + "' already exists");
+      System.exit(-1);
+    }
+
+    int numTablets = Integer.parseInt(props.getProperty(TestProps.CI_COMMON_ACCUMULO_NUM_TABLETS));
+    if (numTablets < 1) {
+      System.err.println("ERROR: numTablets < 1");
+      System.exit(-1);
+    }
+    if (env.getRowMin() >= env.getRowMax()) {
+      System.err.println("ERROR: min >= max");
+      System.exit(-1);
+    }
+
+    conn.tableOperations().create(tableName);
+
+    SortedSet<Text> splits = new TreeSet<>();
+    int numSplits = numTablets - 1;
+    long distance = ((env.getRowMax() - env.getRowMin()) / numTablets) + 1;
+    long split = distance;
+    for (int i = 0; i < numSplits; i++) {
+      String s = String.format("%016x", split + env.getRowMin());
+      while (s.charAt(s.length() - 1) == '0') {
+        s = s.substring(0, s.length() - 1);
+      }
+      splits.add(new Text(s));
+      split += distance;
+    }
+
+    conn.tableOperations().addSplits(tableName, splits);
+    System.out.println("Created Accumulo table '" + tableName + "' with " + numTablets + " tablets");
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/continuous/GenSplits.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/continuous/GenSplits.java b/core/src/main/java/org/apache/accumulo/testing/core/continuous/GenSplits.java
deleted file mode 100644
index be9ef7a..0000000
--- a/core/src/main/java/org/apache/accumulo/testing/core/continuous/GenSplits.java
+++ /dev/null
@@ -1,87 +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.testing.core.continuous;
-
-import java.util.List;
-
-import com.beust.jcommander.JCommander;
-import com.beust.jcommander.Parameter;
-import com.beust.jcommander.ParameterException;
-
-/**
- *
- */
-public class GenSplits {
-
-  static class Opts {
-    @Parameter(names = "--min", description = "minimum row")
-    long minRow = 0;
-
-    @Parameter(names = "--max", description = "maximum row")
-    long maxRow = Long.MAX_VALUE;
-
-    @Parameter(description = "<num tablets>")
-    List<String> args = null;
-  }
-
-  public static void main(String[] args) {
-
-    Opts opts = new Opts();
-    JCommander jcommander = new JCommander(opts);
-    jcommander.setProgramName(GenSplits.class.getSimpleName());
-
-    try {
-      jcommander.parse(args);
-    } catch (ParameterException pe) {
-      System.err.println(pe.getMessage());
-      jcommander.usage();
-      System.exit(-1);
-    }
-
-    if (opts.args == null || opts.args.size() != 1) {
-      jcommander.usage();
-      System.exit(-1);
-    }
-
-    int numTablets = Integer.parseInt(opts.args.get(0));
-
-    if (numTablets < 1) {
-      System.err.println("ERROR: numTablets < 1");
-      System.exit(-1);
-    }
-
-    if (opts.minRow >= opts.maxRow) {
-      System.err.println("ERROR: min >= max");
-      System.exit(-1);
-    }
-
-    int numSplits = numTablets - 1;
-    long distance = ((opts.maxRow - opts.minRow) / numTablets) + 1;
-    long split = distance;
-    for (int i = 0; i < numSplits; i++) {
-
-      String s = String.format("%016x", split + opts.minRow);
-
-      while (s.charAt(s.length() - 1) == '0') {
-        s = s.substring(0, s.length() - 1);
-      }
-
-      System.out.println(s);
-      split += distance;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/continuous/HistData.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/continuous/HistData.java b/core/src/main/java/org/apache/accumulo/testing/core/continuous/HistData.java
deleted file mode 100644
index 2fff363..0000000
--- a/core/src/main/java/org/apache/accumulo/testing/core/continuous/HistData.java
+++ /dev/null
@@ -1,49 +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.testing.core.continuous;
-
-import java.io.Serializable;
-import java.util.Objects;
-
-class HistData<T> implements Comparable<HistData<T>>, Serializable {
-  private static final long serialVersionUID = 1L;
-
-  T bin;
-  long count;
-
-  HistData(T bin) {
-    this.bin = bin;
-    count = 0;
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hashCode(bin) + Objects.hashCode(count);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public boolean equals(Object obj) {
-    return obj == this || (obj != null && obj instanceof HistData && 0 == compareTo((HistData<T>) obj));
-  }
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public int compareTo(HistData<T> o) {
-    return ((Comparable<T>) bin).compareTo(o.bin);
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/continuous/Histogram.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/continuous/Histogram.java b/core/src/main/java/org/apache/accumulo/testing/core/continuous/Histogram.java
deleted file mode 100644
index 0f1ba05..0000000
--- a/core/src/main/java/org/apache/accumulo/testing/core/continuous/Histogram.java
+++ /dev/null
@@ -1,153 +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.testing.core.continuous;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.BufferedOutputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import java.util.TreeSet;
-
-public class Histogram<T> implements Serializable {
-
-  private static final long serialVersionUID = 1L;
-
-  protected long sum;
-  protected HashMap<T,HistData<T>> counts;
-
-  public Histogram() {
-    sum = 0;
-    counts = new HashMap<>();
-  }
-
-  public void addPoint(T x) {
-    addPoint(x, 1);
-  }
-
-  public void addPoint(T x, long y) {
-
-    HistData<T> hd = counts.get(x);
-    if (hd == null) {
-      hd = new HistData<>(x);
-      counts.put(x, hd);
-    }
-
-    hd.count += y;
-    sum += y;
-  }
-
-  public long getCount(T x) {
-    HistData<T> hd = counts.get(x);
-    if (hd == null)
-      return 0;
-    return hd.count;
-  }
-
-  public double getPercentage(T x) {
-    if (getSum() == 0) {
-      return 0;
-    }
-    return (double) getCount(x) / (double) getSum() * 100.0;
-  }
-
-  public long getSum() {
-    return sum;
-  }
-
-  public List<T> getKeysInCountSortedOrder() {
-
-    ArrayList<HistData<T>> sortedCounts = new ArrayList<>(counts.values());
-
-    Collections.sort(sortedCounts, new Comparator<HistData<T>>() {
-      @Override
-      public int compare(HistData<T> o1, HistData<T> o2) {
-        if (o1.count < o2.count)
-          return -1;
-        if (o1.count > o2.count)
-          return 1;
-        return 0;
-      }
-    });
-
-    ArrayList<T> sortedKeys = new ArrayList<>();
-
-    for (Iterator<HistData<T>> iter = sortedCounts.iterator(); iter.hasNext();) {
-      HistData<T> hd = iter.next();
-      sortedKeys.add(hd.bin);
-    }
-
-    return sortedKeys;
-  }
-
-  public void print(StringBuilder out) {
-    TreeSet<HistData<T>> sortedCounts = new TreeSet<>(counts.values());
-
-    int maxValueLen = 0;
-
-    for (Iterator<HistData<T>> iter = sortedCounts.iterator(); iter.hasNext();) {
-      HistData<T> hd = iter.next();
-      if (("" + hd.bin).length() > maxValueLen) {
-        maxValueLen = ("" + hd.bin).length();
-      }
-    }
-
-    double psum = 0;
-
-    for (Iterator<HistData<T>> iter = sortedCounts.iterator(); iter.hasNext();) {
-      HistData<T> hd = iter.next();
-
-      psum += getPercentage(hd.bin);
-
-      out.append(String.format(" %" + (maxValueLen + 1) + "s %,16d %6.2f%s %6.2f%s%n", hd.bin + "", hd.count, getPercentage(hd.bin), "%", psum, "%"));
-    }
-    out.append(String.format("%n %" + (maxValueLen + 1) + "s %,16d %n", "TOTAL", sum));
-  }
-
-  public void save(String file) throws IOException {
-
-    FileOutputStream fos = new FileOutputStream(file);
-    BufferedOutputStream bos = new BufferedOutputStream(fos);
-    PrintStream ps = new PrintStream(bos, false, UTF_8.name());
-
-    TreeSet<HistData<T>> sortedCounts = new TreeSet<>(counts.values());
-    for (Iterator<HistData<T>> iter = sortedCounts.iterator(); iter.hasNext();) {
-      HistData<T> hd = iter.next();
-      ps.println(" " + hd.bin + " " + hd.count);
-    }
-
-    ps.close();
-  }
-
-  public Set<T> getKeys() {
-    return counts.keySet();
-  }
-
-  public void clear() {
-    counts.clear();
-    sum = 0;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/continuous/PrintScanTimeHistogram.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/continuous/PrintScanTimeHistogram.java b/core/src/main/java/org/apache/accumulo/testing/core/continuous/PrintScanTimeHistogram.java
deleted file mode 100644
index 7172f3a..0000000
--- a/core/src/main/java/org/apache/accumulo/testing/core/continuous/PrintScanTimeHistogram.java
+++ /dev/null
@@ -1,95 +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.testing.core.continuous;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.io.BufferedReader;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class PrintScanTimeHistogram {
-
-  private static final Logger log = LoggerFactory.getLogger(PrintScanTimeHistogram.class);
-
-  public static void main(String[] args) throws Exception {
-    Histogram<String> srqHist = new Histogram<>();
-    Histogram<String> fsrHist = new Histogram<>();
-
-    processFile(System.in, srqHist, fsrHist);
-
-    StringBuilder report = new StringBuilder();
-    report.append(String.format("%n *** Single row queries histogram *** %n"));
-    srqHist.print(report);
-    log.info("{}", report);
-
-    report = new StringBuilder();
-    report.append(String.format("%n *** Find start rows histogram *** %n"));
-    fsrHist.print(report);
-    log.info("{}", report);
-  }
-
-  private static void processFile(InputStream ins, Histogram<String> srqHist, Histogram<String> fsrHist) throws FileNotFoundException, IOException {
-    String line;
-    BufferedReader in = new BufferedReader(new InputStreamReader(ins, UTF_8));
-
-    while ((line = in.readLine()) != null) {
-
-      try {
-        String[] tokens = line.split(" ");
-
-        String type = tokens[0];
-        if (type.equals("SRQ")) {
-          long delta = Long.parseLong(tokens[3]);
-          String point = generateHistPoint(delta);
-          srqHist.addPoint(point);
-        } else if (type.equals("FSR")) {
-          long delta = Long.parseLong(tokens[3]);
-          String point = generateHistPoint(delta);
-          fsrHist.addPoint(point);
-        }
-      } catch (Exception e) {
-        log.error("Failed to process line '" + line + "'.", e);
-      }
-    }
-
-    in.close();
-  }
-
-  private static String generateHistPoint(long delta) {
-    String point;
-
-    if (delta / 1000.0 < .1) {
-      point = String.format("%07.2f", delta / 1000.0);
-      if (point.equals("0000.10"))
-        point = "0000.1x";
-    } else if (delta / 1000.0 < 1.0) {
-      point = String.format("%06.1fx", delta / 1000.0);
-      if (point.equals("0001.0x"))
-        point = "0001.xx";
-    } else {
-      point = String.format("%04.0f.xx", delta / 1000.0);
-    }
-    return point;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/continuous/TimeBinner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/continuous/TimeBinner.java b/core/src/main/java/org/apache/accumulo/testing/core/continuous/TimeBinner.java
index d43e2e5..843b251 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/continuous/TimeBinner.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/continuous/TimeBinner.java
@@ -75,7 +75,7 @@ public class TimeBinner {
 
     BufferedReader in = new BufferedReader(new InputStreamReader(System.in, UTF_8));
 
-    String line = null;
+    String line;
 
     HashMap<Long,DoubleWrapper> aggregation1 = new HashMap<>();
     HashMap<Long,DoubleWrapper> aggregation2 = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Environment.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Environment.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Environment.java
deleted file mode 100644
index 09d235e..0000000
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Environment.java
+++ /dev/null
@@ -1,229 +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.testing.core.randomwalk;
-
-import static java.util.Objects.requireNonNull;
-
-import java.io.File;
-import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.util.Properties;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.ClientConfiguration;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.MultiTableBatchWriter;
-import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.client.security.tokens.KerberosToken;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.testing.core.TestProps;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The test environment that is available for randomwalk tests. This includes configuration properties that are available to any randomwalk test and facilities
- * for creating client-side objects. This class is not thread-safe.
- */
-public class Environment {
-
-  private static final Logger log = LoggerFactory.getLogger(Environment.class);
-
-  private final Properties p;
-  private Instance instance = null;
-  private Connector connector = null;
-  private MultiTableBatchWriter mtbw = null;
-
-  /**
-   * Creates a new test environment.
-   *
-   * @param p
-   *          configuration properties
-   * @throws NullPointerException
-   *           if p is null
-   */
-  public Environment(Properties p) {
-    requireNonNull(p);
-    this.p = p;
-  }
-
-  /**
-   * Gets a copy of the configuration properties.
-   *
-   * @return a copy of the configuration properties
-   */
-  Properties copyConfigProperties() {
-    return new Properties(p);
-  }
-
-  /**
-   * Gets a configuration property.
-   *
-   * @param key
-   *          key
-   * @return property value
-   */
-  public String getConfigProperty(String key) {
-    return p.getProperty(key);
-  }
-
-  /**
-   * Gets the configured username.
-   *
-   * @return username
-   */
-  public String getUserName() {
-    return p.getProperty(TestProps.ACCUMULO_USERNAME);
-  }
-
-  /**
-   * Gets the configured password.
-   *
-   * @return password
-   */
-  public String getPassword() {
-    return p.getProperty(TestProps.ACCUMULO_PASSWORD);
-  }
-
-  /**
-   * Gets the configured keytab.
-   *
-   * @return path to keytab
-   */
-  public String getKeytab() {
-    return p.getProperty(TestProps.ACCUMULO_KEYTAB);
-  }
-
-  /**
-   * Gets this process's ID.
-   *
-   * @return pid
-   */
-  public String getPid() {
-    return ManagementFactory.getRuntimeMXBean().getName().split("@")[0];
-  }
-
-
-  public Configuration getHadoopConfiguration() {
-    Configuration config = new Configuration();
-    config.set("mapreduce.framework.name", "yarn");
-    // Setting below are required due to bundled jar breaking default config.
-    // See http://stackoverflow.com/questions/17265002/hadoop-no-filesystem-for-scheme-file
-    config.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName());
-    config.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName());
-    return config;
-  }
-
-  /**
-   * Gets an authentication token based on the configured password.
-   *
-   * @return authentication token
-   */
-  public AuthenticationToken getToken() {
-    String password = getPassword();
-    if (null != password) {
-      return new PasswordToken(getPassword());
-    }
-    String keytab = getKeytab();
-    if (null != keytab) {
-      File keytabFile = new File(keytab);
-      if (!keytabFile.exists() || !keytabFile.isFile()) {
-        throw new IllegalArgumentException("Provided keytab is not a normal file: " + keytab);
-      }
-      try {
-        UserGroupInformation.loginUserFromKeytab(getUserName(), keytabFile.getAbsolutePath());
-        return new KerberosToken();
-      } catch (IOException e) {
-        throw new RuntimeException("Failed to login", e);
-      }
-    }
-    throw new IllegalArgumentException("Must provide password or keytab in configuration");
-  }
-
-  /**
-   * Gets an Accumulo instance object. The same instance is reused after the first call.
-   *
-   * @return instance
-   */
-  public Instance getInstance() {
-    if (instance == null) {
-      String instance = p.getProperty(TestProps.ACCUMULO_INSTANCE);
-      String zookeepers = p.getProperty(TestProps.ZOOKEEPERS);
-      this.instance = new ZooKeeperInstance(ClientConfiguration.loadDefault().withInstance(instance).withZkHosts(zookeepers));
-    }
-    return instance;
-  }
-
-  /**
-   * Gets an Accumulo connector. The same connector is reused after the first call.
-   *
-   * @return connector
-   */
-  public Connector getConnector() throws AccumuloException, AccumuloSecurityException {
-    if (connector == null) {
-      connector = getInstance().getConnector(getUserName(), getToken());
-    }
-    return connector;
-  }
-
-  /**
-   * Gets a multitable batch writer. The same object is reused after the first call unless it is reset.
-   *
-   * @return multitable batch writer
-   * @throws NumberFormatException
-   *           if any of the numeric batch writer configuration properties cannot be parsed
-   * @throws NumberFormatException
-   *           if any configuration property cannot be parsed
-   */
-  public MultiTableBatchWriter getMultiTableBatchWriter() throws AccumuloException, AccumuloSecurityException {
-    if (mtbw == null) {
-      long maxMem = Long.parseLong(p.getProperty(TestProps.RW_BW_MAX_MEM));
-      long maxLatency = Long.parseLong(p.getProperty(TestProps.RW_BW_MAX_LATENCY));
-      int numThreads = Integer.parseInt(p.getProperty(TestProps.RW_BW_NUM_THREADS));
-      mtbw = getConnector().createMultiTableBatchWriter(
-          new BatchWriterConfig().setMaxMemory(maxMem).setMaxLatency(maxLatency, TimeUnit.MILLISECONDS).setMaxWriteThreads(numThreads));
-    }
-    return mtbw;
-  }
-
-  /**
-   * Checks if a multitable batch writer has been created by this wrapper.
-   *
-   * @return true if multitable batch writer is already created
-   */
-  public boolean isMultiTableBatchWriterInitialized() {
-    return mtbw != null;
-  }
-
-  /**
-   * Clears the multitable batch writer previously created and remembered by this wrapper.
-   */
-  public void resetMultiTableBatchWriter() {
-    if (mtbw == null)
-      return;
-    if (!mtbw.isClosed()) {
-      log.warn("Setting non-closed MultiTableBatchWriter to null (leaking resources)");
-    }
-    mtbw = null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Fixture.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Fixture.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Fixture.java
index cfa0e52..5b48c12 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Fixture.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Fixture.java
@@ -23,7 +23,7 @@ public abstract class Fixture {
 
   protected final Logger log = LoggerFactory.getLogger(this.getClass());
 
-  public abstract void setUp(State state, Environment env) throws Exception;
+  public abstract void setUp(State state, RandWalkEnv env) throws Exception;
 
-  public abstract void tearDown(State state, Environment env) throws Exception;
+  public abstract void tearDown(State state, RandWalkEnv env) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Framework.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Framework.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Framework.java
index 0b647bb..43c66b4 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Framework.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Framework.java
@@ -16,10 +16,10 @@
  */
 package org.apache.accumulo.testing.core.randomwalk;
 
-import java.io.FileInputStream;
 import java.util.HashMap;
 import java.util.Properties;
 
+import org.apache.accumulo.testing.core.TestProps;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -42,7 +42,7 @@ public class Framework {
    * @param startName
    *          Full name of starting graph or test
    */
-  public int run(String startName, State state, Environment env) {
+  public int run(String startName, State state, RandWalkEnv env) {
 
     try {
       Node node = getNode(startName);
@@ -86,15 +86,12 @@ public class Framework {
       System.exit(-1);
     }
 
-    Properties props = new Properties();
-    FileInputStream fis = new FileInputStream(args[0]);
-    props.load(fis);
-    fis.close();
+    Properties props = TestProps.loadFromFile(args[0]);
 
     log.info("Running random walk test with module: " + args[1]);
 
     State state = new State();
-    Environment env = new Environment(props);
+    RandWalkEnv env = new RandWalkEnv(props);
     getInstance().run(args[1], state, env);
 
     log.info("Test finished");

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Module.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Module.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Module.java
index 5cfa2e5..addc9b8 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Module.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Module.java
@@ -66,7 +66,7 @@ public class Module extends Node {
     }
 
     @Override
-    public void visit(State state, Environment env, Properties props) {
+    public void visit(State state, RandWalkEnv env, Properties props) {
       String print;
       if ((print = props.getProperty("print")) != null) {
         switch (print) {
@@ -98,7 +98,7 @@ public class Module extends Node {
     }
 
     @Override
-    public void visit(State state, Environment env, Properties props) throws Exception {
+    public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
       throw new Exception("You don't visit aliases!");
     }
 
@@ -185,7 +185,7 @@ public class Module extends Node {
   }
 
   @Override
-  public void visit(final State state, final Environment env, Properties props) throws Exception {
+  public void visit(final State state, final RandWalkEnv env, Properties props) throws Exception {
     int maxHops, maxSec;
     boolean teardown;
 
@@ -321,7 +321,7 @@ public class Module extends Node {
           if (test)
             stopTimer(nextNode);
         } catch (Exception e) {
-          log.debug("Connector belongs to user: " + env.getConnector().whoami());
+          log.debug("Connector belongs to user: " + env.getAccumuloConnector().whoami());
           log.debug("Exception occured at: " + System.currentTimeMillis());
           log.debug("Properties for node: " + nextNodeId);
           for (Entry<Object,Object> entry : nodeProps.entrySet()) {

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Node.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Node.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Node.java
index ed6e82e..7d4e038 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Node.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/Node.java
@@ -37,7 +37,7 @@ public abstract class Node {
    * @param env
    *          test environment
    */
-  public abstract void visit(State state, Environment env, Properties props) throws Exception;
+  public abstract void visit(State state, RandWalkEnv env, Properties props) throws Exception;
 
   @Override
   public boolean equals(Object o) {

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/RandWalkEnv.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/RandWalkEnv.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/RandWalkEnv.java
new file mode 100644
index 0000000..0dd2176
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/RandWalkEnv.java
@@ -0,0 +1,84 @@
+/*
+ * 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.testing.core.randomwalk;
+
+import java.util.Properties;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.MultiTableBatchWriter;
+import org.apache.accumulo.testing.core.TestEnv;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The test environment that is available for randomwalk tests. This includes configuration
+ * properties that are available to any randomwalk test and facilities for creating client-side
+ * objects. This class is not thread-safe.
+ */
+public class RandWalkEnv extends TestEnv {
+
+  private static final Logger log = LoggerFactory.getLogger(RandWalkEnv.class);
+
+  private MultiTableBatchWriter mtbw = null;
+
+  /**
+   * Creates a new test environment.
+   *
+   * @param p configuration properties
+   */
+  public RandWalkEnv(Properties p) {
+    super(p);
+  }
+
+  /**
+   * Gets a multitable batch writer. The same object is reused after the first call unless it is reset.
+   *
+   * @return multitable batch writer
+   * @throws NumberFormatException
+   *           if any of the numeric batch writer configuration properties cannot be parsed
+   * @throws NumberFormatException
+   *           if any configuration property cannot be parsed
+   */
+  public MultiTableBatchWriter getMultiTableBatchWriter() throws AccumuloException, AccumuloSecurityException {
+    if (mtbw == null) {
+      mtbw = getAccumuloConnector().createMultiTableBatchWriter(getBatchWriterConfig());
+    }
+    return mtbw;
+  }
+
+  /**
+   * Checks if a multitable batch writer has been created by this wrapper.
+   *
+   * @return true if multitable batch writer is already created
+   */
+  public boolean isMultiTableBatchWriterInitialized() {
+    return mtbw != null;
+  }
+
+  /**
+   * Clears the multitable batch writer previously created and remembered by this wrapper.
+   */
+  public void resetMultiTableBatchWriter() {
+    if (mtbw == null)
+      return;
+    if (!mtbw.isClosed()) {
+      log.warn("Setting non-closed MultiTableBatchWriter to null (leaking resources)");
+    }
+    mtbw = null;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/BulkImportTest.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/BulkImportTest.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/BulkImportTest.java
index 317a294..1f04cb8 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/BulkImportTest.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/BulkImportTest.java
@@ -18,7 +18,7 @@ package org.apache.accumulo.testing.core.randomwalk.bulk;
 
 import java.util.Properties;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 
 /**
@@ -30,7 +30,7 @@ public abstract class BulkImportTest extends BulkTest {
   public static final String SKIPPED_IMPORT = "skipped.import", TRUE = Boolean.TRUE.toString(), FALSE = Boolean.FALSE.toString();
 
   @Override
-  public void visit(final State state, Environment env, Properties props) throws Exception {
+  public void visit(final State state, RandWalkEnv env, Properties props) throws Exception {
     /**
      * Each visit() is performed sequentially and then submitted to the threadpool which will have async execution. As long as we're checking the state and
      * making decisions about what to do before we submit something to the thread pool, we're fine.
@@ -71,7 +71,7 @@ public abstract class BulkImportTest extends BulkTest {
     }
   }
 
-  private boolean shouldQueueMoreImports(State state, Environment env) throws Exception {
+  private boolean shouldQueueMoreImports(State state, RandWalkEnv env) throws Exception {
     // Only selectively import when it's BulkPlusOne. If we did a BulkPlusOne,
     // we must also do a BulkMinusOne to keep the table consistent
     if (getClass().equals(BulkPlusOne.class)) {

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/BulkMinusOne.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/BulkMinusOne.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/BulkMinusOne.java
index a9bb8f9..7b31d75 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/BulkMinusOne.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/BulkMinusOne.java
@@ -19,7 +19,7 @@ package org.apache.accumulo.testing.core.randomwalk.bulk;
 import static java.nio.charset.StandardCharsets.UTF_8;
 
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 
 public class BulkMinusOne extends BulkImportTest {
@@ -27,7 +27,7 @@ public class BulkMinusOne extends BulkImportTest {
   private static final Value negOne = new Value("-1".getBytes(UTF_8));
 
   @Override
-  protected void runLater(State state, Environment env) throws Exception {
+  protected void runLater(State state, RandWalkEnv env) throws Exception {
     log.info("Decrementing");
     BulkPlusOne.bulkLoadLots(log, state, env, negOne);
   }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/BulkPlusOne.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/BulkPlusOne.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/BulkPlusOne.java
index 32be88b..4ba7935 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/BulkPlusOne.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/BulkPlusOne.java
@@ -31,7 +31,7 @@ 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.file.rfile.RFile;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -57,7 +57,7 @@ public class BulkPlusOne extends BulkImportTest {
 
   private static final Value ONE = new Value("1".getBytes());
 
-  static void bulkLoadLots(Logger log, State state, Environment env, Value value) throws Exception {
+  static void bulkLoadLots(Logger log, State state, RandWalkEnv env, Value value) throws Exception {
     final Path dir = new Path("/tmp", "bulk_" + UUID.randomUUID().toString());
     final Path fail = new Path(dir.toString() + "_fail");
     final DefaultConfiguration defaultConfiguration = AccumuloConfiguration.getDefaultConfiguration();
@@ -97,7 +97,7 @@ public class BulkPlusOne extends BulkImportTest {
       }
       f.close();
     }
-    env.getConnector().tableOperations().importDirectory(Setup.getTableName(), dir.toString(), fail.toString(), true);
+    env.getAccumuloConnector().tableOperations().importDirectory(Setup.getTableName(), dir.toString(), fail.toString(), true);
     fs.delete(dir, true);
     FileStatus[] failures = fs.listStatus(fail);
     if (failures != null && failures.length > 0) {
@@ -109,7 +109,7 @@ public class BulkPlusOne extends BulkImportTest {
   }
 
   @Override
-  protected void runLater(State state, Environment env) throws Exception {
+  protected void runLater(State state, RandWalkEnv env) throws Exception {
     log.info("Incrementing");
     bulkLoadLots(log, state, env, ONE);
   }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/BulkTest.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/BulkTest.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/BulkTest.java
index 8695538..61fb8cf 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/BulkTest.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/BulkTest.java
@@ -18,14 +18,14 @@ package org.apache.accumulo.testing.core.randomwalk.bulk;
 
 import java.util.Properties;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public abstract class BulkTest extends Test {
 
   @Override
-  public void visit(final State state, final Environment env, Properties props) throws Exception {
+  public void visit(final State state, final RandWalkEnv env, Properties props) throws Exception {
     Setup.run(state, () -> {
       try {
         runLater(state, env);
@@ -35,6 +35,6 @@ public abstract class BulkTest extends Test {
     });
   }
 
-  abstract protected void runLater(State state, Environment env) throws Exception;
+  abstract protected void runLater(State state, RandWalkEnv env) throws Exception;
 
 }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Compact.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Compact.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Compact.java
index 356a7c9..bca7547 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Compact.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Compact.java
@@ -16,18 +16,18 @@
  */
 package org.apache.accumulo.testing.core.randomwalk.bulk;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.hadoop.io.Text;
 
 public class Compact extends SelectiveBulkTest {
 
   @Override
-  protected void runLater(State state, Environment env) throws Exception {
+  protected void runLater(State state, RandWalkEnv env) throws Exception {
     final Text[] points = Merge.getRandomTabletRange(state);
     final String rangeString = Merge.rangeToString(points);
     log.info("Compacting " + rangeString);
-    env.getConnector().tableOperations().compact(Setup.getTableName(), points[0], points[1], false, true);
+    env.getAccumuloConnector().tableOperations().compact(Setup.getTableName(), points[0], points[1], false, true);
     log.info("Compaction " + rangeString + " finished");
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/ConsistencyCheck.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/ConsistencyCheck.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/ConsistencyCheck.java
index eb21f30..71ff853 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/ConsistencyCheck.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/ConsistencyCheck.java
@@ -25,20 +25,20 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.hadoop.io.Text;
 
 public class ConsistencyCheck extends SelectiveBulkTest {
 
   @Override
-  protected void runLater(State state, Environment env) throws Exception {
+  protected void runLater(State state, RandWalkEnv env) throws Exception {
     Random rand = (Random) state.get("rand");
     Text row = Merge.getRandomRow(rand);
     log.info("Checking " + row);
-    String user = env.getConnector().whoami();
-    Authorizations auths = env.getConnector().securityOperations().getUserAuthorizations(user);
-    try (Scanner scanner = new IsolatedScanner(env.getConnector().createScanner(Setup.getTableName(), auths))) {
+    String user = env.getAccumuloConnector().whoami();
+    Authorizations auths = env.getAccumuloConnector().securityOperations().getUserAuthorizations(user);
+    try (Scanner scanner = new IsolatedScanner(env.getAccumuloConnector().createScanner(Setup.getTableName(), auths))) {
       scanner.setRange(new Range(row));
       scanner.fetchColumnFamily(BulkPlusOne.CHECK_COLUMN_FAMILY);
       Value v = null;

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Merge.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Merge.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Merge.java
index ebce171..dee6501 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Merge.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Merge.java
@@ -19,17 +19,17 @@ package org.apache.accumulo.testing.core.randomwalk.bulk;
 import java.util.Arrays;
 import java.util.Random;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.hadoop.io.Text;
 
 public class Merge extends SelectiveBulkTest {
 
   @Override
-  protected void runLater(State state, Environment env) throws Exception {
+  protected void runLater(State state, RandWalkEnv env) throws Exception {
     Text[] points = getRandomTabletRange(state);
     log.info("merging " + rangeToString(points));
-    env.getConnector().tableOperations().merge(Setup.getTableName(), points[0], points[1]);
+    env.getAccumuloConnector().tableOperations().merge(Setup.getTableName(), points[0], points[1]);
     log.info("merging " + rangeToString(points) + " complete");
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/SelectiveBulkTest.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/SelectiveBulkTest.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/SelectiveBulkTest.java
index a708942..e4cc165 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/SelectiveBulkTest.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/SelectiveBulkTest.java
@@ -18,7 +18,7 @@ package org.apache.accumulo.testing.core.randomwalk.bulk;
 
 import java.util.Properties;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 
 /**
@@ -27,7 +27,7 @@ import org.apache.accumulo.testing.core.randomwalk.State;
 public abstract class SelectiveBulkTest extends BulkTest {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     if (SelectiveQueueing.shouldQueueOperation(state, env)) {
       super.visit(state, env, props);
     } else {

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/SelectiveQueueing.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/SelectiveQueueing.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/SelectiveQueueing.java
index 59cf8aa..80882b5 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/SelectiveQueueing.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/SelectiveQueueing.java
@@ -19,7 +19,7 @@ package org.apache.accumulo.testing.core.randomwalk.bulk;
 import java.util.concurrent.ThreadPoolExecutor;
 
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -30,10 +30,10 @@ import org.slf4j.LoggerFactory;
 public class SelectiveQueueing {
   private static final Logger log = LoggerFactory.getLogger(SelectiveQueueing.class);
 
-  public static boolean shouldQueueOperation(State state, Environment env) throws Exception {
+  public static boolean shouldQueueOperation(State state, RandWalkEnv env) throws Exception {
     final ThreadPoolExecutor pool = (ThreadPoolExecutor) state.get("pool");
     long queuedThreads = pool.getTaskCount() - pool.getActiveCount() - pool.getCompletedTaskCount();
-    final Connector conn = env.getConnector();
+    final Connector conn = env.getAccumuloConnector();
     int numTservers = conn.instanceOperations().getTabletServers().size();
 
     if (!shouldQueue(queuedThreads, numTservers)) {

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Setup.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Setup.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Setup.java
index 635618f..c9ce9de 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Setup.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Setup.java
@@ -27,7 +27,7 @@ import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.iterators.LongCombiner;
 import org.apache.accumulo.core.iterators.user.SummingCombiner;
 import org.apache.accumulo.core.util.SimpleThreadPool;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.fs.FileSystem;
@@ -38,14 +38,14 @@ public class Setup extends Test {
   static String tableName = null;
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     Random rand = new Random();
     String hostname = InetAddress.getLocalHost().getHostName().replaceAll("[-.]", "_");
     String pid = env.getPid();
     tableName = String.format("bulk_%s_%s_%d", hostname, pid, System.currentTimeMillis());
     log.info("Starting bulk test on " + tableName);
 
-    TableOperations tableOps = env.getConnector().tableOperations();
+    TableOperations tableOps = env.getAccumuloConnector().tableOperations();
     try {
       if (!tableOps.exists(getTableName())) {
         tableOps.create(getTableName());

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Split.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Split.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Split.java
index 4ef212f..6a9270d 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Split.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Split.java
@@ -20,21 +20,21 @@ import java.util.Random;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.hadoop.io.Text;
 
 public class Split extends SelectiveBulkTest {
 
   @Override
-  protected void runLater(State state, Environment env) throws Exception {
+  protected void runLater(State state, RandWalkEnv env) throws Exception {
     SortedSet<Text> splits = new TreeSet<>();
     Random rand = (Random) state.get("rand");
     int count = rand.nextInt(20);
     for (int i = 0; i < count; i++)
       splits.add(new Text(String.format(BulkPlusOne.FMT, (rand.nextLong() & 0x7fffffffffffffffl) % BulkPlusOne.LOTS)));
     log.info("splitting " + splits);
-    env.getConnector().tableOperations().addSplits(Setup.getTableName(), splits);
+    env.getAccumuloConnector().tableOperations().addSplits(Setup.getTableName(), splits);
     log.info("split for " + splits + " finished");
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Verify.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Verify.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Verify.java
index 57aeff3..885be06 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Verify.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/bulk/Verify.java
@@ -31,7 +31,7 @@ import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -41,7 +41,7 @@ public class Verify extends Test {
   static byte[] zero = new byte[] {'0'};
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     ThreadPoolExecutor threadPool = Setup.getThreadPool(state);
     threadPool.shutdown();
     int lastSize = 0;
@@ -58,9 +58,9 @@ public class Verify extends Test {
       return;
     }
 
-    String user = env.getConnector().whoami();
-    Authorizations auths = env.getConnector().securityOperations().getUserAuthorizations(user);
-    Scanner scanner = env.getConnector().createScanner(Setup.getTableName(), auths);
+    String user = env.getAccumuloConnector().whoami();
+    Authorizations auths = env.getAccumuloConnector().securityOperations().getUserAuthorizations(user);
+    Scanner scanner = env.getAccumuloConnector().createScanner(Setup.getTableName(), auths);
     scanner.fetchColumnFamily(BulkPlusOne.CHECK_COLUMN_FAMILY);
     for (Entry<Key,Value> entry : scanner) {
       byte[] value = entry.getValue().get();
@@ -100,7 +100,7 @@ public class Verify extends Test {
     }
 
     log.info("Test successful on table " + Setup.getTableName());
-    env.getConnector().tableOperations().delete(Setup.getTableName());
+    env.getAccumuloConnector().tableOperations().delete(Setup.getTableName());
   }
 
   public static void main(String args[]) throws Exception {

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/AddSplits.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/AddSplits.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/AddSplits.java
index 18e0980..f83f9e9 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/AddSplits.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/AddSplits.java
@@ -26,7 +26,7 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -34,8 +34,8 @@ import org.apache.hadoop.io.Text;
 public class AddSplits extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/BatchScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/BatchScan.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/BatchScan.java
index 970e4df..26b3c27 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/BatchScan.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/BatchScan.java
@@ -33,15 +33,15 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class BatchScan extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/BatchWrite.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/BatchWrite.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/BatchWrite.java
index 39afec0..536e4a9 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/BatchWrite.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/BatchWrite.java
@@ -31,15 +31,15 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class BatchWrite extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/BulkImport.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/BulkImport.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/BulkImport.java
index 9c8eeb4..39a75c6 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/BulkImport.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/BulkImport.java
@@ -37,8 +37,7 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
 import org.apache.accumulo.core.file.rfile.RFile;
 import org.apache.accumulo.core.file.streams.PositionedOutputs;
-import org.apache.accumulo.core.util.CachedConfiguration;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.conf.Configuration;
@@ -95,8 +94,8 @@ public class BulkImport extends Test {
   }
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ChangeAuthorizations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ChangeAuthorizations.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ChangeAuthorizations.java
index 542c61e..28747d6 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ChangeAuthorizations.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ChangeAuthorizations.java
@@ -26,15 +26,15 @@ import java.util.Random;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class ChangeAuthorizations extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ChangePermissions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ChangePermissions.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ChangePermissions.java
index 173af6e..c830096 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ChangePermissions.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ChangePermissions.java
@@ -30,15 +30,15 @@ import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException
 import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class ChangePermissions extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CheckPermission.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CheckPermission.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CheckPermission.java
index 4848423..12e4306 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CheckPermission.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CheckPermission.java
@@ -25,15 +25,15 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class CheckPermission extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CloneTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CloneTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CloneTable.java
index 9697aee..8c4bd95 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CloneTable.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CloneTable.java
@@ -27,15 +27,15 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class CloneTable extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Compact.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Compact.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Compact.java
index cd2dc1a..4ecb34d 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Compact.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Compact.java
@@ -23,7 +23,7 @@ import java.util.Random;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -31,8 +31,8 @@ import org.apache.hadoop.io.Text;
 public class Compact extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ConcurrentFixture.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ConcurrentFixture.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ConcurrentFixture.java
index 388e439..ad2ae35 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ConcurrentFixture.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/ConcurrentFixture.java
@@ -20,7 +20,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.Fixture;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.hadoop.io.Text;
@@ -34,10 +34,10 @@ import org.apache.hadoop.io.Text;
 public class ConcurrentFixture extends Fixture {
 
   @Override
-  public void setUp(State state, Environment env) throws Exception {}
+  public void setUp(State state, RandWalkEnv env) throws Exception {}
 
   @Override
-  public void tearDown(State state, Environment env) throws Exception {}
+  public void tearDown(State state, RandWalkEnv env) throws Exception {}
 
   /**
    *

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Config.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Config.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Config.java
index a640def..23106cf 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Config.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/Config.java
@@ -23,7 +23,7 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
 import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.commons.math3.random.RandomDataGenerator;
@@ -104,14 +104,14 @@ public class Config extends Test {
   // @formatter:on
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
     // reset any previous setting
     Object lastSetting = state.getOkIfAbsent(LAST_SETTING);
     if (lastSetting != null) {
       int choice = Integer.parseInt(lastSetting.toString());
       Property property = settings[choice].property;
       log.debug("Setting " + property.getKey() + " back to " + property.getDefaultValue());
-      env.getConnector().instanceOperations().setProperty(property.getKey(), property.getDefaultValue());
+      env.getAccumuloConnector().instanceOperations().setProperty(property.getKey(), property.getDefaultValue());
     }
     lastSetting = state.getOkIfAbsent(LAST_TABLE_SETTING);
     if (lastSetting != null) {
@@ -119,10 +119,10 @@ public class Config extends Test {
       String table = parts[0];
       int choice = Integer.parseInt(parts[1]);
       Property property = tableSettings[choice].property;
-      if (env.getConnector().tableOperations().exists(table)) {
+      if (env.getAccumuloConnector().tableOperations().exists(table)) {
         log.debug("Setting " + property.getKey() + " on " + table + " back to " + property.getDefaultValue());
         try {
-          env.getConnector().tableOperations().setProperty(table, property.getKey(), property.getDefaultValue());
+          env.getAccumuloConnector().tableOperations().setProperty(table, property.getKey(), property.getDefaultValue());
         } catch (AccumuloException ex) {
           if (ex.getCause() instanceof ThriftTableOperationException) {
             ThriftTableOperationException ttoe = (ThriftTableOperationException) ex.getCause();
@@ -139,10 +139,10 @@ public class Config extends Test {
       String namespace = parts[0];
       int choice = Integer.parseInt(parts[1]);
       Property property = tableSettings[choice].property;
-      if (env.getConnector().namespaceOperations().exists(namespace)) {
+      if (env.getAccumuloConnector().namespaceOperations().exists(namespace)) {
         log.debug("Setting " + property.getKey() + " on " + namespace + " back to " + property.getDefaultValue());
         try {
-          env.getConnector().namespaceOperations().setProperty(namespace, property.getKey(), property.getDefaultValue());
+          env.getAccumuloConnector().namespaceOperations().setProperty(namespace, property.getKey(), property.getDefaultValue());
         } catch (AccumuloException ex) {
           if (ex.getCause() instanceof ThriftTableOperationException) {
             ThriftTableOperationException ttoe = (ThriftTableOperationException) ex.getCause();
@@ -167,13 +167,13 @@ public class Config extends Test {
     }
   }
 
-  private void changeTableSetting(RandomDataGenerator random, State state, Environment env, Properties props) throws Exception {
+  private void changeTableSetting(RandomDataGenerator random, State state, RandWalkEnv env, Properties props) throws Exception {
     // pick a random property
     int choice = random.nextInt(0, tableSettings.length - 1);
     Setting setting = tableSettings[choice];
 
     // pick a random table
-    SortedSet<String> tables = env.getConnector().tableOperations().list().tailSet("ctt").headSet("ctu");
+    SortedSet<String> tables = env.getAccumuloConnector().tableOperations().list().tailSet("ctt").headSet("ctu");
     if (tables.isEmpty())
       return;
     String table = random.nextSample(tables, 1)[0].toString();
@@ -183,7 +183,7 @@ public class Config extends Test {
     state.set(LAST_TABLE_SETTING, table + "," + choice);
     log.debug("Setting " + setting.property.getKey() + " on table " + table + " to " + newValue);
     try {
-      env.getConnector().tableOperations().setProperty(table, setting.property.getKey(), "" + newValue);
+      env.getAccumuloConnector().tableOperations().setProperty(table, setting.property.getKey(), "" + newValue);
     } catch (AccumuloException ex) {
       if (ex.getCause() instanceof ThriftTableOperationException) {
         ThriftTableOperationException ttoe = (ThriftTableOperationException) ex.getCause();
@@ -194,13 +194,13 @@ public class Config extends Test {
     }
   }
 
-  private void changeNamespaceSetting(RandomDataGenerator random, State state, Environment env, Properties props) throws Exception {
+  private void changeNamespaceSetting(RandomDataGenerator random, State state, RandWalkEnv env, Properties props) throws Exception {
     // pick a random property
     int choice = random.nextInt(0, tableSettings.length - 1);
     Setting setting = tableSettings[choice];
 
     // pick a random table
-    SortedSet<String> namespaces = env.getConnector().namespaceOperations().list().tailSet("nspc").headSet("nspd");
+    SortedSet<String> namespaces = env.getAccumuloConnector().namespaceOperations().list().tailSet("nspc").headSet("nspd");
     if (namespaces.isEmpty())
       return;
     String namespace = random.nextSample(namespaces, 1)[0].toString();
@@ -210,7 +210,7 @@ public class Config extends Test {
     state.set(LAST_NAMESPACE_SETTING, namespace + "," + choice);
     log.debug("Setting " + setting.property.getKey() + " on namespace " + namespace + " to " + newValue);
     try {
-      env.getConnector().namespaceOperations().setProperty(namespace, setting.property.getKey(), "" + newValue);
+      env.getAccumuloConnector().namespaceOperations().setProperty(namespace, setting.property.getKey(), "" + newValue);
     } catch (AccumuloException ex) {
       if (ex.getCause() instanceof ThriftTableOperationException) {
         ThriftTableOperationException ttoe = (ThriftTableOperationException) ex.getCause();
@@ -221,7 +221,7 @@ public class Config extends Test {
     }
   }
 
-  private void changeSetting(RandomDataGenerator random, State state, Environment env, Properties props) throws Exception {
+  private void changeSetting(RandomDataGenerator random, State state, RandWalkEnv env, Properties props) throws Exception {
     // pick a random property
     int choice = random.nextInt(0, settings.length - 1);
     Setting setting = settings[choice];
@@ -229,7 +229,7 @@ public class Config extends Test {
     long newValue = random.nextLong(setting.min, setting.max);
     state.set(LAST_SETTING, "" + choice);
     log.debug("Setting " + setting.property.getKey() + " to " + newValue);
-    env.getConnector().instanceOperations().setProperty(setting.property.getKey(), "" + newValue);
+    env.getAccumuloConnector().instanceOperations().setProperty(setting.property.getKey(), "" + newValue);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CreateNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CreateNamespace.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CreateNamespace.java
index 71250d8..6c4ff3e 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CreateNamespace.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CreateNamespace.java
@@ -22,15 +22,15 @@ import java.util.Random;
 
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.NamespaceExistsException;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class CreateNamespace extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CreateTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CreateTable.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CreateTable.java
index 648732e..f9ec6ce 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CreateTable.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CreateTable.java
@@ -25,15 +25,15 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class CreateTable extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CreateUser.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CreateUser.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CreateUser.java
index 708d48f..0db7b2b 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CreateUser.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/CreateUser.java
@@ -23,14 +23,14 @@ import java.util.Random;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class CreateUser extends Test {
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/DeleteNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/DeleteNamespace.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/DeleteNamespace.java
index 7f564ae..e0ce1b2 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/DeleteNamespace.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/DeleteNamespace.java
@@ -23,15 +23,15 @@ import java.util.Random;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.NamespaceNotEmptyException;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 
 public class DeleteNamespace extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/DeleteRange.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/DeleteRange.java b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/DeleteRange.java
index f8db0e7..cdbb36e 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/DeleteRange.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/randomwalk/concurrent/DeleteRange.java
@@ -25,7 +25,7 @@ import java.util.Random;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
-import org.apache.accumulo.testing.core.randomwalk.Environment;
+import org.apache.accumulo.testing.core.randomwalk.RandWalkEnv;
 import org.apache.accumulo.testing.core.randomwalk.State;
 import org.apache.accumulo.testing.core.randomwalk.Test;
 import org.apache.hadoop.io.Text;
@@ -33,8 +33,8 @@ import org.apache.hadoop.io.Text;
 public class DeleteRange extends Test {
 
   @Override
-  public void visit(State state, Environment env, Properties props) throws Exception {
-    Connector conn = env.getConnector();
+  public void visit(State state, RandWalkEnv env, Properties props) throws Exception {
+    Connector conn = env.getAccumuloConnector();
 
     Random rand = (Random) state.get("rand");
 


[4/5] accumulo-testing git commit: ACCUMULO-4510 Refactored Continous Ingest tests

Posted by mw...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/start-stats.sh
----------------------------------------------------------------------
diff --git a/continuous/start-stats.sh b/continuous/start-stats.sh
deleted file mode 100755
index 0a90364..0000000
--- a/continuous/start-stats.sh
+++ /dev/null
@@ -1,49 +0,0 @@
-#! /usr/bin/env bash
-
-# 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.
-
-# Start: Resolve Script Directory
-SOURCE="${BASH_SOURCE[0]}"
-while [[ -h "${SOURCE}" ]]; do # resolve $SOURCE until the file is no longer a symlink
-   bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-   SOURCE=$(readlink "${SOURCE}")
-   [[ "${SOURCE}" != /* ]] && SOURCE="${bin}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located
-done
-bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-script=$( basename "${SOURCE}" )
-# Stop: Resolve Script Directory
-
-CONTINUOUS_CONF_DIR=${CONTINUOUS_CONF_DIR:-${bin}}
-. "$CONTINUOUS_CONF_DIR/continuous-env.sh"
-
-mkdir -p "$CONTINUOUS_LOG_DIR"
-
-CONFIG_OUT=$CONTINUOUS_LOG_DIR/$(date +%Y%m%d%H%M%S)_$(hostname)_config.out
-
-cat "$ACCUMULO_CONF_DIR/accumulo-env.sh" > "$CONFIG_OUT"
-echo >> "$CONFIG_OUT"
-echo -e "config -np\nconfig -t $TABLE -np\nquit" | "$ACCUMULO_HOME/bin/accumulo" shell -u "$USER" -p "$PASS" >> "$CONFIG_OUT"
-echo >> "$CONFIG_OUT"
-cat "$CONTINUOUS_CONF_DIR/continuous-env.sh" >> "$CONFIG_OUT"
-echo >> "$CONFIG_OUT"
-wc -l "$CONTINUOUS_CONF_DIR/walkers.txt" >> "$CONFIG_OUT"
-wc -l "$CONTINUOUS_CONF_DIR/ingesters.txt" >> "$CONFIG_OUT"
-wc -l "$CONTINUOUS_CONF_DIR/scanners.txt" >> "$CONFIG_OUT"
-wc -l "$CONTINUOUS_CONF_DIR/batch_walkers.txt" >> "$CONFIG_OUT"
-
-
-nohup "$ACCUMULO_HOME/bin/accumulo" org.apache.accumulo.test.continuous.ContinuousStatsCollector --table "$TABLE" -i "$INSTANCE_NAME" -z "$ZOO_KEEPERS" -u "$USER" -p "$PASS" >"$CONTINUOUS_LOG_DIR/$(date +%Y%m%d%H%M%S)_$(hostname)_stats.out" 2>"$CONTINUOUS_LOG_DIR/$(date +%Y%m%d%H%M%S)_$(hostname)_stats.err" &
-

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/start-walkers.sh
----------------------------------------------------------------------
diff --git a/continuous/start-walkers.sh b/continuous/start-walkers.sh
deleted file mode 100755
index d9bbff4..0000000
--- a/continuous/start-walkers.sh
+++ /dev/null
@@ -1,41 +0,0 @@
-#! /usr/bin/env bash
-
-# 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.
-
-# Start: Resolve Script Directory
-SOURCE="${BASH_SOURCE[0]}"
-while [[ -h "${SOURCE}" ]]; do # resolve $SOURCE until the file is no longer a symlink
-   bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-   SOURCE=$(readlink "${SOURCE}")
-   [[ "${SOURCE}" != /* ]] && SOURCE="${bin}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located
-done
-bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-script=$( basename "${SOURCE}" )
-# Stop: Resolve Script Directory
-
-CONTINUOUS_CONF_DIR=${CONTINUOUS_CONF_DIR:-${bin}}
-. "$CONTINUOUS_CONF_DIR/continuous-env.sh"
-
-DEBUG_OPT=''
-if [[ "$DEBUG_WALKER" == "on" ]] ; then
-   DEBUG_OPT="--debug $CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_walk.log";
-fi
-
-AUTH_OPT=''
-[[ -n "$AUTHS" ]] && AUTH_OPT="--auths \"$AUTHS\""
-
-pssh -h "$CONTINUOUS_CONF_DIR/walkers.txt" "mkdir -p $CONTINUOUS_LOG_DIR; nohup $ACCUMULO_HOME/bin/accumulo org.apache.accumulo.test.continuous.ContinuousWalk $DEBUG_OPT $AUTH_OPT -i $INSTANCE_NAME -z $ZOO_KEEPERS -u $USER -p $PASS --table $TABLE --min $MIN --max $MAX --sleep $SLEEP_TIME >$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_walk.out 2>$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_walk.err &" < /dev/null
-

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/stop-agitator.sh
----------------------------------------------------------------------
diff --git a/continuous/stop-agitator.sh b/continuous/stop-agitator.sh
deleted file mode 100755
index d8f30e4..0000000
--- a/continuous/stop-agitator.sh
+++ /dev/null
@@ -1,51 +0,0 @@
-#! /usr/bin/env bash
-# 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.
-
-# Start: Resolve Script Directory
-SOURCE="${BASH_SOURCE[0]}"
-while [[ -h "${SOURCE}" ]]; do # resolve $SOURCE until the file is no longer a symlink
-   bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-   SOURCE=$(readlink "${SOURCE}")
-   [[ "${SOURCE}" != /* ]] && SOURCE="${bin}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located
-done
-bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-script=$( basename "${SOURCE}" )
-# Stop: Resolve Script Directory
-
-CONTINUOUS_CONF_DIR=${CONTINUOUS_CONF_DIR:-${bin}}
-. "$CONTINUOUS_CONF_DIR/continuous-env.sh"
-
-# Try to use sudo when we wouldn't normally be able to kill the processes
-[[ -n $AGITATOR_USER ]] || AGITATOR_USER=$(whoami)
-if [[ $AGITATOR_USER == root ]]; then
-  echo "Stopping all processes matching 'agitator.pl' as root"
-  pkill -f agitator.pl 2>/dev/null
-elif [[ $AGITATOR_USER == "$ACCUMULO_USER" ]];  then
-  echo "Stopping all processes matching 'datanode-agitator.pl' as $HDFS_USER"
-  sudo -u "$HDFS_USER" pkill -f datanode-agitator.pl 2>/dev/null
-  echo "Stopping all processes matching 'hdfs-agitator.pl' as $HDFS_USER"
-  sudo -u "$HDFS_USER" pkill -f hdfs-agitator.pl 2>/dev/null
-  echo "Stopping all processes matching 'agitator.pl' as $AGITATOR_USER"
-  pkill -f agitator.pl 2>/dev/null 2>/dev/null
-else
-  echo "Stopping all processes matching 'datanode-agitator.pl' as $HDFS_USER"
-  sudo -u "$HDFS_USER" pkill -f datanode-agitator.pl 2>/dev/null
-  echo "Stopping all processes matching 'hdfs-agitator.pl' as $HDFS_USER"
-  sudo -u "$HDFS_USER" pkill -f hdfs-agitator.pl 2>/dev/null
-  echo "Stopping all processes matching 'agitator.pl' as $ACCUMULO_USER"
-  sudo -u "$ACCUMULO_USER" pkill -f agitator.pl 2>/dev/null
-fi
-

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/stop-batchwalkers.sh
----------------------------------------------------------------------
diff --git a/continuous/stop-batchwalkers.sh b/continuous/stop-batchwalkers.sh
deleted file mode 100755
index 4696387..0000000
--- a/continuous/stop-batchwalkers.sh
+++ /dev/null
@@ -1,33 +0,0 @@
-#! /usr/bin/env bash
-
-# 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.
-
-# Start: Resolve Script Directory
-SOURCE="${BASH_SOURCE[0]}"
-while [[ -h "${SOURCE}" ]]; do # resolve $SOURCE until the file is no longer a symlink
-   bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-   SOURCE=$(readlink "${SOURCE}")
-   [[ "${SOURCE}" != /* ]] && SOURCE="${bin}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located
-done
-bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-script=$( basename "${SOURCE}" )
-# Stop: Resolve Script Directory
-
-CONTINUOUS_CONF_DIR=${CONTINUOUS_CONF_DIR:-${bin}}
-. "$CONTINUOUS_CONF_DIR/continuous-env.sh"
-
-pssh -h "$CONTINUOUS_CONF_DIR/batch_walkers.txt" "pkill -f '[o]rg.apache.accumulo.test.continuous.ContinuousBatchWalker'" < /dev/null
-

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/stop-ingest.sh
----------------------------------------------------------------------
diff --git a/continuous/stop-ingest.sh b/continuous/stop-ingest.sh
deleted file mode 100755
index d159bf7..0000000
--- a/continuous/stop-ingest.sh
+++ /dev/null
@@ -1,33 +0,0 @@
-#! /usr/bin/env bash
-
-# 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.
-
-# Start: Resolve Script Directory
-SOURCE="${BASH_SOURCE[0]}"
-while [[ -h "${SOURCE}" ]]; do # resolve $SOURCE until the file is no longer a symlink
-   bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-   SOURCE=$(readlink "${SOURCE}")
-   [[ "${SOURCE}" != /* ]] && SOURCE="${bin}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located
-done
-bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-script=$( basename "${SOURCE}" )
-# Stop: Resolve Script Directory
-
-CONTINUOUS_CONF_DIR=${CONTINUOUS_CONF_DIR:-${bin}}
-. "$CONTINUOUS_CONF_DIR/continuous-env.sh"
-
-pssh -h "$CONTINUOUS_CONF_DIR/ingesters.txt" "pkill -f '[o]rg.apache.accumulo.test.continuous.ContinuousIngest'" < /dev/null
-

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/stop-scanners.sh
----------------------------------------------------------------------
diff --git a/continuous/stop-scanners.sh b/continuous/stop-scanners.sh
deleted file mode 100755
index cf927b0..0000000
--- a/continuous/stop-scanners.sh
+++ /dev/null
@@ -1,33 +0,0 @@
-#! /usr/bin/env bash
-
-# 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.
-
-# Start: Resolve Script Directory
-SOURCE="${BASH_SOURCE[0]}"
-while [[ -h "${SOURCE}" ]]; do # resolve $SOURCE until the file is no longer a symlink
-   bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-   SOURCE=$(readlink "${SOURCE}")
-   [[ "${SOURCE}" != /* ]] && SOURCE="${bin}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located
-done
-bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-script=$( basename "${SOURCE}" )
-# Stop: Resolve Script Directory
-
-CONTINUOUS_CONF_DIR=${CONTINUOUS_CONF_DIR:-${bin}}
-. "$CONTINUOUS_CONF_DIR/continuous-env.sh"
-
-pssh -h "$CONTINUOUS_CONF_DIR/scanners.txt" "pkill -f '[o]rg.apache.accumulo.test.continuous.ContinuousScanner'" < /dev/null
-

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/stop-stats.sh
----------------------------------------------------------------------
diff --git a/continuous/stop-stats.sh b/continuous/stop-stats.sh
deleted file mode 100755
index 9886eec..0000000
--- a/continuous/stop-stats.sh
+++ /dev/null
@@ -1,33 +0,0 @@
-#! /usr/bin/env bash
-
-# 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.
-
-# Start: Resolve Script Directory
-SOURCE="${BASH_SOURCE[0]}"
-while [[ -h "${SOURCE}" ]]; do # resolve $SOURCE until the file is no longer a symlink
-   bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-   SOURCE=$(readlink "${SOURCE}")
-   [[ "${SOURCE}" != /* ]] && SOURCE="${bin}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located
-done
-bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-script=$( basename "${SOURCE}" )
-# Stop: Resolve Script Directory
-
-CONTINUOUS_CONF_DIR=${CONTINUOUS_CONF_DIR:-${bin}}
-. "$CONTINUOUS_CONF_DIR/continuous-env.sh"
-
-pkill -f org.apache.accumulo.test.continuous.ContinuousStatsCollector
-

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/stop-walkers.sh
----------------------------------------------------------------------
diff --git a/continuous/stop-walkers.sh b/continuous/stop-walkers.sh
deleted file mode 100755
index 2c22cfa..0000000
--- a/continuous/stop-walkers.sh
+++ /dev/null
@@ -1,33 +0,0 @@
-#! /usr/bin/env bash
-
-# 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.
-
-# Start: Resolve Script Directory
-SOURCE="${BASH_SOURCE[0]}"
-while [[ -h "${SOURCE}" ]]; do # resolve $SOURCE until the file is no longer a symlink
-   bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-   SOURCE=$(readlink "${SOURCE}")
-   [[ "${SOURCE}" != /* ]] && SOURCE="${bin}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located
-done
-bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-script=$( basename "${SOURCE}" )
-# Stop: Resolve Script Directory
-
-CONTINUOUS_CONF_DIR=${CONTINUOUS_CONF_DIR:-${bin}}
-. "$CONTINUOUS_CONF_DIR/continuous-env.sh"
-
-pssh -h "$CONTINUOUS_CONF_DIR/walkers.txt" "pkill -f '[o]rg.apache.accumulo.test.continuous.ContinuousWalk'" < /dev/null
-

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/tserver-agitator.pl
----------------------------------------------------------------------
diff --git a/continuous/tserver-agitator.pl b/continuous/tserver-agitator.pl
deleted file mode 100755
index 0e65a50..0000000
--- a/continuous/tserver-agitator.pl
+++ /dev/null
@@ -1,134 +0,0 @@
-#! /usr/bin/env perl
-
-# 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.
-
-
-use POSIX qw(strftime);
-use Cwd qw();
-
-if(scalar(@ARGV) != 4 && scalar(@ARGV) != 2){
-  print "Usage : tserver-agitator.pl <min sleep before kill in minutes>[:max sleep before kill in minutes] <min sleep before tup in minutes>[:max sleep before tup in minutes] [<min kill> <max kill>]\n";
-  exit(1);
-}
-
-my $ACCUMULO_HOME;
-if( defined $ENV{'ACCUMULO_HOME'} ){
-  $ACCUMULO_HOME = $ENV{'ACCUMULO_HOME'};
-} else {
-  $cwd=Cwd::cwd();
-  $ACCUMULO_HOME=$cwd . '/../../..';
-}
-
-print "ACCUMULO_HOME=$ACCUMULO_HOME\n";
-
-@sleeprange1 = split(/:/, $ARGV[0]);
-$sleep1 = $sleeprange1[0];
-
-@sleeprange2 = split(/:/, $ARGV[1]);
-$sleep2 = $sleeprange2[0];
-
-if (scalar(@sleeprange1) > 1) {
-  $sleep1max = $sleeprange1[1] + 1;
-} else {
-  $sleep1max = $sleep1;
-}
-
-if ($sleep1 > $sleep1max) {
-  die("sleep1 > sleep1max $sleep1 > $sleep1max");
-}
-
-if (scalar(@sleeprange2) > 1) {
-  $sleep2max = $sleeprange2[1] + 1;
-} else {
-  $sleep2max = $sleep2;
-}
-
-if($sleep2 > $sleep2max){
-  die("sleep2 > sleep2max $sleep2 > $sleep2max");
-}
-
-if(defined $ENV{'ACCUMULO_CONF_DIR'}){
-  $ACCUMULO_CONF_DIR = $ENV{'ACCUMULO_CONF_DIR'};
-}else{
-  $ACCUMULO_CONF_DIR = $ACCUMULO_HOME . '/conf';
-}
-
-if(scalar(@ARGV) == 4){
-  $minKill = $ARGV[2];
-  $maxKill = $ARGV[3];
-}else{
-  $minKill = 1;
-  $maxKill = 1;
-}
-
-if($minKill > $maxKill){
-  die("minKill > maxKill $minKill > $maxKill");
-}
-
-@tserversRaw = `cat $ACCUMULO_CONF_DIR/tservers`;
-chomp(@tserversRaw);
-
-for $tserver (@tserversRaw){
-  if($tserver eq "" || substr($tserver,0,1) eq "#"){
-    next;
-  }
-
-  push(@tservers, $tserver);
-}
-
-
-if(scalar(@tservers) < $maxKill){
-  print STDERR "WARN setting maxKill to ".scalar(@tservers)."\n";
-  $maxKill = scalar(@tservers);
-}
-
-if ($minKill > $maxKill){
-  print STDERR "WARN setting minKill to equal maxKill\n";
-  $minKill = $maxKill;
-}
-
-while(1){
-
-  $numToKill = int(rand($maxKill - $minKill + 1)) + $minKill;
-  %killed = {};
-  $server = "";
-
-  for($i = 0; $i < $numToKill; $i++){
-    while($server eq "" || $killed{$server} != undef){
-      $index = int(rand(scalar(@tservers)));
-      $server = $tservers[$index];
-    }
-
-    $killed{$server} = 1;
-
-    $t = strftime "%Y%m%d %H:%M:%S", localtime;
-
-    print STDERR "$t Killing tserver on $server\n";
-    # We're the accumulo user, just run the commandj
-    system("$ACCUMULO_HOME/bin/stop-server.sh $server 'accumulo-start.jar' tserver KILL");
-  }
-
-  $nextsleep2 = int(rand($sleep2max - $sleep2)) + $sleep2;
-  sleep($nextsleep2 * 60);
-  $t = strftime "%Y%m%d %H:%M:%S", localtime;
-  print STDERR "$t Running tup\n";
-  # restart the as them as the accumulo user
-  system("$ACCUMULO_HOME/bin/tup.sh");
-
-  $nextsleep1 = int(rand($sleep1max - $sleep1)) + $sleep1;
-  sleep($nextsleep1 * 60);
-}
-

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/walkers.txt.example
----------------------------------------------------------------------
diff --git a/continuous/walkers.txt.example b/continuous/walkers.txt.example
deleted file mode 100644
index b59052d..0000000
--- a/continuous/walkers.txt.example
+++ /dev/null
@@ -1,17 +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.
-
-host3
-host4

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/TestEnv.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/TestEnv.java b/core/src/main/java/org/apache/accumulo/testing/core/TestEnv.java
new file mode 100644
index 0000000..55fecb7
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/TestEnv.java
@@ -0,0 +1,179 @@
+package org.apache.accumulo.testing.core;
+
+import static java.util.Objects.requireNonNull;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ClientConfiguration;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.client.security.tokens.KerberosToken;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.tools.CLI;
+import org.apache.hadoop.security.UserGroupInformation;
+
+public class TestEnv {
+
+  protected final Properties p;
+  private Instance instance = null;
+  private Connector connector = null;
+
+  /**
+   * Creates new test environment using provided properties
+   *
+   * @param p Properties
+   */
+  public TestEnv(Properties p) {
+    requireNonNull(p);
+    this.p = p;
+  }
+
+  /**
+   * Gets a copy of the configuration properties.
+   *
+   * @return a copy of the configuration properties
+   */
+  public Properties copyConfigProperties() {
+    return new Properties(p);
+  }
+
+  /**
+   * Gets a configuration property.
+   *
+   * @param key key
+   * @return property value
+   */
+  public String getConfigProperty(String key) {
+    return p.getProperty(key);
+  }
+
+  /**
+   * Gets the configured username.
+   *
+   * @return username
+   */
+  public String getAccumuloUserName() {
+    return p.getProperty(TestProps.ACCUMULO_USERNAME);
+  }
+
+  /**
+   * Gets the configured password.
+   *
+   * @return password
+   */
+  public String getAccumuloPassword() {
+    return p.getProperty(TestProps.ACCUMULO_PASSWORD);
+  }
+
+  /**
+   * Gets the configured keytab.
+   *
+   * @return path to keytab
+   */
+  public String getAccumuloKeytab() {
+    return p.getProperty(TestProps.ACCUMULO_KEYTAB);
+  }
+
+  /**
+   * Gets this process's ID.
+   *
+   * @return pid
+   */
+  public String getPid() {
+    return ManagementFactory.getRuntimeMXBean().getName().split("@")[0];
+  }
+
+
+  public Configuration getHadoopConfiguration() {
+    Configuration config = new Configuration();
+    config.set("mapreduce.framework.name", "yarn");
+    // Setting below are required due to bundled jar breaking default config.
+    // See http://stackoverflow.com/questions/17265002/hadoop-no-filesystem-for-scheme-file
+    config.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName());
+    config.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName());
+    return config;
+  }
+
+  /**
+   * Gets an authentication token based on the configured password.
+   */
+  public AuthenticationToken getToken() {
+    String password = getAccumuloPassword();
+    if (null != password) {
+      return new PasswordToken(getAccumuloPassword());
+    }
+    String keytab = getAccumuloKeytab();
+    if (null != keytab) {
+      File keytabFile = new File(keytab);
+      if (!keytabFile.exists() || !keytabFile.isFile()) {
+        throw new IllegalArgumentException("Provided keytab is not a normal file: " + keytab);
+      }
+      try {
+        UserGroupInformation.loginUserFromKeytab(getAccumuloUserName(), keytabFile.getAbsolutePath());
+        return new KerberosToken();
+      } catch (IOException e) {
+        throw new RuntimeException("Failed to login", e);
+      }
+    }
+    throw new IllegalArgumentException("Must provide password or keytab in configuration");
+  }
+
+  public String getAccumuloInstanceName() {
+    return p.getProperty(TestProps.ACCUMULO_INSTANCE);
+  }
+
+  public String getZookeepers() {
+    return p.getProperty(TestProps.ZOOKEEPERS);
+  }
+
+  public ClientConfiguration getClientConfiguration() {
+    return ClientConfiguration.loadDefault().withInstance(getAccumuloInstanceName())
+        .withZkHosts(getZookeepers());
+  }
+
+  /**
+   * Gets an Accumulo instance object. The same instance is reused after the first call.
+   */
+  public Instance getAccumuloInstance() {
+    if (instance == null) {
+      this.instance = new ZooKeeperInstance(getClientConfiguration());
+    }
+    return instance;
+  }
+
+  /**
+   * Gets an Accumulo connector. The same connector is reused after the first call.
+   */
+  public Connector getAccumuloConnector() throws AccumuloException, AccumuloSecurityException {
+    if (connector == null) {
+      connector = getAccumuloInstance().getConnector(getAccumuloUserName(), getToken());
+    }
+    return connector;
+  }
+
+  public BatchWriterConfig getBatchWriterConfig() {
+    int numThreads = Integer.parseInt(p.getProperty(TestProps.BW_NUM_THREADS));
+    long maxLatency = Long.parseLong(p.getProperty(TestProps.BW_MAX_LATENCY_MS));
+    long maxMemory = Long.parseLong(p.getProperty(TestProps.BW_MAX_MEM_BYTES));
+
+    BatchWriterConfig config = new BatchWriterConfig();
+    config.setMaxWriteThreads(numThreads);
+    config.setMaxLatency(maxLatency, TimeUnit.MILLISECONDS);
+    config.setMaxMemory(maxMemory);
+    return config;
+  }
+
+  public int getScannerBatchSize() {
+    return Integer.parseInt(p.getProperty(TestProps.SCANNER_BATCH_SIZE));
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/TestProps.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/TestProps.java b/core/src/main/java/org/apache/accumulo/testing/core/TestProps.java
index f8ce9ca..e134c7f 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/TestProps.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/TestProps.java
@@ -17,11 +17,21 @@
 
 package org.apache.accumulo.testing.core;
 
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.Properties;
+
 public class TestProps {
 
   private static final String PREFIX = "test.";
-  private static final String RANDOMWALK = PREFIX + "randomwalk.";
   private static final String COMMON = PREFIX + "common.";
+  private static final String CI = PREFIX + "ci.";
+  private static final String CI_COMMON = CI + "common.";
+  private static final String CI_INGEST = CI + "ingest.";
+  private static final String CI_WALKER = CI + "walker.";
+  private static final String CI_BW = CI + "batch.walker.";
+  private static final String CI_SCANNER = CI + "scanner.";
+  private static final String CI_VERIFY = CI + "verify.";
 
   /** Common properties **/
   // Zookeeper connection string
@@ -38,16 +48,80 @@ public class TestProps {
   public static final String YARN_CONTAINER_MEMORY_MB = COMMON + "yarn.container.memory.mb";
   // Number of cores given to each YARN container
   public static final String YARN_CONTAINER_CORES = COMMON + "yarn.container.cores";
+  // Max memory (in bytes) each batch writer will use to buffer writes
+  public static final String BW_MAX_MEM_BYTES = COMMON + "bw.max.memory.bytes";
+  // Max the maximum time (in ms) each batch writer will buffer data
+  public static final String BW_MAX_LATENCY_MS = COMMON + "bw.max.latency.ms";
+  // Number of threads each batch writer will use to write data
+  public static final String BW_NUM_THREADS = COMMON + "bw.num.threads";
+  // Number of thread for each batch scanner
+  public static final String BS_NUM_THREADS = COMMON + "bw.num.threads";
+  // Number of key/value entries to pull during scan
+  public static final String SCANNER_BATCH_SIZE = COMMON + "scanner.batch.size";
+
+  /** Continuous ingest test properties **/
+  /** Common **/
+  // Accumulo table used by continuous ingest tests
+  public static final String CI_COMMON_ACCUMULO_TABLE = CI_COMMON + "accumulo.table";
+  // Number of tablets that should exist in Accumulo table when created
+  public static final String CI_COMMON_ACCUMULO_NUM_TABLETS = CI_COMMON + "accumulo.num.tablets";
+  // Optional authorizations (in CSV format) that if specified will be randomly selected by scanners
+  // and walkers
+  public static final String CI_COMMON_AUTHS = CI_COMMON + "auths";
+
+  /** Ingest **/
+  // Number of entries each ingest client should write
+  public static final String CI_INGEST_CLIENT_ENTRIES = CI_INGEST + "client.entries";
+  // Minimum random row to generate
+  public static final String CI_INGEST_ROW_MIN = CI_INGEST + "row.min";
+  // Maximum random row to generate
+  public static final String CI_INGEST_ROW_MAX = CI_INGEST + "row.max";
+  // Maximum number of random column families to generate
+  public static final String CI_INGEST_MAX_CF = CI_INGEST + "max.cf";
+  // Maximum number of random column qualifiers to generate
+  public static final String CI_INGEST_MAX_CQ = CI_INGEST + "max.cq";
+  // Optional visibilities (in CSV format) that if specified will be randomly selected by ingesters for
+  // each linked list
+  public static final String CI_INGEST_VISIBILITIES = CI_INGEST + "visibilities";
+  // Checksums will be generated during ingest if set to true
+  public static final String CI_INGEST_CHECKSUM = CI_INGEST + "checksum";
+
+  /** Batch Walker **/
+  // Sleep time between batch scans (in ms)
+  public static final String CI_BW_SLEEP_MS = CI_BW + "sleep.ms";
+  // Scan batch size
+  public static final String CI_BW_BATCH_SIZE = CI_BW + "batch.size";
+
+  /** Walker **/
+  // Sleep time between scans (in ms)
+  public static final String CI_WALKER_SLEEP_MS = CI_WALKER + "sleep.ms";
+
+  /** Scanner **/
+  // Sleep time between scans (in ms)
+  public static final String CI_SCANNER_SLEEP_MS = CI_SCANNER + "sleep.ms";
+  // Scanner entries
+  public static final String CI_SCANNER_ENTRIES = CI_SCANNER + "entries";
 
+  /** Verify **/
+  // Maximum number of mapreduce mappers
+  public static final String CI_VERIFY_MAX_MAPS = CI_VERIFY + "max.maps";
+  // Number of mapreduce reducers
+  public static final String CI_VERIFY_REDUCERS = CI_VERIFY + "reducers";
+  // Perform the verification directly on the files while the table is offline"
+  public static final String CI_VERIFY_SCAN_OFFLINE = CI_VERIFY + "scan.offline";
+  // Comma separated list of auths to use for verify
+  public static final String CI_VERIFY_AUTHS = CI_VERIFY + "auths";
+  // Location in HDFS to store output
+  public static final String CI_VERIFY_OUTPUT_DIR = CI_VERIFY + "output.dir";
 
-  /** Random walk properties **/
-  // Number of random walker (if running in YARN)
-  public static final String RW_NUM_WALKERS = RANDOMWALK + "num.walkers";
-  // Max memory for multi-table batch writer
-  public static final String RW_BW_MAX_MEM = RANDOMWALK + "bw.max.mem";
-  // Max latency in milliseconds for multi-table batch writer
-  public static final String RW_BW_MAX_LATENCY = RANDOMWALK + "bw.max.latency";
-  // Number of write thread for multi-table batch writer
-  public static final String RW_BW_NUM_THREADS = RANDOMWALK + "bw.num.threads";
+  public static Properties loadFromFile(String propsFilePath) throws IOException {
+    return loadFromStream(new FileInputStream(propsFilePath));
+  }
 
+  public static Properties loadFromStream(FileInputStream fis) throws IOException {
+    Properties props = new Properties();
+    props.load(fis);
+    fis.close();
+    return props;
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousBatchWalker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousBatchWalker.java b/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousBatchWalker.java
index e89f2eb..0282c2b 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousBatchWalker.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousBatchWalker.java
@@ -21,13 +21,11 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
+import java.util.Properties;
 import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.accumulo.core.cli.BatchScannerOpts;
-import org.apache.accumulo.core.cli.ClientOnDefaultTable;
-import org.apache.accumulo.core.cli.ScannerOpts;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
@@ -35,50 +33,44 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.testing.core.TestProps;
 import org.apache.hadoop.io.Text;
 
-import com.beust.jcommander.Parameter;
-import com.beust.jcommander.validators.PositiveInteger;
 import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 public class ContinuousBatchWalker {
 
-  static class Opts extends ContinuousWalk.Opts {
-    @Parameter(names = "--numToScan", description = "Number rows to scan between sleeps", required = true, validateWith = PositiveInteger.class)
-    long numToScan = 0;
-  }
-
   public static void main(String[] args) throws Exception {
 
-    Opts opts = new Opts();
-    ScannerOpts scanOpts = new ScannerOpts();
-    BatchScannerOpts bsOpts = new BatchScannerOpts();
-    ClientOnDefaultTable clientOpts = new ClientOnDefaultTable("ci");
-    clientOpts.parseArgs(ContinuousBatchWalker.class.getName(), args, scanOpts, bsOpts, opts);
+    Properties props = TestProps.loadFromFile(args[0]);
+
+    ContinuousEnv env = new ContinuousEnv(props);
+
+    Authorizations auths = env.getRandomAuthorizations();
+    Connector conn = env.getAccumuloConnector();
+    Scanner scanner = ContinuousUtil.createScanner(conn, env.getAccumuloTableName(), auths);
+    int scanBatchSize = Integer.parseInt(props.getProperty(TestProps.CI_BW_BATCH_SIZE));
+    scanner.setBatchSize(scanBatchSize);
 
     Random r = new Random();
-    Authorizations auths = opts.randomAuths.getAuths(r);
 
-    Connector conn = clientOpts.getConnector();
-    Scanner scanner = ContinuousUtil.createScanner(conn, clientOpts.getTableName(), auths);
-    scanner.setBatchSize(scanOpts.scanBatchSize);
+    int scanThreads = Integer.parseInt(props.getProperty(TestProps.BS_NUM_THREADS));
 
     while (true) {
-      BatchScanner bs = conn.createBatchScanner(clientOpts.getTableName(), auths, bsOpts.scanThreads);
-      bs.setTimeout(bsOpts.scanTimeout, TimeUnit.MILLISECONDS);
+      BatchScanner bs = conn.createBatchScanner(env.getAccumuloTableName(), auths, scanThreads);
 
-      Set<Text> batch = getBatch(scanner, opts.min, opts.max, scanOpts.scanBatchSize, r);
+      Set<Text> batch = getBatch(scanner, env.getRowMin(), env.getRowMax(), scanBatchSize, r);
       List<Range> ranges = new ArrayList<>(batch.size());
 
       for (Text row : batch) {
         ranges.add(new Range(row));
       }
 
-      runBatchScan(scanOpts.scanBatchSize, bs, batch, ranges);
+      runBatchScan(scanBatchSize, bs, batch, ranges);
 
-      sleepUninterruptibly(opts.sleepTime, TimeUnit.MILLISECONDS);
+      int bwSleepMs = Integer.parseInt(props.getProperty(TestProps.CI_BW_SLEEP_MS));
+      sleepUninterruptibly(bwSleepMs, TimeUnit.MILLISECONDS);
     }
-
   }
 
   private static void runBatchScan(int batchSize, BatchScanner bs, Set<Text> batch, List<Range> ranges) {
@@ -117,7 +109,6 @@ public class ContinuousBatchWalker {
     } else {
       System.out.printf("BRQ %d %d %d %d %d%n", t1, (t2 - t1), rowsSeen.size(), count, (int) (rowsSeen.size() / ((t2 - t1) / 1000.0)));
     }
-
   }
 
   private static void addRow(int batchSize, Value v) {
@@ -171,5 +162,4 @@ public class ContinuousBatchWalker {
 
     return ret;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousEnv.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousEnv.java b/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousEnv.java
new file mode 100644
index 0000000..7907ffd
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousEnv.java
@@ -0,0 +1,66 @@
+package org.apache.accumulo.testing.core.continuous;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.testing.core.TestEnv;
+import org.apache.accumulo.testing.core.TestProps;
+
+class ContinuousEnv extends TestEnv {
+
+  private List<Authorizations> authList;
+
+  ContinuousEnv(Properties props) {
+    super(props);
+  }
+
+  /**
+   * @return Accumulo authorizations list
+   */
+  private List<Authorizations> getAuthList() {
+    if (authList == null) {
+      String authValue = p.getProperty(TestProps.CI_COMMON_AUTHS);
+      if (authValue == null || authValue.trim().isEmpty()) {
+        authList = Collections.singletonList(Authorizations.EMPTY);
+      } else {
+        authList = new ArrayList<>();
+        for (String a : authValue.split("|")) {
+          authList.add(new Authorizations(a.split(",")));
+        }
+      }
+    }
+    return authList;
+  }
+
+  /**
+   * @return random authorization
+   */
+  Authorizations getRandomAuthorizations() {
+    Random r = new Random();
+    return getAuthList().get(r.nextInt(getAuthList().size()));
+  }
+
+  long getRowMin() {
+    return Long.parseLong(p.getProperty(TestProps.CI_INGEST_ROW_MIN));
+  }
+
+  long getRowMax() {
+    return Long.parseLong(p.getProperty(TestProps.CI_INGEST_ROW_MAX));
+  }
+
+  int getMaxColF() {
+    return Integer.parseInt(p.getProperty(TestProps.CI_INGEST_MAX_CF));
+  }
+
+  int getMaxColQ() {
+    return Integer.parseInt(p.getProperty(TestProps.CI_INGEST_MAX_CQ));
+  }
+
+  String getAccumuloTableName() {
+    return p.getProperty(TestProps.CI_COMMON_ACCUMULO_TABLE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousIngest.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousIngest.java b/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousIngest.java
index 4681cb8..f260e78 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousIngest.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousIngest.java
@@ -18,18 +18,15 @@ package org.apache.accumulo.testing.core.continuous;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
-import java.io.BufferedReader;
-import java.io.InputStreamReader;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Properties;
 import java.util.Random;
 import java.util.UUID;
 import java.util.zip.CRC32;
 import java.util.zip.Checksum;
 
-import org.apache.accumulo.core.cli.BatchWriterOpts;
-import org.apache.accumulo.core.cli.ClientOnDefaultTable;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.MutationsRejectedException;
@@ -40,9 +37,7 @@ import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.trace.CountSampler;
 import org.apache.accumulo.core.trace.Trace;
 import org.apache.accumulo.core.util.FastFormat;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
+import org.apache.accumulo.testing.core.TestProps;
 import org.apache.hadoop.io.Text;
 
 public class ContinuousIngest {
@@ -51,49 +46,44 @@ public class ContinuousIngest {
 
   private static List<ColumnVisibility> visibilities;
 
-  private static void initVisibilities(ContinuousOpts opts) throws Exception {
-    if (opts.visFile == null) {
-      visibilities = Collections.singletonList(new ColumnVisibility());
-      return;
-    }
-
-    visibilities = new ArrayList<>();
-
-    FileSystem fs = FileSystem.get(new Configuration());
-    BufferedReader in = new BufferedReader(new InputStreamReader(fs.open(new Path(opts.visFile)), UTF_8));
-
-    String line;
-
-    while ((line = in.readLine()) != null) {
-      visibilities.add(new ColumnVisibility(line));
-    }
-
-    in.close();
-  }
-
   private static ColumnVisibility getVisibility(Random rand) {
     return visibilities.get(rand.nextInt(visibilities.size()));
   }
 
   public static void main(String[] args) throws Exception {
 
-    ContinuousOpts opts = new ContinuousOpts();
-    BatchWriterOpts bwOpts = new BatchWriterOpts();
-    ClientOnDefaultTable clientOpts = new ClientOnDefaultTable("ci");
-    clientOpts.parseArgs(ContinuousIngest.class.getName(), args, bwOpts, opts);
+    if (args.length != 1) {
+      System.err.println("Usage: ContinuousIngest <propsPath>");
+      System.exit(-1);
+    }
+
+    Properties props = TestProps.loadFromFile(args[0]);
+
+    String vis = props.getProperty(TestProps.CI_INGEST_VISIBILITIES);
+    if (vis == null) {
+      visibilities = Collections.singletonList(new ColumnVisibility());
+    } else {
+      visibilities = new ArrayList<>();
+      for (String v : vis.split(",")) {
+        visibilities.add(new ColumnVisibility(v.trim()));
+      }
+    }
 
-    initVisibilities(opts);
+    ContinuousEnv env = new ContinuousEnv(props);
 
-    if (opts.min < 0 || opts.max < 0 || opts.max <= opts.min) {
+    long rowMin = env.getRowMin();
+    long rowMax = env.getRowMax();
+    if (rowMin < 0 || rowMax < 0 || rowMax <= rowMin) {
       throw new IllegalArgumentException("bad min and max");
     }
-    Connector conn = clientOpts.getConnector();
 
-    if (!conn.tableOperations().exists(clientOpts.getTableName())) {
-      throw new TableNotFoundException(null, clientOpts.getTableName(), "Consult the README and create the table before starting ingest.");
+    Connector conn = env.getAccumuloConnector();
+    String tableName = env.getAccumuloTableName();
+    if (!conn.tableOperations().exists(tableName)) {
+      throw new TableNotFoundException(null, tableName, "Consult the README and create the table before starting ingest.");
     }
 
-    BatchWriter bw = conn.createBatchWriter(clientOpts.getTableName(), bwOpts.getBatchWriterConfig());
+    BatchWriter bw = conn.createBatchWriter(tableName, env.getBatchWriterConfig());
     bw = Trace.wrapAll(bw, new CountSampler(1024));
 
     Random r = new Random();
@@ -117,61 +107,65 @@ public class ContinuousIngest {
 
     long lastFlushTime = System.currentTimeMillis();
 
+    int maxColF = env.getMaxColF();
+    int maxColQ = env.getMaxColQ();
+    boolean checksum = Boolean.parseBoolean(props.getProperty(TestProps.CI_INGEST_CHECKSUM));
+    long numEntries = Long.parseLong(props.getProperty(TestProps.CI_INGEST_CLIENT_ENTRIES));
+
     out: while (true) {
       // generate first set of nodes
       ColumnVisibility cv = getVisibility(r);
 
       for (int index = 0; index < flushInterval; index++) {
-        long rowLong = genLong(opts.min, opts.max, r);
+        long rowLong = genLong(rowMin, rowMax, r);
         prevRows[index] = rowLong;
         firstRows[index] = rowLong;
 
-        int cf = r.nextInt(opts.maxColF);
-        int cq = r.nextInt(opts.maxColQ);
+        int cf = r.nextInt(maxColF);
+        int cq = r.nextInt(maxColQ);
 
         firstColFams[index] = cf;
         firstColQuals[index] = cq;
 
-        Mutation m = genMutation(rowLong, cf, cq, cv, ingestInstanceId, count, null, r, opts.checksum);
+        Mutation m = genMutation(rowLong, cf, cq, cv, ingestInstanceId, count, null,
+                                 checksum);
         count++;
         bw.addMutation(m);
       }
 
       lastFlushTime = flush(bw, count, flushInterval, lastFlushTime);
-      if (count >= opts.num)
+      if (count >= numEntries)
         break out;
 
       // generate subsequent sets of nodes that link to previous set of nodes
       for (int depth = 1; depth < maxDepth; depth++) {
         for (int index = 0; index < flushInterval; index++) {
-          long rowLong = genLong(opts.min, opts.max, r);
+          long rowLong = genLong(rowMin, rowMax, r);
           byte[] prevRow = genRow(prevRows[index]);
           prevRows[index] = rowLong;
-          Mutation m = genMutation(rowLong, r.nextInt(opts.maxColF), r.nextInt(opts.maxColQ), cv, ingestInstanceId, count, prevRow, r, opts.checksum);
+          Mutation m = genMutation(rowLong, r.nextInt(maxColF), r.nextInt(maxColQ), cv, ingestInstanceId, count, prevRow, checksum);
           count++;
           bw.addMutation(m);
         }
 
         lastFlushTime = flush(bw, count, flushInterval, lastFlushTime);
-        if (count >= opts.num)
+        if (count >= numEntries)
           break out;
       }
 
       // create one big linked list, this makes all of the first inserts
       // point to something
       for (int index = 0; index < flushInterval - 1; index++) {
-        Mutation m = genMutation(firstRows[index], firstColFams[index], firstColQuals[index], cv, ingestInstanceId, count, genRow(prevRows[index + 1]), r,
-            opts.checksum);
+        Mutation m = genMutation(firstRows[index], firstColFams[index], firstColQuals[index], cv, ingestInstanceId, count, genRow(prevRows[index + 1]), checksum);
         count++;
         bw.addMutation(m);
       }
       lastFlushTime = flush(bw, count, flushInterval, lastFlushTime);
-      if (count >= opts.num)
+      if (count >= numEntries)
         break out;
     }
 
     bw.close();
-    clientOpts.stopTracing();
   }
 
   private static long flush(BatchWriter bw, long count, final int flushInterval, long lastFlushTime) throws MutationsRejectedException {
@@ -183,8 +177,9 @@ public class ContinuousIngest {
     return lastFlushTime;
   }
 
-  public static Mutation genMutation(long rowLong, int cfInt, int cqInt, ColumnVisibility cv, byte[] ingestInstanceId, long count, byte[] prevRow, Random r,
-      boolean checksum) {
+  static Mutation genMutation(long rowLong, int cfInt, int cqInt, ColumnVisibility cv,
+                              byte[] ingestInstanceId, long count, byte[] prevRow,
+                              boolean checksum) {
     // Adler32 is supposed to be faster, but according to wikipedia is not good for small data.... so used CRC32 instead
     CRC32 cksum = null;
 
@@ -207,15 +202,15 @@ public class ContinuousIngest {
     return m;
   }
 
-  public static final long genLong(long min, long max, Random r) {
-    return ((r.nextLong() & 0x7fffffffffffffffl) % (max - min)) + min;
+  static long genLong(long min, long max, Random r) {
+    return ((r.nextLong() & 0x7fffffffffffffffL) % (max - min)) + min;
   }
 
-  static final byte[] genRow(long min, long max, Random r) {
+  static byte[] genRow(long min, long max, Random r) {
     return genRow(genLong(min, max, r));
   }
 
-  static final byte[] genRow(long rowLong) {
+  static byte[] genRow(long rowLong) {
     return FastFormat.toZeroPaddedString(rowLong, 16, 16, EMPTY_BYTES);
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousMoru.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousMoru.java b/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousMoru.java
index c2902ee..560e2ff 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousMoru.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousMoru.java
@@ -19,12 +19,11 @@ package org.apache.accumulo.testing.core.continuous;
 import static java.nio.charset.StandardCharsets.UTF_8;
 
 import java.io.IOException;
+import java.util.Properties;
 import java.util.Random;
 import java.util.Set;
 import java.util.UUID;
 
-import org.apache.accumulo.core.cli.BatchWriterOpts;
-import org.apache.accumulo.core.cli.MapReduceClientOnDefaultTable;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
 import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
@@ -34,6 +33,7 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.testing.core.TestProps;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.io.Text;
@@ -43,13 +43,11 @@ import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
-import com.beust.jcommander.Parameter;
-import com.beust.jcommander.validators.PositiveInteger;
-
 /**
- * A map only job that reads a table created by continuous ingest and creates doubly linked list. This map reduce job tests the ability of a map only job to
- * read and write to accumulo at the same time. This map reduce job mutates the table in such a way that it should not create any undefined nodes.
- *
+ * A map only job that reads a table created by continuous ingest and creates doubly linked list.
+ * This map reduce job tests the ability of a map only job to read and write to accumulo at the
+ * same time. This map reduce job mutates the table in such a way that it should not create any
+ * undefined nodes.
  */
 public class ContinuousMoru extends Configured implements Tool {
   private static final String PREFIX = ContinuousMoru.class.getSimpleName() + ".";
@@ -59,8 +57,8 @@ public class ContinuousMoru extends Configured implements Tool {
   private static final String MIN = PREFIX + "MIN";
   private static final String CI_ID = PREFIX + "CI_ID";
 
-  static enum Counts {
-    SELF_READ;
+  enum Counts {
+    SELF_READ
   }
 
   public static class CMapper extends Mapper<Key,Value,Text,Mutation> {
@@ -105,43 +103,36 @@ public class ContinuousMoru extends Configured implements Tool {
         if (offset > 0) {
           long rowLong = Long.parseLong(new String(val, offset, 16, UTF_8), 16);
           Mutation m = ContinuousIngest.genMutation(rowLong, random.nextInt(max_cf), random.nextInt(max_cq), EMPTY_VIS, iiId, count++, key.getRowData()
-              .toArray(), random, true);
+              .toArray(), true);
           context.write(null, m);
         }
 
       } else {
-        context.getCounter(Counts.SELF_READ).increment(1l);
+        context.getCounter(Counts.SELF_READ).increment(1L);
       }
     }
   }
 
-  static class Opts extends ContinuousOpts {
-    @Parameter(names = "--maxColF", description = "maximum column family value to use", converter = ShortConverter.class)
-    short maxColF = Short.MAX_VALUE;
-
-    @Parameter(names = "--maxColQ", description = "maximum column qualifier value to use", converter = ShortConverter.class)
-    short maxColQ = Short.MAX_VALUE;
-
-    @Parameter(names = "--maxMappers", description = "the maximum number of mappers to use", required = true, validateWith = PositiveInteger.class)
-    int maxMaps = 0;
-  }
-
   @Override
   public int run(String[] args) throws IOException, InterruptedException, ClassNotFoundException, AccumuloSecurityException {
-    Opts opts = new Opts();
-    BatchWriterOpts bwOpts = new BatchWriterOpts();
-    MapReduceClientOnDefaultTable clientOpts = new MapReduceClientOnDefaultTable("ci");
-    clientOpts.parseArgs(ContinuousMoru.class.getName(), args, bwOpts, opts);
+
+    Properties props = TestProps.loadFromFile(args[0]);
+    ContinuousEnv env = new ContinuousEnv(props);
 
     Job job = Job.getInstance(getConf(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
     job.setJarByClass(this.getClass());
 
     job.setInputFormatClass(AccumuloInputFormat.class);
-    clientOpts.setAccumuloConfigs(job);
+
+    AccumuloInputFormat.setConnectorInfo(job, env.getAccumuloUserName(), env.getToken());
+    AccumuloInputFormat.setInputTableName(job, env.getAccumuloTableName());
+    AccumuloInputFormat.setZooKeeperInstance(job, env.getClientConfiguration());
+
+    int maxMaps = Integer.parseInt(props.getProperty(TestProps.CI_VERIFY_MAX_MAPS));
 
     // set up ranges
     try {
-      Set<Range> ranges = clientOpts.getConnector().tableOperations().splitRangeByTablets(clientOpts.getTableName(), new Range(), opts.maxMaps);
+      Set<Range> ranges = env.getAccumuloConnector().tableOperations().splitRangeByTablets(env.getAccumuloTableName(), new Range(), maxMaps);
       AccumuloInputFormat.setRanges(job, ranges);
       AccumuloInputFormat.setAutoAdjustRanges(job, false);
     } catch (Exception e) {
@@ -149,31 +140,28 @@ public class ContinuousMoru extends Configured implements Tool {
     }
 
     job.setMapperClass(CMapper.class);
-
     job.setNumReduceTasks(0);
-
     job.setOutputFormatClass(AccumuloOutputFormat.class);
-    AccumuloOutputFormat.setBatchWriterOptions(job, bwOpts.getBatchWriterConfig());
+    AccumuloOutputFormat.setBatchWriterOptions(job, env.getBatchWriterConfig());
+    AccumuloOutputFormat.setConnectorInfo(job, env.getAccumuloUserName(), env.getToken());
+    AccumuloOutputFormat.setCreateTables(job, true);
+    AccumuloOutputFormat.setDefaultTableName(job, env.getAccumuloTableName());
+    AccumuloOutputFormat.setZooKeeperInstance(job, env.getClientConfiguration());
 
     Configuration conf = job.getConfiguration();
-    conf.setLong(MIN, opts.min);
-    conf.setLong(MAX, opts.max);
-    conf.setInt(MAX_CF, opts.maxColF);
-    conf.setInt(MAX_CQ, opts.maxColQ);
+    conf.setLong(MIN, env.getRowMin());
+    conf.setLong(MAX, env.getRowMax());
+    conf.setInt(MAX_CF, env.getMaxColF());
+    conf.setInt(MAX_CQ, env.getMaxColQ());
     conf.set(CI_ID, UUID.randomUUID().toString());
 
     job.waitForCompletion(true);
-    clientOpts.stopTracing();
     return job.isSuccessful() ? 0 : 1;
   }
 
-  /**
-   *
-   * @param args
-   *          instanceName zookeepers username password table columns outputpath
-   */
   public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(CachedConfiguration.getInstance(), new ContinuousMoru(), args);
+    ContinuousEnv env = new ContinuousEnv(TestProps.loadFromFile(args[0]));
+    int res = ToolRunner.run(env.getHadoopConfiguration(), new ContinuousMoru(), args);
     if (res != 0)
       System.exit(res);
   }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousQuery.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousQuery.java b/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousQuery.java
deleted file mode 100644
index 8180383..0000000
--- a/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousQuery.java
+++ /dev/null
@@ -1,74 +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.testing.core.continuous;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-
-import java.util.Map.Entry;
-import java.util.Random;
-
-import org.apache.accumulo.core.cli.ClientOnDefaultTable;
-import org.apache.accumulo.core.cli.ClientOpts.TimeConverter;
-import org.apache.accumulo.core.cli.ScannerOpts;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.hadoop.io.Text;
-
-import com.beust.jcommander.Parameter;
-
-public class ContinuousQuery {
-
-  public static class Opts extends ContinuousOpts {
-    @Parameter(names = "--sleep", description = "the time to wait between queries", converter = TimeConverter.class)
-    long sleepTime = 100;
-  }
-
-  public static void main(String[] args) throws Exception {
-    Opts opts = new Opts();
-    ScannerOpts scanOpts = new ScannerOpts();
-    ClientOnDefaultTable clientOpts = new ClientOnDefaultTable("ci");
-    clientOpts.parseArgs(ContinuousQuery.class.getName(), args, scanOpts, opts);
-
-    Connector conn = clientOpts.getConnector();
-    Scanner scanner = ContinuousUtil.createScanner(conn, clientOpts.getTableName(), clientOpts.auths);
-    scanner.setBatchSize(scanOpts.scanBatchSize);
-
-    Random r = new Random();
-
-    while (true) {
-      byte[] row = ContinuousIngest.genRow(opts.min, opts.max, r);
-
-      int count = 0;
-
-      long t1 = System.currentTimeMillis();
-      scanner.setRange(new Range(new Text(row)));
-      for (Entry<Key,Value> entry : scanner) {
-        ContinuousWalk.validate(entry.getKey(), entry.getValue());
-        count++;
-      }
-      long t2 = System.currentTimeMillis();
-
-      System.out.printf("SRQ %d %s %d %d%n", t1, new String(row, UTF_8), (t2 - t1), count);
-
-      if (opts.sleepTime > 0)
-        Thread.sleep(opts.sleepTime);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousScanner.java b/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousScanner.java
index 42e0ea8..162e64d 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousScanner.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousScanner.java
@@ -20,49 +20,44 @@ import static java.nio.charset.StandardCharsets.UTF_8;
 
 import java.util.Iterator;
 import java.util.Map.Entry;
+import java.util.Properties;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.accumulo.core.cli.ClientOnDefaultTable;
-import org.apache.accumulo.core.cli.ScannerOpts;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.testing.core.TestProps;
 import org.apache.hadoop.io.Text;
 
-import com.beust.jcommander.Parameter;
-import com.beust.jcommander.validators.PositiveInteger;
 import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 public class ContinuousScanner {
 
-  static class Opts extends ContinuousWalk.Opts {
-    @Parameter(names = "--numToScan", description = "Number rows to scan between sleeps", required = true, validateWith = PositiveInteger.class)
-    long numToScan = 0;
-  }
-
   public static void main(String[] args) throws Exception {
-    Opts opts = new Opts();
-    ScannerOpts scanOpts = new ScannerOpts();
-    ClientOnDefaultTable clientOpts = new ClientOnDefaultTable("ci");
-    clientOpts.parseArgs(ContinuousScanner.class.getName(), args, scanOpts, opts);
+
+    Properties props = TestProps.loadFromFile(args[0]);
+    ContinuousEnv env = new ContinuousEnv(props);
 
     Random r = new Random();
 
     long distance = 1000000000000l;
 
-    Connector conn = clientOpts.getConnector();
-    Authorizations auths = opts.randomAuths.getAuths(r);
-    Scanner scanner = ContinuousUtil.createScanner(conn, clientOpts.getTableName(), auths);
-    scanner.setBatchSize(scanOpts.scanBatchSize);
+    Connector conn = env.getAccumuloConnector();
+    Authorizations auths = env.getRandomAuthorizations();
+    Scanner scanner = ContinuousUtil.createScanner(conn, env.getAccumuloTableName(), auths);
+    scanner.setBatchSize(env.getScannerBatchSize());
 
-    double delta = Math.min(.05, .05 / (opts.numToScan / 1000.0));
+    int numToScan = Integer.parseInt(props.getProperty(TestProps.CI_SCANNER_ENTRIES));
+    int scannerSleepMs = Integer.parseInt(props.getProperty(TestProps.CI_SCANNER_SLEEP_MS));
+
+    double delta = Math.min(.05, .05 / (numToScan / 1000.0));
 
     while (true) {
-      long startRow = ContinuousIngest.genLong(opts.min, opts.max - distance, r);
+      long startRow = ContinuousIngest.genLong(env.getRowMin(), env.getRowMax() - distance, r);
       byte[] scanStart = ContinuousIngest.genRow(startRow);
       byte[] scanStop = ContinuousIngest.genRow(startRow + distance);
 
@@ -83,13 +78,13 @@ public class ContinuousScanner {
 
       // System.out.println("P1 " +count +" "+((1-delta) * numToScan)+" "+((1+delta) * numToScan)+" "+numToScan);
 
-      if (count < (1 - delta) * opts.numToScan || count > (1 + delta) * opts.numToScan) {
+      if (count < (1 - delta) * numToScan || count > (1 + delta) * numToScan) {
         if (count == 0) {
           distance = distance * 10;
           if (distance < 0)
             distance = 1000000000000l;
         } else {
-          double ratio = (double) opts.numToScan / count;
+          double ratio = (double) numToScan / count;
           // move ratio closer to 1 to make change slower
           ratio = ratio - (ratio - 1.0) * (2.0 / 3.0);
           distance = (long) (ratio * distance);
@@ -100,8 +95,9 @@ public class ContinuousScanner {
 
       System.out.printf("SCN %d %s %d %d%n", t1, new String(scanStart, UTF_8), (t2 - t1), count);
 
-      if (opts.sleepTime > 0)
-        sleepUninterruptibly(opts.sleepTime, TimeUnit.MILLISECONDS);
+      if (scannerSleepMs > 0) {
+        sleepUninterruptibly(scannerSleepMs, TimeUnit.MILLISECONDS);
+      }
     }
 
   }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousStatsCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousStatsCollector.java b/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousStatsCollector.java
deleted file mode 100644
index 818e387..0000000
--- a/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousStatsCollector.java
+++ /dev/null
@@ -1,206 +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.testing.core.continuous;
-
-import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
-import java.util.Timer;
-import java.util.TimerTask;
-
-import org.apache.accumulo.core.cli.ScannerOpts;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.impl.ClientContext;
-import org.apache.accumulo.core.client.impl.Credentials;
-import org.apache.accumulo.core.client.impl.MasterClient;
-import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.core.client.impl.thrift.ThriftNotActiveServiceException;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.data.impl.KeyExtent;
-import org.apache.accumulo.core.iterators.ColumnFamilyCounter;
-import org.apache.accumulo.core.master.thrift.MasterClientService;
-import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
-import org.apache.accumulo.core.master.thrift.TableInfo;
-import org.apache.accumulo.core.master.thrift.TabletServerStatus;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
-import org.apache.accumulo.core.trace.Tracer;
-import org.apache.accumulo.core.util.CachedConfiguration;
-import org.apache.accumulo.core.util.Stat;
-import org.apache.accumulo.server.ServerConstants;
-import org.apache.accumulo.server.cli.ClientOnRequiredTable;
-import org.apache.accumulo.server.conf.ServerConfigurationFactory;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.fs.VolumeManagerImpl;
-import org.apache.accumulo.server.util.TableInfoUtil;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.ContentSummary;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.ClusterStatus;
-import org.apache.hadoop.mapred.JobClient;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class ContinuousStatsCollector {
-
-  private static final Logger log = LoggerFactory.getLogger(ContinuousStatsCollector.class);
-
-  static class StatsCollectionTask extends TimerTask {
-
-    private final String tableId;
-    private final Opts opts;
-    private final int scanBatchSize;
-
-    public StatsCollectionTask(Opts opts, int scanBatchSize) {
-      this.opts = opts;
-      this.scanBatchSize = scanBatchSize;
-      this.tableId = Tables.getNameToIdMap(opts.getInstance()).get(opts.getTableName());
-      System.out
-          .println("TIME TABLET_SERVERS TOTAL_ENTRIES TOTAL_INGEST TOTAL_QUERY TABLE_RECS TABLE_RECS_IN_MEM TABLE_INGEST TABLE_QUERY TABLE_TABLETS TABLE_TABLETS_ONLINE"
-              + " ACCUMULO_DU ACCUMULO_DIRS ACCUMULO_FILES TABLE_DU TABLE_DIRS TABLE_FILES"
-              + " MAP_TASK MAX_MAP_TASK REDUCE_TASK MAX_REDUCE_TASK TASK_TRACKERS BLACK_LISTED MIN_FILES/TABLET MAX_FILES/TABLET AVG_FILES/TABLET STDDEV_FILES/TABLET");
-    }
-
-    @Override
-    public void run() {
-      try {
-        String acuStats = getACUStats();
-        String fsStats = getFSStats();
-        String mrStats = getMRStats();
-        String tabletStats = getTabletStats();
-
-        System.out.println(System.currentTimeMillis() + " " + acuStats + " " + fsStats + " " + mrStats + " " + tabletStats);
-      } catch (Exception e) {
-        log.error(System.currentTimeMillis() + " - Failed to collect stats", e);
-      }
-    }
-
-    private String getTabletStats() throws Exception {
-
-      Connector conn = opts.getConnector();
-      Scanner scanner = conn.createScanner(MetadataTable.NAME, opts.auths);
-      scanner.setBatchSize(scanBatchSize);
-      scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
-      scanner.addScanIterator(new IteratorSetting(1000, "cfc", ColumnFamilyCounter.class.getName()));
-      scanner.setRange(new KeyExtent(tableId, null, null).toMetadataRange());
-
-      Stat s = new Stat();
-
-      int count = 0;
-      for (Entry<Key,Value> entry : scanner) {
-        count++;
-        s.addStat(Long.parseLong(entry.getValue().toString()));
-      }
-
-      if (count > 0)
-        return String.format("%d %d %.3f %.3f", s.getMin(), s.getMax(), s.getAverage(), s.getStdDev());
-      else
-        return "0 0 0 0";
-
-    }
-
-    private String getFSStats() throws Exception {
-      VolumeManager fs = VolumeManagerImpl.get();
-      long length1 = 0, dcount1 = 0, fcount1 = 0;
-      long length2 = 0, dcount2 = 0, fcount2 = 0;
-      for (String dir : ServerConstants.getTablesDirs()) {
-        ContentSummary contentSummary = fs.getContentSummary(new Path(dir));
-        length1 += contentSummary.getLength();
-        dcount1 += contentSummary.getDirectoryCount();
-        fcount1 += contentSummary.getFileCount();
-        contentSummary = fs.getContentSummary(new Path(dir, tableId));
-        length2 += contentSummary.getLength();
-        dcount2 += contentSummary.getDirectoryCount();
-        fcount2 += contentSummary.getFileCount();
-      }
-
-      return "" + length1 + " " + dcount1 + " " + fcount1 + " " + length2 + " " + dcount2 + " " + fcount2;
-    }
-
-    private String getACUStats() throws Exception {
-
-      MasterClientService.Iface client = null;
-      while (true) {
-        try {
-          ClientContext context = new ClientContext(opts.getInstance(), new Credentials(opts.getPrincipal(), opts.getToken()), new ServerConfigurationFactory(
-              opts.getInstance()).getConfiguration());
-          client = MasterClient.getConnectionWithRetry(context);
-          MasterMonitorInfo stats = client.getMasterStats(Tracer.traceInfo(), context.rpcCreds());
-
-          TableInfo all = new TableInfo();
-          Map<String,TableInfo> tableSummaries = new HashMap<>();
-
-          for (TabletServerStatus server : stats.tServerInfo) {
-            for (Entry<String,TableInfo> info : server.tableMap.entrySet()) {
-              TableInfo tableSummary = tableSummaries.get(info.getKey());
-              if (tableSummary == null) {
-                tableSummary = new TableInfo();
-                tableSummaries.put(info.getKey(), tableSummary);
-              }
-              TableInfoUtil.add(tableSummary, info.getValue());
-              TableInfoUtil.add(all, info.getValue());
-            }
-          }
-
-          TableInfo ti = tableSummaries.get(tableId);
-
-          return "" + stats.tServerInfo.size() + " " + all.recs + " " + (long) all.ingestRate + " " + (long) all.queryRate + " " + ti.recs + " "
-              + ti.recsInMemory + " " + (long) ti.ingestRate + " " + (long) ti.queryRate + " " + ti.tablets + " " + ti.onlineTablets;
-
-        } catch (ThriftNotActiveServiceException e) {
-          // Let it loop, fetching a new location
-          log.debug("Contacted a Master which is no longer active, retrying");
-          sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
-        } finally {
-          if (client != null)
-            MasterClient.close(client);
-        }
-      }
-    }
-
-  }
-
-  private static String getMRStats() throws Exception {
-    Configuration conf = CachedConfiguration.getInstance();
-    // No alternatives for hadoop 20
-    JobClient jc = new JobClient(new org.apache.hadoop.mapred.JobConf(conf));
-
-    ClusterStatus cs = jc.getClusterStatus(false);
-
-    return "" + cs.getMapTasks() + " " + cs.getMaxMapTasks() + " " + cs.getReduceTasks() + " " + cs.getMaxReduceTasks() + " " + cs.getTaskTrackers() + " "
-        + cs.getBlacklistedTrackers();
-
-  }
-
-  static class Opts extends ClientOnRequiredTable {}
-
-  public static void main(String[] args) {
-    Opts opts = new Opts();
-    ScannerOpts scanOpts = new ScannerOpts();
-    opts.parseArgs(ContinuousStatsCollector.class.getName(), args, scanOpts);
-    Timer jtimer = new Timer();
-
-    jtimer.schedule(new StatsCollectionTask(opts, scanOpts.scanBatchSize), 0, 30000);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousVerify.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousVerify.java b/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousVerify.java
index 64f8a35..430bf3b 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousVerify.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousVerify.java
@@ -22,16 +22,16 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Properties;
 import java.util.Random;
 import java.util.Set;
 
-import org.apache.accumulo.core.cli.MapReduceClientOnDefaultTable;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.testing.core.TestProps;
 import org.apache.accumulo.testing.core.continuous.ContinuousWalk.BadChecksumException;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
@@ -47,13 +47,9 @@ import org.apache.hadoop.util.ToolRunner;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.beust.jcommander.Parameter;
-import com.beust.jcommander.validators.PositiveInteger;
-
 /**
  * A map reduce job that verifies a table created by continuous ingest. It verifies that all referenced nodes are defined.
  */
-
 public class ContinuousVerify extends Configured implements Tool {
 
   public static final VLongWritable DEF = new VLongWritable(-1);
@@ -76,7 +72,7 @@ public class ContinuousVerify extends Configured implements Tool {
       try {
         ContinuousWalk.validate(key, data);
       } catch (BadChecksumException bce) {
-        context.getCounter(Counts.CORRUPT).increment(1l);
+        context.getCounter(Counts.CORRUPT).increment(1L);
         if (corrupt < 1000) {
           log.error("Bad checksum : " + key);
         } else if (corrupt == 1000) {
@@ -100,7 +96,7 @@ public class ContinuousVerify extends Configured implements Tool {
     }
   }
 
-  public static enum Counts {
+  public enum Counts {
     UNREFERENCED, UNDEFINED, REFERENCED, CORRUPT
   }
 
@@ -131,95 +127,82 @@ public class ContinuousVerify extends Configured implements Tool {
         }
 
         context.write(new Text(ContinuousIngest.genRow(key.get())), new Text(sb.toString()));
-        context.getCounter(Counts.UNDEFINED).increment(1l);
+        context.getCounter(Counts.UNDEFINED).increment(1L);
 
       } else if (defCount > 0 && refs.size() == 0) {
-        context.getCounter(Counts.UNREFERENCED).increment(1l);
+        context.getCounter(Counts.UNREFERENCED).increment(1L);
       } else {
-        context.getCounter(Counts.REFERENCED).increment(1l);
+        context.getCounter(Counts.REFERENCED).increment(1L);
       }
 
     }
   }
 
-  static class Opts extends MapReduceClientOnDefaultTable {
-    @Parameter(names = "--output", description = "location in HDFS to store the results; must not exist")
-    String outputDir = "/tmp/continuousVerify";
-
-    @Parameter(names = "--maxMappers", description = "the maximum number of mappers to use", validateWith = PositiveInteger.class)
-    int maxMaps = 1;
-
-    @Parameter(names = "--reducers", description = "the number of reducers to use", validateWith = PositiveInteger.class)
-    int reducers = 1;
-
-    @Parameter(names = "--offline", description = "perform the verification directly on the files while the table is offline")
-    boolean scanOffline = false;
-
-    public Opts() {
-      super("ci");
-    }
-  }
-
   @Override
   public int run(String[] args) throws Exception {
-    Opts opts = new Opts();
-    opts.parseArgs(this.getClass().getName(), args);
+
+    Properties props = TestProps.loadFromFile(args[0]);
+    ContinuousEnv env = new ContinuousEnv(props);
 
     Job job = Job.getInstance(getConf(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
     job.setJarByClass(this.getClass());
 
     job.setInputFormatClass(AccumuloInputFormat.class);
-    opts.setAccumuloConfigs(job);
 
-    Set<Range> ranges = null;
-    String clone = opts.getTableName();
-    Connector conn = null;
+    boolean scanOffline = Boolean.parseBoolean(props.getProperty(TestProps.CI_VERIFY_SCAN_OFFLINE));
+    String tableName = env.getAccumuloTableName();
+    int maxMaps = Integer.parseInt(props.getProperty(TestProps.CI_VERIFY_MAX_MAPS));
+    int reducers = Integer.parseInt(props.getProperty(TestProps.CI_VERIFY_REDUCERS));
+    String outputDir = props.getProperty(TestProps.CI_VERIFY_OUTPUT_DIR);
+
+    Set<Range> ranges;
+    String clone = "";
+    Connector conn = env.getAccumuloConnector();
 
-    if (opts.scanOffline) {
+    if (scanOffline) {
       Random random = new Random();
-      clone = opts.getTableName() + "_" + String.format("%016x", (random.nextLong() & 0x7fffffffffffffffl));
-      conn = opts.getConnector();
-      conn.tableOperations().clone(opts.getTableName(), clone, true, new HashMap<String,String>(), new HashSet<String>());
-      ranges = conn.tableOperations().splitRangeByTablets(opts.getTableName(), new Range(), opts.maxMaps);
+      clone = tableName + "_" + String.format("%016x", (random.nextLong() & 0x7fffffffffffffffL));
+      conn.tableOperations().clone(tableName, clone, true, new HashMap<>(), new HashSet<>());
+      ranges = conn.tableOperations().splitRangeByTablets(tableName, new Range(), maxMaps);
       conn.tableOperations().offline(clone);
       AccumuloInputFormat.setInputTableName(job, clone);
       AccumuloInputFormat.setOfflineTableScan(job, true);
     } else {
-      ranges = opts.getConnector().tableOperations().splitRangeByTablets(opts.getTableName(), new Range(), opts.maxMaps);
+      ranges = conn.tableOperations().splitRangeByTablets(tableName, new Range(), maxMaps);
+      AccumuloInputFormat.setInputTableName(job, tableName);
     }
-
+    
     AccumuloInputFormat.setRanges(job, ranges);
     AccumuloInputFormat.setAutoAdjustRanges(job, false);
+    AccumuloInputFormat.setConnectorInfo(job, env.getAccumuloUserName(), env.getToken());
+    AccumuloInputFormat.setZooKeeperInstance(job, env.getClientConfiguration());
 
     job.setMapperClass(CMapper.class);
     job.setMapOutputKeyClass(LongWritable.class);
     job.setMapOutputValueClass(VLongWritable.class);
 
     job.setReducerClass(CReducer.class);
-    job.setNumReduceTasks(opts.reducers);
+    job.setNumReduceTasks(reducers);
 
     job.setOutputFormatClass(TextOutputFormat.class);
 
-    job.getConfiguration().setBoolean("mapred.map.tasks.speculative.execution", opts.scanOffline);
+    job.getConfiguration().setBoolean("mapred.map.tasks.speculative.execution", scanOffline);
 
-    TextOutputFormat.setOutputPath(job, new Path(opts.outputDir));
+    TextOutputFormat.setOutputPath(job, new Path(outputDir));
 
     job.waitForCompletion(true);
 
-    if (opts.scanOffline) {
+    if (scanOffline) {
       conn.tableOperations().delete(clone);
     }
-    opts.stopTracing();
     return job.isSuccessful() ? 0 : 1;
   }
 
-  /**
-   *
-   * @param args
-   *          instanceName zookeepers username password table columns outputpath
-   */
   public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(CachedConfiguration.getInstance(), new ContinuousVerify(), args);
+
+    ContinuousEnv env = new ContinuousEnv(TestProps.loadFromFile(args[0]));
+
+    int res = ToolRunner.run(env.getHadoopConfiguration(), new ContinuousVerify(), args);
     if (res != 0)
       System.exit(res);
   }

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousWalk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousWalk.java b/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousWalk.java
index 2335fd4..49c10c9 100644
--- a/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousWalk.java
+++ b/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousWalk.java
@@ -18,106 +18,49 @@ package org.apache.accumulo.testing.core.continuous;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
 import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
 import java.util.Map.Entry;
+import java.util.Properties;
 import java.util.Random;
 import java.util.zip.CRC32;
 
-import org.apache.accumulo.core.cli.ClientOnDefaultTable;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.trace.Span;
 import org.apache.accumulo.core.trace.Trace;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
+import org.apache.accumulo.testing.core.TestProps;
 import org.apache.hadoop.io.Text;
 
-import com.beust.jcommander.IStringConverter;
-import com.beust.jcommander.Parameter;
-
 public class ContinuousWalk {
 
-  static public class Opts extends ContinuousQuery.Opts {
-    class RandomAuthsConverter implements IStringConverter<RandomAuths> {
-      @Override
-      public RandomAuths convert(String value) {
-        try {
-          return new RandomAuths(value);
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-      }
-    }
-
-    @Parameter(names = "--authsFile", description = "read the authorities to use from a file")
-    RandomAuths randomAuths = new RandomAuths();
-  }
-
   static class BadChecksumException extends RuntimeException {
     private static final long serialVersionUID = 1L;
 
-    public BadChecksumException(String msg) {
+    BadChecksumException(String msg) {
       super(msg);
     }
 
   }
 
-  static class RandomAuths {
-    private List<Authorizations> auths;
-
-    RandomAuths() {
-      auths = Collections.singletonList(Authorizations.EMPTY);
-    }
-
-    RandomAuths(String file) throws IOException {
-      if (file == null) {
-        auths = Collections.singletonList(Authorizations.EMPTY);
-        return;
-      }
-
-      auths = new ArrayList<>();
-
-      FileSystem fs = FileSystem.get(new Configuration());
-      BufferedReader in = new BufferedReader(new InputStreamReader(fs.open(new Path(file)), UTF_8));
-      try {
-        String line;
-        while ((line = in.readLine()) != null) {
-          auths.add(new Authorizations(line.split(",")));
-        }
-      } finally {
-        in.close();
-      }
-    }
-
-    Authorizations getAuths(Random r) {
-      return auths.get(r.nextInt(auths.size()));
-    }
-  }
-
   public static void main(String[] args) throws Exception {
-    Opts opts = new Opts();
-    ClientOnDefaultTable clientOpts = new ClientOnDefaultTable("ci");
-    clientOpts.parseArgs(ContinuousWalk.class.getName(), args, opts);
 
-    Connector conn = clientOpts.getConnector();
+    Properties props = TestProps.loadFromFile(args[0]);
+    ContinuousEnv env = new ContinuousEnv(props);
+
+    Connector conn = env.getAccumuloConnector();
 
     Random r = new Random();
 
     ArrayList<Value> values = new ArrayList<>();
 
+    int sleepTime = Integer.parseInt(props.getProperty(TestProps.CI_WALKER_SLEEP_MS));
+
     while (true) {
-      Scanner scanner = ContinuousUtil.createScanner(conn, clientOpts.getTableName(), opts.randomAuths.getAuths(r));
-      String row = findAStartRow(opts.min, opts.max, scanner, r);
+      Scanner scanner = ContinuousUtil.createScanner(conn, env.getAccumuloTableName(), env.getRandomAuthorizations());
+      String row = findAStartRow(env.getRowMin(), env.getRowMax(), scanner, r);
 
       while (row != null) {
 
@@ -146,12 +89,12 @@ public class ContinuousWalk {
           row = null;
         }
 
-        if (opts.sleepTime > 0)
-          Thread.sleep(opts.sleepTime);
+        if (sleepTime > 0)
+          Thread.sleep(sleepTime);
       }
 
-      if (opts.sleepTime > 0)
-        Thread.sleep(opts.sleepTime);
+      if (sleepTime > 0)
+        Thread.sleep(sleepTime);
     }
   }
 
@@ -197,7 +140,7 @@ public class ContinuousWalk {
     return -1;
   }
 
-  static String getPrevRow(Value value) {
+  private static String getPrevRow(Value value) {
 
     byte[] val = value.get();
     int offset = getPrevRowOffset(val);
@@ -208,7 +151,7 @@ public class ContinuousWalk {
     return null;
   }
 
-  static int getChecksumOffset(byte val[]) {
+  private static int getChecksumOffset(byte val[]) {
     if (val[val.length - 1] != ':') {
       if (val[val.length - 9] != ':')
         throw new IllegalArgumentException(new String(val, UTF_8));


[5/5] accumulo-testing git commit: ACCUMULO-4510 Refactored Continous Ingest tests

Posted by mw...@apache.org.
ACCUMULO-4510 Refactored Continous Ingest tests

* Continuous ingest applications can be launched locally or in YARN and
  are now configured by accumulo-testing.properties file
* Applications are now launched using the accumulo-testing command
  rather than custom bash scripts
* Removed continuous stats collector as its uses Accumulo
  internals and corresponding report generation code.
* Agitator was seperated from continuous ingests. It is
  run by accumulo-testing command and configured by
  accumulo-testing-env.sh


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

Branch: refs/heads/master
Commit: fc3ddfc4f9a2a259a04d58e87fb97da89a6c5dbc
Parents: b81229d
Author: Mike Walch <mw...@apache.org>
Authored: Thu Jan 12 11:03:41 2017 -0500
Committer: Mike Walch <mw...@apache.org>
Committed: Wed Jan 25 12:19:14 2017 -0500

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 README.md                                       |  96 +++++++-
 bin/accumulo-testing                            | 163 ++++++++++++-
 conf/accumulo-testing-env.sh.example            |  28 +++
 conf/accumulo-testing.properties.example        |  81 ++++++-
 continuous/README.md                            | 103 ---------
 continuous/analyze-missing.pl                   | 127 ----------
 continuous/analyze-missing.sh                   |  23 --
 continuous/batch_walkers.txt.example            |  16 --
 continuous/continuous-env.sh.example            | 131 -----------
 continuous/datanode-agitator.pl                 | 140 ------------
 continuous/hdfs-agitator.pl                     | 217 ------------------
 continuous/ingesters.txt.example                |  17 --
 continuous/master-agitator.pl                   |  92 --------
 continuous/report.pl                            | 120 ----------
 continuous/run-moru.sh                          |  37 ---
 continuous/run-verify.sh                        |  42 ----
 continuous/scanners.txt.example                 |  16 --
 continuous/start-agitator.sh                    |  72 ------
 continuous/start-batchwalkers.sh                |  42 ----
 continuous/start-ingest.sh                      |  45 ----
 continuous/start-scanners.sh                    |  41 ----
 continuous/start-stats.sh                       |  49 ----
 continuous/start-walkers.sh                     |  41 ----
 continuous/stop-agitator.sh                     |  51 -----
 continuous/stop-batchwalkers.sh                 |  33 ---
 continuous/stop-ingest.sh                       |  33 ---
 continuous/stop-scanners.sh                     |  33 ---
 continuous/stop-stats.sh                        |  33 ---
 continuous/stop-walkers.sh                      |  33 ---
 continuous/tserver-agitator.pl                  | 134 -----------
 continuous/walkers.txt.example                  |  17 --
 .../apache/accumulo/testing/core/TestEnv.java   | 179 +++++++++++++++
 .../apache/accumulo/testing/core/TestProps.java |  94 +++++++-
 .../core/continuous/ContinuousBatchWalker.java  |  44 ++--
 .../testing/core/continuous/ContinuousEnv.java  |  66 ++++++
 .../core/continuous/ContinuousIngest.java       | 105 ++++-----
 .../testing/core/continuous/ContinuousMoru.java |  74 +++---
 .../core/continuous/ContinuousQuery.java        |  74 ------
 .../core/continuous/ContinuousScanner.java      |  42 ++--
 .../continuous/ContinuousStatsCollector.java    | 206 -----------------
 .../core/continuous/ContinuousVerify.java       |  89 +++----
 .../testing/core/continuous/ContinuousWalk.java |  91 ++------
 .../testing/core/continuous/CreateTable.java    |  74 ++++++
 .../testing/core/continuous/GenSplits.java      |  87 -------
 .../testing/core/continuous/HistData.java       |  49 ----
 .../testing/core/continuous/Histogram.java      | 153 -------------
 .../core/continuous/PrintScanTimeHistogram.java |  95 --------
 .../testing/core/continuous/TimeBinner.java     |   2 +-
 .../testing/core/randomwalk/Environment.java    | 229 -------------------
 .../testing/core/randomwalk/Fixture.java        |   4 +-
 .../testing/core/randomwalk/Framework.java      |  11 +-
 .../testing/core/randomwalk/Module.java         |   8 +-
 .../accumulo/testing/core/randomwalk/Node.java  |   2 +-
 .../testing/core/randomwalk/RandWalkEnv.java    |  84 +++++++
 .../core/randomwalk/bulk/BulkImportTest.java    |   6 +-
 .../core/randomwalk/bulk/BulkMinusOne.java      |   4 +-
 .../core/randomwalk/bulk/BulkPlusOne.java       |   8 +-
 .../testing/core/randomwalk/bulk/BulkTest.java  |   6 +-
 .../testing/core/randomwalk/bulk/Compact.java   |   6 +-
 .../core/randomwalk/bulk/ConsistencyCheck.java  |  10 +-
 .../testing/core/randomwalk/bulk/Merge.java     |   6 +-
 .../core/randomwalk/bulk/SelectiveBulkTest.java |   4 +-
 .../core/randomwalk/bulk/SelectiveQueueing.java |   6 +-
 .../testing/core/randomwalk/bulk/Setup.java     |   6 +-
 .../testing/core/randomwalk/bulk/Split.java     |   6 +-
 .../testing/core/randomwalk/bulk/Verify.java    |  12 +-
 .../core/randomwalk/concurrent/AddSplits.java   |   6 +-
 .../core/randomwalk/concurrent/BatchScan.java   |   6 +-
 .../core/randomwalk/concurrent/BatchWrite.java  |   6 +-
 .../core/randomwalk/concurrent/BulkImport.java  |   7 +-
 .../concurrent/ChangeAuthorizations.java        |   6 +-
 .../concurrent/ChangePermissions.java           |   6 +-
 .../randomwalk/concurrent/CheckPermission.java  |   6 +-
 .../core/randomwalk/concurrent/CloneTable.java  |   6 +-
 .../core/randomwalk/concurrent/Compact.java     |   6 +-
 .../concurrent/ConcurrentFixture.java           |   6 +-
 .../core/randomwalk/concurrent/Config.java      |  30 +--
 .../randomwalk/concurrent/CreateNamespace.java  |   6 +-
 .../core/randomwalk/concurrent/CreateTable.java |   6 +-
 .../core/randomwalk/concurrent/CreateUser.java  |   6 +-
 .../randomwalk/concurrent/DeleteNamespace.java  |   6 +-
 .../core/randomwalk/concurrent/DeleteRange.java |   6 +-
 .../core/randomwalk/concurrent/DeleteTable.java |   6 +-
 .../core/randomwalk/concurrent/DropUser.java    |   6 +-
 .../randomwalk/concurrent/IsolatedScan.java     |   6 +-
 .../core/randomwalk/concurrent/ListSplits.java  |   6 +-
 .../core/randomwalk/concurrent/Merge.java       |   6 +-
 .../randomwalk/concurrent/OfflineTable.java     |   6 +-
 .../randomwalk/concurrent/RenameNamespace.java  |   6 +-
 .../core/randomwalk/concurrent/RenameTable.java |   6 +-
 .../core/randomwalk/concurrent/Replication.java |  10 +-
 .../core/randomwalk/concurrent/ScanTable.java   |   6 +-
 .../core/randomwalk/concurrent/Setup.java       |   4 +-
 .../core/randomwalk/conditional/Compact.java    |   6 +-
 .../core/randomwalk/conditional/Flush.java      |   6 +-
 .../core/randomwalk/conditional/Init.java       |   6 +-
 .../core/randomwalk/conditional/Merge.java      |   6 +-
 .../core/randomwalk/conditional/Setup.java      |  10 +-
 .../core/randomwalk/conditional/Split.java      |   6 +-
 .../core/randomwalk/conditional/TearDown.java   |   4 +-
 .../core/randomwalk/conditional/Transfer.java   |   6 +-
 .../core/randomwalk/conditional/Verify.java     |   6 +-
 .../testing/core/randomwalk/image/Commit.java   |   4 +-
 .../core/randomwalk/image/ImageFixture.java     |  12 +-
 .../testing/core/randomwalk/image/ScanMeta.java |   6 +-
 .../testing/core/randomwalk/image/TableOp.java  |   6 +-
 .../testing/core/randomwalk/image/Verify.java   |   6 +-
 .../testing/core/randomwalk/image/Write.java    |   4 +-
 .../core/randomwalk/multitable/Commit.java      |   4 +-
 .../core/randomwalk/multitable/CopyTable.java   |  20 +-
 .../core/randomwalk/multitable/CreateTable.java |   8 +-
 .../core/randomwalk/multitable/DropTable.java   |   6 +-
 .../multitable/MultiTableFixture.java           |   8 +-
 .../randomwalk/multitable/OfflineTable.java     |   8 +-
 .../core/randomwalk/multitable/Write.java       |   4 +-
 .../randomwalk/security/AlterSystemPerm.java    |   6 +-
 .../core/randomwalk/security/AlterTable.java    |   8 +-
 .../randomwalk/security/AlterTablePerm.java     |  16 +-
 .../core/randomwalk/security/Authenticate.java  |  10 +-
 .../core/randomwalk/security/ChangePass.java    |   8 +-
 .../core/randomwalk/security/CreateTable.java   |   8 +-
 .../core/randomwalk/security/CreateUser.java    |   8 +-
 .../core/randomwalk/security/DropTable.java     |  14 +-
 .../core/randomwalk/security/DropUser.java      |   8 +-
 .../randomwalk/security/SecurityFixture.java    |  10 +-
 .../core/randomwalk/security/SetAuths.java      |  10 +-
 .../core/randomwalk/security/TableOp.java       |   6 +-
 .../core/randomwalk/security/Validate.java      |  12 +-
 .../randomwalk/security/WalkingSecurity.java    |  12 +-
 .../core/randomwalk/sequential/BatchVerify.java |   6 +-
 .../core/randomwalk/sequential/Commit.java      |   4 +-
 .../randomwalk/sequential/MapRedVerify.java     |  16 +-
 .../sequential/SequentialFixture.java           |  12 +-
 .../core/randomwalk/sequential/Write.java       |   4 +-
 .../core/randomwalk/shard/BulkInsert.java       |  13 +-
 .../core/randomwalk/shard/CloneIndex.java       |   8 +-
 .../testing/core/randomwalk/shard/Commit.java   |   4 +-
 .../core/randomwalk/shard/CompactFilter.java    |  10 +-
 .../testing/core/randomwalk/shard/Delete.java   |   4 +-
 .../core/randomwalk/shard/DeleteSomeDocs.java   |   8 +-
 .../core/randomwalk/shard/DeleteWord.java       |   8 +-
 .../core/randomwalk/shard/ExportIndex.java      |  29 ++-
 .../testing/core/randomwalk/shard/Flush.java    |   6 +-
 .../testing/core/randomwalk/shard/Grep.java     |   8 +-
 .../testing/core/randomwalk/shard/Insert.java   |   4 +-
 .../testing/core/randomwalk/shard/Merge.java    |  12 +-
 .../testing/core/randomwalk/shard/Reindex.java  |   8 +-
 .../testing/core/randomwalk/shard/Search.java   |  10 +-
 .../core/randomwalk/shard/ShardFixture.java     |  14 +-
 .../testing/core/randomwalk/shard/Split.java    |   6 +-
 .../core/randomwalk/shard/VerifyIndex.java      |  12 +-
 .../core/randomwalk/unit/CreateTable.java       |   4 +-
 .../core/randomwalk/unit/DeleteTable.java       |   4 +-
 .../testing/core/randomwalk/unit/Ingest.java    |   4 +-
 .../testing/core/randomwalk/unit/Scan.java      |   4 +-
 .../testing/core/randomwalk/unit/Verify.java    |   4 +-
 .../randomwalk/ReplicationRandomWalkIT.java     |   8 +-
 libexec/analyze-missing.pl                      | 127 ++++++++++
 libexec/datanode-agitator.pl                    | 140 ++++++++++++
 libexec/hdfs-agitator.pl                        | 217 ++++++++++++++++++
 libexec/master-agitator.pl                      |  92 ++++++++
 libexec/tserver-agitator.pl                     | 134 +++++++++++
 163 files changed, 2124 insertions(+), 3346 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index f534230..7e54b72 100644
--- a/.gitignore
+++ b/.gitignore
@@ -4,3 +4,4 @@
 /target/
 /*.iml
 /.idea
+/logs/

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index f19b91b..0d2b538 100644
--- a/README.md
+++ b/README.md
@@ -32,9 +32,8 @@ on your machine as well as an Accumulo instance to use for testing.
 The random walk test generates client behavior on an Apache Accumulo instance by randomly walking a
 graph of client operations. 
 
-Before running random walk, review the `test.common.*` and `test.randomwalk.*` properties in
-`accumulo-testing.properties` file. A test module must also be specified. See the [modules][modules]
-directory for a list of available ones.
+Before running random walk, review the `test.common.*` properties in `accumulo-testing.properties`
+file. A test module must also be specified. See the [modules] directory for a list of available ones.
 
 The command below will start a single random walker in a local process using the [Image.xml][image]
 module.
@@ -46,9 +45,94 @@ walkers in 5 containers in YARN using the Image.xml module.
 
         ./bin/accumulo-testing rw-yarn 5 Image.xml
 
-This command will create an application in YARN and exit when all containers for the test have started.
-While its running, you can view logs for each random walker using the YARN resource manager. The YARN
-application can be killed at any time using the YARN resource manager or command line tool.
+This command will create an application in YARN and exit when all containers for the test have
+started. While its running, you can view logs for each random walker using the YARN resource manager.
+The YARN application can be killed at any time using the YARN resource manager or command line tool.
+
+## Continuous Ingest & Query
+
+The Continuous Ingest test runs many ingest clients that continually create linked lists of data
+in Accumulo. During ingest, query applications can be run to continously walk and verify the the
+linked lists and put a query load on Accumulo. At some point, the ingest clients are stopped and
+a MapReduce job is run to ensure that there are no holes in any linked list.
+
+The nodes in the linked list are random. This causes each linked list to spread across the table.
+Therefore, if one part of a table loses data, then it will be detected by references in another
+part of table.
+
+Before running any of the Continuous Ingest applications, make sure that the
+`accumulo-testing.properties` file exists in `conf/` and review all properties with the
+`test.ci.*` prefix.
+
+First, run the command below to create an Accumulo table for the continuous ingest tests. The name of the
+table is set by the property `test.ci.common.accumulo.table` (its value defaults to `ci`) in the file
+`accumulo-testing.properties`:
+
+          ./bin/accumulo-testing ci-createtable
+
+The continuous ingest tests have several applications that can either be started in a local process
+or run in multiple containers across a cluster using YARN. The `ci-local` command starts a local
+application which will run continuously until you stop using `ctrl-c`:
+
+          ./bin/accumulo-testing ci-local <application>
+
+The `ci-yarn` command starts an application in `<num>` containers in YARN. All containers will run
+continuously performing the same work until you kill the application in YARN. The logs for the
+application can be viewed using the YARN resource manager.
+
+          ./bin/accumulo-testing ci-yarn <num> <application>
+
+Below is a list of available continuous ingest applications. You should run the `ingest` application
+first to add data to your table.
+
+* `ingest` - Inserts data into Accumulo that will form a random graph.
+* `walk` - Randomly walks the graph created by ingest application using scanner. Each walker
+  produces detailed statistics on query/scan times.
+* `batchwalk` - Randomly walks the graph created by ingest using a batch scanner.
+* `scan` - Scans the graph
+
+The continuous ingest test has two MapReduce jobs that are used to verify and stress
+Accumulo and have the following command:
+
+          ./bin/accumulo-testing ci-mapred <application>
+
+Below is a list of available MapReduce applications:
+
+1. `verify` - Runs a MapReduce job that verifies all data created by continuous ingest. Before
+running, review all `test.ci.verify.*` properties. Do not run ingest while running this command as
+it will cause erroneous reporting of UNDEFINED nodes. Each entry, except for the first batch of
+entries, inserted by continuous ingest references a previously flushed entry. Since we are
+referencing flushed entries, they should always exist. The MapReduce job checks that all referenced
+entries exist. If it finds any that do not exist it will increment the UNDEFINED counter and emit
+the referenced but undefined node.  The MapReduce job produces two other counts: REFERENCED and
+UNREFERENCED. It is expected that these two counts are non zero. REFERENCED counts nodes that are
+defined and referenced. UNREFERENCED counts nodes that defined and unreferenced, these are the
+latest nodes inserted.
+
+2. `moru` - Runs a MapReduce job that stresses Accumulo by reading and writing the continuous ingest
+table. This MapReduce job will write out an entry for every entry in the table (except for ones
+created by the MapReduce job itself). Stop ingest before running this MapReduce job. Do not run more
+than one instance of this MapReduce job concurrently against a table.
+
+## Agitator
+
+The agitator will periodically kill the Accumulo master, tablet server, and Hadoop data node
+processes on random nodes. Before running the agitator you should create `accumulo-testing-env.sh`
+in `conf/` and review all of the agitator settings. The command below will start the agitator:
+
+            ./bin/accumulo-testing agitator start
+
+You can run this script as root and it will properly start processes as the user you configured in
+`accumulo-testing-env.sh` (`AGTR_HDFS_USER` for the data node and `AGTR_ACCUMULO_USER` for Accumulo
+processes). If you run it as yourself and the `AGTR_HDFS_USER` and `AGTR_ACCUMULO_USER` values are
+the same as your user, the agitator will not change users. In the case where you run the agitator as
+a non-privileged user which isn't the same as `AGTR_HDFS_USER` or `AGTR_ACCUMULO_USER`, the agitator
+will attempt to `sudo` to these users, which relies on correct configuration of sudo. Also, be sure
+that your `AGTR_HDFS_USER` has password-less `ssh` configured.
+
+Run the command below stop the agitator:
+
+            ./bin/accumulo-testing agitator stop
 
 [modules]: core/src/main/resources/randomwalk/modules
 [image]: core/src/main/resources/randomwalk/modules/Image.xml

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/bin/accumulo-testing
----------------------------------------------------------------------
diff --git a/bin/accumulo-testing b/bin/accumulo-testing
index dc6f5da..7cf7206 100755
--- a/bin/accumulo-testing
+++ b/bin/accumulo-testing
@@ -25,8 +25,16 @@ function print_usage() {
 Usage: accumulo-testing <command> (<argument>)
 
 Possible commands:
-  rw-local <module>         Runs randomwalk <module> in local java process
-  rw-yarn <num> <module>    Runs randomwalk <module> in <num> containers on YARN
+  agitator <command>            Runs agitator <command>. Available commands: start, stop
+  ci-createtable                Creates Accumulo table with splits for continuous ingest tests
+  ci-local <application>        Runs continuous ingest <application> in local java process
+                                Applications: ingest, walk, batchwalk, scan
+  ci-yarn <num> <application>   Runs continuous ingest <application> in <num> containers in YARN
+                                Applications: ingest, walk, batchwalk, scan
+  ci-mapred <application>       Runs continuous ingest mapreduce <application>
+                                Applications: verify, moru  
+  rw-local <module>             Runs randomwalk <module> in local java process
+  rw-yarn <num> <module>        Runs randomwalk <module> in <num> containers on YARN
 EOF
 }
 
@@ -69,8 +77,143 @@ function build_shade_jar() {
   fi
 }
 
+function determine_app_main() {
+  ci_package="org.apache.accumulo.testing.core.continuous"
+  case "$1" in 
+    ingest)
+      ci_main="${ci_package}.ContinuousIngest"
+      ;;
+    walk)
+      ci_main="${ci_package}.ContinuousWalk"
+      ;;
+    batchwalk)
+      ci_main="${ci_package}.ContinuousBatchWalker"
+      ;;
+    scan)
+      ci_main="${ci_package}.ContinuousScanner"
+      ;;
+    *)
+      echo "Unknown application: $1"
+      print_usage
+      exit 1    
+  esac
+}
+
+function determine_mapred_main() {
+  ci_package="org.apache.accumulo.testing.core.continuous"
+  case "$1" in 
+    verify)
+      ci_main="${ci_package}.ContinuousVerify"
+      ;;
+    moru)
+      ci_main="${ci_package}.ContinuousMoru"
+      ;;
+    *)
+      echo "Unknown application: $1"
+      print_usage
+      exit 1    
+  esac
+}
+
+function start_agitator() {
+  mkdir -p "${at_home}/logs"
+  log_base="${at_home}/logs/$(date +%Y%m%d%H%M%S)_$(hostname)"
+  libexec="${at_home}/libexec"
+  master_log="${log_base}_master-agitator"
+  tserver_log="${log_base}_tserver-agitator"
+  datanode_log="${log_base}_datanode-agitator"
+  master_cmd="nohup ${libexec}/master-agitator.pl $AGTR_MASTER_KILL_SLEEP_TIME $AGTR_MASTER_RESTART_SLEEP_TIME"
+  tserver_cmd="nohup ${libexec}/tserver-agitator.pl $AGTR_TSERVER_KILL_SLEEP_TIME $AGTR_TSERVER_RESTART_SLEEP_TIME $AGTR_TSERVER_MIN_KILL $AGTR_TSERVER_MAX_KILL"
+  datanode_cmd="nohup ${libexec}/datanode-agitator.pl $AGTR_DATANODE_KILL_SLEEP_TIME $AGTR_DATANODE_RESTART_SLEEP_TIME $HADOOP_PREFIX $AGTR_DATANODE_MIN_KILL $AGTR_DATANODE_MAX_KILL"
+  [[ -n $AGITATOR_USER ]] || AGITATOR_USER=$(whoami)
+
+  if [[ $AGITATOR_USER == root ]];  then
+    echo "Running master-agitator and tserver-agitator as $AGTR_ACCUMULO_USER using su. Running datanode-agitator as $AGTR_HDFS_USER using su."
+    su -c "$master_cmd >${master_log}.out 2>${master_log}.err" & -m - "$AGTR_ACCUMULO_USER"
+    su -c "$tserver_cmd >${tserver_log}.out 2>${tserver_log}.err" & -m - "$AGTR_ACCUMULO_USER"
+    su -c "$datanode_cmd >${datanode_log}.out 2>${datanode_log}.err" & -m - "$AGTR_HDFS_USER"
+  elif [[ $AGITATOR_USER == "$AGTR_ACCUMULO_USER" ]]; then
+    echo "Running master-agitator and tserver-agitator as $AGITATOR_USER Running datanode-agitator as $AGTR_HDFS_USER using sudo."
+    $master_cmd > "${master_log}.out" 2> "${master_log}.err" &
+    $tserver_cmd > "${tserver_log}.out" 2> "${tserver_log}.err" &
+    sudo -u "$AGTR_HDFS_USER" "$datanode_cmd" > "${datanode_log}.out" 2> "${datanode_log}.err" &
+  else
+    echo "Running master-agitator and tserver-agitator as $AGTR_ACCUMULO_USER using sudo. Running datanode-agitator as $AGTR_HDFS_USER using sudo."
+    # Not root, and not the accumulo user, hope you can sudo to it
+    sudo -u "$AGTR_ACCUMULO_USER" "$master_cmd" > "${master_log}.out" 2> "${master_log}.err" &
+    sudo -u "$AGTR_ACCUMULO_USER" "$tserver_cmd" > "${tserver_log}.out" 2> "${tserver_log}.err" &
+    sudo -u "$AGTR_HDFS_USER" "$datanode_cmd" > "${datanode_log}.out" 2> "${datanode_log}.err" & -m - "$AGTR_HDFS_USER"
+  fi
+
+  if ${AGTR_HDFS:-false} ; then
+    agitator_log=${log_base}_hdfs-agitator
+    sudo -u "$AGTR_HDFS_SUPERUSER" nohup "${libexec}/hdfs-agitator.pl" --sleep "${AGTR_HDFS_SLEEP_TIME}" --hdfs-cmd "${AGTR_HDFS_COMMAND}" --superuser "${AGTR_HDFS_SUPERUSER}" >"${agitator_log}.out" 2>"${agitator_log}.err" &
+  fi
+}
+
+function stop_agitator() {
+  [[ -n $AGITATOR_USER ]] || AGITATOR_USER=$(whoami)
+  if [[ $AGITATOR_USER == root ]]; then
+    echo "Stopping all processes matching 'agitator.pl' as root"
+    pkill -f agitator.pl 2>/dev/null
+  elif [[ $AGITATOR_USER == "$AGTR_ACCUMULO_USER" ]];  then
+    echo "Stopping all processes matching 'datanode-agitator.pl' as $AGTR_HDFS_USER"
+    sudo -u "$AGTR_HDFS_USER" pkill -f datanode-agitator.pl 2>/dev/null
+    echo "Stopping all processes matching 'hdfs-agitator.pl' as $AGTR_HDFS_USER"
+    sudo -u "$AGTR_HDFS_USER" pkill -f hdfs-agitator.pl 2>/dev/null
+    echo "Stopping all processes matching 'agitator.pl' as $AGITATOR_USER"
+    pkill -f agitator.pl 2>/dev/null 2>/dev/null
+  else
+    echo "Stopping all processes matching 'datanode-agitator.pl' as $AGTR_HDFS_USER"
+    sudo -u "$AGTR_HDFS_USER" pkill -f datanode-agitator.pl 2>/dev/null
+    echo "Stopping all processes matching 'hdfs-agitator.pl' as $AGTR_HDFS_USER"
+    sudo -u "$AGTR_HDFS_USER" pkill -f hdfs-agitator.pl 2>/dev/null
+    echo "Stopping all processes matching 'agitator.pl' as $AGTR_ACCUMULO_USER"
+    sudo -u "$AGTR_ACCUMULO_USER" pkill -f agitator.pl 2>/dev/null
+  fi
+}
+
 randomwalk_main="org.apache.accumulo.testing.core.randomwalk.Framework"
 case "$1" in
+ci-createtable)
+  build_shade_jar
+  java -Dlog4j.configuration="file:$log4j_config" -cp "$at_shaded_jar" org.apache.accumulo.testing.core.continuous.CreateTable "$at_props"
+  ;;
+ci-local)
+  if [ -z "$2" ]; then
+    echo "ERROR: <application> needs to be set"
+    print_usage
+    exit 1
+  fi
+  determine_app_main "$2"
+  build_shade_jar
+  java -Dlog4j.configuration="file:$log4j_config" -cp "$at_shaded_jar" "$ci_main" "$at_props"
+  ;;
+ci-yarn)
+  if [ -z "$2" ]; then
+    echo "ERROR: <num> needs to be set"
+    print_usage
+    exit 1
+  fi
+  if [ -z "$3" ]; then
+    echo "ERROR: <application> needs to be set"
+    print_usage
+    exit 1
+  fi
+  determine_app_main "$3"
+  build_shade_jar
+  mvn compile -P yarn-test-runner -D hadoop.version="$HADOOP_VERSION" -D exec.args="-t AccumuloCITest-$3 -j $at_shaded_jar -m $ci_main -n $2 -p $at_props -l $log4j_config -a ./accumulo-testing.properties"
+  ;;
+ci-mapred)
+  if [ -z "$2" ]; then
+    echo "ERROR: <application> needs to be set"
+    print_usage
+    exit 1
+  fi
+  determine_mapred_main "$2"
+  build_shade_jar
+  "$HADOOP_PREFIX"/bin/yarn jar "$at_shaded_jar" "$ci_main" "$at_props"
+  ;;
 rw-local)
   if [ -z "$2" ]; then
     echo "ERROR: <module> needs to be set"
@@ -92,7 +235,21 @@ rw-yarn)
     exit 1
   fi
   build_shade_jar
-  mvn compile -P yarn-test-runner -D hadoop.version="$HADOOP_VERSION" -D exec.args="-t AccumuloRandomWalkTest -j $at_shaded_jar -m $randomwalk_main -n $2 -p $at_props -l $log4j_config -a ./accumulo-testing.properties $3"
+  mvn compile -P yarn-test-runner -D hadoop.version="$HADOOP_VERSION" -D exec.args="-t AccumuloRWTest-$3 -j $at_shaded_jar -m $randomwalk_main -n $2 -p $at_props -l $log4j_config -a ./accumulo-testing.properties $3"
+  ;;
+agitator)
+  case "$2" in
+    start)
+      start_agitator
+      ;;
+    stop)
+      stop_agitator
+      ;;
+    *)
+      echo "ERROR: unknown command - $2"
+      print_usage
+      exit 1
+  esac
   ;;
 *)
   echo "Unknown command: $1"

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/conf/accumulo-testing-env.sh.example
----------------------------------------------------------------------
diff --git a/conf/accumulo-testing-env.sh.example b/conf/accumulo-testing-env.sh.example
index 2e3554e..3f71463 100644
--- a/conf/accumulo-testing-env.sh.example
+++ b/conf/accumulo-testing-env.sh.example
@@ -24,3 +24,31 @@ test -z "$HADOOP_CONF_DIR" && export HADOOP_CONF_DIR=/path/to/hadoop/etc/hadoop
 export ACCUMULO_VERSION=`accumulo version`
 export HADOOP_VERSION=`hadoop version | head -n1 | awk '{print $2}'`
 export ZOOKEEPER_VERSION=3.4.9
+
+# Agitator
+# ========
+# Accumulo user
+AGTR_ACCUMULO_USER=$(whoami)
+# Time (in minutes) between killing Accumulo masters
+AGTR_MASTER_KILL_SLEEP_TIME=60
+AGTR_MASTER_RESTART_SLEEP_TIME=2
+# Time (in minutes) between killing Accumulo tservers
+AGTR_TSERVER_KILL_SLEEP_TIME=20
+AGTR_TSERVER_RESTART_SLEEP_TIME=10
+# Min and max number of Accumulo tservers that the agitator will kill at once
+AGTR_TSERVER_MIN_KILL=1
+AGTR_TSERVER_MAX_KILL=1
+# Amount of time (in minutes) the agitator should sleep before killing datanodes
+AGTR_DATANODE_KILL_SLEEP_TIME=20
+# Amount of time (in minutes) the agitator should wait before restarting datanodes
+AGTR_DATANODE_RESTART_SLEEP_TIME=10
+# Min and max number of datanodes the agitator will kill at once
+AGTR_DATANODE_MIN_KILL=1
+AGTR_DATANODE_MAX_KILL=1
+# HDFS agitation
+AGTR_HDFS_USER=$(whoami)
+AGTR_HDFS=false
+AGTR_HDFS_SLEEP_TIME=10
+AGTR_HDFS_SUPERUSER=hdfs
+AGTR_HDFS_COMMAND="${HADOOP_PREFIX:-/usr/lib/hadoop}/share/hadoop/hdfs/bin/hdfs"
+AGTR_HDFS_SUDO=$(which sudo)

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/conf/accumulo-testing.properties.example
----------------------------------------------------------------------
diff --git a/conf/accumulo-testing.properties.example b/conf/accumulo-testing.properties.example
index 05cbaf5..1d43e12 100644
--- a/conf/accumulo-testing.properties.example
+++ b/conf/accumulo-testing.properties.example
@@ -31,14 +31,77 @@ test.common.zookeepers=localhost:2181
 test.common.yarn.container.memory.mb=1024
 # Number of cores given to each container (if running in YARN)
 test.common.yarn.container.cores=1
+# Max memory (in bytes) each batch writer will use to buffer writes
+test.common.bw.max.memory.bytes=100000000
+# Max latency (in milliseconds) that each batch writer will buffer data
+test.common.bw.max.latency.ms=600000
+# Number of write thread for each batch writer
+test.common.bw.num.threads=4
+# Number of threads used by batch scanner
+test.common.bs.num.threads=8
+# Number of key/value entries to pull during scan
+test.common.scanner.batch.size=1000
 
-########################
-# Random walk properties
-########################
+###################################
+# Continuous ingest test properties
+###################################
 
-# Max memory for multi-table batch writer
-test.randomwalk.bw.max.mem=100000000
-# Max latency in milliseconds for multi-table batch writer
-test.randomwalk.bw.max.latency=600000
-# Number of write thread for multi-table batch writer
-test.randomwalk.bw.num.threads=4
+# Common
+# ------
+# Accumulo table used by continuous tests
+test.ci.common.accumulo.table=ci
+# Number of tablets that should exist in Accumulo table when created
+test.ci.common.accumulo.num.tablets=20
+# Optional authorizations that if specified will be randomly selected by scanners and walkers
+# Format: a,b|a,b,c|c
+test.ci.common.auths=
+
+# Ingest
+# ------
+# Number of entries each ingest client should write
+test.ci.ingest.client.entries=9223372036854775807
+# Minimum random row to generate
+test.ci.ingest.row.min=0
+# Maximum random row to generate
+test.ci.ingest.row.max=9223372036854775807
+# Maximum number of random column families to generate
+test.ci.ingest.max.cf=32767
+# Maximum number of random column qualifiers to generate
+test.ci.ingest.max.cq=32767
+# Optional visibilities (in CSV format) that if specified will be randomly selected by ingesters for
+# each linked list
+test.ci.ingest.visibilities=
+# Checksums will be generated during ingest if set to true
+test.ci.ingest.checksum=true
+
+# Batch walker
+# ------------
+# Sleep time between batch scans (in ms)
+test.ci.batch.walker.sleep.ms=10000
+# Scan batch size
+test.ci.batch.walker.batch.size=10000
+
+# Walker
+# ------
+# Sleep time between scans (in ms)
+test.ci.walker.sleep.ms=10000
+
+# Scanner
+# -------
+# Sleep time between scans (in ms)
+test.ci.scanner.sleep.ms=10000
+# Scanner entries
+test.ci.scanner.entries=5000
+
+# Verify
+# -----
+# Maximum number of mapreduce mappers
+test.ci.verify.max.maps=64
+# Number of mapreduce reducers
+test.ci.verify.reducers=64
+# Perform the verification directly on the files while the table is offline
+test.ci.verify.scan.offline=false
+# Comma separated list of auths to use for verify
+test.ci.verify.auths=
+# Location in HDFS to store output. Must not exist.
+test.ci.verify.output.dir=/tmp/ci-verify

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/README.md
----------------------------------------------------------------------
diff --git a/continuous/README.md b/continuous/README.md
deleted file mode 100644
index 31ee4bd..0000000
--- a/continuous/README.md
+++ /dev/null
@@ -1,103 +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.
--->
-
-Continuous Query and Ingest
-===========================
-
-This directory contains a suite of scripts for placing continuous query and
-ingest load on accumulo.  The purpose of these script is two-fold. First,
-place continuous load on accumulo to see if breaks.  Second, collect
-statistics in order to understand how accumulo behaves.  To run these scripts
-copy all of the `.example` files and modify them.  You can put these scripts in
-the current directory or define a `CONTINUOUS_CONF_DIR` where the files will be
-read from. These scripts rely on `pssh`. Before running any script you may need
-to use `pssh` to create the log directory on each machine (if you want it local).
-Also, create the table "ci" before running. You can run
-`org.apache.accumulo.test.continuous.GenSplits` to generate splits points for a
-continuous ingest table.
-
-The following ingest scripts insert data into accumulo that will form a random
-graph.
-
-> $ start-ingest.sh  
-> $ stop-ingest.sh
-
-The following query scripts randomly walk the graph created by the ingesters.
-Each walker produce detailed statistics on query/scan times.
-
-> $ start-walkers.sh  
-> $ stop-walker.sh
-
-The following scripts start and stop batch walkers.
-
-> $ start-batchwalkers.sh  
-> $ stop-batchwalkers.sh
-
-And the following scripts start and stop scanners.
-
-> $ start-scanners.sh
-> $ stop-scanners.sh
-
-In addition to placing continuous load, the following scripts start and stop a
-service that continually collect statistics about accumulo and HDFS.
-
-> $ start-stats.sh  
-> $ stop-stats.sh
-
-Optionally, start the agitator to periodically kill the tabletserver and/or datanode
-process(es) on random nodes. You can run this script as root and it will properly start
-processes as the user you configured in `continuous-env.sh` (`HDFS_USER` for the Datanode and
-`ACCUMULO_USER` for Accumulo processes). If you run it as yourself and the `HDFS_USER` and
-`ACCUMULO_USER` values are the same as your user, the agitator will not change users. In
-the case where you run the agitator as a non-privileged user which isn't the same as `HDFS_USER`
-or `ACCUMULO_USER`, the agitator will attempt to `sudo` to these users, which relies on correct
-configuration of sudo. Also, be sure that your `HDFS_USER` has password-less `ssh` configured.
-
-> $ start-agitator.sh  
-> $ stop-agitator.sh
-
-Start all three of these services and let them run for a few hours. Then run
-`report.pl` to generate a simple HTML report containing plots and histograms
-showing what has transpired.
-
-A MapReduce job to verify all data created by continuous ingest can be run
-with the following command.  Before running the command modify the `VERIFY_*`
-variables in `continuous-env.sh` if needed.  Do not run ingest while running this
-command, this will cause erroneous reporting of UNDEFINED nodes. The MapReduce
-job will scan a reference after it has scanned the definition.
-
-> $ run-verify.sh
-
-Each entry, except for the first batch of entries, inserted by continuous
-ingest references a previously flushed entry.  Since we are referencing flushed
-entries, they should always exist.  The MapReduce job checks that all
-referenced entries exist.  If it finds any that do not exist it will increment
-the UNDEFINED counter and emit the referenced but undefined node.  The MapReduce
-job produces two other counts : REFERENCED and UNREFERENCED.  It is
-expected that these two counts are non zero.  REFERENCED counts nodes that are
-defined and referenced.  UNREFERENCED counts nodes that defined and
-unreferenced, these are the latest nodes inserted.
-
-To stress accumulo, run the following script which starts a MapReduce job
-that reads and writes to your continuous ingest table.  This MapReduce job
-will write out an entry for every entry in the table (except for ones created
-by the MapReduce job itself). Stop ingest before running this MapReduce job.
-Do not run more than one instance of this MapReduce job concurrently against a
-table.
-
-> $ run-moru.sh
-

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/analyze-missing.pl
----------------------------------------------------------------------
diff --git a/continuous/analyze-missing.pl b/continuous/analyze-missing.pl
deleted file mode 100755
index 5cce1b1..0000000
--- a/continuous/analyze-missing.pl
+++ /dev/null
@@ -1,127 +0,0 @@
-#! /usr/bin/env perl
-
-# 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.
-
-
-use POSIX qw(strftime);
-
-if(scalar(@ARGV) != 4){
-	print "Usage : analyze-missing.pl <accumulo home> <continuous log dir> <user> <pass> \n";
-	exit(1);
-}
-
-$ACCUMULO_HOME=$ARGV[0];
-$CONTINUOUS_LOG_DIR=$ARGV[1];
-$USER=$ARGV[2];
-$PASS=$ARGV[3];
-
-
-@missing = `grep MIS $CONTINUOUS_LOG_DIR/*.err`;
-
-
-
-for $miss (@missing) {
-	chomp($miss);
-	($file, $type, $time, $row) = split(/[: ]/, $miss);
-
-	substr($file, -3, 3, "out");
-
-	$prevRowLine = `grep -B 1 $row $file | grep SRQ | grep -v $row`;
-
-	@prla = split(/\s+/, $prevRowLine);
-	$prevRow = $prla[2];
-#	print $prevRow."\n";
-
-	$aScript = `mktemp /tmp/miss_script.XXXXXXXXXX`;
-	chomp($aScript);
-	open(AS, ">$aScript") || die;
-
-	print AS "table ci\n";
-	print AS "scan -b $prevRow -e $prevRow\n";
-	print AS "scan -b $row -e $row\n";
-	print AS "quit\n";
-	close(AS);
-
-	$exist = 0;
-	$ingestIDSame = 0;
-	$ingestId = "";
-	$count = 0;
-
-	@entries = `$ACCUMULO_HOME/bin/accumulo shell -u $USER -p $PASS -f $aScript | grep $row`;
-	system("rm $aScript");
-
-	for $entry (@entries){
-		chomp($entry);
-		@entryA = split(/[: ]+/, $entry);
-		if($entryA[0] eq $row){
-			$exist = 1;
-
-			if($entryA[4] eq $ingestId){
-				$ingestIDSame = 1;
-			}
-		}else{
-			$ingestId = $entryA[4];
-			$count = hex($entryA[5]);
-		}
-	}
-
-
-	#look in ingest logs
-	@ingestLogs = `ls  $CONTINUOUS_LOG_DIR/*ingest*.out`;
-	@flushTimes = ();
-	chomp(@ingestLogs);
-	for $ingestLog (@ingestLogs){
-		open(IL, "<$ingestLog") || die;
-		
-
-		while($firstLine = <IL>){
-			chomp($firstLine);
-			if($firstLine =~ /UUID.*/){
-				last;
-			}
-		}
-
-		@iinfo = split(/\s+/,$firstLine);
-		if($iinfo[2] eq $ingestId){
-			while($line = <IL>){
-				if($line =~ /FLUSH (\d+) \d+ \d+ (\d+) \d+/){
-					push(@flushTimes, $1);
-					if(scalar(@flushTimes) > 3){
-						shift(@flushTimes);
-					}
-					if($count < $2){
-						last;
-					}
-				}
-			}
-		}
-		
-		
-
-		close(IL);
-	
-		if(scalar(@flushTimes) > 0){
-			last;
-		}
-	} 
-
-	$its0 = strftime "%m/%d/%Y_%H:%M:%S", gmtime($flushTimes[0]/1000);
-	$its1 = strftime "%m/%d/%Y_%H:%M:%S", gmtime($flushTimes[1]/1000);
-	$mts = strftime "%m/%d/%Y_%H:%M:%S", gmtime($time/1000);
-
-	print "$row $exist $ingestIDSame $prevRow $ingestId   $its0   $its1   $mts\n";
-}
-

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/analyze-missing.sh
----------------------------------------------------------------------
diff --git a/continuous/analyze-missing.sh b/continuous/analyze-missing.sh
deleted file mode 100755
index e2cfbb1..0000000
--- a/continuous/analyze-missing.sh
+++ /dev/null
@@ -1,23 +0,0 @@
-#! /usr/bin/env bash
-
-# 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.
-
-
-CONTINUOUS_CONF_DIR=${CONTINUOUS_CONF_DIR:-$ACCUMULO_HOME/test/system/continuous/}
-. "$CONTINUOUS_CONF_DIR/continuous-env.sh"
-
-./analyze-missing.pl "$ACCUMULO_HOME" "$CONTINUOUS_LOG_DIR" "$USER" "$PASS"
-

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/batch_walkers.txt.example
----------------------------------------------------------------------
diff --git a/continuous/batch_walkers.txt.example b/continuous/batch_walkers.txt.example
deleted file mode 100644
index 63fb8bb..0000000
--- a/continuous/batch_walkers.txt.example
+++ /dev/null
@@ -1,16 +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.
-
-localhost

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/continuous-env.sh.example
----------------------------------------------------------------------
diff --git a/continuous/continuous-env.sh.example b/continuous/continuous-env.sh.example
deleted file mode 100644
index 0abd8c3..0000000
--- a/continuous/continuous-env.sh.example
+++ /dev/null
@@ -1,131 +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.
-
-# set this to an absolute path that exist on every machine
-# Inherit values from environment if they are already set.
-HADOOP_HOME=${HADOOP_HOME:-/opt/hadoop}
-HADOOP_PREFIX=${HADOOP_PREFIX:-$HADOOP_HOME}
-ACCUMULO_HOME=${ACCUMULO_HOME:-/opt/accumulo}
-ACCUMULO_CONF_DIR=${ACCUMULO_CONF_DIR:-$ACCUMULO_HOME/conf}
-JAVA_HOME=${JAVA_HOME:-/opt/java}
-ZOOKEEPER_HOME=${ZOOKEEPER_HOME:-/opt/zookeeper}
-
-CONTINUOUS_LOG_DIR=$ACCUMULO_HOME/test/system/continuous/logs
-INSTANCE_NAME=instance
-ZOO_KEEPERS=zhost1,zhost2
-ACCUMULO_USER=$(whoami)
-HDFS_USER=$(whoami)
-USER=user
-PASS=pass
-TABLE=ci
-
-#set debug to on to enable logging of accumulo client debugging
-DEBUG_INGEST=off
-DEBUG_WALKER=off
-DEBUG_BATCH_WALKER=off
-DEBUG_SCANNER=off
-
-#the number of entries each client should write
-NUM=9223372036854775807
-
-#the minimum random row to generate
-MIN=0
-
-#the maximum random row to generate
-MAX=9223372036854775807
-
-#the maximum number of random column families to generate
-MAX_CF=32767
-
-#the maximum number of random column qualifiers to generate
-MAX_CQ=32767
-
-#an optional file in hdfs containing visibilites.  If left blank, then column
-#visibility will not be set.  If specified then a random line will be selected
-#from the file and used for column visibility for each linked list.
-VISIBILITIES=''
-
-#the max memory (in bytes) each ingester will use to buffer writes
-MAX_MEM=100000000
-
-#the maximum time (in millis) each ingester will buffer data
-MAX_LATENCY=600000
-
-#the number of threads each ingester will use to write data
-NUM_THREADS=4
-
-#the amount of time (in millis) to sleep between each query
-SLEEP_TIME=10
-
-#an optional file in hdfs containing line of comma seperated auths.  If
-#specified, walkers will randomly select lines from this file and use that to
-#set auths.
-AUTHS=''
-
-#determines if checksum are generated, may want to turn of when performance testing
-CHECKSUM=true
-
-#the amount of time (in minutes) the agitator should sleep before killing tservers
-TSERVER_KILL_SLEEP_TIME=20
-
-#the amount of time (in minutes) the agitator should sleep after killing
-# before restarting tservers
-TSERVER_RESTART_SLEEP_TIME=10
-
-#the minimum and maximum number of tservers the agitator will kill at once
-TSERVER_MIN_KILL=1
-TSERVER_MAX_KILL=1
-
-#the amount of time (in minutes) the agitator should sleep before killing datanodes
-DATANODE_KILL_SLEEP_TIME=20
-
-#the amount of time (in minutes) the agitator should sleep after killing
-# before restarting datanodes
-DATANODE_RESTART_SLEEP_TIME=10
-
-#the minimum and maximum number of datanodes the agitator will kill at once
-DATANODE_MIN_KILL=1
-DATANODE_MAX_KILL=1
-
-#time in minutes between killing masters
-MASTER_KILL_SLEEP_TIME=60
-MASTER_RESTART_SLEEP_TIME=2
-
-#Do we want to perturb HDFS? Only works on HDFS versions with HA, i.e. Hadoop 2
-# AGITATE_HDFS=true
-AGITATE_HDFS=false
-AGITATE_HDFS_SLEEP_TIME=10
-AGITATE_HDFS_SUPERUSER=hdfs
-AGITATE_HDFS_COMMAND="${HADOOP_PREFIX:-/usr/lib/hadoop}/share/hadoop/hdfs/bin/hdfs"
-AGITATE_HDFS_SUDO=$(which sudo)
-
-#settings for the verification map reduce job
-VERIFY_OUT=/tmp/continuous_verify
-VERIFY_MAX_MAPS=64
-VERIFY_REDUCERS=64
-SCAN_OFFLINE=false
-#comma separated list of auths to use for verify
-VERIFY_AUTHS=''
-
-#settings related to the batch walker
-# sleep in seconds
-BATCH_WALKER_SLEEP=1800
-BATCH_WALKER_BATCH_SIZE=10000
-BATCH_WALKER_THREADS=8
-
-#settings related to scanners
-# sleep in seconds
-SCANNER_SLEEP_TIME=10
-SCANNER_ENTRIES=5000

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/datanode-agitator.pl
----------------------------------------------------------------------
diff --git a/continuous/datanode-agitator.pl b/continuous/datanode-agitator.pl
deleted file mode 100755
index a98bb66..0000000
--- a/continuous/datanode-agitator.pl
+++ /dev/null
@@ -1,140 +0,0 @@
-#! /usr/bin/env perl
-
-# 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.
-
-
-use POSIX qw(strftime);
-use Cwd qw();
-
-if(scalar(@ARGV) != 5 && scalar(@ARGV) != 3){
-  print "Usage : datanode-agitator.pl <min sleep before kill in minutes>[:max sleep before kill in minutes] <min sleep before restart in minutes>[:max sleep before restart in minutes] HADOOP_PREFIX [<min kill> <max kill>]\n";
-  exit(1);
-}
-
-my $ACCUMULO_HOME;
-if( defined $ENV{'ACCUMULO_HOME'} ){
-  $ACCUMULO_HOME = $ENV{'ACCUMULO_HOME'};
-} else {
-  $cwd=Cwd::cwd();
-  $ACCUMULO_HOME=$cwd . '/../../..';
-}
-$HADOOP_PREFIX=$ARGV[2];
-
-print "ACCUMULO_HOME=$ACCUMULO_HOME\n";
-print "HADOOP_PREFIX=$HADOOP_PREFIX\n";
-
-@sleeprange1 = split(/:/, $ARGV[0]);
-$sleep1 = $sleeprange1[0];
-
-@sleeprange2 = split(/:/, $ARGV[1]);
-$sleep2 = $sleeprange2[0];
-
-if (scalar(@sleeprange1) > 1) {
-  $sleep1max = $sleeprange1[1] + 1;
-} else {
-  $sleep1max = $sleep1;
-}
-
-if ($sleep1 > $sleep1max) {
-  die("sleep1 > sleep1max $sleep1 > $sleep1max");
-}
-
-if (scalar(@sleeprange2) > 1) {
-  $sleep2max = $sleeprange2[1] + 1;
-} else {
-  $sleep2max = $sleep2;
-}
-
-if($sleep2 > $sleep2max){
-  die("sleep2 > sleep2max $sleep2 > $sleep2max");
-}
-
-if(defined $ENV{'ACCUMULO_CONF_DIR'}){
-  $ACCUMULO_CONF_DIR = $ENV{'ACCUMULO_CONF_DIR'};
-}else{
-  $ACCUMULO_CONF_DIR = $ACCUMULO_HOME . '/conf';
-}
-
-if(scalar(@ARGV) == 5){
-  $minKill = $ARGV[3];
-  $maxKill = $ARGV[4];
-}else{
-  $minKill = 1;
-  $maxKill = 1;
-}
-
-if($minKill > $maxKill){
-  die("minKill > maxKill $minKill > $maxKill");
-}
-
-@tserversRaw = `cat $ACCUMULO_CONF_DIR/tservers`;
-chomp(@tserversRaw);
-
-for $tserver (@tserversRaw){
-  if($tserver eq "" || substr($tserver,0,1) eq "#"){
-    next;
-  }
-
-  push(@tservers, $tserver);
-}
-
-
-if(scalar(@tservers) < $maxKill){
-  print STDERR "WARN setting maxKill to ".scalar(@tservers)."\n";
-  $maxKill = scalar(@tservers);
-}
-
-if ($minKill > $maxKill){
-  print STDERR "WARN setting minKill to equal maxKill\n";
-  $minKill = $maxKill;
-}
-
-while(1){
-
-  $numToKill = int(rand($maxKill - $minKill + 1)) + $minKill;
-  %killed = ();
-  $server = "";
-
-  for($i = 0; $i < $numToKill; $i++){
-    while($server eq "" || $killed{$server} != undef){
-      $index = int(rand(scalar(@tservers)));
-      $server = $tservers[$index];
-    }
-
-    $killed{$server} = 1;
-
-    $t = strftime "%Y%m%d %H:%M:%S", localtime;
-
-    print STDERR "$t Killing datanode on $server\n";
-    system("ssh $server \"pkill -9 -f '[p]roc_datanode'\"");
-  }
-
-  $nextsleep2 = int(rand($sleep2max - $sleep2)) + $sleep2;
-  sleep($nextsleep2 * 60);
-
-  foreach $restart (keys %killed) {
-
-    $t = strftime "%Y%m%d %H:%M:%S", localtime;
-
-    print STDERR "$t Starting datanode on $restart\n";
-    # We can just start as we're the HDFS user
-    system("ssh $restart '$HADOOP_PREFIX/sbin/hadoop-daemon.sh start datanode'");
-  }
-
-  $nextsleep1 = int(rand($sleep1max - $sleep1)) + $sleep1;
-  sleep($nextsleep1 * 60);
-}
-

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/hdfs-agitator.pl
----------------------------------------------------------------------
diff --git a/continuous/hdfs-agitator.pl b/continuous/hdfs-agitator.pl
deleted file mode 100755
index 85eab32..0000000
--- a/continuous/hdfs-agitator.pl
+++ /dev/null
@@ -1,217 +0,0 @@
-#! /usr/bin/env perl
-
-# 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.
-
-use strict;
-use warnings;
-use POSIX qw(strftime);
-use Getopt::Long;
-use Pod::Usage;
-
-my $help = 0;
-my $man = 0;
-my $sleep = 10;
-my $superuser = 'hdfs';
-my $hdfsCmd;
-if( defined $ENV{'HADOOP_PREFIX'} ){
-  $hdfsCmd = $ENV{'HADOOP_PREFIX'} . '/share/hadoop/hdfs/bin/hdfs';
-}
-my $sudo;
-my $nameservice;
-
-GetOptions('help|?' => \$help, 'man' => \$man, 'sleep=i' => \$sleep, 'nameservice=s' => \$nameservice, 'superuser=s' => \$superuser, 'hdfs-cmd=s' => \$hdfsCmd, 'sudo:s' => \$sudo) or pod2usage(2);
-pod2usage(-exitval => 0, -verbose => 1) if $help;
-pod2usage(-exitval => 0, -verbose => 2) if $man;
-pod2usage(-exitval => 1, -verbose => 1, -message => '$HADOOP_PREFIX not defined and no hdfs-cmd given. please use --hdfs-cmd to specify where your hdfs cli is.') if not defined $hdfsCmd;
-pod2usage(-exitval => 1, -verbose => 1, -message => "Your specified hdfs cli '$hdfsCmd' is not executable.") if not -x $hdfsCmd;
-if( defined $sudo and "" eq $sudo ){
-  $sudo = `which sudo`;
-  pod2usage(-exitval => 1, -verbose => 1, -message => "Error attempting to find the sudo command, please specify it with --sudo /path/to/sudo") if 0 != $?;
-  chomp($sudo);
-}
-if( defined $sudo ){
-  pod2usage(-exitval => 1, -verbose => 1, -message => "Your specified sudo command '$sudo' is not executable.") if not -x $sudo;
-}
-
-my $needsudo = defined $sudo;
-my $haadmin = "$hdfsCmd haadmin";
-if($needsudo) {
-  $haadmin = "$sudo -u $superuser $haadmin";
-  print STDERR "Starting HDFS agitator, configured to fail over every $sleep minutes. will run hdfs command '$hdfsCmd' as user '$superuser' via '$sudo'.\n";
-} else {
-  print STDERR "Starting HDFS agitator, configured to fail over every $sleep minutes. will run hdfs command '$hdfsCmd' as the current user.\n";
-}
-while(1){
-  sleep($sleep * 60);
-  my $t = strftime "%Y%m%d %H:%M:%S", localtime;
-  my @failServices;
-  if( defined $nameservice ){
-    @failServices = ($nameservice);
-  } else {
-    my $nameservicesRaw = `$hdfsCmd getconf -confKey dfs.nameservices`;
-    if(0 != $?) {
-      print STDERR "$t HDFS CLI failed. please see --help to set it correctly\n";
-      exit(1);
-    }
-    chomp($nameservicesRaw);
-    my @nameservices = split(/,/, $nameservicesRaw);
-    if(1 > scalar(@nameservices)) {
-      print STDERR "$t No HDFS NameServices found. Are you sure you're running in HA?\n";
-      exit(1);
-    }
-    if(rand(1) < .5){
-      my $serviceToFail = $nameservices[int(rand(scalar(@nameservices)))];
-      print STDERR "$t Failing over nameservice $serviceToFail\n";
-      @failServices = ($serviceToFail);
-    } else {
-      print STDERR "$t Failing over all nameservices\n";
-      @failServices = @nameservices;
-    }
-  }
-  for my $toFail (@failServices){
-    my $namenodesRaw = `$hdfsCmd getconf -confKey dfs.ha.namenodes.$toFail`;
-    if(0 != $?) {
-      print STDERR "$t HDFS CLI failed to look up namenodes in service $toFail.\n";
-      exit(1);
-    }
-    chomp($namenodesRaw);
-    my @namenodes = split(/,/, $namenodesRaw);
-    if(2 > scalar(@namenodes)) {
-      print STDERR "$t WARN NameService $toFail does not have at least 2 namenodes according to the HDFS configuration, skipping.\n";
-      next;
-    }
-    my $active;
-    for my $namenode (@namenodes){
-      my $status = `$haadmin -ns $toFail -getServiceState $namenode`;
-      if(0 != $?) {
-        if($needsudo) {
-          print STDERR "$t WARN Error while attempting to get the service state of $toFail :: $namenode\n";
-          $status = 'error';
-        } else {
-          print STDERR "$t WARN Current user may not run the HDFS haadmin utility, attempting to sudo to the $superuser user.\n";
-          $needsudo = 1;
-          if(not defined $sudo) {
-            $sudo = `which sudo`;
-            pod2usage(-exitval => 1, -verbose => 1, -message => "Error attempting to find the sudo command, please specify it with --sudo") if 0 != $?;
-            chomp($sudo);
-            pod2usage(-exitval => 1, -verbose => 1, -message => "The sudo command '$sudo' is not executable. please specify sudo with --sudo") if not -x $sudo;
-          }
-          $haadmin = "$sudo -u $superuser $haadmin";
-          redo;
-        }
-      }
-      chomp($status);
-      if( 'active' eq $status ){
-        $active = $namenode;
-        last;
-      }
-    }
-    if( defined $active ){
-      my @standby = grep { $_ ne $active } @namenodes;
-      my $newActive = $standby[int(rand(scalar(@standby)))];
-      print STDERR "$t Transitioning nameservice $toFail from $active to $newActive\n";
-      my $cmd = "$haadmin -ns $toFail -failover $active $newActive";
-      print "$t $cmd\n";
-      system($cmd);
-    } else {
-      my $newActive = $namenodes[int(rand(scalar(@namenodes)))];
-      print STDERR "$t WARN nameservice $toFail did not have an active namenode. Transitioning a random namenode to active. This will fail if HDFS is configured for automatic failover.\n";
-      my $cmd = "$haadmin -ns $toFail -transitionToActive $newActive";
-      print "$t $cmd\n";
-      system($cmd);
-    }
-  }
-}
-__END__
-
-=head1 NAME
-
-hdfs-agitator - causes HDFS to failover
-
-=head1 DESCRIPTION
-
-Sleeps for a configurable amount of time, then causes a NameNode failover in one
-or more HDFS NameServices. If a given NameService does not have an Active
-NameNode when it comes time to failover, a random standby is promoted.
-
-Only works on HDFS versions that support HA configurations and the haadmin
-command. In order to function, the user running this script must be able to
-use the haadmin command. This requires access to an HDFS superuser. By default,
-it will attempt to sudo to perform calls.
-
-=head1 SYNOPSIS
-
-hdfs-agitator [options]
-
-  Options:
-    --help         Brief help message
-    --man          Full documentation
-    --sleep        Time to sleep between failovers in minutes. Default 10
-    --superuser    HDFS superuser. Default 'hdfs'
-    --hdfs-cmd     hdfs command path. Default '$HADOOP_PREFIX/share/hadoop/hdfs/bin/hdfs'
-    --nameservice  Limit failovers to specified nameservice. Default all nameservices
-    --sudo         command to call to sudo to the HDFS superuser. Default 'sudo' if needed.
-
-=head1 OPTIONS
-
-=over 8
-
-=item B<--sleep>
-
-Sleep the given number of minutes between attempts to fail over nameservices.
-
-=item B<--nameservice>
-
-Limit failover attempts to the given nameservice. By default, we attempt ot list
-all known nameservices and choose either one or all of them to failover in a
-given cycle.
-
-=item B<--superuser>
-
-An HDFS superuser capable of running the haadmin command. Defaults to "hdfs".
-
-=item B<--hdfs-cmd>
-
-Path to the HDFS cli. Will be used both for non-administrative commands (e.g.
-listing the nameservices and serviceids in a given nameservice) and admin-only
-actions such as checking status and failing over.
-
-Defaults to using $HADOOP_PREFIX.
-
-=item B<--sudo>
-
-Command to allow us to act as the given HDFS superuser. By default we assume the current user
-can run HDFS administrative commands. When this argument is specified we will instead attempt
-to use the HDFS superuser instead. If given an argument, it will be called like
-sudo, i.e. "sudo -u $superuser $cmd". Defaults to "sudo" on the shell's path.
-
-=back
-
-=head1 SEE ALSO
-
-See the Apache Hadoop documentation on configuring HDFS HA
-
-=over 8
-
-=item B<HA with QJM>
-
-http://hadoop.apache.org/docs/r2.2.0/hadoop-yarn/hadoop-yarn-site/HDFSHighAvailabilityWithQJM.html#Administrative_commands
-
-=item B<HA with NFS>
-
-http://hadoop.apache.org/docs/r2.2.0/hadoop-yarn/hadoop-yarn-site/HDFSHighAvailabilityWithNFS.html#Administrative_commands
-
-=back

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/ingesters.txt.example
----------------------------------------------------------------------
diff --git a/continuous/ingesters.txt.example b/continuous/ingesters.txt.example
deleted file mode 100644
index b66d790..0000000
--- a/continuous/ingesters.txt.example
+++ /dev/null
@@ -1,17 +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.
-
-host1
-host2

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/master-agitator.pl
----------------------------------------------------------------------
diff --git a/continuous/master-agitator.pl b/continuous/master-agitator.pl
deleted file mode 100755
index d87f17e..0000000
--- a/continuous/master-agitator.pl
+++ /dev/null
@@ -1,92 +0,0 @@
-#! /usr/bin/env perl
-
-# 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.
-
-
-use POSIX qw(strftime);
-use Cwd qw();
-
-if(scalar(@ARGV) != 2){
-	print "Usage : master-agitator.pl <sleep before kill in minutes> <sleep before start in minutes>\n";
-	exit(1);
-}
-
-my $ACCUMULO_HOME;
-if( defined $ENV{'ACCUMULO_HOME'} ){
-  $ACCUMULO_HOME = $ENV{'ACCUMULO_HOME'};
-} else {
-  $cwd=Cwd::cwd();
-  $ACCUMULO_HOME=$cwd . '/../../..';
-}
-
-if(defined $ENV{'ACCUMULO_CONF_DIR'}){
-        $ACCUMULO_CONF_DIR = $ENV{'ACCUMULO_CONF_DIR'};
-}else{
-	$ACCUMULO_CONF_DIR = $ACCUMULO_HOME . '/conf';
-}
-
-$sleep1 = $ARGV[0];
-$sleep2 = $ARGV[1];
-
-@mastersRaw = `cat $ACCUMULO_CONF_DIR/masters`;
-chomp(@mastersRaw);
-
-for $master (@mastersRaw){
-	if($master eq "" || substr($master,0,1) eq "#"){
-		next;
-	}
-
-	push(@masters, $master);
-}
-
-
-while(1){
-	sleep($sleep1 * 60);
-	$t = strftime "%Y%m%d %H:%M:%S", localtime;
-	if(rand(1) < .5){
-		$masterNodeToWack = $masters[int(rand(scalar(@masters)))];
-		print STDERR "$t Killing master on $masterNodeToWack\n";
-		$cmd = "ssh $masterNodeToWack \"pkill -f '[ ]org.apache.accumulo.start.*master'\"";
-		print "$t $cmd\n";
-		system($cmd);
-	}else{
-		print STDERR "$t Killing all masters\n";
-		$cmd = "pssh -h $ACCUMULO_CONF_DIR/masters \"pkill -f '[ ]org.apache.accumulo.start.*master'\" < /dev/null";
-		print "$t $cmd\n";
-		system($cmd);
-
-		$file = '';
-		if (-e "$ACCUMULO_CONF_DIR/gc") {
-			$file = 'gc';
-		} else {
-			$file = 'masters';
-		}
-
-		$cmd = "pssh -h $ACCUMULO_CONF_DIR/$file \"pkill -f '[ ]org.apache.accumulo.start.*gc'\" < /dev/null";
-		print "$t $cmd\n";
-		system($cmd);
-	}
-
-	sleep($sleep2 * 60);
-	$t = strftime "%Y%m%d %H:%M:%S", localtime;
-	print STDERR "$t Running start-all\n";
-
-	$cmd = "pssh -h $ACCUMULO_CONF_DIR/masters \"$ACCUMULO_HOME/bin/accumulo-service master start\" < /dev/null";
-	print "$t $cmd\n";
-	system($cmd);
-}
-
-

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/report.pl
----------------------------------------------------------------------
diff --git a/continuous/report.pl b/continuous/report.pl
deleted file mode 100755
index d1902b6..0000000
--- a/continuous/report.pl
+++ /dev/null
@@ -1,120 +0,0 @@
-#! /usr/bin/env perl
-
-# 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.
-
-
-use POSIX qw(strftime);
-
-if(scalar(@ARGV) == 2 && $ARGV[0] eq "-bt"){
-	$BIN_TIME=$ARGV[1];
-}elsif(scalar(@ARGV) == 0){
-	$BIN_TIME=900;
-}else{
-	print "Usage : report.pl [-bt <bin time>]\n";
-	exit;
-}
-
-
-$LOG_DIR = "logs";
-$ACCUMULO_HOME="../../..";
-$REPORT_DIR = strftime "report_%Y%m%d%H%M", localtime;
-
-mkdir("$REPORT_DIR");
-
-open (HTML, ">$REPORT_DIR/report.html");
-
-print HTML "<html><body>\n";
-
-$misCount = `grep MIS $LOG_DIR/*_walk.err | wc -l`;
-
-if($misCount > 0){
-	print HTML "<HR width=50% size=4>\n";
-	print HTML "<center><P><B color=red>WARNING : The walkers saw missing nodes, this should not happen</B><P></center>\n";
-	print HTML "<HR width=50% size=4>\n";
-}
-
-plot("cat $LOG_DIR/*_stats.out", $BIN_TIME, 0, 2, "AVG", "entries", "Entries over time");
-plot("cat $LOG_DIR/*_stats.out", $BIN_TIME, 0, 3, "AMM", "ingest_rate", "Ingest rate over time");
-plot("egrep 'SRQ|FSR' $LOG_DIR/*_walk.out", $BIN_TIME, 1, 3, "AMM", "query_latency", "Row lookup latency (in milliseconds) over time");
-plot("egrep 'SRQ|FSR' $LOG_DIR/*_walk.out", 3600, 1, 3, "COUNT", "query_count", "# rows looked up in each hour");
-plot("grep 'BRQ' $LOG_DIR/*_batch_walk.out", $BIN_TIME, 1, 5, "AMM", "batch_walk_rate", "batch walkers average lookup rate" );
-plot("cat $LOG_DIR/*_stats.out", $BIN_TIME, 0, 10, "AVG", "tablets", "Table tablets online over time");
-plot("cat $LOG_DIR/*_stats.out", $BIN_TIME, 0, 25, "AMM_HACK1", "files_per_tablet", "Files per tablet");
-plot("cat $LOG_DIR/*_stats.out", $BIN_TIME, 0, 1, "AVG", "tservers", "Tablet servers over time");
-plot("cat $LOG_DIR/*_stats.out", $BIN_TIME, 0, 11, "AVG", "du", "HDFS usage over time");
-plot("cat $LOG_DIR/*_stats.out", $BIN_TIME, 0, 12, "AVG", "dirs", "HDFS # dirs over time");
-plot("cat $LOG_DIR/*_stats.out", $BIN_TIME, 0, 13, "AVG", "files", "HDFS # files over time");
-plot("cat $LOG_DIR/*_stats.out", $BIN_TIME, 0, 17, "AVG", "maps", "# map task over time");
-plot("cat $LOG_DIR/*_stats.out", $BIN_TIME, 0, 19, "AVG", "reduces", "# reduce task over time");
-
-print HTML "<P><h2>Config</h2>\n";
-print HTML "<UL>\n";
-for $config_file (glob("$LOG_DIR/*_config.out")){
-	@path = split(/\//,$config_file);
-        $file_name = $path[$path - 1];
-	system("cp $config_file $REPORT_DIR/$file_name");
-	print HTML "<li><a href='$file_name'>$file_name</a>\n";
-}
-print HTML "</UL>\n";
-
-
-print HTML "<P><h2>Lookup times histogram</h2>\n";
-print HTML "<pre>\n";
-print HTML `cat $LOG_DIR/*_walk.out | $ACCUMULO_HOME/bin/accumulo org.apache.accumulo.test.continuous.PrintScanTimeHistogram`;
-print HTML "</pre>\n";
-
-print HTML "</body></html>\n";
-close(HTML);
-
-sub plot {
-	my $cmd = shift(@_);
-	my $period = shift(@_);
-	my $time_col = shift(@_);
-	my $data_col = shift(@_);
-	my $op = shift(@_);
-	my $output = shift(@_);
-	my $title = shift(@_);
-
-	system("$cmd | $ACCUMULO_HOME/bin/accumulo org.apache.accumulo.test.continuous.TimeBinner --period $period --timeColumn $time_col --dataColumn $data_col --operation $op --dateFormat MM/dd/yy-HH:mm:ss > $REPORT_DIR/$output.dat");
-	gnuplot("$REPORT_DIR/$output.dat", "$REPORT_DIR/$output.png", $op eq "AMM" || $op eq "AMM_HACK1");
-
-	print HTML "<P><h2>$title</h2><img src='$output.png'>\n";
-}
-
-sub gnuplot {
-	my $input = shift(@_);
-	my $output = shift(@_);
-	my $yerr = shift(@_);
-
-	open(GNUP, "|gnuplot > $output");	
-
-	print GNUP "set xdata time\n";
-	print GNUP "set timefmt \"%m/%d/%y-%H:%M:%S\"\n";
-	print GNUP "set format x \"%m/%d\"\n";
-	print GNUP "set offsets 1,1,1,1\n";
-	print GNUP "set size 1.25,1.25\n";
-	print GNUP "set terminal png\n";
-	if($yerr){
-		print GNUP "plot \"$input\" using 1:2:3:4 with yerrorlines\n";
-	}else{
-		print GNUP "plot \"$input\" using 1:2\n";
-	}
-
-	close(GNUP);
-}
-	
-
-

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/run-moru.sh
----------------------------------------------------------------------
diff --git a/continuous/run-moru.sh b/continuous/run-moru.sh
deleted file mode 100755
index 3c73ddb..0000000
--- a/continuous/run-moru.sh
+++ /dev/null
@@ -1,37 +0,0 @@
-#! /usr/bin/env bash
-
-# 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.
-
-
-# Start: Resolve Script Directory
-SOURCE="${BASH_SOURCE[0]}"
-while [[ -h "${SOURCE}" ]]; do # resolve $SOURCE until the file is no longer a symlink
-   bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-   SOURCE="$(readlink "${SOURCE}")"
-   [[ "${SOURCE}" != /* ]] && SOURCE="${bin}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located
-done
-bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-script=$( basename "${SOURCE}" )
-# Stop: Resolve Script Directory
-
-CONTINUOUS_CONF_DIR=${CONTINUOUS_CONF_DIR:-${bin}}
-. $CONTINUOUS_CONF_DIR/continuous-env.sh
-. $ACCUMULO_CONF_DIR/accumulo-env.sh
-
-SERVER_LIBJAR="$ACCUMULO_HOME/lib/accumulo-test.jar"
-
-"$ACCUMULO_HOME/contrib/tool.sh" "$SERVER_LIBJAR" org.apache.accumulo.test.continuous.ContinuousMoru -libjars "$SERVER_LIBJAR" -i "$INSTANCE_NAME" -z "$ZOO_KEEPERS" -u "$USER" -p "$PASS" --table "$TABLE" --min "$MIN" --max "$MAX" --maxColF "$MAX_CF" --maxColQ "$MAX_CQ" --batchMemory "$MAX_MEM" --batchLatency "$MAX_LATENCY" --batchThreads "$NUM_THREADS" --maxMappers "$VERIFY_MAX_MAPS"
-

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/run-verify.sh
----------------------------------------------------------------------
diff --git a/continuous/run-verify.sh b/continuous/run-verify.sh
deleted file mode 100755
index aa56643..0000000
--- a/continuous/run-verify.sh
+++ /dev/null
@@ -1,42 +0,0 @@
-#! /usr/bin/env bash
-
-# 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.
-
-
-# Start: Resolve Script Directory
-SOURCE="${BASH_SOURCE[0]}"
-while [[ -h "${SOURCE}" ]]; do # resolve $SOURCE until the file is no longer a symlink
-   bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-   SOURCE=$(readlink "${SOURCE}")
-   [[ "${SOURCE}" != /* ]] && SOURCE="${bin}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located
-done
-bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-script=$( basename "${SOURCE}" )
-# Stop: Resolve Script Directory
-
-CONTINUOUS_CONF_DIR=${CONTINUOUS_CONF_DIR:-${bin}}
-. $CONTINUOUS_CONF_DIR/continuous-env.sh
-. $ACCUMULO_CONF_DIR/accumulo-env.sh
-
-SERVER_LIBJAR="$ACCUMULO_HOME/lib/accumulo-test.jar"
-
-AUTH_OPT="";
-[[ -n $VERIFY_AUTHS ]] && AUTH_OPT="--auths $VERIFY_AUTHS"
-
-SCAN_OPT=--offline
-[[ $SCAN_OFFLINE == false ]] && SCAN_OPT=
-
-"$ACCUMULO_HOME/contrib/tool.sh" "$SERVER_LIBJAR" org.apache.accumulo.test.continuous.ContinuousVerify -Dmapreduce.job.reduce.slowstart.completedmaps=0.95 -libjars "$SERVER_LIBJAR" "$AUTH_OPT" -i "$INSTANCE_NAME" -z "$ZOO_KEEPERS" -u "$USER" -p "$PASS" --table "$TABLE" --output "$VERIFY_OUT" --maxMappers "$VERIFY_MAX_MAPS" --reducers "$VERIFY_REDUCERS" "$SCAN_OPT"

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/scanners.txt.example
----------------------------------------------------------------------
diff --git a/continuous/scanners.txt.example b/continuous/scanners.txt.example
deleted file mode 100644
index 63fb8bb..0000000
--- a/continuous/scanners.txt.example
+++ /dev/null
@@ -1,16 +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.
-
-localhost

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/start-agitator.sh
----------------------------------------------------------------------
diff --git a/continuous/start-agitator.sh b/continuous/start-agitator.sh
deleted file mode 100755
index a44cd83..0000000
--- a/continuous/start-agitator.sh
+++ /dev/null
@@ -1,72 +0,0 @@
-#! /usr/bin/env bash
-
-# 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.
-
-# Start: Resolve Script Directory
-SOURCE="${BASH_SOURCE[0]}"
-while [[ -h "${SOURCE}" ]]; do # resolve $SOURCE until the file is no longer a symlink
-   bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-   SOURCE=$(readlink "${SOURCE}")
-   [[ "${SOURCE}" != /* ]] && SOURCE="${bin}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located
-done
-bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-script=$( basename "${SOURCE}" )
-# Stop: Resolve Script Directory
-
-CONTINUOUS_CONF_DIR=${CONTINUOUS_CONF_DIR:-${bin}}
-. "$CONTINUOUS_CONF_DIR/continuous-env.sh"
-
-mkdir -p "$CONTINUOUS_LOG_DIR"
-
-LOG_BASE="${CONTINUOUS_LOG_DIR}/$(date +%Y%m%d%H%M%S)_$(hostname)"
-
-# Start agitators for datanodes, tservers, and the master
-[[ -n $AGITATOR_USER ]] || AGITATOR_USER=$(whoami)
-if [[ $AGITATOR_USER == root ]];  then
-  echo "Running master-agitator and tserver-agitator as $ACCUMULO_USER using su. Running datanode-agitator as $HDFS_USER using su."
-
-  # Change to the correct user if started as root
-  su -c "nohup ${bin}/master-agitator.pl $MASTER_KILL_SLEEP_TIME $MASTER_RESTART_SLEEP_TIME >${LOG_BASE}_master-agitator.out 2>${LOG_BASE}_master-agitator.err &" -m - "$ACCUMULO_USER"
-
-  su -c "nohup ${bin}/tserver-agitator.pl $TSERVER_KILL_SLEEP_TIME $TSERVER_RESTART_SLEEP_TIME $TSERVER_MIN_KILL $TSERVER_MAX_KILL >${LOG_BASE}_tserver-agitator.out 2>${LOG_BASE}_tserver-agitator.err &" -m - "$ACCUMULO_USER"
-
-  su -c "nohup ${bin}/datanode-agitator.pl $DATANODE_KILL_SLEEP_TIME $DATANODE_RESTART_SLEEP_TIME $HADOOP_PREFIX $DATANODE_MIN_KILL $DATANODE_MAX_KILL >${LOG_BASE}_datanode-agitator.out 2>${LOG_BASE}_datanode-agitator.err &" -m - "$HDFS_USER"
-
-elif [[ $AGITATOR_USER == "$ACCUMULO_USER" ]]; then
-  echo "Running master-agitator and tserver-agitator as $AGITATOR_USER Running datanode-agitator as $HDFS_USER using sudo."
-  # Just run the master-agitator if we're the accumulo user
-  nohup "${bin}/master-agitator.pl" "$MASTER_KILL_SLEEP_TIME" "$MASTER_RESTART_SLEEP_TIME" >"${LOG_BASE}_master-agitator.out" 2>"${LOG_BASE}_master-agitator.err" &
-
-  nohup "${bin}/tserver-agitator.pl" "$TSERVER_KILL_SLEEP_TIME" "$TSERVER_RESTART_SLEEP_TIME" "$TSERVER_MIN_KILL" "$TSERVER_MAX_KILL" >"${LOG_BASE}_tserver-agitator.out" 2>"${LOG_BASE}_tserver-agitator.err" &
-
-  sudo -u "$HDFS_USER" nohup "${bin}/datanode-agitator.pl" "$DATANODE_KILL_SLEEP_TIME" "$DATANODE_RESTART_SLEEP_TIME" "$HADOOP_PREFIX" "$DATANODE_MIN_KILL" "$DATANODE_MAX_KILL" >"${LOG_BASE}_datanode-agitator.out" 2>"${LOG_BASE}_datanode-agitator.err" &
-
-else
-  echo "Running master-agitator and tserver-agitator as $ACCUMULO_USER using sudo. Running datanode-agitator as $HDFS_USER using sudo."
-
-  # Not root, and not the accumulo user, hope you can sudo to it
-  sudo -u "$ACCUMULO_USER" "nohup ${bin}/master-agitator.pl $MASTER_KILL_SLEEP_TIME $MASTER_RESTART_SLEEP_TIME >${LOG_BASE}_master-agitator.out 2>${LOG_BASE}_master-agitator.err &"
-
-  sudo -u "$ACCUMULO_USER" "nohup ${bin}/tserver-agitator.pl $TSERVER_KILL_SLEEP_TIME $TSERVER_RESTART_SLEEP_TIME $TSERVER_MIN_KILL $TSERVER_MAX_KILL >${LOG_BASE}_tserver-agitator.out 2>${LOG_BASE}_tserver-agitator.err &"
-
-  sudo -u "$HDFS_USER" "nohup ${bin}/datanode-agitator.pl $DATANODE_KILL_SLEEP_TIME $DATANODE_RESTART_SLEEP_TIME $HADOOP_PREFIX $DATANODE_MIN_KILL $DATANODE_MAX_KILL >${LOG_BASE}_datanode-agitator.out 2>${LOG_BASE}_datanode-agitator.err &" -m - "$HDFS_USER"
-
-fi
-
-if ${AGITATE_HDFS:-false} ; then
-  AGITATOR_LOG=${LOG_BASE}_hdfs-agitator
-  sudo -u "$AGITATE_HDFS_SUPERUSER" nohup "${bin}/hdfs-agitator.pl" --sleep "${AGITATE_HDFS_SLEEP_TIME}" --hdfs-cmd "${AGITATE_HDFS_COMMAND}" --superuser "${AGITATE_HDFS_SUPERUSER}" >"${AGITATOR_LOG}.out" 2>"${AGITATOR_LOG}.err" &
-fi

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/start-batchwalkers.sh
----------------------------------------------------------------------
diff --git a/continuous/start-batchwalkers.sh b/continuous/start-batchwalkers.sh
deleted file mode 100755
index 7d4efff..0000000
--- a/continuous/start-batchwalkers.sh
+++ /dev/null
@@ -1,42 +0,0 @@
-#! /usr/bin/env bash
-
-# 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.
-
-
-# Start: Resolve Script Directory
-SOURCE="${BASH_SOURCE[0]}"
-while [[ -h "${SOURCE}" ]]; do # resolve $SOURCE until the file is no longer a symlink
-   bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-   SOURCE=$(readlink "${SOURCE}")
-   [[ "${SOURCE}" != /* ]] && SOURCE="${bin}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located
-done
-bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-script=$( basename "${SOURCE}" )
-# Stop: Resolve Script Directory
-
-CONTINUOUS_CONF_DIR=${CONTINUOUS_CONF_DIR:-${bin}}
-. "$CONTINUOUS_CONF_DIR/continuous-env.sh"
-
-DEBUG_OPT=''
-if [[ $DEBUG_BATCH_WALKER == on ]] ; then
-	DEBUG_OPT="--debug $CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_batch_walk.log";
-fi
-
-AUTH_OPT=''
-[[ -n $AUTHS ]] && AUTH_OPT="--auths \"$AUTHS\""
-
-pssh -h "$CONTINUOUS_CONF_DIR/batch_walkers.txt" "mkdir -p $CONTINUOUS_LOG_DIR; nohup $ACCUMULO_HOME/bin/accumulo org.apache.accumulo.test.continuous.ContinuousBatchWalker $DEBUG_OPT $AUTH_OPT -i $INSTANCE_NAME -z $ZOO_KEEPERS -u $USER -p $PASS --table $TABLE --min $MIN --max $MAX --sleep $BATCH_WALKER_SLEEP --numToScan $BATCH_WALKER_BATCH_SIZE --scanThreads $BATCH_WALKER_THREADS >$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_batch_walk.out 2>$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_batch_walk.err &" < /dev/null
-

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/start-ingest.sh
----------------------------------------------------------------------
diff --git a/continuous/start-ingest.sh b/continuous/start-ingest.sh
deleted file mode 100755
index 8cc7d07..0000000
--- a/continuous/start-ingest.sh
+++ /dev/null
@@ -1,45 +0,0 @@
-#! /usr/bin/env bash
-
-# 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.
-
-
-# Start: Resolve Script Directory
-SOURCE="${BASH_SOURCE[0]}"
-while [[ -h "${SOURCE}" ]]; do # resolve $SOURCE until the file is no longer a symlink
-   bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-   SOURCE=$(readlink "${SOURCE}")
-   [[ "${SOURCE}" != /* ]] && SOURCE="${bin}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located
-done
-bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-script=$( basename "${SOURCE}" )
-# Stop: Resolve Script Directory
-
-CONTINUOUS_CONF_DIR=${CONTINUOUS_CONF_DIR:-${bin}}
-. "$CONTINUOUS_CONF_DIR/continuous-env.sh"
-
-DEBUG_OPT=''
-if [[ $DEBUG_INGEST == on ]] ; then
-	DEBUG_OPT="--debug $CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_ingest.log";
-fi
-
-VIS_OPT=''
-[[ -n $VISIBILITIES ]] && VIS_OPT="--visibilities \"$VISIBILITIES\""
-
-CHECKSUM_OPT='--addCheckSum'
-[[ $CHECKSUM == false ]] && CHECKSUM_OPT=''
-
-pssh -h "$CONTINUOUS_CONF_DIR/ingesters.txt" "mkdir -p $CONTINUOUS_LOG_DIR; nohup $ACCUMULO_HOME/bin/accumulo org.apache.accumulo.test.continuous.ContinuousIngest $DEBUG_OPT $VIS_OPT -i $INSTANCE_NAME -z $ZOO_KEEPERS -u $USER -p $PASS --table $TABLE --num $NUM --min $MIN --max $MAX --maxColF $MAX_CF --maxColQ $MAX_CQ --batchMemory $MAX_MEM --batchLatency $MAX_LATENCY --batchThreads $NUM_THREADS $CHECKSUM_OPT >$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_ingest.out 2>$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_ingest.err &" < /dev/null
-

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/fc3ddfc4/continuous/start-scanners.sh
----------------------------------------------------------------------
diff --git a/continuous/start-scanners.sh b/continuous/start-scanners.sh
deleted file mode 100755
index c876768..0000000
--- a/continuous/start-scanners.sh
+++ /dev/null
@@ -1,41 +0,0 @@
-#! /usr/bin/env bash
-
-# 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.
-
-# Start: Resolve Script Directory
-SOURCE="${BASH_SOURCE[0]}"
-while [[ -h "${SOURCE}" ]]; do # resolve $SOURCE until the file is no longer a symlink
-   bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-   SOURCE=$(readlink "${SOURCE}")
-   [[ "${SOURCE}" != /* ]] && SOURCE="${bin}/${SOURCE}" # if $SOURCE was a relative symlink, we need to resolve it relative to the path where the symlink file was located
-done
-bin=$( cd -P "$( dirname "${SOURCE}" )" && pwd )
-script=$( basename "${SOURCE}" )
-# Stop: Resolve Script Directory
-
-CONTINUOUS_CONF_DIR=${CONTINUOUS_CONF_DIR:-${bin}}
-. "$CONTINUOUS_CONF_DIR/continuous-env.sh"
-
-DEBUG_OPT="";
-if [[ "$DEBUG_SCANNER" == "on" ]] ; then
-	DEBUG_OPT="--debug $CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_scanner.log";
-fi
-
-AUTH_OPT="";
-[[ -n "$AUTHS" ]] && AUTH_OPT="--auths \"$AUTHS\""
-
-pssh -h "$CONTINUOUS_CONF_DIR/scanners.txt" "mkdir -p $CONTINUOUS_LOG_DIR; nohup $ACCUMULO_HOME/bin/accumulo org.apache.accumulo.test.continuous.ContinuousScanner $DEBUG_OPT $AUTH_OPT -i $INSTANCE_NAME -z $ZOO_KEEPERS -u $USER -p $PASS --table $TABLE --min $MIN --max $MAX --sleep $SCANNER_SLEEP_TIME --numToScan $SCANNER_ENTRIES >$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_scanner.out 2>$CONTINUOUS_LOG_DIR/\`date +%Y%m%d%H%M%S\`_\`hostname\`_scanner.err &" < /dev/null
-