You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bigtop.apache.org by co...@apache.org on 2016/05/01 02:39:33 UTC

bigtop git commit: BIGTOP-2317. Add smoke tests for QFS

Repository: bigtop
Updated Branches:
  refs/heads/master f69597dc2 -> bfa820006


BIGTOP-2317. Add smoke tests for QFS

Signed-off-by: Konstantin Boudnik <co...@apache.org>


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

Branch: refs/heads/master
Commit: bfa820006204ddb68160a3132958e1478c2d681e
Parents: f69597d
Author: Faraaz Sareshwala <fs...@quantcast.com>
Authored: Thu Feb 11 11:53:44 2016 -0800
Committer: Konstantin Boudnik <co...@apache.org>
Committed: Sat Apr 30 17:38:18 2016 -0700

----------------------------------------------------------------------
 bigtop-deploy/vm/utils/smoke-tests.sh           | 14 ++++--
 bigtop-tests/smoke-tests/qfs/build.gradle       | 45 ++++++++++++++++++++
 .../hadoop/mapreduce/TestHadoopExamples.groovy  | 19 ++++++---
 .../hadoop/mapreduce/TestHadoopSmoke.groovy     | 17 +++++---
 4 files changed, 79 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/bfa82000/bigtop-deploy/vm/utils/smoke-tests.sh
----------------------------------------------------------------------
diff --git a/bigtop-deploy/vm/utils/smoke-tests.sh b/bigtop-deploy/vm/utils/smoke-tests.sh
index 8dac31c..93f795e 100755
--- a/bigtop-deploy/vm/utils/smoke-tests.sh
+++ b/bigtop-deploy/vm/utils/smoke-tests.sh
@@ -36,9 +36,17 @@ export SQOOP_HOME=/usr/lib/sqoop/
 export HIVE_CONF_DIR=/etc/hive/conf/
 export MAHOUT_HOME="/usr/lib/mahout"
 
-su -s /bin/bash $HCFS_USER -c '/usr/bin/hadoop fs -mkdir /user/vagrant /user/root'
-su -s /bin/bash $HCFS_USER -c 'hadoop fs -chmod 777 /user/vagrant'
-su -s /bin/bash $HCFS_USER -c 'hadoop fs -chmod 777 /user/root'
+prep() {
+    HADOOP_COMMAND=$1
+    su -s /bin/bash $HCFS_USER -c "JAVA_LIBRARY_PATH=/usr/lib/qfs $HADOOP_COMMAND fs -mkdir /user/vagrant /user/root"
+    su -s /bin/bash $HCFS_USER -c "JAVA_LIBRARY_PATH=/usr/lib/qfs $HADOOP_COMMAND fs -chmod 777 /user/vagrant"
+    su -s /bin/bash $HCFS_USER -c "JAVA_LIBRARY_PATH=/usr/lib/qfs $HADOOP_COMMAND fs -chmod 777 /user/root"
+}
+
+prep hadoop
+if [[ $SMOKE_TESTS == *"qfs"* ]]; then
+    prep hadoop-qfs
+fi
 
 if [ -f /etc/debian_version ] ; then
     apt-get -y install pig hive flume mahout sqoop

