You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2013/12/21 04:28:37 UTC

[1/9] git commit: ACCUMULO-2067 Use urlencoding method already present in BasicServlet (thanks, [~ecn]).

Updated Branches:
  refs/heads/master 551c6507d -> 2d8a2ecb8


ACCUMULO-2067 Use urlencoding method already present in BasicServlet (thanks, [~ecn]).


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

Branch: refs/heads/master
Commit: 4cf60b651fa86c6ac2a6e9501dbdd2ae5467ff8f
Parents: 6bf68ed
Author: Josh Elser <el...@apache.org>
Authored: Fri Dec 20 13:06:48 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri Dec 20 13:06:48 2013 -0500

----------------------------------------------------------------------
 .../monitor/util/celltypes/TableLinkType.java         | 14 ++------------
 1 file changed, 2 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/4cf60b65/server/monitor/src/main/java/org/apache/accumulo/monitor/util/celltypes/TableLinkType.java
----------------------------------------------------------------------
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/util/celltypes/TableLinkType.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/util/celltypes/TableLinkType.java
index 39a932e..76041d4 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/util/celltypes/TableLinkType.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/util/celltypes/TableLinkType.java
@@ -16,18 +16,14 @@
  */
 package org.apache.accumulo.monitor.util.celltypes;
 
-import java.io.UnsupportedEncodingException;
-import java.net.URLEncoder;
 import java.util.Map;
 
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.monitor.servlets.BasicServlet;
 import org.apache.accumulo.server.client.HdfsZooInstance;
