You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bigtop.apache.org by ja...@apache.org on 2014/10/09 04:27:09 UTC

git commit: BIGTOP 1388. Use cluster failure tests during other tests ith command line parameterization.

Repository: bigtop
Updated Branches:
  refs/heads/master 04802cc59 -> 2a766aa27


BIGTOP 1388. Use cluster failure tests during other tests ith command line parameterization.

Signed-off-by: jay@apache.org <jayunit100>


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

Branch: refs/heads/master
Commit: 2a766aa27ab2db9177f708eddcbcfdd59530aa2f
Parents: 04802cc
Author: Dawson Choong <da...@wandisco.com>
Authored: Wed Oct 8 22:20:56 2014 -0400
Committer: jay@apache.org <jayunit100>
Committed: Wed Oct 8 22:24:10 2014 -0400

----------------------------------------------------------------------
 bigtop-test-framework/README                    |  17 ++
 bigtop-test-framework/pom.xml                   |   5 +
 .../itest/failures/AbstractFailure.groovy       |  44 ++++--
 .../bigtop/itest/failures/FailureVars.groovy    | 155 +++++++++++++++++++
 .../itest/failures/ClusterFailuresTest.groovy   |  44 ++++--
 .../hadoop/mapreduce/TestHadoopExamples.groovy  |  13 ++
 .../itest/iolongevity/FailureExecutor.groovy    |  76 +++++++++
 .../bigtop/itest/iolongevity/TestDFSIO.groovy   |  40 +++--
 .../bigtop/itest/iolongevity/TestSLive.groovy   |  21 +++
 build.gradle                                    |  39 +++++
 10 files changed, 414 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/2a766aa2/bigtop-test-framework/README
----------------------------------------------------------------------
diff --git a/bigtop-test-framework/README b/bigtop-test-framework/README
index 0980a6a..79bfdf3 100644
--- a/bigtop-test-framework/README
+++ b/bigtop-test-framework/README
@@ -65,3 +65,20 @@ on the machine with Bigtop workspace:
   - run following sample commands from your local machine to verify your setup:
       * ssh -i /test_bigtop_ssh_key bigtop@<some cluster node> sudo service crond stop
       * ssh -i /test_bigtop_ssh_key bigtop@<some cluster node> sudo service crond start
+
+The user can now run a series of cluster failures while running hadoop and mapreduce tests. This can be achieved by
+modifying the vars.properties file found in src/main/resources. To enable the properties file, use -DuseProperties=true
+when running tests (ex: mvn verify -f bigtop-tests/test-execution/longevity/pom.xml -DuseProperties=true
+-Dorg.apache.maven-failsafe-plugin.testInclude=**/TestDFSIO*/).
+
+For vars.properties:
+-Use testhost=name to specify host being tested on.
+-Use testremotehost=name to specify remote host name being tested on.
+-Use runall=true to run all cluster failures.
+-Use servicerestart=true to perform a cron or crond service restart.
+-Use servicekill=true to perform a service kill.
+-Use networkshutdown=true to perform network shutdown and restart.
+-Use service=name to specify which service being used for restart/kill. (default is crond).
+-Use failuredelay=time to specify time (in seconds) in between each failure function.
+-Use startdelay=time to specify time (in seconds) before first failure.
+-Use killduration=time to specify the duration (in seconds) a service is down for.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/2a766aa2/bigtop-test-framework/pom.xml
----------------------------------------------------------------------
diff --git a/bigtop-test-framework/pom.xml b/bigtop-test-framework/pom.xml
index 71828d2..4d13b43 100644
--- a/bigtop-test-framework/pom.xml
+++ b/bigtop-test-framework/pom.xml
@@ -61,6 +61,11 @@
   <build>
     <plugins>
       <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <version>2.7</version>
+      </plugin>
+      <plugin>
         <groupId>org.codehaus.groovy</groupId>
         <artifactId>groovy-eclipse-compiler</artifactId>
         <version>${groovy-eclipse-compiler.version}</version>

