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/08/20 06:01:29 UTC

git commit: BIGTOP-1222. Simplify and gradleize a subset of the bigtop smokes.

Repository: bigtop
Updated Branches:
  refs/heads/master 4fca4573b -> da7ce0f3e


BIGTOP-1222. Simplify and gradleize a subset of the bigtop smokes.

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/da7ce0f3
Tree: http://git-wip-us.apache.org/repos/asf/bigtop/tree/da7ce0f3
Diff: http://git-wip-us.apache.org/repos/asf/bigtop/diff/da7ce0f3

Branch: refs/heads/master
Commit: da7ce0f3e3fa2e24fddfb73a60733ded2061c698
Parents: 4fca457
Author: Jay Vyas <ja...@gmail.com>
Authored: Fri Jun 20 23:48:43 2014 -0400
Committer: jay@apache.org <jayunit100>
Committed: Wed Aug 20 00:00:04 2014 -0400

----------------------------------------------------------------------
 .../org/apache/bigtop/itest/TestUtils.groovy    |  12 +-
 bigtop-tests/smoke-tests/README                 |  59 +++++++
 bigtop-tests/smoke-tests/build.gradle           |  71 ++++++++
 .../smoke-tests/flume/TestFlumeNG.groovy        | 112 +++++++++++++
 bigtop-tests/smoke-tests/flume/build.gradle     |  55 +++++++
 bigtop-tests/smoke-tests/flume/conf/flume.conf  |  20 +++
 bigtop-tests/smoke-tests/flume/log4j.properties |   9 ++
 bigtop-tests/smoke-tests/hive/build.gradle      |  58 +++++++
 bigtop-tests/smoke-tests/hive/log4j.properties  |   9 ++
 bigtop-tests/smoke-tests/mahout/build.gradle    |  56 +++++++
 .../smoke-tests/mahout/log4j.properties         |   9 ++
 bigtop-tests/smoke-tests/mapreduce/build.gradle |  59 +++++++
 .../smoke-tests/pig/TestPigSmoke.groovy         |  58 +++++++
 bigtop-tests/smoke-tests/pig/build.gradle       |  37 +++++
 bigtop-tests/smoke-tests/pig/log4j.properties   |   9 ++
 bigtop-tests/smoke-tests/pig/wordcount.pig      |   6 +
 bigtop-tests/smoke-tests/settings.gradle        |  50 ++++++
 .../smoke-tests/sqoop/TestSqoopETLHsql.groovy   | 160 +++++++++++++++++++
 bigtop-tests/smoke-tests/sqoop/build.gradle     |  57 +++++++
 bigtop-tests/smoke-tests/sqoop/log4j.properties |   9 ++
 .../hadoop/mapreduce/TestHadoopExamples.groovy  |  44 ++++-
 .../hadoop/mapreduce/TestHadoopSmoke.groovy     |   2 +-
 .../hivesmoke/HiveBulkScriptExecutor.groovy     |  26 ++-
 .../itest/hivesmoke/TestHiveSmokeBulk.groovy    |  11 ++
 .../src/main/resources/scripts/ql/basic/filter  |   2 +-
 25 files changed, 983 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-test-framework/src/main/groovy/org/apache/bigtop/itest/TestUtils.groovy