-import org.apache.log4j.Logger;
 
 public class TableLinkType extends CellType<String> {
   
-  private static final Logger log = Logger.getLogger(TableLinkType.class);
   private Map<String,String> tidToNameMap;
   
   public TableLinkType() {
@@ -39,15 +35,9 @@ public class TableLinkType extends CellType<String> {
     if (obj == null)
       return "-";
     String tableId = (String) obj;
-    String encodedTableId = tableId;
     // Encode the tableid we use in the link so we construct a correct url
     // e.g. the root table's id of "+r" would not be interpreted properly
-    try {
-      encodedTableId = URLEncoder.encode(tableId, Constants.UTF8.name());
-    } catch (UnsupportedEncodingException e) {
-      log.error("Could not urlencode tableId: " + encodedTableId);
-    }
-    return String.format("<a href='/tables?t=%s'>%s</a>", encodedTableId, displayName(tableId));
+    return String.format("<a href='/tables?t=%s'>%s</a>", BasicServlet.encode(tableId), displayName(tableId));
   }
   
   private String displayName(String tableId) {


[8/9] git commit: ACCUMULO-2080 made tserver use uuid when checking if a walog to be removed is in use

Posted by kt...@apache.org.
ACCUMULO-2080 made tserver use uuid when checking if a walog to be removed is in use


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

Branch: refs/heads/master
Commit: 3aace980689967414aa2fd390c4e00dff1db11ca
Parents: bed8071
Author: Keith Turner <kt...@apache.org>
Authored: Fri Dec 20 22:12:58 2013 -0500
Committer: Keith Turner <kt...@apache.org>
Committed: Fri Dec 20 22:20:01 2013 -0500

----------------------------------------------------------------------
 .../java/org/apache/accumulo/tserver/Tablet.java  |  4 ++--
 .../org/apache/accumulo/tserver/TabletServer.java | 18 +++++++++++-------
 .../accumulo/tserver/log/TabletServerLogger.java  |  6 +++---
 3 files changed, 16 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/3aace980/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
index d904037..f0fb524 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
@@ -3682,11 +3682,11 @@ public class Tablet {
 
   private Set<DfsLogger> currentLogs = new HashSet<DfsLogger>();
 
-  public Set<String> getCurrentLogs() {
+  public Set<String> getCurrentLogFiles() {
     Set<String> result = new HashSet<String>();
     synchronized (currentLogs) {
       for (DfsLogger log : currentLogs) {
-        result.add(log.toString());
+        result.add(log.getFileName());
       }
     }
     return result;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3aace980/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 322a1a5..d2b01ac 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -2451,20 +2451,24 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       String myname = getClientAddressString();
       myname = myname.replace(':', '+');
       Set<String> loggers = new HashSet<String>();
-      logger.getLoggers(loggers);
+      logger.getLogFiles(loggers);
+      Set<String> loggerUUIDs = new HashSet<String>();
+      for (String logger : loggers)
+        loggerUUIDs.add(new Path(logger).getName());
+
       nextFile: for (String filename : filenames) {
+        String uuid = new Path(filename).getName();
         // skip any log we're currently using
-        for (String logger : loggers) {
-          if (logger.contains(filename))
-            continue nextFile;
-        }
+        if (loggerUUIDs.contains(uuid))
+          continue nextFile;
+
         List<Tablet> onlineTabletsCopy = new ArrayList<Tablet>();
         synchronized (onlineTablets) {
           onlineTabletsCopy.addAll(onlineTablets.values());
         }
         for (Tablet tablet : onlineTabletsCopy) {
-          for (String current : tablet.getCurrentLogs()) {
-            if (current.contains(filename)) {
+          for (String current : tablet.getCurrentLogFiles()) {
+            if (current.contains(uuid)) {
               log.info("Attempted to delete " + filename + " from tablet " + tablet.getExtent());
               continue nextFile;
             }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3aace980/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index 7fd2766..a276a97 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@ -36,9 +36,9 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.tserver.Tablet;
+import org.apache.accumulo.tserver.Tablet.CommitSession;
 import org.apache.accumulo.tserver.TabletMutations;
 import org.apache.accumulo.tserver.TabletServer;
-import org.apache.accumulo.tserver.Tablet.CommitSession;
 import org.apache.accumulo.tserver.log.DfsLogger.LoggerOperation;
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
@@ -161,11 +161,11 @@ public class TabletServerLogger {
     return result[0];
   }
   
-  public void getLoggers(Set<String> loggersOut) {
+  public void getLogFiles(Set<String> loggersOut) {
     logSetLock.readLock().lock();
     try {
       for (DfsLogger logger : loggers) {
-        loggersOut.add(logger.toString());
+        loggersOut.add(logger.getFileName());
       }
     } finally {
       logSetLock.readLock().unlock();


[6/9] git commit: Merge branch '1.5.1-SNAPSHOT' into 1.6.0-SNAPSHOT

Posted by kt...@apache.org.
Merge branch '1.5.1-SNAPSHOT' into 1.6.0-SNAPSHOT


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

Branch: refs/heads/master
Commit: 66c7848a1fa201bd6ce3576d09756aa0681f57d1
Parents: 4de0c1d 0d58654
Author: Josh Elser <el...@apache.org>
Authored: Fri Dec 20 19:29:57 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri Dec 20 19:29:57 2013 -0500

----------------------------------------------------------------------
 conf/examples/1GB/native-standalone/accumulo-env.sh   | 2 +-
 conf/examples/1GB/standalone/accumulo-env.sh          | 2 +-
 conf/examples/2GB/native-standalone/accumulo-env.sh   | 2 +-
 conf/examples/2GB/standalone/accumulo-env.sh          | 2 +-
 conf/examples/3GB/native-standalone/accumulo-env.sh   | 2 +-
 conf/examples/3GB/standalone/accumulo-env.sh          | 2 +-
 conf/examples/512MB/native-standalone/accumulo-env.sh | 2 +-
 conf/examples/512MB/standalone/accumulo-env.sh        | 2 +-
 8 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/66c7848a/conf/examples/1GB/native-standalone/accumulo-env.sh
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/66c7848a/conf/examples/1GB/standalone/accumulo-env.sh
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/66c7848a/conf/examples/2GB/native-standalone/accumulo-env.sh
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/66c7848a/conf/examples/2GB/standalone/accumulo-env.sh
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/66c7848a/conf/examples/3GB/native-standalone/accumulo-env.sh
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/66c7848a/conf/examples/3GB/standalone/accumulo-env.sh
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/66c7848a/conf/examples/512MB/native-standalone/accumulo-env.sh
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/66c7848a/conf/examples/512MB/standalone/accumulo-env.sh
----------------------------------------------------------------------


[2/9] git commit: ACCUMULO-1971 Clean up agitator scripts for clarity and ease of multi-user systems.

Posted by kt...@apache.org.
ACCUMULO-1971 Clean up agitator scripts for clarity and ease of multi-user systems.

Squashed commit of the following:

commit ff62ce94dd72e19386f773d52ab3fc52cf8e2609
Author: Josh Elser <el...@apache.org>
Date:   Fri Dec 20 14:05:58 2013 -0500

    ACCUMULO-1971 Fix the invocation of the agitator scripts to account for the removal of the (unnecessary) user argument.

commit 3c0969856741d6404553974b0a558c318d42a1f8
Author: Josh Elser <el...@apache.org>
Date:   Wed Dec 18 00:33:00 2013 -0500

    ACCUMULO-1971 Change the logic to kill a tserver and datanode each time.

    Previously, when this functionality was in the same perl script, 33% would restart the tserver, 33% would restart the datanode and the
    remaining would restart both. Since we can't reliably determine this, every cycle of the agitator will kill and restart
    a process (datanode and tserver) but they may not be on the same host.

commit 780def9cdc91c65fd9f678f198c20150373d1eec
Author: Josh Elser <el...@apache.org>
Date:   Wed Dec 18 00:32:45 2013 -0500

    ACCUMULO-1971 Remove the unnecessary user arguments.

commit d730c4b7206b5e9a8d847ba0a5bcd8f7ef88cee1
Author: Josh Elser <el...@apache.org>
Date:   Tue Dec 17 23:32:21 2013 -0500

    ACCUMULO-1971 Split up the agitator script into discrete Accumulo and Hadoop components to remove all of the
    user-changing shenanigans.


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

Branch: refs/heads/master
Commit: 9cfdb3c6cd8d1837d31a70007084fc91b682690b
Parents: 30938af
Author: Josh Elser <el...@apache.org>
Authored: Fri Dec 20 18:19:33 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri Dec 20 18:19:33 2013 -0500

----------------------------------------------------------------------
 test/system/continuous/agitator.pl          | 201 -----------------------
 test/system/continuous/datanode-agitator.pl | 131 +++++++++++++++
 test/system/continuous/magitator.pl         |  85 ----------
 test/system/continuous/master-agitator.pl   |  85 ++++++++++
 test/system/continuous/start-agitator.sh    |  50 ++++--
 test/system/continuous/stop-agitator.sh     |  32 +++-
 test/system/continuous/tserver-agitator.pl  | 129 +++++++++++++++
 7 files changed, 413 insertions(+), 300 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/9cfdb3c6/test/system/continuous/agitator.pl
----------------------------------------------------------------------
diff --git a/test/system/continuous/agitator.pl b/test/system/continuous/agitator.pl
deleted file mode 100755
index 49772eb..0000000
--- a/test/system/continuous/agitator.pl
+++ /dev/null
@@ -1,201 +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) != 6 && scalar(@ARGV) != 4){
-  print "Usage : 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] hdfs_user accumulo_user [<min kill> <max kill>]\n";
-  exit(1);
-}
-
-$myself=`whoami`;
-chomp($myself);
-$am_root=($myself eq 'root');
-
-$cwd=Cwd::cwd();
-$ACCUMULO_HOME=$cwd . '/../../..';
-$HADOOP_PREFIX=$ENV{"HADOOP_PREFIX"};
-
-print "Current directory: $cwd\n";
-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';
-}
-
-$HDFS_USER=$ARGV[2];
-$ACCUMULO_USER=$ARGV[3];
-
-$am_hdfs_user=($HDFS_USER eq $myself);
-$am_accumulo_user=($ACCUMULO_USER eq $myself);
-
-if(scalar(@ARGV) == 6){
-  $minKill = $ARGV[4];
-  $maxKill = $ARGV[5];
-}else{
-  $minKill = 1;
-  $maxKill = 1;
-}
-
-if($minKill > $maxKill){
-  die("minKill > maxKill $minKill > $maxKill");
-}
-
-@slavesRaw = `cat $ACCUMULO_CONF_DIR/slaves`;
-chomp(@slavesRaw);
-
-for $slave (@slavesRaw){
-  if($slave eq "" || substr($slave,0,1) eq "#"){
-    next;
-  }
-
-  push(@slaves, $slave);
-}
-
-
-if(scalar(@slaves) < $maxKill){
-  print STDERR "WARN setting maxKill to ".scalar(@slaves)."\n";
-  $maxKill = scalar(@slaves);
-}
-
-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 = "";
-  $kill_tserver = 0;
-  $kill_datanode = 0;
-
-  for($i = 0; $i < $numToKill; $i++){
-    while($server eq "" || $killed{$server} != undef){
-      $index = int(rand(scalar(@slaves)));
-      $server = $slaves[$index];
-    }
-
-    $killed{$server} = 1;
-
-    $t = strftime "%Y%m%d %H:%M:%S", localtime;
-
-    $rn = rand(1);
-    if ($rn <.33) {
-      $kill_tserver = 1;
-      $kill_datanode = 1;
-    } elsif ($rn < .66) {
-      $kill_tserver = 1;
-      $kill_datanode = 0;
-    } else {
-      $kill_tserver = 0;
-      $kill_datanode = 1;
-    }
-
-    print STDERR "$t Killing $server $kill_tserver $kill_datanode\n";
-    if ($kill_tserver) {
-      if ($am_root) {
-        # We're root, switch to the Accumulo user and try to stop gracefully
-        system("su -c '$ACCUMULO_HOME/bin/stop-server.sh $server \"accumulo-start.jar\" tserver KILL' - $ACCUMULO_USER");
-      } elsif ($am_accumulo_user) {
-        # We're the accumulo user, just run the commandj
-        system("$ACCUMULO_HOME/bin/stop-server.sh $server 'accumulo-start.jar' tserver KILL");
-      } else {
-        # We're not the accumulo user, try to use sudo
-        system("sudo -u $ACCUMULO_USER $ACCUMULO_HOME/bin/stop-server.sh $server accumulo-start.jar tserver KILL");
-      }
-    }
-
-    if ($kill_datanode) {
-      if ($am_root) {
-        # We're root, switch to HDFS to ssh and kill the process
-        system("su -c 'ssh $server pkill -9 -f [p]roc_datanode' - $HDFS_USER");
-      } elsif ($am_hdfs_user) {
-        # We're the HDFS user, just kill the process
-        system("ssh $server \"pkill -9 -f '[p]roc_datanode'\"");
-      } else {
-        # We're not the hdfs user, try to use sudo
-        system("sudo -u $HDFS_USER ssh $server pkill -9 -f \'[p]roc_datanode\'");
-      }
-    }
-  }
-
-  $nextsleep2 = int(rand($sleep2max - $sleep2)) + $sleep2;
-  sleep($nextsleep2 * 60);
-  $t = strftime "%Y%m%d %H:%M:%S", localtime;
-  print STDERR "$t Running tup\n";
-  if ($am_root) {
-    # Running as root, su to the accumulo user
-    system("su -c $ACCUMULO_HOME/bin/tup.sh - $ACCUMULO_USER");
-  } elsif ($am_accumulo_user) {
-    # restart the as them as the accumulo user
-    system("$ACCUMULO_HOME/bin/tup.sh");
-  } else {
-    # Not the accumulo user, try to sudo to the accumulo user
-    system("sudo -u $ACCUMULO_USER $ACCUMULO_HOME/bin/tup.sh");
-  }
-
-  if ($kill_datanode) {
-    print STDERR "$t Starting datanode on $server\n";
-    if ($am_root) {
-      # We're root, switch to the HDFS user
-      system("ssh $server 'su -c \"$HADOOP_PREFIX/sbin/hadoop-daemon.sh start datanode\" - $HDFS_USER 2>/dev/null 1>/dev/null'");
-    } elsif ($am_hdfs_user) {
-      # We can just start as we're the HDFS user
-      system("ssh $server '$HADOOP_PREFIX/sbin/hadoop-daemon.sh start datanode'");
-    } else {
-      # Not the HDFS user, have to try sudo
-      system("sudo -u $HDFS_USER ssh $server $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/blob/9cfdb3c6/test/system/continuous/datanode-agitator.pl
----------------------------------------------------------------------
diff --git a/test/system/continuous/datanode-agitator.pl b/test/system/continuous/datanode-agitator.pl
new file mode 100755
index 0000000..f823593
--- /dev/null
+++ b/test/system/continuous/datanode-agitator.pl
@@ -0,0 +1,131 @@
+#! /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);
+}
+
+$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");
+}
+
+@slavesRaw = `cat $ACCUMULO_CONF_DIR/slaves`;
+chomp(@slavesRaw);
+
+for $slave (@slavesRaw){
+  if($slave eq "" || substr($slave,0,1) eq "#"){
+    next;
+  }
+
+  push(@slaves, $slave);
+}
+
+
+if(scalar(@slaves) < $maxKill){
+  print STDERR "WARN setting maxKill to ".scalar(@slaves)."\n";
+  $maxKill = scalar(@slaves);
+}
+
+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(@slaves)));
+      $server = $slaves[$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);
+  $t = strftime "%Y%m%d %H:%M:%S", localtime;
+
+  print STDERR "$t Starting datanode on $server\n";
+  # We can just start as we're the HDFS user
+  system("ssh $server '$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/blob/9cfdb3c6/test/system/continuous/magitator.pl
----------------------------------------------------------------------
diff --git a/test/system/continuous/magitator.pl b/test/system/continuous/magitator.pl
deleted file mode 100755
index a40bfb2..0000000
--- a/test/system/continuous/magitator.pl
+++ /dev/null
@@ -1,85 +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){
-	print "Usage : magitator.pl <sleep before kill in minutes> <sleep before start-all in minutes>\n";
-	exit(1);
-}
-
-$ACCUMULO_HOME="../../..";
-
-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 = "$ACCUMULO_HOME/bin/start-all.sh --notSlaves";
-	print "$t $cmd\n";
-	system($cmd);
-}
-
-

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9cfdb3c6/test/system/continuous/master-agitator.pl
----------------------------------------------------------------------
diff --git a/test/system/continuous/master-agitator.pl b/test/system/continuous/master-agitator.pl
new file mode 100755
index 0000000..a40bfb2
--- /dev/null
+++ b/test/system/continuous/master-agitator.pl
@@ -0,0 +1,85 @@
+#! /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){
+	print "Usage : magitator.pl <sleep before kill in minutes> <sleep before start-all in minutes>\n";
+	exit(1);
+}
+
+$ACCUMULO_HOME="../../..";
+
+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 = "$ACCUMULO_HOME/bin/start-all.sh --notSlaves";
+	print "$t $cmd\n";
+	system($cmd);
+}
+
+

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9cfdb3c6/test/system/continuous/start-agitator.sh
----------------------------------------------------------------------
diff --git a/test/system/continuous/start-agitator.sh b/test/system/continuous/start-agitator.sh
index e476c8d..979899f 100755
--- a/test/system/continuous/start-agitator.sh
+++ b/test/system/continuous/start-agitator.sh
@@ -15,27 +15,57 @@
 # 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/}
+# 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
-export HADOOP_PREFIX
 
 mkdir -p $CONTINUOUS_LOG_DIR
 
-# Agitator needs to handle HDFS and Accumulo - can't switch to a single user and expect it to work
-nohup ./agitator.pl $KILL_SLEEP_TIME $TUP_SLEEP_TIME $HDFS_USER $ACCUMULO_USER $MIN_KILL $MAX_KILL >$CONTINUOUS_LOG_DIR/`date +%Y%m%d%H%M%S`_`hostname`_agitator.out 2>$CONTINUOUS_LOG_DIR/`date +%Y%m%d%H%M%S`_`hostname`_agitator.err &
+LOG_BASE="${CONTINUOUS_LOG_DIR}/`date +%Y%m%d%H%M%S`_`hostname`"
 
+# Start agitators for datanodes, tservers, and the master
 if [[ "`whoami`" == "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 $CONTINUOUS_CONF_DIR/magitator.pl $MASTER_KILL_SLEEP_TIME $MASTER_RESTART_SLEEP_TIME >$CONTINUOUS_LOG_DIR/`date +%Y%m%d%H%M%S`_`hostname`_magitator.out 2>$CONTINUOUS_LOG_DIR/`date +%Y%m%d%H%M%S`_`hostname`_magitator.err &" -m - $ACCUMULO_USER
+  su -c "nohup $CONTINUOUS_CONF_DIR/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 $CONTINUOUS_CONF_DIR/tserver-agitator.pl $KILL_SLEEP_TIME $TUP_SLEEP_TIME $MIN_KILL $MAX_KILL >${LOG_BASE}_tserver-agitator.out 2>${LOG_BASE}_tserver-agitator.err &" -m - $ACCUMULO_USER
+
+  su -c "nohup $CONTINUOUS_CONF_DIR/datanode-agitator.pl $KILL_SLEEP_TIME $TUP_SLEEP_TIME $HADOOP_PREFIX $MIN_KILL $MAX_KILL >${LOG_BASE}_datanode-agitator.out 2>${LOG_BASE}_datanode-agitator.err &" -m - $HDFS_USER
+
 elif [[ "`whoami`" == $ACCUMULO_USER ]]; then
-  # Just run the magitator if we're the accumulo user
-  nohup $CONTINUOUS_CONF_DIR/magitator.pl $MASTER_KILL_SLEEP_TIME $MASTER_RESTART_SLEEP_TIME >$CONTINUOUS_LOG_DIR/`date +%Y%m%d%H%M%S`_`hostname`_magitator.out 2>$CONTINUOUS_LOG_DIR/`date +%Y%m%d%H%M%S`_`hostname`_magitator.err &
+  echo "Running master-agitator and tserver-agitator as `whoami`. Running datanode-agitator as $HDFS_USER using sudo."
+  # Just run the master-agitator if we're the accumulo user
+  nohup $CONTINUOUS_CONF_DIR/master-agitator.pl $MASTER_KILL_SLEEP_TIME $MASTER_RESTART_SLEEP_TIME >${LOG_BASE}_master-agitator.out 2>${LOG_BASE}_master-agitator.err &
+
+  nohup $CONTINUOUS_CONF_DIR/tserver-agitator.pl $KILL_SLEEP_TIME $TUP_SLEEP_TIME $MIN_KILL $MAX_KILL >${LOG_BASE}_tserver-agitator.out 2>${LOG_BASE}_tserver-agitator.err &
+
+  sudo -u $HDFS_USER nohup $CONTINUOUS_CONF_DIR/datanode-agitator.pl $KILL_SLEEP_TIME $TUP_SLEEP_TIME $HADOOP_PREFIX $MIN_KILL $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 -m -u $ACCUMULO_USER "nohup $CONTINUOUS_CONF_DIR/magitator.pl $MASTER_KILL_SLEEP_TIME $MASTER_RESTART_SLEEP_TIME >$CONTINUOUS_LOG_DIR/`date +%Y%m%d%H%M%S`_`hostname`_magitator.out 2>$CONTINUOUS_LOG_DIR/`date +%Y%m%d%H%M%S`_`hostname`_magitator.err &"
+  sudo -u $ACCUMULO_USER "nohup $CONTINUOUS_CONF_DIR/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 $CONTINUOUS_CONF_DIR/tserver-agitator.pl $KILL_SLEEP_TIME $TUP_SLEEP_TIME $MIN_KILL $MAX_KILL >${LOG_BASE}_tserver-agitator.out 2>${LOG_BASE}_tserver-agitator.err &"
+
+  sudo -u $HDFS_USER "nohup $CONTINUOUS_CONF_DIR/datanode-agitator.pl $KILL_SLEEP_TIME $TUP_SLEEP_TIME $HADOOP_PREFIX $MIN_KILL $MAX_KILL >${LOG_BASE}_datanode-agitator.out 2>${LOG_BASE}_datanode-agitator.err &" -m - $HDFS_USER
+
 fi
 
 if ${AGITATE_HDFS:-false} ; then
-  AGITATOR_LOG=${CONTINUOUS_LOG_DIR}/`date +%Y%m%d%H%M%S`_`hostname`_hdfs-agitator
-  nohup ./hdfs-agitator.pl --sleep ${AGITATE_HDFS_SLEEP_TIME} --hdfs-cmd ${AGITATE_HDFS_COMMAND} --superuser ${AGITATE_HDFS_SUPERUSER} --sudo ${AGITATE_HDFS_SUDO} >${AGITATOR_LOG}.out 2>${AGITATOR_LOG}.err &
+  AGITATOR_LOG="${LOG_BASE}_hdfs-agitator"
+  sudo -u $AGITATE_HDFS_SUPERUSER nohup $CONTINUOUS_CONF_DIR/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/blob/9cfdb3c6/test/system/continuous/stop-agitator.sh
----------------------------------------------------------------------
diff --git a/test/system/continuous/stop-agitator.sh b/test/system/continuous/stop-agitator.sh
index 8ce448e..136b451 100755
--- a/test/system/continuous/stop-agitator.sh
+++ b/test/system/continuous/stop-agitator.sh
@@ -14,13 +14,37 @@
 # 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/}
+# 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
-if [[ ("`whoami`" != "root") && ("`whoami`" != $ACCUMULO_USER) ]];  then
-  sudo -u $ACCUMULO_USER pkill -f agitator.pl
+if [[ "`whoami`" == "root" ]]; then 
+  echo "Stopping all processes matching 'agitator.pl' as root"
+  pkill -f agitator.pl 2>/dev/null
+elif [[ "`whoami`" == $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 `whoami`"
+  pkill -f agitator.pl 2>/dev/null 2>/dev/null
 else
-  pkill -f agitator.pl
+  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/blob/9cfdb3c6/test/system/continuous/tserver-agitator.pl
----------------------------------------------------------------------
diff --git a/test/system/continuous/tserver-agitator.pl b/test/system/continuous/tserver-agitator.pl
new file mode 100755
index 0000000..befc097
--- /dev/null
+++ b/test/system/continuous/tserver-agitator.pl
@@ -0,0 +1,129 @@
+#! /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);
+}
+
+$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");
+}
+
+@slavesRaw = `cat $ACCUMULO_CONF_DIR/slaves`;
+chomp(@slavesRaw);
+
+for $slave (@slavesRaw){
+  if($slave eq "" || substr($slave,0,1) eq "#"){
+    next;
+  }
+
+  push(@slaves, $slave);
+}
+
+
+if(scalar(@slaves) < $maxKill){
+  print STDERR "WARN setting maxKill to ".scalar(@slaves)."\n";
+  $maxKill = scalar(@slaves);
+}
+
+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(@slaves)));
+      $server = $slaves[$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);
+}
+


[3/9] git commit: Merge branch '1.5.1-SNAPSHOT' into 1.6.0-SNAPSHOT

Posted by kt...@apache.org.
Merge branch '1.5.1-SNAPSHOT' into 1.6.0-SNAPSHOT


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

Branch: refs/heads/master
Commit: 4de0c1d4b92ae5fd9a737bb6573f5aa8d07deba9
Parents: 4cf60b6 9cfdb3c
Author: Josh Elser <el...@apache.org>
Authored: Fri Dec 20 18:20:41 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri Dec 20 18:20:41 2013 -0500

----------------------------------------------------------------------
 test/system/continuous/agitator.pl          | 201 -----------------------
 test/system/continuous/datanode-agitator.pl | 131 +++++++++++++++
 test/system/continuous/magitator.pl         |  85 ----------
 test/system/continuous/master-agitator.pl   |  85 ++++++++++
 test/system/continuous/start-agitator.sh    |  37 ++++-
 test/system/continuous/stop-agitator.sh     |  19 ++-
 test/system/continuous/tserver-agitator.pl  | 129 +++++++++++++++
 7 files changed, 389 insertions(+), 298 deletions(-)
----------------------------------------------------------------------



[5/9] git commit: Merge branch '1.4.5-SNAPSHOT' into 1.5.1-SNAPSHOT

Posted by kt...@apache.org.
Merge branch '1.4.5-SNAPSHOT' into 1.5.1-SNAPSHOT

Conflicts:
	conf/examples/1GB/native-standalone/accumulo-env.sh
	conf/examples/1GB/standalone/accumulo-env.sh
	conf/examples/2GB/native-standalone/accumulo-env.sh
	conf/examples/2GB/standalone/accumulo-env.sh
	conf/examples/3GB/native-standalone/accumulo-env.sh
	conf/examples/3GB/standalone/accumulo-env.sh
	conf/examples/512MB/native-standalone/accumulo-env.sh
	conf/examples/512MB/standalone/accumulo-env.sh


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

Branch: refs/heads/master
Commit: 0d58654e7e05d2a8c56eae5629d47762f4396491
Parents: 9cfdb3c 55827af
Author: Josh Elser <el...@apache.org>
Authored: Fri Dec 20 19:27:52 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri Dec 20 19:27:52 2013 -0500

----------------------------------------------------------------------
 conf/examples/1GB/native-standalone/accumulo-env.sh   | 2 +-
 conf/examples/1GB/standalone/accumulo-env.sh          | 2 +-
 conf/examples/2GB/native-standalone/accumulo-env.sh   | 2 +-
 conf/examples/2GB/standalone/accumulo-env.sh          | 2 +-
 conf/examples/3GB/native-standalone/accumulo-env.sh   | 2 +-
 conf/examples/3GB/standalone/accumulo-env.sh          | 2 +-
 conf/examples/512MB/native-standalone/accumulo-env.sh | 2 +-
 conf/examples/512MB/standalone/accumulo-env.sh        | 2 +-
 8 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d58654e/conf/examples/1GB/native-standalone/accumulo-env.sh
----------------------------------------------------------------------
diff --cc conf/examples/1GB/native-standalone/accumulo-env.sh
index c683521,b0c79b5..28936d0
--- a/conf/examples/1GB/native-standalone/accumulo-env.sh
+++ b/conf/examples/1GB/native-standalone/accumulo-env.sh
@@@ -50,11 -50,9 +50,11 @@@ test -z "$ACCUMULO_TSERVER_OPTS" && exp
  test -z "$ACCUMULO_MASTER_OPTS"  && export ACCUMULO_MASTER_OPTS="${POLICY} -Xmx128m -Xms128m"
  test -z "$ACCUMULO_MONITOR_OPTS" && export ACCUMULO_MONITOR_OPTS="${POLICY} -Xmx64m -Xms64m" 
  test -z "$ACCUMULO_GC_OPTS"      && export ACCUMULO_GC_OPTS="-Xmx64m -Xms64m"
 -test -z "$ACCUMULO_LOGGER_OPTS"  && export ACCUMULO_LOGGER_OPTS="-Xmx384m -Xms256m"
  test -z "$ACCUMULO_GENERAL_OPTS" && export ACCUMULO_GENERAL_OPTS="-XX:+UseConcMarkSweepGC -XX:CMSInitiatingOccupancyFraction=75"
  test -z "$ACCUMULO_OTHER_OPTS"   && export ACCUMULO_OTHER_OPTS="-Xmx128m -Xms64m"
- export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_HOME/conf/monitor ; echo localhost ) 2>/dev/null | head -1`
 -export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
++export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_HOME/conf/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
  # what do when the JVM runs out of heap memory
  export ACCUMULO_KILL_CMD='kill -9 %p'
 +
 +# Should the monitor bind to all network interfaces -- default: false
 +# export ACCUMULO_MONITOR_BIND_ALL="true"

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d58654e/conf/examples/1GB/standalone/accumulo-env.sh
----------------------------------------------------------------------
diff --cc conf/examples/1GB/standalone/accumulo-env.sh
index 6fe9cf9,eac752c..a0e9517
--- a/conf/examples/1GB/standalone/accumulo-env.sh
+++ b/conf/examples/1GB/standalone/accumulo-env.sh
@@@ -50,11 -50,9 +50,11 @@@ test -z "$ACCUMULO_TSERVER_OPTS" && exp
  test -z "$ACCUMULO_MASTER_OPTS"  && export ACCUMULO_MASTER_OPTS="${POLICY} -Xmx128m -Xms128m"
  test -z "$ACCUMULO_MONITOR_OPTS" && export ACCUMULO_MONITOR_OPTS="${POLICY} -Xmx64m -Xms64m" 
  test -z "$ACCUMULO_GC_OPTS"      && export ACCUMULO_GC_OPTS="-Xmx64m -Xms64m"
 -test -z "$ACCUMULO_LOGGER_OPTS"  && export ACCUMULO_LOGGER_OPTS="-Xmx384m -Xms256m"
  test -z "$ACCUMULO_GENERAL_OPTS" && export ACCUMULO_GENERAL_OPTS="-XX:+UseConcMarkSweepGC -XX:CMSInitiatingOccupancyFraction=75"
  test -z "$ACCUMULO_OTHER_OPTS"   && export ACCUMULO_OTHER_OPTS="-Xmx128m -Xms64m"
- export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_HOME/conf/monitor ; echo localhost ) 2>/dev/null | head -1`
 -export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
++export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_HOME/conf/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
  # what do when the JVM runs out of heap memory
  export ACCUMULO_KILL_CMD='kill -9 %p'
 +
 +# Should the monitor bind to all network interfaces -- default: false
 +# export ACCUMULO_MONITOR_BIND_ALL="true"

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d58654e/conf/examples/2GB/native-standalone/accumulo-env.sh
----------------------------------------------------------------------
diff --cc conf/examples/2GB/native-standalone/accumulo-env.sh
index bbad995,18694c2..1c9965d
--- a/conf/examples/2GB/native-standalone/accumulo-env.sh
+++ b/conf/examples/2GB/native-standalone/accumulo-env.sh
@@@ -49,11 -49,9 +49,11 @@@ test -z "$ACCUMULO_TSERVER_OPTS" && exp
  test -z "$ACCUMULO_MASTER_OPTS"  && export ACCUMULO_MASTER_OPTS="${POLICY} -Xmx256m -Xms256m"
  test -z "$ACCUMULO_MONITOR_OPTS" && export ACCUMULO_MONITOR_OPTS="${POLICY} -Xmx128m -Xms64m" 
  test -z "$ACCUMULO_GC_OPTS"      && export ACCUMULO_GC_OPTS="-Xmx128m -Xms128m"
 -test -z "$ACCUMULO_LOGGER_OPTS"  && export ACCUMULO_LOGGER_OPTS="-Xmx768m -Xms256m"
  test -z "$ACCUMULO_GENERAL_OPTS" && export ACCUMULO_GENERAL_OPTS="-XX:+UseConcMarkSweepGC -XX:CMSInitiatingOccupancyFraction=75"
  test -z "$ACCUMULO_OTHER_OPTS"   && export ACCUMULO_OTHER_OPTS="-Xmx256m -Xms64m"
- export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_HOME/conf/monitor ; echo localhost ) 2>/dev/null | head -1`
 -export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
++export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_HOME/conf/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
  # what do when the JVM runs out of heap memory
  export ACCUMULO_KILL_CMD='kill -9 %p'
 +
 +# Should the monitor bind to all network interfaces -- default: false
 +# export ACCUMULO_MONITOR_BIND_ALL="true"

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d58654e/conf/examples/2GB/standalone/accumulo-env.sh
----------------------------------------------------------------------
diff --cc conf/examples/2GB/standalone/accumulo-env.sh
index b9c14e0,ae5fffe..9c95ba7
--- a/conf/examples/2GB/standalone/accumulo-env.sh
+++ b/conf/examples/2GB/standalone/accumulo-env.sh
@@@ -50,11 -50,9 +50,11 @@@ test -z "$ACCUMULO_TSERVER_OPTS" && exp
  test -z "$ACCUMULO_MASTER_OPTS"  && export ACCUMULO_MASTER_OPTS="${POLICY} -Xmx256m -Xms256m"
  test -z "$ACCUMULO_MONITOR_OPTS" && export ACCUMULO_MONITOR_OPTS="${POLICY} -Xmx128m -Xms64m" 
  test -z "$ACCUMULO_GC_OPTS"      && export ACCUMULO_GC_OPTS="-Xmx128m -Xms128m"
 -test -z "$ACCUMULO_LOGGER_OPTS"  && export ACCUMULO_LOGGER_OPTS="-Xmx768m -Xms256m"
  test -z "$ACCUMULO_GENERAL_OPTS" && export ACCUMULO_GENERAL_OPTS="-XX:+UseConcMarkSweepGC -XX:CMSInitiatingOccupancyFraction=75"
  test -z "$ACCUMULO_OTHER_OPTS"   && export ACCUMULO_OTHER_OPTS="-Xmx256m -Xms64m"
- export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_HOME/conf/monitor ; echo localhost ) 2>/dev/null | head -1`
 -export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
++export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_HOME/conf/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
  # what do when the JVM runs out of heap memory
  export ACCUMULO_KILL_CMD='kill -9 %p'
 +
 +# Should the monitor bind to all network interfaces -- default: false
 +# export ACCUMULO_MONITOR_BIND_ALL="true"

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d58654e/conf/examples/3GB/native-standalone/accumulo-env.sh
----------------------------------------------------------------------
diff --cc conf/examples/3GB/native-standalone/accumulo-env.sh
index a28892a,74308b0..710c82e
--- a/conf/examples/3GB/native-standalone/accumulo-env.sh
+++ b/conf/examples/3GB/native-standalone/accumulo-env.sh
@@@ -49,11 -49,9 +49,11 @@@ test -z "$ACCUMULO_TSERVER_OPTS" && exp
  test -z "$ACCUMULO_MASTER_OPTS"  && export ACCUMULO_MASTER_OPTS="${POLICY} -Xmx1g -Xms1g"
  test -z "$ACCUMULO_MONITOR_OPTS" && export ACCUMULO_MONITOR_OPTS="${POLICY} -Xmx1g -Xms256m" 
  test -z "$ACCUMULO_GC_OPTS"      && export ACCUMULO_GC_OPTS="-Xmx256m -Xms256m"
 -test -z "$ACCUMULO_LOGGER_OPTS"  && export ACCUMULO_LOGGER_OPTS="-Xmx1g -Xms256m"
  test -z "$ACCUMULO_GENERAL_OPTS" && export ACCUMULO_GENERAL_OPTS="-XX:+UseConcMarkSweepGC -XX:CMSInitiatingOccupancyFraction=75"
  test -z "$ACCUMULO_OTHER_OPTS"   && export ACCUMULO_OTHER_OPTS="-Xmx1g -Xms256m"
- export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_HOME/conf/monitor ; echo localhost ) 2>/dev/null | head -1`
 -export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
++export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_HOME/conf/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
  # what do when the JVM runs out of heap memory
  export ACCUMULO_KILL_CMD='kill -9 %p'
 +
 +# Should the monitor bind to all network interfaces -- default: false
 +# export ACCUMULO_MONITOR_BIND_ALL="true"

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d58654e/conf/examples/3GB/standalone/accumulo-env.sh
----------------------------------------------------------------------
diff --cc conf/examples/3GB/standalone/accumulo-env.sh
index 0c3672b,d75dda3..0df95e4
--- a/conf/examples/3GB/standalone/accumulo-env.sh
+++ b/conf/examples/3GB/standalone/accumulo-env.sh
@@@ -49,11 -49,9 +49,11 @@@ test -z "$ACCUMULO_TSERVER_OPTS" && exp
  test -z "$ACCUMULO_MASTER_OPTS"  && export ACCUMULO_MASTER_OPTS="${POLICY} -Xmx1g -Xms1g"
  test -z "$ACCUMULO_MONITOR_OPTS" && export ACCUMULO_MONITOR_OPTS="${POLICY} -Xmx1g -Xms256m" 
  test -z "$ACCUMULO_GC_OPTS"      && export ACCUMULO_GC_OPTS="-Xmx256m -Xms256m"
 -test -z "$ACCUMULO_LOGGER_OPTS"  && export ACCUMULO_LOGGER_OPTS="-Xmx1g -Xms256m"
  test -z "$ACCUMULO_GENERAL_OPTS" && export ACCUMULO_GENERAL_OPTS="-XX:+UseConcMarkSweepGC -XX:CMSInitiatingOccupancyFraction=75"
  test -z "$ACCUMULO_OTHER_OPTS"   && export ACCUMULO_OTHER_OPTS="-Xmx1g -Xms256m"
- export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_HOME/conf/monitor ; echo localhost ) 2>/dev/null | head -1`
 -export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
++export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_HOME/conf/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
  # what do when the JVM runs out of heap memory
  export ACCUMULO_KILL_CMD='kill -9 %p'
 +
 +# Should the monitor bind to all network interfaces -- default: false
 +# export ACCUMULO_MONITOR_BIND_ALL="true"

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d58654e/conf/examples/512MB/native-standalone/accumulo-env.sh
----------------------------------------------------------------------
diff --cc conf/examples/512MB/native-standalone/accumulo-env.sh
index 7f32213,cdd33e9..81728ac
--- a/conf/examples/512MB/native-standalone/accumulo-env.sh
+++ b/conf/examples/512MB/native-standalone/accumulo-env.sh
@@@ -49,11 -49,9 +49,11 @@@ test -z "$ACCUMULO_TSERVER_OPTS" && exp
  test -z "$ACCUMULO_MASTER_OPTS"  && export ACCUMULO_MASTER_OPTS="${POLICY} -Xmx128m -Xms128m"
  test -z "$ACCUMULO_MONITOR_OPTS" && export ACCUMULO_MONITOR_OPTS="${POLICY} -Xmx64m -Xms64m" 
  test -z "$ACCUMULO_GC_OPTS"      && export ACCUMULO_GC_OPTS="-Xmx64m -Xms64m"
 -test -z "$ACCUMULO_LOGGER_OPTS"  && export ACCUMULO_LOGGER_OPTS="-Xmx128m -Xms128m"
  test -z "$ACCUMULO_GENERAL_OPTS" && export ACCUMULO_GENERAL_OPTS="-XX:+UseConcMarkSweepGC -XX:CMSInitiatingOccupancyFraction=75"
  test -z "$ACCUMULO_OTHER_OPTS"   && export ACCUMULO_OTHER_OPTS="-Xmx128m -Xms64m"
- export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_HOME/conf/monitor ; echo localhost ) 2>/dev/null | head -1`
 -export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
++export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_HOME/conf/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
  # what do when the JVM runs out of heap memory
  export ACCUMULO_KILL_CMD='kill -9 %p'
 +
 +# Should the monitor bind to all network interfaces -- default: false
 +# export ACCUMULO_MONITOR_BIND_ALL="true"

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d58654e/conf/examples/512MB/standalone/accumulo-env.sh
----------------------------------------------------------------------
diff --cc conf/examples/512MB/standalone/accumulo-env.sh
index a4c95fc,ba0b0c1..40496df
--- a/conf/examples/512MB/standalone/accumulo-env.sh
+++ b/conf/examples/512MB/standalone/accumulo-env.sh
@@@ -49,11 -49,9 +49,11 @@@ test -z "$ACCUMULO_TSERVER_OPTS" && exp
  test -z "$ACCUMULO_MASTER_OPTS"  && export ACCUMULO_MASTER_OPTS="${POLICY} -Xmx128m -Xms128m"
  test -z "$ACCUMULO_MONITOR_OPTS" && export ACCUMULO_MONITOR_OPTS="${POLICY} -Xmx64m -Xms64m" 
  test -z "$ACCUMULO_GC_OPTS"      && export ACCUMULO_GC_OPTS="-Xmx64m -Xms64m"
 -test -z "$ACCUMULO_LOGGER_OPTS"  && export ACCUMULO_LOGGER_OPTS="-Xmx128m -Xms128m"
  test -z "$ACCUMULO_GENERAL_OPTS" && export ACCUMULO_GENERAL_OPTS="-XX:+UseConcMarkSweepGC -XX:CMSInitiatingOccupancyFraction=75"
  test -z "$ACCUMULO_OTHER_OPTS"   && export ACCUMULO_OTHER_OPTS="-Xmx128m -Xms64m"
- export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_HOME/conf/monitor ; echo localhost ) 2>/dev/null | head -1`
 -export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
++export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_HOME/conf/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
  # what do when the JVM runs out of heap memory
  export ACCUMULO_KILL_CMD='kill -9 %p'
 +
 +# Should the monitor bind to all network interfaces -- default: false
 +# export ACCUMULO_MONITOR_BIND_ALL="true"


[4/9] git commit: ACCUMULO-2078 Fix the grep so that the example configurations are properly interpreted.

Posted by kt...@apache.org.
ACCUMULO-2078 Fix the grep so that the example configurations are properly interpreted.


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

Branch: refs/heads/master
Commit: 55827af327c9e04ac010b821e135c56e5d95faaa
Parents: 36f503c
Author: Josh Elser <el...@apache.org>
Authored: Fri Dec 20 19:20:40 2013 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri Dec 20 19:20:40 2013 -0500

----------------------------------------------------------------------
 conf/examples/1GB/native-standalone/accumulo-env.sh   | 2 +-
 conf/examples/1GB/standalone/accumulo-env.sh          | 2 +-
 conf/examples/2GB/native-standalone/accumulo-env.sh   | 2 +-
 conf/examples/2GB/standalone/accumulo-env.sh          | 2 +-
 conf/examples/3GB/native-standalone/accumulo-env.sh   | 2 +-
 conf/examples/3GB/standalone/accumulo-env.sh          | 2 +-
 conf/examples/512MB/native-standalone/accumulo-env.sh | 2 +-
 conf/examples/512MB/standalone/accumulo-env.sh        | 2 +-
 8 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/55827af3/conf/examples/1GB/native-standalone/accumulo-env.sh
----------------------------------------------------------------------
diff --git a/conf/examples/1GB/native-standalone/accumulo-env.sh b/conf/examples/1GB/native-standalone/accumulo-env.sh
index e301eb8..b0c79b5 100755
--- a/conf/examples/1GB/native-standalone/accumulo-env.sh
+++ b/conf/examples/1GB/native-standalone/accumulo-env.sh
@@ -53,6 +53,6 @@ test -z "$ACCUMULO_GC_OPTS"      && export ACCUMULO_GC_OPTS="-Xmx64m -Xms64m"
 test -z "$ACCUMULO_LOGGER_OPTS"  && export ACCUMULO_LOGGER_OPTS="-Xmx384m -Xms256m"
 test -z "$ACCUMULO_GENERAL_OPTS" && export ACCUMULO_GENERAL_OPTS="-XX:+UseConcMarkSweepGC -XX:CMSInitiatingOccupancyFraction=75"
 test -z "$ACCUMULO_OTHER_OPTS"   && export ACCUMULO_OTHER_OPTS="-Xmx128m -Xms64m"
-export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/masters ; echo localhost ) 2>/dev/null | head -1`
+export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
 # what do when the JVM runs out of heap memory
 export ACCUMULO_KILL_CMD='kill -9 %p'

http://git-wip-us.apache.org/repos/asf/accumulo/blob/55827af3/conf/examples/1GB/standalone/accumulo-env.sh
----------------------------------------------------------------------
diff --git a/conf/examples/1GB/standalone/accumulo-env.sh b/conf/examples/1GB/standalone/accumulo-env.sh
index 9abb597..eac752c 100755
--- a/conf/examples/1GB/standalone/accumulo-env.sh
+++ b/conf/examples/1GB/standalone/accumulo-env.sh
@@ -53,6 +53,6 @@ test -z "$ACCUMULO_GC_OPTS"      && export ACCUMULO_GC_OPTS="-Xmx64m -Xms64m"
 test -z "$ACCUMULO_LOGGER_OPTS"  && export ACCUMULO_LOGGER_OPTS="-Xmx384m -Xms256m"
 test -z "$ACCUMULO_GENERAL_OPTS" && export ACCUMULO_GENERAL_OPTS="-XX:+UseConcMarkSweepGC -XX:CMSInitiatingOccupancyFraction=75"
 test -z "$ACCUMULO_OTHER_OPTS"   && export ACCUMULO_OTHER_OPTS="-Xmx128m -Xms64m"
-export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/masters ; echo localhost ) 2>/dev/null | head -1`
+export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
 # what do when the JVM runs out of heap memory
 export ACCUMULO_KILL_CMD='kill -9 %p'

http://git-wip-us.apache.org/repos/asf/accumulo/blob/55827af3/conf/examples/2GB/native-standalone/accumulo-env.sh
----------------------------------------------------------------------
diff --git a/conf/examples/2GB/native-standalone/accumulo-env.sh b/conf/examples/2GB/native-standalone/accumulo-env.sh
index 701d6e5..18694c2 100755
--- a/conf/examples/2GB/native-standalone/accumulo-env.sh
+++ b/conf/examples/2GB/native-standalone/accumulo-env.sh
@@ -52,6 +52,6 @@ test -z "$ACCUMULO_GC_OPTS"      && export ACCUMULO_GC_OPTS="-Xmx128m -Xms128m"
 test -z "$ACCUMULO_LOGGER_OPTS"  && export ACCUMULO_LOGGER_OPTS="-Xmx768m -Xms256m"
 test -z "$ACCUMULO_GENERAL_OPTS" && export ACCUMULO_GENERAL_OPTS="-XX:+UseConcMarkSweepGC -XX:CMSInitiatingOccupancyFraction=75"
 test -z "$ACCUMULO_OTHER_OPTS"   && export ACCUMULO_OTHER_OPTS="-Xmx256m -Xms64m"
-export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/masters ; echo localhost ) 2>/dev/null | head -1`
+export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
 # what do when the JVM runs out of heap memory
 export ACCUMULO_KILL_CMD='kill -9 %p'

http://git-wip-us.apache.org/repos/asf/accumulo/blob/55827af3/conf/examples/2GB/standalone/accumulo-env.sh
----------------------------------------------------------------------
diff --git a/conf/examples/2GB/standalone/accumulo-env.sh b/conf/examples/2GB/standalone/accumulo-env.sh
index adb95b1..ae5fffe 100755
--- a/conf/examples/2GB/standalone/accumulo-env.sh
+++ b/conf/examples/2GB/standalone/accumulo-env.sh
@@ -53,6 +53,6 @@ test -z "$ACCUMULO_GC_OPTS"      && export ACCUMULO_GC_OPTS="-Xmx128m -Xms128m"
 test -z "$ACCUMULO_LOGGER_OPTS"  && export ACCUMULO_LOGGER_OPTS="-Xmx768m -Xms256m"
 test -z "$ACCUMULO_GENERAL_OPTS" && export ACCUMULO_GENERAL_OPTS="-XX:+UseConcMarkSweepGC -XX:CMSInitiatingOccupancyFraction=75"
 test -z "$ACCUMULO_OTHER_OPTS"   && export ACCUMULO_OTHER_OPTS="-Xmx256m -Xms64m"
-export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/masters ; echo localhost ) 2>/dev/null | head -1`
+export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
 # what do when the JVM runs out of heap memory
 export ACCUMULO_KILL_CMD='kill -9 %p'

