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/01/14 22:09:14 UTC

[1/3] bigtop git commit: BIGTOP-2235. Allow smoke-tests to use filename regexps instead of explicite listings

Repository: bigtop
Updated Branches:
  refs/heads/master 8b0a6a109 -> 937a985e4


BIGTOP-2235. Allow smoke-tests to use filename regexps instead of explicite listings


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

Branch: refs/heads/master
Commit: f368f26f5edf190cef6920a10461fcfa3ba475a3
Parents: 8b0a6a1
Author: Konstantin Boudnik <co...@apache.org>
Authored: Tue Jan 12 23:08:54 2016 -0800
Committer: Konstantin Boudnik <co...@apache.org>
Committed: Thu Jan 14 13:08:16 2016 -0800

----------------------------------------------------------------------
 bigtop-tests/smoke-tests/build.gradle | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/f368f26f/bigtop-tests/smoke-tests/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/build.gradle b/bigtop-tests/smoke-tests/build.gradle
index c3bd973..b9dd63f 100644
--- a/bigtop-tests/smoke-tests/build.gradle
+++ b/bigtop-tests/smoke-tests/build.gradle
@@ -64,7 +64,7 @@ subprojects {
         filename.contains(".java")) {
       keep_this_test =
           tests_to_include().find({
-            it.contains(filename)
+            filename ==~ it
           }) != null;
     }
 


[2/3] bigtop git commit: BIGTOP-2232. Add smoke tests for HDFS

Posted by co...@apache.org.
BIGTOP-2232. Add smoke tests for HDFS


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

Branch: refs/heads/master
Commit: 98d3349b3683415e929dbecbe5526122cbc3480e
Parents: f368f26
Author: Konstantin Boudnik <co...@apache.org>
Authored: Tue Jan 12 23:10:48 2016 -0800
Committer: Konstantin Boudnik <co...@apache.org>
Committed: Thu Jan 14 13:08:17 2016 -0800

----------------------------------------------------------------------
 bigtop-tests/smoke-tests/build.gradle      |  2 +-
 bigtop-tests/smoke-tests/hdfs/build.gradle | 80 +++++++++++++++++++++++++
 2 files changed, 81 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/98d3349b/bigtop-tests/smoke-tests/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/build.gradle b/bigtop-tests/smoke-tests/build.gradle
index b9dd63f..74b7b72 100644
--- a/bigtop-tests/smoke-tests/build.gradle
+++ b/bigtop-tests/smoke-tests/build.gradle
@@ -40,9 +40,9 @@ subprojects {
   ext.BIGTOP_HOME   = rootDir
 
   dependencies {
+    compile group: 'org.apache.bigtop.itest', name: 'itest-common', version: itestVersion, transitive: 'true'
     //needed to avoid groovy not on classpath error.
     testCompile group: 'org.codehaus.groovy', name: 'groovy', version: groovyVersion
-    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'
   }

http://git-wip-us.apache.org/repos/asf/bigtop/blob/98d3349b/bigtop-tests/smoke-tests/hdfs/build.gradle
----------------------------------------------------------------------
diff --git a/bigtop-tests/smoke-tests/hdfs/build.gradle b/bigtop-tests/smoke-tests/hdfs/build.gradle
new file mode 100644
index 0000000..1d8baa2
--- /dev/null
+++ b/bigtop-tests/smoke-tests/hdfs/build.gradle
@@ -0,0 +1,80 @@
+/**
+ * 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 [
+      "TestBlockRecovery.groovy",
+      "TestDistCpIntra.groovy",
+      "TestFileAppend.groovy",
+      "TestFsck.groovy",
+      "TestHDFSQuota.groovy",
+      /* TODO need to figure out how to unpack the resources
+      "TestCmdTest.groovy",
+      "TestCmdText.groovy",
+      "TestCount.groovy",
+      "TestDFSAdmin.groovy",
+      "TestHDFSBalancer.groovy",
+
+      "TestDFSCLI.java",
+      "TestHDFSCLI.java",
+      "TestTextSnappy.groovy"
+      */
+  ];
+}
+
+def junitVersion = '4.11'
+dependencies {
+  compile group: 'junit', name: 'junit', version: junitVersion, transitive: 'true'
+  compile group: 'org.apache.hadoop', name: 'hadoop-common',
+      version: hadoopVersion, classifier: 'tests', transitive: 'true'
+  compile group: 'org.apache.hadoop', name: 'hadoop-hdfs',
+      version: hadoopVersion, classifier: 'tests', transitive: 'true'
+  compile group: 'org.apache.hadoop', name: 'hadoop-common', version: hadoopVersion, transitive: 'true'
+  compile group: 'org.apache.hadoop', name: 'hadoop-hdfs', version: hadoopVersion, transitive: 'true'
+  if (System.env.HADOOP_CONF_DIR) testRuntime files(System.env.HADOOP_CONF_DIR)
+}
+
+sourceSets {
+  main {
+    java {
+      srcDirs = [
+        "${BIGTOP_HOME}/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hcfs",
+        "${BIGTOP_HOME}/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hdfs"
+      ]
+    }
+  }
+  test {
+    groovy {
+      resources {
+        srcDirs = [
+            "${BIGTOP_HOME}/bigtop-tests/test-artifacts/hadoop/src/main/resources",
+            'conf/',
+        ]
+      }
+      srcDirs = [
+        "${BIGTOP_HOME}/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hdfs"
+      ]
+      exclude {
+        FileTreeElement elem -> (doExclude(elem.getName()))
+      }
+    }
+  }
+}
+
+test.doFirst {
+  checkEnv(["HADOOP_CONF_DIR"])
+}


