You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by jo...@apache.org on 2015/10/19 07:45:53 UTC

spark git commit: [SPARK-7018][BUILD] Refactor dev/run-tests-jenkins into Python

Repository: spark
Updated Branches:
  refs/heads/master 94c8fef29 -> d3180c25d


[SPARK-7018][BUILD] Refactor dev/run-tests-jenkins into Python

This commit refactors the `run-tests-jenkins` script into Python. This refactoring was done by brennonyork in #7401; this PR contains a few minor edits from joshrosen in order to bring it up to date with other recent changes.

>From the original PR description (by brennonyork):

Currently a few things are left out that, could and I think should, be smaller JIRA's after this.

1. There are still a few areas where we use environment variables where we don't need to (like `CURRENT_BLOCK`). I might get around to fixing this one in lieu of everything else, but wanted to point that out.
2. The PR tests are still written in bash. I opted to not change those and just rewrite the runner into Python. This is a great follow-on JIRA IMO.
3. All of the linting scripts are still in bash as well and would likely do to just add those in as follow-on JIRA's as well.

Closes #7401.

Author: Brennon York <br...@capitalone.com>

Closes #9161 from JoshRosen/run-tests-jenkins-refactoring.


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

Branch: refs/heads/master
Commit: d3180c25d8cf0899a7238e7d24b35c5ae918cc1d
Parents: 94c8fef
Author: Brennon York <br...@capitalone.com>
Authored: Sun Oct 18 22:45:14 2015 -0700
Committer: Josh Rosen <jo...@databricks.com>
Committed: Sun Oct 18 22:45:27 2015 -0700

----------------------------------------------------------------------
 dev/lint-python                    |   2 +-
 dev/run-tests-codes.sh             |  30 -----
 dev/run-tests-jenkins              | 204 +---------------------------
 dev/run-tests-jenkins.py           | 228 ++++++++++++++++++++++++++++++++
 dev/run-tests.py                   |  20 +--
 dev/sparktestsupport/__init__.py   |  14 ++
 dev/sparktestsupport/shellutils.py |  37 +++++-
 python/run-tests.py                |  19 +--
 8 files changed, 285 insertions(+), 269 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/d3180c25/dev/lint-python
----------------------------------------------------------------------
diff --git a/dev/lint-python b/dev/lint-python
index 575dbb0..0b97213 100755
--- a/dev/lint-python
+++ b/dev/lint-python
@@ -20,7 +20,7 @@
 SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )"
 SPARK_ROOT_DIR="$(dirname "$SCRIPT_DIR")"
 PATHS_TO_CHECK="./python/pyspark/ ./ec2/spark_ec2.py ./examples/src/main/python/ ./dev/sparktestsupport"
-PATHS_TO_CHECK="$PATHS_TO_CHECK ./dev/run-tests.py ./python/run-tests.py"
+PATHS_TO_CHECK="$PATHS_TO_CHECK ./dev/run-tests.py ./python/run-tests.py ./dev/run-tests-jenkins.py"
 PEP8_REPORT_PATH="$SPARK_ROOT_DIR/dev/pep8-report.txt"
 PYLINT_REPORT_PATH="$SPARK_ROOT_DIR/dev/pylint-report.txt"
 PYLINT_INSTALL_INFO="$SPARK_ROOT_DIR/dev/pylint-info.txt"

http://git-wip-us.apache.org/repos/asf/spark/blob/d3180c25/dev/run-tests-codes.sh
----------------------------------------------------------------------
diff --git a/dev/run-tests-codes.sh b/dev/run-tests-codes.sh
deleted file mode 100644
index 1f16790..0000000
--- a/dev/run-tests-codes.sh
+++ /dev/null
@@ -1,30 +0,0 @@
-#!/usr/bin/env bash
-
-#
-# 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
-#
-#    http://www.apache.org/licenses/LICENSE-2.0
-#
-# 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.
-#
-
-readonly BLOCK_GENERAL=10
-readonly BLOCK_RAT=11
-readonly BLOCK_SCALA_STYLE=12
-readonly BLOCK_PYTHON_STYLE=13
-readonly BLOCK_R_STYLE=14
-readonly BLOCK_DOCUMENTATION=15
-readonly BLOCK_BUILD=16
-readonly BLOCK_MIMA=17
-readonly BLOCK_SPARK_UNIT_TESTS=18
-readonly BLOCK_PYSPARK_UNIT_TESTS=19
-readonly BLOCK_SPARKR_UNIT_TESTS=20