http://git-wip-us.apache.org/repos/asf/accumulo/blob/55827af3/conf/examples/3GB/native-standalone/accumulo-env.sh
----------------------------------------------------------------------
diff --git a/conf/examples/3GB/native-standalone/accumulo-env.sh b/conf/examples/3GB/native-standalone/accumulo-env.sh
index 51f96a1..74308b0 100755
--- a/conf/examples/3GB/native-standalone/accumulo-env.sh
+++ b/conf/examples/3GB/native-standalone/accumulo-env.sh
@@ -52,6 +52,6 @@ test -z "$ACCUMULO_GC_OPTS"      && export ACCUMULO_GC_OPTS="-Xmx256m -Xms256m"
 test -z "$ACCUMULO_LOGGER_OPTS"  && export ACCUMULO_LOGGER_OPTS="-Xmx1g -Xms256m"
 test -z "$ACCUMULO_GENERAL_OPTS" && export ACCUMULO_GENERAL_OPTS="-XX:+UseConcMarkSweepGC -XX:CMSInitiatingOccupancyFraction=75"
 test -z "$ACCUMULO_OTHER_OPTS"   && export ACCUMULO_OTHER_OPTS="-Xmx1g -Xms256m"
-export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/masters ; echo localhost ) 2>/dev/null | head -1`
+export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
 # what do when the JVM runs out of heap memory
 export ACCUMULO_KILL_CMD='kill -9 %p'

http://git-wip-us.apache.org/repos/asf/accumulo/blob/55827af3/conf/examples/3GB/standalone/accumulo-env.sh
----------------------------------------------------------------------
diff --git a/conf/examples/3GB/standalone/accumulo-env.sh b/conf/examples/3GB/standalone/accumulo-env.sh
index ad89755..d75dda3 100755
--- a/conf/examples/3GB/standalone/accumulo-env.sh
+++ b/conf/examples/3GB/standalone/accumulo-env.sh
@@ -52,6 +52,6 @@ test -z "$ACCUMULO_GC_OPTS"      && export ACCUMULO_GC_OPTS="-Xmx256m -Xms256m"
 test -z "$ACCUMULO_LOGGER_OPTS"  && export ACCUMULO_LOGGER_OPTS="-Xmx1g -Xms256m"
 test -z "$ACCUMULO_GENERAL_OPTS" && export ACCUMULO_GENERAL_OPTS="-XX:+UseConcMarkSweepGC -XX:CMSInitiatingOccupancyFraction=75"
 test -z "$ACCUMULO_OTHER_OPTS"   && export ACCUMULO_OTHER_OPTS="-Xmx1g -Xms256m"
-export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/masters ; echo localhost ) 2>/dev/null | head -1`
+export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
 # what do when the JVM runs out of heap memory
 export ACCUMULO_KILL_CMD='kill -9 %p'

