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 2017/09/25 14:13:01 UTC

bigtop git commit: BIGTOP-2865: HBase smoke test implementation

Repository: bigtop
Updated Branches:
  refs/heads/master feaf5fdb2 -> fc1205604


BIGTOP-2865: HBase smoke test implementation

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

Branch: refs/heads/master
Commit: fc1205604722ae2eb17f15e68bcdb96546272fc5
Parents: feaf5fd
Author: Jun He <ju...@arm.com>
Authored: Fri Aug 4 15:52:52 2017 +0800
Committer: Konstantin Boudnik <co...@apache.org>
Committed: Mon Sep 25 10:12:50 2017 -0400

----------------------------------------------------------------------
 bigtop-tests/smoke-tests/build.gradle           |  1 +
 bigtop-tests/smoke-tests/hbase/build.gradle     | 81 ++++++++++++++++++++
 .../hbase/smoke/TestHBaseCompression.groovy     |  1 -
 .../bigtop/itest/hbase/smoke/TestHbck.groovy    | 12 +--
 .../itest/hbase/smoke/TestImportTsv.groovy      | 16 ++--
 provisioner/utils/smoke-tests.sh                |  3 +
 6 files changed, 102 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/fc120560/bigtop-tests/smoke-tests/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/build.gradle b/bigtop-tests/smoke-tests/build.gradle
