SERVER-111537 Create an Evergreen task that runs multiple bazel-based resmoke tests (#44986)

GitOrigin-RevId: cfe5a368311f304f71957b8022b792d01870618c
This commit is contained in:
Sean Lyons 2025-12-10 16:16:51 -05:00 committed by MongoDB Bot
parent f23d7f6b80
commit 9f5d68a902
12 changed files with 300 additions and 124 deletions

View File

@ -420,7 +420,6 @@ common:remote_test --remote_download_outputs=minimal
common:remote_test --test_output=summary common:remote_test --test_output=summary
common:remote_test --modify_execution_info=^(CppLink|CppArchive|SolibSymlink|ExtractDebugInfo|StripDebugInfo|CcGenerateIntermediateDwp|CcGenerateDwp)$=-no-remote-cache common:remote_test --modify_execution_info=^(CppLink|CppArchive|SolibSymlink|ExtractDebugInfo|StripDebugInfo|CcGenerateIntermediateDwp|CcGenerateDwp)$=-no-remote-cache
common:remote_test --remote_download_regex=.*(TestLogs.*|report.*\.json$|\.core$|data_archives.*\.tgz$) # Resmoke TestLogs directory, report, core dumps, and data directory archives. common:remote_test --remote_download_regex=.*(TestLogs.*|report.*\.json$|\.core$|data_archives.*\.tgz$) # Resmoke TestLogs directory, report, core dumps, and data directory archives.
test:remote_test --test_timeout=660 # Allow extra 60s for coredump on abort
test:remote_test --test_tag_filters=-incompatible_with_bazel_remote_test test:remote_test --test_tag_filters=-incompatible_with_bazel_remote_test
# Coverage # Coverage

View File

@ -80,6 +80,18 @@ config_setting(
}, },
) )
bool_flag(
name = "installed_dist_test",
build_setting_default = False,
)
config_setting(
name = "installed_dist_test_enabled",
flag_values = {
"//bazel/resmoke:installed_dist_test": "True",
},
)
py_binary( py_binary(
name = "resmoke_config_generator", name = "resmoke_config_generator",
srcs = ["resmoke_config_generator.py"], srcs = ["resmoke_config_generator.py"],

View File

@ -93,14 +93,20 @@ def resmoke_suite_test(
"--log=evg", "--log=evg",
"--cedarReportFile=cedar_report.json", "--cedarReportFile=cedar_report.json",
"--skipSymbolization", # Symbolization is not yet functional, SERVER-103538 "--skipSymbolization", # Symbolization is not yet functional, SERVER-103538
],
"//conditions:default": [],
}) + select({
"//bazel/resmoke:installed_dist_test_enabled": [
"--installDir=dist-test/bin", "--installDir=dist-test/bin",
"--mongoVersionFile=$(location //:.resmoke_mongo_version.yml)", "--mongoVersionFile=$(location //:.resmoke_mongo_version.yml)",
], ],
"//conditions:default": [ "//conditions:default": [
"--installDir=install-dist-test/bin",
"--mongoVersionFile=$(location //bazel/resmoke:resmoke_mongo_version)", "--mongoVersionFile=$(location //bazel/resmoke:resmoke_mongo_version)",
], ],
}) })
deps_path = ":".join(["$(location %s)" % dep for dep in deps])
native.py_test( native.py_test(
name = name, name = name,
# To a user of resmoke_suite_test, the `srcs` is the list of tests to select. However, to the py_test rule, # To a user of resmoke_suite_test, the `srcs` is the list of tests to select. However, to the py_test rule,
@ -122,13 +128,16 @@ def resmoke_suite_test(
"//:generated_resmoke_config", "//:generated_resmoke_config",
"//:empty_jsconfig", "//:empty_jsconfig",
] + select({ ] + select({
"//bazel/resmoke:in_evergreen_enabled": ["//:installed-dist-test", "//:.resmoke_mongo_version.yml"], "//bazel/resmoke:installed_dist_test_enabled": ["//:installed-dist-test", "//:.resmoke_mongo_version.yml"],
"//conditions:default": ["//:install-dist-test", "//bazel/resmoke:resmoke_mongo_version"], "//conditions:default": ["//bazel/resmoke:resmoke_mongo_version"],
}), }),
deps = deps + [ deps = [
resmoke, resmoke,
"//buildscripts:bazel_local_resources", "//buildscripts:bazel_local_resources",
], ] + select({
"//bazel/resmoke:installed_dist_test_enabled": [],
"//conditions:default": deps,
}),
main = resmoke_shim, main = resmoke_shim,
args = [ args = [
"run", "run",
@ -145,6 +154,9 @@ def resmoke_suite_test(
env = { env = {
"LOCAL_RESOURCES": "$(LOCAL_RESOURCES)", "LOCAL_RESOURCES": "$(LOCAL_RESOURCES)",
"GIT_PYTHON_REFRESH": "quiet", # Ignore "Bad git executable" error when importing git python. Git commands will still error if run. "GIT_PYTHON_REFRESH": "quiet", # Ignore "Bad git executable" error when importing git python. Git commands will still error if run.
}, } | select({
"//bazel/resmoke:installed_dist_test_enabled": {},
"//conditions:default": {"DEPS_PATH": deps_path},
}),
**kwargs **kwargs
) )

View File

@ -105,6 +105,15 @@ if __name__ == "__main__":
add_evergreen_build_info(resmoke_args) add_evergreen_build_info(resmoke_args)
if os.environ.get("DEPS_PATH"):
# Modify DEPS_PATH to use os.pathsep, rather than ':'
os.environ["PATH"] += os.pathsep + os.pathsep.join(
[
os.path.dirname(os.path.abspath(path))
for path in os.environ.get("DEPS_PATH").split(":")
]
)
if os.environ.get("TEST_UNDECLARED_OUTPUTS_DIR"): if os.environ.get("TEST_UNDECLARED_OUTPUTS_DIR"):
undeclared_output_dir = os.environ.get("TEST_UNDECLARED_OUTPUTS_DIR") undeclared_output_dir = os.environ.get("TEST_UNDECLARED_OUTPUTS_DIR")
resmoke_args.append(f"--dbpathPrefix={os.path.join(undeclared_output_dir,'data')}") resmoke_args.append(f"--dbpathPrefix={os.path.join(undeclared_output_dir,'data')}")

View File

@ -4,6 +4,7 @@
import errno import errno
import json import json
import os import os
import re
import sys import sys
from optparse import OptionParser from optparse import OptionParser
@ -46,6 +47,25 @@ def check_error(input_count, output_count):
raise ValueError("Both input file and output files exist") raise ValueError("Both input file and output files exist")
def add_bazel_target_info(test_report, report_file):
outputs_path = os.path.dirname(
os.path.dirname(os.path.relpath(report_file, start="bazel-testlogs"))
)
if re.search(r"shard_\d+_of_\d+", os.path.basename(outputs_path)):
target_path = os.path.dirname(outputs_path)
else:
target_path = outputs_path
target = "//" + ":".join(target_path.rsplit("/", 1))
target_string = target.replace("/", "_").replace(":", "_")
for test in test_report.test_infos:
test.test_file = f"{target} - {test.test_file}"
test.group_id = f"{target_string}_{test.group_id}"
test.log_info["log_name"] = os.path.join(outputs_path, test.log_info["log_name"])
test.log_info["logs_to_merge"] = [
os.path.join(outputs_path, log) for log in test.log_info["logs_to_merge"]
]
def main(): def main():
"""Execute Main program.""" """Execute Main program."""
usage = "usage: %prog [options] report1.json report2.json ..." usage = "usage: %prog [options] report1.json report2.json ..."
@ -69,6 +89,13 @@ def main():
action="store_false", action="store_false",
help="Do not exit with a non-zero code if any test in the report fails.", help="Do not exit with a non-zero code if any test in the report fails.",
) )
parser.add_option(
"--add-bazel-target-info",
dest="add_bazel_target_info",
default=False,
action="store_true",
help="Add bazel targets to the test names and log locations.",
)
(options, args) = parser.parse_args() (options, args) = parser.parse_args()
@ -82,7 +109,10 @@ def main():
for report_file in report_files: for report_file in report_files:
try: try:
report_file_json = read_json_file(report_file) report_file_json = read_json_file(report_file)
test_reports.append(report.TestReport.from_dict(report_file_json)) test_report = report.TestReport.from_dict(report_file_json)
if options.add_bazel_target_info:
add_bazel_target_info(test_report, report_file)
test_reports.append(test_report)
except IOError as err: except IOError as err:
# errno.ENOENT is the error code for "No such file or directory". # errno.ENOENT is the error code for "No such file or directory".
if err.errno == errno.ENOENT: if err.errno == errno.ENOENT:

View File

@ -52,8 +52,14 @@ resmoke_suite_test(
resmoke_args = [ resmoke_args = [
"--storageEngineCacheSizeGB=1", "--storageEngineCacheSizeGB=1",
], ],
shard_count = 24,
tags = [ tags = [
"manual", # exclude from expansion of target pattern wildcards (..., :*, :all, etc.) "ci-development-critical-single-variant",
],
deps = [
"//src/mongo/db:mongod",
"//src/mongo/s:mongos",
"//src/mongo/shell:mongo",
], ],
) )
@ -75,6 +81,10 @@ resmoke_suite_test(
tags = [ tags = [
"manual", # exclude from expansion of target pattern wildcards (..., :*, :all, etc.) "manual", # exclude from expansion of target pattern wildcards (..., :*, :all, etc.)
], ],
deps = [
"//src/mongo/db:mongod",
"//src/mongo/shell:mongo",
],
) )
# This is an experimental test target for running the multiversion_sanity_check # This is an experimental test target for running the multiversion_sanity_check
@ -100,6 +110,10 @@ resmoke_suite_test(
tags = [ tags = [
"manual", # exclude from expansion of target pattern wildcards (..., :*, :all, etc.) "manual", # exclude from expansion of target pattern wildcards (..., :*, :all, etc.)
], ],
deps = [
"//src/mongo/db:mongod",
"//src/mongo/shell:mongo",
],
) )
# This is an experimental test target for running the multiversion_sanity_check # This is an experimental test target for running the multiversion_sanity_check
@ -125,4 +139,8 @@ resmoke_suite_test(
tags = [ tags = [
"manual", # exclude from expansion of target pattern wildcards (..., :*, :all, etc.) "manual", # exclude from expansion of target pattern wildcards (..., :*, :all, etc.)
], ],
deps = [
"//src/mongo/db:mongod",
"//src/mongo/shell:mongo",
],
) )