http://git-wip-us.apache.org/repos/asf/bigtop/blob/bfa82000/bigtop-tests/smoke-tests/qfs/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/qfs/build.gradle b/bigtop-tests/smoke-tests/qfs/build.gradle
new file mode 100644
index 0000000..ca25635
--- /dev/null
+++ b/bigtop-tests/smoke-tests/qfs/build.gradle
@@ -0,0 +1,45 @@
+/**
+ * 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_to_include() {
+  def smoke_tests = System.getenv('SMOKE_TESTS')
+  if (smoke_tests == null || !smoke_tests.contains("qfs")) {
+    return [];
+  }
+
+  return [
+    "TestHadoopExamples.groovy"
+  ];
+}
+
+sourceSets {
+  test {
+    groovy {
+      srcDirs = [
+        "${BIGTOP_HOME}/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/mapreduce"
+      ]
+      exclude {
+        FileTreeElement elem -> (doExclude(elem.getName()))
+      }
+    }
+  }
+}
+
+test.doFirst {
+  environment("HADOOP_COMMAND", "hadoop-qfs")
+  checkEnv(["HADOOP_CONF_DIR", "HADOOP_MAPRED_HOME"])
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/bfa82000/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 5a19945..77af19c 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
@@ -42,6 +42,11 @@ class TestHadoopExamples {
 
   private static final String HADOOP_MAPRED_HOME = System.getenv('HADOOP_MAPRED_HOME');
   private static final String HADOOP_CONF_DIR = System.getenv('HADOOP_CONF_DIR');
+
+  // The hadoop command is dynamic in order to support both hadoop over hdfs
+  // and hadoop over qfs easily.
+  private static final String HADOOP_COMMAND = System.getProperty('HADOOP_COMMAND', 'hadoop');
+
   private static String hadoopExamplesJar =
     JarContent.getJarName(HADOOP_MAPRED_HOME, 'hadoop.*examples.*.jar');
   static {
@@ -73,8 +78,8 @@ class TestHadoopExamples {
 
   @AfterClass
   public static void tearDown() {
-    sh.exec("hadoop fs -rmr -skipTrash ${EXAMPLES}",
-      "hadoop fs -rmr -skipTrash ${EXAMPLES_OUT}");
+    sh.exec("${HADOOP_COMMAND} fs -rmr -skipTrash ${EXAMPLES}",
+      "${HADOOP_COMMAND} fs -rmr -skipTrash ${EXAMPLES_OUT}");
   }
 
 
@@ -96,12 +101,12 @@ class TestHadoopExamples {
       LOG.info("MAKING DIRECTORIES ..................... ${EXAMPLES} ${EXAMPLES_OUT}");
 
       //add the files in resources/
-      sh.exec("hadoop fs -put ${source}/*.* .");
+      sh.exec("${HADOOP_COMMAND} 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_COMMAND} fs -put ${source}/${EXAMPLES} ${EXAMPLES}");
+      sh.exec("${HADOOP_COMMAND} fs -mkdir -p ${EXAMPLES_OUT}");
     }
-    sh.exec("hadoop fs -ls ${EXAMPLES}");
+    sh.exec("${HADOOP_COMMAND} fs -ls ${EXAMPLES}");
     assertTrue("Failed asserting that 'examples' were created in the DFS", sh.getRet() == 0);
   }
 
@@ -153,7 +158,7 @@ class TestHadoopExamples {
       || FailureVars.instance.getNetworkShutdown()) {
       runFailureThread();
     }
-    sh.exec("hadoop jar $testJar $testName $testArgs");
+    sh.exec("${HADOOP_COMMAND} jar $testJar $testName $testArgs");
     assertTrue("Example $testName $testJar $testName $testArgs failed", sh.getRet() == 0);
   }
 

http://git-wip-us.apache.org/repos/asf/bigtop/blob/bfa82000/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 63998ac..118bbb6 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
@@ -36,6 +36,11 @@ class TestHadoopSmoke {
   static String hadoopHome = System.getProperty('HADOOP_HOME', '/usr/lib/hadoop')
   static String streamingHome = System.getenv('STREAMING_HOME')
   static String hadoopMapReduceHome = System.getProperty('HADOOP_MAPRED_HOME', '/usr/lib/hadoop-mapreduce')
+
+  // The hadoop command is dynamic in order to support both hadoop over hdfs
+  // and hadoop over qfs easily.
+  private static final String HADOOP_COMMAND = System.getProperty('HADOOP_COMMAND', 'hadoop');
+
   static final String STREAMING_HOME =
     (streamingHome == null) ? hadoopMapReduceHome : streamingHome;
   static String streaming_jar =
@@ -47,7 +52,7 @@ class TestHadoopSmoke {
   static String testDir = "test.hadoopsmoke." + (new Date().getTime())
   static String nn = (new Configuration()).get(DFSConfigKeys.FS_DEFAULT_NAME_KEY)
 
-  String cmd = "hadoop jar ${STREAMING_JAR}" +
+  String cmd = "${HADOOP_COMMAND} jar ${STREAMING_JAR}" +
     " -D mapred.map.tasks=1 -D mapred.reduce.tasks=1 -D mapred.job.name=Experiment"
   String cmd2 = " -input ${testDir}/cachefile/input.txt -mapper map.sh -file map.sh -reducer cat" +
     " -output ${testDir}/cachefile/out -verbose"
@@ -65,15 +70,15 @@ class TestHadoopSmoke {
 
   @AfterClass
   static void tearDown() {
-    sh.exec("hadoop fs -rmr -skipTrash ${testDir}")
+    sh.exec("${HADOOP_COMMAND} fs -rmr -skipTrash ${testDir}")
   }
 
   @Test (timeout = 0x810000l)
   void testCacheArchive() {
-    sh.exec("hadoop fs -rmr ${testDir}/cachefile/out",
+    sh.exec("${HADOOP_COMMAND} fs -rmr ${testDir}/cachefile/out",
       cmd + ' -cacheArchive ' + arg + cmd2)
     logError(sh)
-    sh.exec("hadoop fs -cat ${testDir}/cachefile/out/part-00000")
+    sh.exec("${HADOOP_COMMAND} fs -cat ${testDir}/cachefile/out/part-00000")
     logError(sh)
 
     assertEquals("cache1\t\ncache2\t", sh.out.join('\n'))
@@ -81,10 +86,10 @@ class TestHadoopSmoke {
 
   @Test (timeout = 0x810000l)
   void testArchives() {
-    sh.exec("hadoop fs -rmr ${testDir}/cachefile/out",
+    sh.exec("${HADOOP_COMMAND} fs -rmr ${testDir}/cachefile/out",
       cmd + ' -archives ' + arg + cmd2)
     logError(sh)
-    sh.exec("hadoop fs -cat ${testDir}/cachefile/out/part-00000")
+    sh.exec("${HADOOP_COMMAND} fs -cat ${testDir}/cachefile/out/part-00000")
     logError(sh)
 
     assertEquals("cache1\t\ncache2\t", sh.out.join('\n'))