http://git-wip-us.apache.org/repos/asf/bigtop/blob/2a766aa2/bigtop-test-framework/src/main/groovy/org/apache/bigtop/itest/failures/AbstractFailure.groovy
----------------------------------------------------------------------
diff --git a/bigtop-test-framework/src/main/groovy/org/apache/bigtop/itest/failures/AbstractFailure.groovy b/bigtop-test-framework/src/main/groovy/org/apache/bigtop/itest/failures/AbstractFailure.groovy
index d4f117d..f88358f 100644
--- a/bigtop-test-framework/src/main/groovy/org/apache/bigtop/itest/failures/AbstractFailure.groovy
+++ b/bigtop-test-framework/src/main/groovy/org/apache/bigtop/itest/failures/AbstractFailure.groovy
@@ -69,7 +69,7 @@ public abstract class AbstractFailure implements Runnable {
   /**
    * How long (in milliseconds) shall we wait before executing first failure.
    */
-  protected long startDelay = 0
+  protected long failureDelay = 0
 
   /**
    * How long failure thread waits before next check if failure is over and it should call restore commands.
@@ -88,11 +88,11 @@ public abstract class AbstractFailure implements Runnable {
    * Constructor allowing to set all params.
    *
    * @param hosts list of hosts the failure will be running against
-   * @param startDelay how long (in millisecs) failure will wait before starting
+   * @param failureDelay how long (in millisecs) failure will wait before starting
    */
-  public AbstractFailure(List<String> hosts, long startDelay) {
+  public AbstractFailure(List<String> hosts, long failureDelay) {
     this.hosts = hosts
-    this.startDelay = startDelay
+    this.failureDelay = failureDelay
   }
 
   /**
@@ -101,22 +101,36 @@ public abstract class AbstractFailure implements Runnable {
   @Override
   public void run() {
     try {
-      if (startDelay > 0) {
+      if(failureDelay > 0) {
         try {
-          Thread.sleep(startDelay)
+          Thread.sleep(failureDelay)
         } catch (InterruptedException e) {
-          Thread.currentThread().interrupt()
           return
         }
       }
-
-      runFailCommands()
-
-      while (!Thread.currentThread().isInterrupted()) {
-        try {
-          Thread.sleep(SLEEP_TIME)
-        } catch (InterruptedException e) {
-          return
+      if(FailureVars.instance.getServiceRestart().equals("true")
+        || FailureVars.instance.getServiceKill().equals("true")
+        || FailureVars.instance.getNetworkShutdown().equals("true")) {
+        runFailCommands()
+        Thread.sleep(FailureVars.instance.getKillDuration())
+      }
+      else {
+        if (failureDelay > 0) {
+          try {
+            Thread.sleep(failureDelay)
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt()
+            return
+          }
+        }
+        runFailCommands()
+
+        while (!Thread.currentThread().isInterrupted()) {
+          try {
+            Thread.sleep(SLEEP_TIME)
+          } catch (InterruptedException e) {
+            return
+          }
         }
       }
     } finally {

http://git-wip-us.apache.org/repos/asf/bigtop/blob/2a766aa2/bigtop-test-framework/src/main/groovy/org/apache/bigtop/itest/failures/FailureVars.groovy
----------------------------------------------------------------------
diff --git a/bigtop-test-framework/src/main/groovy/org/apache/bigtop/itest/failures/FailureVars.groovy b/bigtop-test-framework/src/main/groovy/org/apache/bigtop/itest/failures/FailureVars.groovy
new file mode 100644
index 0000000..6bc1ca3
--- /dev/null
+++ b/bigtop-test-framework/src/main/groovy/org/apache/bigtop/itest/failures/FailureVars.groovy
@@ -0,0 +1,155 @@
+package org.apache.bigtop.itest.failures
+
+import org.apache.bigtop.itest.shell.OS
+import org.apache.bigtop.itest.shell.Shell
+import java.io.BufferedReader
+import java.io.IOException;
+
+/**
+ * This class manages objects, variables, and command line parameter values for cluster failure testing.
+ * By default, all parameters are off or set to false.
+ */
+public class FailureVars {
+
+  private final String CRON_SERVICE;
+  private final int SLEEP_TIME = 100;
+  private static FailureVars instance = null;
+  private String propertyFile = "/vars.properties"
+  private Boolean useProperties = System.getProperty("useProperties", Boolean.FALSE.toString());
+  private String testHost;
+  private String testRemoteHost;
+  private String runFailures;
+  private String serviceRestart;
+  private String serviceKill;
+  private String networkShutdown;
+  private String service;
+  private String failureDelay;
+  private String startDelay;
+  private String killDuration;
+
+  {
+    switch (OS.linux_flavor) {
+      case ~/(?is).*(redhat|centos|rhel|fedora|enterpriseenterpriseserver).*/:
+        CRON_SERVICE = "crond"
+        break;
+      default:
+        CRON_SERVICE = "cron"
+    }
+  }
+
+  protected FailureVars() {
+    if(useProperties) {
+      loadProps();
+    }
+  }
+
+  public static FailureVars getInstance() {
+    if(instance == null) {
+      instance = new FailureVars();
+    }
+    return instance;
+  }
+
+  private void loadProps() {
+    try {
+      BufferedReader is = new BufferedReader (new InputStreamReader(getClass().getResourceAsStream(propertyFile)));
+      System.out.println("Input Stream Location: " + is);
+      Properties props = new Properties();
+      props.load(is);
+      is.close();
+      testHost = props.getProperty("testhost", "localhost");
+      testRemoteHost = props.getProperty("testremotehost", "apache.org");
+      runFailures = props.getProperty("runall", Boolean.FALSE.toString());
+      serviceRestart = props.getProperty("servicerestart", Boolean.FALSE.toString());
+      serviceKill = props.getProperty("servicekill", Boolean.FALSE.toString());
+      networkShutdown = props.getProperty("networkshutdown", Boolean.FALSE.toString());
+      service = props.getProperty("service", "crond");
+      failureDelay = props.getProperty("failuredelay", Integer.toString(0));
+      startDelay = props.getProperty("startdelay", Integer.toString(0));
+      killDuration = props.getProperty("killduration", Integer.toString(0));
+    }
+    catch(IOException ioe) {
+      System.out.println(ioe);
+    }
+  }
+
+  public void setKillDuration(String killDuration) {
+    this.killDuration = Long.toString(killDuration);
+  }
+
+  public void setTestHost(String testHost) {
+    this.testHost = testHost;
+  }
+
+  public void setTestRemoteHost(String testRemoteHost) {
+    this.testRemoteHost = testRemoteHost;
+  }
+
+  public void setRunAll(String runFailures) {
+    this.runFailures = runFailures;
+  }
+
+  public void setRestart(String serviceRestart) {
+    this.serviceRestart = serviceRestart;
+  }
+
+  public void setKill(String serviceKill) {
+    this.serviceKill = serviceKill;
+  }
+
+  public void setShutdown(String networkShutdown) {
+    this.networkShutdown = networkShutdown;
+  }
+
+  public void setFailureDelay(long failureDelay) {
+    this.failureDelay = Long.toString(failureDelay);
+  }
+
+  public void setService(String service) {
+    this.service = service;
+  }
+
+  public long getKillDuration() {
+    return Long.parseLong(killDuration)*1000;
+  }
+  public String getTestHost() {
+    return testHost;
+  }
+
+  public String getService() {
+    return service;
+  }
+
+  public String getTestRemoteHost() {
+    return testRemoteHost;
+  }
+
+  public long getStartDelay() {
+    return Long.parseLong(startDelay)*1000;
+  }
+
+
+  public String getRunFailures() {
+    return runFailures;
+  }
+
+  public String getServiceRestart() {
+    return serviceRestart;
+  }
+
+  public  String getServiceKill() {
+    return serviceKill;
+  }
+
+  public String getNetworkShutdown() {
+    return networkShutdown;
+  }
+
+  public long getFailureDelay() {
+    return Long.parseLong(failureDelay)*1000;
+  }
+
+  int getSleepTime() {
+    return SLEEP_TIME;
+  }
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/2a766aa2/bigtop-test-framework/src/test/groovy/org/apache/bigtop/itest/failures/ClusterFailuresTest.groovy
----------------------------------------------------------------------
diff --git a/bigtop-test-framework/src/test/groovy/org/apache/bigtop/itest/failures/ClusterFailuresTest.groovy b/bigtop-test-framework/src/test/groovy/org/apache/bigtop/itest/failures/ClusterFailuresTest.groovy
index af6b36b..1f099be 100644
--- a/bigtop-test-framework/src/test/groovy/org/apache/bigtop/itest/failures/ClusterFailuresTest.groovy
+++ b/bigtop-test-framework/src/test/groovy/org/apache/bigtop/itest/failures/ClusterFailuresTest.groovy
@@ -19,15 +19,19 @@
 package org.apache.bigtop.itest.failures
 
 import org.apache.bigtop.itest.shell.OS
+import org.junit.Before
 import org.junit.Test
 import org.apache.bigtop.itest.shell.Shell
 
 public class ClusterFailuresTest {
   private Shell rootShell = new Shell("/bin/bash", "root")
   private final int SLEEP_TIME = 100
-  private final String TEST_HOST = "localhost"
-  private final String TEST_REMOTE_HOST = "apache.org"
   private final String CRON_SERVICE
+  private String testHost;
+  private String testRemoteHost;
+  private String serviceRestart;
+  private String serviceKill;
+  private String networkShutdown;
 
   {
     switch (OS.linux_flavor) {
@@ -39,12 +43,22 @@ public class ClusterFailuresTest {
     }
   }
 
+  @Before
+  void configureVars() {
+    def f = new FailureVars();
+    testHost = f.getTestHost();
+    testRemoteHost = f.getTestRemoteHost();
+    serviceRestart = f.getServiceRestart();
+    serviceKill = f.getServiceKill();
+    networkShutdown = f.getNetworkShutdown();
+  }
+
   @Test
   void testServiceRestart() {
     startCron()
     assert isCronRunning(), "$CRON_SERVICE service isn't running before the test:"
 
-    def cronKilled = new ServiceRestartFailure([TEST_HOST], "$CRON_SERVICE")
+    def cronKilled = new ServiceRestartFailure([testHost], "$CRON_SERVICE")
     Thread t = new Thread(cronKilled)
     t.start()
 
@@ -81,7 +95,7 @@ public class ClusterFailuresTest {
     startCron()
     assert isCronRunning(), "$CRON_SERVICE service isn't running before the test:"
 
-    def cronKilled = new ServiceKilledFailure([TEST_HOST], "$CRON_SERVICE")
+    def cronKilled = new ServiceKilledFailure([testHost], "$CRON_SERVICE")
     Thread t = new Thread(cronKilled)
     t.start()
 
@@ -109,34 +123,34 @@ public class ClusterFailuresTest {
   @Test
   void testNetworkShutdown() {
     //make sure there are no blocking rules
-    rootShell.exec("iptables -D INPUT -s $TEST_REMOTE_HOST -j DROP")
-    rootShell.exec("iptables -D OUTPUT -d $TEST_REMOTE_HOST -j DROP")
+    rootShell.exec("iptables -D INPUT -s $testRemoteHost -j DROP")
+    rootShell.exec("iptables -D OUTPUT -d $testRemoteHost -j DROP")
 
-    assert isRemoteHostReachable(), "No ping to $TEST_REMOTE_HOST, which is used for network failures test:"
+    assert isRemoteHostReachable(), "No ping to $testRemoteHost, which is used for network failures test:"
 
-    def networkShutdown = new NetworkShutdownFailure(TEST_HOST, [TEST_REMOTE_HOST])
+    def networkShutdown = new NetworkShutdownFailure(testHost, [testRemoteHost])
     Thread t = new Thread(networkShutdown)
     t.start()
 
     while (isRemoteHostReachable()) {
-      println "$TEST_REMOTE_HOST is still reachable..."
+      println "$testRemoteHost is still reachable..."
       Thread.sleep(SLEEP_TIME)
     }
 
     try{
-      assert !isRemoteHostReachable(), "Connection to $TEST_REMOTE_HOST hasn't been killed as expected:"
-      println "$TEST_REMOTE_HOST isn't reachable. Good."
+      assert !isRemoteHostReachable(), "Connection to $testRemoteHost hasn't been killed as expected:"
+      println "$testRemoteHost isn't reachable. Good."
     } finally {
       t.interrupt()
     }
 
     while (!isRemoteHostReachable()) {
-      println "$TEST_REMOTE_HOST isn't reachable..."
+      println "$testRemoteHost isn't reachable..."
       Thread.sleep(SLEEP_TIME)
     }
 
-    assert isRemoteHostReachable(), "Connection to $TEST_REMOTE_HOST hasn't been restored after the test:"
-    println "$TEST_REMOTE_HOST is reachable again. Good."
+    assert isRemoteHostReachable(), "Connection to $testRemoteHost hasn't been restored after the test:"
+    println "$testRemoteHost is reachable again. Good."
   }
 
   private boolean isCronRunning() {
@@ -148,6 +162,6 @@ public class ClusterFailuresTest {
   }
 
   private boolean isRemoteHostReachable() {
-    return rootShell.exec("ping -qc 1 $TEST_REMOTE_HOST").ret == 0 ? true : false
+    return rootShell.exec("ping -qc 1 $testRemoteHost").ret == 0 ? true : false
   }
 }

http://git-wip-us.apache.org/repos/asf/bigtop/blob/2a766aa2/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/mapreduce/TestHadoopExamples.groovy
----------------------------------------------------------------------
diff --git a/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/mapreduce/TestHadoopExamples.groovy b/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/mapreduce/TestHadoopExamples.groovy
index 6d290ef..213250c 100644
--- a/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/mapreduce/TestHadoopExamples.groovy
+++ b/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/mapreduce/TestHadoopExamples.groovy
@@ -18,6 +18,7 @@
 
 package org.apache.bigtop.itest.hadoop.mapreduce
 
+import org.apache.bigtop.itest.failures.FailureVars
 import org.junit.BeforeClass
 import org.junit.AfterClass
 import static org.junit.Assert.assertNotNull
@@ -145,8 +146,20 @@ class TestHadoopExamples {
 
   @Test
   void testMRExample() {
+    if(FailureVars.instance.getRunFailures().equals("true")
+        || FailureVars.instance.getServiceRestart().equals("true")
+        || FailureVars.instance.getServiceKill().equals("true")
+        || FailureVars.instance.getNetworkShutdown().equals("true")) {
+      runFailureThread();
+    }
     sh.exec("hadoop jar $testJar $testName $testArgs");
     assertTrue("Example $testName $testJar $testName $testArgs failed", sh.getRet() == 0);
   }
+
+  private void runFailureThread() {
+    FailureExecutor failureExecutor = new FailureExecutor();
+    Thread failureThread = new Thread(failureExecutor, "TestHadoopExamples");
+    failureThread.start();
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/bigtop/blob/2a766aa2/bigtop-tests/test-artifacts/longevity/src/main/groovy/org/apache/bigtop/itest/iolongevity/FailureExecutor.groovy
----------------------------------------------------------------------
diff --git a/bigtop-tests/test-artifacts/longevity/src/main/groovy/org/apache/bigtop/itest/iolongevity/FailureExecutor.groovy b/bigtop-tests/test-artifacts/longevity/src/main/groovy/org/apache/bigtop/itest/iolongevity/FailureExecutor.groovy
new file mode 100644
index 0000000..79212b2
--- /dev/null
+++ b/bigtop-tests/test-artifacts/longevity/src/main/groovy/org/apache/bigtop/itest/iolongevity/FailureExecutor.groovy
@@ -0,0 +1,76 @@
+package org.apache.bigtop.itest.failures
+
+import org.apache.bigtop.itest.failures.FailureVars
+import org.apache.bigtop.itest.failures.NetworkShutdownFailure
+import org.apache.bigtop.itest.failures.ServiceKilledFailure
+import org.apache.bigtop.itest.failures.ServiceRestartFailure
+import org.apache.bigtop.itest.failures.FailureConstants
+import org.apache.bigtop.itest.shell.OS
+import org.junit.Test
+import org.apache.bigtop.itest.shell.Shell
+
+/**
+ * A runnable that executes the cluster failure threads.
+ * Used to run in parallel to hadoop jobs to test their completion.
+ */
+public class FailureExecutor implements Runnable {
+
+  private String restart = FailureVars.instance.getServiceRestart();
+  private String kill = FailureVars.instance.getServiceKill();
+  private String shutdown = FailureVars.instance.getNetworkShutdown();
+  private String testHost = FailureVars.instance.getTestHost();
+  private String testRemoteHost = FailureVars.instance.getTestRemoteHost();
+  private long failureDelay = FailureVars.instance.getFailureDelay();
+  private long startDelay = FailureVars.instance.getStartDelay();
+
+  Thread restartThread = null;
+  Thread killThread = null;
+  Thread shutdownThread = null;
+
+  public void run() {
+    if (startDelay > 0) {
+      try {
+        Thread.sleep(startDelay)
+      } catch (InterruptedException e) {}
+    }
+    if(restart != null && restart.equals("true")) {
+      serviceRestartExec();
+    }
+    if(kill != null && kill.equals("true")) {
+      serviceKillExec();
+    }
+    if(shutdown != null && shutdown.equals("true")) {
+      networkShutdownExec();
+    }
+  }
+
+  public void serviceRestartExec() {
+    System.out.println("Restarting services...")
+    def srf = new ServiceRestartFailure([testHost],
+      FailureVars.instance.CRON_SERVICE, failureDelay);
+    restartThread = new Thread(srf, "restartThread");
+    restartThread.start();
+    restartThread.join();
+    System.out.println("Finished restarting services.\n");
+  }
+
+  public void serviceKillExec() {
+    System.out.println("Killing services....")
+    def skf = new ServiceKilledFailure([testHost],
+      FailureVars.instance.CRON_SERVICE, failureDelay);
+    killThread = new Thread(skf, "killThread");
+    killThread.start();
+    killThread.join();
+    System.out.println("Finished killing services.\n");
+  }
+
+  public void networkShutdownExec() {
+    System.out.println("Shutting down network...")
+    def nsf = new NetworkShutdownFailure(testHost,
+      [testRemoteHost], failureDelay);
+    shutdownThread = new Thread(nsf)
+    shutdownThread.start();
+    shutdownThread.join();
+    System.out.println("Finished restarting network.\n");
+  }
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/2a766aa2/bigtop-tests/test-artifacts/longevity/src/main/groovy/org/apache/bigtop/itest/iolongevity/TestDFSIO.groovy
----------------------------------------------------------------------
diff --git a/bigtop-tests/test-artifacts/longevity/src/main/groovy/org/apache/bigtop/itest/iolongevity/TestDFSIO.groovy b/bigtop-tests/test-artifacts/longevity/src/main/groovy/org/apache/bigtop/itest/iolongevity/TestDFSIO.groovy
index e32d56e..755ebad 100644
--- a/bigtop-tests/test-artifacts/longevity/src/main/groovy/org/apache/bigtop/itest/iolongevity/TestDFSIO.groovy
+++ b/bigtop-tests/test-artifacts/longevity/src/main/groovy/org/apache/bigtop/itest/iolongevity/TestDFSIO.groovy
@@ -19,6 +19,8 @@
 package org.apache.bigtop.itest.iolongevity
 
 import org.apache.bigtop.itest.JarContent
+import org.apache.bigtop.itest.failures.FailureExecutor
+import org.apache.bigtop.itest.failures.FailureVars
 import org.apache.bigtop.itest.shell.Shell
 import static org.apache.bigtop.itest.LogErrorsUtils.logError
 import static org.junit.Assert.assertNotNull
@@ -37,9 +39,9 @@ public class TestDFSIO {
   private static final String BENCHMARKS_DIR = "/benchmarks/TestDFSIO";
   private static final String HADOOP_MAPRED_HOME = System.getenv('HADOOP_MAPRED_HOME');
 
-  private final String numFiles = System.getProperty("numFiles", "10");
-  private final String sizeFile = System.getProperty("sizeFile", "10MB");
-  private final int numOfIterations = Integer.getInteger("numOfIterations", 1);
+  private final String NUM_FILES = System.getProperty("numFiles", "10");
+  private final String SIZE_FILE = System.getProperty("sizeFile", "10MB");
+  private final int NUM_ITERATIONS = Integer.getInteger("numOfIterations", 1);
   private final int TIMEOUT = 5000;
 
   private String DFSIO_TEMPLATE;
@@ -56,6 +58,11 @@ public class TestDFSIO {
   }
 
   @Before
+  void configureVars() {
+    def failureVars = new FailureVars();
+  }
+
+  @Before
   public void cleanup() {
     sh.exec("hadoop fs -rm -r " + BENCHMARKS_DIR);
     Thread.sleep(TIMEOUT);
@@ -65,19 +72,26 @@ public class TestDFSIO {
 
   @Test
   public void testDFSIO() {
+    if(FailureVars.instance.getRunFailures().equals("true")
+    || FailureVars.instance.getServiceRestart().equals("true")
+    || FailureVars.instance.getServiceKill().equals("true")
+    || FailureVars.instance.getNetworkShutdown().equals("true")) {
+      runFailureThread();
+    }
+
     DFSIO_TEMPLATE = "hadoop jar " + hadoopTestJar + " TestDFSIO %s ";
     final String DFSIO_ARGS_TEMPLATE = "-nrFiles %s -fileSize %s -resFile %s";
 
     final String writeSuccess = BENCHMARKS_DIR + "/io_write/_SUCCESS";
     final String appendSuccess = BENCHMARKS_DIR + "/io_append/_SUCCESS";
     final String readSuccess = BENCHMARKS_DIR + "/io_read/_SUCCESS";
-    String argStr = String.format(DFSIO_ARGS_TEMPLATE, numFiles, sizeFile, result);
+    String argStr = String.format(DFSIO_ARGS_TEMPLATE, NUM_FILES, SIZE_FILE, result);
 
-    for (int counter = 0; counter < numOfIterations; counter++) {
+    for (int counter = 0; counter < NUM_ITERATIONS; counter++) {
       executeCmd(String.format(DFSIO_TEMPLATE + argStr, WRITE_CMD), writeSuccess);
       executeCmd(String.format(DFSIO_TEMPLATE + argStr, APPEND_CMD), appendSuccess);
       executeCmd(String.format(DFSIO_TEMPLATE + argStr, READ_CMD), readSuccess);
-      argStr = String.format(DFSIO_ARGS_TEMPLATE, numFiles, sizeFile, result + "." + counter);
+      argStr = String.format(DFSIO_ARGS_TEMPLATE, NUM_FILES, SIZE_FILE, result + "." + counter);
     }
   }
 
@@ -88,10 +102,16 @@ public class TestDFSIO {
     sh.exec("hadoop fs -ls " + expectedFile);
     boolean success = false;
     sh.getOut().each { str ->
-        if (str.contains(expectedFile)) {
-            success = true;
-        }
+      if (str.contains(expectedFile)) {
+        success = true;
+      }
     }
     assertTrue("File " + expectedFile + " was not found", success);
   }
-}
\ No newline at end of file
+
+  private void runFailureThread() {
+    FailureExecutor failureExecutor = new FailureExecutor();
+    Thread failureThread = new Thread(failureExecutor, "DFSIO");
+    failureThread.start();
+  }
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/2a766aa2/bigtop-tests/test-artifacts/longevity/src/main/groovy/org/apache/bigtop/itest/iolongevity/TestSLive.groovy
----------------------------------------------------------------------
diff --git a/bigtop-tests/test-artifacts/longevity/src/main/groovy/org/apache/bigtop/itest/iolongevity/TestSLive.groovy b/bigtop-tests/test-artifacts/longevity/src/main/groovy/org/apache/bigtop/itest/iolongevity/TestSLive.groovy
index 58ba465..f4ce338 100644
--- a/bigtop-tests/test-artifacts/longevity/src/main/groovy/org/apache/bigtop/itest/iolongevity/TestSLive.groovy
+++ b/bigtop-tests/test-artifacts/longevity/src/main/groovy/org/apache/bigtop/itest/iolongevity/TestSLive.groovy
@@ -19,7 +19,10 @@
 package org.apache.bigtop.itest.iolongevity
 
 import org.apache.bigtop.itest.JarContent
+import org.apache.bigtop.itest.failures.FailureExecutor
+import org.apache.bigtop.itest.failures.FailureVars
 import org.apache.bigtop.itest.shell.Shell
+import org.junit.Before
 import org.junit.BeforeClass
 import org.junit.Test
 import static org.apache.bigtop.itest.LogErrorsUtils.logError
@@ -46,6 +49,11 @@ public class TestSLive {
   private final int numOfIterations = Integer.getInteger("numOfIterations", 1);
   static String[] sliveCmds
 
+  @Before
+  void configureVars() {
+    def failureVars = new FailureVars();
+  }
+
   @BeforeClass
   static void setUp() throws IOException {
     assertNotNull("Can't find hadoop.mapreduce.client.jobclient.tests.jar",
@@ -97,6 +105,13 @@ public class TestSLive {
 
   @Test
   public void testSlive() {
+    if(FailureVars.instance.getRunFailures().equals("true")
+        || FailureVars.instance.getServiceRestart().equals("true")
+        || FailureVars.instance.getServiceKill().equals("true")
+        || FailureVars.instance.getNetworkShutdown().equals("true")) {
+      runFailureThread();
+    }
+
     String suffix = ""
     for (int counter = 0; counter < numOfIterations; counter++) {
       setupDir()
@@ -118,4 +133,10 @@ public class TestSLive {
       suffix = "." + counter
     }
   }
+
+  private void runFailureThread() {
+    FailureExecutor failureExecutor = new FailureExecutor();
+    Thread failureThread = new Thread(failureExecutor, "SLive");
+    failureThread.start();
+  }
 }

http://git-wip-us.apache.org/repos/asf/bigtop/blob/2a766aa2/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index f36056b..4a740f2 100644
--- a/build.gradle
+++ b/build.gradle
@@ -53,6 +53,45 @@ task installCommon(type:Exec) {
 task installAllLocalArtifacts() {
 
 }
+task installArtifact() {
+
+}
+
+/**
+ * Allows user to specify which artifacts to install by dynamically generating tasks.
+ */
+def artifactToInstall = {
+  def final BASE_DIR = projectDir.absolutePath
+  def final TEST_DIR = "bigtop-tests/test-artifacts"
+  def final ARTIFACT_DIR = "${BASE_DIR}/${TEST_DIR}"
+
+  File srcDir
+  srcDir = file("${TEST_DIR}")
+  def artifactFiles = files {srcDir.listFiles()}
+  def artifactCollection = []
+  artifactFiles.each { File file ->
+    artifactCollection.add(file.name)
+  }
+  artifactCollection.each { artifact ->
+    task "install-${artifact}" << {
+      description 'Installs ${artifact} artifact with Maven'
+      def final PATH = "${ARTIFACT_DIR}/$artifact/pom.xml"
+      def final WRAPPER = "mvn clean install -f " + PATH
+      installTopLevel
+      installCommon
+      installConf
+      installiTest
+      exec {
+        workingDir '.'
+        commandLine WRAPPER.split(" ")
+      }
+    }
+  }
+}
+
+project.afterEvaluate{
+  artifactToInstall()
+}
 
 installTestArtifacts.mustRunAfter installiTest
 installiTest.mustRunAfter installTopLevel