[3/3] bigtop git commit: BIGTOP-2234. TestBlockRecovery incorrectly fails on a single node cluster

Posted by co...@apache.org.
BIGTOP-2234. TestBlockRecovery incorrectly fails on a single node cluster


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

Branch: refs/heads/master
Commit: 937a985e4f36758bbe22a1bf25fcb07463f7eef0
Parents: 98d3349
Author: Konstantin Boudnik <co...@apache.org>
Authored: Tue Jan 12 23:06:37 2016 -0800
Committer: Konstantin Boudnik <co...@apache.org>
Committed: Thu Jan 14 13:08:17 2016 -0800

----------------------------------------------------------------------
 .../itest/hadoop/hdfs/TestBlockRecovery.groovy  | 76 ++++++++++++--------
 1 file changed, 45 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/937a985e/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hdfs/TestBlockRecovery.groovy
----------------------------------------------------------------------
diff --git a/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hdfs/TestBlockRecovery.groovy b/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hdfs/TestBlockRecovery.groovy
index 2a3f2d9..88f20f2 100644
--- a/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hdfs/TestBlockRecovery.groovy
+++ b/bigtop-tests/test-artifacts/hadoop/src/main/groovy/org/apache/bigtop/itest/hadoop/hdfs/TestBlockRecovery.groovy
@@ -39,6 +39,9 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
  three datanodes to allow block recovery.
  Block replication must be set to a minimum value of 2
  for this test to work properly.
