diff --git a/.bazelrc b/.bazelrc index 68964ff59af59..78355e88da7d5 100644 --- a/.bazelrc +++ b/.bazelrc @@ -150,6 +150,11 @@ test:asan-buildkite --test_env=LD_PRELOAD="/usr/lib/x86_64-linux-gnu/libasan.so. # CI configuration: aquery:ci --color=no aquery:ci --noshow_progress + +test:ci-base --test_output=errors +test:ci-base --test_verbose_timeout_warnings +test:ci-base --flaky_test_attempts=3 + build:ci --color=yes build:ci --curses=no build:ci --keep_going @@ -157,23 +162,20 @@ build:ci --progress_report_interval=100 build:ci --show_progress_rate_limit=15 build:ci --ui_actions_shown=1024 build:ci --show_timestamps -test:ci --flaky_test_attempts=3 # Disable test result caching because py_test under Bazel can import from outside of sandbox, but Bazel only looks at # declared dependencies to determine if a result should be cached. More details at: # https://github.com/bazelbuild/bazel/issues/7091, https://github.com/bazelbuild/rules_python/issues/382 +test:ci --config=ci-base test:ci --nocache_test_results test:ci --spawn_strategy=local -test:ci --test_output=errors test:ci --experimental_ui_max_stdouterr_bytes=-1 -test:ci --test_verbose_timeout_warnings + +test:ci-debug --config=ci-base test:ci-debug -c dbg test:ci-debug --copt="-g" -test:ci-debug --flaky_test_attempts=3 # Disable test result caching for the same reason above. test:ci-debug --nocache_test_results test:ci-debug --spawn_strategy=local -test:ci-debug --test_output=errors -test:ci-debug --test_verbose_timeout_warnings aquery:get-toolchain --include_commandline=false aquery:get-toolchain --noimplicit_deps diff --git a/.buildkite/cicd.rayci.yml b/.buildkite/cicd.rayci.yml index 34e5505ec66d5..394cd758ad447 100644 --- a/.buildkite/cicd.rayci.yml +++ b/.buildkite/cicd.rayci.yml @@ -3,9 +3,10 @@ steps: - label: ":coral: reef: ci+release tooling tests" key: reef-tests commands: - - bazel run //ci/ray_ci:test_in_docker -- //ci/ray_ci/... //release/... ci + - bazel run //ci/ray_ci:test_in_docker -- + //ci/ray_ci/... //release/... //ci/pipeline/... ci --only-tags=release_unit,ci_unit - --parallelism-per-worker 2 + --cache-test-results --parallelism-per-worker 2 --build-name oss-ci-base_test --build-type skip instance_type: small diff --git a/.buildkite/core.rayci.yml b/.buildkite/core.rayci.yml index 1278a38518100..8b8740738c68a 100644 --- a/.buildkite/core.rayci.yml +++ b/.buildkite/core.rayci.yml @@ -275,14 +275,14 @@ steps: instance_type: medium commands: - bazel run //ci/ray_ci:test_in_docker -- //:all //src/... core --build-type clang - --parallelism-per-worker 2 + --cache-test-results --parallelism-per-worker 2 - label: ":ray: core: cpp asan tests" tags: core_cpp instance_type: medium commands: - bazel run //ci/ray_ci:test_in_docker -- //:all //src/... core --build-type asan-clang - --parallelism-per-worker 2 + --cache-test-results --parallelism-per-worker 2 - label: ":ray: core: cpp ubsan tests" tags: core_cpp @@ -290,7 +290,7 @@ steps: commands: - bazel run //ci/ray_ci:test_in_docker -- //:all //src/... core --build-type ubsan --except-tags no_ubsan - --parallelism-per-worker 2 + --cache-test-results --parallelism-per-worker 2 - label: ":ray: core: cpp tsan tests" tags: core_cpp @@ -298,7 +298,7 @@ steps: commands: - bazel run //ci/ray_ci:test_in_docker -- //:all //src/... core --build-type tsan-clang --except-tags no_tsan - --parallelism-per-worker 2 + --cache-test-results --parallelism-per-worker 2 - label: ":ray: core: flaky tests" key: core_flaky_tests diff --git a/.buildkite/hooks/post-command b/.buildkite/hooks/post-command index fbe6bb48e1e08..fec2f93b1a662 100644 --- a/.buildkite/hooks/post-command +++ b/.buildkite/hooks/post-command @@ -16,6 +16,7 @@ if [ -d "/tmp/artifacts/test-summaries" ] && [ "$(ls -A /tmp/artifacts/test-summ # Only upload annotations if there are at least 2 files in the directory: # 1 header and 1 failed test. if [ "$(find /tmp/artifacts/test-summaries -maxdepth 1 -name '*.txt' | wc -l)" -ge 2 ]; then + echo "Test summaries for ${BUILDKITE_JOB_ID} ${BUILDKITE_LABEL}" | buildkite-agent annotate --job "${BUILDKITE_JOB_ID}" --style error --context "${BUILDKITE_JOB_ID}" cat /tmp/artifacts/test-summaries/*.txt | head -n 20 | buildkite-agent annotate --job "${BUILDKITE_JOB_ID}" --append --style error --context "${BUILDKITE_JOB_ID}" fi diff --git a/BUILD.bazel b/BUILD.bazel index c87c6e6068405..bde8294507001 100644 --- a/BUILD.bazel +++ b/BUILD.bazel @@ -118,18 +118,16 @@ ray_cc_library( "src/ray/rpc/rpc_chaos.cc", "src/ray/rpc/server_call.cc", ], - hdrs = glob([ - "src/ray/rpc/rpc_chaos.h", + hdrs = [ "src/ray/rpc/client_call.h", "src/ray/rpc/common.h", "src/ray/rpc/grpc_client.h", - "src/ray/rpc/retryable_grpc_client.h", "src/ray/rpc/grpc_server.h", "src/ray/rpc/metrics_agent_client.h", + "src/ray/rpc/retryable_grpc_client.h", + "src/ray/rpc/rpc_chaos.h", "src/ray/rpc/server_call.h", - "src/ray/rpc/grpc_util.h", - "src/ray/raylet_client/*.h", - ]), + ] + glob(["src/ray/raylet_client/*.h"]), deps = [ ":stats_metric", "//src/ray/common:asio", @@ -487,6 +485,7 @@ ray_cc_binary( deps = [ ":raylet_lib", "//src/ray/util", + "//src/ray/util:cmd_line_utils", "@com_github_gflags_gflags//:gflags", ], ) @@ -722,6 +721,8 @@ ray_cc_library( "//src/ray/protobuf:common_cc_proto", "//src/ray/protobuf:runtime_env_agent_cc_proto", "//src/ray/util", + "//src/ray/util:cmd_line_utils", + "//src/ray/util:container_util", "//src/ray/util:throttler", "@boost//:asio", "@boost//:beast", @@ -792,6 +793,7 @@ ray_cc_library( ":worker_rpc", "//src/ray/protobuf:worker_cc_proto", "//src/ray/util", + "//src/ray/util:container_util", "//src/ray/util:shared_lru", "@boost//:circular_buffer", "@boost//:fiber", @@ -1977,6 +1979,7 @@ ray_cc_library( ":pubsub_lib", ":ray_common", ":redis_store_client", + "//src/ray/util:container_util", "//src/ray/util:sequencer", "//src/ray/protobuf:usage_cc_proto", ], @@ -2232,6 +2235,7 @@ ray_cc_library( ], deps = [ ":redis_client", + "//src/ray/util:container_util", "@com_google_absl//absl/strings:str_format", "@com_google_absl//absl/synchronization", ], diff --git a/ci/pipeline/BUILD.bazel b/ci/pipeline/BUILD.bazel new file mode 100644 index 0000000000000..a15b9412ef52c --- /dev/null +++ b/ci/pipeline/BUILD.bazel @@ -0,0 +1,20 @@ +load("@rules_python//python:defs.bzl", "py_test") +load("@py_deps_buildkite//:requirements.bzl", ci_require = "requirement") + +py_test( + name = "test_conditional_testing", + size = "small", + srcs = ["test_conditional_testing.py"], + data = [ + ":determine_tests_to_run.py", + ], + exec_compatible_with = ["//:hermetic_python"], + tags = [ + "ci_unit", + "team:ci", + ], + deps = [ + ci_require("bazel-runfiles"), + ci_require("pytest"), + ], +) diff --git a/ci/pipeline/determine_tests_to_run.py b/ci/pipeline/determine_tests_to_run.py index 26d3b844168db..295cc6cc143c6 100644 --- a/ci/pipeline/determine_tests_to_run.py +++ b/ci/pipeline/determine_tests_to_run.py @@ -1,5 +1,4 @@ -# Script used for checking changes for incremental testing cases -from __future__ import absolute_import, division, print_function +#!/usr/bin/env python3 import argparse import os diff --git a/ci/pipeline/test_conditional_testing.py b/ci/pipeline/test_conditional_testing.py new file mode 100644 index 0000000000000..b5cf5c721458f --- /dev/null +++ b/ci/pipeline/test_conditional_testing.py @@ -0,0 +1,84 @@ +import os +import subprocess +import sys +import tempfile + +import runfiles +import pytest + +_REPO_NAME = "com_github_ray_project_ray" +_runfiles = runfiles.Create() + + +def test_conditional_testing_pull_request(): + script = _runfiles.Rlocation(_REPO_NAME + "/ci/pipeline/determine_tests_to_run.py") + + test_cases = [ + (["ci/pipeline/test_conditional_testing.py"], {"lint", "tools"}), + ( + ["python/ray/data/__init__.py"], + {"lint", "data", "linux_wheels", "macos_wheels", "ml", "train"}, + ), + ] + + with tempfile.TemporaryDirectory() as origin, tempfile.TemporaryDirectory() as workdir: + subprocess.check_call(["git", "init", "--bare"], cwd=origin) + subprocess.check_call(["git", "init"], cwd=workdir) + subprocess.check_call( + ["git", "config", "user.email", "rayci@ray.io"], cwd=workdir + ) + subprocess.check_call( + ["git", "config", "user.name", "Ray CI Test"], cwd=workdir + ) + subprocess.check_call(["git", "remote", "add", "origin", origin], cwd=workdir) + + with open(os.path.join(workdir, "README.md"), "w") as f: + f.write("# README\n") + subprocess.check_call(["git", "add", "README.md"], cwd=workdir) + subprocess.check_call(["git", "commit", "-m", "init with readme"], cwd=workdir) + subprocess.check_call(["git", "push", "origin", "master"], cwd=workdir) + + for test_case in test_cases: + subprocess.check_call( + ["git", "checkout", "-B", "pr01", "master"], cwd=workdir + ) + + add_files = test_case[0] + for f in add_files: + dirname = os.path.dirname(f) + if dirname: + os.makedirs(os.path.join(workdir, dirname), exist_ok=True) + with open(os.path.join(workdir, f), "w") as f: + f.write("...\n") + + subprocess.check_call(["git", "add", "."], cwd=workdir) + subprocess.check_call( + ["git", "commit", "-m", "add test files"], cwd=workdir + ) + commit = ( + subprocess.check_output( + ["git", "show", "HEAD", "-q", "--format=%H"], cwd=workdir + ) + .decode() + .strip() + ) + + envs = os.environ.copy() + envs["BUILDKITE"] = "true" + envs["BUILDKITE_PULL_REQUEST_BASE_BRANCH"] = "master" + envs["BUILDKITE_PULL_REQUEST"] = "true" + envs["BUILDKITE_COMMIT"] = commit + + output = ( + subprocess.check_output([sys.executable, script], env=envs, cwd=workdir) + .decode() + .strip() + ) + tags = output.split() + + want = test_case[1] + assert want == set(tags), f"want {want}, got {tags}" + + +if __name__ == "__main__": + sys.exit(pytest.main(["-vv", __file__])) diff --git a/ci/ray_ci/test_linux_tester_container.py b/ci/ray_ci/test_linux_tester_container.py index a0f0d7d8f5af0..2759f61417a32 100644 --- a/ci/ray_ci/test_linux_tester_container.py +++ b/ci/ray_ci/test_linux_tester_container.py @@ -90,7 +90,10 @@ def _mock_popen(input: List[str]) -> None: return_value=None, ): LinuxTesterContainer( - "team", network="host", build_type="debug", test_envs=["ENV_01", "ENV_02"] + "team", + network="host", + build_type="debug", + test_envs=["ENV_01", "ENV_02"], )._run_tests_in_docker(["t1", "t2"], [0, 1], "/tmp", ["v=k"], "flag") input_str = inputs[-1] assert "--env ENV_01 --env ENV_02 --env BUILDKITE" in input_str @@ -111,6 +114,12 @@ def _mock_popen(input: List[str]) -> None: assert "--gpus" not in input_str assert f"--runs_per_test {RUN_PER_FLAKY_TEST} " in input_str + LinuxTesterContainer("team")._run_tests_in_docker( + ["t1", "t2"], [], "/tmp", ["v=k"], cache_test_results=True + ) + input_str = inputs[-1] + assert "--cache_test_results=auto" in input_str.split() + def test_run_script_in_docker() -> None: def _mock_check_output(input: List[str]) -> bytes: @@ -181,6 +190,7 @@ def _mock_run_tests_in_docker( test_envs: List[str], test_arg: Optional[str] = None, run_flaky_tests: Optional[bool] = False, + cache_test_results: Optional[bool] = False, ) -> MockPopen: return MockPopen(test_targets) diff --git a/ci/ray_ci/tester.py b/ci/ray_ci/tester.py index 732e4836671ce..ea1fa9da7c056 100644 --- a/ci/ray_ci/tester.py +++ b/ci/ray_ci/tester.py @@ -74,6 +74,13 @@ type=str, help=("Only include tests with the given tags."), ) +@click.option( + "--cache-test-results", + is_flag=True, + show_default=True, + default=False, + help=("If cache and use test results in bazel cache."), +) @click.option( "--run-flaky-tests", is_flag=True, @@ -185,6 +192,7 @@ def main( operating_system: str, except_tags: str, only_tags: str, + cache_test_results: bool, run_flaky_tests: bool, run_high_impact_tests: bool, skip_ray_installation: bool, @@ -249,6 +257,7 @@ def main( test_arg, is_bisect_run=bisect_run_test_target is not None, run_flaky_tests=run_flaky_tests, + cache_test_results=cache_test_results, ) sys.exit(0 if success else 42) diff --git a/ci/ray_ci/tester_container.py b/ci/ray_ci/tester_container.py index 204760bb11fb0..a193ccd15ebfe 100644 --- a/ci/ray_ci/tester_container.py +++ b/ci/ray_ci/tester_container.py @@ -74,6 +74,7 @@ def run_tests( test_arg: Optional[str] = None, is_bisect_run: bool = False, run_flaky_tests: bool = False, + cache_test_results: bool = False, ) -> bool: """ Run tests parallelly in docker. Return whether all tests pass. @@ -101,8 +102,9 @@ def run_tests( gpu_ids[i], bazel_log_dir_host, self.test_envs, - test_arg, - run_flaky_tests, + test_arg=test_arg, + run_flaky_tests=run_flaky_tests, + cache_test_results=cache_test_results, ) for i in range(len(chunks)) ] @@ -218,6 +220,7 @@ def _run_tests_in_docker( test_envs: List[str], test_arg: Optional[str] = None, run_flaky_tests: bool = False, + cache_test_results: bool = False, ) -> subprocess.Popen: logger.info("Running tests: %s", test_targets) commands = [ @@ -252,6 +255,8 @@ def _run_tests_in_docker( test_cmd += f"--test_env {env} " if test_arg: test_cmd += f"--test_arg {test_arg} " + if cache_test_results: + test_cmd += "--cache_test_results=auto " if run_flaky_tests: test_cmd += f"--runs_per_test {RUN_PER_FLAKY_TEST} " test_cmd += f"{' '.join(test_targets)}" diff --git a/ci/run/bazel_export_options b/ci/run/bazel_export_options index bbd7f3e2ab678..6c459f796ca18 100755 --- a/ci/run/bazel_export_options +++ b/ci/run/bazel_export_options @@ -10,7 +10,6 @@ event_json_flag=--build_event_json_file=$(mktemp /tmp/bazel_event_logs/bazel_log logs_archive_flag=--test_env=RAY_TEST_FAILURE_LOGS_ARCHIVE_DIR="${artifact_mount_path}"/failed_test_logs summary_directory_flag=--test_env=RAY_TEST_SUMMARY_DIR="${artifact_mount_path}"/test-summaries summary_directory_host_flag=--test_env=RAY_TEST_SUMMARY_DIR_HOST=/tmp/artifacts/test-summaries -buildkite_env_vars="--test_env=BUILDKITE_COMMIT --test_env=BUILDKITE_LABEL --test_env=BUILDKITE_JOB_ID" mujoco_env_var="--test_env=LD_LIBRARY_PATH=/root/.mujoco/mujoco210/bin" -echo "${event_json_flag} ${logs_archive_flag} ${summary_directory_flag} ${summary_directory_host_flag} ${buildkite_env_vars} ${mujoco_env_var}" +echo "${event_json_flag} ${logs_archive_flag} ${summary_directory_flag} ${summary_directory_host_flag} ${mujoco_env_var}" diff --git a/cpp/BUILD.bazel b/cpp/BUILD.bazel index 3fe9ae458ff78..409d7b601339a 100644 --- a/cpp/BUILD.bazel +++ b/cpp/BUILD.bazel @@ -60,6 +60,7 @@ cc_library( "//:global_state_accessor_lib", "//:ray_common", "//src/ray/util", + "//src/ray/util:cmd_line_utils", "@boost//:callable_traits", "@boost//:dll", "@com_google_absl//absl/flags:flag", diff --git a/cpp/src/ray/util/process_helper.cc b/cpp/src/ray/util/process_helper.cc index 598f41ab8cbc7..381cc7acabb88 100644 --- a/cpp/src/ray/util/process_helper.cc +++ b/cpp/src/ray/util/process_helper.cc @@ -18,6 +18,7 @@ #include #include "ray/common/ray_config.h" +#include "ray/util/cmd_line_utils.h" #include "ray/util/process.h" #include "ray/util/util.h" #include "src/ray/protobuf/gcs.pb.h" diff --git a/doc/source/ray-core/handling-dependencies.rst b/doc/source/ray-core/handling-dependencies.rst index 19c563c9ab572..3a4f2915d1bf4 100644 --- a/doc/source/ray-core/handling-dependencies.rst +++ b/doc/source/ray-core/handling-dependencies.rst @@ -302,6 +302,77 @@ For details, head to the :ref:`API Reference `. ``conda`` environments must have the same Python version as the Ray cluster. Do not list ``ray`` in the ``conda`` dependencies, as it will be automatically installed. +.. _use-uv-for-package-management: + +Using ``uv`` for package management +""""""""""""""""""""""""""""""""""" + +The recommended approach for package management with `uv` in runtime environments is through `uv run`. + +This method offers several key advantages: +First, it keeps dependencies synchronized between your driver and Ray workers. +Additionally, it provides full support for `pyproject.toml` including editable +packages. It also allows you to lock package versions using `uv lock`. +For more details, see the `UV scripts documentation `_. + +Create a file `pyproject.toml` in your working directory like the following: + +.. code-block:: toml + + [project] + + name = "test" + + version = "0.1" + + dependencies = [ + "emoji", + "ray", + ] + + +And then a `test.py` like the following: + +.. testcode:: + :skipif: True + + import ray + + # Add `--isolated` to avoid uv problems with concurrent environment setup + # (see https://github.com/astral-sh/uv/issues/11219). + ray.init(runtime_env={"working_dir": ".", "py_executable": "uv run --isolated"}) + + @ray.remote + def message(entity): + import emoji + return emoji.emojize(entity + " rocks :thumbs_up:") + + entities = [ + "Ray", + "Ray Serve", + "Ray Data", + "Ray Train", + "Ray RLlib", + "Ray Tune", + ] + results = [message.remote(entity) for entity in entities] + for result in results: + print(ray.get(result)) + + +and run the driver script with `uv run test.py`. For reproducibility it's +recommended to freeze the package versions by running `uv lock`. + +This workflow also supports editable packages, for example, you can use +`uv add --editable ./path/to/package` where `./path/to/package` +must be inside your `working_dir` so it's available to all +workers. + +Instead of the `pyproject.toml` file, you can also use a `requirements.txt` +file and use `uv run --with-requirements requirements.txt` for your `py_executable` +or use the `--with` flag to specify individual requirements. + + Library Development """"""""""""""""""" @@ -380,6 +451,12 @@ The ``runtime_env`` is a Python dictionary or a Python class :class:`ray.runtime Note: For option (1), if the local directory contains a ``.gitignore`` file, the files and paths specified there are not uploaded to the cluster. You can disable this by setting the environment variable `RAY_RUNTIME_ENV_IGNORE_GITIGNORE=1` on the machine doing the uploading. +- ``py_executable`` (str): Specifies the executable used for running the Ray workers. It can include arguments as well. The executable can be + located in the `working_dir`. This runtime environment is useful to run workers in a custom debugger or profiler as well as to run workers + in an environment set up by a package manager like `UV` (see :ref:`here `). + + Note: ``py_executable`` is new functionality and currently experimental. If you have some requirements or run into any problems, raise issues in `github `__. + - ``excludes`` (List[str]): When used with ``working_dir`` or ``py_modules``, specifies a list of files or paths to exclude from being uploaded to the cluster. This field uses the pattern-matching syntax used by ``.gitignore`` files: see ``_ for details. Note: In accordance with ``.gitignore`` syntax, if there is a separator (``/``) at the beginning or middle (or both) of the pattern, then the pattern is interpreted relative to the level of the ``working_dir``. diff --git a/doc/source/rllib/checkpoints.rst b/doc/source/rllib/checkpoints.rst index 9f52d2f20f31d..e4448cf641c63 100644 --- a/doc/source/rllib/checkpoints.rst +++ b/doc/source/rllib/checkpoints.rst @@ -400,7 +400,7 @@ for example a particular :py:class:`~ray.rllib.core.rl_module.rl_module.RLModule .. testcode:: - from ray import train, tune + from ray import tune # Reuse the preceding PPOConfig (`config`). # Inject custom callback code that runs right after algorithm's initialization. @@ -414,7 +414,7 @@ for example a particular :py:class:`~ray.rllib.core.rl_module.rl_module.RLModule results = tune.Tuner( config.algo_class, param_space=config, - run_config=train.RunConfig(stop={"num_env_steps_sampled_lifetime": 8000}) + run_config=tune.RunConfig(stop={"num_env_steps_sampled_lifetime": 8000}) ).fit() .. tab-item:: Swap out one RLModule and continue multi-agent training @@ -454,7 +454,7 @@ for example a particular :py:class:`~ray.rllib.core.rl_module.rl_module.RLModule results = tune.Tuner( multi_agent_config.algo_class, param_space=multi_agent_config, - run_config=train.RunConfig(stop={"num_env_steps_sampled_lifetime": 8000}) + run_config=tune.RunConfig(stop={"num_env_steps_sampled_lifetime": 8000}) ).fit() .. testcode:: diff --git a/doc/source/rllib/doc_code/replay_buffer_demo.py b/doc/source/rllib/doc_code/replay_buffer_demo.py index 3446bf23982ba..0fc3dfe3637de 100644 --- a/doc/source/rllib/doc_code/replay_buffer_demo.py +++ b/doc/source/rllib/doc_code/replay_buffer_demo.py @@ -4,7 +4,7 @@ import random import numpy as np -from ray import air, tune +from ray import tune from ray.rllib.utils.replay_buffers import ReplayBuffer, StorageUnit from ray.rllib.utils.annotations import override from ray.rllib.utils.typing import SampleBatchType @@ -103,7 +103,7 @@ def sample( tune.Tuner( "DQN", param_space=config, - run_config=air.RunConfig( + run_config=tune.RunConfig( stop={"training_iteration": 1}, ), ).fit() @@ -168,7 +168,7 @@ def sample( tune.Tuner( "DQN", param_space=config.to_dict(), - run_config=air.RunConfig( + run_config=tune.RunConfig( stop={"env_runners/episode_return_mean": 40, "training_iteration": 7}, ), ).fit() diff --git a/doc/source/rllib/key-concepts.rst b/doc/source/rllib/key-concepts.rst index 4759e07aef996..7617e99f71532 100644 --- a/doc/source/rllib/key-concepts.rst +++ b/doc/source/rllib/key-concepts.rst @@ -100,7 +100,7 @@ The following examples demonstrate this on RLlib's :py:class:`~ray.rllib.algorit .. testcode:: - from ray import train, tune + from ray import tune from ray.rllib.algorithms.ppo import PPOConfig # Configure. @@ -118,7 +118,7 @@ The following examples demonstrate this on RLlib's :py:class:`~ray.rllib.algorit "PPO", param_space=config, # Train for 4000 timesteps (2 iterations). - run_config=train.RunConfig(stop={"num_env_steps_sampled_lifetime": 4000}), + run_config=tune.RunConfig(stop={"num_env_steps_sampled_lifetime": 4000}), ).fit() .. _rllib-key-concepts-environments: diff --git a/doc/source/rllib/rllib-offline.rst b/doc/source/rllib/rllib-offline.rst index 6630bdf497452..8b4981b4877ea 100644 --- a/doc/source/rllib/rllib-offline.rst +++ b/doc/source/rllib/rllib-offline.rst @@ -67,7 +67,7 @@ this agent and then use its policy to record expert data to local disk. EVALUATION_RESULTS, EPISODE_RETURN_MEAN, ) - from ray import train, tune + from ray import tune # Configure the PPO algorithm. config = ( @@ -99,13 +99,13 @@ this agent and then use its policy to record expert data to local disk. tuner = tune.Tuner( "PPO", param_space=config, - run_config=train.RunConfig( + run_config=tune.RunConfig( stop={ metric: 450.0, }, name="docs_rllib_offline_pretrain_ppo", verbose=2, - checkpoint_config=train.CheckpointConfig( + checkpoint_config=tune.CheckpointConfig( checkpoint_frequency=1, checkpoint_at_end=True, ), @@ -246,7 +246,7 @@ data needs to be linked in the configuration of the algorithm (through the ``inp .. code-block:: python - from ray import train, tune + from ray import tune from ray.rllib.algorithms.bc import BCConfig # Setup the config for behavior cloning. @@ -314,11 +314,11 @@ data needs to be linked in the configuration of the algorithm (through the ``inp tuner = tune.Tuner( "BC", param_space=config, - run_config=train.RunConfig( + run_config=tune.RunConfig( name="docs_rllib_offline_bc", # Stop behavior cloning when we reach 450 in return. stop={metric: 450.0}, - checkpoint_config=train.CheckpointConfig( + checkpoint_config=tune.CheckpointConfig( # Only checkpoint at the end to be faster. checkpoint_frequency=0, checkpoint_at_end=True, diff --git a/python/ray/_private/ray_constants.py b/python/ray/_private/ray_constants.py index 166fc42791b77..62046fc63d968 100644 --- a/python/ray/_private/ray_constants.py +++ b/python/ray/_private/ray_constants.py @@ -387,9 +387,9 @@ def env_set_by_user(key): # created. CALL_STACK_LINE_DELIMITER = " | " -# The default gRPC max message size is 4 MiB, we use a larger number of 250 MiB +# The default gRPC max message size is 4 MiB, we use a larger number of 512 MiB # NOTE: This is equal to the C++ limit of (RAY_CONFIG::max_grpc_message_size) -GRPC_CPP_MAX_MESSAGE_SIZE = 250 * 1024 * 1024 +GRPC_CPP_MAX_MESSAGE_SIZE = 512 * 1024 * 1024 # The gRPC send & receive max length for "dashboard agent" server. # NOTE: This is equal to the C++ limit of RayConfig::max_grpc_message_size diff --git a/python/ray/_private/runtime_env/agent/runtime_env_agent.py b/python/ray/_private/runtime_env/agent/runtime_env_agent.py index c08dfbc203784..c02163d577a90 100644 --- a/python/ray/_private/runtime_env/agent/runtime_env_agent.py +++ b/python/ray/_private/runtime_env/agent/runtime_env_agent.py @@ -29,6 +29,7 @@ from ray._private.runtime_env.py_modules import PyModulesPlugin from ray._private.runtime_env.working_dir import WorkingDirPlugin from ray._private.runtime_env.nsight import NsightPlugin +from ray._private.runtime_env.py_executable import PyExecutablePlugin from ray._private.runtime_env.mpi import MPIPlugin from ray.core.generated import ( runtime_env_agent_pb2, @@ -216,6 +217,7 @@ def __init__( self._py_modules_plugin = PyModulesPlugin( self._runtime_env_dir, self._gcs_aio_client ) + self._py_executable_plugin = PyExecutablePlugin() self._java_jars_plugin = JavaJarsPlugin( self._runtime_env_dir, self._gcs_aio_client ) @@ -238,6 +240,7 @@ def __init__( self._pip_plugin, self._conda_plugin, self._py_modules_plugin, + self._py_executable_plugin, self._java_jars_plugin, self._container_plugin, self._nsight_plugin, diff --git a/python/ray/_private/runtime_env/packaging.py b/python/ray/_private/runtime_env/packaging.py index 87412e06abf13..692f309034e9b 100644 --- a/python/ray/_private/runtime_env/packaging.py +++ b/python/ray/_private/runtime_env/packaging.py @@ -17,6 +17,7 @@ RAY_RUNTIME_ENV_URI_PIN_EXPIRATION_S_DEFAULT, RAY_RUNTIME_ENV_URI_PIN_EXPIRATION_S_ENV_VAR, RAY_RUNTIME_ENV_IGNORE_GITIGNORE, + GRPC_CPP_MAX_MESSAGE_SIZE, ) from ray._private.runtime_env.conda_utils import exec_cmd_stream_to_logger from ray._private.runtime_env.protocol import Protocol @@ -34,7 +35,7 @@ # The size is bounded by the max gRPC message size. # Keep in sync with max_grpc_message_size in ray_config_def.h. GCS_STORAGE_MAX_SIZE = int( - os.environ.get("RAY_max_grpc_message_size", 500 * 1024 * 1024) + os.environ.get("RAY_max_grpc_message_size", GRPC_CPP_MAX_MESSAGE_SIZE) ) RAY_PKG_PREFIX = "_ray_pkg_" diff --git a/python/ray/_private/runtime_env/py_executable.py b/python/ray/_private/runtime_env/py_executable.py new file mode 100644 index 0000000000000..d764565be0dcf --- /dev/null +++ b/python/ray/_private/runtime_env/py_executable.py @@ -0,0 +1,46 @@ +import logging + +from typing import List, Optional + +from ray._private.runtime_env.context import RuntimeEnvContext +from ray._private.runtime_env.plugin import RuntimeEnvPlugin + +default_logger = logging.getLogger(__name__) + + +class PyExecutablePlugin(RuntimeEnvPlugin): + """This plugin allows running Ray workers with a custom Python executable. + + You can use it with + `ray.init(runtime_env={"py_executable": " "})`. If you specify + a `working_dir` in the runtime environment, the executable will have access + to the working directory, for example, to a requirements.txt for a package manager, + a script for a debugger, or the executable could be a shell script in the + working directory. You can also use this plugin to run worker processes + in a custom profiler or use a custom Python interpreter or `python` with + custom arguments. + """ + + name = "py_executable" + + def __init__(self): + pass + + async def create( + self, + uri: Optional[str], + runtime_env: "RuntimeEnv", # noqa: F821 + context: RuntimeEnvContext, + logger: logging.Logger = default_logger, + ) -> int: + return 0 + + def modify_context( + self, + uris: List[str], + runtime_env: "RuntimeEnv", # noqa: F821 + context: RuntimeEnvContext, + logger: Optional[logging.Logger] = default_logger, + ): + logger.info("Running py_executable plugin") + context.py_executable = runtime_env.py_executable() diff --git a/python/ray/_private/state.py b/python/ray/_private/state.py index ccdc709a4f1ba..751bdf748987a 100644 --- a/python/ray/_private/state.py +++ b/python/ray/_private/state.py @@ -1,7 +1,7 @@ import json import logging from collections import defaultdict -from typing import Dict +from typing import Dict, Optional from ray._private.protobuf_compat import message_to_dict @@ -82,7 +82,10 @@ def _really_init_global_state(self): self.global_state_accessor.connect() def actor_table( - self, actor_id: str, job_id: ray.JobID = None, actor_state_name: str = None + self, + actor_id: Optional[str], + job_id: Optional[ray.JobID] = None, + actor_state_name: Optional[str] = None, ): """Fetch and parse the actor table information for a single actor ID. @@ -932,7 +935,9 @@ def node_ids(): def actors( - actor_id: str = None, job_id: ray.JobID = None, actor_state_name: str = None + actor_id: Optional[str] = None, + job_id: Optional[ray.JobID] = None, + actor_state_name: Optional[str] = None, ): """Fetch actor info for one or more actor IDs (for debugging only). diff --git a/python/ray/data/BUILD b/python/ray/data/BUILD index 294402b349bba..1aec6b04237b8 100644 --- a/python/ray/data/BUILD +++ b/python/ray/data/BUILD @@ -503,7 +503,7 @@ py_test( py_test( name = "test_sort", - size = "large", + size = "enormous", srcs = ["tests/test_sort.py"], tags = ["team:data", "exclusive"], deps = ["//:ray_lib", ":conftest"], diff --git a/python/ray/data/_internal/arrow_block.py b/python/ray/data/_internal/arrow_block.py index 220efa65e6a67..d01449e1bd556 100644 --- a/python/ray/data/_internal/arrow_block.py +++ b/python/ray/data/_internal/arrow_block.py @@ -422,20 +422,31 @@ def sum_of_squared_diffs_from_mean( ignore_nulls, ) - def sort_and_partition( - self, boundaries: List[T], sort_key: "SortKey" - ) -> List["Block"]: + def sort(self, sort_key: "SortKey") -> Block: + assert ( + sort_key.get_columns() + ), f"Sorting columns couldn't be empty (got {sort_key.get_columns()})" + if self._table.num_rows == 0: # If the pyarrow table is empty we may not have schema # so calling sort_indices() will raise an error. - return [self._empty_table() for _ in range(len(boundaries) + 1)] + return self._empty_table() context = DataContext.get_current() sort = get_sort_transform(context) - table = sort(self._table, sort_key) - if len(boundaries) == 0: + return sort(self._table, sort_key) + + def sort_and_partition( + self, boundaries: List[T], sort_key: "SortKey" + ) -> List["Block"]: + table = self.sort(sort_key) + + if table.num_rows == 0: + return [self._empty_table() for _ in range(len(boundaries) + 1)] + elif len(boundaries) == 0: return [table] + return find_partitions(table, boundaries, sort_key) def combine(self, sort_key: "SortKey", aggs: Tuple["AggregateFn"]) -> Block: diff --git a/python/ray/data/_internal/batcher.py b/python/ray/data/_internal/batcher.py index d27ed089f03fb..328507da93a38 100644 --- a/python/ray/data/_internal/batcher.py +++ b/python/ray/data/_internal/batcher.py @@ -1,9 +1,13 @@ +import warnings from typing import Optional from ray.data._internal.arrow_block import ArrowBlockAccessor from ray.data._internal.arrow_ops import transform_pyarrow from ray.data._internal.delegating_block_builder import DelegatingBlockBuilder +from ray.data._internal.execution.util import memory_string +from ray.data._internal.util import get_total_obj_store_mem_on_node from ray.data.block import Block, BlockAccessor +from ray.util import log_once # pyarrow.Table.slice is slow when the table has many chunks # so we combine chunks into a single one to make slice faster @@ -208,12 +212,19 @@ def __init__( # Round it up internally to `batch_size` since our algorithm requires it. # This is harmless since it only offers extra randomization. shuffle_buffer_min_size = batch_size - self._buffer_min_size = shuffle_buffer_min_size + self._min_rows_to_yield_batch = shuffle_buffer_min_size + self._min_rows_to_trigger_compaction = int( + shuffle_buffer_min_size * SHUFFLE_BUFFER_COMPACTION_RATIO + ) self._builder = DelegatingBlockBuilder() self._shuffle_buffer: Block = None self._batch_head = 0 self._done_adding = False + self._total_object_store_nbytes = get_total_obj_store_mem_on_node() + self._total_num_rows_added = 0 + self._total_nbytes_added = 0 + def add(self, block: Block): """Add a block to the shuffle buffer. @@ -222,8 +233,49 @@ def add(self, block: Block): Args: block: Block to add to the shuffle buffer. """ - if BlockAccessor.for_block(block).num_rows() > 0: + # Because Arrow tables are memory mapped, blocks in the builder reside in object + # store memory and not local heap memory. So, if you specify a large buffer size + # and there isn't enough object store memory on the node, you encounter + # spilling. + if ( + self._estimated_min_nbytes_in_buffers is not None + and self._estimated_min_nbytes_in_buffers > self._total_object_store_nbytes + and log_once("shuffle_buffer_mem_warning") + ): + warnings.warn( + "The node you're iterating on has " + f"{memory_string(self._total_object_store_nbytes)} object " + "store memory, but the shuffle buffer is estimated to use " + f"{memory_string(self._estimated_min_nbytes_in_buffers)}. If you don't " + f"decrease the shuffle buffer size from " + f"{self._min_rows_to_yield_batch} rows, you might encounter spilling." + ) + + block_accessor = BlockAccessor.for_block(block) + if block_accessor.num_rows() > 0: self._builder.add_block(block) + self._total_num_rows_added += block_accessor.num_rows() + self._total_nbytes_added += block_accessor.size_bytes() + + @property + def _average_row_nbytes(self) -> Optional[int]: + """Return the average number of bytes per row added to this batcher.""" + return ( + self._total_nbytes_added // self._total_num_rows_added + if self._total_num_rows_added > 0 + else None + ) + + @property + def _estimated_min_nbytes_in_buffers(self) -> Optional[int]: + """Return the estimated minimum number of bytes across all buffers. + + This includes data in both the compacted and uncompacted buffers. + """ + if self._average_row_nbytes is None: + return None + + return self._average_row_nbytes * self._min_rows_to_trigger_compaction def done_adding(self) -> bool: """Indicate to the batcher that no more blocks will be added to the batcher. @@ -234,30 +286,31 @@ def done_adding(self) -> bool: def has_any(self) -> bool: """Whether this batcher has any data.""" - return self._buffer_size() > 0 + return self._num_rows() > 0 def has_batch(self) -> bool: """Whether this batcher has any batches.""" - buffer_size = self._buffer_size() + num_rows = self._num_rows() if not self._done_adding: # Delay pulling of batches until the buffer is large enough in order to # amortize compaction overhead. - return self._materialized_buffer_size() >= self._buffer_min_size or ( - buffer_size - self._batch_size - >= self._buffer_min_size * SHUFFLE_BUFFER_COMPACTION_RATIO + return ( + self._num_compacted_rows() >= self._min_rows_to_yield_batch + or num_rows - self._batch_size >= self._min_rows_to_trigger_compaction ) else: - return buffer_size >= self._batch_size + return num_rows >= self._batch_size - def _buffer_size(self) -> int: - """Return shuffle buffer size.""" - buffer_size = self._builder.num_rows() - buffer_size += self._materialized_buffer_size() - return buffer_size + def _num_rows(self) -> int: + """Return the total number of rows that haven't been yielded yet. - def _materialized_buffer_size(self) -> int: - """Return materialized (compacted portion of) shuffle buffer size.""" + This includes rows in both the compacted and uncompacted buffers. + """ + return self._num_compacted_rows() + self._num_uncompacted_rows() + + def _num_compacted_rows(self) -> int: + """Return number of unyielded rows in the compacted (shuffle) buffer.""" if self._shuffle_buffer is None: return 0 # The size of the concrete (materialized) shuffle buffer, adjusting @@ -268,6 +321,10 @@ def _materialized_buffer_size(self) -> int: BlockAccessor.for_block(self._shuffle_buffer).num_rows() - self._batch_head, ) + def _num_uncompacted_rows(self) -> int: + """Return number of unyielded rows in the uncompacted buffer.""" + return self._builder.num_rows() + def next_batch(self) -> Block: """Get the next shuffled batch from the shuffle buffer. @@ -278,9 +335,9 @@ def next_batch(self) -> Block: # Add rows in the builder to the shuffle buffer. Note that we delay compaction # as much as possible to amortize the concatenation overhead. Compaction is # only necessary when the materialized buffer size falls below the min size. - if self._builder.num_rows() > 0 and ( + if self._num_uncompacted_rows() > 0 and ( self._done_adding - or self._materialized_buffer_size() <= self._buffer_min_size + or self._num_compacted_rows() <= self._min_rows_to_yield_batch ): if self._shuffle_buffer is not None: if self._batch_head > 0: diff --git a/python/ray/data/_internal/datasource/parquet_datasource.py b/python/ray/data/_internal/datasource/parquet_datasource.py index be097b33d7904..288020bca3e9e 100644 --- a/python/ray/data/_internal/datasource/parquet_datasource.py +++ b/python/ray/data/_internal/datasource/parquet_datasource.py @@ -10,6 +10,7 @@ List, Literal, Optional, + Tuple, Union, ) @@ -161,20 +162,7 @@ class ParquetDatasource(Datasource): cost of some potential performance and/or compatibility penalties. """ - _FUTURE_FILE_EXTENSIONS = [ - "parquet", - # Snappy compression - "parquet.snappy", - "snappy.parquet", - # Gzip compression - "parquet.gz", - # Brotili compression - "parquet.br", - # Lz4 compression - "parquet.lz4", - # Zstd compression - "parquet.zst", - ] + _FUTURE_FILE_EXTENSIONS = ["parquet"] def __init__( self, @@ -195,8 +183,6 @@ def __init__( ): _check_pyarrow_version() - import pyarrow as pa - self._supports_distributed_reads = not _is_local_scheme(paths) if not self._supports_distributed_reads and ray.util.client.ray.is_connected(): raise ValueError( @@ -252,39 +238,30 @@ def __init__( # duplicating the partition data, we disable PyArrow's partitioning. dataset_kwargs["partitioning"] = None + pq_ds = get_parquet_dataset(paths, filesystem, dataset_kwargs) + # `read_schema` is the schema object that will be used to perform # read operations. # It should be None, unless user has specified the schema or columns. - # We don't use the inferred schema for read, because the pyarrow only infers - # schema based on the first file. Thus, files with different schemas will end - # up producing blocks with wrong schema. + # We don't use the inferred schema for read, because we infer the schema based + # on the first file. Thus, files with different schemas will end up producing + # blocks with wrong schema. # See https://github.com/ray-project/ray/issues/47960 for more context. read_schema = schema - pq_ds = get_parquet_dataset(paths, filesystem, dataset_kwargs) - - if schema is None: - schema = pq_ds.schema - schema = _add_partition_fields_to_schema(partitioning, schema, pq_ds) + inferred_schema = _infer_schema( + pq_ds, schema, columns, partitioning, _block_udf + ) - if columns: - schema = pa.schema( - [schema.field(column) for column in columns], schema.metadata + # Users can pass both data columns and partition columns in the 'columns' + # argument. To prevent PyArrow from complaining about missing columns, we + # separate the partition columns from the data columns. When we read the + # fragments, we pass the data columns to PyArrow and add the partition + # columns manually. + data_columns, partition_columns = None, None + if columns is not None: + data_columns, partition_columns = _infer_data_and_partition_columns( + columns, pq_ds.fragments[0], partitioning ) - read_schema = schema - - check_for_legacy_tensor_type(schema) - - if _block_udf is not None: - # Try to infer dataset schema by passing dummy table through UDF. - dummy_table = schema.empty_table() - try: - schema = _block_udf(dummy_table).schema.with_metadata(schema.metadata) - except Exception: - logger.debug( - "Failed to infer schema of dataset by passing dummy table " - "through UDF due to the following exception:", - exc_info=True, - ) try: prefetch_remote_args = {} @@ -320,9 +297,10 @@ def __init__( self._meta_provider = meta_provider self._block_udf = _block_udf self._to_batches_kwargs = to_batch_kwargs - self._columns = columns + self._data_columns = data_columns + self._partition_columns = partition_columns self._read_schema = read_schema - self._schema = schema + self._inferred_schema = inferred_schema self._file_metadata_shuffler = None self._include_paths = include_paths self._partitioning = partitioning @@ -334,7 +312,7 @@ def __init__( sample_infos = sample_fragments( self._pq_fragments, to_batches_kwargs=to_batch_kwargs, - columns=columns, + columns=data_columns, schema=self._read_schema, local_scheduling=self._local_scheduling, ) @@ -395,7 +373,7 @@ def get_read_tasks(self, parallelism: int) -> List[ReadTask]: meta = self._meta_provider( paths, - self._schema, + self._inferred_schema, num_fragments=len(fragments), prefetched_metadata=metadata, ) @@ -411,7 +389,8 @@ def get_read_tasks(self, parallelism: int) -> List[ReadTask]: block_udf, to_batches_kwargs, default_read_batch_size_rows, - columns, + data_columns, + partition_columns, read_schema, include_paths, partitioning, @@ -419,7 +398,8 @@ def get_read_tasks(self, parallelism: int) -> List[ReadTask]: self._block_udf, self._to_batches_kwargs, self._default_read_batch_size_rows, - self._columns, + self._data_columns, + self._partition_columns, self._read_schema, self._include_paths, self._partitioning, @@ -430,7 +410,8 @@ def get_read_tasks(self, parallelism: int) -> List[ReadTask]: block_udf, to_batches_kwargs, default_read_batch_size_rows, - columns, + data_columns, + partition_columns, read_schema, f, include_paths, @@ -458,7 +439,8 @@ def read_fragments( block_udf, to_batches_kwargs, default_read_batch_size_rows, - columns, + data_columns, + partition_columns, schema, serialized_fragments: List[SerializedFragment], include_paths: bool, @@ -487,17 +469,17 @@ def read_fragments( partitions = parse(fragment.path) # Filter out partitions that aren't in the user-specified columns list. - if columns is not None: + if partition_columns is not None: partitions = { field_name: value for field_name, value in partitions.items() - if field_name in columns + if field_name in partition_columns } def get_batch_iterable(): return fragment.to_batches( use_threads=use_threads, - columns=columns, + columns=data_columns, schema=schema, batch_size=batch_size, **to_batches_kwargs, @@ -720,9 +702,7 @@ def _add_partitions_to_table( for field_name, value in partitions.items(): column = pa.array([value] * len(table)) field_index = table.schema.get_field_index(field_name) - if field_index != -1: - table = table.set_column(field_index, field_name, column) - else: + if field_index == -1: table = table.append_column(field_name, column) return table @@ -768,3 +748,75 @@ def emit_file_extensions_future_warning(future_file_extensions: List[str]): "backwards compatibility, set `file_extensions=None` explicitly.", FutureWarning, ) + + +def _infer_schema( + parquet_dataset, schema, columns, partitioning, _block_udf +) -> "pyarrow.Schema": + """Infer the schema of read data using the user-specified parameters.""" + import pyarrow as pa + + inferred_schema = schema + + if schema is None: + inferred_schema = parquet_dataset.schema + inferred_schema = _add_partition_fields_to_schema( + partitioning, inferred_schema, parquet_dataset + ) + + if columns: + inferred_schema = pa.schema( + [inferred_schema.field(column) for column in columns], + inferred_schema.metadata, + ) + + if _block_udf is not None: + # Try to infer dataset schema by passing dummy table through UDF. + dummy_table = inferred_schema.empty_table() + try: + inferred_schema = _block_udf(dummy_table).schema.with_metadata( + inferred_schema.metadata + ) + except Exception: + logger.debug( + "Failed to infer schema of dataset by passing dummy table " + "through UDF due to the following exception:", + exc_info=True, + ) + + check_for_legacy_tensor_type(inferred_schema) + return inferred_schema + + +def _infer_data_and_partition_columns( + user_specified_columns: List[str], + fragment: "ParquetFileFragment", + partitioning: Optional[Partitioning], +) -> Tuple[List[str], List[str]]: + """Infer which columns are in the files and which columns are partition columns. + + This function uses the schema and path of the first file to infer what columns + represent. + + Args: + user_specified_columns: A list of column names that the user specified. + fragment: The first fragment in the dataset. + partitioning: The partitioning scheme used to partition the data. + + Returns: + A tuple of lists of column names. The first list contains the columns that are + in the file, and the second list contains the columns that are partition + columns. + """ + data_columns = [ + column + for column in user_specified_columns + if column in fragment.physical_schema.names + ] + if partitioning is not None: + parse = PathPartitionParser(partitioning) + partitions = parse(fragment.path) + partition_columns = [ + column for column in user_specified_columns if column in partitions + ] + return data_columns, partition_columns diff --git a/python/ray/data/_internal/logical/operators/all_to_all_operator.py b/python/ray/data/_internal/logical/operators/all_to_all_operator.py index 16a09f44d8209..81603935f594e 100644 --- a/python/ray/data/_internal/logical/operators/all_to_all_operator.py +++ b/python/ray/data/_internal/logical/operators/all_to_all_operator.py @@ -147,6 +147,7 @@ def __init__( input_op: LogicalOperator, key: Optional[str], aggs: List[AggregateFn], + num_partitions: Optional[int] = None, batch_format: Optional[str] = "default", ): super().__init__( @@ -160,4 +161,5 @@ def __init__( ) self._key = key self._aggs = aggs + self._num_partitions = num_partitions self._batch_format = batch_format diff --git a/python/ray/data/_internal/pandas_block.py b/python/ray/data/_internal/pandas_block.py index 54dd12f29f08b..258da88b4b467 100644 --- a/python/ray/data/_internal/pandas_block.py +++ b/python/ray/data/_internal/pandas_block.py @@ -500,17 +500,27 @@ def sum_of_squared_diffs_from_mean( on, ) + def sort(self, sort_key: "SortKey"): + assert ( + sort_key.get_columns() + ), f"Sorting columns couldn't be empty (got {sort_key.get_columns()})" + + if self._table.shape[0] == 0: + return self._empty_table() + + columns, ascending = sort_key.to_pandas_sort_args() + return self._table.sort_values(by=columns, ascending=ascending) + def sort_and_partition( self, boundaries: List[T], sort_key: "SortKey" ) -> List[Block]: - if self._table.shape[0] == 0: + table = self.sort(sort_key) + + if table.shape[0] == 0: # If the pyarrow table is empty we may not have schema # so calling sort_indices() will raise an error. return [self._empty_table() for _ in range(len(boundaries) + 1)] - - columns, ascending = sort_key.to_pandas_sort_args() - table = self._table.sort_values(by=columns, ascending=ascending) - if len(boundaries) == 0: + elif len(boundaries) == 0: return [table] return find_partitions(table, boundaries, sort_key) diff --git a/python/ray/data/_internal/planner/aggregate.py b/python/ray/data/_internal/planner/aggregate.py index 2d32719dc8492..2e88cc675a603 100644 --- a/python/ray/data/_internal/planner/aggregate.py +++ b/python/ray/data/_internal/planner/aggregate.py @@ -18,18 +18,24 @@ from ray.data._internal.stats import StatsDict from ray.data._internal.util import unify_block_metadata_schema from ray.data.aggregate import AggregateFn -from ray.data.context import DataContext +from ray.data.context import DataContext, ShuffleStrategy def generate_aggregate_fn( key: Optional[Union[str, List[str]]], aggs: List[AggregateFn], batch_format: str, + data_context: DataContext, _debug_limit_shuffle_execution_to_num_blocks: Optional[int] = None, ) -> AllToAllTransformFn: """Generate function to aggregate blocks by the specified key column or key function. """ + assert data_context.shuffle_strategy in [ + ShuffleStrategy.SORT_SHUFFLE_PULL_BASED, + ShuffleStrategy.SORT_SHUFFLE_PUSH_BASED, + ] + if len(aggs) == 0: raise ValueError("Aggregate requires at least one aggregation") @@ -72,10 +78,15 @@ def fn( aggs=aggs, batch_format=batch_format, ) - if DataContext.get_current().use_push_based_shuffle: + + if data_context.shuffle_strategy == ShuffleStrategy.SORT_SHUFFLE_PUSH_BASED: scheduler = PushBasedShuffleTaskScheduler(agg_spec) - else: + elif data_context.shuffle_strategy == ShuffleStrategy.SORT_SHUFFLE_PULL_BASED: scheduler = PullBasedShuffleTaskScheduler(agg_spec) + else: + raise ValueError( + f"Invalid shuffle strategy '{data_context.shuffle_strategy}'" + ) return scheduler.execute( refs, diff --git a/python/ray/data/_internal/planner/plan_all_to_all_op.py b/python/ray/data/_internal/planner/plan_all_to_all_op.py index 7677ab6a0fc00..a536c7a5b69ff 100644 --- a/python/ray/data/_internal/planner/plan_all_to_all_op.py +++ b/python/ray/data/_internal/planner/plan_all_to_all_op.py @@ -34,21 +34,25 @@ def plan_all_to_all_op( input_physical_dag = physical_children[0] target_max_block_size = None + if isinstance(op, RandomizeBlocks): fn = generate_randomize_blocks_fn(op) # Randomize block order does not actually compute anything, so we # want to inherit the upstream op's target max block size. + elif isinstance(op, RandomShuffle): debug_limit_shuffle_execution_to_num_blocks = data_context.get_config( "debug_limit_shuffle_execution_to_num_blocks", None ) fn = generate_random_shuffle_fn( + data_context, op._seed, op._num_outputs, op._ray_remote_args, debug_limit_shuffle_execution_to_num_blocks, ) target_max_block_size = data_context.target_shuffle_max_block_size + elif isinstance(op, Repartition): debug_limit_shuffle_execution_to_num_blocks = None if op._shuffle: @@ -59,16 +63,22 @@ def plan_all_to_all_op( fn = generate_repartition_fn( op._num_outputs, op._shuffle, + data_context, debug_limit_shuffle_execution_to_num_blocks, ) + elif isinstance(op, Sort): debug_limit_shuffle_execution_to_num_blocks = data_context.get_config( "debug_limit_shuffle_execution_to_num_blocks", None ) fn = generate_sort_fn( - op._sort_key, op._batch_format, debug_limit_shuffle_execution_to_num_blocks + op._sort_key, + op._batch_format, + data_context, + debug_limit_shuffle_execution_to_num_blocks, ) target_max_block_size = data_context.target_shuffle_max_block_size + elif isinstance(op, Aggregate): debug_limit_shuffle_execution_to_num_blocks = data_context.get_config( "debug_limit_shuffle_execution_to_num_blocks", None @@ -77,6 +87,7 @@ def plan_all_to_all_op( op._key, op._aggs, op._batch_format, + data_context, debug_limit_shuffle_execution_to_num_blocks, ) target_max_block_size = data_context.target_shuffle_max_block_size diff --git a/python/ray/data/_internal/planner/random_shuffle.py b/python/ray/data/_internal/planner/random_shuffle.py index 88e5b255cd0e9..ccfa96de0a72a 100644 --- a/python/ray/data/_internal/planner/random_shuffle.py +++ b/python/ray/data/_internal/planner/random_shuffle.py @@ -14,10 +14,11 @@ ) from ray.data._internal.planner.exchange.shuffle_task_spec import ShuffleTaskSpec from ray.data._internal.stats import StatsDict -from ray.data.context import DataContext +from ray.data.context import DataContext, ShuffleStrategy def generate_random_shuffle_fn( + data_context: DataContext, seed: Optional[int], num_outputs: Optional[int] = None, ray_remote_args: Optional[Dict[str, Any]] = None, @@ -61,7 +62,7 @@ def upstream_map_fn(blocks): upstream_map_fn=upstream_map_fn, ) - if DataContext.get_current().use_push_based_shuffle: + if data_context.shuffle_strategy == ShuffleStrategy.SORT_SHUFFLE_PUSH_BASED: if num_outputs is not None: raise NotImplementedError( "Push-based shuffle doesn't support setting num_blocks yet." diff --git a/python/ray/data/_internal/planner/repartition.py b/python/ray/data/_internal/planner/repartition.py index 73059f7035350..951d9cbb586e7 100644 --- a/python/ray/data/_internal/planner/repartition.py +++ b/python/ray/data/_internal/planner/repartition.py @@ -17,12 +17,13 @@ SplitRepartitionTaskScheduler, ) from ray.data._internal.stats import StatsDict -from ray.data.context import DataContext +from ray.data.context import DataContext, ShuffleStrategy def generate_repartition_fn( num_outputs: int, shuffle: bool, + data_context: DataContext, _debug_limit_shuffle_execution_to_num_blocks: Optional[int] = None, ) -> AllToAllTransformFn: """Generate function to partition each records of blocks.""" @@ -55,7 +56,7 @@ def upstream_map_fn(blocks): upstream_map_fn=upstream_map_fn, ) - if DataContext.get_current().use_push_based_shuffle: + if data_context.shuffle_strategy == ShuffleStrategy.SORT_SHUFFLE_PUSH_BASED: scheduler = PushBasedShuffleTaskScheduler(shuffle_spec) else: scheduler = PullBasedShuffleTaskScheduler(shuffle_spec) diff --git a/python/ray/data/_internal/planner/sort.py b/python/ray/data/_internal/planner/sort.py index ffb936d74bea4..15ec32f6c0c93 100644 --- a/python/ray/data/_internal/planner/sort.py +++ b/python/ray/data/_internal/planner/sort.py @@ -15,12 +15,13 @@ from ray.data._internal.planner.exchange.sort_task_spec import SortKey, SortTaskSpec from ray.data._internal.stats import StatsDict from ray.data._internal.util import unify_block_metadata_schema -from ray.data.context import DataContext +from ray.data.context import DataContext, ShuffleStrategy def generate_sort_fn( sort_key: SortKey, batch_format: str, + data_context: DataContext, _debug_limit_shuffle_execution_to_num_blocks: Optional[int] = None, ) -> AllToAllTransformFn: """Generate function to sort blocks by the specified key column or key function.""" @@ -63,7 +64,7 @@ def fn( boundaries=boundaries, sort_key=sort_key, batch_format=batch_format ) - if DataContext.get_current().use_push_based_shuffle: + if data_context.shuffle_strategy == ShuffleStrategy.SORT_SHUFFLE_PUSH_BASED: scheduler = PushBasedShuffleTaskScheduler(sort_spec) else: scheduler = PullBasedShuffleTaskScheduler(sort_spec) diff --git a/python/ray/data/_internal/table_block.py b/python/ray/data/_internal/table_block.py index 2164fcb99ea75..f69ab4a790e9e 100644 --- a/python/ray/data/_internal/table_block.py +++ b/python/ray/data/_internal/table_block.py @@ -19,7 +19,7 @@ from ray.data._internal.row import TableRow from ray.data._internal.size_estimator import SizeEstimator from ray.data._internal.util import MiB -from ray.data.block import Block, BlockAccessor +from ray.data.block import Block, BlockAccessor, BlockType if TYPE_CHECKING: from ray.data._internal.planner.exchange.sort_task_spec import SortKey @@ -266,7 +266,7 @@ def sample(self, n_samples: int, sort_key: "SortKey") -> Any: def normalize_block_types( cls, blocks: List[Block], - normalize_type: Optional[str] = None, + normalize_type: Optional[BlockType] = None, ) -> List[Block]: """Normalize input blocks to the specified `normalize_type`. If the blocks are already all of the same type, returns the original blocks. @@ -293,12 +293,9 @@ def normalize_block_types( if len(seen_types) <= 1: return blocks - if normalize_type == "arrow": - results = [BlockAccessor.for_block(block).to_arrow() for block in blocks] - elif normalize_type == "pandas": - results = [BlockAccessor.for_block(block).to_pandas() for block in blocks] - else: - results = [BlockAccessor.for_block(block).to_default() for block in blocks] + results = [ + cls.try_convert_block_type(block, normalize_type) for block in blocks + ] if any(not isinstance(block, type(results[0])) for block in results): raise ValueError( @@ -308,3 +305,12 @@ def normalize_block_types( "with block normalization." ) return results + + @classmethod + def try_convert_block_type(cls, block: Block, block_type: BlockType): + if block_type == BlockType.ARROW: + return BlockAccessor.for_block(block).to_arrow() + elif block_type == BlockType.PANDAS: + return BlockAccessor.for_block(block).to_pandas() + else: + return BlockAccessor.for_block(block).to_default() diff --git a/python/ray/data/_internal/util.py b/python/ray/data/_internal/util.py index 537940c5a9b21..af45eb8c87c52 100644 --- a/python/ray/data/_internal/util.py +++ b/python/ray/data/_internal/util.py @@ -1088,10 +1088,6 @@ def _run_transforming_worker(worker_id: int): non_empty_queues.append(output_queue) yield item - assert ( - non_empty_queues + empty_queues == remaining_output_queues - ), "Exhausted non-trailing queue!" - remaining_output_queues = non_empty_queues finally: @@ -1504,3 +1500,16 @@ def keys_equal(keys1, keys2): if not ((is_nan(k1) and is_nan(k2)) or k1 == k2): return False return True + + +def get_total_obj_store_mem_on_node() -> int: + """Return the total object store memory on the current node. + + This function incurs an RPC. Use it cautiously. + """ + node_id = ray.get_runtime_context().get_node_id() + total_resources_per_node = ray._private.state.total_resources_per_node() + assert ( + node_id in total_resources_per_node + ), f"Expected node '{node_id}' to be in resources: {total_resources_per_node}" + return total_resources_per_node[node_id]["object_store_memory"] diff --git a/python/ray/data/block.py b/python/ray/data/block.py index a0d09b12569e2..3ccc34f5d91fc 100644 --- a/python/ray/data/block.py +++ b/python/ray/data/block.py @@ -476,6 +476,10 @@ def sample(self, n_samples: int, sort_key: "SortKey") -> "Block": """Return a random sample of items from this block.""" raise NotImplementedError + def sort(self, sort_key: "SortKey") -> "Block": + """Returns new block sorted according to provided `sort_key`""" + raise NotImplementedError + def sort_and_partition( self, boundaries: List[T], sort_key: "SortKey" ) -> List["Block"]: @@ -495,7 +499,10 @@ def merge_sorted_blocks( @staticmethod def aggregate_combined_blocks( - blocks: List[Block], sort_key: "SortKey", aggs: Tuple["AggregateFn"] + blocks: List[Block], + sort_key: "SortKey", + aggs: Tuple["AggregateFn"], + finalize: bool = True, ) -> Tuple[Block, BlockMetadata]: """Aggregate partially combined and sorted blocks.""" raise NotImplementedError diff --git a/python/ray/data/context.py b/python/ray/data/context.py index 3e9ad30584a38..d32e14412335e 100644 --- a/python/ray/data/context.py +++ b/python/ray/data/context.py @@ -1,3 +1,4 @@ +import enum import logging import os import threading @@ -22,6 +23,15 @@ _context_lock = threading.Lock() +@DeveloperAPI(stability="alpha") +class ShuffleStrategy(str, enum.Enum): + """Shuffle strategy determines shuffling algorithm employed by operations + like aggregate, repartition, etc""" + + SORT_SHUFFLE_PULL_BASED = "sort_shuffle_pull_based" + SORT_SHUFFLE_PUSH_BASED = "sort_shuffle_push_based" + + # We chose 128MiB for default: With streaming execution and num_cpus many concurrent # tasks, the memory footprint will be about 2 * num_cpus * target_max_block_size ~= RAM # * DEFAULT_OBJECT_STORE_MEMORY_LIMIT_FRACTION * 0.3 (default object store memory @@ -56,6 +66,10 @@ os.environ.get("RAY_DATA_PUSH_BASED_SHUFFLE", None) ) +DEFAULT_SHUFFLE_STRATEGY = os.environ.get( + "RAY_DATA_DEFAULT_SHUFFLE_STRATEGY", ShuffleStrategy.SORT_SHUFFLE_PULL_BASED +) + DEFAULT_SCHEDULING_STRATEGY = "SPREAD" # This default enables locality-based scheduling in Ray for tasks where arg data @@ -70,7 +84,7 @@ DEFAULT_DECODING_SIZE_ESTIMATION_ENABLED = True -DEFAULT_MIN_PARALLELISM = 200 +DEFAULT_MIN_PARALLELISM = env_integer("RAY_DATA_DEFAULT_MIN_PARALLELISM", 200) DEFAULT_ENABLE_TENSOR_EXTENSION_CASTING = True @@ -173,6 +187,25 @@ def _execution_options_factory() -> "ExecutionOptions": return ExecutionOptions() +def _deduce_default_shuffle_algorithm() -> ShuffleStrategy: + if DEFAULT_USE_PUSH_BASED_SHUFFLE: + logger.warning( + "RAY_DATA_PUSH_BASED_SHUFFLE is deprecated, please use " + "RAY_DATA_DEFAULT_SHUFFLE_STRATEGY to set shuffling strategy" + ) + + return ShuffleStrategy.SORT_SHUFFLE_PUSH_BASED + else: + vs = [s for s in ShuffleStrategy] # noqa: C416 + + assert DEFAULT_SHUFFLE_STRATEGY in vs, ( + f"RAY_DATA_DEFAULT_SHUFFLE_STRATEGY has to be one of the [{','.join(vs)}] " + f"(got {DEFAULT_SHUFFLE_STRATEGY})" + ) + + return DEFAULT_SHUFFLE_STRATEGY + + @DeveloperAPI @dataclass class DataContext: @@ -285,8 +318,19 @@ class DataContext: streaming_read_buffer_size: int = DEFAULT_STREAMING_READ_BUFFER_SIZE enable_pandas_block: bool = DEFAULT_ENABLE_PANDAS_BLOCK actor_prefetcher_enabled: bool = DEFAULT_ACTOR_PREFETCHER_ENABLED + + ################################################################ + # Sort-based shuffling configuration + ################################################################ + use_push_based_shuffle: bool = DEFAULT_USE_PUSH_BASED_SHUFFLE + + _shuffle_strategy: ShuffleStrategy = _deduce_default_shuffle_algorithm() + pipeline_push_based_shuffle_reduce_tasks: bool = True + + ################################################################ + scheduling_strategy: SchedulingStrategyT = DEFAULT_SCHEDULING_STRATEGY scheduling_strategy_large_args: SchedulingStrategyT = ( DEFAULT_SCHEDULING_STRATEGY_LARGE_ARGS @@ -386,6 +430,12 @@ def __setattr__(self, name: str, value: Any) -> None: "`retried_io_errors` instead.", DeprecationWarning, ) + elif name == "use_push_based_shuffle": + warnings.warn( + "`use_push_based_shuffle` is deprecated, please configure " + "`shuffle_strategy` instead.", + DeprecationWarning, + ) super().__setattr__(name, value) @@ -436,6 +486,22 @@ def _set_current(context: "DataContext") -> None: global _default_context _default_context = context + @property + def shuffle_strategy(self) -> ShuffleStrategy: + if self.use_push_based_shuffle: + logger.warning( + "`use_push_based_shuffle` is deprecated, please configure " + "`shuffle_strategy` instead.", + ) + + return ShuffleStrategy.SORT_SHUFFLE_PUSH_BASED + + return self._shuffle_strategy + + @shuffle_strategy.setter + def shuffle_strategy(self, value: ShuffleStrategy) -> None: + self._shuffle_strategy = value + def get_config(self, key: str, default: Any = None) -> Any: """Get the value for a key-value style config. diff --git a/python/ray/data/dataset.py b/python/ray/data/dataset.py index 69dc520f7ca4d..d523d8e3516d7 100644 --- a/python/ray/data/dataset.py +++ b/python/ray/data/dataset.py @@ -2102,6 +2102,7 @@ def union(self, *other: List["Dataset"]) -> "Dataset": def groupby( self, key: Union[str, List[str], None], + num_partitions: Optional[int] = None, ) -> "GroupedData": """Group rows of a :class:`Dataset` according to a column. @@ -2130,7 +2131,11 @@ def normalize_variety(group: pd.DataFrame) -> pd.DataFrame: Args: key: A column name or list of column names. - If this is ``None``, place all rows in a single group. + If this is ``None``, place all rows in a single group. + + num_partitions: Number of partitions data will be partitioned into (only + relevant if hash-shuffling strategy is used). When not set defaults + to `DataContext.min_parallelism`. Returns: A lazy :class:`~ray.data.grouped_data.GroupedData`. @@ -2149,7 +2154,10 @@ def normalize_variety(group: pd.DataFrame) -> pd.DataFrame: # input validation. SortKey(key).validate_schema(self.schema(fetch_if_missing=False)) - return GroupedData(self, key) + if num_partitions is not None and num_partitions <= 0: + raise ValueError("`num_partitions` must be a positive integer") + + return GroupedData(self, key, num_partitions=num_partitions) @AllToAllAPI @ConsumptionAPI diff --git a/python/ray/data/grouped_data.py b/python/ray/data/grouped_data.py index 6912160b54ceb..66f36706ba10e 100644 --- a/python/ray/data/grouped_data.py +++ b/python/ray/data/grouped_data.py @@ -29,14 +29,17 @@ def __init__( self, dataset: Dataset, key: Optional[Union[str, List[str]]], + *, + num_partitions: Optional[int], ): """Construct a dataset grouped by key (internal API). The constructor is not part of the GroupedData API. Use the ``Dataset.groupby()`` method to construct one. """ - self._dataset = dataset - self._key = key + self._dataset: Dataset = dataset + self._key: Optional[Union[str, List[str]]] = key + self._num_partitions: Optional[int] = num_partitions def __repr__(self) -> str: return ( @@ -62,6 +65,7 @@ def aggregate(self, *aggs: AggregateFn) -> Dataset: self._dataset._logical_plan.dag, key=self._key, aggs=aggs, + num_partitions=self._num_partitions, ) logical_plan = LogicalPlan(op, self._dataset.context) return Dataset( diff --git a/python/ray/data/tests/block_batching/test_util.py b/python/ray/data/tests/block_batching/test_util.py index 852433df87e5c..0f67097d9508b 100644 --- a/python/ray/data/tests/block_batching/test_util.py +++ b/python/ray/data/tests/block_batching/test_util.py @@ -19,6 +19,9 @@ from ray.data._internal.util import make_async_gen +logger = logging.getLogger(__file__) + + def block_generator(num_rows: int, num_blocks: int): for _ in range(num_blocks): yield pa.table({"foo": [1] * num_rows}) @@ -131,7 +134,39 @@ def gen(base_iterator): assert e.match("Fail") -logger = logging.getLogger(__file__) +@pytest.mark.parametrize("buffer_size", [0, 1, 2]) +def test_make_async_gen_varying_seq_lengths(buffer_size: int): + """Tests that iterators of varying lengths are handled appropriately""" + + def _gen(base_iterator): + worker_id = next(base_iterator) + + # Make workers produce sequences increasing the same order + # as worker-ids (so that for left workers sequences run out first) + target_length = worker_id + 1 + + return iter([f"worker_{worker_id}:{i}" for i in range(target_length)]) + + num_seqs = 3 + + iterator = make_async_gen( + base_iterator=iter(list(range(num_seqs))), + fn=_gen, + # Make sure individual elements are handle by diff workers + num_workers=num_seqs, + queue_buffer_size=buffer_size, + ) + + seq = list(iterator) + + assert [ + "worker_0:0", + "worker_1:0", + "worker_2:0", + "worker_1:1", + "worker_2:1", + "worker_2:2", + ] == seq @pytest.mark.parametrize("buffer_size", [0, 1, 2]) diff --git a/python/ray/data/tests/conftest.py b/python/ray/data/tests/conftest.py index 0ac8b6a67e077..3a2352260e95b 100644 --- a/python/ray/data/tests/conftest.py +++ b/python/ray/data/tests/conftest.py @@ -17,6 +17,7 @@ from ray.air.util.tensor_extensions.arrow import ArrowTensorArray from ray.data import Schema from ray.data.block import BlockExecStats, BlockMetadata +from ray.data.context import ShuffleStrategy from ray.data.tests.mock_server import * # noqa # Trigger pytest hook to automatically zip test cluster logs to archive dir on failure @@ -265,13 +266,34 @@ def restore_data_context(request): ray.data.context.DataContext._set_current(original) +@pytest.fixture(params=[s for s in ShuffleStrategy]) # noqa: C416 +def configure_shuffle_method(request): + shuffle_strategy = request.param + + ctx = ray.data.context.DataContext.get_current() + + original_shuffle_strategy = ctx.shuffle_strategy + + ctx.shuffle_strategy = shuffle_strategy + + yield request.param + + ctx.shuffle_strategy = original_shuffle_strategy + + @pytest.fixture(params=[True, False]) -def use_push_based_shuffle(request): +def use_polars(request): + use_polars = request.param + ctx = ray.data.context.DataContext.get_current() - original = ctx.use_push_based_shuffle - ctx.use_push_based_shuffle = request.param + + original_use_polars = ctx.use_polars + + ctx.use_polars = use_polars + yield request.param - ctx.use_push_based_shuffle = original + + ctx.use_polars = original_use_polars @pytest.fixture(params=[True, False]) diff --git a/python/ray/data/tests/test_all_to_all.py b/python/ray/data/tests/test_all_to_all.py index 127666435a5f5..7ae235c3b0969 100644 --- a/python/ray/data/tests/test_all_to_all.py +++ b/python/ray/data/tests/test_all_to_all.py @@ -13,7 +13,7 @@ _ref_bundles_iterator_to_block_refs_list, ) from ray.data.aggregate import AggregateFn, Count, Max, Mean, Min, Quantile, Std, Sum -from ray.data.context import DataContext +from ray.data.context import DataContext, ShuffleStrategy from ray.data.tests.conftest import * # noqa from ray.data.tests.util import named_values from ray.tests.conftest import * # noqa @@ -21,7 +21,7 @@ RANDOM_SEED = 123 -def test_empty_shuffle(ray_start_regular_shared): +def test_empty_shuffle(ray_start_regular_shared_2_cpus): ds = ray.data.range(100, override_num_blocks=100) ds = ds.filter(lambda x: x) ds = ds.map_batches(lambda x: x) @@ -29,7 +29,7 @@ def test_empty_shuffle(ray_start_regular_shared): ds.show() -def test_repartition_shuffle(ray_start_regular_shared): +def test_repartition_shuffle(ray_start_regular_shared_2_cpus): ds = ray.data.range(20, override_num_blocks=10) assert ds._plan.initial_num_blocks() == 10 assert ds.sum() == 190 @@ -50,7 +50,7 @@ def test_repartition_shuffle(ray_start_regular_shared): assert large._block_num_rows() == [500] * 20 -def test_repartition_noshuffle(ray_start_regular_shared): +def test_repartition_noshuffle(ray_start_regular_shared_2_cpus): ds = ray.data.range(20, override_num_blocks=10) assert ds._plan.initial_num_blocks() == 10 assert ds.sum() == 190 @@ -82,7 +82,7 @@ def test_repartition_noshuffle(ray_start_regular_shared): assert large._block_num_rows() == [500] * 20 -def test_repartition_shuffle_arrow(ray_start_regular_shared): +def test_repartition_shuffle_arrow(ray_start_regular_shared_2_cpus): ds = ray.data.range(20, override_num_blocks=10) assert ds._plan.initial_num_blocks() == 10 assert ds.count() == 20 @@ -103,7 +103,7 @@ def test_repartition_shuffle_arrow(ray_start_regular_shared): assert large._block_num_rows() == [500] * 20 -def test_unique(ray_start_regular_shared): +def test_unique(ray_start_regular_shared_2_cpus): ds = ray.data.from_items([3, 2, 3, 1, 2, 3]) assert set(ds.unique("item")) == {1, 2, 3} @@ -117,7 +117,7 @@ def test_unique(ray_start_regular_shared): @pytest.mark.parametrize("batch_format", ["pandas", "pyarrow"]) -def test_unique_with_nulls(ray_start_regular_shared, batch_format): +def test_unique_with_nulls(ray_start_regular_shared_2_cpus, batch_format): ds = ray.data.from_items([3, 2, 3, 1, 2, 3, None]) assert set(ds.unique("item")) == {1, 2, 3, None} assert len(ds.unique("item")) == 4 @@ -175,24 +175,24 @@ def test_unique_with_nulls(ray_start_regular_shared, batch_format): assert len(ds3.unique("col3")) == 3 -def test_grouped_dataset_repr(ray_start_regular_shared): +def test_grouped_dataset_repr(ray_start_regular_shared_2_cpus): ds = ray.data.from_items([{"key": "spam"}, {"key": "ham"}, {"key": "spam"}]) assert repr(ds.groupby("key")) == f"GroupedData(dataset={ds!r}, key='key')" -def test_groupby_arrow(ray_start_regular_shared, use_push_based_shuffle): +def test_groupby_arrow(ray_start_regular_shared_2_cpus, configure_shuffle_method): # Test empty dataset. agg_ds = ray.data.range(10).filter(lambda r: r["id"] > 10).groupby("value").count() assert agg_ds.count() == 0 -def test_groupby_none(ray_start_regular_shared): +def test_groupby_none(ray_start_regular_shared_2_cpus): ds = ray.data.range(10) assert ds.groupby(None).min().take_all() == [{"min(id)": 0}] assert ds.groupby(None).max().take_all() == [{"max(id)": 9}] -def test_groupby_errors(ray_start_regular_shared): +def test_groupby_errors(ray_start_regular_shared_2_cpus): ds = ray.data.range(100) ds.groupby(None).count().show() # OK with pytest.raises(ValueError): @@ -212,7 +212,7 @@ def test_map_groups_with_gpus(shutdown_only): assert rows == [{"id": 0}] -def test_map_groups_with_actors(ray_start_regular_shared): +def test_map_groups_with_actors(ray_start_regular_shared_2_cpus): class Identity: def __call__(self, batch): return batch @@ -224,7 +224,7 @@ def __call__(self, batch): assert rows == [{"id": 0}] -def test_map_groups_with_actors_and_args(ray_start_regular_shared): +def test_map_groups_with_actors_and_args(ray_start_regular_shared_2_cpus): class Fn: def __init__(self, x: int, y: Optional[int] = None): self.x = x @@ -250,7 +250,7 @@ def __call__(self, batch, q: int, r: Optional[int] = None): assert rows == [{"x": 0, "y": 1, "q": 2, "r": 3}] -def test_groupby_large_udf_returns(ray_start_regular_shared): +def test_groupby_large_udf_returns(ray_start_regular_shared_2_cpus): # Test for https://github.com/ray-project/ray/issues/44861. # Each UDF return is 128 MiB. If Ray Data doesn't incrementally yield outputs, the @@ -267,7 +267,7 @@ def create_large_data(group): @pytest.mark.parametrize("keys", ["A", ["A", "B"]]) -def test_agg_inputs(ray_start_regular_shared, keys): +def test_agg_inputs(ray_start_regular_shared_2_cpus, keys): xs = list(range(100)) ds = ray.data.from_items([{"A": (x % 3), "B": x, "C": (x % 2)} for x in xs]) @@ -304,7 +304,7 @@ def check_accumulate_merge(a, r): output.take_all() -def test_agg_errors(ray_start_regular_shared): +def test_agg_errors(ray_start_regular_shared_2_cpus): from ray.data.aggregate import Max ds = ray.data.range(100) @@ -318,7 +318,7 @@ def test_agg_errors(ray_start_regular_shared): @pytest.mark.parametrize("num_parts", [1, 30]) -def test_groupby_agg_name_conflict(ray_start_regular_shared, num_parts): +def test_groupby_agg_name_conflict(ray_start_regular_shared_2_cpus, num_parts): # Test aggregation name conflict. xs = list(range(100)) grouped_ds = ( @@ -351,7 +351,7 @@ def test_groupby_agg_name_conflict(ray_start_regular_shared, num_parts): @pytest.mark.parametrize("ds_format", ["pyarrow", "numpy", "pandas"]) -def test_groupby_nans(ray_start_regular_shared, ds_format): +def test_groupby_nans(ray_start_regular_shared_2_cpus, ds_format): ds = ray.data.from_items( [ 1.0, @@ -369,9 +369,12 @@ def test_groupby_nans(ray_start_regular_shared, ds_format): @pytest.mark.parametrize("num_parts", [1, 30]) -@pytest.mark.parametrize("ds_format", ["arrow", "pandas"]) +@pytest.mark.parametrize("ds_format", ["pyarrow", "pandas"]) def test_groupby_tabular_count( - ray_start_regular_shared, ds_format, num_parts, use_push_based_shuffle + ray_start_regular_shared_2_cpus, + ds_format, + num_parts, + configure_shuffle_method, ): # Test built-in count aggregation seed = int(time.time()) @@ -386,8 +389,9 @@ def _to_pandas(ds): ds = ray.data.from_items([{"A": (x % 3), "B": x} for x in xs]).repartition( num_parts ) - if ds_format == "pandas": - ds = _to_pandas(ds) + + ds = ds.map_batches(lambda x: x, batch_size=None, batch_format=ds_format) + agg_ds = ds.groupby("A").count() assert agg_ds.count() == 3 assert list(agg_ds.sort("A").iter_rows()) == [ @@ -400,7 +404,10 @@ def _to_pandas(ds): @pytest.mark.parametrize("num_parts", [1, 30]) @pytest.mark.parametrize("ds_format", ["pyarrow", "pandas"]) def test_groupby_multiple_keys_tabular_count( - ray_start_regular_shared, ds_format, num_parts, use_push_based_shuffle + ray_start_regular_shared_2_cpus, + ds_format, + num_parts, + configure_shuffle_method, ): # Test built-in count aggregation print(f"Seeding RNG for test_groupby_arrow_count with: {RANDOM_SEED}") @@ -426,9 +433,12 @@ def test_groupby_multiple_keys_tabular_count( @pytest.mark.parametrize("num_parts", [1, 30]) -@pytest.mark.parametrize("ds_format", ["arrow", "pandas"]) +@pytest.mark.parametrize("ds_format", ["pyarrow", "pandas"]) def test_groupby_tabular_sum( - ray_start_regular_shared, ds_format, num_parts, use_push_based_shuffle + ray_start_regular_shared_2_cpus, + ds_format, + num_parts, + configure_shuffle_method, ): # Test built-in sum aggregation seed = int(time.time()) @@ -437,14 +447,13 @@ def test_groupby_tabular_sum( xs = list(range(100)) random.shuffle(xs) - def _to_pandas(ds): - return ds.map_batches(lambda x: x, batch_size=None, batch_format="pandas") + def _to_batch_format(ds): + return ds.map_batches(lambda x: x, batch_size=None, batch_format=ds_format) ds = ray.data.from_items([{"A": (x % 3), "B": x} for x in xs]).repartition( num_parts ) - if ds_format == "pandas": - ds = _to_pandas(ds) + ds = _to_batch_format(ds) agg_ds = ds.groupby("A").sum("B") assert agg_ds.count() == 3 @@ -458,8 +467,7 @@ def _to_pandas(ds): ds = ray.data.from_items( [{"A": (x % 3), "B": x} for x in xs] + [{"A": 0, "B": None}] ).repartition(num_parts) - if ds_format == "pandas": - ds = _to_pandas(ds) + ds = _to_batch_format(ds) nan_grouped_ds = ds.groupby("A") nan_agg_ds = nan_grouped_ds.sum("B") assert nan_agg_ds.count() == 3 @@ -485,8 +493,7 @@ def _to_pandas(ds): ds = ray.data.from_items([{"A": (x % 3), "B": None} for x in xs]).repartition( num_parts ) - if ds_format == "pandas": - ds = _to_pandas(ds) + ds = _to_batch_format(ds) nan_agg_ds = ds.groupby("A").sum("B") assert nan_agg_ds.count() == 3 pd.testing.assert_frame_equal( @@ -502,7 +509,7 @@ def _to_pandas(ds): @pytest.mark.parametrize("num_parts", [1, 30]) @pytest.mark.parametrize("ds_format", ["arrow", "pandas"]) -def test_global_tabular_sum(ray_start_regular_shared, ds_format, num_parts): +def test_global_tabular_sum(ray_start_regular_shared_2_cpus, ds_format, num_parts): seed = int(time.time()) print(f"Seeding RNG for test_global_arrow_sum with: {seed}") random.seed(seed) @@ -543,7 +550,7 @@ def _to_pandas(ds): @pytest.mark.parametrize("num_parts", [1, 30]) @pytest.mark.parametrize("ds_format", ["arrow", "pandas"]) -def test_groupby_tabular_min(ray_start_regular_shared, ds_format, num_parts): +def test_groupby_tabular_min(ray_start_regular_shared_2_cpus, ds_format, num_parts): # Test built-in min aggregation seed = int(time.time()) print(f"Seeding RNG for test_groupby_tabular_min with: {seed}") @@ -617,7 +624,7 @@ def _to_pandas(ds): @pytest.mark.parametrize("num_parts", [1, 30]) @pytest.mark.parametrize("ds_format", ["arrow", "pandas"]) -def test_groupby_tabular_max(ray_start_regular_shared, ds_format, num_parts): +def test_groupby_tabular_max(ray_start_regular_shared_2_cpus, ds_format, num_parts): # Test built-in max aggregation seed = int(time.time()) print(f"Seeding RNG for test_groupby_tabular_max with: {seed}") @@ -691,7 +698,7 @@ def _to_pandas(ds): @pytest.mark.parametrize("num_parts", [1, 30]) @pytest.mark.parametrize("ds_format", ["arrow", "pandas"]) -def test_groupby_tabular_mean(ray_start_regular_shared, ds_format, num_parts): +def test_groupby_tabular_mean(ray_start_regular_shared_2_cpus, ds_format, num_parts): # Test built-in mean aggregation seed = int(time.time()) print(f"Seeding RNG for test_groupby_tabular_mean with: {seed}") @@ -765,7 +772,7 @@ def _to_pandas(ds): @pytest.mark.parametrize("num_parts", [1, 30]) @pytest.mark.parametrize("ds_format", ["arrow", "pandas"]) -def test_groupby_tabular_std(ray_start_regular_shared, ds_format, num_parts): +def test_groupby_tabular_std(ray_start_regular_shared_2_cpus, ds_format, num_parts): # Test built-in std aggregation seed = int(time.time()) print(f"Seeding RNG for test_groupby_tabular_std with: {seed}") @@ -826,7 +833,7 @@ def _to_arrow(ds): @pytest.mark.parametrize("num_parts", [1, 30]) -def test_groupby_arrow_multicolumn(ray_start_regular_shared, num_parts): +def test_groupby_arrow_multicolumn(ray_start_regular_shared_2_cpus, num_parts): # Test built-in mean aggregation on multiple columns seed = int(time.time()) print(f"Seeding RNG for test_groupby_arrow_multicolumn with: {seed}") @@ -861,7 +868,7 @@ def test_groupby_arrow_multicolumn(ray_start_regular_shared, num_parts): assert result_row["mean(B)"] == df["B"].mean() -def test_groupby_agg_bad_on(ray_start_regular_shared): +def test_groupby_agg_bad_on(ray_start_regular_shared_2_cpus): # Test bad on for groupby aggregation xs = list(range(100)) df = pd.DataFrame({"A": [x % 3 for x in xs], "B": xs, "C": [2 * x for x in xs]}) @@ -902,7 +909,7 @@ def test_groupby_agg_bad_on(ray_start_regular_shared): @pytest.mark.parametrize("num_parts", [1, 30]) -def test_groupby_arrow_multi_agg(ray_start_regular_shared, num_parts): +def test_groupby_arrow_multi_agg(ray_start_regular_shared_2_cpus, num_parts): seed = int(time.time()) print(f"Seeding RNG for test_groupby_arrow_multi_agg with: {seed}") random.seed(seed) @@ -959,7 +966,7 @@ def test_groupby_arrow_multi_agg(ray_start_regular_shared, num_parts): @pytest.mark.parametrize("num_parts", [1, 30]) -def test_groupby_arrow_multi_agg_alias(ray_start_regular_shared, num_parts): +def test_groupby_arrow_multi_agg_alias(ray_start_regular_shared_2_cpus, num_parts): seed = int(time.time()) print(f"Seeding RNG for test_groupby_arrow_multi_agg with: {seed}") random.seed(seed) @@ -1018,7 +1025,9 @@ def test_groupby_arrow_multi_agg_alias(ray_start_regular_shared, num_parts): @pytest.mark.parametrize("num_parts", [1, 2, 30]) -def test_groupby_map_groups_for_none_groupkey(ray_start_regular_shared, num_parts): +def test_groupby_map_groups_for_none_groupkey( + ray_start_regular_shared_2_cpus, num_parts +): ds = ray.data.from_items(list(range(100))) mapped = ( ds.repartition(num_parts) @@ -1029,7 +1038,7 @@ def test_groupby_map_groups_for_none_groupkey(ray_start_regular_shared, num_part assert mapped.take_all() == named_values("out", [99]) -def test_groupby_map_groups_perf(ray_start_regular_shared): +def test_groupby_map_groups_perf(ray_start_regular_shared_2_cpus): data_list = [x % 100 for x in range(5000000)] ds = ray.data.from_pandas(pd.DataFrame({"A": data_list})) start = time.perf_counter() @@ -1041,7 +1050,7 @@ def test_groupby_map_groups_perf(ray_start_regular_shared): @pytest.mark.parametrize("num_parts", [1, 2, 3, 30]) -def test_groupby_map_groups_for_pandas(ray_start_regular_shared, num_parts): +def test_groupby_map_groups_for_pandas(ray_start_regular_shared_2_cpus, num_parts): df = pd.DataFrame({"A": "a a b".split(), "B": [1, 1, 3], "C": [4, 6, 5]}) grouped = ray.data.from_pandas(df).repartition(num_parts).groupby("A") @@ -1062,7 +1071,7 @@ def test_groupby_map_groups_for_pandas(ray_start_regular_shared, num_parts): @pytest.mark.parametrize("num_parts", [1, 2, 3, 30]) -def test_groupby_map_groups_for_arrow(ray_start_regular_shared, num_parts): +def test_groupby_map_groups_for_arrow(ray_start_regular_shared_2_cpus, num_parts): at = pa.Table.from_pydict({"A": "a a b".split(), "B": [1, 1, 3], "C": [4, 6, 5]}) grouped = ray.data.from_arrow(at).repartition(num_parts).groupby("A") @@ -1087,7 +1096,7 @@ def normalize(at: pa.Table): assert result.equals(expected) -def test_groupby_map_groups_for_numpy(ray_start_regular_shared): +def test_groupby_map_groups_for_numpy(ray_start_regular_shared_2_cpus): ds = ray.data.from_items( [ {"group": 1, "value": 1}, @@ -1107,7 +1116,7 @@ def func(group): assert result.equals(expected) -def test_groupby_map_groups_with_different_types(ray_start_regular_shared): +def test_groupby_map_groups_with_different_types(ray_start_regular_shared_2_cpus): ds = ray.data.from_items( [ {"group": 1, "value": 1}, @@ -1127,7 +1136,9 @@ def func(group): @pytest.mark.parametrize("num_parts", [1, 30]) -def test_groupby_map_groups_multiple_batch_formats(ray_start_regular_shared, num_parts): +def test_groupby_map_groups_multiple_batch_formats( + ray_start_regular_shared_2_cpus, num_parts +): # Reproduces https://github.com/ray-project/ray/issues/39206 def identity(batch): return batch @@ -1150,7 +1161,7 @@ def identity(batch): ] -def test_groupby_map_groups_extra_args(ray_start_regular_shared): +def test_groupby_map_groups_extra_args(ray_start_regular_shared_2_cpus): ds = ray.data.from_items( [ {"group": 1, "value": 1}, @@ -1175,7 +1186,10 @@ def func(df, a, b, c): @pytest.mark.parametrize("num_parts", [1, 30]) @pytest.mark.parametrize("ds_format", ["pyarrow", "pandas", "numpy"]) def test_groupby_map_groups_multicolumn( - ray_start_regular_shared, ds_format, num_parts, use_push_based_shuffle + ray_start_regular_shared_2_cpus, + ds_format, + num_parts, + configure_shuffle_method, ): # Test built-in count aggregation random.seed(RANDOM_SEED) @@ -1203,7 +1217,10 @@ def test_groupby_map_groups_multicolumn( @pytest.mark.parametrize("num_parts", [1, 30]) @pytest.mark.parametrize("ds_format", ["pyarrow", "pandas", "numpy"]) def test_groupby_map_groups_multicolumn_with_nan( - ray_start_regular_shared, ds_format, num_parts, use_push_based_shuffle + ray_start_regular_shared_2_cpus, + ds_format, + num_parts, + configure_shuffle_method, ): # Test with some NaN values rng = np.random.default_rng(RANDOM_SEED) @@ -1263,13 +1280,13 @@ def func(x, y): assert "MapBatches(func)" in ds.__repr__() -def test_random_block_order_schema(ray_start_regular_shared): +def test_random_block_order_schema(ray_start_regular_shared_2_cpus): df = pd.DataFrame({"a": np.random.rand(10), "b": np.random.rand(10)}) ds = ray.data.from_pandas(df).randomize_block_order() ds.schema().names == ["a", "b"] -def test_random_block_order(ray_start_regular_shared, restore_data_context): +def test_random_block_order(ray_start_regular_shared_2_cpus, restore_data_context): ctx = DataContext.get_current() ctx.execution_options.preserve_order = True @@ -1293,7 +1310,7 @@ def test_random_block_order(ray_start_regular_shared, restore_data_context): # tests should only be carefully reordered to retain this invariant! -def test_random_shuffle(shutdown_only, use_push_based_shuffle): +def test_random_shuffle(shutdown_only, configure_shuffle_method): r1 = ray.data.range(100).random_shuffle().take(999) r2 = ray.data.range(100).random_shuffle().take(999) assert r1 != r2, (r1, r2) @@ -1388,7 +1405,9 @@ def test_random_shuffle_check_random(shutdown_only): prev = x -def test_random_shuffle_with_custom_resource(ray_start_cluster, use_push_based_shuffle): +def test_random_shuffle_with_custom_resource( + ray_start_cluster, configure_shuffle_method +): cluster = ray_start_cluster # Create two nodes which have different custom resources. cluster.add_node( @@ -1410,7 +1429,7 @@ def test_random_shuffle_with_custom_resource(ray_start_cluster, use_push_based_s assert "2 nodes used" not in ds.stats() -def test_random_shuffle_spread(ray_start_cluster, use_push_based_shuffle): +def test_random_shuffle_spread(ray_start_cluster, configure_shuffle_method): cluster = ray_start_cluster cluster.add_node( resources={"bar:1": 100}, @@ -1439,7 +1458,7 @@ def get_node_id(): locations.extend(location_data[block]["node_ids"]) assert "2 nodes used" in ds.stats() - if not use_push_based_shuffle: + if configure_shuffle_method != ShuffleStrategy.SORT_SHUFFLE_PUSH_BASED: # We don't check this for push-based shuffle since it will try to # colocate reduce tasks to improve locality. assert set(locations) == {node1_id, node2_id} diff --git a/python/ray/data/tests/test_batcher.py b/python/ray/data/tests/test_batcher.py index 65394b09d82b0..2dce1053261eb 100644 --- a/python/ray/data/tests/test_batcher.py +++ b/python/ray/data/tests/test_batcher.py @@ -48,7 +48,7 @@ def add_and_check( assert batcher._batch_head == 0 assert batcher._builder.num_rows() == pending_buffer_size - assert batcher._materialized_buffer_size() == materialized_buffer_size + assert batcher._num_compacted_rows() == materialized_buffer_size def next_and_check( current_cursor, @@ -71,7 +71,7 @@ def next_and_check( assert len(batch) == batch_size assert batcher._builder.num_rows() == pending_buffer_size - assert batcher._materialized_buffer_size() == materialized_buffer_size + assert batcher._num_compacted_rows() == materialized_buffer_size if should_have_batch_after: assert batcher.has_batch() @@ -247,6 +247,22 @@ def test_local_shuffle_determinism(batch_size, local_shuffle_buffer_size): assert all(batch_map[batch["id"][0]]["id"] == batch["id"]) +def test_local_shuffle_buffer_warns_if_too_large(shutdown_only): + ray.shutdown() + ray.init(object_store_memory=128 * 1024 * 1024) + + # Each row is 16 MiB * 8 = 128 MiB + ds = ray.data.range_tensor(2, shape=(16, 1024, 1024)) + + # Test that Ray Data emits a warning if the local shuffle buffer size would cause + # spilling. + with pytest.warns(UserWarning, match="shuffle buffer"): + # Each row is 128 MiB and the shuffle buffer size is 2 rows, so expect at least + # 256 MiB of memory usage > 128 MiB total on node. + batches = ds.iter_batches(batch_size=1, local_shuffle_buffer_size=2) + next(iter(batches)) + + if __name__ == "__main__": import sys diff --git a/python/ray/data/tests/test_execution_optimizer.py b/python/ray/data/tests/test_execution_optimizer.py index 86777f1928a33..528ef15a98316 100644 --- a/python/ray/data/tests/test_execution_optimizer.py +++ b/python/ray/data/tests/test_execution_optimizer.py @@ -104,7 +104,7 @@ def _check_valid_plan_and_result( assert op in ds.stats(), f"Operator {op} not found: {ds.stats()}" -def test_read_operator(ray_start_regular_shared): +def test_read_operator(ray_start_regular_shared_2_cpus): ctx = DataContext.get_current() planner = Planner() op = get_parquet_read_logical_op() @@ -139,7 +139,7 @@ def read_fn(): ray.data.read_datasource(StubDatasource()).materialize() -def test_split_blocks_operator(ray_start_regular_shared): +def test_split_blocks_operator(ray_start_regular_shared_2_cpus): ctx = DataContext.get_current() planner = Planner() @@ -175,7 +175,7 @@ def test_split_blocks_operator(ray_start_regular_shared): assert up_physical_op.name == "ReadParquet->SplitBlocks(10)" -def test_from_operators(ray_start_regular_shared): +def test_from_operators(ray_start_regular_shared_2_cpus): ctx = DataContext.get_current() op_classes = [ @@ -198,7 +198,7 @@ def test_from_operators(ray_start_regular_shared): assert physical_op._logical_operators == [op] -def test_from_items_e2e(ray_start_regular_shared): +def test_from_items_e2e(ray_start_regular_shared_2_cpus): data = ["Hello", "World"] ds = ray.data.from_items(data) assert ds.take_all() == named_values("item", data), ds @@ -209,7 +209,7 @@ def test_from_items_e2e(ray_start_regular_shared): _check_usage_record(["FromItems"]) -def test_map_operator_udf_name(ray_start_regular_shared): +def test_map_operator_udf_name(ray_start_regular_shared_2_cpus): # Test the name of the Map operator with different types of UDF. def normal_function(x): return x @@ -253,7 +253,7 @@ def method(self, x): assert op.name == f"Map({expected_name})" -def test_map_batches_operator(ray_start_regular_shared): +def test_map_batches_operator(ray_start_regular_shared_2_cpus): ctx = DataContext.get_current() planner = Planner() @@ -274,14 +274,14 @@ def test_map_batches_operator(ray_start_regular_shared): assert physical_op._logical_operators == [op] -def test_map_batches_e2e(ray_start_regular_shared): +def test_map_batches_e2e(ray_start_regular_shared_2_cpus): ds = ray.data.range(5) ds = ds.map_batches(column_udf("id", lambda x: x)) assert extract_values("id", ds.take_all()) == list(range(5)), ds _check_usage_record(["ReadRange", "MapBatches"]) -def test_map_rows_operator(ray_start_regular_shared): +def test_map_rows_operator(ray_start_regular_shared_2_cpus): ctx = DataContext.get_current() planner = Planner() @@ -299,14 +299,14 @@ def test_map_rows_operator(ray_start_regular_shared): assert isinstance(physical_op.input_dependencies[0], MapOperator) -def test_map_rows_e2e(ray_start_regular_shared): +def test_map_rows_e2e(ray_start_regular_shared_2_cpus): ds = ray.data.range(5) ds = ds.map(column_udf("id", lambda x: x + 1)) assert extract_values("id", ds.take_all()) == [1, 2, 3, 4, 5], ds _check_usage_record(["ReadRange", "Map"]) -def test_filter_operator(ray_start_regular_shared): +def test_filter_operator(ray_start_regular_shared_2_cpus): ctx = DataContext.get_current() planner = Planner() @@ -328,14 +328,14 @@ def test_filter_operator(ray_start_regular_shared): ) -def test_filter_e2e(ray_start_regular_shared): +def test_filter_e2e(ray_start_regular_shared_2_cpus): ds = ray.data.range(5) ds = ds.filter(fn=lambda x: x["id"] % 2 == 0) - assert extract_values("id", ds.take_all()) == [0, 2, 4], ds + assert sorted(extract_values("id", ds.take_all())) == [0, 2, 4], ds _check_usage_record(["ReadRange", "Filter"]) -def test_project_operator_select(ray_start_regular_shared): +def test_project_operator_select(ray_start_regular_shared_2_cpus): """ Checks that the physical plan is properly generated for the Project operator from select columns. @@ -358,7 +358,7 @@ def test_project_operator_select(ray_start_regular_shared): assert isinstance(physical_op.input_dependency, TaskPoolMapOperator) -def test_project_operator_rename(ray_start_regular_shared): +def test_project_operator_rename(ray_start_regular_shared_2_cpus): """ Checks that the physical plan is properly generated for the Project operator from rename columns. @@ -382,7 +382,7 @@ def test_project_operator_rename(ray_start_regular_shared): assert isinstance(physical_op.input_dependency, TaskPoolMapOperator) -def test_flat_map(ray_start_regular_shared): +def test_flat_map(ray_start_regular_shared_2_cpus): ctx = DataContext.get_current() planner = Planner() @@ -404,14 +404,14 @@ def test_flat_map(ray_start_regular_shared): ) -def test_flat_map_e2e(ray_start_regular_shared): +def test_flat_map_e2e(ray_start_regular_shared_2_cpus): ds = ray.data.range(2) ds = ds.flat_map(fn=lambda x: [{"id": x["id"]}, {"id": x["id"]}]) assert extract_values("id", ds.take_all()) == [0, 0, 1, 1], ds _check_usage_record(["ReadRange", "FlatMap"]) -def test_column_ops_e2e(ray_start_regular_shared): +def test_column_ops_e2e(ray_start_regular_shared_2_cpus): ds = ray.data.range(2) ds = ds.add_column(fn=lambda df: df.iloc[:, 0], col="new_col") assert ds.take_all() == [{"id": 0, "new_col": 0}, {"id": 1, "new_col": 1}], ds @@ -426,7 +426,7 @@ def test_column_ops_e2e(ray_start_regular_shared): _check_usage_record(["ReadRange", "MapBatches"]) -def test_random_sample_e2e(ray_start_regular_shared): +def test_random_sample_e2e(ray_start_regular_shared_2_cpus): import math def ensure_sample_size_close(dataset, sample_percent=0.5): @@ -447,7 +447,7 @@ def ensure_sample_size_close(dataset, sample_percent=0.5): _check_usage_record(["ReadRange", "MapBatches"]) -def test_random_shuffle_operator(ray_start_regular_shared): +def test_random_shuffle_operator(ray_start_regular_shared_2_cpus): ctx = DataContext.get_current() planner = Planner() @@ -472,7 +472,7 @@ def test_random_shuffle_operator(ray_start_regular_shared): assert physical_op._logical_operators == [op] -def test_random_shuffle_e2e(ray_start_regular_shared, use_push_based_shuffle): +def test_random_shuffle_e2e(ray_start_regular_shared_2_cpus, configure_shuffle_method): ds = ray.data.range(12, override_num_blocks=4) r1 = extract_values("id", ds.random_shuffle(seed=0).take_all()) r2 = extract_values("id", ds.random_shuffle(seed=1024).take_all()) @@ -486,7 +486,7 @@ def test_random_shuffle_e2e(ray_start_regular_shared, use_push_based_shuffle): "shuffle", [True, False], ) -def test_repartition_operator(ray_start_regular_shared, shuffle): +def test_repartition_operator(ray_start_regular_shared_2_cpus, shuffle): ctx = DataContext.get_current() planner = Planner() @@ -518,7 +518,9 @@ def test_repartition_operator(ray_start_regular_shared, shuffle): "shuffle", [True, False], ) -def test_repartition_e2e(ray_start_regular_shared, use_push_based_shuffle, shuffle): +def test_repartition_e2e( + ray_start_regular_shared_2_cpus, configure_shuffle_method, shuffle +): def _check_repartition_usage_and_stats(ds): _check_usage_record(["ReadRange", "Repartition"]) ds_stats: DatasetStats = ds._plan.stats() @@ -551,7 +553,7 @@ def _check_repartition_usage_and_stats(ds): assert ds._plan.initial_num_blocks() == 4, ds._plan.initial_num_blocks() assert ds.sum() == sum(range(22)) if shuffle: - assert ds._block_num_rows() == [6, 6, 6, 4], ds._block_num_rows() + assert ds._block_num_rows() == [9, 9, 4, 0], ds._block_num_rows() else: assert ds._block_num_rows() == [5, 6, 5, 6], ds._block_num_rows() _check_repartition_usage_and_stats(ds) @@ -565,7 +567,7 @@ def _check_repartition_usage_and_stats(ds): @pytest.mark.parametrize("preserve_order", (True, False)) -def test_union_operator(ray_start_regular_shared, preserve_order): +def test_union_operator(ray_start_regular_shared_2_cpus, preserve_order): ctx = DataContext.get_current() planner = Planner() @@ -596,7 +598,7 @@ def test_union_operator(ray_start_regular_shared, preserve_order): @pytest.mark.parametrize("preserve_order", (True, False)) -def test_union_e2e(ray_start_regular_shared, preserve_order): +def test_union_e2e(ray_start_regular_shared_2_cpus, preserve_order): execution_options = ExecutionOptions(preserve_order=preserve_order) ctx = ray.data.DataContext.get_current() ctx.execution_options = execution_options @@ -642,7 +644,7 @@ def test_union_e2e(ray_start_regular_shared, preserve_order): assert ds2.take_all() == (ds2_result + ds_result * 2) -def test_read_map_batches_operator_fusion(ray_start_regular_shared): +def test_read_map_batches_operator_fusion(ray_start_regular_shared_2_cpus): ctx = DataContext.get_current() # Test that Read is fused with MapBatches. @@ -671,7 +673,7 @@ def test_read_map_batches_operator_fusion(ray_start_regular_shared): assert physical_op._logical_operators == [read_op, op] -def test_read_map_chain_operator_fusion(ray_start_regular_shared): +def test_read_map_chain_operator_fusion(ray_start_regular_shared_2_cpus): ctx = DataContext.get_current() # Test that a chain of different map operators are fused. @@ -702,7 +704,7 @@ def test_read_map_chain_operator_fusion(ray_start_regular_shared): def test_read_map_batches_operator_fusion_compatible_remote_args( - ray_start_regular_shared, + ray_start_regular_shared_2_cpus, ): ctx = DataContext.get_current() @@ -755,7 +757,7 @@ def test_read_map_batches_operator_fusion_compatible_remote_args( def test_read_map_batches_operator_fusion_incompatible_remote_args( - ray_start_regular_shared, + ray_start_regular_shared_2_cpus, ): ctx = DataContext.get_current() @@ -800,7 +802,7 @@ def test_read_map_batches_operator_fusion_incompatible_remote_args( def test_read_map_batches_operator_fusion_compute_tasks_to_actors( - ray_start_regular_shared, + ray_start_regular_shared_2_cpus, ): ctx = DataContext.get_current() @@ -823,7 +825,7 @@ def test_read_map_batches_operator_fusion_compute_tasks_to_actors( def test_read_map_batches_operator_fusion_compute_read_to_actors( - ray_start_regular_shared, + ray_start_regular_shared_2_cpus, ): ctx = DataContext.get_current() @@ -844,7 +846,7 @@ def test_read_map_batches_operator_fusion_compute_read_to_actors( def test_read_map_batches_operator_fusion_incompatible_compute( - ray_start_regular_shared, + ray_start_regular_shared_2_cpus, ): ctx = DataContext.get_current() @@ -869,7 +871,7 @@ def test_read_map_batches_operator_fusion_incompatible_compute( def test_read_map_batches_operator_fusion_min_rows_per_bundled_input( - ray_start_regular_shared, + ray_start_regular_shared_2_cpus, ): ctx = DataContext.get_current() @@ -904,7 +906,7 @@ def test_read_map_batches_operator_fusion_min_rows_per_bundled_input( def test_read_map_batches_operator_fusion_with_randomize_blocks_operator( - ray_start_regular_shared, + ray_start_regular_shared_2_cpus, ): # Note: We currently do not fuse MapBatches->RandomizeBlocks. # This test is to ensure that we don't accidentally fuse them. @@ -925,7 +927,7 @@ def fn(batch): def test_read_map_batches_operator_fusion_with_random_shuffle_operator( - ray_start_regular_shared, use_push_based_shuffle + ray_start_regular_shared_2_cpus, configure_shuffle_method ): # Note: we currently only support fusing MapOperator->AllToAllOperator. def fn(batch): @@ -990,7 +992,7 @@ def fn(_): @pytest.mark.parametrize("shuffle", (True, False)) def test_read_map_batches_operator_fusion_with_repartition_operator( - ray_start_regular_shared, shuffle, use_push_based_shuffle + ray_start_regular_shared_2_cpus, shuffle, configure_shuffle_method ): def fn(batch): return {"id": [x + 1 for x in batch["id"]]} @@ -1011,7 +1013,9 @@ def fn(batch): _check_usage_record(["ReadRange", "MapBatches", "Repartition"]) -def test_read_map_batches_operator_fusion_with_sort_operator(ray_start_regular_shared): +def test_read_map_batches_operator_fusion_with_sort_operator( + ray_start_regular_shared_2_cpus, +): # Note: We currently do not fuse MapBatches->Sort. # This test is to ensure that we don't accidentally fuse them, until # we implement it later. @@ -1031,7 +1035,7 @@ def fn(batch): def test_read_map_batches_operator_fusion_with_aggregate_operator( - ray_start_regular_shared, + ray_start_regular_shared_2_cpus, ): from ray.data.aggregate import AggregateFn @@ -1061,7 +1065,7 @@ def fn(batch): def test_read_map_chain_operator_fusion_e2e( - ray_start_regular_shared, + ray_start_regular_shared_2_cpus, ): ds = ray.data.range(10, override_num_blocks=2) ds = ds.filter(lambda x: x["id"] % 2 == 0) @@ -1090,7 +1094,7 @@ def test_read_map_chain_operator_fusion_e2e( _check_usage_record(["ReadRange", "Filter", "Map", "MapBatches", "FlatMap"]) -def test_write_fusion(ray_start_regular_shared, tmp_path): +def test_write_fusion(ray_start_regular_shared_2_cpus, tmp_path): ds = ray.data.range(10, override_num_blocks=2) ds.write_csv(tmp_path) assert "ReadRange->Write" in ds._write_ds.stats() @@ -1127,7 +1131,7 @@ def test_write_fusion(ray_start_regular_shared, tmp_path): ], ) def test_map_fusion_with_concurrency_arg( - ray_start_regular_shared, + ray_start_regular_shared_2_cpus, up_use_actor, up_concurrency, down_use_actor, @@ -1168,7 +1172,7 @@ def map(row): assert name not in stats, stats -def test_write_operator(ray_start_regular_shared, tmp_path): +def test_write_operator(ray_start_regular_shared_2_cpus, tmp_path): ctx = DataContext.get_current() concurrency = 2 @@ -1194,7 +1198,7 @@ def test_write_operator(ray_start_regular_shared, tmp_path): def test_sort_operator( - ray_start_regular_shared, + ray_start_regular_shared_2_cpus, ): ctx = DataContext.get_current() @@ -1217,7 +1221,7 @@ def test_sort_operator( ) -def test_sort_e2e(ray_start_regular_shared, use_push_based_shuffle, tmp_path): +def test_sort_e2e(ray_start_regular_shared_2_cpus, configure_shuffle_method, tmp_path): ds = ray.data.range(100, override_num_blocks=4) ds = ds.random_shuffle() ds = ds.sort("id") @@ -1238,7 +1242,7 @@ def test_sort_e2e(ray_start_regular_shared, use_push_based_shuffle, tmp_path): assert [d["one"] for d in r2] == list(reversed(range(100))) -def test_sort_validate_keys(ray_start_regular_shared): +def test_sort_validate_keys(ray_start_regular_shared_2_cpus): ds = ray.data.range(10) assert extract_values("id", ds.sort("id").take_all()) == list(range(10)) @@ -1283,7 +1287,7 @@ def test_inherit_batch_format_rule(): assert optimized_plan.dag._batch_format == "pandas" -def test_batch_format_on_sort(ray_start_regular_shared): +def test_batch_format_on_sort(ray_start_regular_shared_2_cpus): """Checks that the Sort op can inherit batch_format from upstream ops correctly.""" ds = ray.data.from_items( [ @@ -1308,7 +1312,7 @@ def test_batch_format_on_sort(ray_start_regular_shared): pd.testing.assert_frame_equal(df_actual, df_expected) -def test_batch_format_on_aggregate(ray_start_regular_shared): +def test_batch_format_on_aggregate(ray_start_regular_shared_2_cpus): """Checks that the Aggregate op can inherit batch_format from upstream ops correctly.""" from ray.data.aggregate import AggregateFn @@ -1334,7 +1338,7 @@ def test_batch_format_on_aggregate(ray_start_regular_shared): ) == {"prod": 384} -def test_aggregate_operator(ray_start_regular_shared): +def test_aggregate_operator(ray_start_regular_shared_2_cpus): ctx = DataContext.get_current() planner = Planner() @@ -1360,7 +1364,7 @@ def test_aggregate_operator(ray_start_regular_shared): assert physical_op._logical_operators == [op] -def test_aggregate_e2e(ray_start_regular_shared, use_push_based_shuffle): +def test_aggregate_e2e(ray_start_regular_shared_2_cpus, configure_shuffle_method): ds = ray.data.range(100, override_num_blocks=4) ds = ds.groupby("id").count() assert ds.count() == 100 @@ -1369,7 +1373,7 @@ def test_aggregate_e2e(ray_start_regular_shared, use_push_based_shuffle): _check_usage_record(["ReadRange", "Aggregate"]) -def test_aggregate_validate_keys(ray_start_regular_shared): +def test_aggregate_validate_keys(ray_start_regular_shared_2_cpus): ds = ray.data.range(10) invalid_col_name = "invalid_column" with pytest.raises(ValueError): @@ -1404,7 +1408,7 @@ def test_aggregate_validate_keys(ray_start_regular_shared): ds_named.groupby(invalid_col_name).count() -def test_zip_operator(ray_start_regular_shared): +def test_zip_operator(ray_start_regular_shared_2_cpus): ctx = DataContext.get_current() planner = Planner() @@ -1433,7 +1437,7 @@ def test_zip_operator(ray_start_regular_shared): "num_blocks1,num_blocks2", list(itertools.combinations_with_replacement(range(1, 12), 2)), ) -def test_zip_e2e(ray_start_regular_shared, num_blocks1, num_blocks2): +def test_zip_e2e(ray_start_regular_shared_2_cpus, num_blocks1, num_blocks2): n = 12 ds1 = ray.data.range(n, override_num_blocks=num_blocks1) ds2 = ray.data.range(n, override_num_blocks=num_blocks2).map( @@ -1444,7 +1448,7 @@ def test_zip_e2e(ray_start_regular_shared, num_blocks1, num_blocks2): _check_usage_record(["ReadRange", "Zip"]) -def test_from_dask_e2e(ray_start_regular_shared): +def test_from_dask_e2e(ray_start_regular_shared_2_cpus): import dask.dataframe as dd df = pd.DataFrame({"one": list(range(100)), "two": list(range(100))}) @@ -1462,7 +1466,7 @@ def test_from_dask_e2e(ray_start_regular_shared): _check_usage_record(["FromPandas"]) -def test_from_modin_e2e(ray_start_regular_shared): +def test_from_modin_e2e(ray_start_regular_shared_2_cpus): import modin.pandas as mopd df = pd.DataFrame( @@ -1485,7 +1489,7 @@ def test_from_modin_e2e(ray_start_regular_shared): @pytest.mark.parametrize("enable_pandas_block", [False, True]) -def test_from_pandas_refs_e2e(ray_start_regular_shared, enable_pandas_block): +def test_from_pandas_refs_e2e(ray_start_regular_shared_2_cpus, enable_pandas_block): ctx = ray.data.context.DataContext.get_current() old_enable_pandas_block = ctx.enable_pandas_block ctx.enable_pandas_block = enable_pandas_block @@ -1523,7 +1527,7 @@ def test_from_pandas_refs_e2e(ray_start_regular_shared, enable_pandas_block): ctx.enable_pandas_block = old_enable_pandas_block -def test_from_numpy_refs_e2e(ray_start_regular_shared): +def test_from_numpy_refs_e2e(ray_start_regular_shared_2_cpus): import numpy as np arr1 = np.expand_dims(np.arange(0, 4), axis=1) @@ -1556,7 +1560,7 @@ def test_from_numpy_refs_e2e(ray_start_regular_shared): _check_usage_record(["FromNumpy"]) -def test_from_arrow_refs_e2e(ray_start_regular_shared): +def test_from_arrow_refs_e2e(ray_start_regular_shared_2_cpus): import pyarrow as pa df1 = pd.DataFrame({"one": [1, 2, 3], "two": ["a", "b", "c"]}) @@ -1584,7 +1588,7 @@ def test_from_arrow_refs_e2e(ray_start_regular_shared): _check_usage_record(["FromArrow"]) -def test_from_huggingface_e2e(ray_start_regular_shared): +def test_from_huggingface_e2e(ray_start_regular_shared_2_cpus): import datasets from ray.data.tests.test_huggingface import hfds_assert_equals @@ -1637,7 +1641,7 @@ def test_from_huggingface_e2e(ray_start_regular_shared): sys.version_info >= (3, 12), reason="Skip due to incompatibility tensorflow with Python 3.12+", ) -def test_from_tf_e2e(ray_start_regular_shared): +def test_from_tf_e2e(ray_start_regular_shared_2_cpus): import tensorflow as tf import tensorflow_datasets as tfds @@ -1662,7 +1666,7 @@ def test_from_tf_e2e(ray_start_regular_shared): _check_usage_record(["FromItems"]) -def test_from_torch_e2e(ray_start_regular_shared, tmp_path): +def test_from_torch_e2e(ray_start_regular_shared_2_cpus, tmp_path): import torchvision torch_dataset = torchvision.datasets.FashionMNIST(tmp_path, download=True) @@ -1685,7 +1689,7 @@ def test_from_torch_e2e(ray_start_regular_shared, tmp_path): reason="Limit pushdown currently disabled, see " "https://github.com/ray-project/ray/issues/36295" ) -def test_limit_pushdown(ray_start_regular_shared): +def test_limit_pushdown(ray_start_regular_shared_2_cpus): def f1(x): return x @@ -1766,7 +1770,7 @@ def f2(x): def test_execute_to_legacy_block_list( - ray_start_regular_shared, + ray_start_regular_shared_2_cpus, ): ds = ray.data.range(10) # Stats not initialized until `ds.iter_rows()` is called @@ -1781,7 +1785,7 @@ def test_execute_to_legacy_block_list( def test_streaming_executor( - ray_start_regular_shared, + ray_start_regular_shared_2_cpus, ): ds = ray.data.range(100, override_num_blocks=4) ds = ds.map_batches(lambda x: x) @@ -1799,7 +1803,7 @@ def test_streaming_executor( def test_schema_partial_execution( - ray_start_regular_shared, + ray_start_regular_shared_2_cpus, ): fields = [ ("sepal.length", pa.float64()), @@ -1833,7 +1837,9 @@ def check_transform_fns(op, expected_types): @pytest.mark.skip("Needs zero-copy optimization for read->map_batches.") -def test_zero_copy_fusion_eliminate_build_output_blocks(ray_start_regular_shared): +def test_zero_copy_fusion_eliminate_build_output_blocks( + ray_start_regular_shared_2_cpus, +): ctx = DataContext.get_current() # Test the EliminateBuildOutputBlocks optimization rule. diff --git a/python/ray/data/tests/test_parquet.py b/python/ray/data/tests/test_parquet.py index 0f766e163d147..3a4630ce9419c 100644 --- a/python/ray/data/tests/test_parquet.py +++ b/python/ray/data/tests/test_parquet.py @@ -1425,6 +1425,19 @@ def write_parquet_file(path, file_index): assert ds1.take_all() == ds2.take_all() +def test_read_file_with_partition_values(ray_start_regular_shared, tmp_path): + # Typically, partition values are excluded from the Parquet file and are instead + # encoded in the directory structure. However, in some cases, partition values + # are also included in the Parquet file. This test verifies that case. + table = pa.Table.from_pydict({"data": [0], "year": [2024]}) + os.makedirs(tmp_path / "year=2024") + pq.write_table(table, tmp_path / "year=2024" / "data.parquet") + + ds = ray.data.read_parquet(tmp_path) + + assert ds.take_all() == [{"data": 0, "year": 2024}] + + def test_read_null_data_in_first_file(tmp_path, ray_start_regular_shared): # The `read_parquet` implementation might infer the schema from the first file. # This test ensures that implementation handles the case where the first file has no diff --git a/python/ray/data/tests/test_sort.py b/python/ray/data/tests/test_sort.py index 78a2838d4deb7..daf86ee2428e9 100644 --- a/python/ray/data/tests/test_sort.py +++ b/python/ray/data/tests/test_sort.py @@ -14,7 +14,7 @@ ) from ray.data._internal.planner.exchange.sort_task_spec import SortKey, SortTaskSpec from ray.data.block import BlockAccessor -from ray.data.context import DataContext +from ray.data.context import DataContext, ShuffleStrategy from ray.data.tests.conftest import * # noqa from ray.data.tests.util import extract_values from ray.tests.conftest import * # noqa @@ -71,7 +71,7 @@ def test_sort_multiple_keys_produces_equally_sized_blocks(ray_start_regular): ), num_rows_per_block -def test_sort_simple(ray_start_regular, use_push_based_shuffle): +def test_sort_simple(ray_start_regular, configure_shuffle_method): num_items = 100 parallelism = 4 xs = list(range(num_items)) @@ -97,7 +97,7 @@ def test_sort_simple(ray_start_regular, use_push_based_shuffle): def test_sort_partition_same_key_to_same_block( - ray_start_regular, use_push_based_shuffle + ray_start_regular, configure_shuffle_method ): num_items = 100 xs = [1] * num_items @@ -115,9 +115,12 @@ def test_sort_partition_same_key_to_same_block( @pytest.mark.parametrize("num_items,parallelism", [(100, 1), (1000, 4)]) -@pytest.mark.parametrize("use_polars", [False, True]) def test_sort_arrow( - ray_start_regular, num_items, parallelism, use_push_based_shuffle, use_polars + ray_start_regular, + num_items, + parallelism, + configure_shuffle_method, + use_polars, ): ctx = ray.data.context.DataContext.get_current() @@ -159,9 +162,29 @@ def assert_sorted(sorted_ds, expected_rows): ctx.use_polars = original_use_polars -@pytest.mark.parametrize("use_polars", [False, True]) +def test_sort(ray_start_regular, use_polars): + import random + + import pyarrow as pa + + N = 100 + r = random.Random(0xDEED) + + ints = [r.randint(0, 10) for _ in range(N)] + floats = [r.normalvariate(0, 5) for _ in range(N)] + t = pa.Table.from_pydict({"ints": ints, "floats": floats}) + + sorted_block = BlockAccessor.for_block(t).sort(SortKey(["ints", "floats"])) + + sorted_tuples = list(zip(*sorted(zip(ints, floats)))) + + assert sorted_block == pa.Table.from_pydict( + {"ints": sorted_tuples[0], "floats": sorted_tuples[1]} + ) + + def test_sort_arrow_with_empty_blocks( - ray_start_regular, use_push_based_shuffle, use_polars + ray_start_regular, configure_shuffle_method, use_polars ): ctx = ray.data.context.DataContext.get_current() @@ -246,7 +269,9 @@ def test_sort_with_multiple_keys(ray_start_regular, descending, batch_format): @pytest.mark.parametrize("num_items,parallelism", [(100, 1), (1000, 4)]) -def test_sort_pandas(ray_start_regular, num_items, parallelism, use_push_based_shuffle): +def test_sort_pandas( + ray_start_regular, num_items, parallelism, configure_shuffle_method +): a = list(reversed(range(num_items))) b = [f"{x:03}" for x in range(num_items)] shard = int(np.ceil(num_items / parallelism)) @@ -275,7 +300,7 @@ def assert_sorted(sorted_ds, expected_rows): assert_sorted(ds.sort(key="a", descending=True), zip(a, b)) -def test_sort_pandas_with_empty_blocks(ray_start_regular, use_push_based_shuffle): +def test_sort_pandas_with_empty_blocks(ray_start_regular, configure_shuffle_method): assert ( BlockAccessor.for_block(pa.Table.from_pydict({})) .sample(10, SortKey("A")) @@ -316,7 +341,7 @@ def test_sort_pandas_with_empty_blocks(ray_start_regular, use_push_based_shuffle assert ds.sort("id").count() == 0 -def test_sort_with_one_block(shutdown_only, use_push_based_shuffle): +def test_sort_with_one_block(shutdown_only, configure_shuffle_method): ray.init(num_cpus=8) ctx = ray.data.DataContext.get_current() ctx.execution_options.verbose_progress = True @@ -485,7 +510,7 @@ def test_push_based_shuffle_stats(ray_start_cluster): ctx.use_push_based_shuffle = original -def test_sort_multinode(ray_start_cluster, use_push_based_shuffle): +def test_sort_multinode(ray_start_cluster, configure_shuffle_method): cluster = ray_start_cluster cluster.add_node( resources={"bar:1": 100}, @@ -635,15 +660,13 @@ def check_pipelined(refs): ] -@pytest.mark.parametrize("use_push_based_shuffle", [False, True]) @pytest.mark.parametrize( "shuffle_op", SHUFFLE_ALL_TO_ALL_OPS, ) def test_debug_limit_shuffle_execution_to_num_blocks( - ray_start_regular, restore_data_context, use_push_based_shuffle, shuffle_op + ray_start_regular, restore_data_context, configure_shuffle_method, shuffle_op ): - DataContext.get_current().use_push_based_shuffle = use_push_based_shuffle shuffle_fn = shuffle_op parallelism = 100 @@ -660,10 +683,9 @@ def test_debug_limit_shuffle_execution_to_num_blocks( assert shuffled_ds._plan.initial_num_blocks() == 1 -@pytest.mark.parametrize("use_push_based_shuffle", [False, True]) -def test_memory_usage(ray_start_regular, restore_data_context, use_push_based_shuffle): - DataContext.get_current().use_push_based_shuffle = use_push_based_shuffle - +def test_memory_usage( + ray_start_regular, restore_data_context, configure_shuffle_method +): parallelism = 2 ds = ray.data.range(int(1e8), override_num_blocks=parallelism) ds = ds.random_shuffle().materialize() @@ -675,12 +697,11 @@ def test_memory_usage(ray_start_regular, restore_data_context, use_push_based_sh assert op_stats.memory["max"] < 2000 -@pytest.mark.parametrize("use_push_based_shuffle", [False, True]) @pytest.mark.parametrize("under_threshold", [False, True]) def test_sort_object_ref_warnings( ray_start_regular, restore_data_context, - use_push_based_shuffle, + configure_shuffle_method, under_threshold, propagate_logs, caplog, @@ -691,10 +712,9 @@ def test_sort_object_ref_warnings( warning_str = "Execution is estimated to use" warning_str_with_bytes = ( "Execution is estimated to use at least " - f"{90 if use_push_based_shuffle else 300}KB" + f"{90 if configure_shuffle_method == ShuffleStrategy.SORT_SHUFFLE_PUSH_BASED else 300}KB" ) - DataContext.get_current().use_push_based_shuffle = use_push_based_shuffle if not under_threshold: DataContext.get_current().warn_on_driver_memory_usage_bytes = 10_000 @@ -710,12 +730,11 @@ def test_sort_object_ref_warnings( assert warning_str_with_bytes in caplog.text -@pytest.mark.parametrize("use_push_based_shuffle", [False, True]) @pytest.mark.parametrize("under_threshold", [False, True]) def test_sort_inlined_objects_warnings( ray_start_regular, restore_data_context, - use_push_based_shuffle, + configure_shuffle_method, under_threshold, propagate_logs, caplog, @@ -723,7 +742,7 @@ def test_sort_inlined_objects_warnings( # Test that we warn iff expected driver memory usage from # storing tiny Ray objects on driver heap is higher than # the configured threshold. - if use_push_based_shuffle: + if configure_shuffle_method == ShuffleStrategy.SORT_SHUFFLE_PUSH_BASED: warning_strs = [ "More than 3MB of driver memory used", "More than 7MB of driver memory used", @@ -733,7 +752,6 @@ def test_sort_inlined_objects_warnings( "More than 8MB of driver memory used", ] - DataContext.get_current().use_push_based_shuffle = use_push_based_shuffle if not under_threshold: DataContext.get_current().warn_on_driver_memory_usage_bytes = 3_000_000 diff --git a/python/ray/data/tests/test_zip.py b/python/ray/data/tests/test_zip.py index 7692aa3936492..942c598c930f9 100644 --- a/python/ray/data/tests/test_zip.py +++ b/python/ray/data/tests/test_zip.py @@ -27,7 +27,7 @@ def test_zip(ray_start_regular_shared): @pytest.mark.parametrize( "num_blocks1,num_blocks2", - list(itertools.combinations_with_replacement(range(1, 12), 2)), + list(itertools.combinations_with_replacement([1, 2, 4, 16], 2)), ) def test_zip_different_num_blocks_combinations( ray_start_regular_shared, num_blocks1, num_blocks2 diff --git a/python/ray/runtime_env/runtime_env.py b/python/ray/runtime_env/runtime_env.py index d04247b75a777..29237403eda6c 100644 --- a/python/ray/runtime_env/runtime_env.py +++ b/python/ray/runtime_env/runtime_env.py @@ -279,6 +279,7 @@ class MyClass: known_fields: Set[str] = { "py_modules", + "py_executable", "java_jars", "working_dir", "conda", @@ -312,6 +313,7 @@ def __init__( self, *, py_modules: Optional[List[str]] = None, + py_executable: Optional[str] = None, working_dir: Optional[str] = None, pip: Optional[List[str]] = None, conda: Optional[Union[Dict[str, str], str]] = None, @@ -331,6 +333,8 @@ def __init__( runtime_env = kwargs if py_modules is not None: runtime_env["py_modules"] = py_modules + if py_executable is not None: + runtime_env["py_executable"] = py_executable if working_dir is not None: runtime_env["working_dir"] = working_dir if pip is not None: @@ -515,6 +519,9 @@ def py_modules(self) -> List[str]: return list(self["py_modules"]) return [] + def py_executable(self) -> Optional[str]: + return self.get("py_executable", None) + def java_jars(self) -> List[str]: if "java_jars" in self: return list(self["java_jars"]) diff --git a/python/ray/tests/BUILD b/python/ray/tests/BUILD index 11c0969ad1f27..c99dd894c602b 100644 --- a/python/ray/tests/BUILD +++ b/python/ray/tests/BUILD @@ -187,6 +187,7 @@ py_test_module_list( "test_runtime_env_plugin.py", "test_runtime_env_setup_func.py", "test_runtime_env_strong_type.py", + "test_runtime_env_py_executable.py", "test_runtime_env_fork_process.py", "test_serialization.py", "test_shuffle.py", @@ -573,6 +574,7 @@ py_test_module_list( "test_runtime_env_conda_and_pip_4.py", "test_runtime_env_conda_and_pip_5.py", "test_runtime_env_uv.py", + "test_runtime_env_uv_run.py", ], size = "large", tags = ["exclusive", "post_wheel_build", "team:core"], diff --git a/python/ray/tests/conftest.py b/python/ray/tests/conftest.py index ad9d983b4f893..2a9a7fc7428c6 100644 --- a/python/ray/tests/conftest.py +++ b/python/ray/tests/conftest.py @@ -14,7 +14,7 @@ from contextlib import contextmanager from pathlib import Path from tempfile import gettempdir -from typing import List, Optional, Tuple +from typing import List, Optional from unittest import mock import psutil import pytest @@ -552,6 +552,13 @@ def ray_start_regular_shared(request): yield res +@pytest.fixture(scope="module") +def ray_start_regular_shared_2_cpus(request): + param = getattr(request, "param", {}) + with _ray_start(num_cpus=2, **param) as res: + yield res + + @pytest.fixture(scope="module", params=[{"local_mode": True}, {"local_mode": False}]) def ray_start_shared_local_modes(request): param = getattr(request, "param", {}) @@ -1198,7 +1205,6 @@ def append_short_test_summary(rep): # ":" is not legal in filenames in windows test_name = test_name.replace(":", "$") - header_file = os.path.join(summary_dir, "000_header.txt") summary_file = os.path.join(summary_dir, test_name + ".txt") if rep.passed and os.path.exists(summary_file): @@ -1219,13 +1225,6 @@ def append_short_test_summary(rep): if not hasattr(rep.longrepr, "chain"): return - if not os.path.exists(header_file): - with open(header_file, "wt") as fp: - test_label = os.environ.get("BUILDKITE_LABEL", "Unknown") - job_id = os.environ.get("BUILDKITE_JOB_ID") - - fp.write(f"### Pytest failures for: [{test_label}](#{job_id})\n\n") - # Use `wt` here to overwrite so we only have one result per test (exclude retries) with open(summary_file, "wt") as fp: fp.write(_get_markdown_annotation(rep)) @@ -1244,12 +1243,10 @@ def _get_markdown_annotation(rep) -> str: markdown += "
\n" markdown += f"{short_message}\n\n" - # Add link to test definition + # Add location to the test definition test_file, test_lineno, _test_node = rep.location - test_path, test_url = _get_repo_github_path_and_link( - os.path.abspath(test_file), test_lineno - ) - markdown += f"Link to test: [{test_path}:{test_lineno}]({test_url})\n\n" + test_path = os.path.abspath(test_file) + markdown += f"Test location: {test_path}:{test_lineno}\n\n" # Print main traceback markdown += "##### Traceback\n\n" @@ -1257,27 +1254,20 @@ def _get_markdown_annotation(rep) -> str: markdown += str(main_tb) markdown += "\n```\n\n" - # Print link to test definition in github - path, url = _get_repo_github_path_and_link(main_loc.path, main_loc.lineno) - markdown += f"[{path}:{main_loc.lineno}]({url})\n\n" + # Print test definition location + markdown += f"{main_loc.path}:{main_loc.lineno}\n\n" # If this is a longer exception chain, users can expand the full traceback if len(rep.longrepr.chain) > 1: markdown += "
Full traceback\n\n" - # Here we just print each traceback and the link to the respective - # lines in GutHub + # Here we just print each traceback and the respective lines. for tb, loc, _ in rep.longrepr.chain: - if loc: - path, url = _get_repo_github_path_and_link(loc.path, loc.lineno) - github_link = f"[{path}:{loc.lineno}]({url})\n\n" - else: - github_link = "" - markdown += "```\n" markdown += str(tb) markdown += "\n```\n\n" - markdown += github_link + if loc: + markdown += f"{loc.path}:{loc.lineno}\n\n" markdown += "
\n" @@ -1300,19 +1290,6 @@ def _get_pip_packages() -> List[str]: return ["invalid"] -def _get_repo_github_path_and_link(file: str, lineno: int) -> Tuple[str, str]: - base_url = "https://github.com/ray-project/ray/blob/{commit}/{path}#L{lineno}" - - commit = os.environ.get("BUILDKITE_COMMIT") - - if not commit: - return file, "" - - path = file.split("com_github_ray_project_ray/")[-1] - - return path, base_url.format(commit=commit, path=path, lineno=lineno) - - def create_ray_logs_for_failed_test(rep): """Creates artifact zip of /tmp/ray/session_latest/logs for failed tests""" diff --git a/python/ray/tests/test_runtime_env_py_executable.py b/python/ray/tests/test_runtime_env_py_executable.py new file mode 100644 index 0000000000000..9a470bbdde69d --- /dev/null +++ b/python/ray/tests/test_runtime_env_py_executable.py @@ -0,0 +1,55 @@ +import os +import pytest +import sys +import tempfile +from pathlib import Path + +import ray + + +@pytest.fixture(scope="function") +def tmp_working_dir(): + """A test fixture, which writes a test file.""" + with tempfile.TemporaryDirectory() as tmp_dir: + path = Path(tmp_dir) + + script_file = path / "start.sh" + with script_file.open(mode="w") as f: + f.write(sys.executable + " $@") + + yield tmp_dir + + +@pytest.mark.skipif(sys.platform == "win32", reason="Not ported to Windows yet.") +def test_simple_py_executable(shutdown_only): + runtime_env = { + "py_executable": "env RAY_TEST_PY_EXECUTABLE_ENV_EXAMPLE=1 " + sys.executable + } + ray.init(runtime_env=runtime_env) + + @ray.remote + def f(): + return os.environ["RAY_TEST_PY_EXECUTABLE_ENV_EXAMPLE"] + + assert ray.get(f.remote()) == "1" + + +@pytest.mark.skipif(sys.platform == "win32", reason="Not ported to Windows yet.") +def test_py_executable_with_working_dir(shutdown_only, tmp_working_dir): + tmp_dir = tmp_working_dir + + runtime_env = {"working_dir": tmp_dir, "py_executable": "bash start.sh"} + ray.init(runtime_env=runtime_env) + + @ray.remote + def f(): + return "hello" + + assert ray.get(f.remote()) == "hello" + + +if __name__ == "__main__": + if os.environ.get("PARALLEL_CI"): + sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) + else: + sys.exit(pytest.main(["-sv", __file__])) diff --git a/python/ray/tests/test_runtime_env_uv_run.py b/python/ray/tests/test_runtime_env_uv_run.py new file mode 100644 index 0000000000000..67f9ef65d6114 --- /dev/null +++ b/python/ray/tests/test_runtime_env_uv_run.py @@ -0,0 +1,150 @@ +# End-to-end tests for using "uv run" with py_executable + +import os +from pathlib import Path +import pytest +import subprocess +import sys +import tempfile + +import ray + + +@pytest.fixture(scope="function") +def with_uv(): + import platform + import stat + import tarfile + from urllib import request + + arch = "aarch64" if platform.machine() in ["aarch64", "arm64"] else "i686" + system = "unknown-linux-gnu" if platform.system() == "Linux" else "apple-darwin" + name = f"uv-{arch}-{system}" + url = f"https://github.com/astral-sh/uv/releases/download/0.5.27/{name}.tar.gz" + with tempfile.TemporaryDirectory() as tmp_dir: + with request.urlopen(request.Request(url), timeout=15.0) as response: + with tarfile.open(fileobj=response, mode="r|*") as tar: + tar.extractall(tmp_dir) + uv = Path(tmp_dir) / name / "uv" + uv.chmod(uv.stat().st_mode | stat.S_IEXEC) + yield uv + + +PYPROJECT_TOML = """ +[project] +name = "test" +version = "0.1" +dependencies = [ + "emoji", +] +requires-python = ">=3.9" +""" + + +@pytest.fixture(scope="function") +def tmp_working_dir(): + """A test fixture, which writes a pyproject.toml.""" + with tempfile.TemporaryDirectory() as tmp_dir: + path = Path(tmp_dir) + + script_file = path / "pyproject.toml" + with script_file.open(mode="w") as f: + f.write(PYPROJECT_TOML) + + yield tmp_dir + + +@pytest.mark.skipif(sys.platform == "win32", reason="Not ported to Windows yet.") +def test_uv_run_simple(shutdown_only, with_uv): + uv = with_uv + + runtime_env = { + "py_executable": f"{uv} run --with emoji --no-project", + } + ray.init(runtime_env=runtime_env) + + @ray.remote + def emojize(): + import emoji + + return emoji.emojize("Ray rocks :thumbs_up:") + + assert ray.get(emojize.remote()) == "Ray rocks 👍" + + +@pytest.mark.skipif(sys.platform == "win32", reason="Not ported to Windows yet.") +def test_uv_run_pyproject(shutdown_only, with_uv, tmp_working_dir): + uv = with_uv + tmp_dir = tmp_working_dir + + ray.init( + runtime_env={ + "working_dir": tmp_dir, + # We want to run in the system environment so the current installation of Ray can be found here + "py_executable": f"env PYTHONPATH={':'.join(sys.path)} {uv} run --python-preference=only-system", + } + ) + + @ray.remote + def emojize(): + import emoji + + return emoji.emojize("Ray rocks :thumbs_up:") + + assert ray.get(emojize.remote()) == "Ray rocks 👍" + + +@pytest.mark.skipif(sys.platform == "win32", reason="Not ported to Windows yet.") +def test_uv_run_editable(shutdown_only, with_uv, tmp_working_dir): + uv = with_uv + tmp_dir = tmp_working_dir + + subprocess.run( + ["git", "clone", "https://github.com/carpedm20/emoji/", "emoji_copy"], + cwd=tmp_dir, + ) + + subprocess.run( + ["git", "reset", "--hard", "08c5cc4789d924ad4215e2fb2ee8f0b19a0d421f"], + cwd=Path(tmp_dir) / "emoji_copy", + ) + + subprocess.run( + [uv, "add", "--editable", "./emoji_copy"], + cwd=tmp_dir, + ) + + # Now edit the package + content = "" + with open(Path(tmp_dir) / "emoji_copy" / "emoji" / "core.py") as f: + content = f.read() + + content = content.replace( + "return pattern.sub(replace, string)", 'return "The package was edited"' + ) + + with open(Path(tmp_dir) / "emoji_copy" / "emoji" / "core.py", "w") as f: + f.write(content) + + ray.init( + runtime_env={ + "working_dir": tmp_dir, + # We want to run in the system environment so the current installation of Ray can be found here + "py_executable": f"env PYTHONPATH={':'.join(sys.path)} {uv} run --python-preference=only-system", + } + ) + + @ray.remote + def emojize(): + import emoji + + return emoji.emojize("Ray rocks :thumbs_up:") + + assert ray.get(emojize.remote()) == "The package was edited" + + +if __name__ == "__main__": + if os.environ.get("PARALLEL_CI"): + sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) + else: + sys.exit(pytest.main(["-sv", __file__])) diff --git a/python/ray/train/_internal/storage.py b/python/ray/train/_internal/storage.py index 05970988862e3..35332963f46bb 100644 --- a/python/ray/train/_internal/storage.py +++ b/python/ray/train/_internal/storage.py @@ -529,7 +529,6 @@ def persist_current_checkpoint(self, checkpoint: "Checkpoint") -> "Checkpoint": Checkpoint: A Checkpoint pointing to the persisted checkpoint location. """ # TODO(justinvyu): Fix this cyclical import. - from ray.train._checkpoint import Checkpoint logger.debug( "Copying checkpoint files to storage path:\n" @@ -555,7 +554,7 @@ def persist_current_checkpoint(self, checkpoint: "Checkpoint") -> "Checkpoint": destination_filesystem=self.storage_filesystem, ) - persisted_checkpoint = Checkpoint( + persisted_checkpoint = checkpoint.__class__( filesystem=self.storage_filesystem, path=self.checkpoint_fs_path, ) diff --git a/python/ray/tune/trainable/trainable.py b/python/ray/tune/trainable/trainable.py index 0714e8b6778ae..6ed58cafb801e 100644 --- a/python/ray/tune/trainable/trainable.py +++ b/python/ray/tune/trainable/trainable.py @@ -15,7 +15,6 @@ import ray.cloudpickle as ray_pickle from ray.air._internal.util import exception_cause, skip_exceptions from ray.air.constants import TIME_THIS_ITER_S, TIMESTAMP, TRAINING_ITERATION -from ray.train import Checkpoint from ray.train._internal.checkpoint_manager import _TrainingResult from ray.train._internal.storage import StorageContext, _exists_at_fs_path from ray.train.constants import DEFAULT_STORAGE_PATH @@ -432,7 +431,7 @@ def _report_class_trainable_checkpoint( f"Got {checkpoint_dict_or_path} != {checkpoint_dir}" ) - local_checkpoint = Checkpoint.from_directory(checkpoint_dir) + local_checkpoint = ray.tune.Checkpoint.from_directory(checkpoint_dir) metrics = self._last_result.copy() if self._last_result else {} @@ -502,10 +501,16 @@ def save(self, checkpoint_dir: Optional[str] = None) -> _TrainingResult: # Update the checkpoint result to include auto-filled metrics. checkpoint_result.metrics.update(self._last_result) + print(f"checkpoint_result: {checkpoint_result}") + print( + f"type(checkpoint_result.checkpoint): {type(checkpoint_result.checkpoint)}" + ) return checkpoint_result @DeveloperAPI - def restore(self, checkpoint_path: Union[str, Checkpoint, _TrainingResult]): + def restore( + self, checkpoint_path: Union[str, "ray.tune.Checkpoint", _TrainingResult] + ): """Restores training state from a given model checkpoint. These checkpoints are returned from calls to save(). @@ -517,22 +522,15 @@ def restore(self, checkpoint_path: Union[str, Checkpoint, _TrainingResult]): `checkpoint_path` should match with the return from ``save()``. Args: - checkpoint_path: Path to restore checkpoint from. If this - path does not exist on the local node, it will be fetched - from external (cloud) storage if available, or restored - from a remote node. - checkpoint_node_ip: If given, try to restore - checkpoint from this node if it doesn't exist locally or - on cloud storage. - fallback_to_latest: If True, will try to recover the - latest available checkpoint if the given ``checkpoint_path`` - could not be found. - + checkpoint_path: training result that was returned by a + previous call to `save()`. """ - # TODO(justinvyu): Clean up this interface + # TODO(justinvyu): This also supports restoring from a Checkpoint object + # or a path, which are legacy APIs that RLlib depends on. + # RLlib should remove this dependency since `restore` is a DeveloperAPI. if isinstance(checkpoint_path, str): - checkpoint_path = Checkpoint.from_directory(checkpoint_path) - if isinstance(checkpoint_path, Checkpoint): + checkpoint_path = ray.tune.Checkpoint.from_directory(checkpoint_path) + if isinstance(checkpoint_path, ray.tune.Checkpoint): checkpoint_result = _TrainingResult(checkpoint=checkpoint_path, metrics={}) else: checkpoint_result: _TrainingResult = checkpoint_path diff --git a/rllib/algorithms/algorithm.py b/rllib/algorithms/algorithm.py index 9bebd2d7f712e..75bd576a2937c 100644 --- a/rllib/algorithms/algorithm.py +++ b/rllib/algorithms/algorithm.py @@ -34,10 +34,10 @@ import tree # pip install dm_tree import ray -from ray.air.constants import TRAINING_ITERATION +from ray.tune.result import TRAINING_ITERATION from ray._private.usage.usage_lib import TagKey, record_extra_usage_tag from ray.actor import ActorHandle -from ray.train import Checkpoint +from ray.tune import Checkpoint import ray.cloudpickle as pickle from ray.rllib.algorithms.algorithm_config import AlgorithmConfig from ray.rllib.algorithms.registry import ALGORITHMS_CLASS_TO_NAME as ALL_ALGORITHMS @@ -518,6 +518,7 @@ def __init__( timestr = datetime.today().strftime("%Y-%m-%d_%H-%M-%S") env_descr_for_dir = re.sub("[/\\\\]", "-", str(env_descr)) logdir_prefix = f"{type(self).__name__}_{env_descr_for_dir}_{timestr}" + if not os.path.exists(DEFAULT_STORAGE_PATH): # Possible race condition if dir is created several times on # rollout workers @@ -807,7 +808,7 @@ def setup(self, config: AlgorithmConfig) -> None: env_steps_sampled=self.metrics.peek( (ENV_RUNNER_RESULTS, NUM_ENV_STEPS_SAMPLED_LIFETIME), default=0 ), - rl_module_state=rl_module_state, + rl_module_state=rl_module_state[COMPONENT_RL_MODULE], ) elif self.eval_env_runner_group: self.eval_env_runner.set_state(rl_module_state) @@ -816,7 +817,7 @@ def setup(self, config: AlgorithmConfig) -> None: env_steps_sampled=self.metrics.peek( (ENV_RUNNER_RESULTS, NUM_ENV_STEPS_SAMPLED_LIFETIME), default=0 ), - rl_module_state=rl_module_state, + rl_module_state=rl_module_state[COMPONENT_RL_MODULE], ) # TODO (simon): Update modules in DataWorkers. @@ -2751,7 +2752,7 @@ def default_resource_request( driver = { # Sampling and training is not done concurrently when local is # used, so pick the max. - "CPU": max(num_cpus_per_learner, cf.num_cpus_for_main_process), + "CPU": (max(num_cpus_per_learner, cf.num_cpus_for_main_process)), "GPU": cf.num_gpus_per_learner, } # Training is done on n remote Learners. @@ -2810,8 +2811,14 @@ def default_resource_request( # resources for remote learner workers learner_bundles = [] - if cf.enable_rl_module_and_learner and cf.num_learners > 0: - learner_bundles = cls._get_learner_bundles(cf) + if cf.enable_rl_module_and_learner: + if cf.num_learners > 0: + learner_bundles = cls._get_learner_bundles(cf) + # Aggregation actors (for the local learner). + else: + learner_bundles = [ + {"CPU": 1} for _ in range(cf.num_aggregator_actors_per_learner) + ] bundles = [driver] + rollout_bundles + evaluation_bundles + learner_bundles diff --git a/rllib/algorithms/appo/appo.py b/rllib/algorithms/appo/appo.py index 68267b876637a..a59636df752d7 100644 --- a/rllib/algorithms/appo/appo.py +++ b/rllib/algorithms/appo/appo.py @@ -57,7 +57,6 @@ class APPOConfig(IMPALAConfig): .. testcode:: from ray.rllib.algorithms.appo import APPOConfig - from ray import air from ray import tune config = APPOConfig() @@ -68,7 +67,7 @@ class APPOConfig(IMPALAConfig): # Use to_dict() to get the old-style python config dict when running with tune. tune.Tuner( "APPO", - run_config=air.RunConfig( + run_config=tune.RunConfig( stop={"training_iteration": 1}, verbose=0, ), diff --git a/rllib/algorithms/appo/default_appo_rl_module.py b/rllib/algorithms/appo/default_appo_rl_module.py index ec9bfa5ab0a36..dc0ee394da03d 100644 --- a/rllib/algorithms/appo/default_appo_rl_module.py +++ b/rllib/algorithms/appo/default_appo_rl_module.py @@ -50,8 +50,7 @@ def forward_target(self, batch: Dict[str, Any]) -> Dict[str, Any]: @OverrideToImplementCustomLogic_CallToSuperRecommended @override(DefaultPPORLModule) def get_non_inference_attributes(self) -> List[str]: - # Get the NON inference-only attributes from the parent class - # `PPOTorchRLModule`. + # Get the NON inference-only attributes from the parent class. ret = super().get_non_inference_attributes() # Add the two (APPO) target networks to it (NOT needed in # inference-only mode). diff --git a/rllib/algorithms/cql/torch/cql_torch_learner.py b/rllib/algorithms/cql/torch/cql_torch_learner.py index 4d74e2f22c736..3fffd0f60fb7a 100644 --- a/rllib/algorithms/cql/torch/cql_torch_learner.py +++ b/rllib/algorithms/cql/torch/cql_torch_learner.py @@ -1,6 +1,6 @@ from typing import Dict -from ray.air.constants import TRAINING_ITERATION +from ray.tune.result import TRAINING_ITERATION from ray.rllib.algorithms.sac.sac_learner import ( LOGPS_KEY, QF_LOSS_KEY, diff --git a/rllib/algorithms/dqn/dqn.py b/rllib/algorithms/dqn/dqn.py index b328b664e0d32..25979093cce4b 100644 --- a/rllib/algorithms/dqn/dqn.py +++ b/rllib/algorithms/dqn/dqn.py @@ -95,7 +95,6 @@ class DQNConfig(AlgorithmConfig): .. testcode:: from ray.rllib.algorithms.dqn.dqn import DQNConfig - from ray import air from ray import tune config = ( @@ -107,7 +106,7 @@ class DQNConfig(AlgorithmConfig): ) tune.Tuner( "DQN", - run_config=air.RunConfig(stop={"training_iteration":1}), + run_config=tune.RunConfig(stop={"training_iteration":1}), param_space=config, ).fit() diff --git a/rllib/algorithms/impala/impala.py b/rllib/algorithms/impala/impala.py index fb55d154af3c9..021860ae9390d 100644 --- a/rllib/algorithms/impala/impala.py +++ b/rllib/algorithms/impala/impala.py @@ -89,7 +89,6 @@ class IMPALAConfig(AlgorithmConfig): .. testcode:: from ray.rllib.algorithms.impala import IMPALAConfig - from ray import air from ray import tune config = ( @@ -103,7 +102,7 @@ class IMPALAConfig(AlgorithmConfig): tune.Tuner( "IMPALA", param_space=config, - run_config=air.RunConfig(stop={"training_iteration": 1}), + run_config=tune.RunConfig(stop={"training_iteration": 1}), ).fit() """ @@ -672,8 +671,10 @@ def _func(actor, p): self.metrics.log_value( (AGGREGATOR_ACTOR_RESULTS, "num_env_steps_aggregated_lifetime"), self.config.train_batch_size_per_learner + * (self.config.num_learners or 1) * len(data_packages_for_learner_group), reduce="sum", + with_throughput=True, ) else: @@ -733,7 +734,7 @@ def _func(actor, p): shuffle_batch_per_epoch=self.config.shuffle_batch_per_epoch, ) # TODO (sven): Rename this metric into a more fitting name: ex. - # `NUM_LEARNER_UPDATED_SINCE_LAST_WEIGHTS_SYNC` + # `NUM_LEARNER_UPDATED_SINCE_LAST_WEIGHTS_SYNC`. self.metrics.log_value( NUM_TRAINING_STEP_CALLS_SINCE_LAST_SYNCH_WORKER_WEIGHTS, 1, @@ -866,8 +867,10 @@ def _pre_queue_batch_refs( # `batch_refs` is a list of tuple(aggregator_actor_id, ObjRef[MABatch]). # Each ObjRef[MABatch] was returned by one AggregatorActor from a single - # `get_batch()` call and the underlying MABatch is already located on a - # particular GPU (matching one particular Learner). + # `get_batch()` call. + # TODO (sven): Add this comment, once valid: + # .. and the underlying MABatch is already located on a particular GPU + # (matching one particular Learner). for agg_actor_id, ma_batch_ref in batch_refs: learner_actor_id = self._aggregator_actor_to_learner[agg_actor_id] self._ma_batches_being_built[learner_actor_id].append(ma_batch_ref) diff --git a/rllib/algorithms/impala/impala_learner.py b/rllib/algorithms/impala/impala_learner.py index c9c8d700241da..b9a6b541abc64 100644 --- a/rllib/algorithms/impala/impala_learner.py +++ b/rllib/algorithms/impala/impala_learner.py @@ -113,11 +113,25 @@ def update_from_batch( self.before_gradient_based_update(timesteps=timesteps or {}) - self._gpu_loader_in_queue.put(batch) - self.metrics.log_value( - (ALL_MODULES, QUEUE_SIZE_GPU_LOADER_QUEUE), - self._gpu_loader_in_queue.qsize(), - ) + if self.config.num_gpus_per_learner > 0: + self._gpu_loader_in_queue.put(batch) + self.metrics.log_value( + (ALL_MODULES, QUEUE_SIZE_GPU_LOADER_QUEUE), + self._gpu_loader_in_queue.qsize(), + ) + else: + if isinstance(self._learner_thread_in_queue, CircularBuffer): + ts_dropped = self._learner_thread_in_queue.add(batch) + self.metrics.log_value( + (ALL_MODULES, LEARNER_THREAD_ENV_STEPS_DROPPED), + ts_dropped, + reduce="sum", + ) + else: + # Enqueue to Learner thread's in-queue. + _LearnerThread.enqueue( + self._learner_thread_in_queue, batch, self.metrics + ) return self.metrics.reduce() diff --git a/rllib/algorithms/marwil/marwil.py b/rllib/algorithms/marwil/marwil.py index 2ff4801be81d6..79ba6af93e3cd 100644 --- a/rllib/algorithms/marwil/marwil.py +++ b/rllib/algorithms/marwil/marwil.py @@ -100,7 +100,7 @@ class MARWILConfig(AlgorithmConfig): from pathlib import Path from ray.rllib.algorithms.marwil import MARWILConfig - from ray import train, tune + from ray import tune # Get the base path (to ray/rllib) base_path = Path(__file__).parents[2] @@ -144,7 +144,7 @@ class MARWILConfig(AlgorithmConfig): tuner = tune.Tuner( "MARWIL", param_space=config, - run_config=train.RunConfig( + run_config=tune.RunConfig( stop={"training_iteration": 1}, ), ) diff --git a/rllib/algorithms/ppo/ppo.py b/rllib/algorithms/ppo/ppo.py index 864ad9a2d7dba..b811782ed1044 100644 --- a/rllib/algorithms/ppo/ppo.py +++ b/rllib/algorithms/ppo/ppo.py @@ -78,7 +78,6 @@ class PPOConfig(AlgorithmConfig): .. testcode:: from ray.rllib.algorithms.ppo import PPOConfig - from ray import air from ray import tune config = ( @@ -93,7 +92,7 @@ class PPOConfig(AlgorithmConfig): tune.Tuner( "PPO", - run_config=air.RunConfig(stop={"training_iteration": 1}), + run_config=tune.RunConfig(stop={"training_iteration": 1}), param_space=config, ).fit() diff --git a/rllib/algorithms/tests/test_callbacks_on_env_runner.py b/rllib/algorithms/tests/test_callbacks_on_env_runner.py index b87701e858c49..87a9ed21d6c52 100644 --- a/rllib/algorithms/tests/test_callbacks_on_env_runner.py +++ b/rllib/algorithms/tests/test_callbacks_on_env_runner.py @@ -4,7 +4,7 @@ import gymnasium as gym import ray -from ray import train, tune +from ray import tune from ray.rllib.callbacks.callbacks import RLlibCallback from ray.rllib.algorithms.ppo import PPOConfig from ray.rllib.env.env_runner import EnvRunner @@ -198,7 +198,7 @@ def test_tune_trial_id_visible_in_callbacks(self): tune.Tuner( trainable=config.algo_class, param_space=config, - run_config=train.RunConfig(stop={"training_iteration": 1}), + run_config=tune.RunConfig(stop={"training_iteration": 1}), ).fit() diff --git a/rllib/benchmarks/torch_compile/run_ppo_with_inference_bm.py b/rllib/benchmarks/torch_compile/run_ppo_with_inference_bm.py index 743b0ee703134..0711c5df8e944 100644 --- a/rllib/benchmarks/torch_compile/run_ppo_with_inference_bm.py +++ b/rllib/benchmarks/torch_compile/run_ppo_with_inference_bm.py @@ -2,11 +2,11 @@ import gymnasium as gym -from ray import tune, air -from ray.air.constants import TRAINING_ITERATION +from ray import tune from ray.rllib.algorithms.ppo import PPOConfig from ray.rllib.env.wrappers.atari_wrappers import wrap_atari_for_new_api_stack from ray.tune.registry import register_env +from ray.tune.result import TRAINING_ITERATION # Note: # To run this benchmark you need to have a ray cluster of at least @@ -84,7 +84,7 @@ def _env_creator(cfg): tuner = tune.Tuner( "PPO", - run_config=air.RunConfig( + run_config=tune.RunConfig( stop={TRAINING_ITERATION: 1 if pargs.smoke_test else pargs.num_iters}, ), param_space=config, diff --git a/rllib/core/learner/learner_group.py b/rllib/core/learner/learner_group.py index 0826a8e3eced1..f3b73c333ca3f 100644 --- a/rllib/core/learner/learner_group.py +++ b/rllib/core/learner/learner_group.py @@ -385,15 +385,17 @@ def _learner_update( **_kwargs, ) if _return_state and result: - result["_rl_module_state_after_update"] = _learner.get_state( - # Only return the state of those RLModules that actually returned - # results and thus got probably updated. - components=[ - COMPONENT_RL_MODULE + "/" + mid - for mid in result - if mid != ALL_MODULES - ], - inference_only=True, + result["_rl_module_state_after_update"] = ray.put( + _learner.get_state( + # Only return the state of those RLModules that actually returned + # results and thus got probably updated. + components=[ + COMPONENT_RL_MODULE + "/" + mid + for mid in result + if mid != ALL_MODULES + ], + inference_only=True, + )[COMPONENT_RL_MODULE] ) return result diff --git a/rllib/env/env_runner_group.py b/rllib/env/env_runner_group.py index 8e1e53e534dfd..64439ad6a899c 100644 --- a/rllib/env/env_runner_group.py +++ b/rllib/env/env_runner_group.py @@ -414,7 +414,11 @@ def sync_env_runner_states( if env_steps_sampled is not None else {} ), - **(rl_module_state if rl_module_state is not None else {}), + **( + {COMPONENT_RL_MODULE: rl_module_state} + if rl_module_state is not None + else {} + ), } ) return @@ -496,7 +500,7 @@ def sync_env_runner_states( # Update the rl_module component of the EnvRunner states, if necessary: if rl_module_state: - env_runner_states.update(rl_module_state) + env_runner_states.update({COMPONENT_RL_MODULE: rl_module_state}) # If we do NOT want remote EnvRunners to get their Connector states updated, # only update the local worker here (with all state components) and then remove diff --git a/rllib/env/multi_agent_env_runner.py b/rllib/env/multi_agent_env_runner.py index 9a2c8d32ec46f..4650b98c7fe70 100644 --- a/rllib/env/multi_agent_env_runner.py +++ b/rllib/env/multi_agent_env_runner.py @@ -6,6 +6,7 @@ import gymnasium as gym +import ray from ray.rllib.algorithms.algorithm_config import AlgorithmConfig from ray.rllib.callbacks.utils import make_callback from ray.rllib.core import ( @@ -780,7 +781,10 @@ def set_state(self, state: StateDict) -> None: # Only update the weigths, if this is the first synchronization or # if the weights of this `EnvRunner` lacks behind the actual ones. if weights_seq_no == 0 or self._weights_seq_no < weights_seq_no: - self.module.set_state(state[COMPONENT_RL_MODULE]) + rl_module_state = state[COMPONENT_RL_MODULE] + if isinstance(rl_module_state, ray.ObjectRef): + rl_module_state = ray.get(rl_module_state) + self.module.set_state(rl_module_state) # Update weights_seq_no, if the new one is > 0. if weights_seq_no > 0: diff --git a/rllib/env/single_agent_env_runner.py b/rllib/env/single_agent_env_runner.py index f2c8fc75f9d77..d2198c43c9aad 100644 --- a/rllib/env/single_agent_env_runner.py +++ b/rllib/env/single_agent_env_runner.py @@ -5,6 +5,7 @@ from typing import Collection, DefaultDict, List, Optional, Union import gymnasium as gym +import ray from gymnasium.wrappers.vector import DictInfoToList from ray.rllib.algorithms.algorithm_config import AlgorithmConfig @@ -543,6 +544,8 @@ def set_state(self, state: StateDict) -> None: # if the weights of this `EnvRunner` lacks behind the actual ones. if weights_seq_no == 0 or self._weights_seq_no < weights_seq_no: rl_module_state = state[COMPONENT_RL_MODULE] + if isinstance(rl_module_state, ray.ObjectRef): + rl_module_state = ray.get(rl_module_state) if ( isinstance(rl_module_state, dict) and DEFAULT_MODULE_ID in rl_module_state diff --git a/rllib/examples/_old_api_stack/connectors/self_play_with_policy_checkpoint.py b/rllib/examples/_old_api_stack/connectors/self_play_with_policy_checkpoint.py index 46cdd3285c26d..9e12ce5f8f9ac 100644 --- a/rllib/examples/_old_api_stack/connectors/self_play_with_policy_checkpoint.py +++ b/rllib/examples/_old_api_stack/connectors/self_play_with_policy_checkpoint.py @@ -10,8 +10,8 @@ import tempfile import ray -from ray import air, tune -from ray.air.constants import TRAINING_ITERATION +from ray import tune +from ray.tune.result import TRAINING_ITERATION from ray.rllib.callbacks.callbacks import RLlibCallback from ray.rllib.algorithms.sac import SACConfig from ray.rllib.env.utils import try_import_pyspiel @@ -111,9 +111,9 @@ def main(checkpoint_dir): tuner = tune.Tuner( "SAC", param_space=config.to_dict(), - run_config=air.RunConfig( + run_config=tune.RunConfig( stop=stop, - checkpoint_config=air.CheckpointConfig( + checkpoint_config=tune.CheckpointConfig( checkpoint_at_end=True, checkpoint_frequency=10, ), diff --git a/rllib/examples/_old_api_stack/parametric_actions_cartpole.py b/rllib/examples/_old_api_stack/parametric_actions_cartpole.py index e5df2f821c49a..22bde41d1b95c 100644 --- a/rllib/examples/_old_api_stack/parametric_actions_cartpole.py +++ b/rllib/examples/_old_api_stack/parametric_actions_cartpole.py @@ -19,8 +19,8 @@ import os import ray -from ray import air, tune -from ray.air.constants import TRAINING_ITERATION +from ray import tune +from ray.tune.result import TRAINING_ITERATION from ray.rllib.examples.envs.classes.parametric_actions_cartpole import ( ParametricActionsCartPole, ) @@ -111,7 +111,7 @@ results = tune.Tuner( args.run, - run_config=air.RunConfig(stop=stop, verbose=1), + run_config=tune.RunConfig(stop=stop, verbose=1), param_space=config, ).fit() diff --git a/rllib/examples/_old_api_stack/parametric_actions_cartpole_embeddings_learnt_by_model.py b/rllib/examples/_old_api_stack/parametric_actions_cartpole_embeddings_learnt_by_model.py index 476e8b81eece6..9a7ae5060df15 100644 --- a/rllib/examples/_old_api_stack/parametric_actions_cartpole_embeddings_learnt_by_model.py +++ b/rllib/examples/_old_api_stack/parametric_actions_cartpole_embeddings_learnt_by_model.py @@ -19,8 +19,8 @@ import os import ray -from ray import air, tune -from ray.air.constants import TRAINING_ITERATION +from ray import tune +from ray.tune.result import TRAINING_ITERATION from ray.rllib.examples.envs.classes.parametric_actions_cartpole import ( ParametricActionsCartPoleNoEmbeddings, ) @@ -97,7 +97,7 @@ results = tune.Tuner( args.run, - run_config=air.RunConfig(stop=stop, verbose=2), + run_config=tune.RunConfig(stop=stop, verbose=2), param_space=config, ).fit() diff --git a/rllib/examples/centralized_critic.py b/rllib/examples/centralized_critic.py index 14380b7899084..39d70d19d2088 100644 --- a/rllib/examples/centralized_critic.py +++ b/rllib/examples/centralized_critic.py @@ -26,8 +26,8 @@ import os import ray -from ray import air, tune -from ray.air.constants import TRAINING_ITERATION +from ray import tune +from ray.tune.result import TRAINING_ITERATION from ray.rllib.algorithms.ppo.ppo import PPO, PPOConfig from ray.rllib.algorithms.ppo.ppo_tf_policy import ( PPOTF1Policy, @@ -311,7 +311,7 @@ def get_default_policy_class(cls, config): tuner = tune.Tuner( CentralizedCritic, param_space=config.to_dict(), - run_config=air.RunConfig(stop=stop, verbose=1), + run_config=tune.RunConfig(stop=stop, verbose=1), ) results = tuner.fit() diff --git a/rllib/examples/checkpoints/continue_training_from_checkpoint.py b/rllib/examples/checkpoints/continue_training_from_checkpoint.py index 449489340de0d..567468fc8df4c 100644 --- a/rllib/examples/checkpoints/continue_training_from_checkpoint.py +++ b/rllib/examples/checkpoints/continue_training_from_checkpoint.py @@ -84,7 +84,7 @@ import re import time -from ray import train, tune +from ray import tune from ray.rllib.algorithms.algorithm_config import AlgorithmConfig from ray.rllib.callbacks.callbacks import RLlibCallback from ray.rllib.examples.envs.classes.multi_agent import MultiAgentCartPole @@ -200,9 +200,9 @@ def on_train_result(self, *, algorithm, metrics_logger, result, **kwargs): tuner = tune.Tuner( trainable=config.algo_class, param_space=config, - run_config=train.RunConfig( + run_config=tune.RunConfig( callbacks=tune_callbacks, - checkpoint_config=train.CheckpointConfig( + checkpoint_config=tune.CheckpointConfig( checkpoint_frequency=args.checkpoint_freq, checkpoint_at_end=args.checkpoint_at_end, ), diff --git a/rllib/examples/checkpoints/restore_1_of_n_agents_from_checkpoint.py b/rllib/examples/checkpoints/restore_1_of_n_agents_from_checkpoint.py index acbbb83118943..e531e23a98ee0 100644 --- a/rllib/examples/checkpoints/restore_1_of_n_agents_from_checkpoint.py +++ b/rllib/examples/checkpoints/restore_1_of_n_agents_from_checkpoint.py @@ -47,7 +47,7 @@ from pathlib import Path -from ray.air.constants import TRAINING_ITERATION +from ray.tune.result import TRAINING_ITERATION from ray.rllib.algorithms.callbacks import DefaultCallbacks from ray.rllib.core import ( COMPONENT_LEARNER, diff --git a/rllib/examples/curriculum/curriculum_learning.py b/rllib/examples/curriculum/curriculum_learning.py index d08cc35c224ff..c5efe06f73038 100644 --- a/rllib/examples/curriculum/curriculum_learning.py +++ b/rllib/examples/curriculum/curriculum_learning.py @@ -56,7 +56,7 @@ """ from functools import partial -from ray.air.constants import TRAINING_ITERATION +from ray.tune.result import TRAINING_ITERATION from ray.rllib.algorithms.algorithm import Algorithm from ray.rllib.callbacks.callbacks import RLlibCallback from ray.rllib.connectors.env_to_module import FlattenObservations diff --git a/rllib/examples/debugging/deterministic_training.py b/rllib/examples/debugging/deterministic_training.py index 9e7a8960c56e4..ac75c91c258b3 100644 --- a/rllib/examples/debugging/deterministic_training.py +++ b/rllib/examples/debugging/deterministic_training.py @@ -7,8 +7,7 @@ import argparse import ray -from ray import air, tune -from ray.air.constants import TRAINING_ITERATION +from ray import tune from ray.rllib.core import DEFAULT_MODULE_ID from ray.rllib.examples.envs.classes.env_using_remote_actor import ( CartPoleWithRemoteParamServer, @@ -18,6 +17,7 @@ from ray.rllib.utils.metrics.learner_info import LEARNER_INFO from ray.rllib.utils.test_utils import check from ray.tune.registry import get_trainable_cls +from ray.tune.result import TRAINING_ITERATION parser = argparse.ArgumentParser() parser.add_argument("--run", type=str, default="PPO") @@ -76,15 +76,15 @@ results1 = tune.Tuner( args.run, param_space=config.to_dict(), - run_config=air.RunConfig( - stop=stop, verbose=1, failure_config=air.FailureConfig(fail_fast="raise") + run_config=tune.RunConfig( + stop=stop, verbose=1, failure_config=tune.FailureConfig(fail_fast="raise") ), ).fit() results2 = tune.Tuner( args.run, param_space=config.to_dict(), - run_config=air.RunConfig( - stop=stop, verbose=1, failure_config=air.FailureConfig(fail_fast="raise") + run_config=tune.RunConfig( + stop=stop, verbose=1, failure_config=tune.FailureConfig(fail_fast="raise") ), ).fit() diff --git a/rllib/examples/envs/external_envs/cartpole_server.py b/rllib/examples/envs/external_envs/cartpole_server.py index 43f25c9a52885..65d86b14ef3ec 100755 --- a/rllib/examples/envs/external_envs/cartpole_server.py +++ b/rllib/examples/envs/external_envs/cartpole_server.py @@ -30,8 +30,7 @@ import os import ray -from ray import air, tune -from ray.air.constants import TRAINING_ITERATION +from ray import tune from ray.rllib.env.policy_server_input import PolicyServerInput from ray.rllib.utils.metrics import ( ENV_RUNNER_RESULTS, @@ -40,6 +39,7 @@ ) from ray.tune.logger import pretty_print from ray.tune.registry import get_trainable_cls +from ray.tune.result import TRAINING_ITERATION SERVER_ADDRESS = "localhost" # In this example, the user can run the policy server with @@ -272,5 +272,7 @@ def _input(ioctx): } tune.Tuner( - args.run, param_space=config, run_config=air.RunConfig(stop=stop, verbose=2) + args.run, + param_space=config, + run_config=tune.RunConfig(stop=stop, verbose=2), ).fit() diff --git a/rllib/examples/envs/greyscale_env.py b/rllib/examples/envs/greyscale_env.py index 2f0e5ffc95600..7df56f5477842 100644 --- a/rllib/examples/envs/greyscale_env.py +++ b/rllib/examples/envs/greyscale_env.py @@ -24,7 +24,7 @@ resize_v1, ) -from ray.air.constants import TRAINING_ITERATION +from ray.tune.result import TRAINING_ITERATION from ray.rllib.algorithms.ppo import PPOConfig from ray.rllib.env import PettingZooEnv from ray.rllib.utils.metrics import ( @@ -34,7 +34,6 @@ ) from ray.tune.registry import register_env from ray import tune -from ray import air parser = argparse.ArgumentParser() @@ -116,7 +115,7 @@ def env_creator(config): tune.Tuner( "PPO", param_space=config.to_dict(), - run_config=air.RunConfig( + run_config=tune.RunConfig( stop={ TRAINING_ITERATION: args.stop_iters, NUM_ENV_STEPS_SAMPLED_LIFETIME: args.stop_timesteps, diff --git a/rllib/examples/envs/unity3d_env_local.py b/rllib/examples/envs/unity3d_env_local.py index d334125ee4e81..f923c8beeed0a 100644 --- a/rllib/examples/envs/unity3d_env_local.py +++ b/rllib/examples/envs/unity3d_env_local.py @@ -27,8 +27,8 @@ import os import ray -from ray import air, tune -from ray.air.constants import TRAINING_ITERATION +from ray import tune +from ray.tune.result import TRAINING_ITERATION from ray.rllib.algorithms.ppo import PPOConfig from ray.rllib.env.wrappers.unity3d_env import Unity3DEnv from ray.rllib.utils.metrics import ( @@ -196,10 +196,10 @@ results = tune.Tuner( "PPO", param_space=config.to_dict(), - run_config=air.RunConfig( + run_config=tune.RunConfig( stop=stop, verbose=1, - checkpoint_config=air.CheckpointConfig( + checkpoint_config=tune.CheckpointConfig( checkpoint_frequency=5, checkpoint_at_end=True, ), diff --git a/rllib/examples/evaluation/custom_evaluation.py b/rllib/examples/evaluation/custom_evaluation.py index 9a2a796a08a37..26cf33fc00f01 100644 --- a/rllib/examples/evaluation/custom_evaluation.py +++ b/rllib/examples/evaluation/custom_evaluation.py @@ -65,7 +65,7 @@ """ from typing import Tuple -from ray.air.constants import TRAINING_ITERATION +from ray.tune.result import TRAINING_ITERATION from ray.rllib.algorithms.algorithm import Algorithm from ray.rllib.algorithms.algorithm_config import AlgorithmConfig from ray.rllib.env.env_runner_group import EnvRunnerGroup diff --git a/rllib/examples/evaluation/evaluation_parallel_to_training.py b/rllib/examples/evaluation/evaluation_parallel_to_training.py index 841496c2aca45..afc99df62c766 100644 --- a/rllib/examples/evaluation/evaluation_parallel_to_training.py +++ b/rllib/examples/evaluation/evaluation_parallel_to_training.py @@ -68,7 +68,7 @@ """ from typing import Optional -from ray.air.constants import TRAINING_ITERATION +from ray.tune.result import TRAINING_ITERATION from ray.rllib.algorithms.algorithm import Algorithm from ray.rllib.callbacks.callbacks import RLlibCallback from ray.rllib.examples.envs.classes.multi_agent import MultiAgentCartPole diff --git a/rllib/examples/inference/policy_inference_after_training_with_attention.py b/rllib/examples/inference/policy_inference_after_training_with_attention.py index 1e594066d18f5..b67c806cd1e2d 100644 --- a/rllib/examples/inference/policy_inference_after_training_with_attention.py +++ b/rllib/examples/inference/policy_inference_after_training_with_attention.py @@ -12,8 +12,7 @@ import os import ray -from ray import air, tune -from ray.air.constants import TRAINING_ITERATION +from ray import tune from ray.rllib.algorithms.algorithm import Algorithm from ray.rllib.utils.metrics import ( ENV_RUNNER_RESULTS, @@ -21,6 +20,7 @@ NUM_ENV_STEPS_SAMPLED_LIFETIME, ) from ray.tune.registry import get_trainable_cls +from ray.tune.result import TRAINING_ITERATION parser = argparse.ArgumentParser() parser.add_argument( @@ -116,10 +116,10 @@ tuner = tune.Tuner( args.run, param_space=config, - run_config=air.RunConfig( + run_config=tune.RunConfig( stop=stop, verbose=2, - checkpoint_config=air.CheckpointConfig( + checkpoint_config=tune.CheckpointConfig( checkpoint_frequency=1, checkpoint_at_end=True, ), diff --git a/rllib/examples/inference/policy_inference_after_training_with_lstm.py b/rllib/examples/inference/policy_inference_after_training_with_lstm.py index 39c6ac6aa5887..950e88ceee892 100644 --- a/rllib/examples/inference/policy_inference_after_training_with_lstm.py +++ b/rllib/examples/inference/policy_inference_after_training_with_lstm.py @@ -12,8 +12,7 @@ import os import ray -from ray import air, tune -from ray.air.constants import TRAINING_ITERATION +from ray import tune from ray.rllib.algorithms.algorithm import Algorithm from ray.rllib.utils.metrics import ( ENV_RUNNER_RESULTS, @@ -21,6 +20,7 @@ NUM_ENV_STEPS_SAMPLED_LIFETIME, ) from ray.tune.registry import get_trainable_cls +from ray.tune.result import TRAINING_ITERATION parser = argparse.ArgumentParser() parser.add_argument( @@ -111,10 +111,10 @@ tuner = tune.Tuner( args.run, param_space=config, - run_config=air.RunConfig( + run_config=tune.RunConfig( stop=stop, verbose=2, - checkpoint_config=air.CheckpointConfig( + checkpoint_config=tune.CheckpointConfig( checkpoint_frequency=1, checkpoint_at_end=True, ), diff --git a/rllib/examples/learners/ppo_load_rl_modules.py b/rllib/examples/learners/ppo_load_rl_modules.py index 61cfe730a4652..d07eb19eb9d46 100644 --- a/rllib/examples/learners/ppo_load_rl_modules.py +++ b/rllib/examples/learners/ppo_load_rl_modules.py @@ -4,8 +4,8 @@ import tempfile import ray -from ray import air, tune -from ray.air.constants import TRAINING_ITERATION +from ray import tune +from ray.tune.result import TRAINING_ITERATION from ray.rllib.algorithms.ppo import PPOConfig from ray.rllib.algorithms.ppo.ppo_catalog import PPOCatalog from ray.rllib.algorithms.ppo.tf.ppo_tf_rl_module import PPOTfRLModule @@ -69,9 +69,9 @@ def _parse_args(): tuner = tune.Tuner( "PPO", param_space=config.to_dict(), - run_config=air.RunConfig( + run_config=tune.RunConfig( stop={TRAINING_ITERATION: 1}, - failure_config=air.FailureConfig(fail_fast="raise"), + failure_config=tune.FailureConfig(fail_fast="raise"), ), ) tuner.fit() diff --git a/rllib/examples/multi_agent/rock_paper_scissors_heuristic_vs_learned.py b/rllib/examples/multi_agent/rock_paper_scissors_heuristic_vs_learned.py index 6f474e8e3c69a..5535b7f45a9c0 100644 --- a/rllib/examples/multi_agent/rock_paper_scissors_heuristic_vs_learned.py +++ b/rllib/examples/multi_agent/rock_paper_scissors_heuristic_vs_learned.py @@ -32,7 +32,7 @@ import gymnasium as gym from pettingzoo.classic import rps_v2 -from ray.air.constants import TRAINING_ITERATION +from ray.tune.result import TRAINING_ITERATION from ray.rllib.connectors.env_to_module import FlattenObservations from ray.rllib.core.rl_module.default_model_config import DefaultModelConfig from ray.rllib.core.rl_module.multi_rl_module import MultiRLModuleSpec diff --git a/rllib/examples/multi_agent/self_play_league_based_with_open_spiel.py b/rllib/examples/multi_agent/self_play_league_based_with_open_spiel.py index 859d4d9c01dda..51e53111514a4 100644 --- a/rllib/examples/multi_agent/self_play_league_based_with_open_spiel.py +++ b/rllib/examples/multi_agent/self_play_league_based_with_open_spiel.py @@ -35,7 +35,7 @@ import torch import ray -from ray.air.constants import TRAINING_ITERATION +from ray.tune.result import TRAINING_ITERATION from ray.rllib.core.rl_module.default_model_config import DefaultModelConfig from ray.rllib.core.rl_module.multi_rl_module import MultiRLModuleSpec from ray.rllib.core.rl_module.rl_module import RLModuleSpec diff --git a/rllib/examples/multi_agent/self_play_with_open_spiel.py b/rllib/examples/multi_agent/self_play_with_open_spiel.py index 37be03d53622c..2a368996910a2 100644 --- a/rllib/examples/multi_agent/self_play_with_open_spiel.py +++ b/rllib/examples/multi_agent/self_play_with_open_spiel.py @@ -23,7 +23,7 @@ import numpy as np import torch -from ray.air.constants import TRAINING_ITERATION +from ray.tune.result import TRAINING_ITERATION from ray.rllib.core.rl_module.default_model_config import DefaultModelConfig from ray.rllib.core.rl_module.multi_rl_module import MultiRLModuleSpec from ray.rllib.core.rl_module.rl_module import RLModuleSpec diff --git a/rllib/examples/offline_rl/custom_input_api.py b/rllib/examples/offline_rl/custom_input_api.py index 789e64a2a357e..6c6cd515abae5 100644 --- a/rllib/examples/offline_rl/custom_input_api.py +++ b/rllib/examples/offline_rl/custom_input_api.py @@ -16,8 +16,7 @@ import os import ray -from ray import air, tune -from ray.air.constants import TRAINING_ITERATION +from ray import tune from ray.rllib.offline import JsonReader, ShuffledInput, IOContext, InputReader from ray.rllib.utils.metrics import ( ENV_RUNNER_RESULTS, @@ -25,6 +24,7 @@ EVALUATION_RESULTS, ) from ray.tune.registry import get_trainable_cls, register_input +from ray.tune.result import TRAINING_ITERATION parser = argparse.ArgumentParser() parser.add_argument( @@ -129,6 +129,6 @@ def input_creator(ioctx: IOContext) -> InputReader: } tuner = tune.Tuner( - args.run, param_space=config, run_config=air.RunConfig(stop=stop, verbose=1) + args.run, param_space=config, run_config=tune.RunConfig(stop=stop, verbose=1) ) tuner.fit() diff --git a/rllib/examples/offline_rl/pretrain_bc_single_agent_evaluate_as_multi_agent.py b/rllib/examples/offline_rl/pretrain_bc_single_agent_evaluate_as_multi_agent.py index d965add335527..bf819d78f216e 100644 --- a/rllib/examples/offline_rl/pretrain_bc_single_agent_evaluate_as_multi_agent.py +++ b/rllib/examples/offline_rl/pretrain_bc_single_agent_evaluate_as_multi_agent.py @@ -56,7 +56,6 @@ import gymnasium as gym from ray import tune -from ray.air.constants import TRAINING_ITERATION from ray.rllib.algorithms.bc import BCConfig from ray.rllib.examples.envs.classes.multi_agent import MultiAgentCartPole from ray.rllib.examples._old_api_stack.policy.random_policy import RandomPolicy @@ -70,7 +69,7 @@ add_rllib_example_script_args, run_rllib_example_script_experiment, ) -from ray.train.constants import TIME_TOTAL_S +from ray.tune.result import TIME_TOTAL_S, TRAINING_ITERATION from ray.tune.registry import register_env parser = add_rllib_example_script_args( diff --git a/rllib/examples/ray_tune/custom_experiment.py b/rllib/examples/ray_tune/custom_experiment.py index 66ce75c11eb62..54628f3103aa1 100644 --- a/rllib/examples/ray_tune/custom_experiment.py +++ b/rllib/examples/ray_tune/custom_experiment.py @@ -41,7 +41,7 @@ from typing import Dict import numpy as np -from ray import train, tune +from ray import tune from ray.rllib.algorithms.ppo import PPOConfig from ray.rllib.utils.framework import try_import_torch from ray.rllib.utils.metrics import NUM_ENV_STEPS_SAMPLED_LIFETIME @@ -69,7 +69,7 @@ def my_experiment(config: Dict): train_results = algo_high_lr.train() # Add the phase to the result dict. train_results["phase"] = 1 - train.report(train_results) + tune.report(train_results) phase_high_lr_time = train_results[NUM_ENV_STEPS_SAMPLED_LIFETIME] checkpoint_training_high_lr = algo_high_lr.save() algo_high_lr.stop() @@ -85,7 +85,7 @@ def my_experiment(config: Dict): train_results["phase"] = 2 # keep time moving forward train_results[NUM_ENV_STEPS_SAMPLED_LIFETIME] += phase_high_lr_time - train.report(train_results) + tune.report(train_results) checkpoint_training_low_lr = algo_low_lr.save() algo_low_lr.stop() @@ -149,7 +149,7 @@ def my_experiment(config: Dict): # evaluation results. results = {**train_results, **eval_results} # Report everything. - train.report(results) + tune.report(results) if __name__ == "__main__": diff --git a/rllib/examples/ray_tune/custom_logger.py b/rllib/examples/ray_tune/custom_logger.py index 9823e47daaec5..12e09ba636afb 100644 --- a/rllib/examples/ray_tune/custom_logger.py +++ b/rllib/examples/ray_tune/custom_logger.py @@ -50,7 +50,7 @@ """ -from ray import air, tune +from ray import tune from ray.rllib.algorithms.ppo import PPOConfig from ray.rllib.core import DEFAULT_MODULE_ID from ray.rllib.utils.metrics import ( @@ -126,7 +126,7 @@ def flush(self): results = tune.Tuner( config.algo_class, param_space=config, - run_config=air.RunConfig( + run_config=tune.RunConfig( stop=stop, verbose=2, # Plugin our own logger. diff --git a/rllib/examples/ray_tune/custom_progress_reporter.py b/rllib/examples/ray_tune/custom_progress_reporter.py index 092b0710db574..d2f7c010aad1c 100644 --- a/rllib/examples/ray_tune/custom_progress_reporter.py +++ b/rllib/examples/ray_tune/custom_progress_reporter.py @@ -1,12 +1,12 @@ """Example showing how to set up a custom progress reporter for an RLlib Algorithm. -The script sets the `progress_reporter` arg in the air.RunConfig and passes that to +The script sets the `progress_reporter` arg in the tune.RunConfig and passes that to Tune's Tuner: ``` tune.Tuner( param_space=..., # <- your RLlib config - run_config=air.RunConfig( + run_config=tune.RunConfig( progress_reporter=[some already instantiated TuneReporterBase object], ), ) @@ -43,8 +43,8 @@ +-------+-------------------+------------------+------------------+------------------+ """ -from ray import air, tune -from ray.air.constants import TRAINING_ITERATION +from ray import tune +from ray.tune.result import TRAINING_ITERATION from ray.rllib.algorithms.ppo import PPOConfig from ray.rllib.examples.envs.classes.multi_agent import MultiAgentCartPole from ray.rllib.utils.metrics import ( @@ -110,7 +110,7 @@ results = tune.Tuner( config.algo_class, param_space=config, - run_config=air.RunConfig( + run_config=tune.RunConfig( stop=stop, verbose=2, # Plugin our own progress reporter. diff --git a/rllib/examples/replay_buffer_api.py b/rllib/examples/replay_buffer_api.py index 5d87a5ef5cd3b..0bdc6403d2290 100644 --- a/rllib/examples/replay_buffer_api.py +++ b/rllib/examples/replay_buffer_api.py @@ -12,8 +12,8 @@ import argparse import ray -from ray import air, tune -from ray.air.constants import TRAINING_ITERATION +from ray import tune +from ray.tune.result import TRAINING_ITERATION from ray.rllib.algorithms.dqn import DQNConfig from ray.rllib.utils.framework import try_import_tf from ray.rllib.utils.metrics import NUM_ENV_STEPS_SAMPLED_LIFETIME @@ -74,7 +74,7 @@ results = tune.Tuner( config.algo_class, param_space=config, - run_config=air.RunConfig(stop=stop_config), + run_config=tune.RunConfig(stop=stop_config), ).fit() ray.shutdown() diff --git a/rllib/policy/policy.py b/rllib/policy/policy.py index 7f14e7f875c99..9306856a26807 100644 --- a/rllib/policy/policy.py +++ b/rllib/policy/policy.py @@ -24,7 +24,6 @@ import ray import ray.cloudpickle as pickle from ray.actor import ActorHandle -from ray.train import Checkpoint from ray.rllib.models.action_dist import ActionDistribution from ray.rllib.models.catalog import ModelCatalog from ray.rllib.models.modelv2 import ModelV2 @@ -72,6 +71,7 @@ TensorStructType, TensorType, ) +from ray.tune import Checkpoint tf1, tf, tfv = try_import_tf() torch, _ = try_import_torch() diff --git a/rllib/tests/test_custom_resource.py b/rllib/tests/test_custom_resource.py index 16fe37238ce66..4a03052940bed 100644 --- a/rllib/tests/test_custom_resource.py +++ b/rllib/tests/test_custom_resource.py @@ -1,9 +1,9 @@ import pytest import ray -from ray import air, tune -from ray.air.constants import TRAINING_ITERATION +from ray import tune from ray.tune.registry import get_trainable_cls +from ray.tune.result import TRAINING_ITERATION @pytest.mark.parametrize("algorithm", ["PPO", "IMPALA"]) @@ -32,7 +32,7 @@ def test_custom_resource(algorithm): tune.Tuner( algorithm, param_space=config, - run_config=air.RunConfig(stop=stop, verbose=0), + run_config=tune.RunConfig(stop=stop, verbose=0), tune_config=tune.TuneConfig(num_samples=1), ).fit() diff --git a/rllib/tests/test_placement_groups.py b/rllib/tests/test_placement_groups.py index 2d268a9a4b10f..789e606f5eff8 100644 --- a/rllib/tests/test_placement_groups.py +++ b/rllib/tests/test_placement_groups.py @@ -2,12 +2,12 @@ import unittest import ray -from ray import air, tune -from ray.air.constants import TRAINING_ITERATION +from ray import tune from ray.rllib.algorithms.ppo import PPO, PPOConfig from ray.tune import Callback from ray.tune.experiment import Trial from ray.tune.execution.placement_groups import PlacementGroupFactory +from ray.tune.result import TRAINING_ITERATION trial_executor = None @@ -65,7 +65,7 @@ def default_resource_request(cls, config): tune.Tuner( "my_trainable", param_space=config, - run_config=air.RunConfig( + run_config=tune.RunConfig( stop={TRAINING_ITERATION: 2}, verbose=2, callbacks=[_TestCallback()], @@ -95,7 +95,7 @@ def test_default_resource_request(self): tune.Tuner( PPO, param_space=config, - run_config=air.RunConfig( + run_config=tune.RunConfig( stop={TRAINING_ITERATION: 2}, verbose=2, callbacks=[_TestCallback()], @@ -119,7 +119,7 @@ def test_default_resource_request_plus_manual_leads_to_error(self): tune.Tuner( tune.with_resources(PPO, PlacementGroupFactory([{"CPU": 1}])), param_space=config, - run_config=air.RunConfig(stop={TRAINING_ITERATION: 2}, verbose=2), + run_config=tune.RunConfig(stop={TRAINING_ITERATION: 2}, verbose=2), ).fit() except ValueError as e: assert "have been automatically set to" in e.args[0] diff --git a/rllib/tuned_examples/bc/cartpole_bc.py b/rllib/tuned_examples/bc/cartpole_bc.py index fbfd864b754ca..857857f0f9bd1 100644 --- a/rllib/tuned_examples/bc/cartpole_bc.py +++ b/rllib/tuned_examples/bc/cartpole_bc.py @@ -1,7 +1,7 @@ import warnings from pathlib import Path -from ray.air.constants import TRAINING_ITERATION +from ray.tune.result import TRAINING_ITERATION from ray.rllib.algorithms.bc import BCConfig from ray.rllib.core.rl_module.default_model_config import DefaultModelConfig from ray.rllib.utils.metrics import ( diff --git a/rllib/tuned_examples/bc/pendulum_bc.py b/rllib/tuned_examples/bc/pendulum_bc.py index 098a35e235b2b..c56403b79392d 100644 --- a/rllib/tuned_examples/bc/pendulum_bc.py +++ b/rllib/tuned_examples/bc/pendulum_bc.py @@ -1,6 +1,6 @@ from pathlib import Path -from ray.air.constants import TRAINING_ITERATION +from ray.tune.result import TRAINING_ITERATION from ray.rllib.algorithms.bc import BCConfig from ray.rllib.core.rl_module.default_model_config import DefaultModelConfig from ray.rllib.utils.metrics import ( diff --git a/rllib/tuned_examples/dqn/benchmark_dqn_atari.py b/rllib/tuned_examples/dqn/benchmark_dqn_atari.py index 4cbe1fa6e204c..d08ba3a741494 100644 --- a/rllib/tuned_examples/dqn/benchmark_dqn_atari.py +++ b/rllib/tuned_examples/dqn/benchmark_dqn_atari.py @@ -10,7 +10,7 @@ NUM_ENV_STEPS_SAMPLED_LIFETIME, ) from ray.tune import Stopper -from ray import train, tune +from ray import tune # Might need `gymnasium[atari, other]` to be installed. @@ -359,7 +359,7 @@ def stop_all(self): tuner = tune.Tuner( "DQN", param_space=config, - run_config=train.RunConfig( + run_config=tune.RunConfig( stop=BenchmarkStopper(benchmark_envs=benchmark_envs), name="benchmark_dqn_atari", ), diff --git a/rllib/tuned_examples/dqn/benchmark_dqn_atari_rllib_preprocessing.py b/rllib/tuned_examples/dqn/benchmark_dqn_atari_rllib_preprocessing.py index 2e8deb84d3541..763ad80069446 100644 --- a/rllib/tuned_examples/dqn/benchmark_dqn_atari_rllib_preprocessing.py +++ b/rllib/tuned_examples/dqn/benchmark_dqn_atari_rllib_preprocessing.py @@ -8,7 +8,7 @@ NUM_ENV_STEPS_SAMPLED_LIFETIME, ) from ray.tune import Stopper -from ray import train, tune +from ray import tune # Might need `gymnasium[atari, other]` to be installed. @@ -357,7 +357,7 @@ def stop_all(self): tuner = tune.Tuner( "DQN", param_space=config, - run_config=train.RunConfig( + run_config=tune.RunConfig( stop=BenchmarkStopper(benchmark_envs=benchmark_envs), name="benchmark_dqn_atari_rllib_preprocessing", ), diff --git a/rllib/tuned_examples/ppo/benchmark_ppo_mujoco.py b/rllib/tuned_examples/ppo/benchmark_ppo_mujoco.py index e29a99ebc155d..5bcc13616abed 100644 --- a/rllib/tuned_examples/ppo/benchmark_ppo_mujoco.py +++ b/rllib/tuned_examples/ppo/benchmark_ppo_mujoco.py @@ -5,7 +5,7 @@ NUM_ENV_STEPS_SAMPLED_LIFETIME, ) from ray.tune import Stopper -from ray import train, tune +from ray import tune # Needs the following packages to be installed on Ubuntu: # sudo apt-get libosmesa-dev @@ -129,7 +129,7 @@ def stop_all(self): tuner = tune.Tuner( "PPO", param_space=config, - run_config=train.RunConfig( + run_config=tune.RunConfig( stop=BenchmarkStopper(benchmark_envs=benchmark_envs), name="benchmark_ppo_mujoco", ), diff --git a/rllib/tuned_examples/ppo/benchmark_ppo_mujoco_pb2.py b/rllib/tuned_examples/ppo/benchmark_ppo_mujoco_pb2.py index c3a4c37c1437a..7aaf5cff2328b 100644 --- a/rllib/tuned_examples/ppo/benchmark_ppo_mujoco_pb2.py +++ b/rllib/tuned_examples/ppo/benchmark_ppo_mujoco_pb2.py @@ -2,7 +2,7 @@ from ray.rllib.algorithms.ppo.ppo import PPOConfig from ray.rllib.utils.metrics import NUM_ENV_STEPS_SAMPLED_LIFETIME from ray.tune.schedulers.pb2 import PB2 -from ray import train, tune +from ray import tune # Needs the following packages to be installed on Ubuntu: # sudo apt-get libosmesa-dev @@ -126,7 +126,7 @@ tuner = tune.Tuner( "PPO", param_space=config, - run_config=train.RunConfig( + run_config=tune.RunConfig( stop=stop_criteria, name="benchmark_ppo_mujoco_pb2_" + env, ), @@ -149,7 +149,7 @@ tuner = tune.Tuner( "PPO", param_space=best_result.config, - run_config=train.RunConfig( + run_config=tune.RunConfig( stop=stop_criteria, name="benchmark_ppo_mujoco_pb2_" + env + "_best", ), diff --git a/rllib/tuned_examples/sac/benchmark_sac_mujoco.py b/rllib/tuned_examples/sac/benchmark_sac_mujoco.py index 2ec59d9f3af8b..f454ffc9d359b 100644 --- a/rllib/tuned_examples/sac/benchmark_sac_mujoco.py +++ b/rllib/tuned_examples/sac/benchmark_sac_mujoco.py @@ -5,7 +5,7 @@ NUM_ENV_STEPS_SAMPLED_LIFETIME, ) from ray.tune import Stopper -from ray import train, tune +from ray import tune # Needs the following packages to be installed on Ubuntu: # sudo apt-get libosmesa-dev @@ -132,7 +132,7 @@ def stop_all(self): tuner = tune.Tuner( "SAC", param_space=config, - run_config=train.RunConfig( + run_config=tune.RunConfig( stop=BenchmarkStopper(benchmark_envs=benchmark_envs), name="benchmark_sac_mujoco", ), diff --git a/rllib/tuned_examples/sac/benchmark_sac_mujoco_pb2.py b/rllib/tuned_examples/sac/benchmark_sac_mujoco_pb2.py index 8ac4faf0b1888..b1dce97b972a4 100644 --- a/rllib/tuned_examples/sac/benchmark_sac_mujoco_pb2.py +++ b/rllib/tuned_examples/sac/benchmark_sac_mujoco_pb2.py @@ -6,7 +6,7 @@ EPISODE_RETURN_MEAN, ) from ray.tune.schedulers.pb2 import PB2 -from ray import train, tune +from ray import tune # Needs the following packages to be installed on Ubuntu: # sudo apt-get libosmesa-dev @@ -119,7 +119,7 @@ tuner = tune.Tuner( "SAC", param_space=config, - run_config=train.RunConfig( + run_config=tune.RunConfig( stop=stop_criteria, name="benchmark_sac_mujoco_pb2_" + env, ), @@ -142,7 +142,7 @@ tuner = tune.Tuner( "SAC", param_space=best_result.config, - run_config=train.RunConfig( + run_config=tune.RunConfig( stop=stop_criteria, name="benchmark_sac_mujoco_pb2_" + env + "_best", ), diff --git a/rllib/utils/actor_manager.py b/rllib/utils/actor_manager.py index 85500a4e483fa..b875e4e8cb388 100644 --- a/rllib/utils/actor_manager.py +++ b/rllib/utils/actor_manager.py @@ -678,33 +678,36 @@ def probe_unhealthy_actors( """ # Collect recently restored actors (from `self._fetch_result` calls other than # the one triggered here via the `ping`). - restored_actors = list(self._restored_actors) - self._restored_actors.clear() + already_restored_actors = list(self._restored_actors) - # Probe all unhealthy actors via a simple `ping()`. + # Which actors are currently marked unhealthy? unhealthy_actor_ids = [ actor_id for actor_id in self.actor_ids() if not self.is_actor_healthy(actor_id) ] - # No unhealthy actors currently -> Return recently restored ones. - if not unhealthy_actor_ids: - return restored_actors - # Some unhealthy actors -> `ping()` all of them to trigger a new fetch and - # capture all restored ones. - remote_results = self.foreach_actor( - func=lambda actor: actor.ping(), - remote_actor_ids=unhealthy_actor_ids, - healthy_only=False, # We specifically want to ping unhealthy actors. - timeout_seconds=timeout_seconds, - mark_healthy=mark_healthy, - ) + # gather the just restored ones (b/c of a successful `ping` response). + just_restored_actors = [] + if unhealthy_actor_ids: + remote_results = self.foreach_actor( + func=lambda actor: actor.ping(), + remote_actor_ids=unhealthy_actor_ids, + healthy_only=False, # We specifically want to ping unhealthy actors. + timeout_seconds=timeout_seconds, + return_obj_refs=False, + mark_healthy=mark_healthy, + ) + just_restored_actors = [ + result.actor_id for result in remote_results if result.ok + ] - # Return previously restored actors AND actors restored via the `ping()` call. - return restored_actors + [ - result.actor_id for result in remote_results if result.ok - ] + # Clear out previously restored actors (b/c of other successful request + # responses, outside of this method). + self._restored_actors.clear() + + # Return all restored actors (previously and just). + return already_restored_actors + just_restored_actors def _call_actors( self, diff --git a/rllib/utils/checkpoints.py b/rllib/utils/checkpoints.py index 1c8e9531fc34a..1e0d9ad8a9c4c 100644 --- a/rllib/utils/checkpoints.py +++ b/rllib/utils/checkpoints.py @@ -27,7 +27,7 @@ ) from ray.rllib.utils.serialization import NOT_SERIALIZABLE, serialize_type from ray.rllib.utils.typing import StateDict -from ray.train import Checkpoint +from ray.tune import Checkpoint from ray.tune.utils.file_transfer import sync_dir_between_nodes from ray.util import log_once from ray.util.annotations import PublicAPI diff --git a/rllib/utils/test_utils.py b/rllib/utils/test_utils.py index 6da24e42e2728..eb271636e8b1c 100644 --- a/rllib/utils/test_utils.py +++ b/rllib/utils/test_utils.py @@ -25,8 +25,7 @@ import tree # pip install dm_tree import ray -from ray import train, tune -from ray.air.constants import TRAINING_ITERATION +from ray import tune from ray.air.integrations.wandb import WandbLoggerCallback, WANDB_ENV_VAR from ray.rllib.core import DEFAULT_MODULE_ID, Columns from ray.rllib.env.wrappers.atari_wrappers import is_atari, wrap_deepmind @@ -42,10 +41,8 @@ ) from ray.rllib.utils.typing import ResultDict from ray.rllib.utils.error import UnsupportedSpaceException - - from ray.tune import CLIReporter - +from ray.tune.result import TRAINING_ITERATION if TYPE_CHECKING: from ray.rllib.algorithms import Algorithm, AlgorithmConfig @@ -1279,11 +1276,11 @@ def run_rllib_example_script_experiment( results = tune.Tuner( trainable or config.algo_class, param_space=config, - run_config=train.RunConfig( + run_config=tune.RunConfig( stop=stop, verbose=args.verbose, callbacks=tune_callbacks, - checkpoint_config=train.CheckpointConfig( + checkpoint_config=tune.CheckpointConfig( checkpoint_frequency=args.checkpoint_freq, checkpoint_at_end=args.checkpoint_at_end, ), @@ -1511,14 +1508,14 @@ def check_reproducibilty( results1 = tune.Tuner( algo_class, param_space=algo_config.to_dict(), - run_config=train.RunConfig(stop=stop_dict, verbose=1), + run_config=tune.RunConfig(stop=stop_dict, verbose=1), ).fit() results1 = results1.get_best_result().metrics results2 = tune.Tuner( algo_class, param_space=algo_config.to_dict(), - run_config=train.RunConfig(stop=stop_dict, verbose=1), + run_config=tune.RunConfig(stop=stop_dict, verbose=1), ).fit() results2 = results2.get_best_result().metrics diff --git a/src/ray/common/BUILD b/src/ray/common/BUILD index cda0f5ee1d6ee..040c02e0a9180 100644 --- a/src/ray/common/BUILD +++ b/src/ray/common/BUILD @@ -20,6 +20,7 @@ ray_cc_library( ":ray_object", "//src/ray/protobuf:common_cc_proto", "//src/ray/util", + "//src/ray/util:cmd_line_utils", "@boost//:optional", "@com_google_googletest//:gtest", ], @@ -210,6 +211,7 @@ ray_cc_library( ":ray_config", "//src/ray/util", "//src/ray/util:array", + "//src/ray/util:function_traits", "@boost//:asio", "@com_google_absl//absl/container:flat_hash_map", "@com_google_absl//absl/synchronization", diff --git a/src/ray/common/task/task_spec.h b/src/ray/common/task/task_spec.h index ac4a38c92ceee..38635e85f31fe 100644 --- a/src/ray/common/task/task_spec.h +++ b/src/ray/common/task/task_spec.h @@ -28,7 +28,6 @@ #include "ray/common/id.h" #include "ray/common/scheduling/resource_set.h" #include "ray/common/task/task_common.h" -#include "ray/util/container_util.h" extern "C" { #include "ray/thirdparty/sha256.h" diff --git a/src/ray/common/test_util.cc b/src/ray/common/test_util.cc index 9eed7903b8f16..aec57526255f4 100644 --- a/src/ray/common/test_util.cc +++ b/src/ray/common/test_util.cc @@ -23,6 +23,7 @@ #include "ray/common/ray_config.h" #include "ray/common/ray_object.h" #include "ray/common/test_util.h" +#include "ray/util/cmd_line_utils.h" #include "ray/util/filesystem.h" #include "ray/util/logging.h" #include "ray/util/process.h" diff --git a/src/ray/core_worker/core_worker.cc b/src/ray/core_worker/core_worker.cc index 1a9142efdf706..5f40a6523c1af 100644 --- a/src/ray/core_worker/core_worker.cc +++ b/src/ray/core_worker/core_worker.cc @@ -32,6 +32,7 @@ #include "ray/core_worker/transport/task_receiver.h" #include "ray/gcs/gcs_client/gcs_client.h" #include "ray/gcs/pb_util.h" +#include "ray/util/container_util.h" #include "ray/util/event.h" #include "ray/util/subreaper.h" #include "ray/util/util.h" diff --git a/src/ray/gcs/gcs_client/accessor.cc b/src/ray/gcs/gcs_client/accessor.cc index b2caf9a755dec..57b2e4da9a037 100644 --- a/src/ray/gcs/gcs_client/accessor.cc +++ b/src/ray/gcs/gcs_client/accessor.cc @@ -19,6 +19,7 @@ #include "ray/common/asio/instrumented_io_context.h" #include "ray/common/common_protocol.h" #include "ray/gcs/gcs_client/gcs_client.h" +#include "ray/util/container_util.h" namespace ray { namespace gcs { diff --git a/src/ray/object_manager/pull_manager.h b/src/ray/object_manager/pull_manager.h index 216696993d94c..a643a14b2cee2 100644 --- a/src/ray/object_manager/pull_manager.h +++ b/src/ray/object_manager/pull_manager.h @@ -31,6 +31,7 @@ #include "ray/object_manager/ownership_based_object_directory.h" #include "ray/rpc/object_manager/object_manager_client.h" #include "ray/rpc/object_manager/object_manager_server.h" +#include "ray/util/container_util.h" #include "ray/util/counter_map.h" namespace ray { diff --git a/src/ray/raylet/main.cc b/src/ray/raylet/main.cc index 921effc0c7157..1d04b6622559f 100644 --- a/src/ray/raylet/main.cc +++ b/src/ray/raylet/main.cc @@ -26,6 +26,7 @@ #include "ray/gcs/gcs_client/gcs_client.h" #include "ray/raylet/raylet.h" #include "ray/stats/stats.h" +#include "ray/util/cmd_line_utils.h" #include "ray/util/event.h" #include "ray/util/process.h" #include "ray/util/subreaper.h" diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index db1da21f1fcc5..083821a33e7d8 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -41,6 +41,7 @@ #include "ray/raylet/worker_pool.h" #include "ray/rpc/node_manager/node_manager_client.h" #include "ray/stats/metric_defs.h" +#include "ray/util/cmd_line_utils.h" #include "ray/util/event.h" #include "ray/util/event_label.h" #include "ray/util/util.h" diff --git a/src/ray/raylet/scheduling/cluster_resource_manager.h b/src/ray/raylet/scheduling/cluster_resource_manager.h index 38cb927918662..d6fca1a3a0aea 100644 --- a/src/ray/raylet/scheduling/cluster_resource_manager.h +++ b/src/ray/raylet/scheduling/cluster_resource_manager.h @@ -26,6 +26,7 @@ #include "ray/common/scheduling/cluster_resource_data.h" #include "ray/common/scheduling/fixed_point.h" #include "ray/raylet/scheduling/local_resource_manager.h" +#include "ray/util/container_util.h" #include "ray/util/logging.h" #include "src/ray/protobuf/gcs.pb.h" diff --git a/src/ray/raylet/scheduling/cluster_task_manager.h b/src/ray/raylet/scheduling/cluster_task_manager.h index 99f550d0ae847..f03d17830cd46 100644 --- a/src/ray/raylet/scheduling/cluster_task_manager.h +++ b/src/ray/raylet/scheduling/cluster_task_manager.h @@ -25,7 +25,6 @@ #include "ray/raylet/scheduling/local_task_manager_interface.h" #include "ray/raylet/scheduling/scheduler_resource_reporter.h" #include "ray/raylet/scheduling/scheduler_stats.h" -#include "ray/util/container_util.h" namespace ray { namespace raylet { diff --git a/src/ray/util/BUILD b/src/ray/util/BUILD index 7074971c0db72..d5a9c9c5175ea 100644 --- a/src/ray/util/BUILD +++ b/src/ray/util/BUILD @@ -37,7 +37,6 @@ ray_cc_library( ], ) -# TODO(hjiang): filesystem and logging has interdependency, we should split them into three targets: filesystem, logging, ray_check_macros. ray_cc_library( name = "logging", hdrs = ["logging.h"], @@ -193,22 +192,19 @@ ray_cc_library( ], ) -# A giant 'util' target is split since PR https://github.com/ray-project/ray/pull/49938, here we keep the 'util' target for API compatibility. -# -# TODO(hjiang): We include a bunch of misc util function/class inside of the class, should split into multiple files and build targets. +# TODO(hjiang): Split URL related functions into a separate util target. ray_cc_library( name = "util", hdrs = ["util.h"], srcs = ["util.cc"], deps = [ - ":cmd_line_utils", - ":container_util", - ":function_traits", + ":filesystem", ":logging", ":macros", ":process", - ":string_utils", "//:sha256", + "@boost//:asio", + "@com_google_absl//absl/container:flat_hash_map", ], ) @@ -242,6 +238,11 @@ ray_cc_library( ray_cc_library( name = "compat", hdrs = ["compat.h"], + srcs = ["compat.cc"], + deps = [ + ":logging", + "//src/ray/common:status", + ], ) ray_cc_library( diff --git a/src/ray/util/compat.cc b/src/ray/util/compat.cc new file mode 100644 index 0000000000000..c4ccf3610dbe1 --- /dev/null +++ b/src/ray/util/compat.cc @@ -0,0 +1,102 @@ +// Copyright 2025 The Ray Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include "ray/util/compat.h" + +#include + +#include "ray/util/logging.h" + +#if defined(__APPLE__) || defined(__linux__) +#include +#if defined(HAVE_FDATASYNC) && !HAVE_DECL_FDATASYNC +extern int fdatasync(int fildes); +#endif +#elif defined(_WIN32) +#include +#endif + +namespace ray { + +#if defined(__APPLE__) || defined(__linux__) +Status CompleteWrite(MEMFD_TYPE_NON_UNIQUE fd, const char *data, size_t len) { + const ssize_t ret = write(fd, data, len); + if (ret == -1) { + return Status::IOError("") << "Fails to write to file because " << strerror(errno); + } + if (ret != static_cast(len)) { + return Status::IOError("") << "Fails to write all requested bytes, requests to write " + << len << " bytes, but actually write " << ret << " bytes"; + } + return Status::OK(); +} +Status Flush(MEMFD_TYPE_NON_UNIQUE fd) { +#if HAVE_FULLFSYNC + // On macOS and iOS, fsync() doesn't guarantee durability past power + // failures. fcntl(F_FULLFSYNC) is required for that purpose. Some + // filesystems don't support fcntl(F_FULLFSYNC), and require a fallback to + // fsync(). + if (::fcntl(fd, F_FULLFSYNC) == 0) { + return Status::OK(); + } +#endif // HAVE_FULLFSYNC + +#if HAVE_FDATASYNC + int ret = ::fdatasync(fd) == 0; +#else + int ret = ::fsync(fd) == 0; +#endif // HAVE_FDATASYNC + + RAY_CHECK(ret != -1 || errno != EIO) << "Fails to flush to file " << strerror(errno); + if (ret == -1) { + return Status::IOError("") << "Fails to flush file because " << strerror(errno); + } + return Status::OK(); +} +Status Close(MEMFD_TYPE_NON_UNIQUE fd) { + const int ret = close(fd); + if (ret != 0) { + return Status::IOError("") << "Fails to flush file because " << strerror(errno); + } + return Status::OK(); +} +#elif defined(_WIN32) +Status CompleteWrite(MEMFD_TYPE_NON_UNIQUE fd, const char *data, size_t len) { + DWORD bytes_written; + BOOL success = WriteFile(fd, data, (DWORD)len, &bytes_written, NULL); + if (!success) { + return Status::IOError("") << "Fails to write to file"; + } + if ((DWORD)len != bytes_written) { + return Status::IOError("") << "Fails to write all requested bytes, requests to write " + << len << " bytes, but actually write " << bytes_written + << " bytes"; + } + return Status::OK(); +} +Status Flush(MEMFD_TYPE_NON_UNIQUE fd) { + if (!FlushFileBuffers(fd)) { + return Status::IOError("") << "Fails to flush file"; + } + return Status::OK(); +} +Status Close(MEMFD_TYPE_NON_UNIQUE fd) { + if (!CloseHandle(fd)) { + return Status::IOError("") << "Fails to close file handle"; + } + return Status::OK(); +} +#endif + +} // namespace ray diff --git a/src/ray/util/compat.h b/src/ray/util/compat.h index 371192331084a..dd9d426048e22 100644 --- a/src/ray/util/compat.h +++ b/src/ray/util/compat.h @@ -31,6 +31,8 @@ #pragma once +#include "ray/common/status.h" + // Workaround for multithreading on XCode 9, see // https://issues.apache.org/jira/browse/ARROW-1622 and // https://github.com/tensorflow/tensorflow/issues/13220#issuecomment-331579775 @@ -47,6 +49,10 @@ mach_port_t pthread_mach_thread_np(pthread_t); #endif /* _MACH_PORT_T */ #endif /* __APPLE__ */ +#if defined(__APPLE__) || defined(__linux__) +#include +#endif + #ifdef _WIN32 #ifndef _WINDOWS_ #ifndef WIN32_LEAN_AND_MEAN // Sorry for the inconvenience. Please include any related @@ -72,3 +78,27 @@ mach_port_t pthread_mach_thread_np(pthread_t); // since fd values can get re-used by the operating system. #define MEMFD_TYPE std::pair #define INVALID_UNIQUE_FD_ID 0 + +namespace ray { +#if defined(__APPLE__) || defined(__linux__) +inline int GetStdoutFd() { return STDOUT_FILENO; } +inline int GetStderrFd() { return STDERR_FILENO; } +inline MEMFD_TYPE_NON_UNIQUE GetStdoutHandle() { return STDOUT_FILENO; } +inline MEMFD_TYPE_NON_UNIQUE GetStderrHandle() { return STDERR_FILENO; } +#elif defined(_WIN32) +inline int GetStdoutFd() { return _fileno(stdout); } +inline int GetStderrFd() { return _fileno(stderr); } +inline MEMFD_TYPE_NON_UNIQUE GetStdoutHandle() { return GetStdHandle(STD_OUTPUT_HANDLE); } +inline MEMFD_TYPE_NON_UNIQUE GetStderrHandle() { return GetStdHandle(STD_ERROR_HANDLE); } +#endif + +// Write the whole content into file descriptor, if any error happens, or actual written +// content is less than expected, IO error status will be returned. +Status CompleteWrite(MEMFD_TYPE_NON_UNIQUE fd, const char *data, size_t len); +// Flush the given file descriptor, if EIO happens, error message is logged and process +// exits directly. Reference to fsyncgate: https://wiki.postgresql.org/wiki/Fsync_Errors +Status Flush(MEMFD_TYPE_NON_UNIQUE fd); +// Close the given file descriptor, if any error happens, IO error status will be +// returned. +Status Close(MEMFD_TYPE_NON_UNIQUE fd); +} // namespace ray diff --git a/src/ray/util/macros.h b/src/ray/util/macros.h index 6ef537cb80086..0a81b92bc2300 100644 --- a/src/ray/util/macros.h +++ b/src/ray/util/macros.h @@ -66,3 +66,10 @@ #else #define __suppress_ubsan__(type) #endif + +// Portable code for unreachable +#if defined(_MSC_VER) +#define UNREACHABLE __assume(0) +#else +#define UNREACHABLE __builtin_unreachable() +#endif diff --git a/src/ray/util/pipe_logger.h b/src/ray/util/pipe_logger.h index 7c2377466fbe9..4d75742d3f76c 100644 --- a/src/ray/util/pipe_logger.h +++ b/src/ray/util/pipe_logger.h @@ -30,23 +30,9 @@ #include "ray/util/util.h" #include "spdlog/logger.h" -#if defined(__APPLE__) || defined(__linux__) -#include -#elif defined(_WIN32) -#include -#endif - namespace ray { -// Environmenr variable, which indicates the pipe size of read. -// -// TODO(hjiang): Should document the env variable after end-to-end integration has -// finished. -inline constexpr std::string_view kPipeLogReadBufSizeEnv = "RAY_PIPE_LOG_READ_BUF_SIZE"; - // File handle requires active destruction via owner calling [Close]. -// -// TODO(hjiang): Wrap fd with spdlog sink to manage stream flush and close. class RedirectionFileHandle { public: RedirectionFileHandle() = default; @@ -100,17 +86,8 @@ class RedirectionFileHandle { MEMFD_TYPE_NON_UNIQUE GetWriteHandle() const { return write_handle_; } // Write the given data into redirection handle; currently only for testing usage. - // - // TODO(hjiang): Use platform compatible API, see - // https://github.com/ray-project/ray/pull/50170 void CompleteWrite(const char *data, size_t len) { -#if defined(__APPLE__) || defined(__linux__) - [[maybe_unused]] auto x = write(write_handle_, data, len); -#elif defined(_WIN32) - DWORD bytes_written; - [[maybe_unused]] auto x = - WriteFile(write_handle_, data, (DWORD)len, &bytes_written, NULL); -#endif + RAY_CHECK_OK(::ray::CompleteWrite(write_handle_, data, len)); } private: diff --git a/src/ray/util/spdlog_fd_sink.h b/src/ray/util/spdlog_fd_sink.h index f2130b5ee41ee..914c8d56c0363 100644 --- a/src/ray/util/spdlog_fd_sink.h +++ b/src/ray/util/spdlog_fd_sink.h @@ -19,12 +19,6 @@ #include "ray/util/compat.h" #include "ray/util/util.h" -#if defined(__APPLE__) || defined(__linux__) -#include -#elif defined(_WIN32) -#include -#endif - namespace ray { // A sink which logs to the file descriptor. @@ -39,26 +33,9 @@ class non_owned_fd_sink final : public spdlog::sinks::base_sink { void sink_it_(const spdlog::details::log_msg &msg) override { spdlog::memory_buf_t formatted; spdlog::sinks::base_sink::formatter_->format(msg, formatted); - -#if defined(__APPLE__) || defined(__linux__) - RAY_CHECK_EQ(write(fd_, formatted.data(), formatted.size()), - static_cast(formatted.size())) - << "Fails to write because " << strerror(errno); -#elif defined(_WIN32) - DWORD bytes_written; - BOOL success = - WriteFile(fd_, formatted.data(), (DWORD)formatted.size(), &bytes_written, NULL); - RAY_CHECK(success); - RAY_CHECK_EQ((DWORD)formatted.size(), bytes_written); -#endif - } - void flush_() override { -#if defined(__APPLE__) || defined(__linux__) - RAY_CHECK_EQ(fdatasync(fd_), 0) << "Fails to flush file because " << strerror(errno); -#elif defined(_WIN32) - RAY_CHECK(FlushFileBuffers(fd_)); -#endif + RAY_CHECK_OK(CompleteWrite(fd_, formatted.data(), formatted.size())); } + void flush_() override { RAY_CHECK_OK(Flush(fd_)); } private: MEMFD_TYPE_NON_UNIQUE fd_; diff --git a/src/ray/util/stream_redirection_utils.cc b/src/ray/util/stream_redirection_utils.cc index 7812065f55cdf..e67d32c9dc7d7 100644 --- a/src/ray/util/stream_redirection_utils.cc +++ b/src/ray/util/stream_redirection_utils.cc @@ -32,14 +32,6 @@ namespace ray { namespace { -#if defined(__APPLE__) || defined(__linux__) -int GetStdoutHandle() { return STDOUT_FILENO; } -int GetStderrHandle() { return STDERR_FILENO; } -#elif defined(_WIN32) -int GetStdoutHandle() { return _fileno(stdout); } -int GetStderrHandle() { return _fileno(stderr); } -#endif - // TODO(hjiang): Revisit later, should be able to save some heap allocation with // absl::InlinedVector. // @@ -89,12 +81,12 @@ void FlushOnRedirectedStream(int stream_fd) { } // namespace void RedirectStdout(const StreamRedirectionOption &opt) { - RedirectStream(GetStdoutHandle(), opt); + RedirectStream(GetStdoutFd(), opt); } void RedirectStderr(const StreamRedirectionOption &opt) { - RedirectStream(GetStderrHandle(), opt); + RedirectStream(GetStderrFd(), opt); } -void FlushOnRedirectedStdout() { FlushOnRedirectedStream(GetStdoutHandle()); } -void FlushOnRedirectedStderr() { FlushOnRedirectedStream(GetStderrHandle()); } +void FlushOnRedirectedStdout() { FlushOnRedirectedStream(GetStdoutFd()); } +void FlushOnRedirectedStderr() { FlushOnRedirectedStream(GetStderrFd()); } } // namespace ray diff --git a/src/ray/util/tests/BUILD b/src/ray/util/tests/BUILD index 6c76d6be8fc8e..65d55f0cb856a 100644 --- a/src/ray/util/tests/BUILD +++ b/src/ray/util/tests/BUILD @@ -18,6 +18,7 @@ ray_cc_test( tags = ["team:core"], deps = [ "//src/ray/util", + "//src/ray/util:function_traits", "@com_google_googletest//:gtest_main", ], ) @@ -41,6 +42,7 @@ ray_cc_test( linkstatic = True, deps = [ "//src/ray/util", + "//src/ray/util:container_util", "@com_google_absl//absl/container:flat_hash_map", "@com_google_absl//absl/container:flat_hash_set", "@com_google_googletest//:gtest_main", @@ -229,7 +231,6 @@ ray_cc_test( "//src/ray/util", "//src/ray/util:pipe_logger", "@com_google_googletest//:gtest_main", - "@com_google_absl//absl/cleanup", ], size = "small", tags = ["team:core"], @@ -287,3 +288,15 @@ ray_cc_test( size = "small", tags = ["team:core"], ) + +ray_cc_test( + name = "compat_test", + srcs = ["compat_test.cc"], + deps = [ + "//src/ray/util:compat", + "//src/ray/util:filesystem", + "@com_google_googletest//:gtest_main", + ], + size = "small", + tags = ["team:core"], +) diff --git a/src/ray/util/tests/compat_test.cc b/src/ray/util/tests/compat_test.cc new file mode 100644 index 0000000000000..8c3646a6e7137 --- /dev/null +++ b/src/ray/util/tests/compat_test.cc @@ -0,0 +1,52 @@ +// Copyright 2025 The Ray Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include "ray/util/compat.h" + +#include + +#include + +#include "ray/common/status_or.h" +#include "ray/util/filesystem.h" + +#if defined(__APPLE__) || defined(__linux__) +#include +#include +#include +#include +#elif defined(_WIN32) +#include +#endif + +namespace ray { + +namespace { + +constexpr std::string_view kContent = "helloworld"; + +TEST(CompatTest, WriteTest) { + MEMFD_TYPE_NON_UNIQUE fd = GetStdoutHandle(); + + testing::internal::CaptureStdout(); + RAY_CHECK_OK(CompleteWrite(fd, kContent.data(), kContent.length())); + RAY_CHECK_OK(Flush(fd)); + const std::string stdout_content = testing::internal::GetCapturedStdout(); + + EXPECT_EQ(stdout_content, kContent); +} + +} // namespace + +} // namespace ray diff --git a/src/ray/util/tests/pipe_logger_test.cc b/src/ray/util/tests/pipe_logger_test.cc index f8bf32248c5cd..fee6c97554fcc 100644 --- a/src/ray/util/tests/pipe_logger_test.cc +++ b/src/ray/util/tests/pipe_logger_test.cc @@ -22,7 +22,6 @@ #include #include -#include "absl/cleanup/cleanup.h" #include "ray/common/test/testing.h" #include "ray/util/filesystem.h" #include "ray/util/temporary_directory.h" @@ -39,11 +38,6 @@ TEST(PipeLoggerTest, RedirectionTest) { ScopedTemporaryDirectory scoped_directory; const auto test_file_path = scoped_directory.GetDirectory() / GenerateUUIDV4(); - // Delete temporary file. - absl::Cleanup cleanup_test_file = [&test_file_path]() { - EXPECT_TRUE(std::filesystem::remove(test_file_path)); - }; - // Take the default option, which doesn't have rotation enabled. StreamRedirectionOption stream_redirection_opt{}; stream_redirection_opt.file_path = test_file_path.string(); @@ -63,11 +57,6 @@ TEST(PipeLoggerTestWithTee, RedirectionWithTee) { ScopedTemporaryDirectory scoped_directory; const auto test_file_path = scoped_directory.GetDirectory() / GenerateUUIDV4(); - // Delete temporary file. - absl::Cleanup cleanup_test_file = [&test_file_path]() { - EXPECT_TRUE(std::filesystem::remove(test_file_path)); - }; - StreamRedirectionOption stream_redirection_opt{}; stream_redirection_opt.file_path = test_file_path.string(); stream_redirection_opt.tee_to_stdout = true; @@ -98,12 +87,6 @@ TEST(PipeLoggerTestWithTee, RotatedRedirectionWithTee) { const auto log_file_path2 = scoped_directory.GetDirectory() / absl::StrFormat("%s.1", uuid); - // Delete temporary file. - absl::Cleanup cleanup_test_file = [&log_file_path1, &log_file_path2]() { - EXPECT_TRUE(std::filesystem::remove(log_file_path1)); - EXPECT_TRUE(std::filesystem::remove(log_file_path2)); - }; - StreamRedirectionOption stream_redirection_opt{}; stream_redirection_opt.file_path = test_file_path.string(); stream_redirection_opt.rotation_max_size = 5; diff --git a/src/ray/util/util.cc b/src/ray/util/util.cc index 1a8d8ee595167..e18b6a85c44d3 100644 --- a/src/ray/util/util.cc +++ b/src/ray/util/util.cc @@ -34,7 +34,7 @@ #include "absl/strings/match.h" #include "ray/util/filesystem.h" #include "ray/util/logging.h" -#include "ray/util/string_utils.h" +#include "ray/util/process.h" std::string EndpointToUrl( const boost::asio::generic::basic_endpoint &ep, diff --git a/src/ray/util/util.h b/src/ray/util/util.h index affed40741aab..a7df2c97b8010 100644 --- a/src/ray/util/util.h +++ b/src/ray/util/util.h @@ -44,25 +44,14 @@ #include #include -// TODO(hjiang): Revisit these inclusion when we split `util` dependency targets; keep -// them here for backward compatibility and avoid breaking too much compilation. #include "absl/container/flat_hash_map.h" -#include "ray/util/cmd_line_utils.h" #include "ray/util/logging.h" #include "ray/util/macros.h" -#include "ray/util/process.h" #ifdef _WIN32 #include // to ensure getpid() on Windows #endif -// Portable code for unreachable -#if defined(_MSC_VER) -#define UNREACHABLE __assume(0) -#else -#define UNREACHABLE __builtin_unreachable() -#endif - // Boost forward-declarations (to avoid forcing slow header inclusions) namespace boost::asio::generic {