You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ad...@apache.org on 2019/04/24 16:40:08 UTC

[kudu] branch master updated (70c2992 -> 10ea0ce)

This is an automated email from the ASF dual-hosted git repository.

adar pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git.


    from 70c2992  [docs] KUDU-2395 Add nscd to requirements
     new f02bd5d  consensus_peers-test: SIGSEGV in TearDown when SetUp fails
     new 3707cf3  dist-test: support test result reporting
     new 10ea0ce  dist-test: support RUN_FLAKY_ONLY

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 build-support/dist_test.py                         | 58 +++++++++++++++++++---
 build-support/jenkins/build-and-test.sh            |  7 ---
 build-support/run-test.sh                          | 12 +----
 .../org/apache/kudu/gradle/DistTestTask.java       | 43 +++++++++++++---
 src/kudu/consensus/consensus_peers-test.cc         |  8 ++-
 5 files changed, 94 insertions(+), 34 deletions(-)


[kudu] 03/03: dist-test: support RUN_FLAKY_ONLY

Posted by ad...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

adar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 10ea0ce5a636a050a1207f7ab5ecf63d178683f5
Author: Adar Dembo <ad...@cloudera.com>
AuthorDate: Sun Apr 21 21:17:04 2019 -0700

    dist-test: support RUN_FLAKY_ONLY
    
    The flaky test list is somewhat short so there's no real reason for the
    tests to be distributed. However, over the years we've observed that the
    dist-test runtime environment is different enough from non-dist-test
    environments that a test may be flaky in one but not the other. Since the
    purpose of flaky test resistance is to improve the precommit experience, and
    since all precommit tests run in dist-test, the test runs that power the
    flaky test dashboard (i.e. RUN_FLAKY_ONLY=1) should follow suit and also run
    in dist-test.
    
    Change-Id: Id5a063512f82341a48031911de12c44f902a2723
    Reviewed-on: http://gerrit.cloudera.org:8080/13072
    Tested-by: Adar Dembo <ad...@cloudera.com>
    Reviewed-by: Grant Henke <gr...@apache.org>
---
 build-support/dist_test.py                         | 34 +++++++++++++++++-----
 build-support/jenkins/build-and-test.sh            |  7 -----
 .../org/apache/kudu/gradle/DistTestTask.java       |  7 ++---
 3 files changed, 30 insertions(+), 18 deletions(-)

diff --git a/build-support/dist_test.py b/build-support/dist_test.py
index b26ebc9..2cadad5 100755
--- a/build-support/dist_test.py
+++ b/build-support/dist_test.py
@@ -59,6 +59,9 @@ MAX_TASKS_PER_JOB=10000
 # of retries, so we have to subtract 1.
 FLAKY_TEST_RETRIES = int(os.environ.get('KUDU_FLAKY_TEST_ATTEMPTS', 1)) - 1
 
+# Whether to only run flaky tests or to run all tests.
+RUN_FLAKY_ONLY = int(os.environ.get('RUN_FLAKY_ONLY', 0))
+
 # Whether to retry all failed C++ tests, rather than just known flaky tests.
 # Since Java flaky tests are not reported by the test server, Java tests are
 # always retried, regardless of this value.