+
+ If passwordless setup isn't configured for user HDFS the test
+ will be skipped as well.
  */
 public class TestBlockRecovery {
 
@@ -46,6 +49,7 @@ public class TestBlockRecovery {
 
   private static Configuration conf;
 
+  private static final String SSH_HDFS_ID = "~/.ssh/id_hdfsuser"
   private static final String corruptContent = "0123456789";
   private static final String HDFS_TEST_DIR = "/tmp/TestBlockRecovery$corruptContent";
   private static final String fsFilePath = HDFS_TEST_DIR + "/file0";
@@ -64,18 +68,36 @@ public class TestBlockRecovery {
   private static String initialBlockChecksum;
   private static String fileContent;
 
-  private static def dataDirs = [];
+  private static String [] dataDirs;
   private static def nodesBeforeRecovery = [];
   private static def nodesAfterRecovery = [];
 
-  private static short numberOfDataNodes;
-  private static short repFactor;
-
-  private static final long fileLen = 10;
-  private static final long SEED = 0;
-
   @BeforeClass
   public static void setUp() {
+    shHDFS.exec("rm -rf $localTestDir");
+    shHDFS.exec("mkdir $localTestDir");
+    shHDFS.exec("hadoop fs -rm -r $fsFilePath");
+    Thread.sleep(TIMEOUT);
+    shHDFS.exec("hadoop fs -mkdir -p $HDFS_TEST_DIR && hadoop fs -chmod 777 $HDFS_TEST_DIR");
+    assertTrue("Failed to create input directory", shHDFS.getRet() == 0);
+
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    // deletion of test files
+    shHDFS.exec("rm -rf $localTestDir");
+    assertTrue("Could not delete test directory $localTestDir", shHDFS.getRet() == 0);
+  }
+
+  @Test
+  public void testBlockRecovery() {
+
+    short numberOfDataNodes;
+    short repFactor;
+
+    final long fileLen = 10;
+    final long SEED = 0;
     /* Find datanode data directory, make file, add content, ensure replication
      * is set to guarantee any chosen datanode will have block,
      * get block and its location, perform checksum before corrupting block
@@ -84,20 +106,21 @@ public class TestBlockRecovery {
     conf = new HdfsConfiguration();
     FileSystem fileSys = DistributedFileSystem.get(conf);
     conf.addResource("hdfs-site.xml");
-    dataDirs = conf.get("dfs.data.dir").split(",");
-    if (dataDirs == null)
-      dataDirs = conf.get("dfs.datanode.data.dir").split(",");
+    def confDataDirs = conf.get("dfs.data.dir")
+    if (confDataDirs == null)
+      confDataDirs = conf.get("dfs.datanode.data.dir")
+    // data.dirs might be configured w/ explicit file:// auth.
+    // Should be stripped-off
+    dataDirs = confDataDirs.split(",")*.replaceFirst(~/file:\/\//,'');
 
     numberOfDataNodes = shHDFS.exec("hdfs dfsadmin -report | grep ^Name | wc -l").getOut()[0] as short;
+    // to recover a block at least two non-corrupted replicas should exist
     Assume.assumeTrue(numberOfDataNodes >= 3);
+    // If passwordless setup isn't configured for user HDFS the test needs to
+    // be skipped
+    Assume.assumeTrue(shHDFS.exec("[ -f ${SSH_HDFS_ID} ]").getRet() == 0);
 
-    shHDFS.exec("rm -rf $localTestDir");
-    shHDFS.exec("mkdir $localTestDir");
-    shHDFS.exec("hadoop fs -rm -r $fsFilePath");
-    Thread.sleep(TIMEOUT);
-    shHDFS.exec("hadoop fs -mkdir -p $HDFS_TEST_DIR && hadoop fs -chmod 777 $HDFS_TEST_DIR");
-    assertTrue("Failed to create input directory", shHDFS.getRet() == 0);
-
+    // snapshot of everything before the corruption happens
     repFactor = (numberOfDataNodes - 1);
     try {
     	DFSTestUtil.createFile(fileSys, new Path(fsFilePath), fileLen, repFactor, SEED);
@@ -105,6 +128,7 @@ public class TestBlockRecovery {
 	assert "Exception should not be thrown"
     }
     fileContent = shHDFS.exec("hadoop fs -cat $fsFilePath").getOut()[0];
+    assertTrue("File $fsFilePath doesn't exist", shHDFS.getRet() == 0);
 
     shHDFS.exec("hdfs fsck $fsFilePath -blocks -locations -files > $outputFile");
     assertTrue("Could not write output to file", shHDFS.getRet() == 0);
@@ -125,19 +149,6 @@ public class TestBlockRecovery {
 
     initialBlockChecksum = shHDFS.exec("cksum $blockLocation").getOut()[0].split(" ")[0];
     assertTrue("Could not obtain checksum for block $blockToTest", shHDFS.getRet() == 0);
-  }
-
-  @AfterClass
-  public static void tearDown() {
-    // deletion of test files
-    shHDFS.exec("hadoop fs -rm -r -skipTrash $fsFilePath");
-    assertTrue("Could not delete file $fsFilePath", shHDFS.getRet() == 0);
-    shHDFS.exec("rm -rf $localTestDir");
-    assertTrue("Could not delete test directory $localTestDir", shHDFS.getRet() == 0);
-  }
-
-  @Test
-  public void testBlockRecovery() {
     // corrupt block
     shHDFS.exec("echo $corruptContent > $blockLocation");
     assertTrue("Could not write to file", shHDFS.getRet() == 0);
@@ -182,7 +193,7 @@ public class TestBlockRecovery {
     while (cksumAttempt < 3) {
       if (corruptBlockChecksum != initialBlockChecksum) {
         sleep(sleepTime);
-        corruptBlockChecksum = shHDFS.exec("ssh -o StrictHostKeyChecking=no -i ~/.ssh/id_hdfsuser " +
+        corruptBlockChecksum = shHDFS.exec("ssh -o StrictHostKeyChecking=no -i ${SSH_HDFS_ID} " +
           "$blockRecoveryNode 'cksum `find ${dataDirs.join(' ')}" +
           " -name $blockToTest 2>/dev/null | grep $blockToTest` '").getOut()[0].split(" ")[0];
         ++cksumAttempt;
@@ -196,6 +207,9 @@ public class TestBlockRecovery {
       }
     }
     assertTrue("Block has not recovered", success);
+    // Let's remove the garbage after the test
+    shHDFS.exec("hadoop fs -rm -r -skipTrash $fsFilePath");
+    assertTrue("Could not delete file $fsFilePath", shHDFS.getRet() == 0);
   }
 
 }