index ebb79b8..c151be3 100644
--- a/bigtop-tests/smoke-tests/build.gradle
+++ b/bigtop-tests/smoke-tests/build.gradle
@@ -29,6 +29,7 @@ subprojects {
 
   ext.groovyVersion = '2.4.10'
   ext.hadoopVersion = '2.7.4'
+  ext.hbaseVersion = '1.1.9'
   ext.solrVersion = '4.6.0'
   ext.slf4jVersion = '1.6.6'
   // itest needs be greater than or equal to = 1.0.0

http://git-wip-us.apache.org/repos/asf/bigtop/blob/fc120560/bigtop-tests/smoke-tests/hbase/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/hbase/build.gradle b/bigtop-tests/smoke-tests/hbase/build.gradle
new file mode 100644
index 0000000..0ca084f
--- /dev/null
+++ b/bigtop-tests/smoke-tests/hbase/build.gradle
@@ -0,0 +1,81 @@
+/**
+ * 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() {
+  return [
+    "TestHBaseBalancer.groovy",
+    "TestHBaseCompression.groovy",
+    "TestHBaseImportExport.groovy",
+    "TestHBasePigSmoke.groovy",
+    "TestHbck.groovy",
+    "TestImportTsv.groovy",
+    "IncrementalPELoad.java",
+    "TestCopyTable.java",
+    "TestHBaseSmoke.java",
+    "TestHFileOutputFormat.java",
+    "TestLoadIncrementalHFiles.java",
+  ];
+}
+
+def junitVersion = '4.11'
+dependencies {
+  compile group: 'junit', name: 'junit', version: junitVersion, transitive: 'true'
+  compile group: 'org.apache.hbase', name: 'hbase-common',
+      version: hbaseVersion, classifier: 'tests', transitive: 'true'
+  compile group: 'org.apache.hbase', name: 'hbase-server',
+      version: hbaseVersion, classifier: 'tests', transitive: 'true'
+  compile group: 'org.apache.hadoop', name: 'hadoop-common', version: hadoopVersion, transitive: 'true'
+  compile group: 'org.apache.hadoop', name: 'hadoop-client', version: hadoopVersion, transitive: 'true'
+  compile group: 'org.apache.hbase', name: 'hbase-common', version: hbaseVersion, transitive: 'true'
+  compile group: 'org.apache.hbase', name: 'hbase-server', version: hbaseVersion, transitive: 'true'
+}
+
+sourceSets {
+  main {
+    java {
+      srcDirs = [
+        "${BIGTOP_HOME}/bigtop-tests/test-artifacts/hbase/src/main/groovy/org/apache/bigtop/itest/hbase/smoke",
+        "${BIGTOP_HOME}/bigtop-tests/test-artifacts/hbase/src/main/groovy/org/apache/bigtop/itest/hbase/system",
+        "${BIGTOP_HOME}/bigtop-tests/test-artifacts/hbase/src/main/groovy/org/apache/bigtop/itest/hbase/util"
+      ]
+    }
+  }
+  test {
+    groovy {
+      resources {
+        srcDirs = [
+            "${BIGTOP_HOME}/bigtop-tests/test-artifacts/hbase/src/main/resources",
+        ]
+      }
+      srcDirs = [
+        "${BIGTOP_HOME}/bigtop-tests/test-artifacts/hbase/src/main/groovy/org/apache/bigtop/itest/hbase/smoke"
+      ]
+      exclude {
+        FileTreeElement elem -> (doExclude(elem.getName()))
+      }
+    }
+  }
+}
+
+test.doFirst {
+  checkEnv(["HBASE_HOME", "HBASE_CONF_DIR"])
+}
+
+test {
+  // Change the default location where test data is picked up
+  systemProperty 'test.resources.dir', "${buildDir}/resources/test/"
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/fc120560/bigtop-tests/test-artifacts/hbase/src/main/groovy/org/apache/bigtop/itest/hbase/smoke/TestHBaseCompression.groovy
----------------------------------------------------------------------
diff --git a/bigtop-tests/test-artifacts/hbase/src/main/groovy/org/apache/bigtop/itest/hbase/smoke/TestHBaseCompression.groovy b/bigtop-tests/test-artifacts/hbase/src/main/groovy/org/apache/bigtop/itest/hbase/smoke/TestHBaseCompression.groovy
index fba1d77..7ef1ae1 100644
--- a/bigtop-tests/test-artifacts/hbase/src/main/groovy/org/apache/bigtop/itest/hbase/smoke/TestHBaseCompression.groovy
+++ b/bigtop-tests/test-artifacts/hbase/src/main/groovy/org/apache/bigtop/itest/hbase/smoke/TestHBaseCompression.groovy
@@ -85,7 +85,6 @@ class TestHBaseCompression {
   }
 
   @Test
-  @Ignore("BIGTOP-218")
   void testSnappyCompression() {
     _testCompression("snappy");
   }

http://git-wip-us.apache.org/repos/asf/bigtop/blob/fc120560/bigtop-tests/test-artifacts/hbase/src/main/groovy/org/apache/bigtop/itest/hbase/smoke/TestHbck.groovy
----------------------------------------------------------------------
diff --git a/bigtop-tests/test-artifacts/hbase/src/main/groovy/org/apache/bigtop/itest/hbase/smoke/TestHbck.groovy b/bigtop-tests/test-artifacts/hbase/src/main/groovy/org/apache/bigtop/itest/hbase/smoke/TestHbck.groovy
index b616450..bee9ca4 100644
--- a/bigtop-tests/test-artifacts/hbase/src/main/groovy/org/apache/bigtop/itest/hbase/smoke/TestHbck.groovy
+++ b/bigtop-tests/test-artifacts/hbase/src/main/groovy/org/apache/bigtop/itest/hbase/smoke/TestHbck.groovy
@@ -26,13 +26,15 @@ import static org.apache.bigtop.itest.LogErrorsUtils.logError
  * Validates the HBase cluster health.
  */
 public class TestHbck {
+  private static final String USER = System.properties["hcfs.root.username"]?
+			"${System.properties['hcfs.root.username']}": "hdfs";
   static Shell sh = new Shell("/bin/bash -s")
   String[] hbckCmds = [
-    "hbase hbck",
-    "hbase hbck -details",
-    "hbase hbck -timelag 120",
-    "hbase hbck -summary",
-    "hbase hbck -metaonly"
+    "HADOOP_USER_NAME=$USER hbase hbck",
+    "HADOOP_USER_NAME=$USER hbase hbck -details",
+    "HADOOP_USER_NAME=$USER hbase hbck -timelag 120",
+    "HADOOP_USER_NAME=$USER hbase hbck -summary",
+    "HADOOP_USER_NAME=$USER hbase hbck -metaonly"
   ]
 
   @Test

http://git-wip-us.apache.org/repos/asf/bigtop/blob/fc120560/bigtop-tests/test-artifacts/hbase/src/main/groovy/org/apache/bigtop/itest/hbase/smoke/TestImportTsv.groovy
----------------------------------------------------------------------
diff --git a/bigtop-tests/test-artifacts/hbase/src/main/groovy/org/apache/bigtop/itest/hbase/smoke/TestImportTsv.groovy b/bigtop-tests/test-artifacts/hbase/src/main/groovy/org/apache/bigtop/itest/hbase/smoke/TestImportTsv.groovy
index d34ab8f..2a8b4cb 100644
--- a/bigtop-tests/test-artifacts/hbase/src/main/groovy/org/apache/bigtop/itest/hbase/smoke/TestImportTsv.groovy
+++ b/bigtop-tests/test-artifacts/hbase/src/main/groovy/org/apache/bigtop/itest/hbase/smoke/TestImportTsv.groovy
@@ -51,6 +51,9 @@ public class TestImportTsv {
   private static final String OUTDIR = "/user/$USER/import_movies_output";
 
   private static final String HBASE_HOME = System.getenv("HBASE_HOME");
+  private static final String HBASE_RESOURCE = System.properties["test.resources.dir"]?
+			"${System.properties['test.resources.dir']}": ".";
+
   static {
     assertNotNull("HBASE_HOME has to be set to run this test",
       HBASE_HOME);
@@ -79,17 +82,18 @@ public class TestImportTsv {
         sh.getRet() == 0);
     }
     // load data into HDFS
-    sh.exec("hadoop fs -put movies.tsv $DATADIR1/items",
-      "hadoop fs -put movies.psv $DATADIR2/items");
+    sh.exec("hadoop fs -put $HBASE_RESOURCE/movies.tsv $DATADIR1/items",
+      "hadoop fs -put $HBASE_RESOURCE/movies.psv $DATADIR2/items");
     assertTrue("setup failed", sh.getRet() == 0);
   }
 
   @AfterClass
   public static void cleanUp() {
     // delete data and junk from HDFS
-    sh.exec("hadoop fs -rmr -skipTrash $DATADIR1",
-      "hadoop fs -rmr -skipTrash $DATADIR2",
-      "hadoop fs -rmr -skipTrash /user/$USER/partitions_*");
+    sh.exec("hadoop fs -rm -r -skipTrash $DATADIR1",
+      "hadoop fs -rm -r -skipTrash $DATADIR2",
+      "hadoop fs -rm -r -skipTrash /user/$USER/hbase*",
+      "hadoop fs -rm -r -skipTrash /user/$USER/orig_*");
     assertTrue("teardown failed", sh.getRet() == 0);
   }
 
@@ -200,7 +204,7 @@ public class TestImportTsv {
     Configuration conf = HBaseConfiguration.create();
     HBaseAdmin admin = new HBaseAdmin(conf);
     HTable table = new HTable(conf, Bytes.toBytes(tableName));
-    new File("movies.tsv").eachLine { line ->
+    new File("$HBASE_RESOURCE/movies.tsv").eachLine { line ->
       String[] tokens = line.split("\t");
       byte[] row = Bytes.toBytes(tokens[0]);
       Get g = new Get(row);

http://git-wip-us.apache.org/repos/asf/bigtop/blob/fc120560/provisioner/utils/smoke-tests.sh
----------------------------------------------------------------------
diff --git a/provisioner/utils/smoke-tests.sh b/provisioner/utils/smoke-tests.sh
index 93f795e..503d3b8 100755
--- a/provisioner/utils/smoke-tests.sh
+++ b/provisioner/utils/smoke-tests.sh
@@ -35,6 +35,9 @@ export FLUME_HOME=/usr/lib/flume/
 export SQOOP_HOME=/usr/lib/sqoop/
 export HIVE_CONF_DIR=/etc/hive/conf/
 export MAHOUT_HOME="/usr/lib/mahout"
+export HBASE_HOME=/usr/lib/hbase
+export HBASE_CONF_DIR=/usr/lib/hbase/conf
+export ZOOKEEPER_HOME=/usr/lib/zookeeper
 
 prep() {
     HADOOP_COMMAND=$1