@@ -478,9 +481,16 @@ def run_tests(parser, options):
   Gets all of the test command lines from 'ctest', isolates them,
   creates a task list, and submits the tasks to the testing service.
   """
-  executions = get_test_executions(options.tests_regex)
+  flakies = get_flakies()
+  if RUN_FLAKY_ONLY:
+    if options.tests_regex:
+      raise Exception("Cannot use RUN_FLAKY_ONLY with --tests-regex")
+    tests_regex = "|".join(["^" + re.escape(f) for f in flakies])
+  else:
+    tests_regex = options.tests_regex
+  executions = get_test_executions(tests_regex)
   if not executions:
-    raise Exception("No matching tests found for pattern %s" % options.tests_regex)
+    raise Exception("No matching tests found for pattern %s" % tests_regex)
   if options.extra_args:
     if options.extra_args[0] == '--':
       del options.extra_args[0]
@@ -494,7 +504,7 @@ def run_tests(parser, options):
   run_isolate(staging)
   retry_all = RETRY_ALL_TESTS > 0
   create_task_json(staging,
-                   flaky_test_set=get_flakies(),
+                   flaky_test_set=flakies,
                    replicate_tasks=options.num_instances,
                    retry_all_tests=retry_all)
   submit_tasks(staging, options)
@@ -591,13 +601,21 @@ def get_gradle_cmd_line(options):
   return cmd
 
 def run_java_tests(parser, options):
-  subprocess.check_call(get_gradle_cmd_line(options),
-                        cwd=rel_to_abs("java"))
+  flakies = get_flakies()
+  cmd = get_gradle_cmd_line(options)
+  if RUN_FLAKY_ONLY:
+    for f in flakies:
+      # As per the Gradle docs[1], test classes are included by filename, so we
+      # need to convert the class names into file paths.
+      #
+      # 1. https://docs.gradle.org/current/javadoc/org/gradle/api/tasks/testing/Test.html
+      cmd.extend([ "--classes", "%s.class" % f.replace(".", "/") ])
+  subprocess.check_call(cmd, cwd=rel_to_abs("java"))
   staging = StagingDir(rel_to_abs("java/build/dist-test"))
   run_isolate(staging)
   retry_all = RETRY_ALL_TESTS > 0
   create_task_json(staging,
-                   flaky_test_set=get_flakies(),
+                   flaky_test_set=flakies,
                    replicate_tasks=1,
                    retry_all_tests=retry_all)
   submit_tasks(staging, options)
@@ -609,7 +627,9 @@ def loop_java_test(parser, options):
   if options.num_instances < 1:
     parser.error("--num-instances must be >= 1")
   cmd = get_gradle_cmd_line(options)
-  cmd.extend([ "--classes", "**/%s" % options.pattern ])
+  # Test classes are matched by filename, so unless we convert dots into forward
+  # slashes, package name prefixes will never match anything.
+  cmd.extend([ "--classes", "**/%s.class" % options.pattern.replace(".", "/") ])
   subprocess.check_call(cmd, cwd=rel_to_abs("java"))
   staging = StagingDir(rel_to_abs("java/build/dist-test"))
   run_isolate(staging)
diff --git a/build-support/jenkins/build-and-test.sh b/build-support/jenkins/build-and-test.sh
index 4eaf556..a4e5e31 100755
--- a/build-support/jenkins/build-and-test.sh
+++ b/build-support/jenkins/build-and-test.sh
@@ -223,13 +223,6 @@ if [ "$RUN_FLAKY_ONLY" == "1" -o "$KUDU_FLAKY_TEST_ATTEMPTS" -gt 1 ]; then
   fi
 
   if [ "$RUN_FLAKY_ONLY" == "1" ]; then
-    # TODO(adar): we can't yet pass the flaky test list into the dist-test
-    # machinery (in order to control which tests are executed).
-    if [ "$ENABLE_DIST_TEST" == "1" ]; then
-      echo "Distributed testing is incompatible with RUN_FLAKY_ONLY=1"
-      exit 1
-    fi
-
     test_regex=$(perl -e '
       chomp(my @lines = <>);
       print join("|", map { "^" . quotemeta($_) } @lines);
diff --git a/java/buildSrc/src/main/groovy/org/apache/kudu/gradle/DistTestTask.java b/java/buildSrc/src/main/groovy/org/apache/kudu/gradle/DistTestTask.java
index 9340b25..f05aacc 100644
--- a/java/buildSrc/src/main/groovy/org/apache/kudu/gradle/DistTestTask.java
+++ b/java/buildSrc/src/main/groovy/org/apache/kudu/gradle/DistTestTask.java
@@ -74,8 +74,7 @@ public class DistTestTask extends DefaultTask {
   private boolean collectTmpDir = false;
 
   /**
-   * Called by the build file to add test tasks to be considered for
-   * dist-tests.
+   * Called by build.gradle to add test tasks to be considered for dist-tests.
    */
   public void addTestTask(Test t) {
     testTasks.add(t);
@@ -85,7 +84,7 @@ public class DistTestTask extends DefaultTask {
           description = "Sets test class to be included, '*' is supported.")
   public DistTestTask setClassPattern(List<String> classPattern) {
     for (Test t : testTasks) {
-      // TODO: this is currently requiring a glob like **/*Foo* instead of just *Foo*
+      // TODO: this requires a glob like **/*Foo* instead of just *Foo*
       t.setIncludes(classPattern);
     }
     return this;
@@ -110,7 +109,7 @@ public class DistTestTask extends DefaultTask {
 
   @InputFiles
   public FileCollection getInputClasses() {
-    FileCollection fc = getProject().files(); // Create and empty FileCollection.
+    FileCollection fc = getProject().files(); // Create an empty FileCollection.
     for (Test t : testTasks) {
       fc = fc.plus(t.getCandidateClassFiles());
     }


[kudu] 01/03: consensus_peers-test: SIGSEGV in TearDown when SetUp fails

Posted by ad...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

adar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit f02bd5d423fce63a9288c97a58798829f52665c2
Author: Adar Dembo <ad...@cloudera.com>
AuthorDate: Tue Apr 23 21:39:15 2019 -0700

    consensus_peers-test: SIGSEGV in TearDown when SetUp fails
    
    Change-Id: I269ef42ebf4621c80b44391efabf9b6e96584294
    Reviewed-on: http://gerrit.cloudera.org:8080/13097
    Tested-by: Adar Dembo <ad...@cloudera.com>
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
    Reviewed-by: Grant Henke <gr...@apache.org>
---
 src/kudu/consensus/consensus_peers-test.cc | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)

diff --git a/src/kudu/consensus/consensus_peers-test.cc b/src/kudu/consensus/consensus_peers-test.cc
index 37427c6..73246f5 100644
--- a/src/kudu/consensus/consensus_peers-test.cc
+++ b/src/kudu/consensus/consensus_peers-test.cc
@@ -111,8 +111,12 @@ class ConsensusPeersTest : public KuduTest {
   }
 
   virtual void TearDown() OVERRIDE {
-    ASSERT_OK(log_->WaitUntilAllFlushed());
-    messenger_->Shutdown();
+    if (log_) {
+      ASSERT_OK(log_->WaitUntilAllFlushed());
+    }
+    if (messenger_) {
+      messenger_->Shutdown();
+    }
     if (raft_pool_) {
       // Make sure to drain any tasks from the pool we're using for our delayable
       // proxy before destructing the queue.


[kudu] 02/03: dist-test: support test result reporting

Posted by ad...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

adar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 3707cf33ccbcb015130de356e4e82d1f54636c6e
Author: Adar Dembo <ad...@cloudera.com>
AuthorDate: Thu Apr 18 15:40:25 2019 -0700

    dist-test: support test result reporting
    
    The goal is to be able to use dist-test in any run of build-and-test.sh,
    even those that power the flaky test dashboard.
    
    Because dist-test runs just one test per invocation, it isn't safe to disown
    the reporting process; the slave might disappear mid-report. There wasn't an
    obvious way to know that we're using dist-test from within run-test.sh, so I
    opted to stop disowning altogether. The Java result reporting is synchronous
    (and per-test rather than per-class) so this didn't seem like a big deal.
    
    I was concerned that the additional parallelism might lead to "thundering
    herd" behavior in the test result server (i.e. if all tests finished and
    reported at the same time) so I ran a couple reporting-enabled dist-tests
    looking for load spikes on the server. I didn't see any, likely because:
    - Most tests' running time varies.
    - Tests only start when dist-test slaves become available, not all at once.
    
    RUN_FLAKY_ONLY will be addressed in a follow-up.
    
    Change-Id: Ie5072e2395376e455f63dac6d3debc88526aed07
    Reviewed-on: http://gerrit.cloudera.org:8080/13068
    Tested-by: Adar Dembo <ad...@cloudera.com>
    Reviewed-by: Grant Henke <gr...@apache.org>
---
 build-support/dist_test.py                         | 24 +++++++++++++++
 build-support/run-test.sh                          | 12 +-------
 .../org/apache/kudu/gradle/DistTestTask.java       | 36 ++++++++++++++++++++--
 3 files changed, 58 insertions(+), 14 deletions(-)

diff --git a/build-support/dist_test.py b/build-support/dist_test.py
index 402b6c6..b26ebc9 100755
--- a/build-support/dist_test.py
+++ b/build-support/dist_test.py
@@ -83,6 +83,7 @@ TEST_SHARD_RE = re.compile("\.\d+$")
 
 DEPS_FOR_ALL = \
     ["build-support/stacktrace_addr2line.pl",
+     "build-support/report-test.sh",
      "build-support/run-test.sh",
      "build-support/run_dist_test.py",
      "build-support/java-home-candidates.txt",
@@ -273,6 +274,20 @@ def copy_system_library(lib):
     shutil.copy2(rel_to_abs(lib), dst)
   return dst
 
+def forward_env_var(command_list, var_name, is_required=True):
+  """
+  Extends 'command_list' with the name and value of the environment variable
+  given by 'var_name'.
+
+  Does nothing if the environment variable isn't set or is empty, unless
+  'is_required' is True, in which case an exception is raised.
+  """
+  if not var_name in os.environ or not os.environ.get(var_name):
+    if is_required:
+      raise Exception("required env variable %s is missing" % (var_name,))
+    return
+  command_list.extend(["-e", "%s=%s" % (var_name, os.environ.get(var_name))])
+
 def create_archive_input(staging, execution, dep_extractor,
                          collect_tmpdir=False):
   """