http://git-wip-us.apache.org/repos/asf/spark/blob/d3180c25/dev/run-tests-jenkins
----------------------------------------------------------------------
diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins
index d3b05fa..e79accf 100755
--- a/dev/run-tests-jenkins
+++ b/dev/run-tests-jenkins
@@ -22,207 +22,7 @@
 # Environment variables are populated by the code here:
 #+ https://github.com/jenkinsci/ghprb-plugin/blob/master/src/main/java/org/jenkinsci/plugins/ghprb/GhprbTrigger.java#L139
 
-# Go to the Spark project root directory
-FWDIR="$(cd `dirname $0`/..; pwd)"
+FWDIR="$(cd "`dirname $0`"/..; pwd)"
 cd "$FWDIR"
 
-source "$FWDIR/dev/run-tests-codes.sh"
-
-COMMENTS_URL="https://api.github.com/repos/apache/spark/issues/$ghprbPullId/comments"
-PULL_REQUEST_URL="https://github.com/apache/spark/pull/$ghprbPullId"
-
-# Important Environment Variables
-# ---
-# $ghprbActualCommit
-#+  This is the hash of the most recent commit in the PR.
-#+  The merge-base of this and master is the commit from which the PR was branched.
-# $sha1
-#+  If the patch merges cleanly, this is a reference to the merge commit hash
-#+    (e.g. "origin/pr/2606/merge").
-#+  If the patch does not merge cleanly, it is equal to $ghprbActualCommit.
-#+  The merge-base of this and master in the case of a clean merge is the most recent commit
-#+    against master.
-
-COMMIT_URL="https://github.com/apache/spark/commit/${ghprbActualCommit}"
-# GitHub doesn't auto-link short hashes when submitted via the API, unfortunately. :(
-SHORT_COMMIT_HASH="${ghprbActualCommit:0:7}"
-
-# format: http://linux.die.net/man/1/timeout
-# must be less than the timeout configured on Jenkins (currently 300m)
-TESTS_TIMEOUT="250m"
-
-# Array to capture all tests to run on the pull request. These tests are held under the
-#+ dev/tests/ directory.
-#
-# To write a PR test:
-#+  * the file must reside within the dev/tests directory
-#+  * be an executable bash script
-#+  * accept three arguments on the command line, the first being the Github PR long commit
-#+    hash, the second the Github SHA1 hash, and the final the current PR hash
-#+  * and, lastly, return string output to be included in the pr message output that will
-#+    be posted to Github
-PR_TESTS=(
-  "pr_merge_ability"
-  "pr_public_classes"
-# DISABLED (pwendell) "pr_new_dependencies"
-)
-
-function post_message () {
-  local message=$1
-  local data="{\"body\": \"$message\"}"
-  local HTTP_CODE_HEADER="HTTP Response Code: "
-
-  echo "Attempting to post to Github..."
-
-  local curl_output=$(
-    curl `#--dump-header -` \
-      --silent \
-      --user x-oauth-basic:$GITHUB_OAUTH_KEY \
-      --request POST \
-      --data "$data" \
-      --write-out "${HTTP_CODE_HEADER}%{http_code}\n" \
-      --header "Content-Type: application/json" \
-      "$COMMENTS_URL" #> /dev/null #| "$FWDIR/dev/jq" .id #| head -n 8
-  )
-  local curl_status=${PIPESTATUS[0]}
-
-  if [ "$curl_status" -ne 0 ]; then
-      echo "Failed to post message to GitHub." >&2
-      echo " > curl_status: ${curl_status}" >&2
-      echo " > curl_output: ${curl_output}" >&2
-      echo " > data: ${data}" >&2
-      # exit $curl_status
-  fi
-
-  local api_response=$(
-    echo "${curl_output}" \
-    | grep -v -e "^${HTTP_CODE_HEADER}"
-  )
-
-  local http_code=$(
-    echo "${curl_output}" \
-    | grep -e "^${HTTP_CODE_HEADER}" \
-    | sed -r -e "s/^${HTTP_CODE_HEADER}//g"
-  )
-
-  if [ -n "$http_code" ] && [ "$http_code" -ne "201" ]; then
-      echo " > http_code: ${http_code}." >&2
-      echo " > api_response: ${api_response}" >&2
-      echo " > data: ${data}" >&2
-  fi
-
-  if [ "$curl_status" -eq 0 ] && [ "$http_code" -eq "201" ]; then
-    echo " > Post successful."
-  fi
-}
-
-# post start message
-{
-  start_message="\
-  [Test build ${BUILD_DISPLAY_NAME} has started](${BUILD_URL}consoleFull) for \
-  PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL})."
-
-  post_message "$start_message"
-}
-
-# Environment variable to capture PR test output
-pr_message=""
-# Ensure we save off the current HEAD to revert to
-current_pr_head="`git rev-parse HEAD`"
-
-echo "HEAD:  `git rev-parse HEAD`"
-echo "\$ghprbActualCommit: $ghprbActualCommit"
-echo "\$sha1:  $sha1"
-echo "\$ghprbPullTitle: $ghprbPullTitle"
-
-# Run pull request tests
-for t in "${PR_TESTS[@]}"; do
-  this_test="${FWDIR}/dev/tests/${t}.sh"
-  # Ensure the test can be found and is a file
-  if [ -f "${this_test}" ]; then
-    echo "Running test: $t"
-    this_mssg="$(bash "${this_test}" "${ghprbActualCommit}" "${sha1}" "${current_pr_head}")"
-    # Check if this is the merge test as we submit that note *before* and *after*
-    # the tests run
-    [ "$t" == "pr_merge_ability" ] && merge_note="${this_mssg}"
-    pr_message="${pr_message}\n${this_mssg}"
-    # Ensure, after each test, that we're back on the current PR
-    git checkout -f "${current_pr_head}" &>/dev/null
-  else
-    echo "Cannot find test ${this_test}."
-  fi
-done
-
-# run tests
-{
-  # Marks this build is a pull request build.
-  export AMP_JENKINS_PRB=true
-  if [[ $ghprbPullTitle == *"test-maven"* ]]; then
-    export AMPLAB_JENKINS_BUILD_TOOL="maven"
-  fi
-  if [[ $ghprbPullTitle == *"test-hadoop1.0"* ]]; then
-    export AMPLAB_JENKINS_BUILD_PROFILE="hadoop1.0"
-  elif [[ $ghprbPullTitle == *"test-hadoop2.0"* ]]; then
-    export AMPLAB_JENKINS_BUILD_PROFILE="hadoop2.0"
-  elif [[ $ghprbPullTitle == *"test-hadoop2.2"* ]]; then
-    export AMPLAB_JENKINS_BUILD_PROFILE="hadoop2.2"
-  elif [[ $ghprbPullTitle == *"test-hadoop2.3"* ]]; then
-    export AMPLAB_JENKINS_BUILD_PROFILE="hadoop2.3"
-  fi
-
-  timeout "${TESTS_TIMEOUT}" ./dev/run-tests
-  test_result="$?"
-
-  if [ "$test_result" -eq "124" ]; then
-    fail_message="**[Test build ${BUILD_DISPLAY_NAME} timed out](${BUILD_URL}console)** \
-    for PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL}) \
-    after a configured wait of \`${TESTS_TIMEOUT}\`."
-
-    post_message "$fail_message"
-    exit $test_result
-  elif [ "$test_result" -eq "0" ]; then
-    test_result_note=" * This patch **passes all tests**."
-  else
-    if [ "$test_result" -eq "$BLOCK_GENERAL" ]; then
-      failing_test="some tests"
-    elif [ "$test_result" -eq "$BLOCK_RAT" ]; then
-      failing_test="RAT tests"
-    elif [ "$test_result" -eq "$BLOCK_SCALA_STYLE" ]; then
-      failing_test="Scala style tests"
-    elif [ "$test_result" -eq "$BLOCK_PYTHON_STYLE" ]; then
-      failing_test="Python style tests"
-    elif [ "$test_result" -eq "$BLOCK_R_STYLE" ]; then
-      failing_test="R style tests"
-    elif [ "$test_result" -eq "$BLOCK_DOCUMENTATION" ]; then
-      failing_test="to generate documentation"
-    elif [ "$test_result" -eq "$BLOCK_BUILD" ]; then
-      failing_test="to build"
-    elif [ "$test_result" -eq "$BLOCK_MIMA" ]; then
-      failing_test="MiMa tests"
-    elif [ "$test_result" -eq "$BLOCK_SPARK_UNIT_TESTS" ]; then
-      failing_test="Spark unit tests"
-    elif [ "$test_result" -eq "$BLOCK_PYSPARK_UNIT_TESTS" ]; then
-      failing_test="PySpark unit tests"
-    elif [ "$test_result" -eq "$BLOCK_SPARKR_UNIT_TESTS" ]; then
-      failing_test="SparkR unit tests"
-    else
-      failing_test="some tests"
-    fi
-
-    test_result_note=" * This patch **fails $failing_test**."
-  fi
-}
-
-# post end message
-{
-  result_message="\
-  [Test build ${BUILD_DISPLAY_NAME} has finished](${BUILD_URL}console) for \
-  PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL})."
-
-  result_message="${result_message}\n${test_result_note}"
-  result_message="${result_message}${pr_message}"
-
-  post_message "$result_message"
-}
-
-exit $test_result
+exec python -u ./dev/run-tests-jenkins.py "$@"