View File

@ -1331,9 +1331,9 @@ functions:
OTEL_PARENT_ID: ${otel_parent_id} OTEL_PARENT_ID: ${otel_parent_id}
OTEL_COLLECTOR_DIR: "../build/OTelTraces/" OTEL_COLLECTOR_DIR: "../build/OTelTraces/"
"execute resmoke tests via bazel": &execute_resmoke_tests_via_bazel "execute resmoke tests via bazel sh": &execute_resmoke_tests_via_bazel_sh
command: subprocess.exec command: subprocess.exec
display_name: "execute resmoke tests via bazel" display_name: "execute resmoke tests via bazel sh"
type: test type: test
params: params:
binary: bash binary: bash
@ -1352,6 +1352,12 @@ functions:
args: args:
- "./src/evergreen/resmoke_tests_execute_bazel.sh" - "./src/evergreen/resmoke_tests_execute_bazel.sh"
"execute resmoke tests via bazel":
- *get_version_expansions
- *apply_version_expansions
- *f_expansions_write
- *execute_resmoke_tests_via_bazel_sh
"assume ECR role": &assume_ecr_role "assume ECR role": &assume_ecr_role
command: ec2.assume_role command: ec2.assume_role
params: params:
@ -1973,7 +1979,7 @@ functions:
- *update_task_timeout - *update_task_timeout
- *f_expansions_write - *f_expansions_write
- *sign_macos_dev_binaries - *sign_macos_dev_binaries
- *execute_resmoke_tests_via_bazel - *execute_resmoke_tests_via_bazel_sh
"run generated tests via bazel": "run generated tests via bazel":
- *f_expansions_write - *f_expansions_write
@ -1997,7 +2003,7 @@ functions:
- *get_version_expansions - *get_version_expansions
- *apply_version_expansions - *apply_version_expansions
- *f_expansions_write - *f_expansions_write
- *execute_resmoke_tests_via_bazel - *execute_resmoke_tests_via_bazel_sh
"generate version expansions": &generate_version_expansions "generate version expansions": &generate_version_expansions
command: subprocess.exec command: subprocess.exec
@ -3236,6 +3242,20 @@ functions:
remote_file: ${project}/${build_variant}/${revision}/datafiles/${task_id}- remote_file: ${project}/${build_variant}/${revision}/datafiles/${task_id}-
display_name: "Data files " display_name: "Data files "
"upload bazel test logs":
command: s3.put
params:
aws_key: ${aws_key}
aws_secret: ${aws_secret}
bucket: mciuploads
permissions: private
visibility: signed
content_type: text/plain
local_files_include_filter:
- tmp/bazel-testlogs/*.log
remote_file: ${project}/${build_variant}/${revision}/${task_id}-
display_name: "Bazel test.log: "
"attach wiki page": "attach wiki page":
- *f_expansions_write - *f_expansions_write
- command: subprocess.exec - command: subprocess.exec

View File

@ -486,40 +486,83 @@ tasks:
suite: //buildscripts/resmokeconfig:core suite: //buildscripts/resmokeconfig:core
compiling_for_test: true compiling_for_test: true
bazel_args: >- bazel_args: >-
--test_sharding_strategy=forced=16
--test_arg=--testTimeout=960 --test_arg=--testTimeout=960
--test_timeout=1500 --test_timeout=1500
# Timeouts are 16 minutes for individual tests, the same as the default idle timeout. # Timeouts are 16 minutes for individual tests, the same as the default idle timeout.
# The bazel test timeout is 25 minutes, slightly shorter than the 30 minute commit queue # The bazel test timeout is 25 minutes, slightly shorter than the 30 minute commit queue
# timeout, so that it will timeout before the task would. # timeout, so that it will timeout before the task would.
# Experimental task running the jstestfuzz suite bazel target. To be removed with SERVER-103537. - name: resmoke_tests
- <<: *jstestfuzz_template
name: bazel_jstestfuzz_gen
tags: ["assigned_to_jira_team_devprod_correctness", "experimental"] tags: ["assigned_to_jira_team_devprod_correctness", "experimental"]
depends_on:
- name: version_expansions_gen
variant: generate-tasks-for-version
exec_timeout_secs: 1800 # 30 minutes.
commands: commands:
- func: "generate resmoke tasks" - func: "execute resmoke tests via bazel"
vars: vars:
<<: *jstestfuzz_config_vars targets: //buildscripts/...
num_files: 8 compiling_for_test: true
num_tasks: 2 bazel_args: >-
jstestfuzz_vars: --jsTestsDir ../jstests --test_tag_filters=${resmoke_tests_tag_filter},-incompatible_with_bazel_remote_test
suite: //buildscripts/resmokeconfig:jstestfuzz --test_arg=--testTimeout=960
npm_command: jstestfuzz --test_timeout=1500
# Timeouts are 16 minutes for individual tests, the same as the default idle timeout.
# The bazel test timeout is 25 minutes, slightly shorter than the 30 minute exec
# timeout, so that it will timeout before the task would.
# The task_compile_flags are taken from what is set for archive_dist_test in bazel_compile.sh
task_compile_flags: >-
--verbose_failures
--simple_build_id=True
--features=strip_debug
--separate_debug=False
--define=MONGO_VERSION=${version}
--config=evg
--config=opt_profiled
--jobs=1600
# Experimental task running the multiversion sanity check bazel targets. To be removed with SERVER-103537. task_groups:
- <<: *gen_task_template - name: resmoke_tests_TG
name: bazel_multiversion_sanity_check_gen max_hosts: -1
tags: setup_task_can_fail_task: true
[ setup_task:
"assigned_to_jira_team_devprod_correctness", - command: manifest.load
"experimental", - func: "git get project and add git tag"
"multiversion", - func: "set task expansion macros"
"future_git_tag_incompatible", - func: "f_expansions_write"
] - func: "kill processes"
commands: - func: "cleanup environment"
- func: "generate resmoke tasks" - func: "set up venv"
- func: "initialize multiversion tasks" - func: "upload pip requirements"
- func: "configure evergreen api credentials"
- func: "set up credentials"
- func: "get engflow creds"
teardown_task:
- func: "debug full disk"
- func: "attach bazel invocation"
- func: "upload bazel test logs"
- func: "attach report"
- func: "upload mongodatafiles"
- func: "attach multiversion download links"
- func: "kill processes"
- func: "save mongo coredumps"
- func: "generate hang analyzer tasks"
- func: "save disk statistics"
- func: "save system resource information"
- func: "remove files"
vars: vars:
//buildscripts/resmokeconfig:bazel_multiversion_sanity_check_last_continuous_new_new_old: last_continuous files: >-
//buildscripts/resmokeconfig:bazel_multiversion_sanity_check_last_lts_new_new_old: last_lts src/resmoke_error_code
src/*.gcda.gcov
src/gcov-intermediate-files.tgz
src/*.core src/*.mdmp src/*.core.gz src/*.mdmp.gz
mongo-coredumps.json
src/debugger*.*
src/mongo-hanganalyzer.tgz
diskstats.tgz
system-resource-info.tgz
${report_file|src/report.json}
${archive_file|src/archive.json}
src/network_diagnostics.txt
tasks:
- resmoke_tests

View File

@ -40,22 +40,15 @@ buildvariants:
cron: "0 4 * * 0" # Every week starting 0400 UTC Sunday cron: "0 4 * * 0" # Every week starting 0400 UTC Sunday
stepback: false stepback: false
run_on: run_on:
- amazon2023-arm64-latest-m8g-xlarge - amazon2023-arm64-latest-m8g-4xlarge
depends_on:
- name: archive_dist_test
variant: amazon-linux2023-arm64-static-compile
- name: archive_jstestshell
variant: amazon-linux2023-arm64-static-compile
- name: version_gen
variant: generate-tasks-for-version
expansions: expansions:
compile_variant: amazon-linux2023-arm64-static-compile compile_variant: bazel-integration-tests
evergreen_remote_exec: on
multi_suite_resmoke_task: true
multiversion_platform: amazon2023 multiversion_platform: amazon2023
multiversion_edition: enterprise multiversion_edition: enterprise
multiversion_architecture: aarch64 multiversion_architecture: aarch64
multiversion_link_dir: multiversion_binaries multiversion_link_dir: multiversion_binaries
bazel_compile_flags: >- resmoke_tests_tag_filter: ci-development-critical-single-variant
--default_test_resources=cpu=1,2,4,8
tasks: tasks:
- name: bazel_multiversion_sanity_check_gen - name: resmoke_tests_TG
- name: bazel_jstestfuzz_gen

View File

@ -32,6 +32,7 @@ mciuploads_binary_visibility: public
jstestfuzz_concurrent_num_files: "10" jstestfuzz_concurrent_num_files: "10"
curator_release: "latest" curator_release: "latest"
ext: tgz ext: tgz
resmoke_tests_tag_filter: resmoke_tests_tag_filter_not_set
# Bazel compile expansions. # Bazel compile expansions.
compiling_for_test: "false" compiling_for_test: "false"

View File

@ -85,6 +85,7 @@ bazel_evergreen_shutils::compute_local_arg() {
local_arg+=" --jobs=auto" local_arg+=" --jobs=auto"
elif [[ "$mode" == "test" && "${task_name:-}" == "unit_tests" ]]; then elif [[ "$mode" == "test" && "${task_name:-}" == "unit_tests" ]]; then
local_arg+=" --config=remote_test" local_arg+=" --config=remote_test"
local_arg+=" --test_timeout=660" # Allow extra 60s for coredump on abort
fi fi
if bazel_evergreen_shutils::is_ppc64le; then if bazel_evergreen_shutils::is_ppc64le; then

View File

@ -8,30 +8,29 @@
DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" >/dev/null 2>&1 && pwd)" DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" >/dev/null 2>&1 && pwd)"
. "$DIR/prelude.sh" . "$DIR/prelude.sh"
. "$DIR/bazel_evergreen_shutils.sh"
cd src
set -o errexit set -o errexit
set -o verbose set -o verbose
activate_venv bazel_evergreen_shutils::activate_and_cd_src
source ./evergreen/bazel_evergreen_shutils.sh
BAZEL_BINARY=$(bazel_evergreen_shutils::bazel_get_binary_path) BAZEL_BINARY=$(bazel_evergreen_shutils::bazel_get_binary_path)
# Timeout is set here to avoid the build hanging indefinitely, still allowing if [ -z "${multi_suite_resmoke_task}" ]; then
# for retries. ci_flags="\
TIMEOUT_CMD=""
if [ -n "${build_timeout_seconds}" ]; then
TIMEOUT_CMD="timeout ${build_timeout_seconds}"
fi
ci_flags="\
--//bazel/resmoke:in_evergreen \ --//bazel/resmoke:in_evergreen \
--//bazel/resmoke:installed_dist_test \
--test_output=all \ --test_output=all \
--noincompatible_enable_cc_toolchain_resolution \ --noincompatible_enable_cc_toolchain_resolution \
--repo_env=no_c++_toolchain=1" --repo_env=no_c++_toolchain=1"
else
ci_flags="--//bazel/resmoke:in_evergreen"
# For simple build ID generation:
export compile_variant="${compile_variant}"
export version_id="${version_id}"
fi
if [[ "${evergreen_remote_exec}" == "on" ]]; then if [[ "${evergreen_remote_exec}" == "on" ]]; then
ci_flags="--config=remote_test ${ci_flags}" ci_flags="--config=remote_test ${ci_flags}"
@ -62,74 +61,113 @@ for strategy in "${strategies[@]}"; do
ci_flags+=" --test_arg=--evergreenTestSelectionStrategy=${strategy}" ci_flags+=" --test_arg=--evergreenTestSelectionStrategy=${strategy}"
done done
# If not explicitly specified on the target, pick a shard count that will fully utilize the current machine. ALL_FLAGS="${ci_flags} ${LOCAL_ARG} ${bazel_args:-} ${bazel_compile_flags:-} ${task_compile_flags:-} ${patch_compile_flags:-}"
BUILD_INFO=$(bazel query ${targets} --output build) echo "${ALL_FLAGS}" >.bazel_build_flags
if [[ "$BUILD_INFO" != *"shard_count ="* ]] && [[ "${bazel_args} ${bazel_compile_flags} ${task_compile_flags} ${patch_compile_flags}" != *"test_sharding_strategy"* ]]; then
CPUS=$(nproc) # Save the invocation, intentionally excluding CI specific flags.
SIZE=$(echo $BUILD_INFO | grep "size =" | cut -d '"' -f2) echo "python buildscripts/install_bazel.py" >bazel-invocation.txt
TEST_RESOURCES_CPU=$(echo ${bazel_args} ${bazel_compile_flags} ${task_compile_flags} ${patch_compile_flags} | awk -F'--default_test_resources=cpu=' '{print $2}') echo "bazel test ${bazel_args} ${targets}" >>bazel-invocation.txt
TEST_RESOURCES_CPU=(${TEST_RESOURCES_CPU//,/ })
declare -A SIZES
SIZES=(["small"]=0 ["medium"]=1 ["large"]=2 ["enormous"]=3)
CPUS_PER_SHARD=${TEST_RESOURCES_CPU[${SIZES[$SIZE]}]}
SHARD_COUNT=$((CPUS / $CPUS_PER_SHARD))
ci_flags+=" --test_sharding_strategy=forced=$SHARD_COUNT"
fi
set +o errexit set +o errexit
for i in {1..3}; do # Fetch then test with retries.
eval ${TIMEOUT_CMD} ${BAZEL_BINARY} fetch ${ci_flags} ${bazel_args} ${bazel_compile_flags} ${task_compile_flags} ${patch_compile_flags} ${targets} && RET=0 && break || RET=$? && sleep 60 export RETRY_ON_FAIL=1
if [ $RET -eq 124 ]; then bazel_evergreen_shutils::retry_bazel_cmd 3 "$BAZEL_BINARY" \
echo "Bazel fetch timed out after ${build_timeout_seconds} seconds, retrying..." fetch ${ci_flags} ${bazel_args} ${bazel_compile_flags} ${task_compile_flags} ${patch_compile_flags} ${targets}
else
echo "Bazel fetch failed, retrying..."
fi
$BAZEL_BINARY shutdown
done
# Save the invocation, intentionally excluding ci_flags.
echo "python buildscripts/install_bazel.py" >bazel-invocation.txt
echo "bazel test ${bazel_args} ${bazel_compile_flags} ${task_compile_flags} ${patch_compile_flags} ${targets}" >>bazel-invocation.txt
eval ${BAZEL_BINARY} test ${ci_flags} ${bazel_args} ${bazel_compile_flags} ${task_compile_flags} ${patch_compile_flags} ${targets}
RET=$? RET=$?
if [[ "$RET" == "0" ]]; then
export RETRY_ON_FAIL=0
bazel_evergreen_shutils::retry_bazel_cmd 3 "$BAZEL_BINARY" \
test ${ci_flags} ${bazel_args} ${bazel_compile_flags} ${task_compile_flags} ${patch_compile_flags} ${targets}
RET=$?
if [[ "$RET" -eq 124 ]]; then
echo "Bazel timed out after ${build_timeout_seconds:-<unspecified>} seconds."
elif [[ "$RET" != "0" ]]; then
echo "Errors were found during bazel test, failing the execution"
fi
fi
bazel_evergreen_shutils::write_last_engflow_link
set -o errexit set -o errexit
# Symlink data directories to where Resmoke normally puts them for compatability with post tasks if [ -z "${multi_suite_resmoke_task}" ]; then
# that run for all Resmoke tasks. # Symlink data directories to where Resmoke normally puts them for compatibility with post tasks
find bazel-testlogs/ -path '*data/job*' -name 'job*' -print0 | # that run for all Resmoke tasks.
while IFS= read -r -d '' test_outputs; do find bazel-testlogs/ -path '*data/job*' -name 'job*' -print0 |
source=${workdir}/src/$test_outputs while IFS= read -r -d '' test_outputs; do
target=${workdir}/$(sed 's/.*\.outputs\///' <<<$test_outputs) source=${workdir}/src/$test_outputs
mkdir -p $(dirname $target) target=${workdir}/$(sed 's/.*\.outputs\///' <<<$test_outputs)
ln -sf $source $target mkdir -p $(dirname $target)
done ln -sf $source $target
done
# Symlink test logs to where Evergreen expects them. Evergreen won't read into a symlinked directory, # Symlink test logs to where Evergreen expects them. Evergreen won't read into a symlinked directory,
# so symlink each log file individually. # so symlink each log file individually.
find bazel-testlogs/ -type f -path "*TestLogs/*" -print0 | find bazel-testlogs/ -type f -path "*TestLogs/*" -print0 |
while IFS= read -r -d '' test_outputs; do while IFS= read -r -d '' test_outputs; do
source=${workdir}/src/$test_outputs source=${workdir}/src/$test_outputs
target=${workdir}/$(sed 's/.*\.outputs\///' <<<$test_outputs) target=${workdir}/$(sed 's/.*\.outputs\///' <<<$test_outputs)
mkdir -p $(dirname $target) mkdir -p $(dirname $target)
ln -sf $source $target ln -sf $source $target
done done
# Symlinks archived data directories from multiple tests/shards to a single folder. Evergreen needs a # Symlinks archived data directories from multiple tests/shards to a single folder. Evergreen needs a
# single folder it can glob for s3.put. See the Evergreen function "upload mongodatafiles". # single folder it can glob for s3.put. See the Evergreen function "upload mongodatafiles".
find bazel-testlogs/ -path '*data_archives/*.tgz' -print0 | find bazel-testlogs/ -path '*data_archives/*.tgz' -print0 |
while IFS= read -r -d '' archive; do while IFS= read -r -d '' archive; do
source=${workdir}/src/$archive source=${workdir}/src/$archive
target=${workdir}/$(sed 's/.*\.outputs\///' <<<$archive) target=${workdir}/$(sed 's/.*\.outputs\///' <<<$archive)
echo $source echo $source
echo $target echo $target
mkdir -p $(dirname $target) mkdir -p $(dirname $target)
ln -sf $source $target ln -sf $source $target
done done
# Combine reports from potentially multiple tests/shards. # Combine reports from potentially multiple tests/shards.
find bazel-testlogs/ -name report*.json | xargs $python buildscripts/combine_reports.py --no-report-exit -o report.json find bazel-testlogs/ -name report*.json | xargs $python buildscripts/combine_reports.py --no-report-exit -o report.json
else
# Symlink data directories to where Resmoke normally puts them for compatibility with post tasks
# that run for all Resmoke tasks.
find bazel-testlogs/ -path '*data/job*' -name 'job*' -print0 |
while IFS= read -r -d '' test_outputs; do
source=${workdir}/src/$test_outputs
target=${workdir}/$(sed 's/.*\.outputs\///' <<<$test_outputs)
mkdir -p $(dirname $target)
ln -sf $source $target
done
# Symlinks archived data directories from multiple tests/shards to a single folder. Evergreen needs a
# single folder it can glob for s3.put. See the Evergreen function "upload mongodatafiles".
target_from_undeclared_outputs() {
echo ${1} | sed -e 's/^.*bazel-testlogs\/\(.*\)\/test.outputs.*$/\1/'
}
find bazel-testlogs/ -path '*data_archives/*.tgz' -print0 |
while IFS= read -r -d '' archive; do
source=${workdir}/src/$archive
bazel_target_prefix=$(target_from_undeclared_outputs $archive | sed 's/\//_/g')
target=${workdir}/$(echo $archive | sed -e 's/.*\.outputs\///' -e "s/data_archives\//&$bazel_target_prefix-/g")
mkdir -p $(dirname $target)
ln -sf $source $target
done
# Symlinks test.log from multiple tests/shards to a single folder. Evergreen needs a
# single folder it can glob for s3.put. See the Evergreen function "upload bazel test logs".
find bazel-testlogs/ -path '*test.log' -print0 |
while IFS= read -r -d '' log; do
log_renamed=$(echo $log | sed -e 's/bazel-testlogs\///g' -e 's/\//_/g')
source=${workdir}/src/$log
target=${workdir}/tmp/bazel-testlogs/$log_renamed
mkdir -p $(dirname $target)
ln -sf $source $target
done
echo "format: text-timestamp" >${workdir}/build/TestLogs/log_spec.yaml
echo "version: 0" >>${workdir}/build/TestLogs/log_spec.yaml
# Combine reports from potentially multiple tests/shards.
find bazel-testlogs/ -name report*.json | xargs $python buildscripts/combine_reports.py --no-report-exit --add-bazel-target-info -o report.json
fi
exit $RET exit $RET