----------------------------------------------------------------------
diff --git a/bigtop-test-framework/src/main/groovy/org/apache/bigtop/itest/TestUtils.groovy b/bigtop-test-framework/src/main/groovy/org/apache/bigtop/itest/TestUtils.groovy
index 3020303..f4bc247 100644
--- a/bigtop-test-framework/src/main/groovy/org/apache/bigtop/itest/TestUtils.groovy
+++ b/bigtop-test-framework/src/main/groovy/org/apache/bigtop/itest/TestUtils.groovy
@@ -47,21 +47,21 @@ public class TestUtils {
       sh.exec("hadoop fs -test -e ${inputDir}");
       if (sh.getRet() == 0) {
         sh.exec("hadoop fs -rmr -skipTrash ${inputDir}");
-        assertTrue("Deletion of previous $inputDir from HDFS failed",
+        assertTrue("Deletion of previous $inputDir from the DFS failed",
             sh.getRet() == 0);
       }
       if (inputFiles != null) {
         sh.exec("hadoop fs -mkdir -p ${inputDir}");
-        assertTrue("Could not create input directory to HDFS", sh.getRet() == 0);
+        assertTrue("Could not create input directory to the DFS", sh.getRet() == 0);
         // copy additional files into HDFS input folder
         inputFiles.each {
           sh.exec("hadoop fs -put ${it} ${inputDir}");
-          assertTrue("Could not copy input files into input folder in HDFS", sh.getRet() == 0);
+          assertTrue("Could not copy input files into input folder in the DFS", sh.getRet() == 0);
         }
       } else {
         // copy the entire resource folder into HDFS
         sh.exec("hadoop fs -put ${inputDir} ${inputDir}");
-        assertTrue("Could not copy input directory to HDFS", sh.getRet() == 0);
+        assertTrue("Could not copy input directory to the DFS", sh.getRet() == 0);
       }
     }
 
@@ -70,11 +70,11 @@ public class TestUtils {
       sh.exec("hadoop fs -test -e ${outputDir}");
       if (sh.getRet() == 0) {
         sh.exec("hadoop fs -rmr -skipTrash ${outputDir}");
-        assertTrue("Deletion of previous examples output from HDFS failed",
+        assertTrue("Deletion of previous examples output from the DFS failed",
             sh.getRet() == 0);
       }
       sh.exec("hadoop fs -mkdir -p ${outputDir}");
-      assertTrue("Could not create output directory in HDFS", sh.getRet() == 0);
+      assertTrue("Could not create output directory in DFS", sh.getRet() == 0);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/smoke-tests/README
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/README b/bigtop-tests/smoke-tests/README
new file mode 100644
index 0000000..c8ac345
--- /dev/null
+++ b/bigtop-tests/smoke-tests/README
@@ -0,0 +1,59 @@
+# This is the new smoke testing module for bigtop
+
+## Goals
+
+This gradle based test framework aims at making it extremely simple to
+customize and test Apache BigTop's hadoop distribution.
+
+ - Easily add new tests to bigtop without adding new directories / xml files.
+ - Debug tests easily and make logging extremely transparent.
+ - Geneate easy to read test reports.
+ - Run bigtop tests without needing a jar file.
+ - (todo) Run bigtop tests easily, ideally without gradle/maven
+   (this can be done with gradle wrapper)
+
+So overall, this framework makes testing with apache bigtop easy as well as fun!
+
+## Running
+
+Each directory is a gradle "subproject" mapping to an ecosystem component.
+If you specify, for example, "flume", then the flume test runs.  To test hive as well,
+you would specify "flume,hive", and so on.   An example is below:
+```
+    gradle compileGroovy test -Dsmoke-tests=flume,hive --info
+```
+
+The --info option is a critical feature here:  You can easily debug your tests this way,
+just make sure you have log4j.properties in your conf/ directory locally.
+
+## Adding more tests
+
+To add a new test you will want to add a groovy file to an existing ecosystem test or create a new folder
+(if you are adding a new component to bigtop that has no tests).
+
+This can be done easily by copying an existing test subproject, and simply modifying it
+for your own needs.  The settings.gradle file will automatically see the new folder you have added,
+and run your build.gradle.
+
+## How it works
+
+Subprojects in gradle are configured at the top level by the root build.gradle file.  Each
+subproject is injected with bits of functionality from the top level (i.e. generic functions
+for confirming environment variables are set, calling of the test task, etc).  Each subproject
+then is responsible for defining its source directories and resources, and does little more.
+
+Right now, this library leverages tests both from test-artifacts, as well as locally.
+Hopefully over time all tests will be merged here, because it is easier to maintain
+this script based framework.
+
+## Typical recipes
+
+There are a lot of ways to run the bigtop tests, and not all are applicable to all clusters.
+So, below, we've accumulated some typical modifications which you can easily implement
+when running tests through this gradle interface (rather than as jars).
+
+1) Modify the TestMahoutExamples.groovy file to exclude clustering tests, and re run.
+2) Add new tests to hive , other than the "basic" hive test.
+3) Modify the Sqoop test to write more data via the embedded database.
+4) Add your own custom Groovy test in a directory which calls a shell script of your own desgn.
+And so on.

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/smoke-tests/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/build.gradle b/bigtop-tests/smoke-tests/build.gradle
new file mode 100644
index 0000000..57286e5
--- /dev/null
+++ b/bigtop-tests/smoke-tests/build.gradle
@@ -0,0 +1,71 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */apply plugin: 'groovy'
+
+repositories {
+  mavenCentral()
+}
+
+dependencies {
+}
+
+subprojects {
+
+  ext.hadoopVersion = '2.3.0'
+  //Note you need a snapshot itest to run this.
+  //if you dont have it, change to 0.7.0.
+  ext.itestVersion = '0.8.0-SNAPSHOT'
+  ext.doExclude = { String filename ->
+    //print("Exclude? ${filename} ... ")
+    def keep_this_test = true;
+    if (filename.contains(".groovy") ||
+        filename.contains(".java")) {
+      keep_this_test =
+          tests_to_include().find({
+            it.contains(filename)
+          }) != null;
+    }
+
+    //println("Keep = ${keep_this_test} "+filename);
+    return !keep_this_test;
+  }
+
+  println("Now testing...");
+  test {
+
+    systemProperties['org.apache.bigtop.itest.hivesmoke.TestHiveSmokeBulk.test_include'] = 'basic'
+       testLogging {
+      events "passed", "skipped", "failed"
+    }
+  }
+
+  /**
+   *  Utility function for tests to use to confirm EVN Variables.
+   */
+  ext.checkEnv = { env_vars ->
+    env_vars.each() {
+      def value = System.getenv("${it}")
+      if (value == null || value == "null")
+        throw new GradleException("undeclared env variable: ${it}")
+      else
+        println("ENV VARIABLE: ${it} = $value");
+    }
+  }
+
+test.dependsOn compileGroovy
+compileGroovy.dependsOn clean
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/smoke-tests/flume/TestFlumeNG.groovy
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/flume/TestFlumeNG.groovy b/bigtop-tests/smoke-tests/flume/TestFlumeNG.groovy
new file mode 100644
index 0000000..568e44b
--- /dev/null
+++ b/bigtop-tests/smoke-tests/flume/TestFlumeNG.groovy
@@ -0,0 +1,112 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.bigtop.itest.hadoop.mapreduce
+
+import org.junit.BeforeClass
+import org.junit.AfterClass
+import static org.junit.Assert.assertNotNull
+import org.apache.bigtop.itest.shell.Shell
+import static org.junit.Assert.assertTrue
+import org.junit.Test
+import org.apache.hadoop.conf.Configuration
+import org.apache.bigtop.itest.JarContent
+import org.apache.bigtop.itest.TestUtils
+import org.apache.commons.logging.LogFactory
+import org.apache.commons.logging.Log
+
+import org.junit.runner.RunWith
+
+class TestFlumeNG {
+  static private Log LOG = LogFactory.getLog(Object.class);
+
+  static Shell sh = new Shell("/bin/bash -s");
+
+
+  @AfterClass
+  public static void tearDown() {
+    LOG.warn("Not deleting flume-test from hdfs, delete manually if necessary")
+    //sh.exec("hadoop fs -rmr -skipTrash /tmp/flume-test");
+  }
+
+  @BeforeClass
+  static void setUp() {
+    sh.exec("hadoop fs -rmr -skipTrash /tmp/flume-test");
+    /**
+     *  Start a collecter that will run for 60 seconds.
+     *  This test is HCFS Compatible, see FLUME-2140.
+     */
+    String conf = """\n
+        agent.channels.memory-channel.type = memory
+        agent.channels.memory-channel.capacity = 2000
+        agent.channels.memory-channel.transactionCapacity = 100
+        agent.sources.tail-source.type = exec
+        agent.sources.tail-source.command = tail -F /tmp/flume-smoke.source
+        agent.sources.tail-source.channels = memory-channel
+
+        agent.sinks.log-sink.channel = memory-channel
+        agent.sinks.log-sink.type = logger
+
+        # Define a sink that outputs to the DFS
+        agent.sinks.hdfs-sink.channel = memory-channel
+        agent.sinks.hdfs-sink.type = hdfs
+        agent.sinks.hdfs-sink.hdfs.path = /tmp/flume-test
+        agent.sinks.hdfs-sink.hdfs.fileType = DataStream
+
+        # activate the channels/sinks/sources
+        agent.channels = memory-channel
+        agent.sources = tail-source
+        agent.sinks = log-sink hdfs-sink"""
+    new File("./conf/flume.conf").write(conf);
+
+    //Start the listener...
+
+    Thread.start {
+      sh.exec("timeout 60 flume-ng agent " +
+          "--conf ./conf/ " +
+          "-f ./conf/flume.conf " +
+          "-Dflume.root.logger=DEBUG,console " +
+          "-n agent > /tmp/flumetest.log")
+    }
+
+    LOG.info("Started threaded listener.")
+    LOG.info("Waiting 60 seconds to finish ....")
+    LOG.info("check /tmp/flumetest.log for progress")
+  }
+
+
+  @Test
+  void test() {
+    /**
+     *    Now write to the sink.
+     */
+    File source = new File("/tmp/flume-smoke.source");
+    for (i in 1..100) {
+      Thread.sleep(1000 - i)
+      source.withWriterAppend("UTF-8") {
+        it.write("hello ${i} \n")
+      }
+    }
+
+    LOG.info("RESULTS.........")
+    sh.exec("hadoop fs -cat /tmp/flume-test/* | grep -q hello")
+    assertTrue("Did not detect the contents in the flume SINK." + sh.getOut(), sh.getRet() == 0);
+
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/smoke-tests/flume/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/flume/build.gradle b/bigtop-tests/smoke-tests/flume/build.gradle
new file mode 100644
index 0000000..37ffc0c
--- /dev/null
+++ b/bigtop-tests/smoke-tests/flume/build.gradle
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+apply plugin: 'groovy'
+
+repositories {
+  mavenCentral()
+}
+
+dependencies {
+  //needed to avoid groovy not on classpath error.
+  testCompile module('org.codehaus.groovy:groovy:1.8.0')
+  testCompile group: 'org.apache.bigtop.itest', name: 'itest-common', version: itestVersion, transitive: 'true'
+  testCompile group: 'org.apache.hadoop', name: 'hadoop-common', version: hadoopVersion, transitive: 'true'
+  testCompile group: 'org.apache.hadoop', name: 'hadoop-hdfs', version: hadoopVersion, transitive: 'true'
+}
+
+
+def tests_to_include() {
+  return [
+      "TestFlumeNG.groovy"
+  ];
+}
+
+sourceSets {
+  test {
+    resources {
+      srcDirs = [
+          'conf/'
+      ]
+    }
+    groovy {
+      srcDirs = ["./"]
+      exclude { FileTreeElement elem -> (doExclude(elem.getName())) }
+    }
+  }
+}
+
+test.doFirst {
+  checkEnv(["FLUME_HOME"])
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/smoke-tests/flume/conf/flume.conf
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/flume/conf/flume.conf b/bigtop-tests/smoke-tests/flume/conf/flume.conf
new file mode 100644
index 0000000..c06c1e3
--- /dev/null
+++ b/bigtop-tests/smoke-tests/flume/conf/flume.conf
@@ -0,0 +1,20 @@
+agent.channels.memory-channel.type = memory
+agent.channels.memory-channel.capacity = 2000
+agent.channels.memory-channel.transactionCapacity = 100
+agent.sources.tail-source.type = exec
+agent.sources.tail-source.command = tail -F /tmp/flume-smoke.source
+agent.sources.tail-source.channels = memory-channel
+
+agent.sinks.log-sink.channel = memory-channel
+agent.sinks.log-sink.type = logger
+
+# Define a sink that outputs to the DFS
+agent.sinks.hdfs-sink.channel = memory-channel
+agent.sinks.hdfs-sink.type = hdfs
+agent.sinks.hdfs-sink.hdfs.path = /tmp/flume-test
+agent.sinks.hdfs-sink.hdfs.fileType = DataStream
+
+# activate the channels/sinks/sources
+agent.channels = memory-channel
+agent.sources = tail-source
+agent.sinks = log-sink hdfs-sink
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/smoke-tests/flume/log4j.properties
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/flume/log4j.properties b/bigtop-tests/smoke-tests/flume/log4j.properties
new file mode 100644
index 0000000..02d115f
--- /dev/null
+++ b/bigtop-tests/smoke-tests/flume/log4j.properties
@@ -0,0 +1,9 @@
+# Set root logger level to DEBUG and its only appender to A1.
+log4j.rootLogger=TRACE, A1
+
+# A1 is set to be a ConsoleAppender.
+log4j.appender.A1=org.apache.log4j.ConsoleAppender
+
+# A1 uses PatternLayout.
+log4j.appender.A1.layout=org.apache.log4j.PatternLayout
+log4j.appender.A1.layout.ConversionPattern=%-4r [BIGTOP-TEST-LOG %t] %-5p %c %x --- %m%n

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/smoke-tests/hive/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/hive/build.gradle b/bigtop-tests/smoke-tests/hive/build.gradle
new file mode 100644
index 0000000..447fb00
--- /dev/null
+++ b/bigtop-tests/smoke-tests/hive/build.gradle
@@ -0,0 +1,58 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+apply plugin: 'groovy'
+
+repositories {
+  mavenCentral()
+}
+
+dependencies {
+  //needed to avoid groovy not on classpath error.
+  testCompile module('org.codehaus.groovy:groovy:1.8.0')
+  testCompile group: 'org.apache.bigtop.itest', name: 'itest-common', version: itestVersion, transitive: 'true'
+  testCompile group: 'org.apache.hadoop', name: 'hadoop-common', version: hadoopVersion, transitive: 'true'
+  testCompile group: 'org.apache.hadoop', name: 'hadoop-hdfs', version: hadoopVersion, transitive: 'true'
+}
+
+
+def tests_to_include() {
+  return [
+      "TestHiveSmokeBulk.groovy",
+      "HiveBulkScriptExecutor.groovy"
+  ];
+}
+
+sourceSets {
+  test {
+    resources {
+      srcDirs = [
+          'conf/'
+      ]
+    }
+    groovy {
+      srcDirs = ["$System.env.BIGTOP_HOME"+"/bigtop-tests/test-artifacts/hive/"]
+      exclude { FileTreeElement elem -> (doExclude(elem.getName())) }
+    }
+  }
+}
+
+test.doFirst {
+  // BIGTOP_HOME is necessary here, since we use it to read in from
+  // test-artifacts.
+  checkEnv(["HIVE_HOME", "HIVE_CONF_DIR", "BIGTOP_HOME"])
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/smoke-tests/hive/log4j.properties
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/hive/log4j.properties b/bigtop-tests/smoke-tests/hive/log4j.properties
new file mode 100644
index 0000000..02d115f
--- /dev/null
+++ b/bigtop-tests/smoke-tests/hive/log4j.properties
@@ -0,0 +1,9 @@
+# Set root logger level to DEBUG and its only appender to A1.
+log4j.rootLogger=TRACE, A1
+
+# A1 is set to be a ConsoleAppender.
+log4j.appender.A1=org.apache.log4j.ConsoleAppender
+
+# A1 uses PatternLayout.
+log4j.appender.A1.layout=org.apache.log4j.PatternLayout
+log4j.appender.A1.layout.ConversionPattern=%-4r [BIGTOP-TEST-LOG %t] %-5p %c %x --- %m%n

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/smoke-tests/mahout/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/mahout/build.gradle b/bigtop-tests/smoke-tests/mahout/build.gradle
new file mode 100644
index 0000000..5a562b4
--- /dev/null
+++ b/bigtop-tests/smoke-tests/mahout/build.gradle
@@ -0,0 +1,56 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+
+apply plugin: 'groovy'
+
+repositories {
+  mavenCentral()
+}
+
+dependencies {
+  //needed to avoid groovy not on classpath error.
+  testCompile module('org.codehaus.groovy:groovy:1.8.0')
+  testCompile group: 'org.apache.bigtop.itest', name: 'itest-common', version: itestVersion, transitive: 'true'
+  testCompile group: 'org.apache.hadoop', name: 'hadoop-common', version: hadoopVersion, transitive: 'true'
+  testCompile group: 'org.apache.hadoop', name: 'hadoop-hdfs', version: hadoopVersion, transitive: 'true'
+}
+
+
+def tests_to_include() {
+  return [
+      "TestMahoutExamples.groovy"
+  ];
+}
+
+sourceSets {
+  test {
+    resources {
+      srcDirs = [
+          'conf/'
+      ]
+    }
+    groovy {
+      srcDirs = ["$System.env.BIGTOP_HOME"+"/bigtop-tests/test-artifacts/mahout/"]
+      exclude { FileTreeElement elem -> (doExclude(elem.getName())) }
+    }
+  }
+}
+
+test.doFirst {
+  checkEnv(["BIGTOP_HOME", "MAHOUT_HOME"])
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/smoke-tests/mahout/log4j.properties
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/mahout/log4j.properties b/bigtop-tests/smoke-tests/mahout/log4j.properties
new file mode 100644
index 0000000..02d115f
--- /dev/null
+++ b/bigtop-tests/smoke-tests/mahout/log4j.properties
@@ -0,0 +1,9 @@
+# Set root logger level to DEBUG and its only appender to A1.
+log4j.rootLogger=TRACE, A1
+
+# A1 is set to be a ConsoleAppender.
+log4j.appender.A1=org.apache.log4j.ConsoleAppender
+
+# A1 uses PatternLayout.
+log4j.appender.A1.layout=org.apache.log4j.PatternLayout
+log4j.appender.A1.layout.ConversionPattern=%-4r [BIGTOP-TEST-LOG %t] %-5p %c %x --- %m%n

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/smoke-tests/mapreduce/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/mapreduce/build.gradle b/bigtop-tests/smoke-tests/mapreduce/build.gradle
new file mode 100644
index 0000000..7b67b4f
--- /dev/null
+++ b/bigtop-tests/smoke-tests/mapreduce/build.gradle
@@ -0,0 +1,59 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+
+apply plugin: 'groovy'
+
+repositories {
+  mavenCentral()
+}
+
+dependencies {
+  //needed to avoid groovy not on classpath error.
+  testCompile module('org.codehaus.groovy:groovy:1.8.0')
+  testCompile group: 'org.apache.bigtop.itest', name: 'itest-common', version: itestVersion, transitive: 'true'
+  testCompile group: 'org.apache.hadoop', name: 'hadoop-common', version: hadoopVersion, transitive: 'true'
+  testCompile group: 'org.apache.hadoop', name: 'hadoop-hdfs', version: hadoopVersion, transitive: 'true'
+}
+
+
+def tests_to_include() {
+  return [
+      "TestHadoopExamples.groovy"
+  ];
+}
+
+sourceSets {
+  test {
+    resources {
+      srcDirs = [
+          'conf/',
+      ]
+    }
+    groovy {
+      srcDirs = [
+          "$System.env.BIGTOP_HOME"+"/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/mapreduce"]
+      exclude {
+        FileTreeElement elem -> (doExclude(elem.getName()))
+      }
+    }
+  }
+}
+
+test.doFirst {
+  checkEnv(["HADOOP_CONF_DIR"])
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/smoke-tests/pig/TestPigSmoke.groovy
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/pig/TestPigSmoke.groovy b/bigtop-tests/smoke-tests/pig/TestPigSmoke.groovy
new file mode 100644
index 0000000..9902267
--- /dev/null
+++ b/bigtop-tests/smoke-tests/pig/TestPigSmoke.groovy
@@ -0,0 +1,58 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.bigtop.itest.hadoop.mapreduce
+
+import org.junit.BeforeClass
+import org.junit.AfterClass
+import static org.junit.Assert.assertNotNull
+import org.apache.bigtop.itest.shell.Shell
+import static org.junit.Assert.assertTrue
+import org.junit.Test
+import org.apache.hadoop.conf.Configuration
+import org.apache.bigtop.itest.JarContent
+import org.apache.bigtop.itest.TestUtils
+import org.apache.commons.logging.LogFactory
+import org.apache.commons.logging.Log
+
+import org.junit.runner.RunWith
+
+class TestPigSmoke {
+  static private Log LOG = LogFactory.getLog(Object.class);
+
+  static Shell sh = new Shell("/bin/bash -s");
+
+
+  @AfterClass
+  public static void tearDown() {
+    sh.exec("hadoop fs -rmr -skipTrash pigsmoketest");
+  }
+
+  @BeforeClass
+  static void setUp() {
+    sh.exec("cat /etc/passwd > passwd");
+    sh.exec("hadoop fs -mkdir pigsmoketest");
+    sh.exec("hadoop fs -copyFromLocal passwd pigsmoketest/input");
+  }
+
+  @Test
+  void test() {
+    sh.exec("pig wordcount.pig");
+    assertTrue("Example word count failed. " + sh.getOut() + " " + sh.getErr(), sh.getRet() == 0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/smoke-tests/pig/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/pig/build.gradle b/bigtop-tests/smoke-tests/pig/build.gradle
new file mode 100644
index 0000000..9b3d323
--- /dev/null
+++ b/bigtop-tests/smoke-tests/pig/build.gradle
@@ -0,0 +1,37 @@
+apply plugin: 'groovy'
+
+repositories {
+  mavenCentral()
+}
+
+dependencies {
+  //needed to avoid groovy not on classpath error.
+  testCompile module('org.codehaus.groovy:groovy:1.8.0')
+  testCompile group: 'org.apache.bigtop.itest', name: 'itest-common', version: itestVersion, transitive: 'true'
+  testCompile group: 'org.apache.hadoop', name: 'hadoop-common', version: hadoopVersion, transitive: 'true'
+  testCompile group: 'org.apache.hadoop', name: 'hadoop-hdfs', version: hadoopVersion, transitive: 'true'
+  testCompile group: 'org.hsqldb', name: 'hsqldb', version: '1.8.0.10'
+}
+
+
+def tests_to_include() {
+  return ["TestPigSmoke.groovy"];
+}
+
+sourceSets {
+  test {
+    resources {
+      srcDirs = ['conf/']
+    }
+
+    groovy {
+      srcDirs = ["./"]
+      exclude 'src/main/groovy/org/apache/bigtop/itest/hadoop/hdfs/**'
+      exclude { FileTreeElement elem -> (doExclude(elem.getName())) }
+    }
+  }
+}
+
+test.doFirst {
+  checkEnv(["PIG_HOME"])
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/smoke-tests/pig/log4j.properties
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/pig/log4j.properties b/bigtop-tests/smoke-tests/pig/log4j.properties
new file mode 100644
index 0000000..02d115f
--- /dev/null
+++ b/bigtop-tests/smoke-tests/pig/log4j.properties
@@ -0,0 +1,9 @@
+# Set root logger level to DEBUG and its only appender to A1.
+log4j.rootLogger=TRACE, A1
+
+# A1 is set to be a ConsoleAppender.
+log4j.appender.A1=org.apache.log4j.ConsoleAppender
+
+# A1 uses PatternLayout.
+log4j.appender.A1.layout=org.apache.log4j.PatternLayout
+log4j.appender.A1.layout.ConversionPattern=%-4r [BIGTOP-TEST-LOG %t] %-5p %c %x --- %m%n

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/smoke-tests/pig/wordcount.pig
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/pig/wordcount.pig b/bigtop-tests/smoke-tests/pig/wordcount.pig
new file mode 100644
index 0000000..2b03219
--- /dev/null
+++ b/bigtop-tests/smoke-tests/pig/wordcount.pig
@@ -0,0 +1,6 @@
+A = load 'pigsmoketest/input';
+B = foreach A generate flatten(TOKENIZE((chararray)$0)) as word;
+C = group B by word;
+D = foreach C generate COUNT(B), group;
+store D into 'pig-output-wordcount';
+dump D ;

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/smoke-tests/settings.gradle
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/settings.gradle b/bigtop-tests/smoke-tests/settings.gradle
new file mode 100644
index 0000000..464988c
--- /dev/null
+++ b/bigtop-tests/smoke-tests/settings.gradle
@@ -0,0 +1,50 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+
+def tests_comma_str = System.properties['smoke.tests']
+
+if(tests_comma_str != null) {
+  def tests = tests_comma_str.split(",")
+  tests.each() { t ->
+    println("including $t")
+    include(t)
+  }
+}
+else {
+  println("""
+
+     ******************************************************************************
+     Welcome to the bigtop smoke tests.
+
+     This is an extensible test module, with one directory for every test component.
+
+     To run it, just use the following command, adding components (directories) into
+
+     the smoke.tests system property.
+
+     For example, to run the hive/ and pig/ smoke tests:
+
+         gradle clean compileGroovy test -Dsmoke.tests=hive,pig
+
+     NOTE:  You must run clean.  Otherwise gradle might skip some of the tests.
+
+     ... Exiting now ...
+     ******************************************************************************
+  """)
+  System.exit(1)
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/smoke-tests/sqoop/TestSqoopETLHsql.groovy
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/sqoop/TestSqoopETLHsql.groovy b/bigtop-tests/smoke-tests/sqoop/TestSqoopETLHsql.groovy
new file mode 100644
index 0000000..3dafc99
--- /dev/null
+++ b/bigtop-tests/smoke-tests/sqoop/TestSqoopETLHsql.groovy
@@ -0,0 +1,160 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.bigtop.itest.hadoop.mapreduce
+
+import org.junit.Assert
+import org.junit.BeforeClass
+import org.junit.AfterClass
+import static org.junit.Assert.assertNotNull
+import org.apache.bigtop.itest.shell.Shell
+import static org.junit.Assert.assertTrue
+import org.junit.Test
+import org.apache.hadoop.conf.Configuration
+import org.apache.bigtop.itest.JarContent
+import org.apache.bigtop.itest.TestUtils
+import org.apache.commons.logging.LogFactory
+import org.apache.commons.logging.Log
+import java.lang.reflect.Constructor;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import org.hsqldb.Server;
+import junit.framework.TestCase;
+import junit.framework.TestResult;
+import org.junit.runner.RunWith
+import org.hsqldb.persist.HsqlDatabaseProperties;
+import org.hsqldb.persist.HsqlProperties;
+import java.sql.DriverManager;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.io.PrintWriter;
+
+class TestSqoopHSQLDBETL {
+  static private Log LOG = LogFactory.getLog(Object.class);
+
+  static Shell sh = new Shell("/bin/bash -s");
+
+
+  @AfterClass
+  public static void tearDown() {
+    //sh.exec("hadoop fs -rmr -skipTrash pigsmoketest");
+    server.stop();
+  }
+
+  static Server server;
+
+  @BeforeClass
+  static void setUp() {
+    sh.exec("rm -rf /tmp/sqooptest*")
+    HsqlProperties p = new HsqlProperties();
+    p.setProperty("server.database.0", "file:/tmp/sqooptest");
+    // just use default.
+    p.setProperty("server.dbname.0", "sqooptest");
+    p.setProperty("hsqldb.default_table_type", "TEXT");
+
+    server = new Server();
+    server.setProperties(p);
+    server.setLogWriter(new PrintWriter(System.out, true));
+    // can use custom writer
+    server.setErrWriter(new PrintWriter(System.out, true));
+    // can use custom writer
+    server.start();
+
+  }
+
+  final static String DB_DRIVER = "org.hsqldb.jdbcDriver";
+  static String DB_CONNECTION =
+      "jdbc:hsqldb:hsql://localhost:9001/sqooptest";
+  static String DB_USER = "sa";
+  static String DB_PASSWORD = "";
+
+
+  void execDB(String query) {
+    Connection dbConnection = null;
+    Statement statement = null;
+    try {
+      dbConnection = getDBConnection();
+      statement = dbConnection.createStatement();
+      statement.execute(query);
+    }
+    catch (Exception e) {
+      e.printStackTrace();
+      Assert.fail(e.getMessage() + "")
+    }
+    finally {
+      if (statement != null) statement.close();
+      if (dbConnection != null) dbConnection.close();
+    }
+
+  }
+
+
+  String hsql = "/usr/lib/sqoop/lib/hsqldb-1.8.0.10.jar"
+  String port = "9001"
+
+  @Test
+  void test() {
+    createDB()
+    sh.exec("sqoop import --libjars /usr/lib/sqoop/lib/hsqldb-1.8.0.10.jar --connect \"jdbc:hsqldb:hsql://localhost:9001/sqooptest\" --username \"SA\" --query \'select ANIMALS.* from ANIMALS WHERE \$CONDITIONS\' --split-by ANIMALS.id --target-dir /tmp/sqooptest")
+  }
+
+  void createDB() {
+    String createTableSQL =
+        """
+        CREATE TABLE ANIMALS(
+         NAME VARCHAR(20) NOT NULL,
+         COUNTRY VARCHAR(20) NOT NULL,
+         ID INTEGER NOT NULL)
+	"""
+
+    execDB(createTableSQL);
+    LOG.info("DONE CREATING ANIMALS DB")
+
+    for (i in 1..300) {
+      execDB("INSERT INTO ANIMALS (NAME,COUNTRY,ID) VALUES (\'ELEPHANT\',\'SOUTH AFRICA\',1)");
+    }
+    execDB("INSERT INTO ANIMALS (NAME,COUNTRY,ID) VALUES (\'ELEPHANT\',\'SOUTH AFRICA\',1)");
+    execDB("INSERT INTO ANIMALS (NAME,COUNTRY,ID) VALUES (\'KOALA\', \'AUSTRALIA\', 2)");
+    execDB("INSERT INTO ANIMALS (NAME,COUNTRY,ID) VALUES (\'ZEBRA\', \'USA\', 3)");
+
+    LOG.info("DONE inserting 3 animals");
+  }
+
+  private static Connection getDBConnection() {
+    Connection dbConnection = null;
+    try {
+      Class.forName(DB_DRIVER);
+    }
+    catch (ClassNotFoundException e) {
+      LOG.info(e.getMessage());
+      Assert.fail("Driver not found ${DB_DRIVER}")
+    }
+    try {
+      dbConnection = DriverManager.
+          getConnection(DB_CONNECTION, DB_USER, DB_PASSWORD);
+      return dbConnection;
+    }
+    catch (Exception e) {
+      Assert.fail("Couldnt get connection ")
+    }
+    return dbConnection;
+  }
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/smoke-tests/sqoop/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/sqoop/build.gradle b/bigtop-tests/smoke-tests/sqoop/build.gradle
new file mode 100644
index 0000000..35f4abd
--- /dev/null
+++ b/bigtop-tests/smoke-tests/sqoop/build.gradle
@@ -0,0 +1,57 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+
+apply plugin: 'groovy'
+
+repositories {
+  mavenCentral()
+}
+
+dependencies {
+  //needed to avoid groovy not on classpath error.
+  testCompile module('org.codehaus.groovy:groovy:1.8.0')
+  testCompile group: 'org.apache.bigtop.itest', name: 'itest-common', version: itestVersion, transitive: 'true'
+  testCompile group: 'org.apache.hadoop', name: 'hadoop-common', version: hadoopVersion, transitive: 'true'
+  testCompile group: 'org.apache.hadoop', name: 'hadoop-hdfs', version: hadoopVersion, transitive: 'true'
+  testCompile group: 'org.hsqldb', name: 'hsqldb', version: '1.8.0.10'
+}
+
+
+def tests_to_include() {
+  return [
+      "TestSqoopETLHsql.groovy"
+  ];
+}
+
+sourceSets {
+  test {
+    resources {
+      srcDirs = [
+          'conf/'
+      ]
+    }
+    groovy {
+      srcDirs = ["./"]
+      exclude { FileTreeElement elem -> (doExclude(elem.getName())) }
+    }
+  }
+}
+
+test.doFirst {
+  checkEnv(["SQOOP_HOME"])
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/smoke-tests/sqoop/log4j.properties
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/sqoop/log4j.properties b/bigtop-tests/smoke-tests/sqoop/log4j.properties
new file mode 100644
index 0000000..02d115f
--- /dev/null
+++ b/bigtop-tests/smoke-tests/sqoop/log4j.properties
@@ -0,0 +1,9 @@
+# Set root logger level to DEBUG and its only appender to A1.
+log4j.rootLogger=TRACE, A1
+
+# A1 is set to be a ConsoleAppender.
+log4j.appender.A1=org.apache.log4j.ConsoleAppender
+
+# A1 uses PatternLayout.
+log4j.appender.A1.layout=org.apache.log4j.PatternLayout
+log4j.appender.A1.layout.ConversionPattern=%-4r [BIGTOP-TEST-LOG %t] %-5p %c %x --- %m%n

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/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 1afdaf8..6d290ef 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
@@ -19,6 +19,7 @@
 package org.apache.bigtop.itest.hadoop.mapreduce
 
 import org.junit.BeforeClass
+import org.junit.AfterClass
 import static org.junit.Assert.assertNotNull
 import org.apache.bigtop.itest.shell.Shell
 import static org.junit.Assert.assertTrue
@@ -35,7 +36,7 @@ import org.junit.runner.RunWith
 
 @RunWith(OrderedParameterized.class)
 class TestHadoopExamples {
-  static private Log LOG = LogFactory.getLog(TestHadoopExamples.class);
+  static private Log LOG = LogFactory.getLog(Object.class);
 
   private static final String HADOOP_MAPRED_HOME = System.getenv('HADOOP_MAPRED_HOME');
   private static final String HADOOP_CONF_DIR = System.getenv('HADOOP_CONF_DIR');
@@ -52,20 +53,54 @@ class TestHadoopExamples {
     HADOOP_MAPRED_HOME + "/" + hadoopExamplesJar;
 
   static Shell sh = new Shell("/bin/bash -s");
+
+  /**
+  * Public so that we can run these tests as scripts
+  * and the scripts can manually copy resoruces into DFS
+  * See BIGTOP-1222 for example.
+  */
+  public static final String SOURCE ="bigtop-tests/test-artifacts/hadoop/src/main/resources/"
   private static final String EXAMPLES = "examples";
   private static final String EXAMPLES_OUT = "examples-output";
   private static Configuration conf;
 
   private static String mr_version = System.getProperty("mr.version", "mr2");
   
-    static final String RANDOMTEXTWRITER_TOTALBYTES = (mr_version == "mr1") ?
+  static final String RANDOMTEXTWRITER_TOTALBYTES = (mr_version == "mr1") ?
       "test.randomtextwrite.total_bytes" : "mapreduce.randomtextwriter.totalbytes";
 
+  @AfterClass
+  public static void tearDown() {
+    sh.exec("hadoop fs -rmr -skipTrash ${EXAMPLES}",
+            "hadoop fs -rmr -skipTrash ${EXAMPLES_OUT}");
+  }
+
+
   @BeforeClass
   static void setUp() {
     conf = new Configuration();
-    TestUtils.unpackTestResources(TestHadoopExamples.class, EXAMPLES, null, EXAMPLES_OUT);
-    assertTrue("Failed to copy TestHadoopExamples.class to FS", sh.getRet() == 0);
+    try{
+       //copy examples/ int /user/root/ and
+       //then create examples-output directory
+       TestUtils.unpackTestResources(TestHadoopExamples.class, EXAMPLES, null, EXAMPLES_OUT);
+    }
+    catch(java.lang.Throwable t){
+        LOG.info("Failed to unpack jar resources.  Attemting to use bigtop sources");
+        def source = System.getenv("BIGTOP_HOME")+"/"+SOURCE;
+
+        assertNotNull("Can't copy test input files from bigtop source dir,"+
+                      "and jar specific attempt failed also", examples);
+
+        LOG.info("MAKING DIRECTORIES ..................... ${EXAMPLES} ${EXAMPLES_OUT}");
+
+        //add the files in resources/
+        sh.exec("hadoop fs -put ${source}/*.* .");
+        //add the directories under resources (like examples/)
+        sh.exec("hadoop fs -put ${source}/${EXAMPLES} ${EXAMPLES}");
+        sh.exec("hadoop fs -mkdir -p ${EXAMPLES_OUT}");
+   }
+   sh.exec("hadoop fs -ls ${EXAMPLES}");
+   assertTrue("Failed asserting that 'examples' were created in the DFS", sh.getRet()==0);
   }
 
   static long terasortid = System.currentTimeMillis();
@@ -76,7 +111,6 @@ class TestHadoopExamples {
   //Number of maps and samples for pi
   public static String pi_maps = System.getProperty("pi_maps", "2");
   public static String pi_samples = System.getProperty("pi_samples", "1000");
-
   static LinkedHashMap examples =
     [
         pi                :"${pi_maps} ${pi_samples}",

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/mapreduce/TestHadoopSmoke.groovy
----------------------------------------------------------------------
diff --git a/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/mapreduce/TestHadoopSmoke.groovy b/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/mapreduce/TestHadoopSmoke.groovy
index 0b2970b..da31115 100644
--- a/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/mapreduce/TestHadoopSmoke.groovy
+++ b/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/mapreduce/TestHadoopSmoke.groovy
@@ -56,7 +56,7 @@ class TestHadoopSmoke {
   @BeforeClass
   static void  setUp() throws IOException {
     String[] inputFiles = ["cachedir.jar", "input.txt"];
-    TestUtils.unpackTestResources(TestHadoopSmoke.class, "${testDir}/cachefile", inputFiles, null);
+    //TestUtils.unpackTestResources(TestHadoopSmoke.class, "${testDir}/cachefile", inputFiles, null);
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/test-artifacts/hive/src/main/groovy/org/apache/bigtop/itest/hivesmoke/HiveBulkScriptExecutor.groovy
----------------------------------------------------------------------
diff --git a/bigtop-tests/test-artifacts/hive/src/main/groovy/org/apache/bigtop/itest/hivesmoke/HiveBulkScriptExecutor.groovy b/bigtop-tests/test-artifacts/hive/src/main/groovy/org/apache/bigtop/itest/hivesmoke/HiveBulkScriptExecutor.groovy
index 23444b4..a25214a 100644
--- a/bigtop-tests/test-artifacts/hive/src/main/groovy/org/apache/bigtop/itest/hivesmoke/HiveBulkScriptExecutor.groovy
+++ b/bigtop-tests/test-artifacts/hive/src/main/groovy/org/apache/bigtop/itest/hivesmoke/HiveBulkScriptExecutor.groovy
@@ -17,23 +17,39 @@
  */
 package org.apache.bigtop.itest.hivesmoke
 
+import org.apache.commons.logging.LogFactory
+import org.apache.commons.logging.Log
 import org.apache.bigtop.itest.JarContent
 import org.apache.bigtop.itest.shell.Shell
 import static junit.framework.Assert.assertEquals
 
 public class HiveBulkScriptExecutor {
+  static private Log LOG = LogFactory.getLog(Object.class);
+
   static Shell sh = new Shell("/bin/bash -s");
 
   private File scripts;
   private String location;
 
+  public static final String RESOURCES ="bigtop-tests/test-artifacts/hive/src/main/resources/"
+
   public HiveBulkScriptExecutor(String l) {
     location = l;
     scripts = new File(location);
 
     if (!scripts.exists()) {
-      JarContent.unpackJarContainer(HiveBulkScriptExecutor.class, '.' , null);
-    }
+      try{
+        JarContent.unpackJarContainer(HiveBulkScriptExecutor.class, '.' , null);
+      }
+      //BIGTOP-1222 : Support script execution.
+      catch(Throwable t){
+        LOG.info("Didnt find jar resource, copying resources locally...");
+        def resources = System.getenv("BIGTOP_HOME")+"/"+RESOURCES ;
+        sh.exec("cp -r ${resources}/* .");
+        sh.exec("ls ${l}");
+
+      }
+   }
   }
 
   public List<String> getScripts() {
@@ -47,14 +63,16 @@ public class HiveBulkScriptExecutor {
 
   public void runScript(String test, String extraArgs) {
     String l = "${location}/${test}";
+    String test_command="""diff -u <(\$F < ${l}/actual) <(\$F < ${l}/out)""" ;
     sh.exec("""
     F=cat
     if [ -f ${l}/filter ]; then
       chmod 777 ${l}/filter
       F=${l}/filter
     fi
-    hive ${extraArgs} -v -f ${l}/in > ${l}/actual && diff -u <(\$F < ${l}/actual) <(\$F < ${l}/out)
-    """);
+    hive ${extraArgs} -v -f ${l}/in > ${l}/actual && ${test_command}"""
+    ) ;
+
     assertEquals("Got unexpected output from test script ${test}",
                   0, sh.ret);
   }

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/test-artifacts/hive/src/main/groovy/org/apache/bigtop/itest/hivesmoke/TestHiveSmokeBulk.groovy
----------------------------------------------------------------------
diff --git a/bigtop-tests/test-artifacts/hive/src/main/groovy/org/apache/bigtop/itest/hivesmoke/TestHiveSmokeBulk.groovy b/bigtop-tests/test-artifacts/hive/src/main/groovy/org/apache/bigtop/itest/hivesmoke/TestHiveSmokeBulk.groovy
index 6a0bbff..60ca698 100644
--- a/bigtop-tests/test-artifacts/hive/src/main/groovy/org/apache/bigtop/itest/hivesmoke/TestHiveSmokeBulk.groovy
+++ b/bigtop-tests/test-artifacts/hive/src/main/groovy/org/apache/bigtop/itest/hivesmoke/TestHiveSmokeBulk.groovy
@@ -25,9 +25,15 @@ import org.apache.bigtop.itest.shell.Shell
 import org.apache.bigtop.itest.junit.OrderedParameterized
 import org.junit.runner.RunWith
 import org.junit.runners.Parameterized.Parameters
+import org.apache.commons.logging.LogFactory
+import org.apache.commons.logging.Log
+import static org.junit.Assert.assertTrue
 
 @RunWith(OrderedParameterized.class)
 public class TestHiveSmokeBulk {
+
+  static private Log LOG = LogFactory.getLog(Object.class);
+
   private static String test_include =
     System.getProperty("org.apache.bigtop.itest.hivesmoke.TestHiveSmokeBulk.test_include");
   private static String test_exclude =
@@ -61,6 +67,9 @@ public class TestHiveSmokeBulk {
   @BeforeClass
   public static void setUp() {
     sh.exec("hive -f ./seed.hql");
+    LOG.info(sh.getOut())
+    LOG.info(sh.getErr())
+    assertTrue("FAILED.. "+sh.getOut()+" "+sh.getErr(),sh.getRet()==0);
   }
 
   @AfterClass
@@ -71,6 +80,7 @@ public class TestHiveSmokeBulk {
 
   @Parameters
   public static Map<String, Object[]> readTestCases() {
+    LOG.info("hive includes = " + test_include);
     List<String> tests;
     if (test_include != null) {
       tests = scripts.getScripts().intersect(Arrays.asList(test_include.split(",")));
@@ -79,6 +89,7 @@ public class TestHiveSmokeBulk {
     } else {
       tests = scripts.getScripts();
     }
+    LOG.info("HIVE TESTS = " + tests);
     Map res = [:];
     tests.each {
       res[it] = ([it] as String[]);

http://git-wip-us.apache.org/repos/asf/bigtop/blob/da7ce0f3/bigtop-tests/test-artifacts/hive/src/main/resources/scripts/ql/basic/filter
----------------------------------------------------------------------
diff --git a/bigtop-tests/test-artifacts/hive/src/main/resources/scripts/ql/basic/filter b/bigtop-tests/test-artifacts/hive/src/main/resources/scripts/ql/basic/filter
index 0bba73c..e8e6135 100644
--- a/bigtop-tests/test-artifacts/hive/src/main/resources/scripts/ql/basic/filter
+++ b/bigtop-tests/test-artifacts/hive/src/main/resources/scripts/ql/basic/filter
@@ -1,4 +1,4 @@
 sed -e 's#hdfs://[^/]*/#hdfs://HADOOP/#' \
-    -e 's#Copying file:.*u.data#Copying file: u.data#' \
+    -e '/.*Copying .*u.data$/d' \
     -e '/^Deleted.*u_data$/d' \
     -e '/.*jobconf.xml:an attempt to override final parameter: mapreduce.job.end-notification.*;  Ignoring\./ d'