@@ -324,6 +339,15 @@ def create_archive_input(staging, execution, dep_extractor,
       continue
     command.extend(['-e', '%s=%s' % (k, v)])
 
+  # If test result reporting was requested, forward all relevant environment
+  # variables into the test process so as to enable reporting.
+  if os.environ.get('KUDU_REPORT_TEST_RESULTS', 0):
+    forward_env_var(command, 'KUDU_REPORT_TEST_RESULTS')
+    forward_env_var(command, 'BUILD_CONFIG')
+    forward_env_var(command, 'BUILD_TAG')
+    forward_env_var(command, 'GIT_REVISION')
+    forward_env_var(command, 'TEST_RESULT_SERVER', is_required=False)
+
   if collect_tmpdir:
     command += ["--collect-tmpdir"]
   command.append('--')
diff --git a/build-support/run-test.sh b/build-support/run-test.sh
index c5b9ab8..21d09d0 100755
--- a/build-support/run-test.sh
+++ b/build-support/run-test.sh
@@ -215,17 +215,7 @@ for ATTEMPT_NUMBER in $(seq 1 $TEST_EXECUTION_ATTEMPTS) ; do
 
   if [ -n "$KUDU_REPORT_TEST_RESULTS" ] && [ "$KUDU_REPORT_TEST_RESULTS" -ne 0 ]; then
     echo Reporting results
-    $SOURCE_ROOT/build-support/report-test.sh "$ABS_TEST_PATH" "$LOGFILE" "$STATUS" &
-
-    # On success, we'll do "best effort" reporting, and disown the subprocess.
-    # On failure, we want to upload the failed test log. So, in that case,
-    # wait for the report-test.sh job to finish, lest we accidentally run
-    # a test retry and upload the wrong log.
-    if [ "$STATUS" -eq "0" ]; then
-      disown
-    else
-      wait
-    fi
+    $SOURCE_ROOT/build-support/report-test.sh "$ABS_TEST_PATH" "$LOGFILE" "$STATUS"
   fi
 
   if [ "$STATUS" -eq "0" ]; then
diff --git a/java/buildSrc/src/main/groovy/org/apache/kudu/gradle/DistTestTask.java b/java/buildSrc/src/main/groovy/org/apache/kudu/gradle/DistTestTask.java
index 4d3f5db..9340b25 100644
--- a/java/buildSrc/src/main/groovy/org/apache/kudu/gradle/DistTestTask.java
+++ b/java/buildSrc/src/main/groovy/org/apache/kudu/gradle/DistTestTask.java
@@ -149,7 +149,7 @@ public class DistTestTask extends DefaultTask {
    *
    * Note: This currently fails OSX because dump_base_deps use ldd.
    */
-  List<String> getBaseDeps() throws IOException {
+  private List<String> getBaseDeps() throws IOException {
     Process proc = new ProcessBuilder(distTestBin,
         "internal",
         "dump_base_deps")
@@ -162,6 +162,35 @@ public class DistTestTask extends DefaultTask {
     }
   }
 
+  /**
+   * @return all test result reporting environment variables and their values,
+   *         in a format suitable for consumption by run_dist_test.py.
+   */
+  private List<String> getTestResultReportingEnvironmentVariables() {
+    ImmutableList.Builder<String> args = new ImmutableList.Builder<>();
+    String enabled = System.getenv("KUDU_REPORT_TEST_RESULTS");
+    if (enabled != null && Integer.parseInt(enabled) > 0) {
+      for (String ev : ImmutableList.of("KUDU_REPORT_TEST_RESULTS",
+                                        "BUILD_CONFIG",
+                                        "BUILD_TAG",
+                                        "GIT_REVISION",
+                                        "TEST_RESULT_SERVER")) {
+        String evValue = System.getenv(ev);
+        if (evValue == null || evValue.isEmpty()) {
+          if (ev.equals("TEST_RESULT_SERVER")) {
+            // This one is optional.
+            continue;
+          }
+          throw new RuntimeException(
+              String.format("Required env variable %s is missing", ev));
+        }
+        args.add("-e");
+        args.add(String.format("%s=%s", ev, evValue));
+      }
+    }
+    return args.build();
+  }
+
   private String genIsolate(Path isolateFileDir, Test test, String testClass,
                             List<String> baseDeps) throws IOException {
     Path rootDir = test.getProject().getRootDir().toPath();
@@ -200,8 +229,9 @@ public class DistTestTask extends DefaultTask {
     if (collectTmpDir) {
       cmd.add("--collect-tmpdir");
     }
-    cmd.add("--test-language=java",
-            "--",
+    cmd.add("--test-language=java");
+    cmd.addAll(getTestResultReportingEnvironmentVariables());
+    cmd.add("--",
             "-ea",
             "-cp",
             Joiner.on(":").join(classpath));