http://git-wip-us.apache.org/repos/asf/accumulo/blob/55827af3/conf/examples/512MB/native-standalone/accumulo-env.sh
----------------------------------------------------------------------
diff --git a/conf/examples/512MB/native-standalone/accumulo-env.sh b/conf/examples/512MB/native-standalone/accumulo-env.sh
index b3b733d..cdd33e9 100755
--- a/conf/examples/512MB/native-standalone/accumulo-env.sh
+++ b/conf/examples/512MB/native-standalone/accumulo-env.sh
@@ -52,6 +52,6 @@ test -z "$ACCUMULO_GC_OPTS"      && export ACCUMULO_GC_OPTS="-Xmx64m -Xms64m"
 test -z "$ACCUMULO_LOGGER_OPTS"  && export ACCUMULO_LOGGER_OPTS="-Xmx128m -Xms128m"
 test -z "$ACCUMULO_GENERAL_OPTS" && export ACCUMULO_GENERAL_OPTS="-XX:+UseConcMarkSweepGC -XX:CMSInitiatingOccupancyFraction=75"
 test -z "$ACCUMULO_OTHER_OPTS"   && export ACCUMULO_OTHER_OPTS="-Xmx128m -Xms64m"
-export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/masters ; echo localhost ) 2>/dev/null | head -1`
+export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
 # what do when the JVM runs out of heap memory
 export ACCUMULO_KILL_CMD='kill -9 %p'

http://git-wip-us.apache.org/repos/asf/accumulo/blob/55827af3/conf/examples/512MB/standalone/accumulo-env.sh
----------------------------------------------------------------------
diff --git a/conf/examples/512MB/standalone/accumulo-env.sh b/conf/examples/512MB/standalone/accumulo-env.sh
index 975e20c..ba0b0c1 100755
--- a/conf/examples/512MB/standalone/accumulo-env.sh
+++ b/conf/examples/512MB/standalone/accumulo-env.sh
@@ -52,6 +52,6 @@ test -z "$ACCUMULO_GC_OPTS"      && export ACCUMULO_GC_OPTS="-Xmx64m -Xms64m"
 test -z "$ACCUMULO_LOGGER_OPTS"  && export ACCUMULO_LOGGER_OPTS="-Xmx128m -Xms128m"
 test -z "$ACCUMULO_GENERAL_OPTS" && export ACCUMULO_GENERAL_OPTS="-XX:+UseConcMarkSweepGC -XX:CMSInitiatingOccupancyFraction=75"
 test -z "$ACCUMULO_OTHER_OPTS"   && export ACCUMULO_OTHER_OPTS="-Xmx128m -Xms64m"
-export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/masters ; echo localhost ) 2>/dev/null | head -1`
+export ACCUMULO_LOG_HOST=`(grep -v '^#' $ACCUMULO_CONF_DIR/monitor | egrep -v '^[[:space:]]*$' ; echo localhost ) 2>/dev/null | head -1`
 # what do when the JVM runs out of heap memory
 export ACCUMULO_KILL_CMD='kill -9 %p'


