From 31b51dea681534fca28b762b6eca01b81229215c Mon Sep 17 00:00:00 2001 From: Brennon York Date: Mon, 13 Jul 2015 15:28:53 -0700 Subject: [PATCH 01/32] initial cut of refactored run-tests-jenkins script into python --- dev/run-tests-codes.sh | 29 ---- dev/run-tests-jenkins | 223 +------------------------ dev/run-tests-jenkins.py | 254 +++++++++++++++++++++++++++++ dev/run-tests.py | 16 +- dev/sparktestsupport/__init__.py | 11 ++ dev/sparktestsupport/shellutils.py | 7 +- 6 files changed, 274 insertions(+), 266 deletions(-) delete mode 100644 dev/run-tests-codes.sh create mode 100755 dev/run-tests-jenkins.py diff --git a/dev/run-tests-codes.sh b/dev/run-tests-codes.sh deleted file mode 100644 index f4b238e1b78a..000000000000 --- a/dev/run-tests-codes.sh +++ /dev/null @@ -1,29 +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_DOCUMENTATION=14 -readonly BLOCK_BUILD=15 -readonly BLOCK_MIMA=16 -readonly BLOCK_SPARK_UNIT_TESTS=17 -readonly BLOCK_PYSPARK_UNIT_TESTS=18 -readonly BLOCK_SPARKR_UNIT_TESTS=19 diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index c4d39d95d589..e79accf9e987 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -22,226 +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 180m) -TESTS_TIMEOUT="175m" - -# 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 -} - -function send_archived_logs () { - echo "Archiving unit tests logs..." - - local log_files=$( - find .\ - -name "unit-tests.log" -o\ - -path "./sql/hive/target/HiveCompatibilitySuite.failed" -o\ - -path "./sql/hive/target/HiveCompatibilitySuite.hiveFailed" -o\ - -path "./sql/hive/target/HiveCompatibilitySuite.wrong" - ) - - if [ -z "$log_files" ]; then - echo "> No log files found." >&2 - else - local log_archive="unit-tests-logs.tar.gz" - echo "$log_files" | xargs tar czf ${log_archive} - - local jenkins_build_dir=${JENKINS_HOME}/jobs/${JOB_NAME}/builds/${BUILD_NUMBER} - local scp_output=$(scp ${log_archive} amp-jenkins-master:${jenkins_build_dir}/${log_archive}) - local scp_status="$?" - - if [ "$scp_status" -ne 0 ]; then - echo "Failed to send archived unit tests logs to Jenkins master." >&2 - echo "> scp_status: ${scp_status}" >&2 - echo "> scp_output: ${scp_output}" >&2 - else - echo "> Send successful." - fi - - rm -f ${log_archive} - 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 "GHPRB: $ghprbActualCommit" -echo "SHA1: $sha1" - -# 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 - 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_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 - - send_archived_logs -} - -# 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 "$@" diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py new file mode 100755 index 000000000000..98902410e93f --- /dev/null +++ b/dev/run-tests-jenkins.py @@ -0,0 +1,254 @@ +#!/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 subprocess + +from sparktestsupport import SPARK_HOME, ERROR_CODES +from sparktestsupport.shellutils import exit_from_command_with_retcode, run_cmd, rm_r + + +def print_err(*args): + """ + Given a set of arguments, will print them to the STDERR stream + """ + print(*args, file=sys.stderr) + + +def post_message(mssg, comments_url): + http_code_header = "HTTP Response Code: " + posted_message = json.dumps({"body" : mssg}) + + print("Attempting to post to Github...") + + # we don't want to call `run_cmd` here as, in the event of an error, we DO NOT + # want to print the GITHUB_OAUTH_KEY into the public Jenkins logs + curl_proc = subprocess.Popen(['curl', + '--silent', + '--user', 'x-oauth-basic:' + os.environ['GITHUB_OATH_KEY'], + '--request', 'POST', + '--data', posted_message, + '--write-out', http_code_header + '%{http_code}', + '--header', 'Content-Type: application/json', + comments_url], + stdout=subprocess.PIPE, + stderr=subprocess.PIPE) + curl_stdout, curl_stderr = curl_proc.communicate() + curl_returncode = curl_proc.returncode + # find all lines relevant to the Github API response + api_response = "\n".join([l for l in curl_stdout.split('\n') + if l and not l.startswith(http_code_header)]) + # find the line where `http_code_header` exists, split on ':' to get the + # HTTP response code, and cast to an int + http_code = int(curl_stdout[curl_stdout.find(http_code_header):].split(':')[1]) + + if not curl_returncode == 0: + print_err("Failed to post message to GitHub.") + print_err(" > curl_status:", curl_returncode) + print_err(" > curl_output:", curl_stdout) + print_err(" > data:", posted_message) + + if http_code and not http_code == 201: + print_err(" > http_code:", http_code) + print_err(" > api_response:", api_response) + print_err(" > data:", posted_message) + + if curl_returncode == 0 and http_code == 201: + print(" > Post successful.") + + +def send_archived_logs(): + print("Archiving unit tests logs...") + + log_files = run_cmd(['find', '.', + '-name', 'unit-tests.log', + '-o', '-path', './sql/hive/target/HiveCompatibilitySuite.failed', + '-o', '-path', './sql/hive/target/HiveCompatibilitySuite.hiveFailed', + '-o', '-path', './sql/hive/target/HiveCompatibilitySuite.wrong'], + return_output = True) + + if log_files: + log_archive = "unit-tests-logs.tar.gz" + + run_cmd(['tar', 'czf', log-archive, *log_files]) + + jenkins_build_dir = os.environ["JENKINS_HOME"] + + "/jobs/" + os.environ["JOB_NAME"] + + "/builds/" + os.environ["BUILD_NUMBER"] + + scp_proc = subprocess.Popen(['scp', log_archive, + 'amp-jenkins-master:' + jenkins_build_dir + '/' + log_archive], + stdout=subprocess.PIPE, + stderr=subprocess.PIPE) + scp_stdout, scp_stderr = scp_proc.communicate() + scp_returncode = scp_proc.returncode + + if not scp_returncode == 0: + print_err("Failed to send archived unit tests logs to Jenkins master.") + print_err(" > scp_status:", scp_returncode) + print_err(" > scp_output:", scp_stdout) + else: + print(" > Send successful.") + else: + print_err(" > No log files found.") + + rm_r(log_archive) + + +def run_pr_tests(pr_tests, ghprb_actual_commit, sha1): + # 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: + pr_results.append(run_cmd(['bash', os.path.join(SPARK_HOME, 'dev', 'tests', pr_test), + ghprb_actual_commit, sha1], + return_output=True).strip()) + # Ensure, after each test, that we're back on the current PR + run_cmd(['git', 'checkout', '-f', current_pr_head]) + return pr_results + + +def bind_message_base(build_display_name, build_url, ghprb_pull_id, short_commit_hash, commit_url): + """ + Given base parameters to generate a strong Github message response, binds those + parameters into a closure without the specific message and returns a function + able to generate strong messages for a specific description. + """ + return lambda mssg, post_mssg="":\ + '**[Test build ' + build_display_name + ' ' + mssg + '](' + build_url + + 'console)** for PR ' + ghprb_pull_id + ' at commit [\`' + short_commit_hash + '\`](' + + commit_url + ')' + str(' ' + post_mssg + '.') if post_mssg else '.' + + +def success_result_note(mssg): + return ' * This patch ' + mssg + '.' + + +def failure_result_note(mssg): + return ' * This patch **fails ' + mssg + '**.' + + +def run_tests(tests_timeout): + test_proc = subprocess.Popen(['timeout', + tests_timeout, + os.path.join(SPARK_HOME, 'dev', 'run-tests')]).wait() + test_result = test_proc.returncode + + failure_note_by_errcode = { + ERROR_CODES["BLOCK_GENERAL"] : failure_result_note('some tests'), + ERROR_CODES["BLOCK_RAT"] : failure_result_note('RAT tests'), + ERROR_CODES["BLOCK_SCALA_STYLE"] : failure_result_note('Scala style tests'), + ERROR_CODES["BLOCK_PYTHON_STYLE"] : failure_result_note('Python style tests'), + ERROR_CODES["BLOCK_DOCUMENTATION"] : failure_result_note('to generate documentation'), + ERROR_CODES["BLOCK_BUILD"] : failure_result_note('to build'), + ERROR_CODES["BLOCK_MIMA"] : failure_result_note('MiMa tests'), + ERROR_CODES["BLOCK_SPARK_UNIT_TESTS"] : failure_result_note('Spark unit tests'), + ERROR_CODES["BLOCK_PYSPARK_UNIT_TESTS"] : failure_result_note('PySpark unit tests'), + ERROR_CODES["BLOCK_SPARKR_UNIT_TESTS"] : failure_result_note('SparkR unit tests'), + ERROR_CODES["BLOCK_TIMEOUT"] : failure_result_note('from timeout after a configured wait' + + ' of \`' + tests_timeout + '\`') + } + + if test_result == 0: + test_result_note = success_result_note('passes all tests') + else: + test_result_note = failure_note_by_errcode(test_result) + send_archived_logs() + + return 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"] + sha1 = os.environ["sha1"] + + # Marks this build as a pull request build. + os.environ["AMP_JENKINS_PRB"] = "true" + build_display_name = os.environ["BUILD_DISPLAY_NAME"] + build_url = os.environ["BUILD_URL"] + + comments_url = "https://api.github.com/repos/apache/spark/issues/" + ghprb_pull_id + "/comments" + pull_request_url = "https://github.com/apache/spark/pull/" + ghprb_pull_id + 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 180m) + tests_timeout = "175m" + + # 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" + ] + + # `bind_message_base` returns a function to generate messages for Github posting + github_message = bind_message_base(build_display_name, + build_url, + ghprb_pull_id, + short_commit_hash, + commit_url) + + # post start message + post_message(github_message('has started')) + + pr_test_results = run_pr_tests(pr_tests, ghprb_actual_commit, sha1) + + test_results = run_tests(tests_timeout) + + # post end message + result_message = github_message('has finished') + result_message += '\n' + test_results + for pr_result in pr_test_results: + result_message += pr_result + + post_message(result_message) + + sys.exit(test_result) + + +if __name__ == "__main__": + main() diff --git a/dev/run-tests.py b/dev/run-tests.py index 1f0d218514f9..5884884fddc7 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -26,10 +26,11 @@ 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 # ------------------------------------------------------------------------------------------------- @@ -114,19 +115,6 @@ def determine_modules_to_test(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 diff --git a/dev/sparktestsupport/__init__.py b/dev/sparktestsupport/__init__.py index 12696d98fb98..ba952f45586c 100644 --- a/dev/sparktestsupport/__init__.py +++ b/dev/sparktestsupport/__init__.py @@ -19,3 +19,14 @@ 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_DOCUMENTATION" : 14, + "BLOCK_BUILD" : 15, + "BLOCK_MIMA" : 16, + "BLOCK_SPARK_UNIT_TESTS" : 17, + "BLOCK_PYSPARK_UNIT_TESTS" : 18, + "BLOCK_SPARKR_UNIT_TESTS" : 19, + "BLOCK_TIMEOUT" : 124 } diff --git a/dev/sparktestsupport/shellutils.py b/dev/sparktestsupport/shellutils.py index 12bd0bf3a4fe..d474cc5430ee 100644 --- a/dev/sparktestsupport/shellutils.py +++ b/dev/sparktestsupport/shellutils.py @@ -39,7 +39,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 +48,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) From f2a1dc6eaf6c316809cdf08c5340a8a81de504b3 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Tue, 14 Jul 2015 11:31:36 -0700 Subject: [PATCH 02/32] fixed pep8 issues --- dev/run-tests-jenkins.py | 71 ++++++++++++++++++++-------------------- 1 file changed, 35 insertions(+), 36 deletions(-) diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 98902410e93f..908bbd48e669 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -36,7 +36,7 @@ def print_err(*args): def post_message(mssg, comments_url): http_code_header = "HTTP Response Code: " - posted_message = json.dumps({"body" : mssg}) + posted_message = json.dumps({"body": mssg}) print("Attempting to post to Github...") @@ -84,18 +84,17 @@ def send_archived_logs(): '-o', '-path', './sql/hive/target/HiveCompatibilitySuite.failed', '-o', '-path', './sql/hive/target/HiveCompatibilitySuite.hiveFailed', '-o', '-path', './sql/hive/target/HiveCompatibilitySuite.wrong'], - return_output = True) + return_output=True) if log_files: log_archive = "unit-tests-logs.tar.gz" run_cmd(['tar', 'czf', log-archive, *log_files]) - jenkins_build_dir = os.environ["JENKINS_HOME"] + - "/jobs/" + os.environ["JOB_NAME"] + - "/builds/" + os.environ["BUILD_NUMBER"] + jenkins_build_dir = os.environ["JENKINS_HOME"] + "/jobs/" + os.environ["JOB_NAME"] + + "/builds/" + os.environ["BUILD_NUMBER"] - scp_proc = subprocess.Popen(['scp', log_archive, + scp_proc = subprocess.Popen(['scp', log_archive, 'amp-jenkins-master:' + jenkins_build_dir + '/' + log_archive], stdout=subprocess.PIPE, stderr=subprocess.PIPE) @@ -136,8 +135,8 @@ def bind_message_base(build_display_name, build_url, ghprb_pull_id, short_commit """ return lambda mssg, post_mssg="":\ '**[Test build ' + build_display_name + ' ' + mssg + '](' + build_url + - 'console)** for PR ' + ghprb_pull_id + ' at commit [\`' + short_commit_hash + '\`](' + - commit_url + ')' + str(' ' + post_mssg + '.') if post_mssg else '.' +'console)** for PR ' + ghprb_pull_id + ' at commit [\`' + short_commit_hash + '\`](' + +commit_url + ')' + str(' ' + post_mssg + '.') if post_mssg else '.' def success_result_note(mssg): @@ -149,24 +148,24 @@ def failure_result_note(mssg): def run_tests(tests_timeout): - test_proc = subprocess.Popen(['timeout', - tests_timeout, + test_proc = subprocess.Popen(['timeout', + tests_timeout, os.path.join(SPARK_HOME, 'dev', 'run-tests')]).wait() test_result = test_proc.returncode failure_note_by_errcode = { - ERROR_CODES["BLOCK_GENERAL"] : failure_result_note('some tests'), - ERROR_CODES["BLOCK_RAT"] : failure_result_note('RAT tests'), - ERROR_CODES["BLOCK_SCALA_STYLE"] : failure_result_note('Scala style tests'), - ERROR_CODES["BLOCK_PYTHON_STYLE"] : failure_result_note('Python style tests'), - ERROR_CODES["BLOCK_DOCUMENTATION"] : failure_result_note('to generate documentation'), - ERROR_CODES["BLOCK_BUILD"] : failure_result_note('to build'), - ERROR_CODES["BLOCK_MIMA"] : failure_result_note('MiMa tests'), - ERROR_CODES["BLOCK_SPARK_UNIT_TESTS"] : failure_result_note('Spark unit tests'), - ERROR_CODES["BLOCK_PYSPARK_UNIT_TESTS"] : failure_result_note('PySpark unit tests'), - ERROR_CODES["BLOCK_SPARKR_UNIT_TESTS"] : failure_result_note('SparkR unit tests'), - ERROR_CODES["BLOCK_TIMEOUT"] : failure_result_note('from timeout after a configured wait' + - ' of \`' + tests_timeout + '\`') + ERROR_CODES["BLOCK_GENERAL"]: failure_result_note('some tests'), + ERROR_CODES["BLOCK_RAT"]: failure_result_note('RAT tests'), + ERROR_CODES["BLOCK_SCALA_STYLE"]: failure_result_note('Scala style tests'), + ERROR_CODES["BLOCK_PYTHON_STYLE"]: failure_result_note('Python style tests'), + ERROR_CODES["BLOCK_DOCUMENTATION"]: failure_result_note('to generate documentation'), + ERROR_CODES["BLOCK_BUILD"]: failure_result_note('to build'), + ERROR_CODES["BLOCK_MIMA"]: failure_result_note('MiMa tests'), + ERROR_CODES["BLOCK_SPARK_UNIT_TESTS"]: failure_result_note('Spark unit tests'), + ERROR_CODES["BLOCK_PYSPARK_UNIT_TESTS"]: failure_result_note('PySpark unit tests'), + ERROR_CODES["BLOCK_SPARKR_UNIT_TESTS"]: failure_result_note('SparkR unit tests'), + ERROR_CODES["BLOCK_TIMEOUT"]: failure_result_note('from timeout after a configured wait' + + ' of \`' + tests_timeout + '\`') } if test_result == 0: @@ -182,14 +181,14 @@ 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. + # 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. + # 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"] sha1 = os.environ["sha1"] @@ -211,15 +210,15 @@ def main(): tests_timeout = "175m" # Array to capture all tests to run on the pull request. These tests are held under the - #+ dev/tests/ directory. + # 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 + # * 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" From d202c426cba9a4d71b1e3d89978100f5f8d63236 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Tue, 14 Jul 2015 12:14:20 -0700 Subject: [PATCH 03/32] fixed list bug when finding log files --- dev/run-tests-jenkins.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 908bbd48e669..8bb167039c86 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -89,7 +89,7 @@ def send_archived_logs(): if log_files: log_archive = "unit-tests-logs.tar.gz" - run_cmd(['tar', 'czf', log-archive, *log_files]) + run_cmd(['tar', 'czf', log-archive] + log_files.strip().split('\n')) jenkins_build_dir = os.environ["JENKINS_HOME"] + "/jobs/" + os.environ["JOB_NAME"] + "/builds/" + os.environ["BUILD_NUMBER"] From 3ae1d4984893f1ab1a7c66079db431d739a611d0 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Tue, 14 Jul 2015 12:51:16 -0700 Subject: [PATCH 04/32] correctly escape newlines for long strings --- dev/run-tests-jenkins.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 8bb167039c86..136d10c0b546 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -91,8 +91,8 @@ def send_archived_logs(): run_cmd(['tar', 'czf', log-archive] + log_files.strip().split('\n')) - jenkins_build_dir = os.environ["JENKINS_HOME"] + "/jobs/" + os.environ["JOB_NAME"] + - "/builds/" + os.environ["BUILD_NUMBER"] + jenkins_build_dir = os.environ["JENKINS_HOME"] + "/jobs/" + os.environ["JOB_NAME"] + \ + "/builds/" + os.environ["BUILD_NUMBER"] scp_proc = subprocess.Popen(['scp', log_archive, 'amp-jenkins-master:' + jenkins_build_dir + '/' + log_archive], @@ -133,10 +133,10 @@ def bind_message_base(build_display_name, build_url, ghprb_pull_id, short_commit parameters into a closure without the specific message and returns a function able to generate strong messages for a specific description. """ - return lambda mssg, post_mssg="":\ - '**[Test build ' + build_display_name + ' ' + mssg + '](' + build_url + -'console)** for PR ' + ghprb_pull_id + ' at commit [\`' + short_commit_hash + '\`](' + -commit_url + ')' + str(' ' + post_mssg + '.') if post_mssg else '.' + return lambda mssg, post_mssg="": \ + '**[Test build ' + build_display_name + ' ' + mssg + '](' + build_url + \ + 'console)** for PR ' + ghprb_pull_id + ' at commit [\`' + short_commit_hash + '\`](' + \ + commit_url + ')' + str(' ' + post_mssg + '.') if post_mssg else '.' def success_result_note(mssg): From 2a3d67d9c0cea49a25c8da86bbfea49ecfac2335 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Wed, 15 Jul 2015 09:51:38 -0700 Subject: [PATCH 05/32] fixed numerous bugs, updated to use urllib2 over curl, removed 'find' in place of os.walk --- dev/lint-python | 2 +- dev/run-tests-jenkins.py | 167 ++++++++++++++++--------------- dev/sparktestsupport/__init__.py | 24 +++-- 3 files changed, 99 insertions(+), 94 deletions(-) diff --git a/dev/lint-python b/dev/lint-python index 0c3586462cb3..252f246bdcdf 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" PYTHON_LINT_REPORT_PATH="$SPARK_ROOT_DIR/dev/python-lint-report.txt" cd "$SPARK_ROOT_DIR" diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 136d10c0b546..aead2417eee9 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -21,75 +21,68 @@ import os import sys import json +import urllib2 +import functools import subprocess from sparktestsupport import SPARK_HOME, ERROR_CODES -from sparktestsupport.shellutils import exit_from_command_with_retcode, run_cmd, rm_r +from sparktestsupport.shellutils import run_cmd, rm_r -def print_err(*args): +def print_err(msg): """ Given a set of arguments, will print them to the STDERR stream """ - print(*args, file=sys.stderr) + print(msg, file=sys.stderr) -def post_message(mssg, comments_url): - http_code_header = "HTTP Response Code: " - posted_message = json.dumps({"body": mssg}) - +def post_message_to_github(msg, ghprb_pull_id): print("Attempting to post to Github...") - # we don't want to call `run_cmd` here as, in the event of an error, we DO NOT - # want to print the GITHUB_OAUTH_KEY into the public Jenkins logs - curl_proc = subprocess.Popen(['curl', - '--silent', - '--user', 'x-oauth-basic:' + os.environ['GITHUB_OATH_KEY'], - '--request', 'POST', - '--data', posted_message, - '--write-out', http_code_header + '%{http_code}', - '--header', 'Content-Type: application/json', - comments_url], - stdout=subprocess.PIPE, - stderr=subprocess.PIPE) - curl_stdout, curl_stderr = curl_proc.communicate() - curl_returncode = curl_proc.returncode - # find all lines relevant to the Github API response - api_response = "\n".join([l for l in curl_stdout.split('\n') - if l and not l.startswith(http_code_header)]) - # find the line where `http_code_header` exists, split on ':' to get the - # HTTP response code, and cast to an int - http_code = int(curl_stdout[curl_stdout.find(http_code_header):].split(':')[1]) - - if not curl_returncode == 0: + posted_message = json.dumps({"body": msg}) + request = urllib2.Request("https://api.github.com/repos/apache/spark/issues/" + + ghprb_pull_id + "/comments", + headers={ + "Authorization": "x-oauth-basic:" + os.environ['GITHUB_OATH_KEY'], + "Content-Type": "application/json" + }, + data=posted_message) + try: + response = urllib2.urlopen(request) + except urllib2.URLError as url_e: print_err("Failed to post message to GitHub.") - print_err(" > curl_status:", curl_returncode) - print_err(" > curl_output:", curl_stdout) - print_err(" > data:", posted_message) - - if http_code and not http_code == 201: - print_err(" > http_code:", http_code) - print_err(" > api_response:", api_response) - print_err(" > data:", posted_message) + print_err(" > urllib2_status: %s" % url_e.reason[1]) + print_err(" > data: %s" % posted_message) + 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) - if curl_returncode == 0 and http_code == 201: + if response.getcode() == 201: print(" > Post successful.") def send_archived_logs(): print("Archiving unit tests logs...") - log_files = run_cmd(['find', '.', - '-name', 'unit-tests.log', - '-o', '-path', './sql/hive/target/HiveCompatibilitySuite.failed', - '-o', '-path', './sql/hive/target/HiveCompatibilitySuite.hiveFailed', - '-o', '-path', './sql/hive/target/HiveCompatibilitySuite.wrong'], - return_output=True) + # find any files rescursively with the name 'unit-tests.log' + log_files = [os.path.join(path, f) + for path, _, filenames in os.walk(SPARK_HOME) + for f in filesnames if f == 'unit-tests.log'] + # ensure we have a default list if no 'unit-tests.log' files were found + log_files = log_files if log_files else list() + + # check if any of the three explicit paths exist on the system + log_files += [f for f in ['./sql/hive/target/HiveCompatibilitySuite.failed', + './sql/hive/target/HiveCompatibilitySuite.hiveFailed', + './sql/hive/target/HiveCompatibilitySuite.wrong'] + if os.path.isfile(f)] if log_files: log_archive = "unit-tests-logs.tar.gz" - run_cmd(['tar', 'czf', log-archive] + log_files.strip().split('\n')) + run_cmd(['tar', 'czf', log_archive] + log_files.strip().split('\n')) jenkins_build_dir = os.environ["JENKINS_HOME"] + "/jobs/" + os.environ["JOB_NAME"] + \ "/builds/" + os.environ["BUILD_NUMBER"] @@ -103,8 +96,8 @@ def send_archived_logs(): if not scp_returncode == 0: print_err("Failed to send archived unit tests logs to Jenkins master.") - print_err(" > scp_status:", scp_returncode) - print_err(" > scp_output:", scp_stdout) + print_err(" > scp_status: %s" % scp_returncode) + print_err(" > scp_output: %s" % scp_stdout) else: print(" > Send successful.") else: @@ -113,7 +106,12 @@ def send_archived_logs(): rm_r(log_archive) -def run_pr_tests(pr_tests, ghprb_actual_commit, sha1): +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() @@ -127,31 +125,38 @@ def run_pr_tests(pr_tests, ghprb_actual_commit, sha1): return pr_results -def bind_message_base(build_display_name, build_url, ghprb_pull_id, short_commit_hash, commit_url): - """ - Given base parameters to generate a strong Github message response, binds those - parameters into a closure without the specific message and returns a function - able to generate strong messages for a specific description. - """ - return lambda mssg, post_mssg="": \ - '**[Test build ' + build_display_name + ' ' + mssg + '](' + build_url + \ - 'console)** for PR ' + ghprb_pull_id + ' at commit [\`' + short_commit_hash + '\`](' + \ - commit_url + ')' + str(' ' + post_mssg + '.') if post_mssg else '.' - - -def success_result_note(mssg): - return ' * This patch ' + mssg + '.' - - -def failure_result_note(mssg): - return ' * This patch **fails ' + mssg + '**.' +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](%sconsole)** for PR %s at commit [\`%s\`](%s)%s' % str_args 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_proc = subprocess.Popen(['timeout', tests_timeout, os.path.join(SPARK_HOME, 'dev', 'run-tests')]).wait() - test_result = test_proc.returncode + test_result_code = test_proc.returncode + + def failure_result_note(msg): + ' * This patch **fails ' + msg + '**.' failure_note_by_errcode = { ERROR_CODES["BLOCK_GENERAL"]: failure_result_note('some tests'), @@ -169,12 +174,12 @@ def run_tests(tests_timeout): } if test_result == 0: - test_result_note = success_result_note('passes all tests') + test_result_note = ' * This patch passes all tests.' else: - test_result_note = failure_note_by_errcode(test_result) + test_result_note = failure_note_by_errcode[test_result] send_archived_logs() - return test_result_note + return [test_result_code, test_result_note] def main(): @@ -198,8 +203,6 @@ def main(): build_display_name = os.environ["BUILD_DISPLAY_NAME"] build_url = os.environ["BUILD_URL"] - comments_url = "https://api.github.com/repos/apache/spark/issues/" + ghprb_pull_id + "/comments" - pull_request_url = "https://github.com/apache/spark/pull/" + ghprb_pull_id commit_url = "https://github.com/apache/spark/commit/" + ghprb_actual_commit # GitHub doesn't auto-link short hashes when submitted via the API, unfortunately. :( @@ -209,8 +212,8 @@ def main(): # must be less than the timeout configured on Jenkins (currently 180m) tests_timeout = "175m" - # Array to capture all tests to run on the pull request. These tests are held under the - # dev/tests/ directory. + # 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 @@ -225,28 +228,28 @@ def main(): ] # `bind_message_base` returns a function to generate messages for Github posting - github_message = bind_message_base(build_display_name, + github_message = functools.partial(pr_message, + build_display_name, build_url, ghprb_pull_id, short_commit_hash, commit_url) # post start message - post_message(github_message('has started')) + post_message_to_github(github_message('has started'), ghprb_pull_id) pr_test_results = run_pr_tests(pr_tests, ghprb_actual_commit, sha1) - test_results = run_tests(tests_timeout) + test_result_code, test_result_note = run_tests(tests_timeout) # post end message result_message = github_message('has finished') - result_message += '\n' + test_results - for pr_result in pr_test_results: - result_message += pr_result + result_message += '\n' + test_result_note + result_message += " ".join(pr_results) - post_message(result_message) + post_message_to_github(result_message, ghprb_pull_id) - sys.exit(test_result) + sys.exit(test_result_code) if __name__ == "__main__": diff --git a/dev/sparktestsupport/__init__.py b/dev/sparktestsupport/__init__.py index ba952f45586c..576c01f955b9 100644 --- a/dev/sparktestsupport/__init__.py +++ b/dev/sparktestsupport/__init__.py @@ -19,14 +19,16 @@ 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_DOCUMENTATION" : 14, - "BLOCK_BUILD" : 15, - "BLOCK_MIMA" : 16, - "BLOCK_SPARK_UNIT_TESTS" : 17, - "BLOCK_PYSPARK_UNIT_TESTS" : 18, - "BLOCK_SPARKR_UNIT_TESTS" : 19, - "BLOCK_TIMEOUT" : 124 } +ERROR_CODES = { + "BLOCK_GENERAL" : 10, + "BLOCK_RAT" : 11, + "BLOCK_SCALA_STYLE" : 12, + "BLOCK_PYTHON_STYLE" : 13, + "BLOCK_DOCUMENTATION" : 14, + "BLOCK_BUILD" : 15, + "BLOCK_MIMA" : 16, + "BLOCK_SPARK_UNIT_TESTS" : 17, + "BLOCK_PYSPARK_UNIT_TESTS" : 18, + "BLOCK_SPARKR_UNIT_TESTS" : 19, + "BLOCK_TIMEOUT" : 124 +} From d83c535d5dcfa379b266f18026a374e91a648a64 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Wed, 15 Jul 2015 10:31:16 -0700 Subject: [PATCH 06/32] fixed typos, style, and linting issues --- dev/run-tests-jenkins.py | 83 +++++++++++++++++++--------------------- 1 file changed, 40 insertions(+), 43 deletions(-) diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index aead2417eee9..f7ec1ccc9163 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -49,15 +49,15 @@ def post_message_to_github(msg, ghprb_pull_id): data=posted_message) try: response = urllib2.urlopen(request) - 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) 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) if response.getcode() == 201: print(" > Post successful.") @@ -69,7 +69,7 @@ def send_archived_logs(): # find any files rescursively with the name 'unit-tests.log' log_files = [os.path.join(path, f) for path, _, filenames in os.walk(SPARK_HOME) - for f in filesnames if f == 'unit-tests.log'] + for f in filenames if f == 'unit-tests.log'] # ensure we have a default list if no 'unit-tests.log' files were found log_files = log_files if log_files else list() @@ -91,7 +91,7 @@ def send_archived_logs(): 'amp-jenkins-master:' + jenkins_build_dir + '/' + log_archive], stdout=subprocess.PIPE, stderr=subprocess.PIPE) - scp_stdout, scp_stderr = scp_proc.communicate() + scp_stdout, _ = scp_proc.communicate() scp_returncode = scp_proc.returncode if not scp_returncode == 0: @@ -106,6 +106,24 @@ def send_archived_logs(): rm_r(log_archive) +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](%sconsole)** 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 @@ -125,24 +143,6 @@ def run_pr_checks(pr_tests, ghprb_actual_commit, sha1): return pr_results -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](%sconsole)** for PR %s at commit [\`%s\`](%s)%s' % str_args - - def run_tests(tests_timeout): """ Runs the `dev/run-tests` script and responds with the correct error message @@ -155,28 +155,25 @@ def run_tests(tests_timeout): os.path.join(SPARK_HOME, 'dev', 'run-tests')]).wait() test_result_code = test_proc.returncode - def failure_result_note(msg): - ' * This patch **fails ' + msg + '**.' - failure_note_by_errcode = { - ERROR_CODES["BLOCK_GENERAL"]: failure_result_note('some tests'), - ERROR_CODES["BLOCK_RAT"]: failure_result_note('RAT tests'), - ERROR_CODES["BLOCK_SCALA_STYLE"]: failure_result_note('Scala style tests'), - ERROR_CODES["BLOCK_PYTHON_STYLE"]: failure_result_note('Python style tests'), - ERROR_CODES["BLOCK_DOCUMENTATION"]: failure_result_note('to generate documentation'), - ERROR_CODES["BLOCK_BUILD"]: failure_result_note('to build'), - ERROR_CODES["BLOCK_MIMA"]: failure_result_note('MiMa tests'), - ERROR_CODES["BLOCK_SPARK_UNIT_TESTS"]: failure_result_note('Spark unit tests'), - ERROR_CODES["BLOCK_PYSPARK_UNIT_TESTS"]: failure_result_note('PySpark unit tests'), - ERROR_CODES["BLOCK_SPARKR_UNIT_TESTS"]: failure_result_note('SparkR unit tests'), - ERROR_CODES["BLOCK_TIMEOUT"]: failure_result_note('from timeout after a configured wait' + - ' of \`' + tests_timeout + '\`') + 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_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 == 0: + if test_result_code == 0: test_result_note = ' * This patch passes all tests.' else: - test_result_note = failure_note_by_errcode[test_result] + test_result_note = ' * This patch **fails %s**.' % failure_note_by_errcode[test_result_code] send_archived_logs() return [test_result_code, test_result_note] @@ -238,14 +235,14 @@ def main(): # post start message post_message_to_github(github_message('has started'), ghprb_pull_id) - pr_test_results = run_pr_tests(pr_tests, ghprb_actual_commit, sha1) + 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 - result_message += " ".join(pr_results) + result_message += " ".join(pr_check_results) post_message_to_github(result_message, ghprb_pull_id) From 3acac9075c5abc335f7a1f93bae0da031f880def Mon Sep 17 00:00:00 2001 From: Brennon York Date: Wed, 15 Jul 2015 10:32:29 -0700 Subject: [PATCH 07/32] fixed indenting with list --- dev/run-tests-jenkins.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index f7ec1ccc9163..e15db9016993 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -219,10 +219,11 @@ def main(): # 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" - ] + 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, From 395928d8da7c104a78f9578e823b62a221d478fd Mon Sep 17 00:00:00 2001 From: Brennon York Date: Wed, 15 Jul 2015 10:39:56 -0700 Subject: [PATCH 08/32] fixed typo --- dev/run-tests-jenkins.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index e15db9016993..3b1e5fb1b1db 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -43,7 +43,8 @@ def post_message_to_github(msg, ghprb_pull_id): request = urllib2.Request("https://api.github.com/repos/apache/spark/issues/" + ghprb_pull_id + "/comments", headers={ - "Authorization": "x-oauth-basic:" + os.environ['GITHUB_OATH_KEY'], + "Authorization": "x-oauth-basic: %s" % ( + os.environ['GITHUB_OAUTH_KEY']), "Content-Type": "application/json" }, data=posted_message) From 4765dc8eb7476c2bd21a0c68ada807bf56d96cbc Mon Sep 17 00:00:00 2001 From: Brennon York Date: Wed, 15 Jul 2015 11:15:49 -0700 Subject: [PATCH 09/32] added variables to remove indents, fixed ' : ' => ': ' in maps --- dev/run-tests-jenkins.py | 9 +++++---- dev/sparktestsupport/__init__.py | 22 +++++++++++----------- 2 files changed, 16 insertions(+), 15 deletions(-) diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 3b1e5fb1b1db..32b6d92a49a7 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -39,12 +39,13 @@ def print_err(msg): 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("https://api.github.com/repos/apache/spark/issues/" + - ghprb_pull_id + "/comments", + request = urllib2.Request(url, headers={ - "Authorization": "x-oauth-basic: %s" % ( - os.environ['GITHUB_OAUTH_KEY']), + "Authorization": "x-oauth-basic: %s" % (github_oauth_key), "Content-Type": "application/json" }, data=posted_message) diff --git a/dev/sparktestsupport/__init__.py b/dev/sparktestsupport/__init__.py index 576c01f955b9..7081e8ab45ea 100644 --- a/dev/sparktestsupport/__init__.py +++ b/dev/sparktestsupport/__init__.py @@ -20,15 +20,15 @@ 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_DOCUMENTATION" : 14, - "BLOCK_BUILD" : 15, - "BLOCK_MIMA" : 16, - "BLOCK_SPARK_UNIT_TESTS" : 17, - "BLOCK_PYSPARK_UNIT_TESTS" : 18, - "BLOCK_SPARKR_UNIT_TESTS" : 19, - "BLOCK_TIMEOUT" : 124 + "BLOCK_GENERAL": 10, + "BLOCK_RAT": 11, + "BLOCK_SCALA_STYLE": 12, + "BLOCK_PYTHON_STYLE": 13, + "BLOCK_DOCUMENTATION": 14, + "BLOCK_BUILD": 15, + "BLOCK_MIMA": 16, + "BLOCK_SPARK_UNIT_TESTS": 17, + "BLOCK_PYSPARK_UNIT_TESTS": 18, + "BLOCK_SPARKR_UNIT_TESTS": 19, + "BLOCK_TIMEOUT": 124 } From e90fdebd751566fabf33c746ef72baf3fc515572 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Wed, 15 Jul 2015 11:18:01 -0700 Subject: [PATCH 10/32] moved if statement for urllib2 into the try block --- dev/run-tests-jenkins.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 32b6d92a49a7..6e7673c0e3fc 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -51,6 +51,9 @@ def post_message_to_github(msg, ghprb_pull_id): 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) @@ -61,9 +64,6 @@ def post_message_to_github(msg, ghprb_pull_id): print_err(" > urllib2_status: %s" % url_e.reason[1]) print_err(" > data: %s" % posted_message) - if response.getcode() == 201: - print(" > Post successful.") - def send_archived_logs(): print("Archiving unit tests logs...") From 64c13c19b536012559433716bed3f6b697773f53 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Wed, 15 Jul 2015 11:22:32 -0700 Subject: [PATCH 11/32] updated token passing for github oauth --- dev/run-tests-jenkins.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 6e7673c0e3fc..73e2be7b63f7 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -45,7 +45,7 @@ def post_message_to_github(msg, ghprb_pull_id): posted_message = json.dumps({"body": msg}) request = urllib2.Request(url, headers={ - "Authorization": "x-oauth-basic: %s" % (github_oauth_key), + "Authorization": "token %s" % github_oauth_key, "Content-Type": "application/json" }, data=posted_message) @@ -55,12 +55,12 @@ def post_message_to_github(msg, ghprb_pull_id): if response.getcode() == 201: print(" > Post successful.") except urllib2.HTTPError as http_e: - print_err("Failed to post message to GitHub.") + 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("Failed to post message to Github.") print_err(" > urllib2_status: %s" % url_e.reason[1]) print_err(" > data: %s" % posted_message) From 85aaf76ed1d32998b68dad1b88abd84686fb69e3 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Wed, 15 Jul 2015 14:24:37 -0700 Subject: [PATCH 12/32] added subprocess_check_call() as backport from python 2.7, moved subprocess cmds from python/run-tests.py to sparktestsupport module, fixed bug with pr check runner --- dev/run-tests-jenkins.py | 2 +- dev/sparktestsupport/shellutils.py | 33 ++++++++++++++++++++++++++++-- python/run-tests.py | 19 +---------------- 3 files changed, 33 insertions(+), 21 deletions(-) diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 73e2be7b63f7..56c3eb6ef3c4 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -137,7 +137,7 @@ def run_pr_checks(pr_tests, ghprb_actual_commit, sha1): pr_results = list() for pr_test in pr_tests: - pr_results.append(run_cmd(['bash', os.path.join(SPARK_HOME, 'dev', 'tests', pr_test), + pr_results.append(run_cmd(['bash', os.path.join(SPARK_HOME, 'dev', 'tests', pr_test + '.sh'), ghprb_actual_commit, sha1], return_output=True).strip()) # Ensure, after each test, that we're back on the current PR diff --git a/dev/sparktestsupport/shellutils.py b/dev/sparktestsupport/shellutils.py index d474cc5430ee..b893e05e747b 100644 --- a/dev/sparktestsupport/shellutils.py +++ b/dev/sparktestsupport/shellutils.py @@ -22,6 +22,35 @@ 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))) @@ -49,9 +78,9 @@ def run_cmd(cmd, return_output=False): cmd = cmd.split() try: if return_output: - return subprocess.check_output(cmd) + return subprocess_check_output(cmd) else: - return subprocess.check_call(cmd) + return subprocess_check_call(cmd) except subprocess.CalledProcessError as e: exit_from_command_with_retcode(e.cmd, e.returncode) diff --git a/python/run-tests.py b/python/run-tests.py index cc560779373b..6a2a3411f792 100755 --- a/python/run-tests.py +++ b/python/run-tests.py @@ -31,23 +31,6 @@ 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 @@ def subprocess_check_output(*popenargs, **kwargs): 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 From f75609845b03c3d08e0794fc6c8c1b1acbc134ea Mon Sep 17 00:00:00 2001 From: Brennon York Date: Wed, 15 Jul 2015 14:26:15 -0700 Subject: [PATCH 13/32] removed unnecessary escapes --- dev/run-tests-jenkins.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 56c3eb6ef3c4..4fbc0c11d734 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -123,7 +123,7 @@ def pr_message(build_display_name, short_commit_hash, commit_url, str(' ' + post_msg + '.') if post_msg else '.') - return '**[Test build %s %s](%sconsole)** for PR %s at commit [\`%s\`](%s)%s' % str_args + return '**[Test build %s %s](%sconsole)** for PR %s at commit [`%s`](%s)%s' % str_args def run_pr_checks(pr_tests, ghprb_actual_commit, sha1): From 39f6a1f1efe5175acf21e33e6c56809f618cf4fd Mon Sep 17 00:00:00 2001 From: Brennon York Date: Wed, 15 Jul 2015 14:37:34 -0700 Subject: [PATCH 14/32] changed test process to only return the returncode --- dev/run-tests-jenkins.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 4fbc0c11d734..6c082284c403 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -152,10 +152,9 @@ def run_tests(tests_timeout): @return a tuple containing the test result code and the result note to post to Github """ - test_proc = subprocess.Popen(['timeout', - tests_timeout, - os.path.join(SPARK_HOME, 'dev', 'run-tests')]).wait() - test_result_code = test_proc.returncode + test_result_code = subprocess.Popen(['timeout', + tests_timeout, + os.path.join(SPARK_HOME, 'dev', 'run-tests')]).wait() failure_note_by_errcode = { ERROR_CODES["BLOCK_GENERAL"]: 'some tests', From 7b116650ed613589b8b0ac42dc3feee6809beccf Mon Sep 17 00:00:00 2001 From: Brennon York Date: Wed, 15 Jul 2015 15:00:22 -0700 Subject: [PATCH 15/32] fixed serialization error between int and string, created an error response for when the run-tests script fails to execute --- dev/run-tests-jenkins.py | 1 + dev/run-tests.py | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 6c082284c403..545c3cf1313f 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -157,6 +157,7 @@ def run_tests(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', diff --git a/dev/run-tests.py b/dev/run-tests.py index 5884884fddc7..8b1b466d8042 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -166,7 +166,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('') @@ -401,7 +401,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() From cf110bcfacebd593162f90b1585205a0901349f8 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Wed, 15 Jul 2015 15:20:27 -0700 Subject: [PATCH 16/32] fixed reference error and pep8 bugs --- dev/run-tests-jenkins.py | 9 +++++---- dev/sparktestsupport/__init__.py | 2 +- dev/sparktestsupport/shellutils.py | 1 + 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 545c3cf1313f..f49e1a58c7a0 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -102,10 +102,10 @@ def send_archived_logs(): print_err(" > scp_output: %s" % scp_stdout) else: print(" > Send successful.") - else: - print_err(" > No log files found.") rm_r(log_archive) + else: + print_err(" > No log files found.") def pr_message(build_display_name, @@ -137,7 +137,8 @@ def run_pr_checks(pr_tests, ghprb_actual_commit, sha1): pr_results = list() for pr_test in pr_tests: - pr_results.append(run_cmd(['bash', os.path.join(SPARK_HOME, 'dev', 'tests', pr_test + '.sh'), + 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).strip()) # Ensure, after each test, that we're back on the current PR @@ -157,7 +158,7 @@ def run_tests(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 + 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', diff --git a/dev/sparktestsupport/__init__.py b/dev/sparktestsupport/__init__.py index 7081e8ab45ea..10d3a87e4d26 100644 --- a/dev/sparktestsupport/__init__.py +++ b/dev/sparktestsupport/__init__.py @@ -30,5 +30,5 @@ "BLOCK_SPARK_UNIT_TESTS": 17, "BLOCK_PYSPARK_UNIT_TESTS": 18, "BLOCK_SPARKR_UNIT_TESTS": 19, - "BLOCK_TIMEOUT": 124 + "BLOCK_TIMEOUT": 124 } diff --git a/dev/sparktestsupport/shellutils.py b/dev/sparktestsupport/shellutils.py index b893e05e747b..d280e797077d 100644 --- a/dev/sparktestsupport/shellutils.py +++ b/dev/sparktestsupport/shellutils.py @@ -40,6 +40,7 @@ def subprocess_check_output(*popenargs, **kwargs): 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) From c9e5c5f6b94cfc487d0a1bf023abc5ff547e008e Mon Sep 17 00:00:00 2001 From: Brennon York Date: Wed, 15 Jul 2015 17:52:03 -0700 Subject: [PATCH 17/32] commented out tests and fixed test result output --- dev/run-tests-jenkins.py | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index f49e1a58c7a0..984738cb5988 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -153,9 +153,11 @@ def run_tests(tests_timeout): @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() + # TODO: turn tests back on + # test_result_code = subprocess.Popen(['timeout', + # tests_timeout, + # os.path.join(SPARK_HOME, 'dev', 'run-tests')]).wait() + test_result_code = 0 failure_note_by_errcode = { 1: 'executing the `dev/run-tests` script', # error to denote run-tests script failures @@ -245,8 +247,8 @@ def main(): # post end message result_message = github_message('has finished') - result_message += '\n' + test_result_note - result_message += " ".join(pr_check_results) + result_message += '\n' + test_result_note + '\n' + result_message += '\n'.join(pr_check_results) post_message_to_github(result_message, ghprb_pull_id) From 5272381b61092fd8954d3c25ffae459dcec26b5f Mon Sep 17 00:00:00 2001 From: Brennon York Date: Thu, 16 Jul 2015 08:38:50 -0700 Subject: [PATCH 18/32] printing of the pr results --- dev/run-tests-jenkins.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 984738cb5988..cca850bd2f9a 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -245,6 +245,8 @@ def main(): test_result_code, test_result_note = run_tests(tests_timeout) + print("PR results:", pr_check_results) + # post end message result_message = github_message('has finished') result_message += '\n' + test_result_note + '\n' From 528279a3ee0c03a35417505aca4132ad65393a89 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Thu, 16 Jul 2015 08:55:07 -0700 Subject: [PATCH 19/32] fixed strip issue with PR results, from strip => rstrip --- dev/run-tests-jenkins.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index cca850bd2f9a..c7319646827e 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -140,7 +140,7 @@ def run_pr_checks(pr_tests, ghprb_actual_commit, sha1): 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).strip()) + 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 From 6a9ed32b487f30b1b7762324fb016d3af7a79b1e Mon Sep 17 00:00:00 2001 From: Brennon York Date: Thu, 16 Jul 2015 09:05:31 -0700 Subject: [PATCH 20/32] run-tests-jenkins now properly runs the tests, removed license from file to check RAT tests --- .../scala/org/apache/spark/Accumulators.scala | 16 ---------------- dev/run-tests-jenkins.py | 10 +++------- 2 files changed, 3 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 5a8d17bd9993..6103bc07efa1 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -1,19 +1,3 @@ -/* - * 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. - */ package org.apache.spark diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index c7319646827e..7a2355335137 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -153,11 +153,9 @@ def run_tests(tests_timeout): @return a tuple containing the test result code and the result note to post to Github """ - # TODO: turn tests back on - # test_result_code = subprocess.Popen(['timeout', - # tests_timeout, - # os.path.join(SPARK_HOME, 'dev', 'run-tests')]).wait() - test_result_code = 0 + 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 @@ -245,8 +243,6 @@ def main(): test_result_code, test_result_note = run_tests(tests_timeout) - print("PR results:", pr_check_results) - # post end message result_message = github_message('has finished') result_message += '\n' + test_result_note + '\n' From 22f9be85701d5e8f92c1bb2092961cde666eb40f Mon Sep 17 00:00:00 2001 From: Brennon York Date: Thu, 16 Jul 2015 09:13:07 -0700 Subject: [PATCH 21/32] reverted license --- .../scala/org/apache/spark/Accumulators.scala | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 6103bc07efa1..5a8d17bd9993 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -1,3 +1,19 @@ +/* + * 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. + */ package org.apache.spark From 5d63bc543d95b9d2ecedbf9ab7479e90e40afd64 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Thu, 16 Jul 2015 09:17:27 -0700 Subject: [PATCH 22/32] removed mima to fail binary compatibility checks --- project/MimaExcludes.scala | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 4e4e810ec36e..79089aae2a37 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -83,18 +83,6 @@ object MimaExcludes { "org.apache.spark.streaming.scheduler.InputInfo$"), ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.streaming.scheduler.InputInfo") - ) ++ Seq( - // SPARK-6797 Support YARN modes for SparkR - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.r.PairwiseRRDD.this"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.r.RRDD.createRWorker"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.r.RRDD.this"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.r.StringRRDD.this"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.r.BaseRRDD.this") ) case v if v.startsWith("1.4") => From 33e314315b549aab20b20b7fe4e1ab3cc29ed7a0 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Thu, 16 Jul 2015 09:57:29 -0700 Subject: [PATCH 23/32] reverted mima excludes --- project/MimaExcludes.scala | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 79089aae2a37..4e4e810ec36e 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -83,6 +83,18 @@ object MimaExcludes { "org.apache.spark.streaming.scheduler.InputInfo$"), ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.streaming.scheduler.InputInfo") + ) ++ Seq( + // SPARK-6797 Support YARN modes for SparkR + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.r.PairwiseRRDD.this"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.r.RRDD.createRWorker"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.r.RRDD.this"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.r.StringRRDD.this"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.r.BaseRRDD.this") ) case v if v.startsWith("1.4") => From d82567af7c76720df8baf936c0d6548325406e65 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Thu, 16 Jul 2015 09:59:11 -0700 Subject: [PATCH 24/32] set accumulator test suite to fail --- core/src/test/scala/org/apache/spark/AccumulatorSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index e942d6579b2f..c35d79824cbb 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -47,7 +47,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex val d = sc.parallelize(1 to 20) d.foreach{x => acc += x} - acc.value should be (210) + acc.value should be (42) val longAcc = sc.accumulator(0L) val maxInt = Integer.MAX_VALUE.toLong From 2117c64cf7eab47dbe18831aea9655e136ca1e8c Mon Sep 17 00:00:00 2001 From: Brennon York Date: Thu, 16 Jul 2015 12:07:30 -0700 Subject: [PATCH 25/32] reverted accumulator test in favor of graphx test to only trigger 'graphx' test suite, fixed bug with run-tests-jenkins on list joining versus strings --- core/src/test/scala/org/apache/spark/AccumulatorSuite.scala | 2 +- dev/run-tests-jenkins.py | 2 +- .../src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index c35d79824cbb..e942d6579b2f 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -47,7 +47,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex val d = sc.parallelize(1 to 20) d.foreach{x => acc += x} - acc.value should be (42) + acc.value should be (210) val longAcc = sc.accumulator(0L) val maxInt = Integer.MAX_VALUE.toLong diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 7a2355335137..1438546e695c 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -84,7 +84,7 @@ def send_archived_logs(): if log_files: log_archive = "unit-tests-logs.tar.gz" - run_cmd(['tar', 'czf', log_archive] + log_files.strip().split('\n')) + run_cmd(['tar', 'czf', log_archive] + log_files) jenkins_build_dir = os.environ["JENKINS_HOME"] + "/jobs/" + os.environ["JOB_NAME"] + \ "/builds/" + os.environ["BUILD_NUMBER"] diff --git a/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala index f1ecc9e2219d..fbd6911e9972 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala @@ -27,7 +27,7 @@ class EdgeRDDSuite extends SparkFunSuite with LocalSparkContext { withSpark { sc => val verts = sc.parallelize(List((0L, 0), (1L, 1), (1L, 2), (2L, 3), (2L, 3), (2L, 3))) val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]])) - assert(edges.getStorageLevel == StorageLevel.NONE) + assert(edges.getStorageLevel == 1)//StorageLevel.NONE) edges.cache() assert(edges.getStorageLevel == StorageLevel.MEMORY_ONLY) } From 831376ebc321771b347a8303e332ee6467f3757c Mon Sep 17 00:00:00 2001 From: Brennon York Date: Thu, 16 Jul 2015 12:19:24 -0700 Subject: [PATCH 26/32] fixed scala style check --- .../src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala index fbd6911e9972..91fe6b06dd30 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala @@ -27,7 +27,7 @@ class EdgeRDDSuite extends SparkFunSuite with LocalSparkContext { withSpark { sc => val verts = sc.parallelize(List((0L, 0), (1L, 1), (1L, 2), (2L, 3), (2L, 3), (2L, 3))) val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]])) - assert(edges.getStorageLevel == 1)//StorageLevel.NONE) + assert(edges.getStorageLevel == 1) //StorageLevel.NONE) edges.cache() assert(edges.getStorageLevel == StorageLevel.MEMORY_ONLY) } From d76f559857a1368750a37d703d0c8619d3c1d37a Mon Sep 17 00:00:00 2001 From: Brennon York Date: Thu, 16 Jul 2015 12:28:03 -0700 Subject: [PATCH 27/32] fixed scala style checks --- .../src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala index 91fe6b06dd30..06b9800d7309 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala @@ -27,7 +27,7 @@ class EdgeRDDSuite extends SparkFunSuite with LocalSparkContext { withSpark { sc => val verts = sc.parallelize(List((0L, 0), (1L, 1), (1L, 2), (2L, 3), (2L, 3), (2L, 3))) val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]])) - assert(edges.getStorageLevel == 1) //StorageLevel.NONE) + assert(edges.getStorageLevel == 1) edges.cache() assert(edges.getStorageLevel == StorageLevel.MEMORY_ONLY) } From 05d9ef41665201d8f3c0b8529ff86b92c1cd28ce Mon Sep 17 00:00:00 2001 From: Brennon York Date: Thu, 16 Jul 2015 15:27:23 -0700 Subject: [PATCH 28/32] reverted scala test failure, added pyspark failure --- .../src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala | 2 +- python/pyspark/rdd.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala index 06b9800d7309..f1ecc9e2219d 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala @@ -27,7 +27,7 @@ class EdgeRDDSuite extends SparkFunSuite with LocalSparkContext { withSpark { sc => val verts = sc.parallelize(List((0L, 0), (1L, 1), (1L, 2), (2L, 3), (2L, 3), (2L, 3))) val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]])) - assert(edges.getStorageLevel == 1) + assert(edges.getStorageLevel == StorageLevel.NONE) edges.cache() assert(edges.getStorageLevel == StorageLevel.MEMORY_ONLY) } diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 3218bed5c74f..f01927972228 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -296,7 +296,7 @@ def flatMap(self, f, preservesPartitioning=False): >>> rdd = sc.parallelize([2, 3, 4]) >>> sorted(rdd.flatMap(lambda x: range(1, x)).collect()) - [1, 1, 1, 2, 2, 3] + [1, 1, 1, 2, 2, 3, 3, 3, 4, 4, 4, 4] >>> sorted(rdd.flatMap(lambda x: [(x, x), (x, x)]).collect()) [(2, 2), (2, 2), (3, 3), (3, 3), (4, 4), (4, 4)] """ From 0ad51b40396249fd268ab29839f419203612f2a5 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Mon, 27 Jul 2015 09:07:26 -0700 Subject: [PATCH 29/32] updated to add --absolute-names with tar command for archiving logs --- dev/run-tests-jenkins.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 1438546e695c..d8966737b710 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -84,7 +84,7 @@ def send_archived_logs(): if log_files: log_archive = "unit-tests-logs.tar.gz" - run_cmd(['tar', 'czf', log_archive] + log_files) + run_cmd(['tar', '-czf', '--absolute-names', log_archive] + log_files) jenkins_build_dir = os.environ["JENKINS_HOME"] + "/jobs/" + os.environ["JOB_NAME"] + \ "/builds/" + os.environ["BUILD_NUMBER"] From a218baffb4415255dcc13baef28a3a778917a601 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Tue, 4 Aug 2015 16:52:49 -0700 Subject: [PATCH 30/32] reverting python test --- python/pyspark/rdd.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index f01927972228..3218bed5c74f 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -296,7 +296,7 @@ def flatMap(self, f, preservesPartitioning=False): >>> rdd = sc.parallelize([2, 3, 4]) >>> sorted(rdd.flatMap(lambda x: range(1, x)).collect()) - [1, 1, 1, 2, 2, 3, 3, 3, 4, 4, 4, 4] + [1, 1, 1, 2, 2, 3] >>> sorted(rdd.flatMap(lambda x: [(x, x), (x, x)]).collect()) [(2, 2), (2, 2), (3, 3), (3, 3), (4, 4), (4, 4)] """ From fe397fbf0798d9b26305e86b9ed8667696c20715 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Fri, 16 Oct 2015 14:40:01 -0700 Subject: [PATCH 31/32] updated to account for PR #7883 --- dev/run-tests-jenkins.py | 1 + dev/sparktestsupport/__init__.py | 13 +++++++------ 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index d8966737b710..26a1c4252675 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -163,6 +163,7 @@ def run_tests(tests_timeout): 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', diff --git a/dev/sparktestsupport/__init__.py b/dev/sparktestsupport/__init__.py index 10d3a87e4d26..8ab6d9e37ca2 100644 --- a/dev/sparktestsupport/__init__.py +++ b/dev/sparktestsupport/__init__.py @@ -24,11 +24,12 @@ "BLOCK_RAT": 11, "BLOCK_SCALA_STYLE": 12, "BLOCK_PYTHON_STYLE": 13, - "BLOCK_DOCUMENTATION": 14, - "BLOCK_BUILD": 15, - "BLOCK_MIMA": 16, - "BLOCK_SPARK_UNIT_TESTS": 17, - "BLOCK_PYSPARK_UNIT_TESTS": 18, - "BLOCK_SPARKR_UNIT_TESTS": 19, + "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 } From 206930e5fcccc239485c092d0fb4d418fbbb0178 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Fri, 16 Oct 2015 14:42:14 -0700 Subject: [PATCH 32/32] removed 'send_archived_logs' function --- dev/run-tests-jenkins.py | 44 ---------------------------------------- 1 file changed, 44 deletions(-) diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 26a1c4252675..37601ca365e6 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -65,49 +65,6 @@ def post_message_to_github(msg, ghprb_pull_id): print_err(" > data: %s" % posted_message) -def send_archived_logs(): - print("Archiving unit tests logs...") - - # find any files rescursively with the name 'unit-tests.log' - log_files = [os.path.join(path, f) - for path, _, filenames in os.walk(SPARK_HOME) - for f in filenames if f == 'unit-tests.log'] - # ensure we have a default list if no 'unit-tests.log' files were found - log_files = log_files if log_files else list() - - # check if any of the three explicit paths exist on the system - log_files += [f for f in ['./sql/hive/target/HiveCompatibilitySuite.failed', - './sql/hive/target/HiveCompatibilitySuite.hiveFailed', - './sql/hive/target/HiveCompatibilitySuite.wrong'] - if os.path.isfile(f)] - - if log_files: - log_archive = "unit-tests-logs.tar.gz" - - run_cmd(['tar', '-czf', '--absolute-names', log_archive] + log_files) - - jenkins_build_dir = os.environ["JENKINS_HOME"] + "/jobs/" + os.environ["JOB_NAME"] + \ - "/builds/" + os.environ["BUILD_NUMBER"] - - scp_proc = subprocess.Popen(['scp', log_archive, - 'amp-jenkins-master:' + jenkins_build_dir + '/' + log_archive], - stdout=subprocess.PIPE, - stderr=subprocess.PIPE) - scp_stdout, _ = scp_proc.communicate() - scp_returncode = scp_proc.returncode - - if not scp_returncode == 0: - print_err("Failed to send archived unit tests logs to Jenkins master.") - print_err(" > scp_status: %s" % scp_returncode) - print_err(" > scp_output: %s" % scp_stdout) - else: - print(" > Send successful.") - - rm_r(log_archive) - else: - print_err(" > No log files found.") - - def pr_message(build_display_name, build_url, ghprb_pull_id, @@ -178,7 +135,6 @@ def run_tests(tests_timeout): test_result_note = ' * This patch passes all tests.' else: test_result_note = ' * This patch **fails %s**.' % failure_note_by_errcode[test_result_code] - send_archived_logs() return [test_result_code, test_result_note]