http://git-wip-us.apache.org/repos/asf/spark/blob/d3180c25/dev/run-tests-jenkins.py
----------------------------------------------------------------------
diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py
new file mode 100755
index 0000000..6230043
--- /dev/null
+++ b/dev/run-tests-jenkins.py
@@ -0,0 +1,228 @@
+#!/usr/bin/env python2
+
+#
+# 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
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# 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.
+#
+
+from __future__ import print_function
+import os
+import sys
+import json
+import urllib2
+import functools
+import subprocess
+
+from sparktestsupport import SPARK_HOME, ERROR_CODES
+from sparktestsupport.shellutils import run_cmd
+
+
+def print_err(msg):
+    """
+    Given a set of arguments, will print them to the STDERR stream
+    """
+    print(msg, file=sys.stderr)
+
+
+def post_message_to_github(msg, ghprb_pull_id):
+    print("Attempting to post to Github...")
+
+    url = "https://api.github.com/repos/apache/spark/issues/" + ghprb_pull_id + "/comments"
+    github_oauth_key = os.environ["GITHUB_OAUTH_KEY"]
+
+    posted_message = json.dumps({"body": msg})
+    request = urllib2.Request(url,
+                              headers={
+                                  "Authorization": "token %s" % github_oauth_key,
+                                  "Content-Type": "application/json"
+                              },
+                              data=posted_message)
+    try:
+        response = urllib2.urlopen(request)
+
+        if response.getcode() == 201:
+            print(" > Post successful.")
+    except urllib2.HTTPError as http_e:
+        print_err("Failed to post message to Github.")
+        print_err(" > http_code: %s" % http_e.code)
+        print_err(" > api_response: %s" % http_e.read())
+        print_err(" > data: %s" % posted_message)
+    except urllib2.URLError as url_e:
+        print_err("Failed to post message to Github.")
+        print_err(" > urllib2_status: %s" % url_e.reason[1])
+        print_err(" > data: %s" % posted_message)
+
+
+def pr_message(build_display_name,
+               build_url,
+               ghprb_pull_id,
+               short_commit_hash,
+               commit_url,
+               msg,
+               post_msg=''):
+    # align the arguments properly for string formatting
+    str_args = (build_display_name,
+                msg,
+                build_url,
+                ghprb_pull_id,
+                short_commit_hash,
+                commit_url,
+                str(' ' + post_msg + '.') if post_msg else '.')
+    return '**[Test build %s %s](%sconsoleFull)** for PR %s at commit [`%s`](%s)%s' % str_args
+
+
+def run_pr_checks(pr_tests, ghprb_actual_commit, sha1):
+    """
+    Executes a set of pull request checks to ease development and report issues with various
+    components such as style, linting, dependencies, compatibilities, etc.
+    @return a list of messages to post back to Github
+    """
+    # Ensure we save off the current HEAD to revert to
+    current_pr_head = run_cmd(['git', 'rev-parse', 'HEAD'], return_output=True).strip()
+    pr_results = list()
+
+    for pr_test in pr_tests:
+        test_name = pr_test + '.sh'
+        pr_results.append(run_cmd(['bash', os.path.join(SPARK_HOME, 'dev', 'tests', test_name),
+                                   ghprb_actual_commit, sha1],
+                                  return_output=True).rstrip())
+        # Ensure, after each test, that we're back on the current PR
+        run_cmd(['git', 'checkout', '-f', current_pr_head])
+    return pr_results
+
+
+def run_tests(tests_timeout):
+    """
+    Runs the `dev/run-tests` script and responds with the correct error message
+    under the various failure scenarios.
+    @return a tuple containing the test result code and the result note to post to Github
+    """
+
+    test_result_code = subprocess.Popen(['timeout',
+                                         tests_timeout,
+                                         os.path.join(SPARK_HOME, 'dev', 'run-tests')]).wait()
+
+    failure_note_by_errcode = {
+        1: 'executing the `dev/run-tests` script',  # error to denote run-tests script failures
+        ERROR_CODES["BLOCK_GENERAL"]: 'some tests',
+        ERROR_CODES["BLOCK_RAT"]: 'RAT tests',
+        ERROR_CODES["BLOCK_SCALA_STYLE"]: 'Scala style tests',
+        ERROR_CODES["BLOCK_PYTHON_STYLE"]: 'Python style tests',
+        ERROR_CODES["BLOCK_R_STYLE"]: 'R style tests',
+        ERROR_CODES["BLOCK_DOCUMENTATION"]: 'to generate documentation',
+        ERROR_CODES["BLOCK_BUILD"]: 'to build',
+        ERROR_CODES["BLOCK_MIMA"]: 'MiMa tests',
+        ERROR_CODES["BLOCK_SPARK_UNIT_TESTS"]: 'Spark unit tests',
+        ERROR_CODES["BLOCK_PYSPARK_UNIT_TESTS"]: 'PySpark unit tests',
+        ERROR_CODES["BLOCK_SPARKR_UNIT_TESTS"]: 'SparkR unit tests',
+        ERROR_CODES["BLOCK_TIMEOUT"]: 'from timeout after a configured wait of \`%s\`' % (
+            tests_timeout)
+    }
+
+    if test_result_code == 0:
+        test_result_note = ' * This patch passes all tests.'
+    else:
+        test_result_note = ' * This patch **fails %s**.' % failure_note_by_errcode[test_result_code]
+
+    return [test_result_code, test_result_note]
+
+
+def main():
+    # Important Environment Variables
+    # ---
+    # $ghprbActualCommit
+    #   This is the hash of the most recent commit in the PR.
+    #   The merge-base of this and master is the commit from which the PR was branched.
+    # $sha1
+    #   If the patch merges cleanly, this is a reference to the merge commit hash
+    #     (e.g. "origin/pr/2606/merge").
+    #   If the patch does not merge cleanly, it is equal to $ghprbActualCommit.
+    #   The merge-base of this and master in the case of a clean merge is the most recent commit
+    #     against master.
+    ghprb_pull_id = os.environ["ghprbPullId"]
+    ghprb_actual_commit = os.environ["ghprbActualCommit"]
+    ghprb_pull_title = os.environ["ghprbPullTitle"]
+    sha1 = os.environ["sha1"]
+
+    # Marks this build as a pull request build.
+    os.environ["AMP_JENKINS_PRB"] = "true"
+    # Switch to a Maven-based build if the PR title contains "test-maven":
+    if "test-maven" in ghprb_pull_title:
+        os.environ["AMPLAB_JENKINS_BUILD_TOOL"] = "maven"
+    # Switch the Hadoop profile based on the PR title:
+    if "test-hadoop1.0" in ghprb_pull_title:
+        os.environ["AMPLAB_JENKINS_BUILD_PROFILE"] = "hadoop1.0"
+    if "test-hadoop2.2" in ghprb_pull_title:
+        os.environ["AMPLAB_JENKINS_BUILD_PROFILE"] = "hadoop2.0"
+    if "test-hadoop2.2" in ghprb_pull_title:
+        os.environ["AMPLAB_JENKINS_BUILD_PROFILE"] = "hadoop2.2"
+    if "test-hadoop2.3" in ghprb_pull_title:
+        os.environ["AMPLAB_JENKINS_BUILD_PROFILE"] = "hadoop2.3"
+
+    build_display_name = os.environ["BUILD_DISPLAY_NAME"]
+    build_url = os.environ["BUILD_URL"]
+
+    commit_url = "https://github.com/apache/spark/commit/" + ghprb_actual_commit
+
+    # GitHub doesn't auto-link short hashes when submitted via the API, unfortunately. :(
+    short_commit_hash = ghprb_actual_commit[0:7]
+
+    # format: http://linux.die.net/man/1/timeout
+    # must be less than the timeout configured on Jenkins (currently 300m)
+    tests_timeout = "250m"
+
+    # Array to capture all test names to run on the pull request. These tests are represented
+    # by their file equivalents in the dev/tests/ directory.
+    #
+    # To write a PR test:
+    #   * the file must reside within the dev/tests directory
+    #   * be an executable bash script
+    #   * accept three arguments on the command line, the first being the Github PR long commit
+    #     hash, the second the Github SHA1 hash, and the final the current PR hash
+    #   * and, lastly, return string output to be included in the pr message output that will
+    #     be posted to Github
+    pr_tests = [
+        "pr_merge_ability",
+        "pr_public_classes"
+        # DISABLED (pwendell) "pr_new_dependencies"
+    ]
+
+    # `bind_message_base` returns a function to generate messages for Github posting
+    github_message = functools.partial(pr_message,
+                                       build_display_name,
+                                       build_url,
+                                       ghprb_pull_id,
+                                       short_commit_hash,
+                                       commit_url)
+
+    # post start message
+    post_message_to_github(github_message('has started'), ghprb_pull_id)
+
+    pr_check_results = run_pr_checks(pr_tests, ghprb_actual_commit, sha1)
+
+    test_result_code, test_result_note = run_tests(tests_timeout)
+
+    # post end message
+    result_message = github_message('has finished')
+    result_message += '\n' + test_result_note + '\n'
+    result_message += '\n'.join(pr_check_results)
+
+    post_message_to_github(result_message, ghprb_pull_id)
+
+    sys.exit(test_result_code)
+
+
+if __name__ == "__main__":
+    main()