[9/9] git commit: Merge remote-tracking branch 'origin/1.6.0-SNAPSHOT'

Posted by kt...@apache.org.
Merge remote-tracking branch 'origin/1.6.0-SNAPSHOT'


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

Branch: refs/heads/master
Commit: 2d8a2ecb8fd90458eb9702d5db3215c539070882
Parents: 551c650 3aace98
Author: Keith Turner <kt...@apache.org>
Authored: Fri Dec 20 22:27:43 2013 -0500
Committer: Keith Turner <kt...@apache.org>
Committed: Fri Dec 20 22:27:43 2013 -0500

----------------------------------------------------------------------
 .../1GB/native-standalone/accumulo-env.sh       |   2 +-
 conf/examples/1GB/standalone/accumulo-env.sh    |   2 +-
 .../2GB/native-standalone/accumulo-env.sh       |   2 +-
 conf/examples/2GB/standalone/accumulo-env.sh    |   2 +-
 .../3GB/native-standalone/accumulo-env.sh       |   2 +-
 conf/examples/3GB/standalone/accumulo-env.sh    |   2 +-
 .../512MB/native-standalone/accumulo-env.sh     |   2 +-
 conf/examples/512MB/standalone/accumulo-env.sh  |   2 +-
 .../gc/GarbageCollectWriteAheadLogs.java        |  28 +--
 .../monitor/util/celltypes/TableLinkType.java   |  14 +-
 .../org/apache/accumulo/tserver/Tablet.java     |   4 +-
 .../apache/accumulo/tserver/TabletServer.java   |  18 +-
 .../tserver/log/TabletServerLogger.java         |   6 +-
 test/system/continuous/agitator.pl              | 201 -------------------
 test/system/continuous/datanode-agitator.pl     | 131 ++++++++++++
 test/system/continuous/magitator.pl             |  85 --------
 test/system/continuous/master-agitator.pl       |  85 ++++++++
 test/system/continuous/start-agitator.sh        |  37 +++-
 test/system/continuous/stop-agitator.sh         |  19 +-
 test/system/continuous/tserver-agitator.pl      | 129 ++++++++++++
 20 files changed, 431 insertions(+), 342 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/2d8a2ecb/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
