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/23 22:03:58 UTC

[3/3] accumulo-testing git commit: ACCUMULO-4510 Moved continuous ingest code from Accumulo repo

ACCUMULO-4510 Moved continuous ingest code from Accumulo repo


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

Branch: refs/heads/master
Commit: b81229d780c29369f977696f10656a8d5567b4fd
Parents: c3c8ff1
Author: Mike Walch <mw...@apache.org>
Authored: Wed Jan 11 14:38:26 2017 -0500
Committer: Mike Walch <mw...@apache.org>
Committed: Wed Jan 11 14:38:26 2017 -0500

----------------------------------------------------------------------
 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 +
 .../core/continuous/ContinuousBatchWalker.java  | 175 ++++++++++
 .../core/continuous/ContinuousIngest.java       | 252 +++++++++++++
 .../testing/core/continuous/ContinuousMoru.java | 180 ++++++++++
 .../testing/core/continuous/ContinuousOpts.java |  80 +++++
 .../core/continuous/ContinuousQuery.java        |  74 ++++
 .../core/continuous/ContinuousScanner.java      | 108 ++++++
 .../continuous/ContinuousStatsCollector.java    | 206 +++++++++++
 .../testing/core/continuous/ContinuousUtil.java |  49 +++
 .../core/continuous/ContinuousVerify.java       | 226 ++++++++++++
 .../testing/core/continuous/ContinuousWalk.java | 240 +++++++++++++
 .../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     | 196 +++++++++++
 .../core/continuous/UndefinedAnalyzer.java      | 350 +++++++++++++++++++
 pom.xml                                         |   2 +-
 44 files changed, 4259 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/b81229d7/continuous/README.md
----------------------------------------------------------------------
diff --git a/continuous/README.md b/continuous/README.md
new file mode 100644
index 0000000..31ee4bd
--- /dev/null
+++ b/continuous/README.md
@@ -0,0 +1,103 @@
+<!--
+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/b81229d7/continuous/analyze-missing.pl
----------------------------------------------------------------------
diff --git a/continuous/analyze-missing.pl b/continuous/analyze-missing.pl
new file mode 100755
index 0000000..5cce1b1
--- /dev/null
+++ b/continuous/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/b81229d7/continuous/analyze-missing.sh
----------------------------------------------------------------------
diff --git a/continuous/analyze-missing.sh b/continuous/analyze-missing.sh
new file mode 100755
index 0000000..e2cfbb1
--- /dev/null
+++ b/continuous/analyze-missing.sh
@@ -0,0 +1,23 @@
+#! /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/b81229d7/continuous/batch_walkers.txt.example
----------------------------------------------------------------------
diff --git a/continuous/batch_walkers.txt.example b/continuous/batch_walkers.txt.example
new file mode 100644
index 0000000..63fb8bb
--- /dev/null
+++ b/continuous/batch_walkers.txt.example
@@ -0,0 +1,16 @@
+# 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/b81229d7/continuous/continuous-env.sh.example
----------------------------------------------------------------------
diff --git a/continuous/continuous-env.sh.example b/continuous/continuous-env.sh.example
new file mode 100644
index 0000000..0abd8c3
--- /dev/null
+++ b/continuous/continuous-env.sh.example
@@ -0,0 +1,131 @@
+# 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/b81229d7/continuous/datanode-agitator.pl
----------------------------------------------------------------------
diff --git a/continuous/datanode-agitator.pl b/continuous/datanode-agitator.pl
new file mode 100755
index 0000000..a98bb66
--- /dev/null
+++ b/continuous/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/b81229d7/continuous/hdfs-agitator.pl
----------------------------------------------------------------------
diff --git a/continuous/hdfs-agitator.pl b/continuous/hdfs-agitator.pl
new file mode 100755
index 0000000..85eab32
--- /dev/null
+++ b/continuous/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/b81229d7/continuous/ingesters.txt.example
----------------------------------------------------------------------
diff --git a/continuous/ingesters.txt.example b/continuous/ingesters.txt.example
new file mode 100644
index 0000000..b66d790
--- /dev/null
+++ b/continuous/ingesters.txt.example
@@ -0,0 +1,17 @@
+# 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/b81229d7/continuous/master-agitator.pl
----------------------------------------------------------------------
diff --git a/continuous/master-agitator.pl b/continuous/master-agitator.pl
new file mode 100755
index 0000000..d87f17e
--- /dev/null
+++ b/continuous/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/b81229d7/continuous/report.pl
----------------------------------------------------------------------
diff --git a/continuous/report.pl b/continuous/report.pl
new file mode 100755
index 0000000..d1902b6
--- /dev/null
+++ b/continuous/report.pl
@@ -0,0 +1,120 @@
+#! /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/b81229d7/continuous/run-moru.sh
----------------------------------------------------------------------
diff --git a/continuous/run-moru.sh b/continuous/run-moru.sh
new file mode 100755
index 0000000..3c73ddb
--- /dev/null
+++ b/continuous/run-moru.sh
@@ -0,0 +1,37 @@
+#! /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/b81229d7/continuous/run-verify.sh
----------------------------------------------------------------------
diff --git a/continuous/run-verify.sh b/continuous/run-verify.sh
new file mode 100755
index 0000000..aa56643
--- /dev/null
+++ b/continuous/run-verify.sh
@@ -0,0 +1,42 @@
+#! /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/b81229d7/continuous/scanners.txt.example
----------------------------------------------------------------------
diff --git a/continuous/scanners.txt.example b/continuous/scanners.txt.example
new file mode 100644
index 0000000..63fb8bb
--- /dev/null
+++ b/continuous/scanners.txt.example
@@ -0,0 +1,16 @@
+# 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/b81229d7/continuous/start-agitator.sh
----------------------------------------------------------------------
diff --git a/continuous/start-agitator.sh b/continuous/start-agitator.sh
new file mode 100755
index 0000000..a44cd83
--- /dev/null
+++ b/continuous/start-agitator.sh
@@ -0,0 +1,72 @@
+#! /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/b81229d7/continuous/start-batchwalkers.sh
----------------------------------------------------------------------
diff --git a/continuous/start-batchwalkers.sh b/continuous/start-batchwalkers.sh
new file mode 100755
index 0000000..7d4efff
--- /dev/null
+++ b/continuous/start-batchwalkers.sh
@@ -0,0 +1,42 @@
+#! /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/b81229d7/continuous/start-ingest.sh
----------------------------------------------------------------------
diff --git a/continuous/start-ingest.sh b/continuous/start-ingest.sh
new file mode 100755
index 0000000..8cc7d07
--- /dev/null
+++ b/continuous/start-ingest.sh
@@ -0,0 +1,45 @@
+#! /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/b81229d7/continuous/start-scanners.sh
----------------------------------------------------------------------
diff --git a/continuous/start-scanners.sh b/continuous/start-scanners.sh
new file mode 100755
index 0000000..c876768
--- /dev/null
+++ b/continuous/start-scanners.sh
@@ -0,0 +1,41 @@
+#! /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
+

