From ee2a8da87a9bdc360d8e7b99061ed643fb934fc0 Mon Sep 17 00:00:00 2001 From: Kai Fricke Date: Thu, 22 Sep 2022 11:58:41 +0100 Subject: [PATCH] [ci] Move to new hierarchical docker structure + pipeline (#28641) This PR moves our buildkite pipeline to a new hierarchical structure and will be used with the new buildkite pipeline. When merging this PR, the old behavior will still work, i.e. the old pipeline is still in place. After merging this PR, we can build the base images for the master branch, and then switch the CI pipelines to use the new build structure. Once this switch has been done, the following files will be removed: - `./buildkite/pipeline.yml` - this has been split into pipeline.test.yml and pipeline.build.yml - `./buildkite/Dockerfile` - this has been moved (and split) to `./ci/docker/` - `./buildkite/Dockerfile.gpu` - this has been moved (and split) to `./ci/docker/` The new structure is as follow: - `./ci/docker` contains hierarchical docker files that will be built by the pipeline. - `Dockerfile.base_test` contains common dependencies - `Dockerfile.base_build` inherits from it and adds build-specific dependencies, e.g. llvm, nvm, java - `Dockerfile.base_ml` inherits from `base_test` and adds ML dependencies, e.g. torch, tensorflow - `Dockerfile.base_gpu` depends on a cuda image and otherwise has the same contents as `base_test` and `base_ml` combined In each build, we do the following - `Dockerfile.build` is built on top of `Dockerfile.base_build`. Dependencies are re-installed, which is mostly a no-op (except if they changed from when the base image was built) - `Dockerfile.test` is built on top of `Dockerfile.base_test`, and the extracted Ray installation from`Dockerfile.build` is injected - The same is true respectively for `ml` and `gpu`. The pipelines have been split, and a new attribute `NO_WHEELS_REQUIRED` is added, identifying tests that can be early-started. Early start means that the last available branch image is used and the current code revision is checked out upon it. See https://github.com/ray-project/buildkite-ci-pipelines/ for the pipeline logic. Additionally, this PR identified two CI regressions that haven't been caught previously, namely the minimal install tests that didn't properly install the respective Python versions, and some runtime environment tests that don't work with later Ray versions. These should be addressed separately and I'll create issues for them once this PR is merged. Signed-off-by: Kai Fricke Signed-off-by: Artur Niederfahrenhorst Co-authored-by: Artur Niederfahrenhorst --- .buildkite/Dockerfile | 5 +- .buildkite/Dockerfile.gpu | 3 + .buildkite/README.md | 30 ++ .buildkite/pipeline.build.yml | 571 +++++++++++++++++++++ .buildkite/pipeline.gpu.large.yml | 22 +- .buildkite/pipeline.gpu.yml | 13 +- .buildkite/pipeline.gpu_large.yml | 61 +++ .buildkite/pipeline.macos.yml | 2 + .buildkite/pipeline.ml.yml | 177 +++---- .buildkite/pipeline.test.yml | 30 ++ .buildkite/pipeline.windows.yml | 2 + .buildkite/pipeline.yml | 18 +- .flake8 | 1 + ci/ci.sh | 9 +- ci/docker/README.md | 29 ++ ci/docker/base.build.Dockerfile | 10 + ci/docker/base.gpu.Dockerfile | 69 +++ ci/docker/base.ml.Dockerfile | 15 + ci/docker/base.test.Dockerfile | 65 +++ ci/docker/build.Dockerfile | 35 ++ ci/docker/gpu.Dockerfile | 29 ++ ci/docker/ml.Dockerfile | 18 + ci/docker/test.Dockerfile | 18 + ci/env/install-dependencies.sh | 55 +- ci/env/install-minimal.sh | 10 +- python/ray/tests/test_basic_3.py | 4 + python/ray/tests/test_runtime_env.py | 4 + python/ray/tests/test_runtime_env_2.py | 4 + python/ray/tests/test_state_api.py | 6 +- python/ray/tests/test_state_api_log.py | 2 +- python/ray/tests/test_state_api_summary.py | 2 +- 31 files changed, 1162 insertions(+), 157 deletions(-) create mode 100644 .buildkite/README.md create mode 100644 .buildkite/pipeline.build.yml create mode 100644 .buildkite/pipeline.gpu_large.yml create mode 100644 .buildkite/pipeline.test.yml create mode 100644 ci/docker/README.md create mode 100644 ci/docker/base.build.Dockerfile create mode 100644 ci/docker/base.gpu.Dockerfile create mode 100644 ci/docker/base.ml.Dockerfile create mode 100644 ci/docker/base.test.Dockerfile create mode 100644 ci/docker/build.Dockerfile create mode 100644 ci/docker/gpu.Dockerfile create mode 100644 ci/docker/ml.Dockerfile create mode 100644 ci/docker/test.Dockerfile diff --git a/.buildkite/Dockerfile b/.buildkite/Dockerfile index 13d3d8de279a..6faded1f9e82 100644 --- a/.buildkite/Dockerfile +++ b/.buildkite/Dockerfile @@ -4,7 +4,7 @@ ARG REMOTE_CACHE_URL ARG BUILDKITE_PULL_REQUEST ARG BUILDKITE_COMMIT ARG BUILDKITE_PULL_REQUEST_BASE_BRANCH -ARG PYTHON=3.6 +ARG PYTHON=3.7 ARG INSTALL_DEPENDENCIES ENV DEBIAN_FRONTEND=noninteractive @@ -51,6 +51,9 @@ ENV LC_ALL=en_US.utf8 ENV LANG=en_US.utf8 RUN echo "ulimit -c 0" >> /root/.bashrc +ENV BUILD=1 +ENV DL=1 + # Setup Bazel caches RUN (echo "build --remote_cache=${REMOTE_CACHE_URL}" >> /root/.bazelrc); \ (if [ "${BUILDKITE_PULL_REQUEST}" != "false" ]; then (echo "build --remote_upload_local_results=false" >> /root/.bazelrc); fi); \ diff --git a/.buildkite/Dockerfile.gpu b/.buildkite/Dockerfile.gpu index eabbfd4bf4c6..9232b9d1bfae 100644 --- a/.buildkite/Dockerfile.gpu +++ b/.buildkite/Dockerfile.gpu @@ -53,6 +53,9 @@ ENV LC_ALL=en_US.utf8 ENV LANG=en_US.utf8 RUN echo "ulimit -c 0" >> /root/.bashrc +ENV BUILD=1 +ENV DL=1 + # Setup Bazel caches RUN (echo "build --remote_cache=${REMOTE_CACHE_URL}" >> /root/.bazelrc); \ (if [ "${BUILDKITE_PULL_REQUEST}" != "false" ]; then (echo "build --remote_upload_local_results=false" >> /root/.bazelrc); fi); \ diff --git a/.buildkite/README.md b/.buildkite/README.md new file mode 100644 index 000000000000..9df1111ef3fb --- /dev/null +++ b/.buildkite/README.md @@ -0,0 +1,30 @@ +# Buildkite pipelines + +This directory contains buildkite pipelines used to start CI tests. + +Each step contains a buildkite step that is parsed and executed according to the +[Buildkite pipeline specification](https://buildkite.com/docs/pipelines). + +## Conditions + +An extra optional field `conditions` is defined, which includes conditions under which tests are run. +The script `ci/pipeline/determine_tests_to_run.py` determines changed files in a PR and only kicks off +tests that include at least one of the conditions. If no condition is specified, the test is always run. + +A special case is the `NO_WHEELS_REQUIRED` condition. If this is present, it indicates that the test can +be run with the latest available binaries - in this case the test can be started early, as it will re-use +the latest branch image and only check out the current code revision in the PR. This early kick off will +only trigger on PR builds, not on branch builds. + +## Pipelines + +This directory should be considered with respect to the docker images located in `ci/docker`. + +- `pipeline.build.yml` contains jobs that require build dependencies. This includes all tests that re-build + Ray (e.g. when switching Python versions). The tests are run on the `build.Dockerfile` image. +- `pipeline.test.yml` contains jobs that only require an installed Ray and a small subset of dependencies, + notably exlcuding ML libraries such as Tensorflow or Torch. The tests are run on the `test.Dockerfile` image. +- `pipeline.ml.yml` contains jobs that require ML libraries Tensorflow and Torch to be available. The tests + are run on the `ml.Dockerfile` image. +- `pipeline.gpu.yml` contains jobs that require one GPU. The tests are run on the `gpu.Dockerfile` image. +- `pipeline.gpu.large.yml` contains jobs that require multi-GPUs (currently 4). The tests are run on the `gpu.Dockerfile` image. diff --git a/.buildkite/pipeline.build.yml b/.buildkite/pipeline.build.yml new file mode 100644 index 000000000000..e9b35db67816 --- /dev/null +++ b/.buildkite/pipeline.build.yml @@ -0,0 +1,571 @@ +- label: ":book: Lint" + commands: + - export LINT=1 + - ./ci/env/install-dependencies.sh + - ./ci/ci.sh lint + +- label: ":ferris_wheel: Wheels and Jars" + conditions: + [ + "RAY_CI_LINUX_WHEELS_AFFECTED", + "RAY_CI_JAVA_AFFECTED", + ] + commands: + # Build the wheels and jars + - UPLOAD_WHEELS_AS_ARTIFACTS=1 LINUX_WHEELS=1 LINUX_JARS=1 ./ci/ci.sh build + - bash ./java/build-jar-multiplatform.sh linux + # Upload the wheels and jars + # We don't want to push on PRs, in fact, the copy_files will fail because unauthenticated. + - if [ "$BUILDKITE_PULL_REQUEST" != "false" ]; then exit 0; fi + - pip install -q docker aws_requests_auth boto3 + - ./ci/env/env_info.sh + # Upload to branch directory. + - python .buildkite/copy_files.py --destination branch_wheels --path ./.whl + - python .buildkite/copy_files.py --destination branch_jars --path ./.jar/linux + # Upload to latest directory. + - if [ "$BUILDKITE_BRANCH" == "master" ]; then python .buildkite/copy_files.py --destination wheels --path ./.whl; fi + - if [ "$BUILDKITE_BRANCH" == "master" ]; then python .buildkite/copy_files.py --destination jars --path ./.jar/linux; fi + +- label: ":ferris_wheel: Post-wheel tests" + conditions: ["RAY_CI_LINUX_WHEELS_AFFECTED"] + commands: + - LINUX_WHEELS=1 ./ci/ci.sh build + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - ./ci/env/env_info.sh + - bazel test --config=ci $(./ci/run/bazel_export_options) + --test_tag_filters=post_wheel_build + --test_env=CONDA_EXE + --test_env=CONDA_PYTHON_EXE + --test_env=CONDA_SHLVL + --test_env=CONDA_PREFIX + --test_env=CONDA_DEFAULT_ENV + --test_env=CI + --test_env=RAY_CI_POST_WHEEL_TESTS=True + python/ray/tests/... python/ray/serve/... python/ray/tune/... rllib/... doc/... + +- label: ":ferris_wheel: Debug Wheels" + conditions: + [ + "RAY_CI_LINUX_WHEELS_AFFECTED", + "RAY_CI_JAVA_AFFECTED", + ] + commands: + # Build the debug wheels + - RAY_DEBUG_BUILD=debug LINUX_WHEELS=1 ./ci/ci.sh build + # Upload the wheels. + # We don't want to push on PRs, in fact, the copy_files will fail because unauthenticated. + - if [ "$BUILDKITE_PULL_REQUEST" != "false" ]; then exit 0; fi + - pip install -q docker aws_requests_auth boto3 + - ./ci/env/env_info.sh + # Upload to branch directory. + - python .buildkite/copy_files.py --destination branch_wheels --path ./.whl + # Upload to latest directory. + - if [ "$BUILDKITE_BRANCH" == "master" ]; then python .buildkite/copy_files.py --destination wheels --path ./.whl; fi + +# Not working now. +# - label: ":ferris_wheel: ASAN Wheels" +# conditions: +# [ +# "RAY_CI_LINUX_WHEELS_AFFECTED", +# "RAY_CI_JAVA_AFFECTED", +# ] +# commands: +# # Build the asan wheels +# - RAY_DEBUG_BUILD=asan LINUX_WHEELS=1 ./ci/ci.sh build +# # Upload the wheels. +# # We don't want to push on PRs, in fact, the copy_files will fail because unauthenticated. +# - if [ "$BUILDKITE_PULL_REQUEST" != "false" ]; then exit 0; fi +# - pip install -q docker aws_requests_auth boto3 +# # Upload to branch directory. +# - python .buildkite/copy_files.py --destination branch_wheels --path ./.whl +# # Upload to latest directory. +# - if [ "$BUILDKITE_BRANCH" == "master" ]; then python .buildkite/copy_files.py --destination wheels --path ./.whl; fi + +- label: ":docker: Build Images: py36 (1/2)" + conditions: ["RAY_CI_LINUX_WHEELS_AFFECTED"] + commands: + - LINUX_WHEELS=1 ./ci/ci.sh build + - pip install -q docker aws_requests_auth boto3 + - ./ci/env/env_info.sh + - if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then python .buildkite/copy_files.py --destination docker_login; fi + - python ./ci/build/build-docker-images.py --py-versions py36 --device-types cpu cu101 cu102 cu110 --build-type BUILDKITE --build-base + +- label: ":docker: Build Images: py36 (2/2)" + conditions: ["RAY_CI_LINUX_WHEELS_AFFECTED"] + commands: + - LINUX_WHEELS=1 ./ci/ci.sh build + - pip install -q docker aws_requests_auth boto3 + - ./ci/env/env_info.sh + - if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then python .buildkite/copy_files.py --destination docker_login; fi + - python ./ci/build/build-docker-images.py --py-versions py36 --device-types cu111 cu112 --build-type BUILDKITE --build-base + +- label: ":docker: Build Images: py37 (1/2)" + conditions: ["RAY_CI_LINUX_WHEELS_AFFECTED"] + commands: + - LINUX_WHEELS=1 ./ci/ci.sh build + - pip install -q docker aws_requests_auth boto3 + - ./ci/env/env_info.sh + - if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then python .buildkite/copy_files.py --destination docker_login; fi + - python ./ci/build/build-docker-images.py --py-versions py37 --device-types cpu cu101 cu102 cu110 --build-type BUILDKITE --build-base + +- label: ":docker: Build Images: py37 (2/2)" + conditions: ["RAY_CI_LINUX_WHEELS_AFFECTED"] + commands: + - LINUX_WHEELS=1 ./ci/ci.sh build + - pip install -q docker aws_requests_auth boto3 + - ./ci/env/env_info.sh + - if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then python .buildkite/copy_files.py --destination docker_login; fi + - python ./ci/build/build-docker-images.py --py-versions py37 --device-types cu111 cu112 cu113 cu116 --build-type BUILDKITE --build-base + +- label: ":docker: Build Images: py38 (1/2)" + conditions: ["RAY_CI_LINUX_WHEELS_AFFECTED"] + commands: + - LINUX_WHEELS=1 ./ci/ci.sh build + - pip install -q docker aws_requests_auth boto3 + - ./ci/env/env_info.sh + - if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then python .buildkite/copy_files.py --destination docker_login; fi + - python ./ci/build/build-docker-images.py --py-versions py38 --device-types cpu cu101 cu102 cu110 --build-type BUILDKITE --build-base + +- label: ":docker: Build Images: py38 (2/2)" + conditions: ["RAY_CI_LINUX_WHEELS_AFFECTED"] + commands: + - LINUX_WHEELS=1 ./ci/ci.sh build + - pip install -q docker aws_requests_auth boto3 + - ./ci/env/env_info.sh + - if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then python .buildkite/copy_files.py --destination docker_login; fi + - python ./ci/build/build-docker-images.py --py-versions py38 --device-types cu111 cu112 cu113 cu116 --build-type BUILDKITE --build-base + +- label: ":docker: Build Images: py39 (1/2)" + conditions: ["RAY_CI_LINUX_WHEELS_AFFECTED"] + commands: + - LINUX_WHEELS=1 ./ci/ci.sh build + - pip install -q docker aws_requests_auth boto3 + - ./ci/env/env_info.sh + - if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then python .buildkite/copy_files.py --destination docker_login; fi + - python ./ci/build/build-docker-images.py --py-versions py39 --device-types cpu cu101 cu102 cu110 --build-type BUILDKITE --build-base + +- label: ":docker: Build Images: py39 (2/2)" + conditions: ["RAY_CI_LINUX_WHEELS_AFFECTED"] + commands: + - LINUX_WHEELS=1 ./ci/ci.sh build + - pip install -q docker aws_requests_auth boto3 + - ./ci/env/env_info.sh + - if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then python .buildkite/copy_files.py --destination docker_login; fi + - python ./ci/build/build-docker-images.py --py-versions py39 --device-types cu111 cu112 cu113 cu116 --build-type BUILDKITE --build-base + +- label: ":docker: Build Images: py310 (1/2)" + conditions: ["RAY_CI_LINUX_WHEELS_AFFECTED"] + commands: + - LINUX_WHEELS=1 ./ci/ci.sh build + - pip install -q docker aws_requests_auth boto3 + - ./ci/env/env_info.sh + - if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then python .buildkite/copy_files.py --destination docker_login; fi + - python ./ci/build/build-docker-images.py --py-versions py310 --device-types cpu cu101 cu102 --build-type BUILDKITE --build-base + +- label: ":docker: Build Images: py310 (2/2)" + conditions: ["RAY_CI_LINUX_WHEELS_AFFECTED"] + commands: + - LINUX_WHEELS=1 ./ci/ci.sh build + - pip install -q docker aws_requests_auth boto3 + - if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then python .buildkite/copy_files.py --destination docker_login; fi + - python ./ci/build/build-docker-images.py --py-versions py310 --device-types cu110 cu111 cu112 --build-type BUILDKITE --build-base + +- label: ":java: Java" + conditions: ["RAY_CI_JAVA_AFFECTED"] + commands: + - ./java/test.sh + +- label: ":cpp: Ray CPP Worker" + conditions: [ "RAY_CI_CPP_AFFECTED" ] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - ./ci/ci.sh test_cpp + +- label: ":cpp: Tests" + conditions: [ "RAY_CI_CORE_CPP_AFFECTED" ] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - bazel test --config=ci --config=llvm $(./ci/run/bazel_export_options) + --build_tests_only + -- //:all -rllib/... -core_worker_test + +- label: ":cpp: Tests (ASAN)" + conditions: [ "RAY_CI_CORE_CPP_AFFECTED" ] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - bazel test --config=ci --config=asan-clang $(./ci/run/bazel_export_options) + --build_tests_only + --jobs=2 + -- //:all -//:core_worker_test + +- label: ":cpp: Tests (UBSAN)" + conditions: [ "RAY_CI_CORE_CPP_AFFECTED" ] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - bazel test --config=ci --config=ubsan $(./ci/run/bazel_export_options) + --build_tests_only + --jobs=2 + -- //:all -//:core_worker_test -//:logging_test -//:ray_syncer_test + +- label: ":cpp: Tests (TSAN)" + conditions: [ "RAY_CI_CORE_CPP_AFFECTED" ] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - bazel test --config=ci --config=tsan-clang $(./ci/run/bazel_export_options) + --build_tests_only + --jobs=2 + -- //:all -//:core_worker_test -//:event_test -//:gcs_actor_manager_test + -//:gcs_placement_group_manager_test -//:gcs_placement_group_scheduler_test + -//:gcs_server_rpc_test -//:gcs_client_test -//:gcs_heartbeat_manager_test + -//:metric_exporter_client_test -//:stats_test -//:worker_pool_test + -//:ray_syncer_test + +- label: ":serverless: Dashboard Tests" + conditions: + [ + "RAY_CI_DASHBOARD_AFFECTED", + "RAY_CI_PYTHON_AFFECTED", + ] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - ./ci/env/env_info.sh + - ./dashboard/tests/run_ui_tests.sh + - bazel test --config=ci $(./ci/run/bazel_export_options) python/ray/dashboard/... + +- label: ":serverless: Serve Release Tests" + conditions: + [ + "RAY_CI_SERVE_AFFECTED", + "RAY_CI_PYTHON_AFFECTED", + ] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - TORCH_VERSION=1.6 ./ci/env/install-dependencies.sh + - 'git clone https://github.com/wg/wrk.git /tmp/wrk && pushd /tmp/wrk && make -j && sudo cp wrk /usr/local/bin && popd' + - ./ci/env/env_info.sh + - bazel test --config=ci $(./ci/run/bazel_export_options) + --test_tag_filters=team:serve + release/... + +- label: ":serverless: Serve Tests" + parallelism: 3 + conditions: + [ + "RAY_CI_SERVE_AFFECTED", + "RAY_CI_PYTHON_AFFECTED", + "RAY_CI_ML_AFFECTED", + ] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - TORCH_VERSION=1.6 ./ci/env/install-dependencies.sh + - 'git clone https://github.com/wg/wrk.git /tmp/wrk && pushd /tmp/wrk && make -j && sudo cp wrk /usr/local/bin && popd' + - ./ci/env/env_info.sh + - >- + set -x; + python ./ci/run/bazel-sharding.py + --exclude_manual + --index "\${BUILDKITE_PARALLEL_JOB}" --count "\${BUILDKITE_PARALLEL_JOB_COUNT}" + python/ray/serve/... + > test_shard.txt + - cat test_shard.txt + - bazel test --config=ci $(./ci/run/bazel_export_options) + --test_tag_filters=-post_wheel_build,-py37,-gpu + $(cat test_shard.txt) + + +- label: ":serverless: Serve Tests (Python 3.7)" + conditions: + [ + "RAY_CI_SERVE_AFFECTED", + "RAY_CI_PYTHON_AFFECTED", + ] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - echo "--- Setting up Python 3.7 environment." + - TORCH_VERSION=1.6 ./ci/env/install-dependencies.sh + # Specifying above somehow messes up the Ray install. + # Uninstall and re-install Ray so that we can use Ray Client. + # (Remove thirdparty_files to sidestep an issue with psutil.) + - pip uninstall -y ray && rm -rf /ray/python/ray/thirdparty_files + - ./ci/ci.sh build + - bazel test --config=ci $(./ci/run/bazel_export_options) + --test_tag_filters=team:serve + python/ray/serve/test_gradio + python/ray/serve/test_gradio_visualization + + +- label: ":python: Minimal install 3.6" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - ./ci/ci.sh test_minimal 3.6 + +- label: ":python: Minimal install 3.7" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - ./ci/ci.sh test_minimal 3.7 + +- label: ":python: Minimal install 3.8" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - ./ci/ci.sh test_minimal 3.8 + +- label: ":python: Minimal install 3.9" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - ./ci/ci.sh test_minimal 3.9 + +- label: ":python: Minimal install 3.10" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - ./ci/ci.sh test_minimal 3.10 + +- label: ":python: Default install" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - ./ci/env/install-default.sh + - ./ci/env/env_info.sh + - bazel test --test_output=streamed --config=ci --test_env=RAY_DEFAULT=1 $(./ci/run/bazel_export_options) + python/ray/dashboard/test_dashboard + +- label: ":python: Ray Serve default install" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - ./ci/env/install-serve.sh + - ./ci/env/env_info.sh + - bazel test --test_output=streamed --config=ci --test_env=RAY_DEFAULT=1 $(./ci/run/bazel_export_options) + python/ray/serve/test_deployment_graph + - bazel test --test_output=streamed --config=ci --test_env=RAY_DEFAULT=1 $(./ci/run/bazel_export_options) + python/ray/serve/test_api + +- label: ":python: Release test package unit tests" + conditions: ["ALWAYS"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - pip install -e release/ + - ./ci/env/env_info.sh + - bazel test --config=ci $(./ci/run/bazel_export_options) + --build_tests_only + --test_tag_filters=release_unit + release/... + +- label: ":python: (Small & Client)" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - DL=1 ./ci/env/install-dependencies.sh + - bash ./ci/ci.sh prepare_docker + - ./ci/env/env_info.sh + - bazel test --config=ci $(./ci/run/bazel_export_options) + --test_tag_filters=client_tests,small_size_python_tests + -- python/ray/tests/... + - bazel test --config=ci $(./ci/run/bazel_export_options) + --test_tag_filters=ray_ha + --test_env=DOCKER_HOST=tcp://docker:2376 + --test_env=DOCKER_TLS_VERIFY=1 + --test_env=DOCKER_CERT_PATH=/certs/client + --test_env=DOCKER_TLS_CERTDIR=/certs + -- python/ray/tests/... + +- label: ":python: (Large)" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + parallelism: 3 + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - DL=1 ./ci/env/install-dependencies.sh + - ./ci/env/env_info.sh + - ./ci/ci.sh test_large + +- label: ":python: (Medium A-J)" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - ./ci/env/env_info.sh + - bazel test --config=ci $(./ci/run/bazel_export_options) + --test_tag_filters=-kubernetes,medium_size_python_tests_a_to_j + python/ray/tests/... + +- label: ":python: (Medium K-Z)" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - DL=1 ./ci/env/install-dependencies.sh + - bazel test --config=ci $(./ci/run/bazel_export_options) + --test_tag_filters=-kubernetes,medium_size_python_tests_k_to_z + python/ray/tests/... + +- label: ":redis: (External Redis) (Small & Client)" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - DL=1 ./ci/env/install-dependencies.sh + - ./ci/env/env_info.sh + - bazel test --config=ci $(./scripts/bazel_export_options) + --test_tag_filters=client_tests,small_size_python_tests + --test_env=TEST_EXTERNAL_REDIS=1 + -- python/ray/tests/... + +- label: ":redis: (External Redis) (Large)" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + parallelism: 3 + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - DL=1 ./ci/env/install-dependencies.sh + - ./ci/env/env_info.sh + - TEST_EXTERNAL_REDIS=1 ./ci/ci.sh test_large + +- label: ":redis: (External Redis) (Medium A-J)" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - ./ci/env/env_info.sh + - bazel test --config=ci $(./scripts/bazel_export_options) + --test_tag_filters=-kubernetes,medium_size_python_tests_a_to_j + --test_env=TEST_EXTERNAL_REDIS=1 + -- //python/ray/tests/... + +- label: ":redis: (External Redis) (Medium K-Z)" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - DL=1 ./ci/env/install-dependencies.sh + - ./ci/env/env_info.sh + - bazel test --config=ci $(./scripts/bazel_export_options) + --test_tag_filters=-kubernetes,medium_size_python_tests_k_to_z + --test_env=TEST_EXTERNAL_REDIS=1 + -- //python/ray/tests/... + +- label: ":python: Debug Test" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - pip uninstall -y ray + - RAY_DEBUG_BUILD=debug ./ci/ci.sh build + - ./ci/env/env_info.sh + - bazel test --config=ci-debug $(./ci/run/bazel_export_options) + --test_tag_filters=-kubernetes,debug_tests + python/ray/tests/... + +- label: ":python: (ASAN tests)" + conditions: ["RAY_CI_PYTHON_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh + - pip install "grpcio >= 1.28.1, <= 1.43.0" + - ./ci/env/env_info.sh + - bazel test --config=ci --config=asan $(./ci/run/bazel_export_options) + --config=asan-buildkite + --test_tag_filters=-kubernetes,asan_tests + --test_env=CONDA_EXE + --test_env=CONDA_PYTHON_EXE + --test_env=CONDA_SHLVL + --test_env=CONDA_PREFIX + --test_env=CONDA_DEFAULT_ENV + python/ray/tests/... + +# https://github.com/ray-project/ray/issues/22460 +#- label: ":python: (Privileged test)" + #conditions: ["RAY_CI_PYTHON_AFFECTED"] + #commands: + #- LINUX_WHEELS=1 ./ci/ci.sh build + #- pip install docker + #We build image ray-worker-container:nightly-py36-cpu which have installed podman,but not push it. + #And we save this image to a tarball, so that we can load it to podman image storage in the + #nested-container which run tests. And in this nested-container, Raylet will start ray worker + #process in double-nested-container. + #- python ./ci/build/build-docker-images.py --py-versions py36 --device-types cpu --build-type BUILDKITE --only-build-worker-container + #- mkdir /ray-mount/containers + #- docker save -o /ray-mount/containers/images.tar rayproject/ray-worker-container:nightly-py36-cpu + #- docker run --rm --privileged -v /ray/containers:/var/lib/containers -v /ray:/ray --entrypoint /bin/bash + #rayproject/ray-worker-container:nightly-py36-cpu /ray/ci/build/test-worker-in-container.sh + +- label: ":kubernetes: operator" + conditions: ["RAY_CI_LINUX_WHEELS_AFFECTED"] + commands: + - | + cleanup() { + if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi + python python/ray/tests/kuberay/setup/teardown_kuberay.py || true + kind delete cluster + } + trap cleanup EXIT + - echo "--- Setting up Python 3.7 environment." + - ./ci/env/install-dependencies.sh + # Specifying above somehow messes up the Ray install. + # Uninstall and re-install Ray so that we can use Ray Client. + # (Remove thirdparty_files to sidestep an issue with psutil.) + - pip uninstall -y ray && rm -rf /ray/python/ray/thirdparty_files + - pip install -e /ray/python + - echo "--- Setting up local kind cluster." + - ./ci/k8s/prep-k8s-environment.sh + - echo "--- Building py37-cpu Ray image for the test." + - LINUX_WHEELS=1 ./ci/ci.sh build + - pip install -q docker + - python ci/build/build-docker-images.py --py-versions py37 --device-types cpu --build-type LOCAL --build-base + # Tag the image built in the last step. We want to be sure to distinguish the image from the real Ray nightly. + - docker tag rayproject/ray:nightly-py37-cpu ray-ci:kuberay-test + # Load the image into the kind node. + - kind load docker-image ray-ci:kuberay-test + - echo "--- Setting up KubeRay operator." + - python python/ray/tests/kuberay/setup/setup_kuberay.py + - ./ci/env/env_info.sh + - echo "--- Running the test." + - bazel test --config=ci $(./ci/run/bazel_export_options) + --test_tag_filters=kuberay_operator + --test_env=RAY_IMAGE=docker.io/library/ray-ci:kuberay-test + --test_env=PULL_POLICY=IfNotPresent + --test_env=KUBECONFIG=/root/.kube/config + python/ray/tests/... + +- label: ":book: Documentation" + commands: + - export LINT=1 + - echo "--- Setting up Python 3.7 environment." + - ./ci/env/install-dependencies.sh + # Specifying above somehow messes up the Ray install. + # Uninstall and re-install Ray so that we can use Ray Client + # (remove thirdparty_files to sidestep an issue with psutil). + - pip uninstall -y ray && rm -rf /ray/python/ray/thirdparty_files + - pushd /ray && git clean -f -f -x -d -e .whl -e python/ray/dashboard/client && popd + - bazel clean --expunge + - ./ci/ci.sh build + +- label: ":octopus: Tune multinode tests" + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TUNE_AFFECTED"] + commands: + - LINUX_WHEELS=1 ./ci/ci.sh build + - mkdir -p ~/.docker/cli-plugins/ && curl -SL https://github.com/docker/compose/releases/download/v2.0.1/docker-compose-linux-x86_64 -o ~/.docker/cli-plugins/docker-compose && chmod +x ~/.docker/cli-plugins/docker-compose + - pip install -U docker aws_requests_auth boto3 + - ./ci/env/env_info.sh + - python ./ci/build/build-docker-images.py --py-versions py37 --device-types cpu --build-type LOCAL --build-base + - python ./ci/build/build-multinode-image.py rayproject/ray:nightly-py37-cpu rayproject/ray:multinode-py37 + - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only + --test_tag_filters=multinode,-example,-flaky,-py37,-soft_imports,-gpu_only,-rllib + python/ray/tune/... + --test_env=RAY_HAS_SSH="1" + --test_env=RAY_DOCKER_IMAGE="rayproject/ray:multinode-py37" + --test_env=RAY_TEMPDIR="/ray-mount" + --test_env=RAY_HOSTDIR="/ray" + --test_env=RAY_TESTHOST="dind-daemon" + --test_env=DOCKER_HOST=tcp://docker:2376 + --test_env=DOCKER_TLS_VERIFY=1 + --test_env=DOCKER_CERT_PATH=/certs/client + --test_env=DOCKER_TLS_CERTDIR=/certs + + + +# Test to see if Train can be used without torch, tf, etc. installed +- label: ":steam_locomotive: Train minimal install" + conditions: ["RAY_CI_TRAIN_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - TRAIN_MINIMAL_INSTALL=1 ./ci/env/install-minimal.sh + - ./ci/env/env_info.sh + - python ./ci/env/check_minimal_install.py + - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=minimal python/ray/train/... diff --git a/.buildkite/pipeline.gpu.large.yml b/.buildkite/pipeline.gpu.large.yml index 69cfae6e33bb..b8950f7b1a85 100644 --- a/.buildkite/pipeline.gpu.large.yml +++ b/.buildkite/pipeline.gpu.large.yml @@ -1,8 +1,8 @@ - label: ":tv: :steam_locomotive: Train GPU tests " - conditions: ["RAY_CI_TRAIN_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TRAIN_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - PYTHON=3.7 TRAIN_TESTING=1 TUNE_TESTING=1 INSTALL_HOROVOD=1 ./ci/env/install-dependencies.sh + - TRAIN_TESTING=1 TUNE_TESTING=1 INSTALL_HOROVOD=1 ./ci/env/install-dependencies.sh # Because Python version changed, we need to re-install Ray here - rm -rf ./python/ray/thirdparty_files; rm -rf ./python/ray/pickle5_files; ./ci/ci.sh build - pip install -Ur ./python/requirements_ml_docker.txt @@ -10,10 +10,10 @@ - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=gpu,gpu_only,-ray_air,-torch_1_11 python/ray/train/... - label: ":tv: :steam_locomotive: Train GPU tests (PyTorch 1.11) " - conditions: ["RAY_CI_TRAIN_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TRAIN_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - PYTHON=3.7 TRAIN_TESTING=1 INSTALL_HOROVOD=1 ./ci/env/install-dependencies.sh + - TRAIN_TESTING=1 INSTALL_HOROVOD=1 ./ci/env/install-dependencies.sh # Because Python version changed, we need to re-install Ray here - rm -rf ./python/ray/thirdparty_files; rm -rf ./python/ray/pickle5_files; ./ci/ci.sh build - pip install -Ur ./python/requirements_ml_docker.txt @@ -23,19 +23,19 @@ - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=torch_1_11 python/ray/train/... - label: ":tv: :database: :steam_locomotive: Datasets Train Integration GPU Tests and Examples (Python 3.7)" - conditions: ["RAY_CI_TRAIN_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TRAIN_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - PYTHON=3.7 TRAIN_TESTING=1 DATA_PROCESSING_TESTING=1 ./ci/env/install-dependencies.sh + - TRAIN_TESTING=1 DATA_PROCESSING_TESTING=1 ./ci/env/install-dependencies.sh - pip install -Ur ./python/requirements_ml_docker.txt - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=datasets_train doc/... - label: ":tv: :brain: RLlib: Multi-GPU Tests" - conditions: ["RAY_CI_RLLIB_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - PYTHON=3.7 RLLIB_TESTING=1 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - pip install -Ur ./python/requirements_ml_docker.txt - ./ci/env/env_info.sh # --jobs 2 is necessary as we only need to have at least 2 gpus on the machine @@ -45,7 +45,7 @@ --test_tag_filters=multi_gpu --test_env=RAY_USE_MULTIPROCESSING_CPU_COUNT=1 rllib/... - label: ":tv: :airplane: ML GPU tests (ray/air)" - conditions: ["RAY_CI_ML_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_ML_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - DATA_PROCESSING_TESTING=1 TRAIN_TESTING=1 TUNE_TESTING=1 INSTALL_HOROVOD=1 ./ci/env/install-dependencies.sh @@ -56,10 +56,10 @@ - label: ":tv: :book: Doc GPU tests and examples" conditions: - ["RAY_CI_PYTHON_AFFECTED", "RAY_CI_TUNE_AFFECTED", "RAY_CI_DOC_AFFECTED"] + ["NO_WHEELS_REQUIRED", "RAY_CI_PYTHON_AFFECTED", "RAY_CI_TUNE_AFFECTED", "RAY_CI_DOC_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - DOC_TESTING=1 TRAIN_TESTING=1 TUNE_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - DOC_TESTING=1 TRAIN_TESTING=1 TUNE_TESTING=1 ./ci/env/install-dependencies.sh - pip install -Ur ./python/requirements_ml_docker.txt - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=gpu,-py37,-post_wheel_build doc/... diff --git a/.buildkite/pipeline.gpu.yml b/.buildkite/pipeline.gpu.yml index cf3ad630d479..e5576941b02a 100644 --- a/.buildkite/pipeline.gpu.yml +++ b/.buildkite/pipeline.gpu.yml @@ -1,6 +1,6 @@ # Todo: Enable once tests are available #- label: ":tv: :octopus: Tune GPU tests " -# conditions: ["RAY_CI_TUNE_AFFECTED"] +# conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TUNE_AFFECTED"] # commands: # - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT # - TUNE_TESTING=1 ./ci/env/install-dependencies.sh @@ -9,10 +9,10 @@ # - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=gpu,gpu_only python/ray/tune/... - label: ":tv: :brain: RLlib: GPU Examples {A/B}" - conditions: ["RAY_CI_RLLIB_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - PYTHON=3.7 RLLIB_TESTING=1 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - pip install -Ur ./python/requirements_ml_docker.txt - ./ci/env/env_info.sh # --jobs 1 is necessary as we only have 1 GPU on the machine and running tests in parallel @@ -24,6 +24,7 @@ - label: ":tv: :serverless: Serve Tests" conditions: [ + "NO_WHEELS_REQUIRED", "RAY_CI_SERVE_AFFECTED", "RAY_CI_PYTHON_AFFECTED", "RAY_CI_ML_AFFECTED", @@ -36,7 +37,7 @@ # Todo: enable once tests pass #- label: ":tv: :brain: RLlib: GPU Examples {C/D}" -# conditions: ["RAY_CI_RLLIB_AFFECTED"] +# conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_AFFECTED"] # commands: # - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT # - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh @@ -47,7 +48,7 @@ # Todo: enable once tests pass #- label: ":tv: :brain: RLlib: GPU Examples {E/P}" -# conditions: ["RAY_CI_RLLIB_AFFECTED"] +# conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_AFFECTED"] # commands: # - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT # - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh @@ -59,7 +60,7 @@ # Todo: enable once tests pass #- label: ":tv: :brain: RLlib: GPU Examples {Q/Z}" -# conditions: ["RAY_CI_RLLIB_AFFECTED"] +# conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_AFFECTED"] # commands: # - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT # - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh diff --git a/.buildkite/pipeline.gpu_large.yml b/.buildkite/pipeline.gpu_large.yml new file mode 100644 index 000000000000..b4dc4b4520b4 --- /dev/null +++ b/.buildkite/pipeline.gpu_large.yml @@ -0,0 +1,61 @@ +- label: ":tv: :steam_locomotive: Train GPU tests " + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TRAIN_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - TRAIN_TESTING=1 TUNE_TESTING=1 INSTALL_HOROVOD=1 ./ci/env/install-dependencies.sh + - pip install -Ur ./python/requirements_ml_docker.txt + - ./ci/env/env_info.sh + - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=gpu,gpu_only,-ray_air,-torch_1_11 python/ray/train/... + +- label: ":tv: :steam_locomotive: Train GPU tests (PyTorch 1.11) " + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TRAIN_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - TRAIN_TESTING=1 INSTALL_HOROVOD=1 ./ci/env/install-dependencies.sh + - pip install -Ur ./python/requirements_ml_docker.txt + - pip uninstall torch -y + - pip install -U torch==1.11.0+cu113 --extra-index-url https://download.pytorch.org/whl/cu113 + - ./ci/env/env_info.sh + - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=torch_1_11 python/ray/train/... + +- label: ":tv: :database: :steam_locomotive: Datasets Train Integration GPU Tests and Examples (Python 3.7)" + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TRAIN_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - TRAIN_TESTING=1 DATA_PROCESSING_TESTING=1 ./ci/env/install-dependencies.sh + - pip install -Ur ./python/requirements_ml_docker.txt + - ./ci/env/env_info.sh + - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=datasets_train doc/... + +- label: ":tv: :brain: RLlib: Multi-GPU Tests" + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh + - pip install -Ur ./python/requirements_ml_docker.txt + - ./ci/env/env_info.sh + # --jobs 2 is necessary as we only need to have at least 2 gpus on the machine + # and running tests in parallel would cause timeouts as the other scripts would + # wait for the GPU to become available. + - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --jobs 2 + --test_tag_filters=multi_gpu --test_env=RAY_USE_MULTIPROCESSING_CPU_COUNT=1 rllib/... + +- label: ":tv: :airplane: ML GPU tests (ray/air)" + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_ML_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - DATA_PROCESSING_TESTING=1 TRAIN_TESTING=1 TUNE_TESTING=1 INSTALL_HOROVOD=1 ./ci/env/install-dependencies.sh + - pip install -Ur ./python/requirements_ml_docker.txt + - ./ci/env/env_info.sh + - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=gpu python/ray/air/... + - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=gpu python/ray/train/... + +- label: ":tv: :book: Doc GPU tests and examples" + conditions: + ["NO_WHEELS_REQUIRED", "RAY_CI_PYTHON_AFFECTED", "RAY_CI_TUNE_AFFECTED", "RAY_CI_DOC_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - DOC_TESTING=1 TRAIN_TESTING=1 TUNE_TESTING=1 ./ci/env/install-dependencies.sh + - pip install -Ur ./python/requirements_ml_docker.txt + - ./ci/env/env_info.sh + - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=gpu,-py37,-post_wheel_build doc/... diff --git a/.buildkite/pipeline.macos.yml b/.buildkite/pipeline.macos.yml index def698b03f61..f3bf5f8c1b3c 100644 --- a/.buildkite/pipeline.macos.yml +++ b/.buildkite/pipeline.macos.yml @@ -7,6 +7,8 @@ common: &common RAY_DEFAULT_BUILD: "1" LC_ALL: en_US.UTF-8 LANG: en_US.UTF-8 + BUILD: "1" + DL: "1" prelude_commands: &prelude_commands |- rm -rf /tmp/bazel_event_logs diff --git a/.buildkite/pipeline.ml.yml b/.buildkite/pipeline.ml.yml index 257188273bbd..a87062d163c7 100644 --- a/.buildkite/pipeline.ml.yml +++ b/.buildkite/pipeline.ml.yml @@ -1,5 +1,5 @@ - label: ":airplane: ML tests (ray/air)" - conditions: ["RAY_CI_ML_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_ML_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - DATA_PROCESSING_TESTING=1 INSTALL_HOROVOD=1 ./ci/env/install-dependencies.sh @@ -9,10 +9,10 @@ - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=ray_air python/ray/data/... - label: ":brain: RLlib: Learning discr. actions TF2-static-graph" - conditions: ["RAY_CI_RLLIB_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only @@ -21,10 +21,10 @@ rllib/... - label: ":brain: RLlib: Learning cont. actions TF2-static-graph" - conditions: ["RAY_CI_RLLIB_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only @@ -33,10 +33,10 @@ rllib/... - label: ":brain: RLlib: Learning discr. actions TF2-eager-tracing" - conditions: ["RAY_CI_RLLIB_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only @@ -45,10 +45,10 @@ rllib/... - label: ":brain: RLlib: Learning cont. actions TF2-eager-tracing" - conditions: ["RAY_CI_RLLIB_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only @@ -57,10 +57,10 @@ rllib/... - label: ":brain: RLlib: Learning discr. actions PyTorch" - conditions: ["RAY_CI_RLLIB_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only @@ -69,10 +69,10 @@ rllib/... - label: ":brain: RLlib: Learning cont. actions PyTorch" - conditions: ["RAY_CI_RLLIB_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only @@ -81,10 +81,10 @@ rllib/... - label: ":brain: RLlib: Learning tests w/ 2 fake GPUs TF2-static-graph" - conditions: ["RAY_CI_RLLIB_DIRECTLY_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_DIRECTLY_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only @@ -94,10 +94,10 @@ # TODO: (sven) tf2 (eager) multi-GPU - label: ":brain: RLlib: Learning tests w/ 2 fake GPUs PyTorch" - conditions: ["RAY_CI_RLLIB_DIRECTLY_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_DIRECTLY_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only @@ -106,10 +106,10 @@ rllib/... - label: ":brain: RLlib: Memory leak tests TF2-eager-tracing" - conditions: ["RAY_CI_RLLIB_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only @@ -118,10 +118,10 @@ rllib/... - label: ":brain: RLlib: Memory leak tests PyTorch" - conditions: ["RAY_CI_RLLIB_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only @@ -130,10 +130,10 @@ rllib/... - label: ":brain: RLlib: Quick Agent train.py runs (TODO: obsolete)" - conditions: ["RAY_CI_RLLIB_DIRECTLY_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_DIRECTLY_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=quick_train,-multi_gpu @@ -141,10 +141,10 @@ rllib/... - label: ":brain: RLlib: Algorithm Tests (generic)" - conditions: ["RAY_CI_RLLIB_DIRECTLY_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_DIRECTLY_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh # Test all tests in the `algorithms` dir: - bazel test --config=ci $(./ci/run/bazel_export_options) @@ -154,10 +154,10 @@ rllib/... - label: ":brain: RLlib: Algorithm Tests (specific algos)" - conditions: ["RAY_CI_RLLIB_DIRECTLY_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_DIRECTLY_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh # Test all tests in the `algorithms` dir: - bazel test --config=ci $(./ci/run/bazel_export_options) @@ -167,10 +167,10 @@ rllib/... - label: ":brain: RLlib: Everything else (env-, evaluation-, ... dirs)" - conditions: ["RAY_CI_RLLIB_DIRECTLY_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_DIRECTLY_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh # Test everything that does not have any of the "main" labels: # "learning_tests|quick_train|examples|tests_dir". @@ -181,106 +181,106 @@ rllib/... - label: ":brain: RLlib: Examples {A..B}" - conditions: ["RAY_CI_RLLIB_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=examples_A,examples_B,-multi_gpu --test_env=RAY_USE_MULTIPROCESSING_CPU_COUNT=1 rllib/... - label: ":brain: RLlib: Examples {Ca..Ct}" - conditions: ["RAY_CI_RLLIB_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=examples_C_AtoT,-multi_gpu --test_env=RAY_USE_MULTIPROCESSING_CPU_COUNT=1 rllib/... - label: ":brain: RLlib: Examples {Cu..Cz}" - conditions: ["RAY_CI_RLLIB_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=examples_C_UtoZ,-multi_gpu --test_env=RAY_USE_MULTIPROCESSING_CPU_COUNT=1 rllib/... - label: ":brain: RLlib: Examples {D..P}" - conditions: ["RAY_CI_RLLIB_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=examples_D,examples_E,examples_F,examples_G,examples_H,examples_I,examples_J,examples_K,examples_L,examples_M,examples_N,examples_O,examples_P,-multi_gpu --test_env=RAY_USE_MULTIPROCESSING_CPU_COUNT=1 rllib/... - label: ":brain: RLlib: Examples {Q..Z}" - conditions: ["RAY_CI_RLLIB_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=examples_Q,examples_R,examples_S,examples_T,examples_U,examples_V,examples_W,examples_X,examples_Y,examples_Z,-multi_gpu --test_env=RAY_USE_MULTIPROCESSING_CPU_COUNT=1 rllib/... - label: ":brain: RLlib: tests/ dir (A..L)" - conditions: ["RAY_CI_RLLIB_DIRECTLY_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_DIRECTLY_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=tests_dir_A,tests_dir_B,tests_dir_C,tests_dir_D,tests_dir_E,tests_dir_F,tests_dir_G,tests_dir_H,tests_dir_I,tests_dir_J,tests_dir_K,tests_dir_L --test_env=RAY_USE_MULTIPROCESSING_CPU_COUNT=1 rllib/... - label: ":brain: RLlib: tests/ dir (M..Z (no R))" - conditions: ["RAY_CI_RLLIB_DIRECTLY_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_DIRECTLY_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=tests_dir_M,tests_dir_N,tests_dir_O,tests_dir_P,tests_dir_Q,tests_dir_S,tests_dir_T,tests_dir_U,tests_dir_V,tests_dir_W,tests_dir_X,tests_dir_Y,tests_dir_Z,-multi_gpu --test_env=RAY_USE_MULTIPROCESSING_CPU_COUNT=1 rllib/... - label: ":brain: RLlib: tests/ dir (R)" - conditions: ["RAY_CI_RLLIB_DIRECTLY_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_DIRECTLY_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=tests_dir_R,-multi_gpu --test_env=RAY_USE_MULTIPROCESSING_CPU_COUNT=1 rllib/... - label: ":brain: RLlib: Documentation code/examples" - conditions: ["RAY_CI_RLLIB_DIRECTLY_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_RLLIB_DIRECTLY_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - RLLIB_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - RLLIB_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=documentation --test_env=RAY_USE_MULTIPROCESSING_CPU_COUNT=1 rllib/... - label: ":octopus: Tune tests {A-R; no RLlib}" - conditions: ["RAY_CI_TUNE_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TUNE_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - TUNE_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - TUNE_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=tests_dir_A,tests_dir_B,tests_dir_C,tests_dir_D,tests_dir_E,tests_dir_F,tests_dir_G,tests_dir_H,tests_dir_I,tests_dir_J,tests_dir_K,tests_dir_L,tests_dir_M,tests_dir_N,tests_dir_O,tests_dir_P,tests_dir_Q,tests_dir_R,-example,-py37,-soft_imports,-gpu_only,-rllib python/ray/tune/... - label: ":octopus: Tune tests {S-Z; no RLlib}" - conditions: ["RAY_CI_TUNE_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TUNE_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - TUNE_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - TUNE_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - python ./ci/env/setup_credentials.py sigopt - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only @@ -288,55 +288,33 @@ python/ray/tune/... -- label: ":octopus: Tune multinode tests" - conditions: [ "RAY_CI_TUNE_AFFECTED" ] - commands: - - LINUX_WHEELS=1 ./ci/ci.sh build - - mkdir -p ~/.docker/cli-plugins/ && curl -SL https://github.com/docker/compose/releases/download/v2.0.1/docker-compose-linux-x86_64 -o ~/.docker/cli-plugins/docker-compose && chmod +x ~/.docker/cli-plugins/docker-compose - - pip install -U docker aws_requests_auth boto3 - - ./ci/env/env_info.sh - - python ./ci/build/build-docker-images.py --py-versions py37 --device-types cpu --build-type LOCAL --build-base - - python ./ci/build/build-multinode-image.py rayproject/ray:nightly-py37-cpu rayproject/ray:multinode-py37 - - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only - --test_tag_filters=multinode,-example,-flaky,-py37,-soft_imports,-gpu_only,-rllib - python/ray/tune/... - --test_env=RAY_HAS_SSH="1" - --test_env=RAY_DOCKER_IMAGE="rayproject/ray:multinode-py37" - --test_env=RAY_TEMPDIR="/ray-mount" - --test_env=RAY_HOSTDIR="/ray" - --test_env=RAY_TESTHOST="dind-daemon" - --test_env=DOCKER_HOST=tcp://docker:2376 - --test_env=DOCKER_TLS_VERIFY=1 - --test_env=DOCKER_CERT_PATH=/certs/client - --test_env=DOCKER_TLS_CERTDIR=/certs - - label: ":octopus: Tune examples {w/o tf/pytorch; no RLlib}" - conditions: ["RAY_CI_TUNE_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TUNE_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - TUNE_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - TUNE_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=example,-tf,-pytorch,-py37,-soft_imports,-gpu_only,-rllib python/ray/tune/... - label: ":octopus: Tune examples {w/ tf/pytorch; no RLlib}" - conditions: ["RAY_CI_TUNE_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TUNE_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - TUNE_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - TUNE_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=tf,-pytorch,-py37,-soft_imports,-gpu_only,-rllib python/ray/tune/... - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=-tf,pytorch,-py37,-soft_imports,-gpu_only,-rllib python/ray/tune/... - label: ":octopus: :brain: Tune tests and examples {using RLlib}" - conditions: ["RAY_CI_TUNE_AFFECTED", "RAY_CI_RLLIB_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TUNE_AFFECTED", "RAY_CI_RLLIB_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - TUNE_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - TUNE_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=-gpu_only,rllib python/ray/tune/... - label: ":steam_locomotive: Train tests and examples" - conditions: ["RAY_CI_TRAIN_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TRAIN_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - TRAIN_TESTING=1 INSTALL_HOROVOD=1 ./ci/env/install-dependencies.sh @@ -344,7 +322,7 @@ - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=-gpu_only,-minimal,-tune,-ray_air python/ray/train/... - label: ":steam_locomotive: :octopus: Train + Tune tests and examples" - conditions: ["RAY_CI_TRAIN_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TRAIN_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - TRAIN_TESTING=1 TUNE_TESTING=1 ./ci/env/install-dependencies.sh @@ -352,18 +330,18 @@ - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=tune,-gpu_only,-ray_air python/ray/train/... - label: ":octopus: Tune tests and examples. Python 3.7" - conditions: ["RAY_CI_TUNE_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TUNE_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - TUNE_TESTING=1 PYTHON=3.7 INSTALL_HOROVOD=1 ./ci/env/install-dependencies.sh + - TUNE_TESTING=1 INSTALL_HOROVOD=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=py37,-client python/ray/tune/... - label: ":octopus: ML library integrations tests and examples. Python 3.7" - conditions: ["RAY_CI_TUNE_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TUNE_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - TUNE_TESTING=1 PYTHON=3.7 INSTALL_HOROVOD=1 ./ci/env/install-dependencies.sh + - TUNE_TESTING=1 INSTALL_HOROVOD=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only python/ray/tests/xgboost/... - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only python/ray/tests/horovod/... @@ -371,45 +349,44 @@ # TODO(amogkam): Re-enable Ludwig tests after Ludwig supports Ray 2.0 #- label: ":octopus: Ludwig tests and examples. Python 3.7" -# conditions: ["RAY_CI_TUNE_AFFECTED"] +# conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TUNE_AFFECTED"] # commands: # - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT -# - PYTHON=3.7 INSTALL_LUDWIG=1 INSTALL_HOROVOD=1 ./ci/env/install-dependencies.sh +# - INSTALL_LUDWIG=1 INSTALL_HOROVOD=1 ./ci/env/install-dependencies.sh # - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only python/ray/tests/ludwig/... - label: ":tropical_fish: ML Libraries w/ Ray Client Examples (Python 3.7)." - conditions: ["RAY_CI_TUNE_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_TUNE_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - TUNE_TESTING=1 PYTHON=3.7 INSTALL_HOROVOD=1 ./ci/env/install-dependencies.sh - - rm -rf ./python/ray/thirdparty_files; rm -rf ./python/ray/pickle5_files; ./ci/ci.sh build + - TUNE_TESTING=1 INSTALL_HOROVOD=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=client --test_env=RAY_CLIENT_MODE=1 python/ray/util/dask/... - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=client python/ray/tune/... - label: ":potable_water: Dataset library integrations tests and examples. Python 3.7" - conditions: ["RAY_CI_PYTHON_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_PYTHON_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - DATA_PROCESSING_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - DATA_PROCESSING_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only python/ray/tests/modin/... # Dask tests and examples. - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=-client python/ray/util/dask/... - label: ":potable_water: Dataset tests (Python 3.7)" - conditions: ["RAY_CI_PYTHON_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_PYTHON_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - DATA_PROCESSING_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - DATA_PROCESSING_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=-ray_air python/ray/data/... - label: ":potable_water: Workflow tests (Python 3.7)" - conditions: ["RAY_CI_PYTHON_AFFECTED"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_PYTHON_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - DATA_PROCESSING_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - DATA_PROCESSING_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only python/ray/workflow/... @@ -418,7 +395,7 @@ ["RAY_CI_PYTHON_AFFECTED", "RAY_CI_TUNE_AFFECTED", "RAY_CI_DOC_AFFECTED", "RAY_CI_SERVE_AFFECTED", "RAY_CI_ML_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - DOC_TESTING=1 INSTALL_HOROVOD=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - DOC_TESTING=1 INSTALL_HOROVOD=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=-ray_air,-gpu,-py37,-post_wheel_build doc/... @@ -427,16 +404,16 @@ ["RAY_CI_PYTHON_AFFECTED", "RAY_CI_TUNE_AFFECTED", "RAY_CI_DOC_AFFECTED", "RAY_CI_SERVE_AFFECTED", "RAY_CI_ML_AFFECTED"] commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - DOC_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - DOC_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=ray_air,-needs_credentials,-gpu,-py37,-post_wheel_build doc/... - label: ":book: Doc examples with authentication " - conditions: ["RAY_CI_BRANCH_BUILD"] + conditions: ["NO_WHEELS_REQUIRED", "RAY_CI_BRANCH_BUILD"] commands: - if [ "$BUILDKITE_PULL_REQUEST" != "false" ]; then exit 0; fi - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - - DOC_TESTING=1 PYTHON=3.7 ./ci/env/install-dependencies.sh + - DOC_TESTING=1 ./ci/env/install-dependencies.sh - ./ci/env/env_info.sh - python ./ci/env/setup_credentials.py wandb comet_ml - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=needs_credentials,-gpu,-py37,-post_wheel_build doc/... diff --git a/.buildkite/pipeline.test.yml b/.buildkite/pipeline.test.yml new file mode 100644 index 000000000000..1e43f762037c --- /dev/null +++ b/.buildkite/pipeline.test.yml @@ -0,0 +1,30 @@ +- label: ":book: LinkCheck" + commands: + - export LINT=1 + - ./ci/env/install-dependencies.sh + - ./ci/ci.sh check_sphinx_links + soft_fail: True + + +- label: ":octopus: Tune soft imports test" + conditions: ["RAY_CI_TUNE_AFFECTED"] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + # no TUNE_TESTING=1 on purpose + - ./ci/env/install-dependencies.sh + - ./ci/env/env_info.sh + - bazel test --config=ci $(./ci/run/bazel_export_options) --build_tests_only --test_tag_filters=soft_imports python/ray/tune/... + +- label: ":python: Ray DAG Tests" + conditions: + [ + "RAY_CI_PYTHON_AFFECTED", + ] + commands: + - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT + - pip install -U pydot + - sudo apt-get install -y graphviz + - ./ci/env/env_info.sh + - bazel test --config=ci $(./scripts/bazel_export_options) + --test_tag_filters=ray_dag_tests + python/ray/dag/... diff --git a/.buildkite/pipeline.windows.yml b/.buildkite/pipeline.windows.yml index f21b9c20c4ef..fef6b5187115 100644 --- a/.buildkite/pipeline.windows.yml +++ b/.buildkite/pipeline.windows.yml @@ -14,6 +14,8 @@ prelude_commands: &prelude_commands |- export RAY_DEFAULT_BUILD="1" export LC_ALL="en_US.UTF-8" export LANG="en_US.UTF-8" + export BUILD="1" + export DL="1" powershell ci/pipeline/fix-windows-container-networking.ps1 cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT diff --git a/.buildkite/pipeline.yml b/.buildkite/pipeline.yml index 6450aee27e08..1a67360c5980 100644 --- a/.buildkite/pipeline.yml +++ b/.buildkite/pipeline.yml @@ -1,3 +1,9 @@ +### ATTENTION: THIS FILE IS DEPRECATED AND WILL BE REMOVED SHORTLY +### IT HAS BEEN SPLIT INTO TWO FILES: +### - pipeline.build.yml FOR ALL TESTS THAT REQUIRE A FULL BUILD ENV (E.G. LLVM) +### - pipeline.test.yml FOR THE REMAINING TESTS +### IF YOU CHANGE SOMETHING HERE, CHANGE IT IN THE OTHER LOCATIONS, TOO! + - label: ":ferris_wheel: Wheels and Jars" conditions: [ @@ -174,8 +180,8 @@ commands: - export LINT=1 - echo "--- Setting up Python 3.7 environment." - - PYTHON=3.7 ./ci/env/install-dependencies.sh - # Specifying PYTHON=3.7 above somehow messes up the Ray install. + - ./ci/env/install-dependencies.sh + # Specifying above somehow messes up the Ray install. # Uninstall and re-install Ray so that we can use Ray Client # (remove thirdparty_files to sidestep an issue with psutil). - pip uninstall -y ray && rm -rf /ray/python/ray/thirdparty_files @@ -302,8 +308,8 @@ commands: - cleanup() { if [ "${BUILDKITE_PULL_REQUEST}" = "false" ]; then ./ci/build/upload_build_info.sh; fi }; trap cleanup EXIT - echo "--- Setting up Python 3.7 environment." - - PYTHON=3.7 TORCH_VERSION=1.6 ./ci/env/install-dependencies.sh - # Specifying PYTHON=3.7 above somehow messes up the Ray install. + - TORCH_VERSION=1.6 ./ci/env/install-dependencies.sh + # Specifying above somehow messes up the Ray install. # Uninstall and re-install Ray so that we can use Ray Client. # (Remove thirdparty_files to sidestep an issue with psutil.) - pip uninstall -y ray && rm -rf /ray/python/ray/thirdparty_files @@ -529,8 +535,8 @@ } trap cleanup EXIT - echo "--- Setting up Python 3.7 environment." - - PYTHON=3.7 ./ci/env/install-dependencies.sh - # Specifying PYTHON=3.7 above somehow messes up the Ray install. + - ./ci/env/install-dependencies.sh + # Specifying above somehow messes up the Ray install. # Uninstall and re-install Ray so that we can use Ray Client. # (Remove thirdparty_files to sidestep an issue with psutil.) - pip uninstall -y ray && rm -rf /ray/python/ray/thirdparty_files diff --git a/.flake8 b/.flake8 index e37fb41c96b9..82772b8f7a7c 100644 --- a/.flake8 +++ b/.flake8 @@ -12,6 +12,7 @@ max-line-length = 88 inline-quotes = " ignore = C408 + C417 E121 E123 E126 diff --git a/ci/ci.sh b/ci/ci.sh index d2d665698e3d..2b3aaad7bc4b 100755 --- a/ci/ci.sh +++ b/ci/ci.sh @@ -790,8 +790,13 @@ test_minimal() { bazel test --test_output=streamed --config=ci ${BAZEL_EXPORT_OPTIONS} python/ray/tests/test_runtime_env # shellcheck disable=SC2086 bazel test --test_output=streamed --config=ci ${BAZEL_EXPORT_OPTIONS} python/ray/tests/test_runtime_env_2 - # shellcheck disable=SC2086 - bazel test --test_output=streamed --config=ci ${BAZEL_EXPORT_OPTIONS} python/ray/tests/test_runtime_env_complicated + + # Todo: Make compatible with python 3.9/3.10 + if [ "$1" != "3.9" ] && [ "$1" != "3.10" ]; then + # shellcheck disable=SC2086 + bazel test --test_output=streamed --config=ci ${BAZEL_EXPORT_OPTIONS} python/ray/tests/test_runtime_env_complicated + fi + # shellcheck disable=SC2086 bazel test --test_output=streamed --config=ci ${BAZEL_EXPORT_OPTIONS} python/ray/tests/test_runtime_env_validation # shellcheck disable=SC2086 diff --git a/ci/docker/README.md b/ci/docker/README.md new file mode 100644 index 000000000000..132d36e0c6af --- /dev/null +++ b/ci/docker/README.md @@ -0,0 +1,29 @@ +# CI Docker files + +This directory contains the Dockerfiles used to build the CI system. + +These are _not_ the Dockerfiles that build the `rayproject/ray` releases. These +are found in the `/docker` directory under the root. + +The Dockerfiles are hierarchical and will be built in different places during a CI run. + +## Base images + +The base images are built per-branch either when they are first requested or on a periodic basis +(for the master branch). The base images contain the latest dependencies of the respective branch. +Every per-commit build will always install the latest dependencies to make sure everything is up to date. +However, by using the base images as a source, this will mostly be a no or low cost operation. + +- `base.test.Dockerfile` contains common dependencies for all images +- `base.build.Dockerfile` inherits from `base.test` and installs build dependencies like Java and LLVM +- `base.ml.Dockerfile` inherits from `base.test` and installs ML dependencies like torch/tensorflow +- `base.gpu.Dockerfile` inherits from a CUDA base image and otherwise contains the same content as `base.test` and `base.ml`. + +## Per-commit images + +On every commit, the following images are built in this order: + +- `build.Dockerfile` (based on `base.build`) which will build the Ray binaries +- `test.Dockerfile` (based on `base.test`), where we will inject the built Ray libraries +- `ml.Dockerfile` (based on `base.ml`), where we will inject the built Ray libraries +- `gpu.Dockerfile` (based on `base.ml`), where we will inject the built Ray libraries diff --git a/ci/docker/base.build.Dockerfile b/ci/docker/base.build.Dockerfile new file mode 100644 index 000000000000..8c324fc5d2ac --- /dev/null +++ b/ci/docker/base.build.Dockerfile @@ -0,0 +1,10 @@ +ARG DOCKER_IMAGE_BASE_TEST +FROM $DOCKER_IMAGE_BASE_TEST + +ENV RAY_INSTALL_JAVA=1 + +RUN apt-get install -y -qq \ + maven openjdk-8-jre openjdk-8-jdk + +# init also calls install-dependencies.sh (again) +RUN BUILD=1 ./ci/ci.sh init diff --git a/ci/docker/base.gpu.Dockerfile b/ci/docker/base.gpu.Dockerfile new file mode 100644 index 000000000000..6d33fd2d9b25 --- /dev/null +++ b/ci/docker/base.gpu.Dockerfile @@ -0,0 +1,69 @@ +FROM nvidia/cuda:11.2.0-cudnn8-devel-ubuntu20.04 + +ARG REMOTE_CACHE_URL +ARG BUILDKITE_PULL_REQUEST +ARG BUILDKITE_COMMIT +ARG BUILDKITE_PULL_REQUEST_BASE_BRANCH +ARG PYTHON=3.7 + +ENV DEBIAN_FRONTEND=noninteractive +ENV TZ=America/Los_Angeles + +ENV BUILDKITE=true +ENV CI=true +ENV PYTHON=$PYTHON +ENV RAY_USE_RANDOM_PORTS=1 +ENV RAY_DEFAULT_BUILD=1 +ENV RAY_INSTALL_JAVA=0 +ENV BUILDKITE_PULL_REQUEST=${BUILDKITE_PULL_REQUEST} +ENV BUILDKITE_COMMIT=${BUILDKITE_COMMIT} +ENV BUILDKITE_PULL_REQUEST_BASE_BRANCH=${BUILDKITE_PULL_REQUEST_BASE_BRANCH} +# For wheel build +# https://github.com/docker-library/docker/blob/master/20.10/docker-entrypoint.sh +ENV DOCKER_TLS_CERTDIR=/certs +ENV DOCKER_HOST=tcp://docker:2376 +ENV DOCKER_TLS_VERIFY=1 +ENV DOCKER_CERT_PATH=/certs/client +ENV TRAVIS_COMMIT=${BUILDKITE_COMMIT} +ENV BUILDKITE_BAZEL_CACHE_URL=${REMOTE_CACHE_URL} + +RUN apt-key adv --fetch-keys https://developer.download.nvidia.com/compute/cuda/repos/ubuntu1804/x86_64/3bf863cc.pub +RUN apt-key adv --fetch-keys https://developer.download.nvidia.com/compute/machine-learning/repos/ubuntu1804/x86_64/7fa2af80.pub + +RUN apt-get update -qq && apt-get upgrade -qq +RUN apt-get install -y -qq \ + curl python-is-python3 git build-essential \ + sudo unzip unrar apt-utils dialog tzdata wget rsync \ + language-pack-en tmux cmake gdb vim htop \ + libgtk2.0-dev zlib1g-dev libgl1-mesa-dev \ + clang-format-12 jq \ + clang-tidy-12 clang-12 +# Make using GCC 9 explicit. +RUN update-alternatives --install /usr/bin/gcc gcc /usr/bin/gcc-9 90 --slave /usr/bin/g++ g++ /usr/bin/g++-9 \ + --slave /usr/bin/gcov gcov /usr/bin/gcov-9 +RUN ln -s /usr/bin/clang-format-12 /usr/bin/clang-format && \ + ln -s /usr/bin/clang-tidy-12 /usr/bin/clang-tidy && \ + ln -s /usr/bin/clang-12 /usr/bin/clang + +RUN curl -o- https://get.docker.com | sh + +# System conf for tests +RUN locale -a +ENV LC_ALL=en_US.utf8 +ENV LANG=en_US.utf8 +RUN echo "ulimit -c 0" >> /root/.bashrc + +# Setup Bazel caches +RUN (echo "build --remote_cache=${REMOTE_CACHE_URL}" >> /root/.bazelrc); \ + (if [ "${BUILDKITE_PULL_REQUEST}" != "false" ]; then (echo "build --remote_upload_local_results=false" >> /root/.bazelrc); fi); \ + cat /root/.bazelrc + +# Install some dependencies (miniconda, pip dependencies, etc) +RUN mkdir /ray +WORKDIR /ray + +# Below should be re-run each time +COPY . . + +RUN ./ci/env/install-dependencies.sh init +RUN RLLIB_TESTING=1 TRAIN_TESTING=1 TUNE_TESTING=1 bash --login -i ./ci/env/install-dependencies.sh diff --git a/ci/docker/base.ml.Dockerfile b/ci/docker/base.ml.Dockerfile new file mode 100644 index 000000000000..405ec1df0ca4 --- /dev/null +++ b/ci/docker/base.ml.Dockerfile @@ -0,0 +1,15 @@ +ARG DOCKER_IMAGE_BASE_TEST +FROM $DOCKER_IMAGE_BASE_TEST + +# Move out of working dir /ray +# Delete stale data +WORKDIR / +RUN rm -rf /ray + +RUN mkdir /ray +WORKDIR /ray + +# Copy new ray files +COPY . . + +RUN RLLIB_TESTING=1 TRAIN_TESTING=1 TUNE_TESTING=1 ./ci/env/install-dependencies.sh diff --git a/ci/docker/base.test.Dockerfile b/ci/docker/base.test.Dockerfile new file mode 100644 index 000000000000..e03b05144703 --- /dev/null +++ b/ci/docker/base.test.Dockerfile @@ -0,0 +1,65 @@ +FROM ubuntu:focal + +ARG REMOTE_CACHE_URL +ARG BUILDKITE_PULL_REQUEST +ARG BUILDKITE_COMMIT +ARG BUILDKITE_PULL_REQUEST_BASE_BRANCH +ARG PYTHON=3.7 + +ENV DEBIAN_FRONTEND=noninteractive +ENV TZ=America/Los_Angeles + +ENV BUILDKITE=true +ENV CI=true +ENV PYTHON=$PYTHON +ENV RAY_USE_RANDOM_PORTS=1 +ENV RAY_DEFAULT_BUILD=1 +ENV RAY_INSTALL_JAVA=0 +ENV BUILDKITE_PULL_REQUEST=${BUILDKITE_PULL_REQUEST} +ENV BUILDKITE_COMMIT=${BUILDKITE_COMMIT} +ENV BUILDKITE_PULL_REQUEST_BASE_BRANCH=${BUILDKITE_PULL_REQUEST_BASE_BRANCH} +# For wheel build +# https://github.com/docker-library/docker/blob/master/20.10/docker-entrypoint.sh +ENV DOCKER_TLS_CERTDIR=/certs +ENV DOCKER_HOST=tcp://docker:2376 +ENV DOCKER_TLS_VERIFY=1 +ENV DOCKER_CERT_PATH=/certs/client +ENV TRAVIS_COMMIT=${BUILDKITE_COMMIT} +ENV BUILDKITE_BAZEL_CACHE_URL=${REMOTE_CACHE_URL} + +RUN apt-get update -qq && apt-get upgrade -qq +RUN apt-get install -y -qq \ + curl python-is-python3 git build-essential \ + sudo unzip unrar apt-utils dialog tzdata wget rsync \ + language-pack-en tmux cmake gdb vim htop \ + libgtk2.0-dev zlib1g-dev libgl1-mesa-dev \ + clang-format-12 jq \ + clang-tidy-12 clang-12 +# Make using GCC 9 explicit. +RUN update-alternatives --install /usr/bin/gcc gcc /usr/bin/gcc-9 90 --slave /usr/bin/g++ g++ /usr/bin/g++-9 \ + --slave /usr/bin/gcov gcov /usr/bin/gcov-9 +RUN ln -s /usr/bin/clang-format-12 /usr/bin/clang-format && \ + ln -s /usr/bin/clang-tidy-12 /usr/bin/clang-tidy && \ + ln -s /usr/bin/clang-12 /usr/bin/clang + +RUN curl -o- https://get.docker.com | sh + +# System conf for tests +RUN locale -a +ENV LC_ALL=en_US.utf8 +ENV LANG=en_US.utf8 +RUN echo "ulimit -c 0" >> /root/.bashrc + +# Setup Bazel caches +RUN (echo "build --remote_cache=${REMOTE_CACHE_URL}" >> /root/.bazelrc); \ + (if [ "${BUILDKITE_PULL_REQUEST}" != "false" ]; then (echo "build --remote_upload_local_results=false" >> /root/.bazelrc); fi); \ + cat /root/.bazelrc + +# Install some dependencies (miniconda, pip dependencies, etc) +RUN mkdir /ray +WORKDIR /ray + +# Below should be re-run each time +COPY . . + +RUN ./ci/env/install-dependencies.sh init diff --git a/ci/docker/build.Dockerfile b/ci/docker/build.Dockerfile new file mode 100644 index 000000000000..d8877faaf892 --- /dev/null +++ b/ci/docker/build.Dockerfile @@ -0,0 +1,35 @@ +ARG DOCKER_IMAGE_BASE_BUILD +FROM $DOCKER_IMAGE_BASE_BUILD + +ARG BUILDKITE_PULL_REQUEST +ARG BUILDKITE_COMMIT +ARG BUILDKITE_PULL_REQUEST_BASE_BRANCH + +ENV BUILDKITE_PULL_REQUEST=${BUILDKITE_PULL_REQUEST} +ENV BUILDKITE_COMMIT=${BUILDKITE_COMMIT} +ENV BUILDKITE_PULL_REQUEST_BASE_BRANCH=${BUILDKITE_PULL_REQUEST_BASE_BRANCH} +ENV TRAVIS_COMMIT=${BUILDKITE_COMMIT} + +# Move out of working dir /ray +# Delete stale data +WORKDIR / +RUN rm -rf /ray + +RUN mkdir /ray +WORKDIR /ray + +# Below should be re-run each time +COPY . . + +RUN env + +# init also calls install-dependencies.sh +RUN BUILD=1 bash --login -i ./ci/ci.sh init +RUN bash --login -i ./ci/ci.sh build + +RUN export CC=clang CXX=clang++-12 + +# Run determine test to run +RUN bash --login -i -c "python ./ci/pipeline/determine_tests_to_run.py --output=json > affected_set.json" +RUN cat affected_set.json + diff --git a/ci/docker/gpu.Dockerfile b/ci/docker/gpu.Dockerfile new file mode 100644 index 000000000000..88e87e1514e8 --- /dev/null +++ b/ci/docker/gpu.Dockerfile @@ -0,0 +1,29 @@ +ARG DOCKER_IMAGE_BASE_GPU +FROM $DOCKER_IMAGE_BASE_GPU + +ARG BUILDKITE_PULL_REQUEST +ARG BUILDKITE_COMMIT +ARG BUILDKITE_PULL_REQUEST_BASE_BRANCH + +ENV BUILDKITE_PULL_REQUEST=${BUILDKITE_PULL_REQUEST} +ENV BUILDKITE_COMMIT=${BUILDKITE_COMMIT} +ENV BUILDKITE_PULL_REQUEST_BASE_BRANCH=${BUILDKITE_PULL_REQUEST_BASE_BRANCH} +ENV TRAVIS_COMMIT=${BUILDKITE_COMMIT} + +# Move out of working dir /ray +# Delete stale data +WORKDIR / +RUN rm -rf /ray + +RUN mkdir /ray +WORKDIR /ray + +# Copy new ray files +COPY . . + +RUN env + +# Install Ray +RUN SKIP_BAZEL_BUILD=1 RAY_INSTALL_JAVA=0 bash --login -i -c -- "python3 -m pip install -e /ray/python/" + +RUN RLLIB_TESTING=1 TRAIN_TESTING=1 TUNE_TESTING=1 bash --login -i ./ci/env/install-dependencies.sh diff --git a/ci/docker/ml.Dockerfile b/ci/docker/ml.Dockerfile new file mode 100644 index 000000000000..fcdf7aeb9d14 --- /dev/null +++ b/ci/docker/ml.Dockerfile @@ -0,0 +1,18 @@ +ARG DOCKER_IMAGE_BASE_ML +FROM $DOCKER_IMAGE_BASE_ML + +# Move out of working dir /ray +# Delete stale data +WORKDIR / +RUN rm -rf /ray + +RUN mkdir /ray +WORKDIR /ray + +# Copy new ray files +COPY . . + +# Install Ray +RUN SKIP_BAZEL_BUILD=1 RAY_INSTALL_JAVA=0 bash --login -i -c -- "python3 -m pip install -e /ray/python/" + +RUN RLLIB_TESTING=1 TRAIN_TESTING=1 TUNE_TESTING=1 bash --login -i ./ci/env/install-dependencies.sh diff --git a/ci/docker/test.Dockerfile b/ci/docker/test.Dockerfile new file mode 100644 index 000000000000..a0645b154c2b --- /dev/null +++ b/ci/docker/test.Dockerfile @@ -0,0 +1,18 @@ +ARG DOCKER_IMAGE_BASE_TEST +FROM $DOCKER_IMAGE_BASE_TEST + +# Move out of working dir /ray +# Delete stale data +WORKDIR / +RUN rm -rf /ray + +RUN mkdir /ray +WORKDIR /ray + +# Copy new ray files +COPY . . + +# Install Ray +RUN SKIP_BAZEL_BUILD=1 RAY_INSTALL_JAVA=0 bash --login -i -c -- "python3 -m pip install -e /ray/python/" + +RUN bash --login -i ./ci/env/install-dependencies.sh diff --git a/ci/env/install-dependencies.sh b/ci/env/install-dependencies.sh index 00d7f7879681..29a090a958ad 100755 --- a/ci/env/install-dependencies.sh +++ b/ci/env/install-dependencies.sh @@ -80,7 +80,7 @@ install_miniconda() { conda="$(command -v conda || true)" fi - if [ ! -x "${conda}" ]; then # If no conda is found, install it + if [ ! -x "${conda}" ] || [ "${MINIMAL_INSTALL-}" = 1 ]; then # If no conda is found, install it local miniconda_dir # Keep directories user-independent, to help with Bazel caching case "${OSTYPE}" in linux*) miniconda_dir="/opt/miniconda";; @@ -115,6 +115,9 @@ install_miniconda() { conda="${miniconda_dir}\Scripts\conda.exe" ;; *) + if [ "${MINIMAL_INSTALL-}" = 1 ]; then + rm -rf "${miniconda_dir}" + fi mkdir -p -- "${miniconda_dir}" # We're forced to pass -b for non-interactive mode. # Unfortunately it inhibits PATH modifications as a side effect. @@ -285,23 +288,7 @@ download_mnist() { unzip "${HOME}/data/mnist.zip" -d "${HOME}/data" } -install_dependencies() { - - install_bazel - install_base - install_toolchains - - install_upgrade_pip - if [ -n "${PYTHON-}" ] || [ "${LINT-}" = 1 ] || [ "${MINIMAL_INSTALL-}" = "1" ]; then - install_miniconda - # Upgrade the miniconda pip. - install_upgrade_pip - fi - - install_nvm - if [ -n "${PYTHON-}" ] || [ -n "${LINT-}" ] || [ "${MAC_WHEELS-}" = 1 ]; then - install_node - fi +install_pip_packages() { # Install modules needed in all jobs. alias pip="python -m pip" @@ -310,9 +297,9 @@ install_dependencies() { pip install --no-clean dm-tree==0.1.5 # --no-clean is due to: https://github.com/deepmind/tree/issues/5 fi - if [ -n "${PYTHON-}" ] && [ "${MINIMAL_INSTALL-}" != 1 ]; then + if { [ -n "${PYTHON-}" ] || [ "${DL-}" = "1" ]; } && [ "${MINIMAL_INSTALL-}" != 1 ]; then # Remove this entire section once Serve dependencies are fixed. - if [ "${DOC_TESTING-}" != 1 ] && [ "${TRAIN_TESTING-}" != 1 ] && [ "${TUNE_TESTING-}" != 1 ] && [ "${RLLIB_TESTING-}" != 1 ]; then + if { [ -z "${BUILDKITE-}" ] || [ "${DL-}" = "1" ]; } && [ "${DOC_TESTING-}" != 1 ] && [ "${TRAIN_TESTING-}" != 1 ] && [ "${TUNE_TESTING-}" != 1 ] && [ "${RLLIB_TESTING-}" != 1 ]; then # We want to install the CPU version only. pip install -r "${WORKSPACE_DIR}"/python/requirements/ml/requirements_dl.txt fi @@ -390,7 +377,7 @@ install_dependencies() { # dependencies with Modin. if [ "${INSTALL_LUDWIG-}" = 1 ]; then # TODO: eventually pin this to master. - pip install -U "ludwig[test]">=0.4 jsonschema>=4 + pip install -U "ludwig[test]>=0.4" "jsonschema>=4" fi # Data processing test dependencies. @@ -443,6 +430,32 @@ install_dependencies() { CC=gcc pip install psutil setproctitle==1.2.2 colorama --target="${WORKSPACE_DIR}/python/ray/thirdparty_files" } +install_dependencies() { + install_bazel + + # Only install on buildkite if requested + if [ -z "${BUILDKITE-}" ] || [ "${BUILD-}" = "1" ]; then + install_base + install_toolchains + fi + + if [ -n "${PYTHON-}" ] || [ "${LINT-}" = 1 ] || [ "${MINIMAL_INSTALL-}" = "1" ]; then + install_miniconda + fi + + install_upgrade_pip + + # Only install on buildkite if requested + if [ -z "${BUILDKITE-}" ] || [ "${BUILD-}" = "1" ]; then + install_nvm + if [ -n "${PYTHON-}" ] || [ -n "${LINT-}" ] || [ "${MAC_WHEELS-}" = 1 ]; then + install_node + fi + fi + + install_pip_packages +} + install_dependencies "$@" # Pop caller's shell options (quietly) diff --git a/ci/env/install-minimal.sh b/ci/env/install-minimal.sh index 40db33a5698a..9c3293ca9d73 100755 --- a/ci/env/install-minimal.sh +++ b/ci/env/install-minimal.sh @@ -5,15 +5,15 @@ if [ -z "$1" ]; then PYTHON_VERSION=${PYTHON-3.7} else if [ "$1" = "3.6" ]; then - PYTHON_VERSION=${PYTHON-3.6} + PYTHON_VERSION="3.6" elif [ "$1" = "3.7" ]; then - PYTHON_VERSION=${PYTHON-3.7} + PYTHON_VERSION="3.7" elif [ "$1" = "3.8" ]; then - PYTHON_VERSION=${PYTHON-3.8} + PYTHON_VERSION="3.8" elif [ "$1" = "3.9" ]; then - PYTHON_VERSION=${PYTHON-3.9} + PYTHON_VERSION="3.9" elif [ "$1" = "3.10" ]; then - PYTHON_VERSION=${PYTHON-3.10} + PYTHON_VERSION="3.10" else echo "Unsupported Python version." exit 1 diff --git a/python/ray/tests/test_basic_3.py b/python/ray/tests/test_basic_3.py index d41e2a444781..14c4323b1c49 100644 --- a/python/ray/tests/test_basic_3.py +++ b/python/ray/tests/test_basic_3.py @@ -60,6 +60,10 @@ def collected(self): assert ray.get(test.collected.remote()) +@pytest.mark.skipif( + sys.version_info >= (3, 10, 0), + reason=("Currently not passing for Python 3.10"), +) def test_many_fractional_resources(shutdown_only): ray.init(num_cpus=2, num_gpus=2, resources={"Custom": 2}) diff --git a/python/ray/tests/test_runtime_env.py b/python/ray/tests/test_runtime_env.py index a559e7ba3d42..b9c3174443d0 100644 --- a/python/ray/tests/test_runtime_env.py +++ b/python/ray/tests/test_runtime_env.py @@ -413,6 +413,10 @@ def enable_dev_mode(local_env_var_enabled): @pytest.mark.skipif( sys.platform == "win32", reason="conda in runtime_env unsupported on Windows." ) +@pytest.mark.skipif( + sys.version_info >= (3, 10, 0), + reason=("Currently not passing for Python 3.10"), +) @pytest.mark.parametrize("local_env_var_enabled", [False, True]) @pytest.mark.parametrize("runtime_env_class", [dict, RuntimeEnv]) def test_runtime_env_log_msg( diff --git a/python/ray/tests/test_runtime_env_2.py b/python/ray/tests/test_runtime_env_2.py index c6570128ee51..6a3ed598b00d 100644 --- a/python/ray/tests/test_runtime_env_2.py +++ b/python/ray/tests/test_runtime_env_2.py @@ -13,6 +13,10 @@ @pytest.mark.skipif( sys.platform == "win32", reason="conda in runtime_env unsupported on Windows." ) +@pytest.mark.skipif( + sys.version_info >= (3, 10, 0), + reason=("Currently not passing on Python 3.10"), +) @pytest.mark.parametrize("runtime_env_class", [dict, RuntimeEnv]) @pytest.mark.parametrize( "set_bad_runtime_env_cache_ttl_seconds", diff --git a/python/ray/tests/test_state_api.py b/python/ray/tests/test_state_api.py index 24aabe89de01..9e606f986470 100644 --- a/python/ray/tests/test_state_api.py +++ b/python/ray/tests/test_state_api.py @@ -694,7 +694,7 @@ async def test_api_manager_list_workers(state_api_manager): @pytest.mark.skipif( - sys.version_info <= (3, 7, 0), + sys.version_info < (3, 8, 0), reason=("Not passing in CI although it works locally. Will handle it later."), ) @pytest.mark.asyncio @@ -785,7 +785,7 @@ async def test_api_manager_list_tasks(state_api_manager): @pytest.mark.skipif( - sys.version_info <= (3, 7, 0), + sys.version_info < (3, 8, 0), reason=("Not passing in CI although it works locally. Will handle it later."), ) @pytest.mark.asyncio @@ -897,7 +897,7 @@ async def test_api_manager_list_objects(state_api_manager): @pytest.mark.skipif( - sys.version_info <= (3, 7, 0), + sys.version_info < (3, 8, 0), reason=("Not passing in CI although it works locally. Will handle it later."), ) @pytest.mark.asyncio diff --git a/python/ray/tests/test_state_api_log.py b/python/ray/tests/test_state_api_log.py index 7aa11d640d2d..21019710bc4a 100644 --- a/python/ray/tests/test_state_api_log.py +++ b/python/ray/tests/test_state_api_log.py @@ -28,7 +28,7 @@ from ray.experimental.state.exception import DataSourceUnavailable from ray.experimental.state.state_manager import StateDataSourceClient -if sys.version_info > (3, 7, 0): +if sys.version_info >= (3, 8, 0): from unittest.mock import AsyncMock else: from asyncmock import AsyncMock diff --git a/python/ray/tests/test_state_api_summary.py b/python/ray/tests/test_state_api_summary.py index 541eb17bffc5..dfa2548012c1 100644 --- a/python/ray/tests/test_state_api_summary.py +++ b/python/ray/tests/test_state_api_summary.py @@ -14,7 +14,7 @@ from ray._private.test_utils import wait_for_condition from ray._raylet import ActorID, TaskID, ObjectID -if sys.version_info > (3, 7, 0): +if sys.version_info >= (3, 8, 0): from unittest.mock import AsyncMock else: from asyncmock import AsyncMock