----------------------------------------------------------------------


[7/9] git commit: ACCUMULO-2048 Removed unnecessary path manipulation in walog GC and added sanity check

Posted by kt...@apache.org.
ACCUMULO-2048 Removed unnecessary path manipulation in walog GC and
added sanity check  

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

Branch: refs/heads/master
Commit: bed80715607d6b9b74ad96e3f2a7020e8ad9f149
Parents: 66c7848
Author: Keith Turner <kt...@apache.org>
Authored: Fri Dec 20 19:56:44 2013 -0500
Committer: Keith Turner <kt...@apache.org>
Committed: Fri Dec 20 22:13:49 2013 -0500

----------------------------------------------------------------------
 .../gc/GarbageCollectWriteAheadLogs.java        | 28 +++++++++++---------
 1 file changed, 16 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/bed80715/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
----------------------------------------------------------------------
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
index c4c69b4..111d1e7 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
@@ -42,7 +42,6 @@ import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.fs.VolumeManager.FileType;
 import org.apache.accumulo.server.security.SystemCredentials;
 import org.apache.accumulo.server.util.MetadataTableUtil;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
@@ -231,20 +230,19 @@ public class GarbageCollectWriteAheadLogs {
       InterruptedException {
     int count = 0;
     Iterator<LogEntry> iterator = MetadataTableUtil.getLogEntries(SystemCredentials.get());
+
     while (iterator.hasNext()) {
       for (String entry : iterator.next().logSet) {
-        String parts[] = entry.split("/", 2);
-        String filename = parts[1];
-        Path path;
-        if (filename.contains(":"))
-          path = new Path(filename);
-        else
-          path = fs.getFullPath(FileType.WAL, filename);
-        
-        Path pathFromNN = nameToFileMap.remove(path.getName());
+        String uuid = new Path(entry).getName();
+        if (!isUUID(uuid)) {
+          // fully expect this to be a uuid, if its not then something is wrong and walog GC should not proceed!
+          throw new IllegalArgumentException("Expected uuid, but got " + uuid + " from " + entry);
+        }
+
+        Path pathFromNN = nameToFileMap.remove(uuid);
         if (pathFromNN != null) {
           status.currentLog.inUse++;
-          sortedWALogs.remove(path.getName());
+          sortedWALogs.remove(uuid);
         }
         count++;
       }
@@ -258,7 +256,13 @@ public class GarbageCollectWriteAheadLogs {
     Set<String> servers = new HashSet<String>();
     for (String walDir : ServerConstants.getWalDirs()) {
       Path walRoot = new Path(walDir);
-      FileStatus[] listing = fs.listStatus(walRoot);
+      FileStatus[] listing = null;
+      try {
+        listing = fs.listStatus(walRoot);
+      } catch (FileNotFoundException e) {
+        // ignore dir
+      }
+
       if (listing == null)
         continue;
       for (FileStatus status : listing) {