http://git-wip-us.apache.org/repos/asf/spark/blob/d3180c25/dev/run-tests.py
----------------------------------------------------------------------
diff --git a/dev/run-tests.py b/dev/run-tests.py
index d4d6880..6b4b710 100755
--- a/dev/run-tests.py
+++ b/dev/run-tests.py
@@ -27,10 +27,11 @@ import sys
 import subprocess
 from collections import namedtuple
 
-from sparktestsupport import SPARK_HOME, USER_HOME
+from sparktestsupport import SPARK_HOME, USER_HOME, ERROR_CODES
 from sparktestsupport.shellutils import exit_from_command_with_retcode, run_cmd, rm_r, which
 import sparktestsupport.modules as modules
 
+
 # -------------------------------------------------------------------------------------------------
 # Functions for traversing module dependency graph
 # -------------------------------------------------------------------------------------------------
@@ -130,19 +131,6 @@ def determine_tags_to_exclude(changed_modules):
 # Functions for working with subprocesses and shell tools
 # -------------------------------------------------------------------------------------------------
 
-def get_error_codes(err_code_file):
-    """Function to retrieve all block numbers from the `run-tests-codes.sh`
-    file to maintain backwards compatibility with the `run-tests-jenkins`
-    script"""
-
-    with open(err_code_file, 'r') as f:
-        err_codes = [e.split()[1].strip().split('=')
-                     for e in f if e.startswith("readonly")]
-        return dict(err_codes)
-
-
-ERROR_CODES = get_error_codes(os.path.join(SPARK_HOME, "dev/run-tests-codes.sh"))
-
 
 def determine_java_executable():
     """Will return the path of the java executable that will be used by Spark's
@@ -191,7 +179,7 @@ def determine_java_version(java_exe):
 
 
 def set_title_and_block(title, err_block):
-    os.environ["CURRENT_BLOCK"] = ERROR_CODES[err_block]
+    os.environ["CURRENT_BLOCK"] = str(ERROR_CODES[err_block])
     line_str = '=' * 72
 
     print('')
@@ -467,7 +455,7 @@ def main():
     rm_r(os.path.join(USER_HOME, ".ivy2", "local", "org.apache.spark"))
     rm_r(os.path.join(USER_HOME, ".ivy2", "cache", "org.apache.spark"))
 
-    os.environ["CURRENT_BLOCK"] = ERROR_CODES["BLOCK_GENERAL"]
+    os.environ["CURRENT_BLOCK"] = str(ERROR_CODES["BLOCK_GENERAL"])
 
     java_exe = determine_java_executable()
 

http://git-wip-us.apache.org/repos/asf/spark/blob/d3180c25/dev/sparktestsupport/__init__.py
----------------------------------------------------------------------
diff --git a/dev/sparktestsupport/__init__.py b/dev/sparktestsupport/__init__.py
index 12696d9..8ab6d9e 100644
--- a/dev/sparktestsupport/__init__.py
+++ b/dev/sparktestsupport/__init__.py
@@ -19,3 +19,17 @@ import os
 
 SPARK_HOME = os.path.abspath(os.path.join(os.path.dirname(os.path.realpath(__file__)), "../../"))
 USER_HOME = os.environ.get("HOME")
+ERROR_CODES = {
+    "BLOCK_GENERAL": 10,
+    "BLOCK_RAT": 11,
+    "BLOCK_SCALA_STYLE": 12,
+    "BLOCK_PYTHON_STYLE": 13,
+    "BLOCK_R_STYLE": 14,
+    "BLOCK_DOCUMENTATION": 15,
+    "BLOCK_BUILD": 16,
+    "BLOCK_MIMA": 17,
+    "BLOCK_SPARK_UNIT_TESTS": 18,
+    "BLOCK_PYSPARK_UNIT_TESTS": 19,
+    "BLOCK_SPARKR_UNIT_TESTS": 20,
+    "BLOCK_TIMEOUT": 124
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d3180c25/dev/sparktestsupport/shellutils.py
----------------------------------------------------------------------
diff --git a/dev/sparktestsupport/shellutils.py b/dev/sparktestsupport/shellutils.py
index 12bd0bf..d280e79 100644
--- a/dev/sparktestsupport/shellutils.py
+++ b/dev/sparktestsupport/shellutils.py
@@ -22,6 +22,36 @@ import subprocess
 import sys
 
 
+if sys.version_info >= (2, 7):
+    subprocess_check_output = subprocess.check_output
+    subprocess_check_call = subprocess.check_call
+else:
+    # SPARK-8763
+    # backported from subprocess module in Python 2.7
+    def subprocess_check_output(*popenargs, **kwargs):
+        if 'stdout' in kwargs:
+            raise ValueError('stdout argument not allowed, it will be overridden.')
+        process = subprocess.Popen(stdout=subprocess.PIPE, *popenargs, **kwargs)
+        output, unused_err = process.communicate()
+        retcode = process.poll()
+        if retcode:
+            cmd = kwargs.get("args")
+            if cmd is None:
+                cmd = popenargs[0]
+            raise subprocess.CalledProcessError(retcode, cmd, output=output)
+        return output
+
+    # backported from subprocess module in Python 2.7
+    def subprocess_check_call(*popenargs, **kwargs):
+        retcode = call(*popenargs, **kwargs)
+        if retcode:
+            cmd = kwargs.get("args")
+            if cmd is None:
+                cmd = popenargs[0]
+            raise CalledProcessError(retcode, cmd)
+        return 0
+
+
 def exit_from_command_with_retcode(cmd, retcode):
     print("[error] running", ' '.join(cmd), "; received return code", retcode)
     sys.exit(int(os.environ.get("CURRENT_BLOCK", 255)))
@@ -39,7 +69,7 @@ def rm_r(path):
         os.remove(path)
 
 
-def run_cmd(cmd):
+def run_cmd(cmd, return_output=False):
     """
     Given a command as a list of arguments will attempt to execute the command
     and, on failure, print an error message and exit.