http://git-wip-us.apache.org/repos/asf/accumulo-testing/blob/b81229d7/continuous/start-stats.sh
----------------------------------------------------------------------
diff --git a/continuous/start-stats.sh b/continuous/start-stats.sh
new file mode 100755
index 0000000..0a90364
--- /dev/null
+++ b/continuous/start-stats.sh
@@ -0,0 +1,49 @@
+#! /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/b81229d7/continuous/start-walkers.sh
----------------------------------------------------------------------
diff --git a/continuous/start-walkers.sh b/continuous/start-walkers.sh
new file mode 100755
index 0000000..d9bbff4
--- /dev/null
+++ b/continuous/start-walkers.sh
@@ -0,0 +1,41 @@
+#! /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/b81229d7/continuous/stop-agitator.sh
----------------------------------------------------------------------
diff --git a/continuous/stop-agitator.sh b/continuous/stop-agitator.sh
new file mode 100755
index 0000000..d8f30e4
--- /dev/null
+++ b/continuous/stop-agitator.sh
@@ -0,0 +1,51 @@
+#! /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/b81229d7/continuous/stop-batchwalkers.sh
----------------------------------------------------------------------
diff --git a/continuous/stop-batchwalkers.sh b/continuous/stop-batchwalkers.sh
new file mode 100755
index 0000000..4696387
--- /dev/null
+++ b/continuous/stop-batchwalkers.sh
@@ -0,0 +1,33 @@
+#! /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/b81229d7/continuous/stop-ingest.sh
----------------------------------------------------------------------
diff --git a/continuous/stop-ingest.sh b/continuous/stop-ingest.sh
new file mode 100755
index 0000000..d159bf7
--- /dev/null
+++ b/continuous/stop-ingest.sh
@@ -0,0 +1,33 @@
+#! /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/b81229d7/continuous/stop-scanners.sh
----------------------------------------------------------------------
diff --git a/continuous/stop-scanners.sh b/continuous/stop-scanners.sh
new file mode 100755
index 0000000..cf927b0
--- /dev/null
+++ b/continuous/stop-scanners.sh
@@ -0,0 +1,33 @@
+#! /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/b81229d7/continuous/stop-stats.sh
----------------------------------------------------------------------
diff --git a/continuous/stop-stats.sh b/continuous/stop-stats.sh
new file mode 100755
index 0000000..9886eec
--- /dev/null
+++ b/continuous/stop-stats.sh
@@ -0,0 +1,33 @@
+#! /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/b81229d7/continuous/stop-walkers.sh
----------------------------------------------------------------------
diff --git a/continuous/stop-walkers.sh b/continuous/stop-walkers.sh
new file mode 100755
index 0000000..2c22cfa
--- /dev/null
+++ b/continuous/stop-walkers.sh
@@ -0,0 +1,33 @@
+#! /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/b81229d7/continuous/tserver-agitator.pl
----------------------------------------------------------------------
diff --git a/continuous/tserver-agitator.pl b/continuous/tserver-agitator.pl
new file mode 100755
index 0000000..0e65a50
--- /dev/null
+++ b/continuous/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("$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/b81229d7/continuous/walkers.txt.example
----------------------------------------------------------------------
diff --git a/continuous/walkers.txt.example b/continuous/walkers.txt.example
new file mode 100644
index 0000000..b59052d
--- /dev/null
+++ b/continuous/walkers.txt.example
@@ -0,0 +1,17 @@
+# 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/b81229d7/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
new file mode 100644
index 0000000..e89f2eb
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/testing/core/continuous/ContinuousBatchWalker.java
@@ -0,0 +1,175 @@
+/*
+ * 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.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+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;
+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.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);
+
+    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);
+
+    while (true) {
+      BatchScanner bs = conn.createBatchScanner(clientOpts.getTableName(), auths, bsOpts.scanThreads);
+      bs.setTimeout(bsOpts.scanTimeout, TimeUnit.MILLISECONDS);
+
+      Set<Text> batch = getBatch(scanner, opts.min, opts.max, scanOpts.scanBatchSize, r);
+      List<Range> ranges = new ArrayList<>(batch.size());
+
+      for (Text row : batch) {
+        ranges.add(new Range(row));
+      }
+
+      runBatchScan(scanOpts.scanBatchSize, bs, batch, ranges);
+
+      sleepUninterruptibly(opts.sleepTime, TimeUnit.MILLISECONDS);
+    }
+
+  }
+
+  private static void runBatchScan(int batchSize, BatchScanner bs, Set<Text> batch, List<Range> ranges) {
+    bs.setRanges(ranges);
+
+    Set<Text> rowsSeen = new HashSet<>();
+
+    int count = 0;
+
+    long t1 = System.currentTimeMillis();
+
+    for (Entry<Key,Value> entry : bs) {
+      ContinuousWalk.validate(entry.getKey(), entry.getValue());
+
+      rowsSeen.add(entry.getKey().getRow());
+
+      addRow(batchSize, entry.getValue());
+
+      count++;
+    }
+    bs.close();
+
+    long t2 = System.currentTimeMillis();
+
+    if (!rowsSeen.equals(batch)) {
+      HashSet<Text> copy1 = new HashSet<>(rowsSeen);
+      HashSet<Text> copy2 = new HashSet<>(batch);
+
+      copy1.removeAll(batch);
+      copy2.removeAll(rowsSeen);
+
+      System.out.printf("DIF %d %d %d%n", t1, copy1.size(), copy2.size());
+      System.err.printf("DIF %d %d %d%n", t1, copy1.size(), copy2.size());
+      System.err.println("Extra seen : " + copy1);
+      System.err.println("Not seen   : " + copy2);
+    } 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) {
+    byte[] val = v.get();
+
+    int offset = ContinuousWalk.getPrevRowOffset(val);
+    if (offset > 1) {
+      Text prevRow = new Text();
+      prevRow.set(val, offset, 16);
+      if (rowsToQuery.size() < 3 * batchSize) {
+        rowsToQuery.add(prevRow);
+      }
+    }
+  }
+
+  private static HashSet<Text> rowsToQuery = new HashSet<>();
+
+  private static Set<Text> getBatch(Scanner scanner, long min, long max, int batchSize, Random r) {
+
+    while (rowsToQuery.size() < batchSize) {
+      byte[] scanStart = ContinuousIngest.genRow(min, max, r);
+      scanner.setRange(new Range(new Text(scanStart), null));
+
+      int count = 0;
+
+      long t1 = System.currentTimeMillis();
+
+      Iterator<Entry<Key,Value>> iter = scanner.iterator();
+      while (iter.hasNext() && rowsToQuery.size() < 3 * batchSize) {
+        Entry<Key,Value> entry = iter.next();
+        ContinuousWalk.validate(entry.getKey(), entry.getValue());
+        addRow(batchSize, entry.getValue());
+        count++;
+      }
+
+      long t2 = System.currentTimeMillis();
+
+      System.out.println("FSB " + t1 + " " + (t2 - t1) + " " + count);
+
+      sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+    }
+
+    HashSet<Text> ret = new HashSet<>();
+
+    Iterator<Text> iter = rowsToQuery.iterator();
+
+    for (int i = 0; i < batchSize; i++) {
+      ret.add(iter.next());
+      iter.remove();
+    }
+
+    return ret;
+  }
+
+}