@@ -48,7 +78,10 @@ def run_cmd(cmd):
     if not isinstance(cmd, list):
         cmd = cmd.split()
     try:
-        subprocess.check_call(cmd)
+        if return_output:
+            return subprocess_check_output(cmd)
+        else:
+            return subprocess_check_call(cmd)
     except subprocess.CalledProcessError as e:
         exit_from_command_with_retcode(e.cmd, e.returncode)
 

http://git-wip-us.apache.org/repos/asf/spark/blob/d3180c25/python/run-tests.py
----------------------------------------------------------------------
diff --git a/python/run-tests.py b/python/run-tests.py
index 152f5cc..f5857f8 100755
--- a/python/run-tests.py
+++ b/python/run-tests.py
@@ -31,23 +31,6 @@ if sys.version < '3':
     import Queue
 else:
     import queue as Queue
-if sys.version_info >= (2, 7):
-    subprocess_check_output = subprocess.check_output
-else:
-    # SPARK-8763
-    # backported from subprocess module in Python 2.7
-    def subprocess_check_output(*popenargs, **kwargs):
-        if 'stdout' in kwargs:
-            raise ValueError('stdout argument not allowed, it will be overridden.')
-        process = subprocess.Popen(stdout=subprocess.PIPE, *popenargs, **kwargs)
-        output, unused_err = process.communicate()
-        retcode = process.poll()
-        if retcode:
-            cmd = kwargs.get("args")
-            if cmd is None:
-                cmd = popenargs[0]
-            raise subprocess.CalledProcessError(retcode, cmd, output=output)
-        return output
 
 
 # Append `SPARK_HOME/dev` to the Python path so that we can import the sparktestsupport module
@@ -55,7 +38,7 @@ sys.path.append(os.path.join(os.path.dirname(os.path.realpath(__file__)), "../de
 
 
 from sparktestsupport import SPARK_HOME  # noqa (suppress pep8 warnings)
-from sparktestsupport.shellutils import which  # noqa
+from sparktestsupport.shellutils import which, subprocess_check_output  # noqa
 from sparktestsupport.modules import all_modules  # noqa
 
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org