From 43aa2299e6623c8f8c7c4a1b80133459d0aa68b0 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 21 Jun 2022 15:13:29 -0700 Subject: [PATCH] [api] Annotate as public / move ray-core APIs to _private and add enforcement rule (#25695) Enable checking of the ray core module, excluding serve, workflows, and tune, in ./ci/lint/check_api_annotations.py. This required moving many files to ray._private and associated fixes. --- ci/lint/check_api_annotations.py | 41 ++++-- dashboard/agent.py | 2 +- dashboard/consts.py | 2 +- dashboard/dashboard.py | 6 +- dashboard/head.py | 26 ++-- dashboard/memory_utils.py | 12 +- dashboard/modules/actor/actor_head.py | 32 ++-- dashboard/modules/actor/tests/test_actor.py | 19 ++- dashboard/modules/event/event_agent.py | 13 +- dashboard/modules/event/event_consts.py | 2 +- dashboard/modules/job/cli.py | 12 +- dashboard/modules/job/common.py | 12 +- dashboard/modules/job/job_manager.py | 26 ++-- .../check_cuda_devices.py | 3 +- .../modules/job/tests/test_http_job_server.py | 25 ++-- .../modules/job/tests/test_job_manager.py | 24 ++- dashboard/modules/log/tests/test_log.py | 20 ++- dashboard/modules/node/node_head.py | 29 ++-- dashboard/modules/reporter/reporter_agent.py | 17 +-- dashboard/modules/reporter/reporter_consts.py | 2 +- dashboard/modules/reporter/reporter_head.py | 24 +-- .../modules/reporter/tests/test_reporter.py | 27 ++-- .../modules/runtime_env/runtime_env_consts.py | 2 +- dashboard/modules/snapshot/snapshot_head.py | 29 ++-- .../modules/snapshot/tests/test_actors.py | 15 +- dashboard/modules/test/test_agent.py | 8 +- dashboard/modules/test/test_head.py | 10 +- dashboard/optional_utils.py | 5 +- dashboard/tests/test_dashboard.py | 58 ++++---- .../visualize_dag_during_deployment.py | 4 +- python/ray/__init__.py | 30 +++- python/ray/_private/client_mode_hook.py | 9 +- python/ray/{ => _private}/external_storage.py | 12 +- python/ray/_private/function_manager.py | 27 ++-- python/ray/_private/gcs_utils.py | 27 ++-- python/ray/_private/import_thread.py | 8 +- .../inspect.py => _private/inspect_util.py} | 0 .../{internal => _private}/internal_api.py | 18 +-- python/ray/_private/log_monitor.py | 6 +- python/ray/_private/metrics_agent.py | 17 ++- python/ray/{ => _private}/node.py | 16 +- python/ray/_private/parameter.py | 6 +- python/ray/_private/profiling.py | 7 +- python/ray/{ => _private}/ray_constants.py | 0 python/ray/_private/ray_logging.py | 18 ++- python/ray/_private/ray_option_utils.py | 7 +- python/ray/_private/resource_spec.py | 7 +- python/ray/_private/runtime_env/packaging.py | 8 +- python/ray/{ => _private}/serialization.py | 74 +++++----- python/ray/_private/services.py | 34 ++--- python/ray/_private/signature.py | 4 +- python/ray/{ => _private}/state.py | 13 +- python/ray/{internal => _private}/storage.py | 10 +- python/ray/_private/test_utils.py | 62 ++++---- python/ray/_private/usage/usage_lib.py | 12 +- python/ray/_private/utils.py | 115 +++++++++++++-- python/ray/{ => _private}/worker.py | 137 ++++++++---------- python/ray/{ => _private}/workers/__init__.py | 0 .../{ => _private}/workers/default_worker.py | 27 ++-- .../{ => _private}/workers/setup_worker.py | 2 +- python/ray/_raylet.pyx | 54 +++---- python/ray/actor.py | 100 +++++++------ python/ray/air/util/check_ingest.py | 11 +- .../autoscaler/_private/_kubernetes/config.py | 2 +- .../autoscaler/_private/aws/node_provider.py | 2 +- python/ray/autoscaler/_private/commands.py | 2 +- python/ray/autoscaler/_private/constants.py | 2 +- .../_private/fake_multi_node/node_provider.py | 4 +- .../_private/kuberay/run_autoscaler.py | 6 +- .../autoscaler/_private/legacy_info_string.py | 2 +- .../ray/autoscaler/_private/load_metrics.py | 7 +- python/ray/autoscaler/_private/monitor.py | 8 +- .../_private/resource_demand_scheduler.py | 2 +- python/ray/autoscaler/_private/util.py | 2 +- python/ray/autoscaler/command_runner.py | 3 + python/ray/autoscaler/node_provider.py | 2 + python/ray/autoscaler/sdk/sdk.py | 14 ++ python/ray/client_builder.py | 22 +-- python/ray/cluster_utils.py | 20 ++- python/ray/cross_language.py | 115 +++++++-------- python/ray/dag/class_node.py | 3 + python/ray/dag/dag_node.py | 2 + python/ray/dag/format_utils.py | 58 ++++---- python/ray/dag/function_node.py | 2 + python/ray/dag/input_node.py | 4 + python/ray/dag/tests/test_plot.py | 4 +- python/ray/dag/utils.py | 2 +- python/ray/dag/vis_utils.py | 76 +++++----- python/ray/data/__init__.py | 3 +- python/ray/data/_internal/block_batching.py | 4 +- python/ray/data/_internal/progress_bar.py | 4 +- python/ray/data/_internal/size_estimator.py | 2 +- python/ray/data/_internal/stats.py | 2 +- python/ray/data/context.py | 2 +- python/ray/data/dataset.py | 4 +- python/ray/data/tests/test_object_gc.py | 3 +- python/ray/data/tests/test_optimize.py | 10 +- python/ray/data/tests/test_split.py | 13 +- python/ray/exceptions.py | 45 +++++- .../job/example_job/demo_script.py | 4 +- python/ray/experimental/locations.py | 5 +- .../experimental/packaging/load_package.py | 14 +- python/ray/experimental/raysort/main.py | 13 +- python/ray/experimental/shuffle.py | 9 +- python/ray/experimental/state/api.py | 24 +-- python/ray/experimental/state/state_cli.py | 15 +- .../ray/experimental/state/state_manager.py | 47 +++--- python/ray/includes/function_descriptor.pxi | 2 +- python/ray/includes/object_ref.pxi | 6 +- python/ray/internal/__init__.py | 2 +- python/ray/job_config.py | 6 +- python/ray/ray_operator/operator.py | 17 +-- python/ray/ray_operator/operator_utils.py | 9 +- python/ray/remote_function.py | 42 +++--- python/ray/runtime_context.py | 17 ++- python/ray/runtime_env.py | 27 ++-- python/ray/scripts/scripts.py | 65 ++++----- python/ray/serve/__init__.py | 17 ++- python/ray/serve/api.py | 39 ++--- python/ray/serve/context.py | 6 +- python/ray/serve/deployment_graph_build.py | 6 +- python/ray/serve/logging_utils.py | 4 +- python/ray/serve/storage/kv_store.py | 2 +- python/ray/serve/tests/test_cluster.py | 9 +- .../tests/test_deployment_graph_build.py | 26 ++-- python/ray/serve/tests/test_json_serde.py | 4 +- python/ray/serve/tests/test_persistence.py | 5 +- .../ray/serve/tests/test_pipeline_dag_plot.py | 22 +-- python/ray/serve/tests/test_standalone.py | 30 ++-- python/ray/serve/tests/test_standalone2.py | 11 +- python/ray/serve/utils.py | 17 ++- python/ray/sgd/__init__.py | 1 - python/ray/streaming/__init__.py | 28 ---- python/ray/tests/aws/test_autoscaler_aws.py | 36 ++--- python/ray/tests/conftest.py | 2 +- python/ray/tests/test_actor.py | 35 ++--- python/ray/tests/test_actor_advanced.py | 83 ++++++----- python/ray/tests/test_actor_group.py | 5 +- python/ray/tests/test_actor_resources.py | 62 ++++++-- python/ray/tests/test_advanced.py | 30 ++-- python/ray/tests/test_advanced_2.py | 47 +++--- python/ray/tests/test_advanced_3.py | 27 ++-- python/ray/tests/test_advanced_4.py | 25 ++-- python/ray/tests/test_advanced_5.py | 10 +- python/ray/tests/test_advanced_6.py | 8 +- python/ray/tests/test_advanced_7.py | 19 +-- python/ray/tests/test_advanced_8.py | 29 ++-- python/ray/tests/test_advanced_9.py | 13 +- python/ray/tests/test_asyncio.py | 4 +- python/ray/tests/test_autoscaler.py | 80 +++++----- .../tests/test_autoscaler_drain_node_api.py | 8 +- .../tests/test_autoscaler_fake_scaledown.py | 7 +- python/ray/tests/test_autoscaler_yaml.py | 23 ++- python/ray/tests/test_basic.py | 10 +- python/ray/tests/test_basic_2.py | 32 ++-- python/ray/tests/test_basic_3.py | 5 +- python/ray/tests/test_basic_4.py | 10 +- python/ray/tests/test_basic_5.py | 4 +- python/ray/tests/test_client.py | 41 +++--- python/ray/tests/test_client_builder.py | 11 +- python/ray/tests/test_client_proxy.py | 12 +- python/ray/tests/test_client_references.py | 14 +- python/ray/tests/test_client_terminate.py | 21 +-- python/ray/tests/test_component_failures.py | 12 +- python/ray/tests/test_component_failures_2.py | 7 +- python/ray/tests/test_component_failures_3.py | 10 +- python/ray/tests/test_dashboard.py | 18 +-- .../ray/tests/test_dataclient_disconnect.py | 8 +- python/ray/tests/test_failure.py | 24 +-- python/ray/tests/test_failure_2.py | 22 +-- python/ray/tests/test_failure_4.py | 32 ++-- python/ray/tests/test_gcs_fault_tolerance.py | 44 +++--- python/ray/tests/test_gcs_utils.py | 23 +-- python/ray/tests/test_global_gc.py | 3 +- python/ray/tests/test_global_state.py | 53 +++---- python/ray/tests/test_healthcheck.py | 9 +- python/ray/tests/test_job.py | 16 +- .../ray/tests/test_k8s_operator_unit_tests.py | 2 +- .../ray/tests/test_kill_raylet_signal_log.py | 6 +- python/ray/tests/test_logging.py | 40 ++--- python/ray/tests/test_memstat.py | 17 +-- python/ray/tests/test_metrics.py | 19 +-- python/ray/tests/test_metrics_agent.py | 12 +- python/ray/tests/test_multi_node.py | 13 +- python/ray/tests/test_multi_node_2.py | 17 ++- python/ray/tests/test_multi_node_3.py | 22 +-- python/ray/tests/test_multinode_failures.py | 4 +- python/ray/tests/test_multinode_failures_2.py | 4 +- python/ray/tests/test_namespace.py | 5 +- python/ray/tests/test_object_manager.py | 12 +- python/ray/tests/test_object_spilling.py | 38 +++-- python/ray/tests/test_object_spilling_2.py | 9 +- python/ray/tests/test_object_spilling_3.py | 19 +-- .../ray/tests/test_object_spilling_no_asan.py | 11 +- python/ray/tests/test_output.py | 16 +- python/ray/tests/test_placement_group.py | 21 ++- python/ray/tests/test_placement_group_2.py | 50 +++---- python/ray/tests/test_placement_group_3.py | 46 +++--- python/ray/tests/test_placement_group_5.py | 9 +- python/ray/tests/test_ray_debugger.py | 5 +- python/ray/tests/test_ray_init.py | 17 ++- python/ray/tests/test_raylet_output.py | 9 +- python/ray/tests/test_reconstruction_2.py | 12 +- python/ray/tests/test_reference_counting.py | 25 ++-- python/ray/tests/test_reference_counting_2.py | 9 +- .../tests/test_resource_demand_scheduler.py | 96 ++++++------ python/ray/tests/test_runtime_env.py | 8 +- .../ray/tests/test_runtime_env_packaging.py | 47 +++--- .../tests/test_runtime_env_working_dir_3.py | 13 +- python/ray/tests/test_scheduling.py | 53 ++++--- python/ray/tests/test_scheduling_2.py | 35 ++--- .../ray/tests/test_scheduling_performance.py | 9 +- python/ray/tests/test_serialization.py | 7 +- python/ray/tests/test_state_api.py | 10 +- python/ray/tests/test_storage.py | 9 +- python/ray/tests/test_stress_failure.py | 9 +- python/ray/tests/test_tempfile.py | 9 +- python/ray/tests/test_threaded_actor.py | 6 +- python/ray/tests/test_unhandled_error.py | 2 +- python/ray/tests/test_usage_stats.py | 10 +- .../ray/train/_internal/backend_executor.py | 7 +- python/ray/train/callbacks/logging.py | 16 +- python/ray/train/tests/test_backend.py | 19 +-- python/ray/train/tests/test_worker_group.py | 8 +- .../impl/out_of_band_serialize_dataset.py | 2 +- python/ray/tune/integration/mlflow.py | 12 +- python/ray/tune/ray_trial_executor.py | 36 ++--- python/ray/tune/registry.py | 10 +- .../ray/tune/tests/test_integration_mlflow.py | 15 +- python/ray/tune/trainable.py | 2 +- python/ray/tune/tune.py | 48 +++--- python/ray/tune/utils/resource_updater.py | 4 +- python/ray/tune/utils/trainable.py | 9 +- python/ray/types.py | 3 + python/ray/util/__init__.py | 19 ++- python/ray/util/check_serialize.py | 9 +- python/ray/util/client/__init__.py | 33 +++-- python/ray/util/client/api.py | 28 ++-- python/ray/util/client/common.py | 43 +++--- python/ray/util/client/runtime_context.py | 9 +- python/ray/util/client/server/logservicer.py | 13 +- python/ray/util/client/server/proxier.py | 47 +++--- python/ray/util/client/server/server.py | 75 +++++----- python/ray/util/client/worker.py | 31 ++-- python/ray/util/client_connect.py | 17 ++- python/ray/util/collective/collective.py | 5 +- .../collective_group/gloo_collective_group.py | 30 ++-- .../collective/collective_group/gloo_util.py | 20 +-- .../util/dask/tests/test_dask_scheduler.py | 10 +- python/ray/util/debug.py | 6 + python/ray/util/iter.py | 12 +- python/ray/util/iter_metrics.py | 3 + python/ray/util/joblib/ray_backend.py | 9 +- python/ray/util/ml_utils/dict.py | 10 +- python/ray/util/ml_utils/filelock.py | 8 +- python/ray/util/ml_utils/json.py | 6 +- python/ray/util/ml_utils/mlflow.py | 10 +- python/ray/util/ml_utils/node.py | 3 + python/ray/util/ml_utils/tests/test_mlflow.py | 7 +- python/ray/util/ml_utils/util.py | 8 +- python/ray/util/multiprocessing/pool.py | 25 ++-- python/ray/util/placement_group.py | 35 ++--- python/ray/util/rpdb.py | 60 ++++---- python/ray/util/serialization.py | 8 +- python/ray/{ => util}/serialization_addons.py | 5 + python/ray/util/tracing/tracing_helper.py | 80 +++++----- python/ray/utils.py | 87 ----------- python/ray/workflow/tests/test_events.py | 17 ++- .../workflow/tests/test_inplace_workflows.py | 14 +- .../ray/workflow/tests/test_serialization.py | 19 +-- python/ray/workflow/tests/test_storage.py | 14 +- python/ray/workflow/workflow_context.py | 11 +- python/ray/workflow/workflow_storage.py | 21 ++- .../chaos_test/test_chaos_basic.py | 10 +- .../dask_on_ray/dask_on_ray_sort.py | 19 ++- .../dask_on_ray/large_scale_test.py | 31 ++-- release/nightly_tests/dataset/sort.py | 2 +- .../stress_tests/test_many_tasks.py | 7 +- rllib/evaluation/rollout_worker.py | 50 +++---- rllib/policy/tf_policy.py | 7 +- rllib/policy/torch_policy.py | 22 +-- rllib/policy/torch_policy_v2.py | 7 +- rllib/utils/torch_utils.py | 7 +- src/ray/common/constants.h | 2 +- 284 files changed, 2777 insertions(+), 2625 deletions(-) rename python/ray/{ => _private}/external_storage.py (99%) rename python/ray/{util/inspect.py => _private/inspect_util.py} (100%) rename python/ray/{internal => _private}/internal_api.py (95%) rename python/ray/{ => _private}/node.py (99%) rename python/ray/{ => _private}/ray_constants.py (100%) rename python/ray/{ => _private}/serialization.py (97%) rename python/ray/{ => _private}/state.py (99%) rename python/ray/{internal => _private}/storage.py (99%) rename python/ray/{ => _private}/worker.py (98%) rename python/ray/{ => _private}/workers/__init__.py (100%) rename python/ray/{ => _private}/workers/default_worker.py (92%) rename python/ray/{ => _private}/workers/setup_worker.py (94%) delete mode 100644 python/ray/sgd/__init__.py delete mode 100644 python/ray/streaming/__init__.py rename python/ray/{ => util}/serialization_addons.py (95%) delete mode 100644 python/ray/utils.py diff --git a/ci/lint/check_api_annotations.py b/ci/lint/check_api_annotations.py index 58bf3b8a1..09029cf03 100755 --- a/ci/lint/check_api_annotations.py +++ b/ci/lint/check_api_annotations.py @@ -13,25 +13,30 @@ IGNORE_PATHS = { ".generated.", ".test_utils.", ".annotations.", - ".deprecation", + ".deprecation.", + ".protobuf.", + ".cloudpickle.", } def _fullname(attr): """Fully qualified name of an attribute.""" fullname = "" - if hasattr(attr, "__module__"): - fullname += attr.__module__ - if hasattr(attr, "__name__"): - if fullname: - fullname += "." - fullname += attr.__name__ - if not fullname: - fullname = str(attr) + try: + if hasattr(attr, "__module__"): + fullname += attr.__module__ + if hasattr(attr, "__name__"): + if fullname: + fullname += "." + fullname += attr.__name__ + if not fullname: + fullname = str(attr) + except Exception as e: + print("Error qualifying", e) return fullname -def _ignore(attr): +def _ignore(attr, extra_ignore): """Whether an attr should be ignored from annotation checking.""" attr = _fullname(attr) if "ray." not in attr or "._" in attr: @@ -39,10 +44,13 @@ def _ignore(attr): for path in IGNORE_PATHS: if path in attr: return True + for path in extra_ignore or []: + if path in attr: + return True return False -def verify(symbol, scanned, ok, output, prefix=None): +def verify(symbol, scanned, ok, output, prefix=None, ignore=None): """Recursively verify all child symbols of a given module.""" if not prefix: prefix = symbol.__name__ + "." @@ -53,7 +61,7 @@ def verify(symbol, scanned, ok, output, prefix=None): if child.startswith("_"): continue attr = getattr(symbol, child) - if _ignore(attr): + if _ignore(attr, ignore): continue if (inspect.isclass(attr) or inspect.isfunction(attr)) and prefix in _fullname( attr @@ -68,7 +76,7 @@ def verify(symbol, scanned, ok, output, prefix=None): scanned.add(attr) elif inspect.ismodule(attr): print("Scanning module", attr) - verify(attr, scanned, ok, output, prefix) + verify(attr, scanned, ok, output, prefix, ignore) else: print("Not scanning", attr, type(attr)) @@ -77,8 +85,9 @@ if __name__ == "__main__": import ray.data import ray.rllib import ray.serve - import ray.tune import ray.train + import ray.tune + import ray.workflow output = set() ok = set() @@ -89,10 +98,12 @@ if __name__ == "__main__": verify(ray.rllib, set(), ok, output) verify(ray.air, set(), ok, output) verify(ray.train, set(), ok, output) + verify(ray, set(), ok, output, ignore=["ray.workflow", "ray.tune", "ray.serve"]) + assert len(ok) >= 400, len(ok) # TODO(ekl) enable it for all modules. # verify(ray.serve, set(), ok, output) # verify(ray.tune, set(), ok, output) - # verify(ray, set(), ok, output) + # verify(ray.workflow, set(), ok, output) print("Num ok", len(ok)) print("Num bad", len(output)) diff --git a/dashboard/agent.py b/dashboard/agent.py index 6498796e5..16844620d 100644 --- a/dashboard/agent.py +++ b/dashboard/agent.py @@ -8,12 +8,12 @@ import os import sys import ray +import ray._private.ray_constants as ray_constants import ray._private.services import ray._private.utils import ray.dashboard.consts as dashboard_consts import ray.dashboard.utils as dashboard_utils import ray.experimental.internal_kv as internal_kv -import ray.ray_constants as ray_constants from ray._private.gcs_pubsub import GcsAioPublisher, GcsPublisher from ray._private.gcs_utils import GcsAioClient, GcsClient from ray._private.ray_logging import setup_component_logger diff --git a/dashboard/consts.py b/dashboard/consts.py index 740ef7272..a81d3ebc7 100644 --- a/dashboard/consts.py +++ b/dashboard/consts.py @@ -1,4 +1,4 @@ -from ray.ray_constants import env_integer +from ray._private.ray_constants import env_integer DASHBOARD_LOG_FILENAME = "dashboard.log" DASHBOARD_AGENT_PORT_PREFIX = "DASHBOARD_AGENT_PORT_PREFIX:" diff --git a/dashboard/dashboard.py b/dashboard/dashboard.py index 2db565500..1a08b0439 100644 --- a/dashboard/dashboard.py +++ b/dashboard/dashboard.py @@ -5,12 +5,12 @@ import logging.handlers import platform import traceback +import ray._private.ray_constants as ray_constants +import ray._private.services +import ray._private.utils import ray.dashboard.consts as dashboard_consts import ray.dashboard.head as dashboard_head import ray.dashboard.utils as dashboard_utils -import ray.ray_constants as ray_constants -import ray._private.services -import ray._private.utils from ray._private.gcs_pubsub import GcsPublisher from ray._private.ray_logging import setup_component_logger diff --git a/dashboard/head.py b/dashboard/head.py index a1002de65..37df66f87 100644 --- a/dashboard/head.py +++ b/dashboard/head.py @@ -1,28 +1,26 @@ -import os import asyncio import logging +import os import threading from concurrent.futures import Future from queue import Queue +import ray._private.services +import ray._private.utils +import ray.dashboard.consts as dashboard_consts +import ray.dashboard.utils as dashboard_utils +import ray.experimental.internal_kv as internal_kv +from ray._private import ray_constants +from ray._private.gcs_pubsub import GcsAioErrorSubscriber, GcsAioLogSubscriber +from ray._private.gcs_utils import GcsClient, check_health +from ray.dashboard.datacenter import DataOrganizer +from ray.dashboard.utils import async_loop_forever + try: from grpc import aio as aiogrpc except ImportError: from grpc.experimental import aio as aiogrpc -import ray.experimental.internal_kv as internal_kv -import ray._private.utils -from ray._private.gcs_utils import GcsClient, check_health -import ray._private.services -import ray.dashboard.consts as dashboard_consts -import ray.dashboard.utils as dashboard_utils -from ray import ray_constants -from ray._private.gcs_pubsub import ( - GcsAioErrorSubscriber, - GcsAioLogSubscriber, -) -from ray.dashboard.datacenter import DataOrganizer -from ray.dashboard.utils import async_loop_forever logger = logging.getLogger(__name__) diff --git a/dashboard/memory_utils.py b/dashboard/memory_utils.py index ca26095ad..dd69b8568 100644 --- a/dashboard/memory_utils.py +++ b/dashboard/memory_utils.py @@ -1,14 +1,12 @@ import base64 - +import logging from collections import defaultdict from enum import Enum from typing import List import ray - -from ray._raylet import TaskID, ActorID, JobID -from ray.internal.internal_api import node_stats -import logging +from ray._private.internal_api import node_stats +from ray._raylet import ActorID, JobID, TaskID logger = logging.getLogger(__name__) @@ -378,11 +376,11 @@ def memory_summary( unit="B", num_entries=None, ) -> str: - from ray.dashboard.modules.node.node_head import node_stats_to_dict - # Get terminal size import shutil + from ray.dashboard.modules.node.node_head import node_stats_to_dict + size = shutil.get_terminal_size((80, 20)).columns line_wrap_threshold = 137 diff --git a/dashboard/modules/actor/actor_head.py b/dashboard/modules/actor/actor_head.py index 0724efc57..fa4edd842 100644 --- a/dashboard/modules/actor/actor_head.py +++ b/dashboard/modules/actor/actor_head.py @@ -1,28 +1,30 @@ import asyncio import logging + import aiohttp.web + +import ray._private.ray_constants as ray_constants import ray._private.utils -from ray.dashboard.modules.actor import actor_utils +import ray.dashboard.optional_utils as dashboard_optional_utils +import ray.dashboard.utils as dashboard_utils +from ray._private.gcs_pubsub import GcsAioActorSubscriber +from ray.core.generated import ( + core_worker_pb2, + core_worker_pb2_grpc, + gcs_service_pb2, + gcs_service_pb2_grpc, + node_manager_pb2_grpc, +) +from ray.dashboard.datacenter import DataOrganizer, DataSource +from ray.dashboard.modules.actor import actor_consts, actor_utils +from ray.dashboard.modules.actor.actor_utils import actor_classname_from_func_descriptor +from ray.dashboard.optional_utils import rest_response try: from grpc import aio as aiogrpc except ImportError: from grpc.experimental import aio as aiogrpc -from ray._private.gcs_pubsub import GcsAioActorSubscriber -import ray.ray_constants as ray_constants -import ray.dashboard.utils as dashboard_utils -import ray.dashboard.optional_utils as dashboard_optional_utils -from ray.dashboard.optional_utils import rest_response -from ray.dashboard.modules.actor import actor_consts -from ray.dashboard.modules.actor.actor_utils import actor_classname_from_func_descriptor -from ray.core.generated import node_manager_pb2_grpc -from ray.core.generated import gcs_service_pb2 -from ray.core.generated import gcs_service_pb2_grpc -from ray.core.generated import core_worker_pb2 -from ray.core.generated import core_worker_pb2_grpc -from ray.dashboard.datacenter import DataSource, DataOrganizer - logger = logging.getLogger(__name__) routes = dashboard_optional_utils.ClassMethodRouteTable diff --git a/dashboard/modules/actor/tests/test_actor.py b/dashboard/modules/actor/tests/test_actor.py index 5a53b263e..04a04a6fd 100644 --- a/dashboard/modules/actor/tests/test_actor.py +++ b/dashboard/modules/actor/tests/test_actor.py @@ -1,19 +1,18 @@ +import logging import os import sys -import logging -import requests import time import traceback -import ray + import pytest -import ray.dashboard.utils as dashboard_utils +import requests + +import ray import ray._private.gcs_pubsub as gcs_pubsub -from ray.dashboard.tests.conftest import * # noqa +import ray.dashboard.utils as dashboard_utils +from ray._private.test_utils import format_web_url, wait_until_server_available from ray.dashboard.modules.actor import actor_consts -from ray._private.test_utils import ( - format_web_url, - wait_until_server_available, -) +from ray.dashboard.tests.conftest import * # noqa logger = logging.getLogger(__name__) @@ -152,7 +151,7 @@ def test_kill_actor(ray_start_with_dashboard): pass def f(self): - ray.worker.show_in_dashboard("test") + ray._private.worker.show_in_dashboard("test") return os.getpid() a = Actor.remote() diff --git a/dashboard/modules/event/event_agent.py b/dashboard/modules/event/event_agent.py index 8bcecb4c5..a3520e0c2 100644 --- a/dashboard/modules/event/event_agent.py +++ b/dashboard/modules/event/event_agent.py @@ -1,18 +1,17 @@ -import os import asyncio import logging +import os from typing import Union -import ray.experimental.internal_kv as internal_kv -import ray.ray_constants as ray_constants +import ray._private.ray_constants as ray_constants import ray._private.utils as utils -import ray.dashboard.utils as dashboard_utils import ray.dashboard.consts as dashboard_consts -from ray.dashboard.utils import async_loop_forever, create_task +import ray.dashboard.utils as dashboard_utils +import ray.experimental.internal_kv as internal_kv +from ray.core.generated import event_pb2, event_pb2_grpc from ray.dashboard.modules.event import event_consts from ray.dashboard.modules.event.event_utils import monitor_events -from ray.core.generated import event_pb2 -from ray.core.generated import event_pb2_grpc +from ray.dashboard.utils import async_loop_forever, create_task logger = logging.getLogger(__name__) diff --git a/dashboard/modules/event/event_consts.py b/dashboard/modules/event/event_consts.py index 2520d2803..3efd4b716 100644 --- a/dashboard/modules/event/event_consts.py +++ b/dashboard/modules/event/event_consts.py @@ -1,4 +1,4 @@ -from ray.ray_constants import env_integer +from ray._private.ray_constants import env_integer from ray.core.generated import event_pb2 LOG_ERROR_EVENT_STRING_LENGTH_LIMIT = 1000 diff --git a/dashboard/modules/job/cli.py b/dashboard/modules/job/cli.py index acf7db6b2..9f2253b2a 100644 --- a/dashboard/modules/job/cli.py +++ b/dashboard/modules/job/cli.py @@ -1,18 +1,18 @@ import asyncio import os import pprint -from subprocess import list2cmdline import time +from subprocess import list2cmdline from typing import Optional, Tuple import click -import ray.ray_constants as ray_constants -from ray.autoscaler._private.cli_logger import add_click_logging_options, cli_logger, cf -from ray.job_submission import JobStatus, JobSubmissionClient -from ray.internal.storage import _load_class -from ray.util.annotations import PublicAPI +import ray._private.ray_constants as ray_constants +from ray._private.storage import _load_class +from ray.autoscaler._private.cli_logger import add_click_logging_options, cf, cli_logger from ray.dashboard.modules.dashboard_sdk import parse_runtime_env_args +from ray.job_submission import JobStatus, JobSubmissionClient +from ray.util.annotations import PublicAPI def _get_sdk_client( diff --git a/dashboard/modules/job/common.py b/dashboard/modules/job/common.py index 721328027..cf6a6e52c 100644 --- a/dashboard/modules/job/common.py +++ b/dashboard/modules/job/common.py @@ -1,18 +1,18 @@ +import pickle +import time from dataclasses import dataclass, replace from enum import Enum -import time -from typing import Any, Dict, Optional, Tuple -import pickle from pathlib import Path +from typing import Any, Dict, Optional, Tuple -from ray import ray_constants +from ray._private import ray_constants +from ray._private.runtime_env.packaging import parse_uri from ray.experimental.internal_kv import ( - _internal_kv_initialized, _internal_kv_get, + _internal_kv_initialized, _internal_kv_list, _internal_kv_put, ) -from ray._private.runtime_env.packaging import parse_uri # NOTE(edoakes): these constants should be considered a public API because # they're exposed in the snapshot API. diff --git a/dashboard/modules/job/job_manager.py b/dashboard/modules/job/job_manager.py index 0925e7433..d27492881 100644 --- a/dashboard/modules/job/job_manager.py +++ b/dashboard/modules/job/job_manager.py @@ -1,30 +1,30 @@ import asyncio -from asyncio.tasks import FIRST_COMPLETED import copy -import os import json import logging +import os +import random +import string +import subprocess import time import traceback -import random -import subprocess -import string +from asyncio.tasks import FIRST_COMPLETED from collections import deque -from typing import Any, Dict, Iterator, Tuple, Optional +from typing import Any, Dict, Iterator, Optional, Tuple import ray -from ray.exceptions import RuntimeEnvSetupError -import ray.ray_constants as ray_constants +import ray._private.ray_constants as ray_constants +from ray._private.runtime_env.constants import RAY_JOB_CONFIG_JSON_ENV_VAR from ray.actor import ActorHandle -from ray.job_submission import JobStatus from ray.dashboard.modules.job.common import ( - JobInfo, - JobInfoStorageClient, JOB_ID_METADATA_KEY, JOB_NAME_METADATA_KEY, + JobInfo, + JobInfoStorageClient, ) from ray.dashboard.modules.job.utils import file_tail_iterator -from ray._private.runtime_env.constants import RAY_JOB_CONFIG_JSON_ENV_VAR +from ray.exceptions import RuntimeEnvSetupError +from ray.job_submission import JobStatus logger = logging.getLogger(__name__) @@ -86,7 +86,7 @@ class JobLogStorageClient: /tmp/ray/session_date/logs/job-driver-{job_id}.log """ return os.path.join( - ray.worker._global_node.get_logs_dir_path(), + ray._private.worker._global_node.get_logs_dir_path(), self.JOB_LOGS_PATH.format(job_id=job_id), ) diff --git a/dashboard/modules/job/tests/subprocess_driver_scripts/check_cuda_devices.py b/dashboard/modules/job/tests/subprocess_driver_scripts/check_cuda_devices.py index 845f07bb1..169340b72 100644 --- a/dashboard/modules/job/tests/subprocess_driver_scripts/check_cuda_devices.py +++ b/dashboard/modules/job/tests/subprocess_driver_scripts/check_cuda_devices.py @@ -1,7 +1,8 @@ import os + import ray -cuda_env = ray.ray_constants.NOSET_CUDA_VISIBLE_DEVICES_ENV_VAR +cuda_env = ray._private.ray_constants.NOSET_CUDA_VISIBLE_DEVICES_ENV_VAR assert os.environ[cuda_env] == "1" diff --git a/dashboard/modules/job/tests/test_http_job_server.py b/dashboard/modules/job/tests/test_http_job_server.py index 29f120f96..2846ffd7a 100644 --- a/dashboard/modules/job/tests/test_http_job_server.py +++ b/dashboard/modules/job/tests/test_http_job_server.py @@ -1,31 +1,28 @@ +import json import logging -from pathlib import Path import os import shutil import sys -import json -import yaml import tempfile +from pathlib import Path from typing import Optional - -import pytest from unittest.mock import patch +import pytest +import yaml + import ray -from ray.job_submission import JobSubmissionClient, JobStatus -from ray.dashboard.modules.job.common import CURRENT_VERSION, JobInfo -from ray.dashboard.modules.dashboard_sdk import ( - ClusterInfo, - parse_cluster_info, -) -from ray.dashboard.tests.conftest import * # noqa -from ray.tests.conftest import _ray_start from ray._private.test_utils import ( chdir, format_web_url, wait_for_condition, wait_until_server_available, ) +from ray.dashboard.modules.dashboard_sdk import ClusterInfo, parse_cluster_info +from ray.dashboard.modules.job.common import CURRENT_VERSION, JobInfo +from ray.dashboard.tests.conftest import * # noqa +from ray.job_submission import JobStatus, JobSubmissionClient +from ray.tests.conftest import _ray_start logger = logging.getLogger(__name__) @@ -391,7 +388,7 @@ def test_job_metadata(job_sdk_client): 'python -c"' "import ray;" "ray.init();" - "job_config=ray.worker.global_worker.core_worker.get_job_config();" + "job_config=ray._private.worker.global_worker.core_worker.get_job_config();" "print(dict(sorted(job_config.metadata.items())))" '"' ) diff --git a/dashboard/modules/job/tests/test_job_manager.py b/dashboard/modules/job/tests/test_job_manager.py index ffe62e954..2d5d92032 100644 --- a/dashboard/modules/job/tests/test_job_manager.py +++ b/dashboard/modules/job/tests/test_job_manager.py @@ -1,24 +1,20 @@ import asyncio import os -import psutil -import tempfile +import signal import sys +import tempfile import urllib.request from uuid import uuid4 -import signal +import psutil import pytest import ray -from ray.job_submission import JobStatus -from ray.dashboard.modules.job.common import ( - JOB_ID_METADATA_KEY, - JOB_NAME_METADATA_KEY, -) -from ray.dashboard.modules.job.job_manager import generate_job_id, JobManager +from ray._private.ray_constants import RAY_ADDRESS_ENVIRONMENT_VARIABLE from ray._private.test_utils import SignalActor, async_wait_for_condition -from ray.ray_constants import RAY_ADDRESS_ENVIRONMENT_VARIABLE - +from ray.dashboard.modules.job.common import JOB_ID_METADATA_KEY, JOB_NAME_METADATA_KEY +from ray.dashboard.modules.job.job_manager import JobManager, generate_job_id +from ray.job_submission import JobStatus from ray.tests.conftest import call_ray_start # noqa: F401 TEST_NAMESPACE = "jobs_test_namespace" @@ -395,7 +391,7 @@ class TestRuntimeEnv: 'python -c"' "import ray;" "ray.init();" - "job_config=ray.worker.global_worker.core_worker.get_job_config();" + "job_config=ray._private.worker.global_worker.core_worker.get_job_config();" "print(dict(sorted(job_config.metadata.items())))" '"' ) @@ -716,8 +712,8 @@ async def test_bootstrap_address(job_manager, monkeypatch): cluster might be started with http://ip:{dashboard_port} from previous runs. """ - ip = ray.ray_constants.DEFAULT_DASHBOARD_IP - port = ray.ray_constants.DEFAULT_DASHBOARD_PORT + ip = ray._private.ray_constants.DEFAULT_DASHBOARD_IP + port = ray._private.ray_constants.DEFAULT_DASHBOARD_PORT monkeypatch.setenv("RAY_ADDRESS", f"http://{ip}:{port}") print_ray_address_cmd = ( diff --git a/dashboard/modules/log/tests/test_log.py b/dashboard/modules/log/tests/test_log.py index bb0ba3480..c7f551c95 100644 --- a/dashboard/modules/log/tests/test_log.py +++ b/dashboard/modules/log/tests/test_log.py @@ -1,17 +1,15 @@ -import sys +import html.parser import logging -import requests +import sys import time import traceback -import html.parser import urllib.parse -import pytest -import ray -from ray._private.test_utils import ( - format_web_url, - wait_until_server_available, -) +import pytest +import requests + +import ray +from ray._private.test_utils import format_web_url, wait_until_server_available from ray.dashboard.tests.conftest import * # noqa logger = logging.getLogger(__name__) @@ -44,7 +42,7 @@ def test_log(disable_aiohttp_cache, ray_start_with_dashboard): test_file = "test.log" with open( - f"{ray.worker.global_worker.node.get_logs_dir_path()}/{test_file}", "w" + f"{ray._private.worker.global_worker.node.get_logs_dir_path()}/{test_file}", "w" ) as f: f.write(test_log_text) assert wait_until_server_available(ray_start_with_dashboard["webui_url"]) is True @@ -128,7 +126,7 @@ def test_log_proxy(ray_start_with_dashboard): test_log_text = "test_log_text" test_file = "test.log" with open( - f"{ray.worker.global_worker.node.get_logs_dir_path()}/{test_file}", "w" + f"{ray._private.worker.global_worker.node.get_logs_dir_path()}/{test_file}", "w" ) as f: f.write(test_log_text) while True: diff --git a/dashboard/modules/node/node_head.py b/dashboard/modules/node/node_head.py index 5dee64db2..8a4aa2eca 100644 --- a/dashboard/modules/node/node_head.py +++ b/dashboard/modules/node/node_head.py @@ -1,26 +1,29 @@ import asyncio -import re -import logging import json +import logging +import re + import aiohttp.web import ray._private.utils -from ray import ray_constants +import ray.dashboard.consts as dashboard_consts +import ray.dashboard.optional_utils as dashboard_optional_utils +import ray.dashboard.utils as dashboard_utils +from ray._private import ray_constants +from ray.core.generated import ( + gcs_service_pb2, + gcs_service_pb2_grpc, + node_manager_pb2, + node_manager_pb2_grpc, +) +from ray.dashboard.datacenter import DataOrganizer, DataSource +from ray.dashboard.memory_utils import GroupByType, SortingType from ray.dashboard.modules.node import node_consts from ray.dashboard.modules.node.node_consts import ( - MAX_LOGS_TO_CACHE, LOG_PRUNE_THREASHOLD, + MAX_LOGS_TO_CACHE, ) -import ray.dashboard.utils as dashboard_utils -import ray.dashboard.optional_utils as dashboard_optional_utils -import ray.dashboard.consts as dashboard_consts from ray.dashboard.utils import async_loop_forever -from ray.dashboard.memory_utils import GroupByType, SortingType -from ray.core.generated import node_manager_pb2 -from ray.core.generated import node_manager_pb2_grpc -from ray.core.generated import gcs_service_pb2 -from ray.core.generated import gcs_service_pb2_grpc -from ray.dashboard.datacenter import DataSource, DataOrganizer logger = logging.getLogger(__name__) routes = dashboard_optional_utils.ClassMethodRouteTable diff --git a/dashboard/modules/reporter/reporter_agent.py b/dashboard/modules/reporter/reporter_agent.py index 07b2db9ee..8ecc752de 100644 --- a/dashboard/modules/reporter/reporter_agent.py +++ b/dashboard/modules/reporter/reporter_agent.py @@ -9,20 +9,19 @@ import sys import traceback import warnings +import psutil import ray -import ray.dashboard.modules.reporter.reporter_consts as reporter_consts -from ray.dashboard import k8s_utils -import ray.dashboard.utils as dashboard_utils -import ray.experimental.internal_kv as internal_kv import ray._private.services import ray._private.utils -from ray.core.generated import reporter_pb2 -from ray.core.generated import reporter_pb2_grpc -from ray.ray_constants import DEBUG_AUTOSCALING_STATUS -from ray._private.metrics_agent import MetricsAgent, Gauge, Record +import ray.dashboard.modules.reporter.reporter_consts as reporter_consts +import ray.dashboard.utils as dashboard_utils +import ray.experimental.internal_kv as internal_kv +from ray._private.metrics_agent import Gauge, MetricsAgent, Record +from ray._private.ray_constants import DEBUG_AUTOSCALING_STATUS +from ray.core.generated import reporter_pb2, reporter_pb2_grpc +from ray.dashboard import k8s_utils from ray.util.debug import log_once -import psutil logger = logging.getLogger(__name__) diff --git a/dashboard/modules/reporter/reporter_consts.py b/dashboard/modules/reporter/reporter_consts.py index 7d23a2179..369446212 100644 --- a/dashboard/modules/reporter/reporter_consts.py +++ b/dashboard/modules/reporter/reporter_consts.py @@ -1,4 +1,4 @@ -import ray.ray_constants as ray_constants +import ray._private.ray_constants as ray_constants REPORTER_PREFIX = "RAY_REPORTER:" # The reporter will report its statistics this often (milliseconds). diff --git a/dashboard/modules/reporter/reporter_head.py b/dashboard/modules/reporter/reporter_head.py index 1ab8d2e04..013d08438 100644 --- a/dashboard/modules/reporter/reporter_head.py +++ b/dashboard/modules/reporter/reporter_head.py @@ -1,25 +1,25 @@ import json import logging -import yaml import os + import aiohttp.web +import yaml import ray -import ray.dashboard.utils as dashboard_utils -import ray.dashboard.optional_utils as dashboard_optional_utils -import ray.experimental.internal_kv as internal_kv import ray._private.services import ray._private.utils -from ray.ray_constants import ( - GLOBAL_GRPC_OPTIONS, - DEBUG_AUTOSCALING_STATUS, - DEBUG_AUTOSCALING_STATUS_LEGACY, - DEBUG_AUTOSCALING_ERROR, -) -from ray.core.generated import reporter_pb2 -from ray.core.generated import reporter_pb2_grpc +import ray.dashboard.optional_utils as dashboard_optional_utils +import ray.dashboard.utils as dashboard_utils +import ray.experimental.internal_kv as internal_kv from ray._private.gcs_pubsub import GcsAioResourceUsageSubscriber from ray._private.metrics_agent import PrometheusServiceDiscoveryWriter +from ray._private.ray_constants import ( + DEBUG_AUTOSCALING_ERROR, + DEBUG_AUTOSCALING_STATUS, + DEBUG_AUTOSCALING_STATUS_LEGACY, + GLOBAL_GRPC_OPTIONS, +) +from ray.core.generated import reporter_pb2, reporter_pb2_grpc from ray.dashboard.datacenter import DataSource logger = logging.getLogger(__name__) diff --git a/dashboard/modules/reporter/tests/test_reporter.py b/dashboard/modules/reporter/tests/test_reporter.py index 3f67e003e..d8a54acdc 100644 --- a/dashboard/modules/reporter/tests/test_reporter.py +++ b/dashboard/modules/reporter/tests/test_reporter.py @@ -1,23 +1,24 @@ +import logging import os import sys -import logging -from mock import patch -import requests import time import pytest +import requests + import ray -from ray import ray_constants +from mock import patch +from ray._private import ray_constants +from ray._private.test_utils import ( + RayTestTimeoutException, + fetch_prometheus, + format_web_url, + wait_for_condition, + wait_until_server_available, +) +from ray.dashboard.modules.reporter.reporter_agent import ReporterAgent from ray.dashboard.tests.conftest import * # noqa from ray.dashboard.utils import Bunch -from ray.dashboard.modules.reporter.reporter_agent import ReporterAgent -from ray._private.test_utils import ( - format_web_url, - RayTestTimeoutException, - wait_until_server_available, - wait_for_condition, - fetch_prometheus, -) try: import prometheus_client @@ -148,7 +149,7 @@ def test_prometheus_physical_stats_record(enable_test_module, shutdown_only): def test_case_ip_correct(): components_dict, metric_names, metric_samples = fetch_prometheus(prom_addresses) - raylet_proc = ray.worker._global_node.all_processes[ + raylet_proc = ray._private.worker._global_node.all_processes[ ray_constants.PROCESS_TYPE_RAYLET ][0] raylet_pid = None diff --git a/dashboard/modules/runtime_env/runtime_env_consts.py b/dashboard/modules/runtime_env/runtime_env_consts.py index b4bfc31ab..eb721ce29 100644 --- a/dashboard/modules/runtime_env/runtime_env_consts.py +++ b/dashboard/modules/runtime_env/runtime_env_consts.py @@ -1,4 +1,4 @@ -import ray.ray_constants as ray_constants +import ray._private.ray_constants as ray_constants RUNTIME_ENV_RETRY_TIMES = ray_constants.env_integer("RUNTIME_ENV_RETRY_TIMES", 3) diff --git a/dashboard/modules/snapshot/snapshot_head.py b/dashboard/modules/snapshot/snapshot_head.py index 58c3a57a7..7f1acb3e3 100644 --- a/dashboard/modules/snapshot/snapshot_head.py +++ b/dashboard/modules/snapshot/snapshot_head.py @@ -1,29 +1,24 @@ import asyncio import concurrent.futures -from typing import Any, Dict, List, Optional import hashlib +import json +from typing import Any, Dict, List, Optional + +import aiohttp.web import ray -from ray import ray_constants -from ray.core.generated import gcs_service_pb2 -from ray.core.generated import gcs_pb2 -from ray.core.generated import gcs_service_pb2_grpc +import ray.dashboard.optional_utils as dashboard_optional_utils +import ray.dashboard.utils as dashboard_utils +from ray._private import ray_constants +from ray.core.generated import gcs_pb2, gcs_service_pb2, gcs_service_pb2_grpc +from ray.dashboard.modules.job.common import JOB_ID_METADATA_KEY, JobInfoStorageClient from ray.experimental.internal_kv import ( - _internal_kv_initialized, _internal_kv_get, + _internal_kv_initialized, _internal_kv_list, ) -import ray.dashboard.utils as dashboard_utils -import ray.dashboard.optional_utils as dashboard_optional_utils -from ray.runtime_env import RuntimeEnv from ray.job_submission import JobInfo -from ray.dashboard.modules.job.common import ( - JobInfoStorageClient, - JOB_ID_METADATA_KEY, -) - -import json -import aiohttp.web +from ray.runtime_env import RuntimeEnv routes = dashboard_optional_utils.ClassMethodRouteTable @@ -195,8 +190,8 @@ class APIHead(dashboard_utils.DashboardHeadModule): # Conditionally import serve to prevent ModuleNotFoundError from serve # dependencies when only ray[default] is installed (#17712) try: - from ray.serve.controller import SNAPSHOT_KEY as SERVE_SNAPSHOT_KEY from ray.serve.constants import SERVE_CONTROLLER_NAME + from ray.serve.controller import SNAPSHOT_KEY as SERVE_SNAPSHOT_KEY except Exception: return {} diff --git a/dashboard/modules/snapshot/tests/test_actors.py b/dashboard/modules/snapshot/tests/test_actors.py index 36a146d25..5d55c2dac 100644 --- a/dashboard/modules/snapshot/tests/test_actors.py +++ b/dashboard/modules/snapshot/tests/test_actors.py @@ -1,15 +1,14 @@ +import logging import os import sys -import logging -import requests import time -import ray + import pytest +import requests + +import ray +from ray._private.test_utils import format_web_url, wait_until_server_available from ray.dashboard.tests.conftest import * # noqa -from ray._private.test_utils import ( - format_web_url, - wait_until_server_available, -) logger = logging.getLogger(__name__) @@ -59,7 +58,7 @@ def test_kill_actor_gcs(ray_start_with_dashboard): @ray.remote class Actor: def f(self): - ray.worker.show_in_dashboard("test") + ray._private.worker.show_in_dashboard("test") return os.getpid() def loop(self): diff --git a/dashboard/modules/test/test_agent.py b/dashboard/modules/test/test_agent.py index 0dd39909a..2bf21c4f2 100644 --- a/dashboard/modules/test/test_agent.py +++ b/dashboard/modules/test/test_agent.py @@ -2,11 +2,11 @@ import logging import aiohttp.web -import ray.dashboard.utils as dashboard_utils -import ray.dashboard.optional_utils as dashboard_optional_utils -import ray.dashboard.modules.test.test_utils as test_utils import ray.dashboard.modules.test.test_consts as test_consts -from ray.ray_constants import env_bool +import ray.dashboard.modules.test.test_utils as test_utils +import ray.dashboard.optional_utils as dashboard_optional_utils +import ray.dashboard.utils as dashboard_utils +from ray._private.ray_constants import env_bool logger = logging.getLogger(__name__) routes = dashboard_optional_utils.ClassMethodRouteTable diff --git a/dashboard/modules/test/test_head.py b/dashboard/modules/test/test_head.py index fc1b1798f..4fdd5e27f 100644 --- a/dashboard/modules/test/test_head.py +++ b/dashboard/modules/test/test_head.py @@ -1,14 +1,14 @@ -import time import logging +import time import aiohttp.web -import ray.dashboard.utils as dashboard_utils -import ray.dashboard.optional_utils as dashboard_optional_utils -import ray.dashboard.modules.test.test_utils as test_utils import ray.dashboard.modules.test.test_consts as test_consts +import ray.dashboard.modules.test.test_utils as test_utils +import ray.dashboard.optional_utils as dashboard_optional_utils +import ray.dashboard.utils as dashboard_utils +from ray._private.ray_constants import env_bool from ray.dashboard.datacenter import DataSource -from ray.ray_constants import env_bool logger = logging.getLogger(__name__) routes = dashboard_optional_utils.ClassMethodRouteTable diff --git a/dashboard/optional_utils.py b/dashboard/optional_utils.py index c741e73be..afed9f170 100644 --- a/dashboard/optional_utils.py +++ b/dashboard/optional_utils.py @@ -14,16 +14,17 @@ import traceback from collections import namedtuple from typing import Any, Callable +from aiohttp.web import Response + import ray import ray.dashboard.consts as dashboard_consts -from aiohttp.web import Response +from ray._private.ray_constants import env_bool # All third-party dependencies that are not included in the minimal Ray # installation must be included in this file. This allows us to determine if # the agent has the necessary dependencies to be started. from ray.dashboard.optional_deps import PathLike, RouteDef, aiohttp, hdrs from ray.dashboard.utils import CustomEncoder, to_google_style -from ray.ray_constants import env_bool try: create_task = asyncio.create_task diff --git a/dashboard/tests/test_dashboard.py b/dashboard/tests/test_dashboard.py index 9df44c866..afabde47a 100644 --- a/dashboard/tests/test_dashboard.py +++ b/dashboard/tests/test_dashboard.py @@ -1,43 +1,47 @@ -import os -import sys -import copy -import json -import time -import logging import asyncio -import ipaddress -import subprocess import collections +import copy +import ipaddress +import json +import logging +import os +import subprocess +import sys +import time import numpy as np -import ray -import psutil import pytest import requests -from ray import ray_constants +import ray +import ray.dashboard.consts as dashboard_consts +import ray.dashboard.modules +import ray.dashboard.utils as dashboard_utils +from ray._private import ray_constants +from ray._private.ray_constants import ( + DEBUG_AUTOSCALING_ERROR, + DEBUG_AUTOSCALING_STATUS_LEGACY, +) from ray._private.test_utils import ( format_web_url, + get_error_message, + init_error_pubsub, + run_string_as_driver, wait_for_condition, wait_until_server_available, - run_string_as_driver, wait_until_succeeded_without_exception, - init_error_pubsub, - get_error_message, ) -from ray.ray_constants import DEBUG_AUTOSCALING_STATUS_LEGACY, DEBUG_AUTOSCALING_ERROR from ray.dashboard import dashboard -import ray.dashboard.consts as dashboard_consts -import ray.dashboard.utils as dashboard_utils -import ray.dashboard.modules from ray.dashboard.modules.dashboard_sdk import DEFAULT_DASHBOARD_ADDRESS - -from ray.experimental.state.exception import ServerUnavailable -from ray.experimental.state.common import ListApiOptions, StateResource from ray.experimental.state.api import StateApiClient +from ray.experimental.state.common import ListApiOptions, StateResource +from ray.experimental.state.exception import ServerUnavailable + +import psutil try: import aiohttp.web + import ray.dashboard.optional_utils as dashboard_optional_utils routes = dashboard_optional_utils.ClassMethodRouteTable @@ -106,7 +110,7 @@ def test_basic(ray_start_with_dashboard): gcs_client = make_gcs_client(address_info) ray.experimental.internal_kv._initialize_internal_kv(gcs_client) - all_processes = ray.worker._global_node.all_processes + all_processes = ray._private.worker._global_node.all_processes assert ray_constants.PROCESS_TYPE_DASHBOARD in all_processes assert ray_constants.PROCESS_TYPE_REPORTER not in all_processes dashboard_proc_info = all_processes[ray_constants.PROCESS_TYPE_DASHBOARD][0] @@ -151,7 +155,7 @@ def test_raylet_and_agent_share_fate(shutdown_only): ray.init(include_dashboard=True) p = init_error_pubsub() - node = ray.worker._global_node + node = ray._private.worker._global_node all_processes = node.all_processes raylet_proc_info = all_processes[ray_constants.PROCESS_TYPE_RAYLET][0] raylet_proc = psutil.Process(raylet_proc_info.process.pid) @@ -174,7 +178,7 @@ def test_raylet_and_agent_share_fate(shutdown_only): ray.shutdown() ray.init(include_dashboard=True) - all_processes = ray.worker._global_node.all_processes + all_processes = ray._private.worker._global_node.all_processes raylet_proc_info = all_processes[ray_constants.PROCESS_TYPE_RAYLET][0] raylet_proc = psutil.Process(raylet_proc_info.process.pid) wait_for_condition(lambda: search_agent(raylet_proc.children())) @@ -195,7 +199,7 @@ def test_agent_report_unexpected_raylet_death(shutdown_only): ray.init(include_dashboard=True) p = init_error_pubsub() - node = ray.worker._global_node + node = ray._private.worker._global_node all_processes = node.all_processes raylet_proc_info = all_processes[ray_constants.PROCESS_TYPE_RAYLET][0] raylet_proc = psutil.Process(raylet_proc_info.process.pid) @@ -229,7 +233,7 @@ def test_agent_report_unexpected_raylet_death_large_file(shutdown_only): ray.init(include_dashboard=True) p = init_error_pubsub() - node = ray.worker._global_node + node = ray._private.worker._global_node all_processes = node.all_processes raylet_proc_info = all_processes[ray_constants.PROCESS_TYPE_RAYLET][0] raylet_proc = psutil.Process(raylet_proc_info.process.pid) @@ -788,7 +792,7 @@ def test_dashboard_port_conflict(ray_start_with_dashboard): def test_gcs_check_alive(fast_gcs_failure_detection, ray_start_with_dashboard): assert wait_until_server_available(ray_start_with_dashboard["webui_url"]) is True - all_processes = ray.worker._global_node.all_processes + all_processes = ray._private.worker._global_node.all_processes dashboard_info = all_processes[ray_constants.PROCESS_TYPE_DASHBOARD][0] dashboard_proc = psutil.Process(dashboard_info.process.pid) gcs_server_info = all_processes[ray_constants.PROCESS_TYPE_GCS_SERVER][0] diff --git a/doc/source/serve/doc_code/visualize_dag_during_deployment.py b/doc/source/serve/doc_code/visualize_dag_during_deployment.py index 1c9d66c4c..469b05bf2 100644 --- a/doc/source/serve/doc_code/visualize_dag_during_deployment.py +++ b/doc/source/serve/doc_code/visualize_dag_during_deployment.py @@ -27,11 +27,11 @@ with InputNode() as user_input: dag = combine.bind(m1_output, m2_output, kwargs_output=user_input[2]) # Partial DAG visualization -graph = ray.dag.vis_utils.dag_to_dot(m1_output) +graph = ray.dag.vis_utils._dag_to_dot(m1_output) to_string = graph.to_string() print(to_string) # Entire DAG visualization -graph = ray.dag.vis_utils.dag_to_dot(dag) +graph = ray.dag.vis_utils._dag_to_dot(dag) to_string = graph.to_string() print(to_string) diff --git a/python/ray/__init__.py b/python/ray/__init__.py index 6a903aefa..616579166 100644 --- a/python/ray/__init__.py +++ b/python/ray/__init__.py @@ -1,3 +1,4 @@ +# isort: skip_file import logging import os @@ -132,13 +133,13 @@ from ray._raylet import ( # noqa: E402 _config = _Config() -from ray.state import ( # noqa: E402 +from ray._private.state import ( # noqa: E402 nodes, timeline, cluster_resources, available_resources, ) -from ray.worker import ( # noqa: E402,F401 +from ray._private.worker import ( # noqa: E402,F401 LOCAL_MODE, SCRIPT_MODE, WORKER_MODE, @@ -156,7 +157,6 @@ from ray.worker import ( # noqa: E402,F401 shutdown, wait, ) -import ray.internal # noqa: E402 # We import ray.actor because some code is run in actor.py which initializes # some functions in the worker. @@ -168,6 +168,7 @@ from ray.cross_language import java_function, java_actor_class # noqa: E402 from ray.runtime_context import get_runtime_context # noqa: E402 from ray import autoscaler # noqa:E402 from ray import data # noqa: E402,F401 +from ray import internal # noqa: E402,F401 from ray import util # noqa: E402 from ray import _private # noqa: E402,F401 from ray import workflow # noqa: E402,F401 @@ -175,6 +176,29 @@ from ray import workflow # noqa: E402,F401 # We import ClientBuilder so that modules can inherit from `ray.ClientBuilder`. from ray.client_builder import client, ClientBuilder # noqa: E402 + +class _DeprecationWrapper(object): + def __init__(self, name, real_worker): + self._name = name + self._real_worker = real_worker + self._warned = set() + + def __getattr__(self, attr): + value = getattr(self._real_worker, attr) + if attr not in self._warned: + self._warned.add(attr) + logger.warning( + f"DeprecationWarning: `ray.{self._name}.{attr}` is a private " + "attribute and access will be removed in a future Ray version." + ) + return value + + +# TODO(ekl) remove this entirely after 3rd party libraries are all migrated. +worker = _DeprecationWrapper("worker", ray._private.worker) +ray_constants = _DeprecationWrapper("ray_constants", ray._private.ray_constants) +serialization = _DeprecationWrapper("serialization", ray._private.serialization) + __all__ = [ "__version__", "_config", diff --git a/python/ray/_private/client_mode_hook.py b/python/ray/_private/client_mode_hook.py index b7dc9c267..2d58726ab 100644 --- a/python/ray/_private/client_mode_hook.py +++ b/python/ray/_private/client_mode_hook.py @@ -1,7 +1,7 @@ import os +import threading from contextlib import contextmanager from functools import partial, wraps -import threading # Attr set on func defs to mark they have been converted to client mode. RAY_CLIENT_MODE_ATTR = "__ray_client_mode_key__" @@ -93,10 +93,10 @@ def client_mode_hook(func: callable = None, *, auto_init: bool): if func is None: return partial(client_mode_hook, auto_init=auto_init) - from ray.util.client import ray - @wraps(func) def wrapper(*args, **kwargs): + from ray.util.client import ray + if client_mode_should_convert(auto_init=auto_init): # Legacy code # we only convert init function if RAY_CLIENT_MODE=1 @@ -141,10 +141,11 @@ def client_mode_wrap(func): side, this function is wrapped in a task to facilitate interaction with the GCS. """ - from ray.util.client import ray @wraps(func) def wrapper(*args, **kwargs): + from ray.util.client import ray + # Directly pass this through since `client_mode_wrap` is for # Placement Group APIs if client_mode_should_convert(auto_init=True): diff --git a/python/ray/external_storage.py b/python/ray/_private/external_storage.py similarity index 99% rename from python/ray/external_storage.py rename to python/ray/_private/external_storage.py index 85c6f8d76..f15eeb1c9 100644 --- a/python/ray/external_storage.py +++ b/python/ray/_private/external_storage.py @@ -1,15 +1,15 @@ import abc import logging import os -import shutil import random +import shutil import time import urllib from collections import namedtuple -from typing import List, IO, Tuple, Optional +from typing import IO, List, Optional, Tuple import ray -from ray.ray_constants import DEFAULT_OBJECT_PREFIX +from ray._private.ray_constants import DEFAULT_OBJECT_PREFIX from ray._raylet import ObjectRef ParsedURL = namedtuple("ParsedURL", "base_url, offset, size") @@ -87,7 +87,7 @@ class ExternalStorage(metaclass=abc.ABCMeta): HEADER_LENGTH = 24 def _get_objects_from_store(self, object_refs): - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker # Since the object should always exist in the plasma store before # spilling, it can directly get the object from the local plasma # store. @@ -98,7 +98,7 @@ class ExternalStorage(metaclass=abc.ABCMeta): def _put_object_to_store( self, metadata, data_size, file_like, object_ref, owner_address ): - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker.core_worker.put_file_like_object( metadata, data_size, file_like, object_ref, owner_address ) @@ -370,7 +370,7 @@ class ExternalStorageRayStorageImpl(ExternalStorage): # Override the storage config for unit tests. _force_storage_for_testing: Optional[str] = None, ): - from ray.internal import storage + from ray._private import storage if _force_storage_for_testing: storage._reset() diff --git a/python/ray/_private/function_manager.py b/python/ray/_private/function_manager.py index 901e7e013..a036e2fe6 100644 --- a/python/ray/_private/function_manager.py +++ b/python/ray/_private/function_manager.py @@ -1,36 +1,33 @@ import dis import hashlib -import os import importlib import inspect import json import logging +import os import sys -import time -from typing import Optional import threading +import time import traceback -from collections import ( - namedtuple, - defaultdict, -) +from collections import defaultdict, namedtuple +from typing import Optional import ray import ray._private.profiling as profiling -from ray import ray_constants from ray import cloudpickle as pickle -from ray._raylet import PythonFunctionDescriptor, JobID +from ray._private import ray_constants +from ray._private.inspect_util import ( + is_class_method, + is_function_or_method, + is_static_method, +) +from ray._private.ray_constants import KV_NAMESPACE_FUNCTION_TABLE from ray._private.utils import ( check_oversized_function, ensure_str, format_error_message, ) -from ray.ray_constants import KV_NAMESPACE_FUNCTION_TABLE -from ray.util.inspect import ( - is_function_or_method, - is_class_method, - is_static_method, -) +from ray._raylet import JobID, PythonFunctionDescriptor FunctionExecutionInfo = namedtuple( "FunctionExecutionInfo", ["function", "function_name", "max_calls"] diff --git a/python/ray/_private/gcs_utils.py b/python/ray/_private/gcs_utils.py index be9c193a3..7f67c2d40 100644 --- a/python/ray/_private/gcs_utils.py +++ b/python/ray/_private/gcs_utils.py @@ -1,37 +1,36 @@ import enum import logging -from typing import List, Optional -from functools import wraps import time +from functools import wraps +from typing import List, Optional import grpc import ray -from ray import ray_constants +from ray._private import ray_constants +from ray.core.generated import gcs_service_pb2, gcs_service_pb2_grpc from ray.core.generated.common_pb2 import ErrorType -from ray.core.generated import gcs_service_pb2_grpc -from ray.core.generated import gcs_service_pb2 from ray.core.generated.gcs_pb2 import ( ActorTableData, - GcsNodeInfo, AvailableResources, - JobTableData, - JobConfig, ErrorTableData, GcsEntry, - ResourceUsageBatchData, - ResourcesData, + GcsNodeInfo, + JobConfig, + JobTableData, ObjectTableData, + PlacementGroupTableData, ProfileTableData, - TablePrefix, - TablePubsub, + PubSubMessage, ResourceDemand, ResourceLoad, ResourceMap, + ResourcesData, ResourceTableData, - PubSubMessage, + ResourceUsageBatchData, + TablePrefix, + TablePubsub, WorkerTableData, - PlacementGroupTableData, ) logger = logging.getLogger(__name__) diff --git a/python/ray/_private/import_thread.py b/python/ray/_private/import_thread.py index ce6a4d57d..f8d368cc4 100644 --- a/python/ray/_private/import_thread.py +++ b/python/ray/_private/import_thread.py @@ -1,14 +1,14 @@ -from collections import defaultdict +import logging import threading import traceback +from collections import defaultdict import grpc import ray -from ray import ray_constants -from ray import cloudpickle as pickle import ray._private.profiling as profiling -import logging +from ray import cloudpickle as pickle +from ray._private import ray_constants logger = logging.getLogger(__name__) diff --git a/python/ray/util/inspect.py b/python/ray/_private/inspect_util.py similarity index 100% rename from python/ray/util/inspect.py rename to python/ray/_private/inspect_util.py diff --git a/python/ray/internal/internal_api.py b/python/ray/_private/internal_api.py similarity index 95% rename from python/ray/internal/internal_api.py rename to python/ray/_private/internal_api.py index d52ba43e6..c64733d73 100644 --- a/python/ray/internal/internal_api.py +++ b/python/ray/_private/internal_api.py @@ -1,10 +1,10 @@ import ray -import ray._private.services as services -import ray.worker import ray._private.profiling as profiling +import ray._private.services as services import ray._private.utils as utils -from ray import ray_constants -from ray.state import GlobalState +import ray._private.worker +from ray._private import ray_constants +from ray._private.state import GlobalState from ray._raylet import GcsClientOptions __all__ = ["free", "global_gc"] @@ -14,7 +14,7 @@ MAX_MESSAGE_LENGTH = ray._config.max_grpc_message_size() def global_gc(): """Trigger gc.collect() on all workers in the cluster.""" - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker.core_worker.global_gc() @@ -45,8 +45,7 @@ def memory_summary( def get_store_stats(state, node_manager_address=None, node_manager_port=None): """Returns a formatted string describing memory usage in the cluster.""" - from ray.core.generated import node_manager_pb2 - from ray.core.generated import node_manager_pb2_grpc + from ray.core.generated import node_manager_pb2, node_manager_pb2_grpc # We can ask any Raylet for the global memory info, that Raylet internally # asks all nodes in the cluster for memory stats. @@ -85,8 +84,7 @@ def node_stats( ): """Returns NodeStats object describing memory usage in the cluster.""" - from ray.core.generated import node_manager_pb2 - from ray.core.generated import node_manager_pb2_grpc + from ray.core.generated import node_manager_pb2, node_manager_pb2_grpc # We can ask any Raylet for the global memory info. assert node_manager_address is not None and node_manager_port is not None @@ -192,7 +190,7 @@ def free(object_refs: list, local_only: bool = False): local_only: Whether only deleting the list of objects in local object store or all object stores. """ - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker if isinstance(object_refs, ray.ObjectRef): object_refs = [object_refs] diff --git a/python/ray/_private/log_monitor.py b/python/ray/_private/log_monitor.py index c04c7bba7..441ed2654 100644 --- a/python/ray/_private/log_monitor.py +++ b/python/ray/_private/log_monitor.py @@ -9,12 +9,10 @@ import re import shutil import time import traceback -from typing import Set, List +from typing import Callable, List, Set -from typing import Callable - -import ray.ray_constants as ray_constants import ray._private.gcs_pubsub as gcs_pubsub +import ray._private.ray_constants as ray_constants import ray._private.services as services import ray._private.utils from ray._private.gcs_pubsub import GcsPublisher diff --git a/python/ray/_private/metrics_agent.py b/python/ray/_private/metrics_agent.py index 1f4173a01..e1e6b1189 100644 --- a/python/ray/_private/metrics_agent.py +++ b/python/ray/_private/metrics_agent.py @@ -7,25 +7,24 @@ import traceback from collections import namedtuple from typing import List +from opencensus.metrics.export.value import ValueDouble from opencensus.stats import aggregation from opencensus.stats import measure as measure_module from opencensus.stats import stats as stats_module -from opencensus.stats.view import View -from opencensus.stats.view_data import ViewData from opencensus.stats.aggregation_data import ( CountAggregationData, DistributionAggregationData, LastValueAggregationData, ) -from opencensus.metrics.export.value import ValueDouble +from opencensus.stats.view import View +from opencensus.stats.view_data import ViewData from opencensus.tags import tag_key as tag_key_module from opencensus.tags import tag_map as tag_map_module from opencensus.tags import tag_value as tag_value_module import ray -from ray._private.gcs_utils import GcsClient - import ray._private.prometheus_exporter as prometheus_exporter +from ray._private.gcs_utils import GcsClient from ray.core.generated.metrics_pb2 import Metric logger = logging.getLogger(__name__) @@ -211,7 +210,7 @@ class PrometheusServiceDiscoveryWriter(threading.Thread): gcs_client_options = ray._raylet.GcsClientOptions.from_gcs_address(gcs_address) self.gcs_address = gcs_address - ray.state.state._initialize_global_state(gcs_client_options) + ray._private.state.state._initialize_global_state(gcs_client_options) self.temp_dir = temp_dir self.default_service_discovery_flush_period = 5 super().__init__() @@ -247,13 +246,15 @@ class PrometheusServiceDiscoveryWriter(threading.Thread): def get_target_file_name(self): return os.path.join( - self.temp_dir, ray.ray_constants.PROMETHEUS_SERVICE_DISCOVERY_FILE + self.temp_dir, ray._private.ray_constants.PROMETHEUS_SERVICE_DISCOVERY_FILE ) def get_temp_file_name(self): return os.path.join( self.temp_dir, - "{}_{}".format("tmp", ray.ray_constants.PROMETHEUS_SERVICE_DISCOVERY_FILE), + "{}_{}".format( + "tmp", ray._private.ray_constants.PROMETHEUS_SERVICE_DISCOVERY_FILE + ), ) def run(self): diff --git a/python/ray/node.py b/python/ray/_private/node.py similarity index 99% rename from python/ray/node.py rename to python/ray/_private/node.py index 4552ead61..19f614132 100644 --- a/python/ray/node.py +++ b/python/ray/_private/node.py @@ -14,19 +14,19 @@ import tempfile import threading import time import traceback - -from typing import Optional, Dict from collections import defaultdict +from typing import Dict, Optional + from filelock import FileLock import ray -import ray.ray_constants as ray_constants +import ray._private.ray_constants as ray_constants import ray._private.services import ray._private.utils -from ray.internal import storage +from ray._private import storage from ray._private.gcs_utils import GcsClient from ray._private.resource_spec import ResourceSpec -from ray._private.utils import try_to_create_directory, try_to_symlink, open_log +from ray._private.utils import open_log, try_to_create_directory, try_to_symlink # Logger for this module. It should be configured at the entry point # into the program using Ray. Ray configures it by default automatically @@ -1118,7 +1118,7 @@ class Node: gcs_options = ray._raylet.GcsClientOptions.from_gcs_address( self.gcs_address ) - global_state = ray.state.GlobalState() + global_state = ray._private.state.GlobalState() global_state._initialize_global_state(gcs_options) new_config = global_state.get_system_config() assert self._config.items() <= new_config.items(), ( @@ -1448,7 +1448,7 @@ class Node: object_spilling_config = self._config.get("object_spilling_config", {}) if object_spilling_config: object_spilling_config = json.loads(object_spilling_config) - from ray import external_storage + from ray._private import external_storage storage = external_storage.setup_external_storage( object_spilling_config, self.session_name @@ -1493,7 +1493,7 @@ class Node: self._config["is_external_storage_type_fs"] = is_external_storage_type_fs # Validate external storage usage. - from ray import external_storage + from ray._private import external_storage external_storage.setup_external_storage(deserialized_config, self.session_name) external_storage.reset_external_storage() diff --git a/python/ray/_private/parameter.py b/python/ray/_private/parameter.py index f3e236da1..ee6dc163d 100644 --- a/python/ray/_private/parameter.py +++ b/python/ray/_private/parameter.py @@ -1,10 +1,10 @@ import logging import os -from typing import Optional, List, Dict +from typing import Dict, List, Optional import numpy as np -import ray.ray_constants as ray_constants +import ray._private.ray_constants as ray_constants logger = logging.getLogger(__name__) @@ -19,7 +19,7 @@ class RayParams: It will also kill these processes when Python exits. redis_port: The port that the primary Redis shard should listen to. If None, then it will fall back to - ray.ray_constants.DEFAULT_PORT, or a random port if the default is + ray._private.ray_constants.DEFAULT_PORT, or a random port if the default is not available. redis_shard_ports: A list of the ports to use for the non-primary Redis shards. If None, then it will fall back to the ports right after diff --git a/python/ray/_private/profiling.py b/python/ray/_private/profiling.py index d97c1b33f..09f2f1b48 100644 --- a/python/ray/_private/profiling.py +++ b/python/ray/_private/profiling.py @@ -1,6 +1,7 @@ -import ray import os +import ray + class _NullLogSpan: """A log span context manager that does nothing""" @@ -49,7 +50,7 @@ def profile(event_type, extra_data=None): """ if not PROFILING_ENABLED: return NULL_LOG_SPAN - worker = ray.worker.global_worker - if worker.mode == ray.worker.LOCAL_MODE: + worker = ray._private.worker.global_worker + if worker.mode == ray._private.worker.LOCAL_MODE: return NULL_LOG_SPAN return worker.core_worker.profile_event(event_type.encode("ascii"), extra_data) diff --git a/python/ray/ray_constants.py b/python/ray/_private/ray_constants.py similarity index 100% rename from python/ray/ray_constants.py rename to python/ray/_private/ray_constants.py diff --git a/python/ray/_private/ray_logging.py b/python/ray/_private/ray_logging.py index 54ca682d9..cdf805c74 100644 --- a/python/ray/_private/ray_logging.py +++ b/python/ray/_private/ray_logging.py @@ -3,7 +3,6 @@ import os import sys import threading from logging.handlers import RotatingFileHandler - from typing import Callable import ray @@ -192,9 +191,12 @@ def get_worker_log_file_name(worker_type, job_id=None): worker_name = "io_worker" # Make sure these values are set already. - assert ray.worker._global_node is not None - assert ray.worker.global_worker is not None - filename = f"{worker_name}-" f"{binary_to_hex(ray.worker.global_worker.worker_id)}-" + assert ray._private.worker._global_node is not None + assert ray._private.worker.global_worker is not None + filename = ( + f"{worker_name}-" + f"{binary_to_hex(ray._private.worker.global_worker.worker_id)}-" + ) if job_id: filename += f"{job_id}-" filename += f"{os.getpid()}" @@ -270,12 +272,12 @@ def setup_and_get_worker_interceptor_logger( worker_name = "io_worker" # Make sure these values are set already. - assert ray.worker._global_node is not None - assert ray.worker.global_worker is not None + assert ray._private.worker._global_node is not None + assert ray._private.worker.global_worker is not None filename = ( - f"{ray.worker._global_node.get_session_dir_path()}/logs/" + f"{ray._private.worker._global_node.get_session_dir_path()}/logs/" f"{worker_name}-" - f"{binary_to_hex(ray.worker.global_worker.worker_id)}-" + f"{binary_to_hex(ray._private.worker.global_worker.worker_id)}-" f"{job_id}-{os.getpid()}.{file_extension}" ) handler = StandardFdRedirectionRotatingFileHandler( diff --git a/python/ray/_private/ray_option_utils.py b/python/ray/_private/ray_option_utils.py index 9163ec65f..07fc1a4b6 100644 --- a/python/ray/_private/ray_option_utils.py +++ b/python/ray/_private/ray_option_utils.py @@ -1,12 +1,13 @@ """Manage, parse and validate options for Ray tasks, actors and actor methods.""" -from typing import Dict, Any, Callable, Tuple, Union, Optional from dataclasses import dataclass +from typing import Any, Callable, Dict, Optional, Tuple, Union + +import ray._private.ray_constants as ray_constants from ray.util.placement_group import PlacementGroup from ray.util.scheduling_strategies import ( - PlacementGroupSchedulingStrategy, NodeAffinitySchedulingStrategy, + PlacementGroupSchedulingStrategy, ) -import ray.ray_constants as ray_constants @dataclass diff --git a/python/ray/_private/resource_spec.py b/python/ray/_private/resource_spec.py index 644ab0ea0..0bb9fd7d9 100644 --- a/python/ray/_private/resource_spec.py +++ b/python/ray/_private/resource_spec.py @@ -1,14 +1,14 @@ import importlib.util -from collections import namedtuple import logging import os import re import subprocess import sys +from collections import namedtuple from typing import Optional import ray -import ray.ray_constants as ray_constants +import ray._private.ray_constants as ray_constants try: import GPUtil @@ -158,7 +158,8 @@ class ResourceSpec( node_ip_address = ray.util.get_node_ip_address() # Automatically create a node id resource on each node. This is - # queryable with ray.state.node_ids() and ray.state.current_node_id(). + # queryable with ray._private.state.node_ids() and + # ray._private.state.current_node_id(). resources[NODE_ID_PREFIX + node_ip_address] = 1.0 num_cpus = self.num_cpus diff --git a/python/ray/_private/runtime_env/packaging.py b/python/ray/_private/runtime_env/packaging.py index c784fb6bd..a9816476c 100644 --- a/python/ray/_private/runtime_env/packaging.py +++ b/python/ray/_private/runtime_env/packaging.py @@ -11,6 +11,10 @@ from zipfile import ZipFile from filelock import FileLock +from ray._private.ray_constants import ( + RAY_RUNTIME_ENV_URI_PIN_EXPIRATION_S_DEFAULT, + RAY_RUNTIME_ENV_URI_PIN_EXPIRATION_S_ENV_VAR, +) from ray._private.gcs_utils import GcsAioClient from ray._private.thirdparty.pathspec import PathSpec from ray.experimental.internal_kv import ( @@ -18,10 +22,6 @@ from ray.experimental.internal_kv import ( _internal_kv_put, _pin_runtime_env_uri, ) -from ray.ray_constants import ( - RAY_RUNTIME_ENV_URI_PIN_EXPIRATION_S_DEFAULT, - RAY_RUNTIME_ENV_URI_PIN_EXPIRATION_S_ENV_VAR, -) default_logger = logging.getLogger(__name__) diff --git a/python/ray/serialization.py b/python/ray/_private/serialization.py similarity index 97% rename from python/ray/serialization.py rename to python/ray/_private/serialization.py index cce212da9..7c7ed8adf 100644 --- a/python/ray/serialization.py +++ b/python/ray/_private/serialization.py @@ -2,43 +2,43 @@ import logging import threading import traceback -import ray.cloudpickle as pickle -from ray import ray_constants import ray._private.utils +import ray.cloudpickle as pickle +from ray._private import ray_constants from ray._private.gcs_utils import ErrorType -from ray.core.generated.common_pb2 import RayErrorInfo -from ray.exceptions import ( - RayError, - PlasmaObjectNotAvailable, - RayTaskError, - RayActorError, - TaskCancelledError, - WorkerCrashedError, - ObjectLostError, - ObjectFetchTimedOutError, - ReferenceCountingAssertionError, - OwnerDiedError, - ObjectReconstructionFailedError, - ObjectReconstructionFailedMaxAttemptsExceededError, - ObjectReconstructionFailedLineageEvictedError, - RaySystemError, - RuntimeEnvSetupError, - TaskPlacementGroupRemoved, - ActorPlacementGroupRemoved, - LocalRayletDiedError, - TaskUnschedulableError, - ActorUnschedulableError, -) from ray._raylet import ( + MessagePackSerializedObject, + MessagePackSerializer, + Pickle5SerializedObject, + Pickle5Writer, + RawSerializedObject, split_buffer, unpack_pickle5_buffers, - Pickle5Writer, - Pickle5SerializedObject, - MessagePackSerializer, - MessagePackSerializedObject, - RawSerializedObject, ) -from ray import serialization_addons +from ray.core.generated.common_pb2 import RayErrorInfo +from ray.exceptions import ( + ActorPlacementGroupRemoved, + ActorUnschedulableError, + LocalRayletDiedError, + ObjectFetchTimedOutError, + ObjectLostError, + ObjectReconstructionFailedError, + ObjectReconstructionFailedLineageEvictedError, + ObjectReconstructionFailedMaxAttemptsExceededError, + OwnerDiedError, + PlasmaObjectNotAvailable, + RayActorError, + RayError, + RaySystemError, + RayTaskError, + ReferenceCountingAssertionError, + RuntimeEnvSetupError, + TaskCancelledError, + TaskPlacementGroupRemoved, + TaskUnschedulableError, + WorkerCrashedError, +) +from ray.util import serialization_addons logger = logging.getLogger(__name__) @@ -62,7 +62,7 @@ def _object_ref_deserializer(binary, call_site, owner_address, object_status): # to 'self' here instead, but this function is itself pickled # somewhere, which causes an error. if owner_address: - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker.check_connected() context = worker.get_serialization_context() outer_id = context.get_outer_object_ref() @@ -79,7 +79,7 @@ def _object_ref_deserializer(binary, call_site, owner_address, object_status): def _actor_handle_deserializer(serialized_obj): # If this actor handle was stored in another object, then tell the # core worker. - context = ray.worker.global_worker.get_serialization_context() + context = ray._private.worker.global_worker.get_serialization_context() outer_id = context.get_outer_object_ref() return ray.actor.ActorHandle._deserialization_helper(serialized_obj, outer_id) @@ -96,7 +96,7 @@ class SerializationContext: self._thread_local = threading.local() def actor_handle_reducer(obj): - ray.worker.global_worker.check_connected() + ray._private.worker.global_worker.check_connected() serialized, actor_handle_id = obj._serialization_helper() # Update ref counting for the actor handle self.add_contained_object_ref(actor_handle_id) @@ -105,7 +105,7 @@ class SerializationContext: self._register_cloudpickle_reducer(ray.actor.ActorHandle, actor_handle_reducer) def object_ref_reducer(obj): - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker.check_connected() self.add_contained_object_ref(obj) obj, owner_address, object_status = worker.core_worker.serialize_object_ref( @@ -171,7 +171,9 @@ class SerializationContext: # cloudpickle directly or captured in a remote function/actor), # then pin the object for the lifetime of this worker by adding # a local reference that won't ever be removed. - ray.worker.global_worker.core_worker.add_object_ref_reference(object_ref) + ray._private.worker.global_worker.core_worker.add_object_ref_reference( + object_ref + ) def _deserialize_pickle5_data(self, data): try: diff --git a/python/ray/_private/services.py b/python/ray/_private/services.py index 5c2beb2c3..398c1ee8a 100644 --- a/python/ray/_private/services.py +++ b/python/ray/_private/services.py @@ -4,10 +4,9 @@ import errno import io import json import logging +import mmap import multiprocessing import os -from pathlib import Path -import mmap import random import shutil import signal @@ -15,20 +14,21 @@ import socket import subprocess import sys import time -from typing import Optional, List import uuid - -# Ray modules -import ray -import ray.ray_constants as ray_constants -from ray._raylet import GcsClientOptions -from ray._private.gcs_utils import GcsClient -from ray.core.generated.common_pb2 import Language +from pathlib import Path +from typing import List, Optional # Import psutil and colorama after ray so the packaged version is used. import colorama import psutil +# Ray modules +import ray +import ray._private.ray_constants as ray_constants +from ray._private.gcs_utils import GcsClient +from ray._raylet import GcsClientOptions +from ray.core.generated.common_pb2 import Language + resource = None if sys.platform != "win32": import resource @@ -382,7 +382,7 @@ def wait_for_node( the node appears in the client table. """ gcs_options = GcsClientOptions.from_gcs_address(gcs_address) - global_state = ray.state.GlobalState() + global_state = ray._private.state.GlobalState() global_state._initialize_global_state(gcs_options) start_time = time.time() while time.time() - start_time < timeout: @@ -401,7 +401,7 @@ def get_node_to_connect_for_driver( redis_address, gcs_address, node_ip_address, redis_password=None ): # Get node table from global state accessor. - global_state = ray.state.GlobalState() + global_state = ray._private.state.GlobalState() gcs_options = _get_gcs_client_options(redis_address, redis_password, gcs_address) global_state._initialize_global_state(gcs_options) return global_state.get_node_to_connect_for_driver(node_ip_address) @@ -437,12 +437,12 @@ def remaining_processes_alive(): Exception: An exception is raised if the processes were not started by ray.init(). """ - if ray.worker._global_node is None: + if ray._private.worker._global_node is None: raise RuntimeError( "This process is not in a position to determine " "whether all processes are alive or not." ) - return ray.worker._global_node.remaining_processes_alive() + return ray._private.worker._global_node.remaining_processes_alive() def canonicalize_bootstrap_address(addr: str): @@ -540,8 +540,8 @@ def node_ip_address_from_perspective(address: str): def get_node_ip_address(address="8.8.8.8:53"): - if ray.worker._global_node is not None: - return ray.worker._global_node.node_ip_address + if ray._private.worker._global_node is not None: + return ray._private.worker._global_node.node_ip_address if sys.platform == "darwin" or sys.platform == "win32": # Due to the mac osx/windows firewall, # we use loopback ip as the ip address @@ -2185,7 +2185,7 @@ def start_ray_client_server( """ root_ray_dir = Path(__file__).resolve().parents[1] setup_worker_path = os.path.join( - root_ray_dir, "workers", ray_constants.SETUP_WORKER_FILENAME + root_ray_dir, "_private", "workers", ray_constants.SETUP_WORKER_FILENAME ) ray_client_server_host = ( diff --git a/python/ray/_private/signature.py b/python/ray/_private/signature.py index 9d3fcf9f0..c9c843bf7 100644 --- a/python/ray/_private/signature.py +++ b/python/ray/_private/signature.py @@ -1,8 +1,8 @@ import inspect -from inspect import Parameter import logging +from inspect import Parameter -from ray.util.inspect import is_cython +from ray._private.inspect_util import is_cython # Logger for this module. It should be configured at the entry point # into the program using Ray. Ray provides a default configuration at diff --git a/python/ray/state.py b/python/ray/_private/state.py similarity index 99% rename from python/ray/state.py rename to python/ray/_private/state.py index 6a30248a3..d9d965664 100644 --- a/python/ray/state.py +++ b/python/ray/_private/state.py @@ -1,18 +1,17 @@ -from collections import defaultdict import json import logging +from collections import defaultdict + +from google.protobuf.json_format import MessageToDict import ray - import ray._private.gcs_utils as gcs_utils -from ray.util.annotations import DeveloperAPI -from google.protobuf.json_format import MessageToDict -from ray.core.generated import gcs_pb2 from ray._private.client_mode_hook import client_mode_hook -from ray._private.utils import decode, binary_to_hex, hex_to_binary from ray._private.resource_spec import NODE_ID_PREFIX - +from ray._private.utils import binary_to_hex, decode, hex_to_binary from ray._raylet import GlobalStateAccessor +from ray.core.generated import gcs_pb2 +from ray.util.annotations import DeveloperAPI logger = logging.getLogger(__name__) diff --git a/python/ray/internal/storage.py b/python/ray/_private/storage.py similarity index 99% rename from python/ray/internal/storage.py rename to python/ray/_private/storage.py index d93650e4f..2cfcbdac0 100644 --- a/python/ray/internal/storage.py +++ b/python/ray/_private/storage.py @@ -1,9 +1,9 @@ -from typing import List, Optional, TYPE_CHECKING -from pathlib import Path -import os -import urllib import importlib +import os import re +import urllib +from pathlib import Path +from typing import TYPE_CHECKING, List, Optional from ray._private.client_mode_hook import client_mode_hook @@ -260,7 +260,7 @@ class KVClient: FileNotFoundError if the given path is not found. NotADirectoryError if the given path isn't a valid directory. """ - from pyarrow.fs import FileSelector, LocalFileSystem, FileType + from pyarrow.fs import FileSelector, FileType, LocalFileSystem full_path = self._resolve_path(path) selector = FileSelector(full_path, recursive=False) diff --git a/python/ray/_private/test_utils.py b/python/ray/_private/test_utils.py index 3d0270852..7f5ea3b63 100644 --- a/python/ray/_private/test_utils.py +++ b/python/ray/_private/test_utils.py @@ -1,43 +1,39 @@ import asyncio +import fnmatch import functools import io -import fnmatch +import logging +import math import os import pathlib +import socket import subprocess import sys +import tempfile import time import timeit -import socket -import math import traceback -from typing import Optional, Any, List, Dict -from contextlib import redirect_stdout, redirect_stderr, contextmanager +from contextlib import contextmanager, redirect_stderr, redirect_stdout +from typing import Any, Dict, List, Optional -import yaml -import logging -import tempfile import grpc -from grpc._channel import _InactiveRpcError import numpy as np +import psutil # We must import psutil after ray because we bundle it with ray. +import yaml +from grpc._channel import _InactiveRpcError import ray -import ray._private.services -import ray._private.utils import ray._private.gcs_utils as gcs_utils import ray._private.memory_monitor as memory_monitor -from ray._raylet import GcsClientOptions, GlobalStateAccessor -from ray.core.generated import gcs_pb2 -from ray.core.generated import node_manager_pb2 -from ray.core.generated import node_manager_pb2_grpc -from ray._private.gcs_pubsub import ( - GcsErrorSubscriber, - GcsLogSubscriber, -) +import ray._private.services +import ray._private.utils +from ray._private.gcs_pubsub import GcsErrorSubscriber, GcsLogSubscriber +from ray._private.internal_api import memory_summary from ray._private.tls_utils import generate_self_signed_tls_certs -from ray.util.queue import Queue, _QueueActor, Empty +from ray._raylet import GcsClientOptions, GlobalStateAccessor +from ray.core.generated import gcs_pb2, node_manager_pb2, node_manager_pb2_grpc from ray.scripts.scripts import main as ray_main -from ray.internal.internal_api import memory_summary +from ray.util.queue import Empty, Queue, _QueueActor try: from prometheus_client.parser import text_string_to_metric_families @@ -47,9 +43,6 @@ except (ImportError, ModuleNotFoundError): raise ModuleNotFoundError("`prometheus_client` not found") -import psutil # We must import psutil after ray because we bundle it with ray. - - class RayTestTimeoutException(Exception): """Exception used to identify timeouts from test utilities.""" @@ -282,7 +275,7 @@ def wait_for_num_actors(num_actors, state=None, timeout=10): len( [ _ - for _ in ray.state.actors().values() + for _ in ray._private.state.actors().values() if state is None or _["State"] == state ] ) @@ -328,11 +321,11 @@ def wait_for_num_nodes(num_nodes: int, timeout_s: int): def kill_actor_and_wait_for_failure(actor, timeout=10, retry_interval_ms=100): actor_id = actor._actor_id.hex() - current_num_restarts = ray.state.actors(actor_id)["NumRestarts"] + current_num_restarts = ray._private.state.actors(actor_id)["NumRestarts"] ray.kill(actor) start = time.time() while time.time() - start <= timeout: - actor_status = ray.state.actors(actor_id) + actor_status = ray._private.state.actors(actor_id) if ( actor_status["State"] == convert_actor_state(gcs_utils.ActorTableData.DEAD) or actor_status["NumRestarts"] > current_num_restarts @@ -617,7 +610,8 @@ def get_other_nodes(cluster, exclude_head=False): return [ node for node in cluster.list_all_nodes() - if node._raylet_socket_name != ray.worker._global_node._raylet_socket_name + if node._raylet_socket_name + != ray._private.worker._global_node._raylet_socket_name and (exclude_head is False or node.head is False) ] @@ -629,7 +623,7 @@ def get_non_head_nodes(cluster): def init_error_pubsub(): """Initialize error info pub/sub""" - s = GcsErrorSubscriber(address=ray.worker.global_worker.gcs_client.address) + s = GcsErrorSubscriber(address=ray._private.worker.global_worker.gcs_client.address) s.subscribe() return s @@ -657,7 +651,7 @@ def get_error_message(subscriber, num=1e6, error_type=None, timeout=20): def init_log_pubsub(): """Initialize log pub/sub""" - s = GcsLogSubscriber(address=ray.worker.global_worker.gcs_client.address) + s = GcsLogSubscriber(address=ray._private.worker.global_worker.gcs_client.address) s.subscribe() return s @@ -1007,7 +1001,7 @@ def monitor_memory_usage( """ return self.peak_memory_usage, self.peak_top_n_memory_usage - current_node_ip = ray.worker.global_worker.node_ip_address + current_node_ip = ray._private.worker.global_worker.node_ip_address # Schedule the actor on the current node. memory_monitor_actor = MemoryMonitorActor.options( resources={f"node:{current_node_ip}": 0.001} @@ -1160,8 +1154,8 @@ def get_and_run_node_killer( alive_nodes += 1 return alive_nodes - head_node_ip = ray.worker.global_worker.node_ip_address - head_node_id = ray.worker.global_worker.current_node_id.hex() + head_node_ip = ray._private.worker.global_worker.node_ip_address + head_node_id = ray._private.worker.global_worker.current_node_id.hex() # Schedule the actor on the current node. node_killer = NodeKillerActor.options( resources={f"node:{head_node_ip}": 0.001}, @@ -1306,7 +1300,9 @@ def simulate_storage(storage_type, root=None): yield "file://" + root elif storage_type == "s3": import uuid + from moto import mock_s3 + from ray.tests.mock_s3_server import start_service, stop_process @contextmanager diff --git a/python/ray/_private/usage/usage_lib.py b/python/ray/_private/usage/usage_lib.py index 72ab34a5d..1f503f06a 100644 --- a/python/ray/_private/usage/usage_lib.py +++ b/python/ray/_private/usage/usage_lib.py @@ -56,8 +56,8 @@ import requests import yaml import ray +import ray._private.ray_constants as ray_constants import ray._private.usage.usage_constants as usage_constant -import ray.ray_constants as ray_constants from ray.experimental.internal_kv import _internal_kv_initialized, _internal_kv_put logger = logging.getLogger(__name__) @@ -184,8 +184,8 @@ def record_library_usage(library_usage: str): # it can be reported if the library is imported from # e.g., API server. if ( - ray.worker.global_worker.mode == ray.SCRIPT_MODE - or ray.worker.global_worker.mode == ray.WORKER_MODE + ray._private.worker.global_worker.mode == ray.SCRIPT_MODE + or ray._private.worker.global_worker.mode == ray.WORKER_MODE ): _put_library_usage(library_usage) @@ -195,13 +195,13 @@ def _put_pre_init_library_usages(): # NOTE: When the lib is imported from a worker, ray should # always be initialized, so there's no need to register the # pre init hook. - if ray.worker.global_worker.mode != ray.SCRIPT_MODE: + if ray._private.worker.global_worker.mode != ray.SCRIPT_MODE: return for library_usage in _recorded_library_usages: _put_library_usage(library_usage) -ray.worker._post_init_hooks.append(_put_pre_init_library_usages) +ray._private.worker._post_init_hooks.append(_put_pre_init_library_usages) def _usage_stats_report_url(): @@ -420,7 +420,7 @@ def get_cluster_status_to_report(gcs_client, num_retries: int) -> ClusterStatusT try: cluster_status = ray._private.utils.internal_kv_get_with_retry( gcs_client, - ray.ray_constants.DEBUG_AUTOSCALING_STATUS, + ray._private.ray_constants.DEBUG_AUTOSCALING_STATUS, namespace=None, num_retries=num_retries, ) diff --git a/python/ray/_private/utils.py b/python/ray/_private/utils.py index f71e3e92f..109338497 100644 --- a/python/ray/_private/utils.py +++ b/python/ray/_private/utils.py @@ -3,6 +3,7 @@ import errno import functools import hashlib import importlib +import inspect import logging import multiprocessing import os @@ -12,28 +13,35 @@ import sys import tempfile import threading import time -from typing import Optional, Sequence, Tuple, Any, Union, Dict import uuid -import grpc import warnings +from inspect import signature +from pathlib import Path +from typing import TYPE_CHECKING, Any, Dict, Optional, Sequence, Tuple, Union + +import grpc +import numpy as np + +# Import psutil after ray so the packaged version is used. +import psutil +from google.protobuf import json_format + +import ray +import ray._private.ray_constants as ray_constants +from ray._private.tls_utils import load_certs_from_env +from ray.core.generated.gcs_pb2 import ErrorTableData +from ray.core.generated.runtime_env_common_pb2 import ( + RuntimeEnvInfo as ProtoRuntimeEnvInfo, +) + +if TYPE_CHECKING: + from ray.runtime_env import RuntimeEnv try: from grpc import aio as aiogrpc except ImportError: from grpc.experimental import aio as aiogrpc -import inspect -from inspect import signature -from pathlib import Path -import numpy as np - -import ray -from ray.core.generated.gcs_pb2 import ErrorTableData -import ray.ray_constants as ray_constants -from ray._private.tls_utils import load_certs_from_env - -# Import psutil after ray so the packaged version is used. -import psutil pwd = None if sys.platform != "win32": @@ -670,7 +678,7 @@ def detect_fate_sharing_support_win32(): import ctypes try: - from ctypes.wintypes import BOOL, DWORD, HANDLE, LPVOID, LPCWSTR + from ctypes.wintypes import BOOL, DWORD, HANDLE, LPCWSTR, LPVOID kernel32 = ctypes.WinDLL("kernel32") kernel32.CreateJobObjectW.argtypes = (LPVOID, LPCWSTR) @@ -752,7 +760,7 @@ def detect_fate_sharing_support_linux(): global linux_prctl if linux_prctl is None and sys.platform.startswith("linux"): try: - from ctypes import c_int, c_ulong, CDLL + from ctypes import CDLL, c_int, c_ulong prctl = CDLL(None).prctl prctl.restype = c_int @@ -1349,3 +1357,78 @@ def check_version_info(cluster_metadata): " Python: " + version_info[1] + "\n" ) raise RuntimeError(error_message) + + +def get_runtime_env_info( + runtime_env: "RuntimeEnv", + *, + is_job_runtime_env: bool = False, + serialize: bool = False, +): + """Create runtime env info from runtime env. + + In the user interface, the argument `runtime_env` contains some fields + which not contained in `ProtoRuntimeEnv` but in `ProtoRuntimeEnvInfo`, + such as `eager_install`. This function will extract those fields from + `RuntimeEnv` and create a new `ProtoRuntimeEnvInfo`, and serialize it. + """ + from ray.runtime_env import RuntimeEnvConfig + + proto_runtime_env_info = ProtoRuntimeEnvInfo() + + proto_runtime_env_info.uris[:] = runtime_env.get_uris() + + # Normally, `RuntimeEnv` should guarantee the accuracy of field eager_install, + # but so far, the internal code has not completely prohibited direct + # modification of fields in RuntimeEnv, so we should check it for insurance. + # TODO(Catch-Bull): overload `__setitem__` for `RuntimeEnv`, change the + # runtime_env of all internal code from dict to RuntimeEnv. + + eager_install = runtime_env.get("eager_install") + if is_job_runtime_env or eager_install is not None: + if eager_install is None: + eager_install = True + elif not isinstance(eager_install, bool): + raise TypeError( + f"eager_install must be a boolean. got {type(eager_install)}" + ) + proto_runtime_env_info.runtime_env_eager_install = eager_install + + runtime_env_config = runtime_env.get("config") + if runtime_env_config is None: + runtime_env_config = RuntimeEnvConfig.default_config() + else: + runtime_env_config = RuntimeEnvConfig.parse_and_validate_runtime_env_config( + runtime_env_config + ) + + proto_runtime_env_info.runtime_env_config.CopyFrom( + runtime_env_config.build_proto_runtime_env_config() + ) + + proto_runtime_env_info.serialized_runtime_env = runtime_env.serialize() + + if not serialize: + return proto_runtime_env_info + + return json_format.MessageToJson(proto_runtime_env_info) + + +def parse_runtime_env(runtime_env: Optional[Union[Dict, "RuntimeEnv"]]): + from ray.runtime_env import RuntimeEnv + + # Parse local pip/conda config files here. If we instead did it in + # .remote(), it would get run in the Ray Client server, which runs on + # a remote node where the files aren't available. + if runtime_env: + if isinstance(runtime_env, dict): + return RuntimeEnv(**(runtime_env or {})) + raise TypeError( + "runtime_env must be dict or RuntimeEnv, ", + f"but got: {type(runtime_env)}", + ) + else: + # Keep the new_runtime_env as None. In .remote(), we need to know + # if runtime_env is None to know whether or not to fall back to the + # runtime_env specified in the @ray.remote decorator. + return None diff --git a/python/ray/worker.py b/python/ray/_private/worker.py similarity index 98% rename from python/ray/worker.py rename to python/ray/_private/worker.py index cbcd57dbc..b74d609dd 100644 --- a/python/ray/worker.py +++ b/python/ray/_private/worker.py @@ -1,4 +1,3 @@ -from contextlib import contextmanager import atexit import faulthandler import functools @@ -15,6 +14,7 @@ import traceback import warnings from abc import ABCMeta, abstractmethod from collections.abc import Mapping +from contextlib import contextmanager from dataclasses import dataclass from typing import ( Any, @@ -31,64 +31,53 @@ from typing import ( overload, ) -# Ray modules -import ray.cloudpickle as pickle -import ray._private.memory_monitor as memory_monitor -import ray.internal.storage as storage -from ray.internal.storage import _load_class -import ray.node -import ray.job_config -import ray._private.parameter -import ray.ray_constants as ray_constants -import ray.remote_function -import ray.serialization as serialization -import ray._private.gcs_utils as gcs_utils -import ray._private.services as services -from ray._private.gcs_pubsub import ( - GcsPublisher, - GcsErrorSubscriber, - GcsLogSubscriber, - GcsFunctionKeySubscriber, -) -from ray._private.runtime_env.py_modules import upload_py_modules_if_needed -from ray._private.runtime_env.working_dir import upload_working_dir_if_needed -from ray._private.runtime_env.constants import RAY_JOB_CONFIG_JSON_ENV_VAR -import ray._private.import_thread as import_thread -from ray.util.tracing.tracing_helper import import_from_string -from ray.util.annotations import PublicAPI, DeveloperAPI, Deprecated -from ray.util.debug import log_once -from ray._private import ray_option_utils -import ray import colorama import setproctitle -import ray.state -from ray import ( - ActorID, - JobID, - ObjectRef, - Language, -) +import ray +import ray._private.gcs_utils as gcs_utils +import ray._private.import_thread as import_thread +import ray._private.memory_monitor as memory_monitor +import ray._private.node +import ray._private.parameter import ray._private.profiling as profiling +import ray._private.ray_constants as ray_constants +import ray._private.serialization as serialization +import ray._private.services as services +import ray._private.state +import ray._private.storage as storage -from ray.exceptions import ( - RaySystemError, - RayError, - RayTaskError, - ObjectStoreFullError, -) -from ray._private.function_manager import FunctionActorManager, make_function_table_key -from ray._private.ray_logging import setup_logger -from ray._private.ray_logging import global_worker_stdstream_dispatcher -from ray._private.utils import check_oversized_function -from ray.util.inspect import is_cython -from ray.experimental.internal_kv import ( - _internal_kv_initialized, - _initialize_internal_kv, - _internal_kv_reset, - _internal_kv_get, -) +# Ray modules +import ray.cloudpickle as pickle +import ray.job_config +import ray.remote_function +from ray import ActorID, JobID, Language, ObjectRef +from ray._private import ray_option_utils from ray._private.client_mode_hook import client_mode_hook +from ray._private.function_manager import FunctionActorManager, make_function_table_key +from ray._private.gcs_pubsub import ( + GcsErrorSubscriber, + GcsFunctionKeySubscriber, + GcsLogSubscriber, + GcsPublisher, +) +from ray._private.inspect_util import is_cython +from ray._private.ray_logging import global_worker_stdstream_dispatcher, setup_logger +from ray._private.runtime_env.constants import RAY_JOB_CONFIG_JSON_ENV_VAR +from ray._private.runtime_env.py_modules import upload_py_modules_if_needed +from ray._private.runtime_env.working_dir import upload_working_dir_if_needed +from ray._private.storage import _load_class +from ray._private.utils import check_oversized_function +from ray.exceptions import ObjectStoreFullError, RayError, RaySystemError, RayTaskError +from ray.experimental.internal_kv import ( + _initialize_internal_kv, + _internal_kv_get, + _internal_kv_initialized, + _internal_kv_reset, +) +from ray.util.annotations import Deprecated, DeveloperAPI, PublicAPI +from ray.util.debug import log_once +from ray.util.tracing.tracing_helper import _import_from_string SCRIPT_MODE = 0 WORKER_MODE = 1 @@ -411,7 +400,7 @@ class Worker: functions outside of this class are considered exposed. Attributes: - node (ray.node.Node): The node this worker is attached to. + node (ray._private.node.Node): The node this worker is attached to. mode: The mode of the worker. One of SCRIPT_MODE, LOCAL_MODE, and WORKER_MODE. cached_functions_to_run: A list of functions to run on all of @@ -878,7 +867,7 @@ def get_resource_ids(): if _mode() == LOCAL_MODE: raise RuntimeError( - "ray.worker.get_resource_ids() currently does not work in local_mode." + "ray._private.worker.get_resource_ids() does not work in local_mode." ) return global_worker.core_worker.resource_ids() @@ -985,7 +974,7 @@ per worker process. """ _global_node = None -"""ray.node.Node: The global node object that is created by ray.init().""" +"""ray._private.node.Node: The global node object that is created by ray.init().""" @PublicAPI @@ -1365,7 +1354,7 @@ def init( # shutdown the node in the ray.shutdown call that happens in the atexit # handler. We still spawn a reaper process in case the atexit handler # isn't called. - _global_node = ray.node.Node( + _global_node = ray._private.node.Node( head=True, shutdown_at_exit=False, spawn_reaper=True, ray_params=ray_params ) else: @@ -1419,7 +1408,7 @@ def init( enable_object_reconstruction=_enable_object_reconstruction, metrics_export_port=_metrics_export_port, ) - _global_node = ray.node.Node( + _global_node = ray._private.node.Node( ray_params, head=False, shutdown_at_exit=False, @@ -1502,7 +1491,7 @@ def shutdown(_exiting_interpreter: bool = False): # We need to reset function actor manager to clear the context global_worker.function_actor_manager = FunctionActorManager(global_worker) # Disconnect global state from GCS. - ray.state.state.disconnect() + ray._private.state.state.disconnect() # Shut down the Ray processes. global _global_node @@ -1547,8 +1536,8 @@ def custom_excepthook(type, value, tb): worker_type = gcs_utils.DRIVER worker_info = {"exception": error_message} - ray.state.state._check_connected() - ray.state.state.add_worker(worker_id, worker_type, worker_info) + ray._private.state.state._check_connected() + ray._private.state.state.add_worker(worker_id, worker_type, worker_info) # Call the normal excepthook. normal_excepthook(type, value, tb) @@ -1747,7 +1736,7 @@ def is_initialized() -> bool: Returns: True if ray.init has already been called and false otherwise. """ - return ray.worker.global_worker.connected + return ray._private.worker.global_worker.connected def connect( @@ -1766,7 +1755,7 @@ def connect( """Connect this worker to the raylet, to Plasma, and to GCS. Args: - node (ray.node.Node): The node to connect. + node (ray._private.node.Node): The node to connect. mode: The mode of the worker. One of SCRIPT_MODE, WORKER_MODE, and LOCAL_MODE. log_to_driver: If true, then output from all of the worker processes on all nodes will be directed to the driver. @@ -1796,7 +1785,7 @@ def connect( worker.gcs_client = node.get_gcs_client() assert worker.gcs_client is not None _initialize_internal_kv(worker.gcs_client) - ray.state.state._initialize_global_state( + ray._private.state.state._initialize_global_state( ray._raylet.GcsClientOptions.from_gcs_address(node.gcs_address) ) worker.gcs_publisher = GcsPublisher(address=worker.gcs_client.address) @@ -1814,7 +1803,7 @@ def connect( else: # This is the code path of driver mode. if job_id is None: - job_id = ray.state.next_job_id() + job_id = ray._private.state.next_job_id() if mode is not SCRIPT_MODE and mode is not LOCAL_MODE and setproctitle: process_name = ray_constants.WORKER_PROCESS_TYPE_IDLE_WORKER @@ -2006,7 +1995,7 @@ def connect( if tracing_hook_val is not None: ray.util.tracing.tracing_helper._global_is_tracing_enabled = True if not getattr(ray, "__traced__", False): - _setup_tracing = import_from_string(tracing_hook_val.decode("utf-8")) + _setup_tracing = _import_from_string(tracing_hook_val.decode("utf-8")) _setup_tracing() ray.__traced__ = True @@ -2045,7 +2034,7 @@ def disconnect(exiting_interpreter=False): except AttributeError: ray_actor = None # This can occur during program termination if ray_actor is not None: - ray_actor.ActorClassMethodMetadata.reset_cache() + ray_actor._ActorClassMethodMetadata.reset_cache() @contextmanager @@ -2189,7 +2178,7 @@ def get( if debugger_breakpoint != b"": frame = sys._getframe().f_back - rdb = ray.util.pdb.connect_ray_pdb( + rdb = ray.util.pdb._connect_ray_pdb( host=None, port=None, patch_stdstreams=False, @@ -2230,10 +2219,12 @@ def put( if _owner is None: serialize_owner_address = None elif isinstance(_owner, ray.actor.ActorHandle): - # Ensure `ray.state.state.global_state_accessor` is not None - ray.state.state._check_connected() + # Ensure `ray._private.state.state.global_state_accessor` is not None + ray._private.state.state._check_connected() owner_address = gcs_utils.ActorTableData.FromString( - ray.state.state.global_state_accessor.get_actor_info(_owner._actor_id) + ray._private.state.state.global_state_accessor.get_actor_info( + _owner._actor_id + ) ).address if len(owner_address.worker_id) == 0: raise RuntimeError(f"{_owner} is not alive, it's worker_id is empty!") @@ -2462,7 +2453,7 @@ def cancel(object_ref: "ray.ObjectRef", *, force: bool = False, recursive: bool Raises: TypeError: This is also raised for actor tasks. """ - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker.check_connected() if not isinstance(object_ref, ray.ObjectRef): @@ -2496,7 +2487,7 @@ def _make_remote(function_or_class, options): if inspect.isclass(function_or_class): ray_option_utils.validate_actor_options(options, in_options=False) - return ray.actor.make_actor(function_or_class, options) + return ray.actor._make_actor(function_or_class, options) raise TypeError( "The @ray.remote decorator must be applied to either a function or a class." diff --git a/python/ray/workers/__init__.py b/python/ray/_private/workers/__init__.py similarity index 100% rename from python/ray/workers/__init__.py rename to python/ray/_private/workers/__init__.py diff --git a/python/ray/workers/default_worker.py b/python/ray/_private/workers/default_worker.py similarity index 92% rename from python/ray/workers/default_worker.py rename to python/ray/_private/workers/default_worker.py index f193cf198..affdf70f6 100644 --- a/python/ray/workers/default_worker.py +++ b/python/ray/_private/workers/default_worker.py @@ -1,17 +1,17 @@ import argparse import base64 import json -import time -import sys import os +import sys +import time import ray -import ray.actor -import ray.node -import ray.ray_constants as ray_constants +import ray._private.node +import ray._private.ray_constants as ray_constants import ray._private.utils +import ray.actor from ray._private.parameter import RayParams -from ray._private.ray_logging import get_worker_log_file_name, configure_log_file +from ray._private.ray_logging import configure_log_file, get_worker_log_file_name parser = argparse.ArgumentParser( description=("Parse addresses for the worker to connect to.") @@ -182,7 +182,7 @@ if __name__ == "__main__": gcs_address=args.gcs_address, ) - node = ray.node.Node( + node = ray._private.node.Node( ray_params, head=False, shutdown_at_exit=False, @@ -194,8 +194,7 @@ if __name__ == "__main__": # connect to raylet. Otherwise we may receive requests before the # external storage is intialized. if mode == ray.RESTORE_WORKER_MODE or mode == ray.SPILL_WORKER_MODE: - from ray import external_storage - from ray.internal import storage + from ray._private import external_storage, storage storage._init_storage(args.storage, is_head=False) if args.object_spilling_config: @@ -207,8 +206,8 @@ if __name__ == "__main__": object_spilling_config, node.session_name ) - ray.worker._global_node = node - ray.worker.connect( + ray._private.worker._global_node = node + ray._private.worker.connect( node, mode=mode, runtime_env_hash=args.runtime_env_hash, @@ -217,7 +216,7 @@ if __name__ == "__main__": ) # Add code search path to sys.path, set load_code_from_local. - core_worker = ray.worker.global_worker.core_worker + core_worker = ray._private.worker.global_worker.core_worker code_search_path = core_worker.get_job_config().code_search_path load_code_from_local = False if code_search_path: @@ -226,7 +225,7 @@ if __name__ == "__main__": if os.path.isfile(p): p = os.path.dirname(p) sys.path.insert(0, p) - ray.worker.global_worker.set_load_code_from_local(load_code_from_local) + ray._private.worker.global_worker.set_load_code_from_local(load_code_from_local) # Setup log file. out_file, err_file = node.get_log_file_handles( @@ -235,7 +234,7 @@ if __name__ == "__main__": configure_log_file(out_file, err_file) if mode == ray.WORKER_MODE: - ray.worker.global_worker.main_loop() + ray._private.worker.global_worker.main_loop() elif mode in [ray.RESTORE_WORKER_MODE, ray.SPILL_WORKER_MODE]: # It is handled by another thread in the C++ core worker. # We just need to keep the worker alive. diff --git a/python/ray/workers/setup_worker.py b/python/ray/_private/workers/setup_worker.py similarity index 94% rename from python/ray/workers/setup_worker.py rename to python/ray/_private/workers/setup_worker.py index 6a722846f..23ba980a5 100644 --- a/python/ray/workers/setup_worker.py +++ b/python/ray/_private/workers/setup_worker.py @@ -1,10 +1,10 @@ import argparse import logging +from ray._private.ray_constants import LOGGER_FORMAT, LOGGER_LEVEL from ray._private.ray_logging import setup_logger from ray._private.runtime_env.context import RuntimeEnvContext from ray.core.generated.common_pb2 import Language -from ray.ray_constants import LOGGER_LEVEL, LOGGER_FORMAT logger = logging.getLogger(__name__) diff --git a/python/ray/_raylet.pyx b/python/ray/_raylet.pyx index 4b505467c..3a9e17991 100644 --- a/python/ray/_raylet.pyx +++ b/python/ray/_raylet.pyx @@ -125,12 +125,12 @@ from ray.exceptions import ( AsyncioActorExit, PendingCallsLimitExceeded, ) -from ray import external_storage +from ray._private import external_storage from ray.util.scheduling_strategies import ( PlacementGroupSchedulingStrategy, NodeAffinitySchedulingStrategy, ) -import ray.ray_constants as ray_constants +import ray._private.ray_constants as ray_constants from ray._private.async_compat import sync_to_async, get_new_event_loop from ray._private.client_mode_hook import disable_client_hook import ray._private.gcs_utils as gcs_utils @@ -396,7 +396,7 @@ cdef prepare_args_internal( c_string put_arg_call_site c_vector[CObjectReference] inlined_refs - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker put_threshold = RayConfig.instance().max_direct_call_object_size() total_inlined = 0 rpc_inline_threshold = RayConfig.instance().task_rpc_inlined_bytes_limit() @@ -499,7 +499,7 @@ cdef execute_task( is_application_level_error[0] = False - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker manager = worker.function_actor_manager actor = None cdef: @@ -637,14 +637,14 @@ cdef execute_task( # We deserialize objects in event loop thread to # prevent segfaults. See #7799 async def deserialize_args(): - return (ray.worker.global_worker + return (ray._private.worker.global_worker .deserialize_objects( metadata_pairs, object_refs)) args = core_worker.run_async_func_in_event_loop( deserialize_args, function_descriptor, name_of_concurrency_group_to_execute) else: - args = ray.worker.global_worker.deserialize_objects( + args = ray._private.worker.global_worker.deserialize_objects( metadata_pairs, object_refs) for arg in args: @@ -664,13 +664,13 @@ cdef execute_task( if is_existing: title = f"{title}::Exiting" next_title = f"{next_title}::Exiting" - with ray.worker._changeproctitle(title, next_title): + with ray._private.worker._changeproctitle(title, next_title): if debugger_breakpoint != b"": ray.util.pdb.set_trace( breakpoint_uuid=debugger_breakpoint) outputs = function_executor(*args, **kwargs) next_breakpoint = ( - ray.worker.global_worker.debugger_breakpoint) + ray._private.worker.global_worker.debugger_breakpoint) if next_breakpoint != b"": # If this happens, the user typed "remote" and # there were no more remote calls left in this @@ -684,7 +684,7 @@ cdef execute_task( "RAY_PDB_CONTINUE_{}".format(next_breakpoint), namespace=ray_constants.KV_NAMESPACE_PDB ) - ray.worker.global_worker.debugger_breakpoint = b"" + ray._private.worker.global_worker.debugger_breakpoint = b"" task_exception = False except AsyncioActorExit as e: exit_current_actor_if_asyncio() @@ -834,7 +834,7 @@ cdef CRayStatus task_execution_handler( "occurred while the worker " "was executing a task.") ray._private.utils.push_error_to_driver( - ray.worker.global_worker, + ray._private.worker.global_worker, "worker_crash", traceback_str, job_id=None) @@ -910,7 +910,7 @@ cdef c_vector[c_string] spill_objects_handler( object_refs_to_spill[i].owner_address() .SerializeAsString()) try: - with ray.worker._changeproctitle( + with ray._private.worker._changeproctitle( ray_constants.WORKER_PROCESS_TYPE_SPILL_WORKER, ray_constants.WORKER_PROCESS_TYPE_SPILL_WORKER_IDLE): urls = external_storage.spill_objects( @@ -923,7 +923,7 @@ cdef c_vector[c_string] spill_objects_handler( "was spilling objects: {}".format(err)) logger.exception(exception_str) ray._private.utils.push_error_to_driver( - ray.worker.global_worker, + ray._private.worker.global_worker, "spill_objects_error", traceback.format_exc() + exception_str, job_id=None) @@ -944,7 +944,7 @@ cdef int64_t restore_spilled_objects_handler( object_refs_to_restore, skip_adding_local_ref=False) try: - with ray.worker._changeproctitle( + with ray._private.worker._changeproctitle( ray_constants.WORKER_PROCESS_TYPE_RESTORE_WORKER, ray_constants.WORKER_PROCESS_TYPE_RESTORE_WORKER_IDLE): bytes_restored = external_storage.restore_spilled_objects( @@ -956,7 +956,7 @@ cdef int64_t restore_spilled_objects_handler( logger.exception(exception_str) if os.getenv("RAY_BACKEND_LOG_LEVEL") == "debug": ray._private.utils.push_error_to_driver( - ray.worker.global_worker, + ray._private.worker.global_worker, "restore_objects_error", traceback.format_exc() + exception_str, job_id=None) @@ -987,7 +987,7 @@ cdef void delete_spilled_objects_handler( assert False, ("This line shouldn't be reachable.") # Delete objects. - with ray.worker._changeproctitle( + with ray._private.worker._changeproctitle( proctitle, original_proctitle): external_storage.delete_spilled_objects(urls) @@ -997,7 +997,7 @@ cdef void delete_spilled_objects_handler( "was deleting spilled objects.") logger.exception(exception_str) ray._private.utils.push_error_to_driver( - ray.worker.global_worker, + ray._private.worker.global_worker, "delete_spilled_objects_error", traceback.format_exc() + exception_str, job_id=None) @@ -1005,7 +1005,7 @@ cdef void delete_spilled_objects_handler( cdef void unhandled_exception_handler(const CRayObject& error) nogil: with gil: - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker data = None metadata = None if error.HasData(): @@ -1035,10 +1035,10 @@ cdef void get_py_stack(c_string* stack_out) nogil: while frame and len(msg_frames) < 4: filename = frame.f_code.co_filename # Decode Ray internal frames to add annotations. - if filename.endswith("ray/worker.py"): + if filename.endswith("_private/worker.py"): if frame.f_code.co_name == "put": msg_frames = ["(put object) "] - elif filename.endswith("ray/workers/default_worker.py"): + elif filename.endswith("_private/workers/default_worker.py"): pass elif filename.endswith("ray/remote_function.py"): # TODO(ekl) distinguish between task return objects and @@ -1048,7 +1048,7 @@ cdef void get_py_stack(c_string* stack_out) nogil: # TODO(ekl) distinguish between actor return objects and # arguments. This can only be done in the core worker. msg_frames = ["(actor call) "] - elif filename.endswith("ray/serialization.py"): + elif filename.endswith("_private/serialization.py"): if frame.f_code.co_name == "id_deserializer": msg_frames = ["(deserialize task arg) "] else: @@ -1071,7 +1071,7 @@ cdef shared_ptr[CBuffer] string_to_buffer(c_string& c_str): cdef void terminate_asyncio_thread() nogil: with gil: - core_worker = ray.worker.global_worker.core_worker + core_worker = ray._private.worker.global_worker.core_worker core_worker.stop_and_join_asyncio_threads_if_exist() @@ -1837,7 +1837,7 @@ cdef class CoreWorker: c_actor_id) cdef make_actor_handle(self, ActorHandleSharedPtr c_actor_handle): - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker.check_connected() manager = worker.function_actor_manager @@ -1858,7 +1858,7 @@ cdef class CoreWorker: actor_method_cpu = 0 # Actor is created by non Python worker. actor_class = manager.load_actor_class( job_id, actor_creation_function_descriptor) - method_meta = ray.actor.ActorClassMethodMetadata.create( + method_meta = ray.actor._ActorClassMethodMetadata.create( actor_class, actor_creation_function_descriptor) return ray.actor.ActorHandle(language, actor_id, method_meta.decorators, @@ -2049,14 +2049,14 @@ cdef class CoreWorker: serialized_object = context.serialize(output) data_size = serialized_object.total_bytes metadata_str = serialized_object.metadata - if ray.worker.global_worker.debugger_get_breakpoint: + if ray._private.worker.global_worker.debugger_get_breakpoint: breakpoint = ( - ray.worker.global_worker.debugger_get_breakpoint) + ray._private.worker.global_worker.debugger_get_breakpoint) metadata_str += ( b"," + ray_constants.OBJECT_METADATA_DEBUG_PREFIX + breakpoint.encode()) # Reset debugging context of this worker. - ray.worker.global_worker.debugger_get_breakpoint = b"" + ray._private.worker.global_worker.debugger_get_breakpoint = b"" metadata = string_to_buffer(metadata_str) contained_id = ObjectRefsToVector( serialized_object.contained_object_refs) @@ -2301,7 +2301,7 @@ cdef void async_callback(shared_ptr[CRayObject] obj, data_metadata_pairs = RayObjectsToDataMetadataPairs( objects_to_deserialize) ids_to_deserialize = [ObjectRef(object_ref.Binary())] - result = ray.worker.global_worker.deserialize_objects( + result = ray._private.worker.global_worker.deserialize_objects( data_metadata_pairs, ids_to_deserialize)[0] py_callback = user_callback diff --git a/python/ray/actor.py b/python/ray/actor.py index 5bb826d8c..54ec7e47b 100644 --- a/python/ray/actor.py +++ b/python/ray/actor.py @@ -1,39 +1,38 @@ import inspect import logging import weakref -from typing import Optional, List, Dict, Any +from typing import Any, Dict, List, Optional -import ray.ray_constants as ray_constants -import ray._raylet +import ray._private.ray_constants as ray_constants import ray._private.signature as signature -from ray.utils import get_runtime_env_info, parse_runtime_env -import ray.worker -from ray.util.annotations import PublicAPI, DeveloperAPI -from ray.util.placement_group import configure_placement_group_based_on_context +import ray._private.worker +import ray._raylet +from ray import ActorClassID, Language, cross_language +from ray._private import ray_option_utils +from ray._private.client_mode_hook import ( + client_mode_convert_actor, + client_mode_hook, + client_mode_should_convert, +) +from ray._private.inspect_util import ( + is_class_method, + is_function_or_method, + is_static_method, +) +from ray._private.utils import get_runtime_env_info, parse_runtime_env +from ray._raylet import PythonFunctionDescriptor +from ray.exceptions import AsyncioActorExit +from ray.util.annotations import DeveloperAPI, PublicAPI +from ray.util.placement_group import _configure_placement_group_based_on_context from ray.util.scheduling_strategies import ( PlacementGroupSchedulingStrategy, SchedulingStrategyT, ) - -from ray import ActorClassID, Language -from ray._raylet import PythonFunctionDescriptor -from ray._private.client_mode_hook import client_mode_hook -from ray._private.client_mode_hook import client_mode_should_convert -from ray._private.client_mode_hook import client_mode_convert_actor -from ray import cross_language -from ray.util.inspect import ( - is_function_or_method, - is_class_method, - is_static_method, -) -from ray.exceptions import AsyncioActorExit from ray.util.tracing.tracing_helper import ( + _inject_tracing_into_class, _tracing_actor_creation, _tracing_actor_method_invocation, - _inject_tracing_into_class, ) -from ray._private import ray_option_utils - logger = logging.getLogger(__name__) @@ -88,6 +87,7 @@ def method(*args, **kwargs): # Create objects to wrap method invocations. This is done so that we can # invoke methods with actor.method.remote() instead of actor.method(). +@PublicAPI class ActorMethod: """A class used to invoke an actor method. @@ -200,7 +200,7 @@ class ActorMethod: ) -class ActorClassMethodMetadata(object): +class _ActorClassMethodMetadata(object): """Metadata for all methods in an actor class. This data can be cached. Attributes: @@ -214,7 +214,7 @@ class ActorClassMethodMetadata(object): each actor method. """ - _cache = {} # This cache will be cleared in ray.worker.disconnect() + _cache = {} # This cache will be cleared in ray._private.worker.disconnect() def __init__(self): class_name = type(self).__name__ @@ -286,7 +286,7 @@ class ActorClassMethodMetadata(object): return self -class ActorClassMetadata: +class _ActorClassMetadata: """Metadata for an actor class. Attributes: @@ -353,18 +353,19 @@ class ActorClassMetadata: self.concurrency_groups = concurrency_groups self.scheduling_strategy = scheduling_strategy self.last_export_session_and_job = None - self.method_meta = ActorClassMethodMetadata.create( + self.method_meta = _ActorClassMethodMetadata.create( modified_class, actor_creation_function_descriptor ) +@PublicAPI class ActorClassInheritanceException(TypeError): pass def _process_option_dict(actor_options): _filled_options = {} - arg_names = set(inspect.getfullargspec(ActorClassMetadata.__init__)[0]) + arg_names = set(inspect.getfullargspec(_ActorClassMetadata.__init__)[0]) for k, v in ray_option_utils.actor_options.items(): if k in arg_names: _filled_options[k] = actor_options.get(k, v.default_value) @@ -372,6 +373,7 @@ def _process_option_dict(actor_options): return _filled_options +@PublicAPI class ActorClass: """An actor class. @@ -479,7 +481,7 @@ class ActorClass: modified_class.__ray_actor_class__ ) - self.__ray_metadata__ = ActorClassMetadata( + self.__ray_metadata__ = _ActorClassMetadata( Language.PYTHON, modified_class, actor_creation_function_descriptor, @@ -500,7 +502,7 @@ class ActorClass: actor_options, ): self = ActorClass.__new__(ActorClass) - self.__ray_metadata__ = ActorClassMetadata( + self.__ray_metadata__ = _ActorClassMetadata( language, None, actor_creation_function_descriptor, @@ -718,7 +720,7 @@ class ActorClass: max_task_retries = actor_options["max_task_retries"] max_pending_calls = actor_options["max_pending_calls"] - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker.check_connected() # Check whether the name is already taken. @@ -806,7 +808,7 @@ class ActorClass: actor_placement_resources = resources.copy() actor_placement_resources["CPU"] += 1 if meta.is_cross_language: - creation_args = cross_language.format_args(worker, args, kwargs) + creation_args = cross_language._format_args(worker, args, kwargs) else: function_signature = meta.method_meta.signatures["__init__"] creation_args = signature.flatten_args(function_signature, args, kwargs) @@ -831,7 +833,7 @@ class ActorClass: placement_group_capture_child_tasks = ( worker.should_capture_child_tasks_in_placement_group ) - placement_group = configure_placement_group_based_on_context( + placement_group = _configure_placement_group_based_on_context( placement_group_capture_child_tasks, placement_group_bundle_index, resources, @@ -883,7 +885,7 @@ class ActorClass: if meta.language == Language.CPP: func_name = meta.actor_creation_function_descriptor.function_name meta.actor_creation_function_descriptor = ( - cross_language.get_function_descriptor_for_actor_method( + cross_language._get_function_descriptor_for_actor_method( meta.language, meta.actor_creation_function_descriptor, func_name, @@ -944,6 +946,7 @@ class ActorClass: ) +@PublicAPI class ActorHandle: """A handle to an actor. @@ -1022,8 +1025,8 @@ class ActorHandle: def __del__(self): # Mark that this actor handle has gone out of scope. Once all actor # handles are out of scope, the actor will exit. - if ray.worker: - worker = ray.worker.global_worker + if ray._private.worker: + worker = ray._private.worker.global_worker if worker.connected and hasattr(worker, "core_worker"): worker.core_worker.remove_actor_handle_reference(self._ray_actor_id) @@ -1054,13 +1057,13 @@ class ActorHandle: object_refs: A list of object refs returned by the remote actor method. """ - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker args = args or [] kwargs = kwargs or {} if self._ray_is_cross_language: - list_args = cross_language.format_args(worker, args, kwargs) - function_descriptor = cross_language.get_function_descriptor_for_actor_method( # noqa: E501 + list_args = cross_language._format_args(worker, args, kwargs) + function_descriptor = cross_language._get_function_descriptor_for_actor_method( # noqa: E501 self._ray_actor_language, self._ray_actor_creation_function_descriptor, method_name, @@ -1154,7 +1157,7 @@ class ActorHandle: Returns: A dictionary of the information needed to reconstruct the object. """ - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker.check_connected() if hasattr(worker, "core_worker"): @@ -1188,7 +1191,7 @@ class ActorHandle: to the actor handle. """ - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker.check_connected() if hasattr(worker, "core_worker"): @@ -1217,7 +1220,7 @@ class ActorHandle: return ActorHandle._deserialization_helper, state -def modify_class(cls): +def _modify_class(cls): # cls has been modified. if hasattr(cls, "__ray_actor_class__"): return cls @@ -1236,7 +1239,7 @@ def modify_class(cls): __ray_actor_class__ = cls # The original actor class def __ray_terminate__(self): - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker if worker.mode != ray.LOCAL_MODE: ray.actor.exit_actor() @@ -1256,8 +1259,8 @@ def modify_class(cls): return Class -def make_actor(cls, actor_options): - Class = modify_class(cls) +def _make_actor(cls, actor_options): + Class = _modify_class(cls) _inject_tracing_into_class(Class) if "max_restarts" in actor_options: @@ -1275,6 +1278,7 @@ def make_actor(cls, actor_options): ) +@PublicAPI def exit_actor(): """Intentionally exit the current actor. @@ -1285,13 +1289,13 @@ def exit_actor(): Exception: An exception is raised if this is a driver or this worker is not an actor. """ - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker if worker.mode == ray.WORKER_MODE and not worker.actor_id.is_nil(): # Intentionally disconnect the core worker from the raylet so the # raylet won't push an error message to the driver. - ray.worker.disconnect() + ray._private.worker.disconnect() # Disconnect global state from GCS. - ray.state.state.disconnect() + ray._private.state.state.disconnect() # In asyncio actor mode, we can't raise SystemExit because it will just # quit the asycnio event loop thread, not the main thread. Instead, we diff --git a/python/ray/air/util/check_ingest.py b/python/ray/air/util/check_ingest.py index df70d1617..bd0ba705e 100755 --- a/python/ray/air/util/check_ingest.py +++ b/python/ray/air/util/check_ingest.py @@ -1,14 +1,15 @@ #!/usr/bin/env python -import time -import numpy as np -from typing import Optional import sys +import time +from typing import Optional + +import numpy as np import ray from ray import train -from ray.data.preprocessors import Chain, BatchMapper from ray.air.config import DatasetConfig +from ray.data.preprocessors import BatchMapper, Chain from ray.train.data_parallel_trainer import DataParallelTrainer from ray.util.annotations import DeveloperAPI @@ -150,7 +151,7 @@ if __name__ == "__main__": try: print( "Memory stats at end of ingest:\n\n{}".format( - ray.internal.internal_api.memory_summary(stats_only=True) + ray._private.internal_api.memory_summary(stats_only=True) ) ) except Exception: diff --git a/python/ray/autoscaler/_private/_kubernetes/config.py b/python/ray/autoscaler/_private/_kubernetes/config.py index a86ae7633..fc70d2b9d 100644 --- a/python/ray/autoscaler/_private/_kubernetes/config.py +++ b/python/ray/autoscaler/_private/_kubernetes/config.py @@ -6,7 +6,7 @@ import re from kubernetes import client from kubernetes.client.rest import ApiException -import ray.ray_constants as ray_constants +import ray._private.ray_constants as ray_constants from ray.autoscaler._private._kubernetes import auth_api, core_api, log_prefix logger = logging.getLogger(__name__) diff --git a/python/ray/autoscaler/_private/aws/node_provider.py b/python/ray/autoscaler/_private/aws/node_provider.py index b60cd2da2..29cad9dbf 100644 --- a/python/ray/autoscaler/_private/aws/node_provider.py +++ b/python/ray/autoscaler/_private/aws/node_provider.py @@ -8,7 +8,7 @@ from typing import Any, Dict, List import botocore from boto3.resources.base import ServiceResource -import ray.ray_constants as ray_constants +import ray._private.ray_constants as ray_constants from ray.autoscaler._private.aws.cloudwatch.cloudwatch_helper import ( CLOUDWATCH_AGENT_INSTALLED_AMI_TAG, CLOUDWATCH_AGENT_INSTALLED_TAG, diff --git a/python/ray/autoscaler/_private/commands.py b/python/ray/autoscaler/_private/commands.py index 1de65ebdb..77a47c143 100644 --- a/python/ray/autoscaler/_private/commands.py +++ b/python/ray/autoscaler/_private/commands.py @@ -20,6 +20,7 @@ import yaml import ray import ray._private.services as services from ray._private.usage import usage_lib +from ray._private.worker import global_worker # type: ignore from ray.autoscaler._private import subprocess_output_util as cmd_output_util from ray.autoscaler._private.autoscaler import AutoscalerSummary from ray.autoscaler._private.cli_logger import cf, cli_logger @@ -70,7 +71,6 @@ from ray.autoscaler.tags import ( ) from ray.experimental.internal_kv import _internal_kv_put from ray.util.debug import log_once -from ray.worker import global_worker # type: ignore try: # py3 from shlex import quote diff --git a/python/ray/autoscaler/_private/constants.py b/python/ray/autoscaler/_private/constants.py index 05832cf92..4b6bf039c 100644 --- a/python/ray/autoscaler/_private/constants.py +++ b/python/ray/autoscaler/_private/constants.py @@ -1,7 +1,7 @@ import os import sys -from ray.ray_constants import ( # noqa F401 +from ray._private.ray_constants import ( # noqa F401 AUTOSCALER_RESOURCE_REQUEST_CHANNEL, DEFAULT_OBJECT_STORE_MAX_MEMORY_BYTES, DEFAULT_OBJECT_STORE_MEMORY_PROPORTION, diff --git a/python/ray/autoscaler/_private/fake_multi_node/node_provider.py b/python/ray/autoscaler/_private/fake_multi_node/node_provider.py index db01bab28..c5bac12b9 100644 --- a/python/ray/autoscaler/_private/fake_multi_node/node_provider.py +++ b/python/ray/autoscaler/_private/fake_multi_node/node_provider.py @@ -12,6 +12,7 @@ from typing import Any, Dict, Optional import yaml import ray +from ray._private.ray_constants import DEFAULT_PORT from ray.autoscaler._private.fake_multi_node.command_runner import ( FakeDockerCommandRunner, ) @@ -26,7 +27,6 @@ from ray.autoscaler.tags import ( TAG_RAY_NODE_STATUS, TAG_RAY_USER_NODE_TYPE, ) -from ray.ray_constants import DEFAULT_PORT logger = logging.getLogger(__name__) @@ -326,7 +326,7 @@ class FakeMultiNodeProvider(NodeProvider): "RAY_OVERRIDE_RESOURCES": json.dumps(resources), }, ) - node = ray.node.Node( + node = ray._private.node.Node( ray_params, head=False, shutdown_at_exit=False, spawn_reaper=False ) self._nodes[next_id] = { diff --git a/python/ray/autoscaler/_private/kuberay/run_autoscaler.py b/python/ray/autoscaler/_private/kuberay/run_autoscaler.py index 38813f33c..a5a25d45a 100644 --- a/python/ray/autoscaler/_private/kuberay/run_autoscaler.py +++ b/python/ray/autoscaler/_private/kuberay/run_autoscaler.py @@ -4,7 +4,7 @@ import subprocess import time import ray -from ray import ray_constants +from ray._private import ray_constants from ray._private.ray_logging import setup_component_logger from ray._private.services import get_node_ip_address from ray.autoscaler._private.kuberay.autoscaling_config import AutoscalingConfigProducer @@ -56,7 +56,9 @@ def _setup_logging() -> None: logging_level=ray_constants.LOGGER_LEVEL, # info logging_format=ray_constants.LOGGER_FORMAT, log_dir=os.path.join( - ray._private.utils.get_ray_temp_dir(), ray.node.SESSION_LATEST, "logs" + ray._private.utils.get_ray_temp_dir(), + ray._private.node.SESSION_LATEST, + "logs", ), filename=ray_constants.MONITOR_LOG_FILE_NAME, # monitor.log max_bytes=ray_constants.LOGGING_ROTATE_BYTES, diff --git a/python/ray/autoscaler/_private/legacy_info_string.py b/python/ray/autoscaler/_private/legacy_info_string.py index b88928c1c..830078345 100644 --- a/python/ray/autoscaler/_private/legacy_info_string.py +++ b/python/ray/autoscaler/_private/legacy_info_string.py @@ -1,7 +1,7 @@ import logging +from ray._private.ray_constants import DEBUG_AUTOSCALING_STATUS_LEGACY from ray.experimental.internal_kv import _internal_kv_initialized, _internal_kv_put -from ray.ray_constants import DEBUG_AUTOSCALING_STATUS_LEGACY """This file provides legacy support for the old info string in order to ensure the dashboard's `api/cluster_status` does not break backwards diff --git a/python/ray/autoscaler/_private/load_metrics.py b/python/ray/autoscaler/_private/load_metrics.py index a2918ead3..01f0c12ae 100644 --- a/python/ray/autoscaler/_private/load_metrics.py +++ b/python/ray/autoscaler/_private/load_metrics.py @@ -6,7 +6,7 @@ from typing import Dict, List import numpy as np -import ray.ray_constants +import ray._private.ray_constants from ray._private.gcs_utils import PlacementGroupTableData from ray.autoscaler._private.constants import ( AUTOSCALER_MAX_RESOURCE_DEMAND_VECTOR_SIZE, @@ -282,11 +282,12 @@ class LoadMetrics: for key in total_resources: if key in ["memory", "object_store_memory"]: total = ( - total_resources[key] * ray.ray_constants.MEMORY_RESOURCE_UNIT_BYTES + total_resources[key] + * ray._private.ray_constants.MEMORY_RESOURCE_UNIT_BYTES ) available = ( available_resources[key] - * ray.ray_constants.MEMORY_RESOURCE_UNIT_BYTES + * ray._private.ray_constants.MEMORY_RESOURCE_UNIT_BYTES ) usage_dict[key] = (total - available, total) else: diff --git a/python/ray/autoscaler/_private/monitor.py b/python/ray/autoscaler/_private/monitor.py index 2eacc0632..305027e81 100644 --- a/python/ray/autoscaler/_private/monitor.py +++ b/python/ray/autoscaler/_private/monitor.py @@ -13,8 +13,8 @@ from multiprocessing.synchronize import Event from typing import Any, Callable, Dict, Optional, Union import ray +import ray._private.ray_constants as ray_constants import ray._private.utils -import ray.ray_constants as ray_constants from ray._private.gcs_pubsub import GcsPublisher from ray._private.gcs_utils import GcsClient from ray._private.ray_logging import setup_component_logger @@ -155,7 +155,7 @@ class Monitor: if redis_password is not None: logger.warning("redis_password has been deprecated.") # Set the redis client and mode so _internal_kv works for autoscaler. - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker gcs_client = GcsClient(address=gcs_address) if monitor_ip: @@ -319,7 +319,9 @@ class Monitor: """Fetches resource requests from the internal KV and updates load.""" if not _internal_kv_initialized(): return - data = _internal_kv_get(ray.ray_constants.AUTOSCALER_RESOURCE_REQUEST_CHANNEL) + data = _internal_kv_get( + ray._private.ray_constants.AUTOSCALER_RESOURCE_REQUEST_CHANNEL + ) if data: try: resource_request = json.loads(data) diff --git a/python/ray/autoscaler/_private/resource_demand_scheduler.py b/python/ray/autoscaler/_private/resource_demand_scheduler.py index de7e0d333..c78e7eda2 100644 --- a/python/ray/autoscaler/_private/resource_demand_scheduler.py +++ b/python/ray/autoscaler/_private/resource_demand_scheduler.py @@ -14,7 +14,7 @@ from typing import Dict, List, Optional, Tuple import numpy as np -import ray.ray_constants as ray_constants +import ray._private.ray_constants as ray_constants from ray._private.gcs_utils import PlacementGroupTableData from ray.autoscaler._private.constants import AUTOSCALER_CONSERVE_GPU_NODES from ray.autoscaler._private.util import ( diff --git a/python/ray/autoscaler/_private/util.py b/python/ray/autoscaler/_private/util.py index 3c1ccb39f..6be87b6ca 100644 --- a/python/ray/autoscaler/_private/util.py +++ b/python/ray/autoscaler/_private/util.py @@ -12,8 +12,8 @@ from numbers import Number, Real from typing import Any, Dict, List, Optional, Tuple, Union import ray +import ray._private.ray_constants import ray._private.services as services -import ray.ray_constants from ray.autoscaler._private import constants from ray.autoscaler._private.cli_logger import cli_logger from ray.autoscaler._private.docker import validate_docker_config diff --git a/python/ray/autoscaler/command_runner.py b/python/ray/autoscaler/command_runner.py index 897510141..23ce60a3b 100644 --- a/python/ray/autoscaler/command_runner.py +++ b/python/ray/autoscaler/command_runner.py @@ -1,6 +1,9 @@ from typing import Any, Dict, List, Optional, Tuple +from ray.util.annotations import DeveloperAPI + +@DeveloperAPI class CommandRunnerInterface: """Interface to run commands on a remote cluster node. diff --git a/python/ray/autoscaler/node_provider.py b/python/ray/autoscaler/node_provider.py index 5c93cf929..7001a8217 100644 --- a/python/ray/autoscaler/node_provider.py +++ b/python/ray/autoscaler/node_provider.py @@ -4,10 +4,12 @@ from typing import Any, Dict, List, Optional from ray.autoscaler._private.command_runner import DockerCommandRunner, SSHCommandRunner from ray.autoscaler.command_runner import CommandRunnerInterface +from ray.util.annotations import DeveloperAPI logger = logging.getLogger(__name__) +@DeveloperAPI class NodeProvider: """Interface for getting and returning nodes from a Cloud. diff --git a/python/ray/autoscaler/sdk/sdk.py b/python/ray/autoscaler/sdk/sdk.py index abccf99d7..4299fccdf 100644 --- a/python/ray/autoscaler/sdk/sdk.py +++ b/python/ray/autoscaler/sdk/sdk.py @@ -10,8 +10,10 @@ from ray.autoscaler._private import commands from ray.autoscaler._private.cli_logger import cli_logger from ray.autoscaler._private.event_system import CreateClusterEvent # noqa: F401 from ray.autoscaler._private.event_system import global_event_system # noqa: F401 +from ray.util.annotations import DeveloperAPI +@DeveloperAPI def create_or_update_cluster( cluster_config: Union[dict, str], *, @@ -47,6 +49,7 @@ def create_or_update_cluster( ) +@DeveloperAPI def teardown_cluster( cluster_config: Union[dict, str], workers_only: bool = False, @@ -72,6 +75,7 @@ def teardown_cluster( ) +@DeveloperAPI def run_on_cluster( cluster_config: Union[dict, str], *, @@ -117,6 +121,7 @@ def run_on_cluster( ) +@DeveloperAPI def rsync( cluster_config: Union[dict, str], *, @@ -161,6 +166,7 @@ def rsync( ) +@DeveloperAPI def get_head_node_ip(cluster_config: Union[dict, str]) -> str: """Returns head node IP for given configuration file if exists. @@ -178,6 +184,7 @@ def get_head_node_ip(cluster_config: Union[dict, str]) -> str: return commands.get_head_node_ip(config_file) +@DeveloperAPI def get_worker_node_ips(cluster_config: Union[dict, str]) -> List[str]: """Returns worker node IPs for given configuration file. @@ -195,6 +202,7 @@ def get_worker_node_ips(cluster_config: Union[dict, str]) -> List[str]: return commands.get_worker_node_ips(config_file) +@DeveloperAPI def request_resources( num_cpus: Optional[int] = None, bundles: Optional[List[dict]] = None ) -> None: @@ -235,6 +243,7 @@ def request_resources( return commands.request_resources(num_cpus, bundles) +@DeveloperAPI def configure_logging( log_style: Optional[str] = None, color_mode: Optional[str] = None, @@ -265,6 +274,7 @@ def configure_logging( @contextmanager +@DeveloperAPI def _as_config_file(cluster_config: Union[dict, str]) -> Iterator[str]: if isinstance(cluster_config, dict): tmp = tempfile.NamedTemporaryFile("w", prefix="autoscaler-sdk-tmp-") @@ -276,6 +286,7 @@ def _as_config_file(cluster_config: Union[dict, str]) -> Iterator[str]: yield cluster_config +@DeveloperAPI def bootstrap_config( cluster_config: Dict[str, Any], no_config_cache: bool = False ) -> Dict[str, Any]: @@ -284,6 +295,7 @@ def bootstrap_config( return commands._bootstrap_config(cluster_config, no_config_cache) +@DeveloperAPI def fillout_defaults(config: Dict[str, Any]) -> Dict[str, Any]: """Fillout default values for a cluster_config based on the provider.""" from ray.autoscaler._private.util import fillout_defaults @@ -291,6 +303,7 @@ def fillout_defaults(config: Dict[str, Any]) -> Dict[str, Any]: return fillout_defaults(config) +@DeveloperAPI def register_callback_handler( event_name: str, callback: Union[Callable[[Dict], None], List[Callable[[Dict], None]]], @@ -307,6 +320,7 @@ def register_callback_handler( global_event_system.add_callback_handler(event_name, callback) +@DeveloperAPI def get_docker_host_mount_location(cluster_name: str) -> str: """Return host path that Docker mounts attach to.""" docker_mount_prefix = "/tmp/ray_tmp_mount/{cluster_name}" diff --git a/python/ray/client_builder.py b/python/ray/client_builder.py index 0adcde65d..b735f6197 100644 --- a/python/ray/client_builder.py +++ b/python/ray/client_builder.py @@ -1,23 +1,23 @@ -import os import importlib import inspect import json import logging +import os +import sys import warnings from dataclasses import dataclass -import sys - from typing import Any, Dict, Optional, Tuple -from ray.ray_constants import ( +import ray.util.client_connect +from ray._private.ray_constants import ( RAY_ADDRESS_ENVIRONMENT_VARIABLE, RAY_NAMESPACE_ENVIRONMENT_VARIABLE, RAY_RUNTIME_ENV_ENVIRONMENT_VARIABLE, ) +from ray._private.worker import BaseContext +from ray._private.worker import init as ray_driver_init from ray.job_config import JobConfig -import ray.util.client_connect -from ray.worker import init as ray_driver_init, BaseContext -from ray.util.annotations import Deprecated +from ray.util.annotations import Deprecated, PublicAPI logger = logging.getLogger(__name__) @@ -25,6 +25,7 @@ CLIENT_DOCS_URL = "https://docs.ray.io/en/latest/cluster/ray-client.html" @dataclass +@PublicAPI class ClientContext(BaseContext): """ Basic context manager for a ClientBuilder connection. @@ -67,10 +68,10 @@ class ClientContext(BaseContext): if ray.util.client.ray.is_default() or force_disconnect: # This is the only client connection ray.util.client_connect.disconnect() - elif ray.worker.global_worker.node is None: + elif ray._private.worker.global_worker.node is None: # Already disconnected. return - elif ray.worker.global_worker.node.is_head(): + elif ray._private.worker.global_worker.node.is_head(): logger.debug( "The current Ray Cluster is scoped to this process. " "Disconnecting is not possible as it will shutdown the " @@ -81,6 +82,7 @@ class ClientContext(BaseContext): ray.shutdown() +@Deprecated class ClientBuilder: """ Builder for a Ray Client connection. This class can be subclassed by @@ -163,7 +165,7 @@ class ClientBuilder: _credentials=self._credentials, ray_init_kwargs=self._remote_init_kwargs, ) - get_dashboard_url = ray.remote(ray.worker.get_dashboard_url) + get_dashboard_url = ray.remote(ray._private.worker.get_dashboard_url) dashboard_url = ray.get(get_dashboard_url.options(num_cpus=0).remote()) cxt = ClientContext( dashboard_url=dashboard_url, diff --git a/python/ray/cluster_utils.py b/python/ray/cluster_utils.py index 3cc3ffbde..c1d96f240 100644 --- a/python/ray/cluster_utils.py +++ b/python/ray/cluster_utils.py @@ -1,23 +1,26 @@ import copy -import logging import json -import yaml +import logging import os import subprocess import tempfile import time +import yaml + import ray import ray._private.services +from ray._private import ray_constants from ray._private.client_mode_hook import disable_client_hook -from ray import ray_constants from ray._raylet import GcsClientOptions +from ray.util.annotations import DeveloperAPI logger = logging.getLogger(__name__) cluster_not_supported = os.name == "nt" +@DeveloperAPI class AutoscalingCluster: """Create a local autoscaling cluster for testing. @@ -92,6 +95,7 @@ class AutoscalingCluster: subprocess.check_call(["ray", "stop", "--force"]) +@DeveloperAPI class Cluster: def __init__( self, @@ -124,7 +128,7 @@ class Cluster: self.redis_address = None self.connected = False # Create a new global state accessor for fetching GCS table. - self.global_state = ray.state.GlobalState() + self.global_state = ray._private.state.GlobalState() self._shutdown_at_exit = shutdown_at_exit if not initialize_head and connect: raise RuntimeError("Cannot connect to uninitialized cluster.") @@ -186,7 +190,7 @@ class Cluster: ray_params.update_if_absent(**default_kwargs) with disable_client_hook(): if self.head_node is None: - node = ray.node.Node( + node = ray._private.node.Node( ray_params, head=True, shutdown_at_exit=self._shutdown_at_exit, @@ -209,7 +213,7 @@ class Cluster: # Let grpc pick a port. ray_params.update_if_absent(node_manager_port=0) - node = ray.node.Node( + node = ray._private.node.Node( ray_params, head=False, shutdown_at_exit=self._shutdown_at_exit, @@ -234,7 +238,7 @@ class Cluster: node: Worker node of which all associated processes will be removed. """ - global_node = ray.worker._global_node + global_node = ray._private.worker._global_node if global_node is not None: if node._raylet_socket_name == global_node._raylet_socket_name: ray.shutdown() @@ -269,7 +273,7 @@ class Cluster: """Wait until this node has appeared in the client table. Args: - node (ray.node.Node): The node to wait for. + node (ray._private.node.Node): The node to wait for. timeout: The amount of time in seconds to wait before raising an exception. diff --git a/python/ray/cross_language.py b/python/ray/cross_language.py index fad12af52..9a1d0066a 100644 --- a/python/ray/cross_language.py +++ b/python/ray/cross_language.py @@ -1,11 +1,8 @@ -from __future__ import absolute_import -from __future__ import division -from __future__ import print_function +from __future__ import absolute_import, division, print_function from ray import Language +from ray._raylet import CppFunctionDescriptor, JavaFunctionDescriptor from ray.util.annotations import PublicAPI -from ray._raylet import JavaFunctionDescriptor -from ray._raylet import CppFunctionDescriptor __all__ = [ "java_function", @@ -14,60 +11,6 @@ __all__ = [ ] -def format_args(worker, args, kwargs): - """Format args for various languages. - - Args: - worker: The global worker instance. - args: The arguments for cross language. - kwargs: The keyword arguments for cross language. - - Returns: - List of args and kwargs (if supported). - """ - if not worker.load_code_from_local: - raise ValueError( - "Cross language feature needs --load-code-from-local to be set." - ) - if kwargs: - raise TypeError("Cross language remote functions does not support kwargs.") - return args - - -def get_function_descriptor_for_actor_method( - language: str, actor_creation_function_descriptor, method_name: str, signature: str -): - """Get function descriptor for cross language actor method call. - - Args: - language: Target language. - actor_creation_function_descriptor: - The function signature for actor creation. - method_name: The name of actor method. - signature: The signature for the actor method. When calling Java from Python, - it should be string in the form of "{length_of_args}". - - Returns: - Function descriptor for cross language actor method call. - """ - if language == Language.JAVA: - return JavaFunctionDescriptor( - actor_creation_function_descriptor.class_name, - method_name, - signature, - ) - elif language == Language.CPP: - return CppFunctionDescriptor( - method_name, - "PYTHON", - actor_creation_function_descriptor.class_name, - ) - else: - raise NotImplementedError( - "Cross language remote actor method " f"not support language {language}" - ) - - @PublicAPI(stability="beta") def java_function(class_name: str, function_name: str): """Define a Java function. @@ -135,3 +78,57 @@ def cpp_actor_class(create_function_name: str, class_name: str): CppFunctionDescriptor(create_function_name, "PYTHON", class_name), {}, ) + + +def _format_args(worker, args, kwargs): + """Format args for various languages. + + Args: + worker: The global worker instance. + args: The arguments for cross language. + kwargs: The keyword arguments for cross language. + + Returns: + List of args and kwargs (if supported). + """ + if not worker.load_code_from_local: + raise ValueError( + "Cross language feature needs --load-code-from-local to be set." + ) + if kwargs: + raise TypeError("Cross language remote functions does not support kwargs.") + return args + + +def _get_function_descriptor_for_actor_method( + language: str, actor_creation_function_descriptor, method_name: str, signature: str +): + """Get function descriptor for cross language actor method call. + + Args: + language: Target language. + actor_creation_function_descriptor: + The function signature for actor creation. + method_name: The name of actor method. + signature: The signature for the actor method. When calling Java from Python, + it should be string in the form of "{length_of_args}". + + Returns: + Function descriptor for cross language actor method call. + """ + if language == Language.JAVA: + return JavaFunctionDescriptor( + actor_creation_function_descriptor.class_name, + method_name, + signature, + ) + elif language == Language.CPP: + return CppFunctionDescriptor( + method_name, + "PYTHON", + actor_creation_function_descriptor.class_name, + ) + else: + raise NotImplementedError( + "Cross language remote actor method " f"not support language {language}" + ) diff --git a/python/ray/dag/class_node.py b/python/ray/dag/class_node.py index 1c0819971..d5842c1d4 100644 --- a/python/ray/dag/class_node.py +++ b/python/ray/dag/class_node.py @@ -7,10 +7,12 @@ from ray.dag.constants import ( PREV_CLASS_METHOD_CALL_KEY, DAGNODE_TYPE_KEY, ) +from ray.util.annotations import DeveloperAPI from typing import Any, Dict, List, Optional, Tuple +@DeveloperAPI class ClassNode(DAGNode): """Represents an actor creation in a Ray task DAG.""" @@ -157,6 +159,7 @@ class _UnboundClassMethodNode(object): return self +@DeveloperAPI class ClassMethodNode(DAGNode): """Represents an actor method invocation in a Ray function DAG.""" diff --git a/python/ray/dag/dag_node.py b/python/ray/dag/dag_node.py index a2370cc68..63c23c15d 100644 --- a/python/ray/dag/dag_node.py +++ b/python/ray/dag/dag_node.py @@ -1,5 +1,6 @@ import ray from ray.dag.py_obj_scanner import _PyObjScanner +from ray.util.annotations import DeveloperAPI from typing import ( Optional, @@ -16,6 +17,7 @@ import uuid T = TypeVar("T") +@DeveloperAPI class DAGNode: """Abstract class for a node in a Ray task graph. diff --git a/python/ray/dag/format_utils.py b/python/ray/dag/format_utils.py index 1bde92277..e1ea27858 100644 --- a/python/ray/dag/format_utils.py +++ b/python/ray/dag/format_utils.py @@ -1,15 +1,36 @@ from ray.dag import DAGNode +from ray.util.annotations import DeveloperAPI -def get_indentation(num_spaces=4): +@DeveloperAPI +def get_dag_node_str( + dag_node: DAGNode, + body_line, +): + indent = _get_indentation() + other_args_to_resolve_lines = _get_other_args_to_resolve_lines( + dag_node._bound_other_args_to_resolve + ) + return ( + f"({dag_node.__class__.__name__})(\n" + f"{indent}body={body_line}\n" + f"{indent}args={_get_args_lines(dag_node._bound_args)}\n" + f"{indent}kwargs={_get_kwargs_lines(dag_node._bound_kwargs)}\n" + f"{indent}options={_get_options_lines(dag_node._bound_options)}\n" + f"{indent}other_args_to_resolve={other_args_to_resolve_lines}\n" + f")" + ) + + +def _get_indentation(num_spaces=4): return " " * num_spaces -def get_args_lines(bound_args): +def _get_args_lines(bound_args): """Pretty prints bounded args of a DAGNode, and recursively handle DAGNode in list / dict containers. """ - indent = get_indentation() + indent = _get_indentation() lines = [] for arg in bound_args: if isinstance(arg, DAGNode): @@ -41,14 +62,14 @@ def get_args_lines(bound_args): return args_line -def get_kwargs_lines(bound_kwargs): +def _get_kwargs_lines(bound_kwargs): """Pretty prints bounded kwargs of a DAGNode, and recursively handle DAGNode in list / dict containers. """ # TODO: (jiaodong) Nits, we're missing keys and indentation was a bit off. if not bound_kwargs: return "{}" - indent = get_indentation() + indent = _get_indentation() kwargs_lines = [] for key, val in bound_kwargs.items(): if isinstance(val, DAGNode): @@ -86,11 +107,11 @@ def get_kwargs_lines(bound_kwargs): return kwargs_line -def get_options_lines(bound_options): +def _get_options_lines(bound_options): """Pretty prints .options() in DAGNode. Only prints non-empty values.""" if not bound_options: return "{}" - indent = get_indentation() + indent = _get_indentation() options_lines = [] for key, val in bound_options.items(): if val: @@ -103,10 +124,10 @@ def get_options_lines(bound_options): return options_line -def get_other_args_to_resolve_lines(other_args_to_resolve): +def _get_other_args_to_resolve_lines(other_args_to_resolve): if not other_args_to_resolve: return "{}" - indent = get_indentation() + indent = _get_indentation() other_args_to_resolve_lines = [] for key, val in other_args_to_resolve.items(): if isinstance(val, DAGNode): @@ -132,22 +153,3 @@ def get_other_args_to_resolve_lines(other_args_to_resolve): other_args_to_resolve_line += f"\n{indent}{line}" other_args_to_resolve_line += f"\n{indent}}}" return other_args_to_resolve_line - - -def get_dag_node_str( - dag_node: DAGNode, - body_line, -): - indent = get_indentation() - other_args_to_resolve_lines = get_other_args_to_resolve_lines( - dag_node._bound_other_args_to_resolve - ) - return ( - f"({dag_node.__class__.__name__})(\n" - f"{indent}body={body_line}\n" - f"{indent}args={get_args_lines(dag_node._bound_args)}\n" - f"{indent}kwargs={get_kwargs_lines(dag_node._bound_kwargs)}\n" - f"{indent}options={get_options_lines(dag_node._bound_options)}\n" - f"{indent}other_args_to_resolve={other_args_to_resolve_lines}\n" - f")" - ) diff --git a/python/ray/dag/function_node.py b/python/ray/dag/function_node.py index 02f17c9fe..42dc01bdb 100644 --- a/python/ray/dag/function_node.py +++ b/python/ray/dag/function_node.py @@ -5,8 +5,10 @@ import ray from ray.dag.dag_node import DAGNode from ray.dag.format_utils import get_dag_node_str from ray.dag.constants import DAGNODE_TYPE_KEY +from ray.util.annotations import DeveloperAPI +@DeveloperAPI class FunctionNode(DAGNode): """Represents a bound task node in a Ray task DAG.""" diff --git a/python/ray/dag/input_node.py b/python/ray/dag/input_node.py index aa3a7cea5..9f7c4e897 100644 --- a/python/ray/dag/input_node.py +++ b/python/ray/dag/input_node.py @@ -3,10 +3,12 @@ from typing import Any, Dict, List, Union from ray.dag import DAGNode from ray.dag.format_utils import get_dag_node_str from ray.dag.constants import DAGNODE_TYPE_KEY +from ray.util.annotations import DeveloperAPI IN_CONTEXT_MANAGER = "__in_context_manager__" +@DeveloperAPI class InputNode(DAGNode): """Ray dag node used in DAG building API to mark entrypoints of a DAG. @@ -146,6 +148,7 @@ class InputNode(DAGNode): return node +@DeveloperAPI class InputAttributeNode(DAGNode): """Represents partial access of user input based on an index (int), object attribute or dict key (str). @@ -252,6 +255,7 @@ class InputAttributeNode(DAGNode): return node +@DeveloperAPI class DAGInputData: """If user passed multiple args and kwargs directly to dag.execute(), we generate this wrapper for all user inputs as one object, accessible via diff --git a/python/ray/dag/tests/test_plot.py b/python/ray/dag/tests/test_plot.py index 9cb7688a7..6f9e45cae 100644 --- a/python/ray/dag/tests/test_plot.py +++ b/python/ray/dag/tests/test_plot.py @@ -24,7 +24,7 @@ def test_basic_dag_with_names_plot(): ray.dag.plot(dag, to_file) assert os.path.isfile(to_file) - graph = ray.dag.vis_utils.dag_to_dot(dag) + graph = ray.dag.vis_utils._dag_to_dot(dag) to_string = graph.to_string() assert "tmp1 -> tmp3" in to_string assert "tmp2 -> tmp3" in to_string @@ -52,7 +52,7 @@ def test_basic_dag_without_names_plot(): ray.dag.plot(dag, to_file) assert os.path.isfile(to_file) - graph = ray.dag.vis_utils.dag_to_dot(dag) + graph = ray.dag.vis_utils._dag_to_dot(dag) to_string = graph.to_string() assert "a_5 -> a_4" in to_string assert "a_2 -> a_1" in to_string diff --git a/python/ray/dag/utils.py b/python/ray/dag/utils.py index ab4ebd555..69d8e01e6 100644 --- a/python/ray/dag/utils.py +++ b/python/ray/dag/utils.py @@ -10,7 +10,7 @@ from ray.dag import ( ) -class DAGNodeNameGenerator(object): +class _DAGNodeNameGenerator(object): """ Generate unique suffix for each given Node in the DAG. Apply monotonic increasing id suffix for duplicated names. diff --git a/python/ray/dag/vis_utils.py b/python/ray/dag/vis_utils.py index 9aa3f430b..7ffa960ca 100644 --- a/python/ray/dag/vis_utils.py +++ b/python/ray/dag/vis_utils.py @@ -3,10 +3,42 @@ from ray.dag import DAGNode import os import tempfile -from ray.dag.utils import DAGNodeNameGenerator +from ray.dag.utils import _DAGNodeNameGenerator +from ray.util.annotations import DeveloperAPI -def check_pydot_and_graphviz(): +@DeveloperAPI +def plot(dag: DAGNode, to_file=None): + if to_file is None: + tmp_file = tempfile.NamedTemporaryFile(suffix=".png") + to_file = tmp_file.name + extension = "png" + else: + _, extension = os.path.splitext(to_file) + if not extension: + extension = "png" + else: + extension = extension[1:] + + graph = _dag_to_dot(dag) + graph.write(to_file, format=extension) + + # Render the image directly if running inside a Jupyter notebook + try: + from IPython import display + + return display.Image(filename=to_file) + except ImportError: + pass + + # close temp file if needed + try: + tmp_file.close() + except NameError: + pass + + +def _check_pydot_and_graphviz(): """Check if pydot and graphviz are installed. pydot and graphviz are required for plotting. We check this @@ -28,7 +60,7 @@ def check_pydot_and_graphviz(): ) -def get_nodes_and_edges(dag: DAGNode): +def _get_nodes_and_edges(dag: DAGNode): """Get all unique nodes and edges in the DAG. A basic dfs with memorization to get all unique nodes @@ -50,7 +82,7 @@ def get_nodes_and_edges(dag: DAGNode): return nodes, edges -def dag_to_dot(dag: DAGNode): +def _dag_to_dot(dag: DAGNode): """Create a Dot graph from dag. TODO(lchu): @@ -61,14 +93,14 @@ def dag_to_dot(dag: DAGNode): """ # Step 0: check dependencies and init graph - check_pydot_and_graphviz() + _check_pydot_and_graphviz() import pydot graph = pydot.Dot(rankdir="LR") # Step 1: generate unique name for each node in dag - nodes, edges = get_nodes_and_edges(dag) - name_generator = DAGNodeNameGenerator() + nodes, edges = _get_nodes_and_edges(dag) + name_generator = _DAGNodeNameGenerator() node_names = {} for node in nodes: node_names[node] = name_generator.get_node_name(node) @@ -81,33 +113,3 @@ def dag_to_dot(dag: DAGNode): graph.add_node(pydot.Node(node_names[nodes[0]])) return graph - - -def plot(dag: DAGNode, to_file=None): - if to_file is None: - tmp_file = tempfile.NamedTemporaryFile(suffix=".png") - to_file = tmp_file.name - extension = "png" - else: - _, extension = os.path.splitext(to_file) - if not extension: - extension = "png" - else: - extension = extension[1:] - - graph = dag_to_dot(dag) - graph.write(to_file, format=extension) - - # Render the image directly if running inside a Jupyter notebook - try: - from IPython import display - - return display.Image(filename=to_file) - except ImportError: - pass - - # close temp file if needed - try: - tmp_file.close() - except NameError: - pass diff --git a/python/ray/data/__init__.py b/python/ray/data/__init__.py index ab184ff48..0afc9cd1e 100644 --- a/python/ray/data/__init__.py +++ b/python/ray/data/__init__.py @@ -44,7 +44,8 @@ _cached_cls = None if ray.is_initialized(): _register_arrow_json_readoptions_serializer() else: - ray.worker._post_init_hooks.append(_register_arrow_json_readoptions_serializer) + pass +# ray._internal.worker._post_init_hooks.append(_register_arrow_json_readoptions_serializer) __all__ = [ "ActorPoolStrategy", diff --git a/python/ray/data/_internal/block_batching.py b/python/ray/data/_internal/block_batching.py index a154fbf11..2243f632e 100644 --- a/python/ray/data/_internal/block_batching.py +++ b/python/ray/data/_internal/block_batching.py @@ -1,6 +1,6 @@ import collections import itertools -from typing import TYPE_CHECKING, Dict, Iterable, Iterator, Optional, Union +from typing import TYPE_CHECKING, Iterable, Iterator, Optional, Union, Dict import numpy as np @@ -155,7 +155,7 @@ def _sliding_window(iterable: Iterable, n: int, clear_block_after_read: bool = F for elem in it: block_ref = window.popleft() if clear_block_after_read: - ray.internal.internal_api.free(block_ref, local_only=False) + ray._private.internal_api.free(block_ref, local_only=False) window.append(elem) yield tuple(window) diff --git a/python/ray/data/_internal/progress_bar.py b/python/ray/data/_internal/progress_bar.py index 071843a89..2b56f0188 100644 --- a/python/ray/data/_internal/progress_bar.py +++ b/python/ray/data/_internal/progress_bar.py @@ -1,8 +1,8 @@ -from typing import List, Any import threading +from typing import Any, List import ray -from ray.ray_constants import env_integer +from ray._private.ray_constants import env_integer from ray.types import ObjectRef from ray.util.annotations import PublicAPI diff --git a/python/ray/data/_internal/size_estimator.py b/python/ray/data/_internal/size_estimator.py index bf8641b2b..7f5231c1c 100644 --- a/python/ray/data/_internal/size_estimator.py +++ b/python/ray/data/_internal/size_estimator.py @@ -44,7 +44,7 @@ class SizeEstimator: _ray_initialized = True ray.put(None) return ( - ray.worker.global_worker.get_serialization_context() + ray._private.worker.global_worker.get_serialization_context() .serialize(item) .total_bytes ) diff --git a/python/ray/data/_internal/stats.py b/python/ray/data/_internal/stats.py index 9dbcb4ca2..29cafdd62 100644 --- a/python/ray/data/_internal/stats.py +++ b/python/ray/data/_internal/stats.py @@ -134,7 +134,7 @@ def _get_or_create_stats_actor(): def clear_actor(): _stats_actor[0] = None - ray.worker._post_init_hooks.append(clear_actor) + ray._private.worker._post_init_hooks.append(clear_actor) return _stats_actor[0] diff --git a/python/ray/data/context.py b/python/ray/data/context.py index 4fd7a4635..056e19b38 100644 --- a/python/ray/data/context.py +++ b/python/ray/data/context.py @@ -129,7 +129,7 @@ class DatasetContext: if _default_context: _default_context.block_owner = None - ray.worker._post_init_hooks.append(clear_owner) + ray._private.worker._post_init_hooks.append(clear_owner) _default_context.block_owner = owner return _default_context diff --git a/python/ray/data/dataset.py b/python/ray/data/dataset.py index c9f434802..c7ea6cf70 100644 --- a/python/ray/data/dataset.py +++ b/python/ray/data/dataset.py @@ -1029,7 +1029,7 @@ class Dataset(Generic[T]): def build_node_id_by_actor(actors: List[Any]) -> Dict[Any, str]: """Build a map from a actor to its node_id.""" - actors_state = ray.state.actors() + actors_state = ray._private.state.actors() return { actor: actors_state.get(actor._actor_id.hex(), {}) .get("Address", {}) @@ -3209,7 +3209,7 @@ class Dataset(Generic[T]): state["_last_export_session_and_job"] = None return reconstructor, args, state - context = ray.worker.global_worker.get_serialization_context() + context = ray._private.worker.global_worker.get_serialization_context() try: context._register_cloudpickle_reducer( ray.remote_function.RemoteFunction, _reduce diff --git a/python/ray/data/tests/test_object_gc.py b/python/ray/data/tests/test_object_gc.py index 6abfdfdf7..5dce9d06c 100644 --- a/python/ray/data/tests/test_object_gc.py +++ b/python/ray/data/tests/test_object_gc.py @@ -3,10 +3,9 @@ import time import pytest import ray +from ray._private.internal_api import memory_summary from ray.tests.conftest import * # noqa -from ray.internal.internal_api import memory_summary - def check_no_spill(ctx, pipe, prefetch_blocks: int = 0): # Run .iter_batches() for 10 secs, and we expect no object spilling. diff --git a/python/ray/data/tests/test_optimize.py b/python/ray/data/tests/test_optimize.py index 2380ba20e..9dffcd045 100644 --- a/python/ray/data/tests/test_optimize.py +++ b/python/ray/data/tests/test_optimize.py @@ -1,16 +1,16 @@ -import pytest -import numpy as np -import pandas as pd import os from typing import List +import numpy as np +import pandas as pd +import pytest + import ray +from ray._private.internal_api import memory_summary from ray.data.block import BlockMetadata from ray.data.context import DatasetContext from ray.data.datasource import Datasource, ReadTask from ray.data.datasource.csv_datasource import CSVDatasource -from ray.internal.internal_api import memory_summary - from ray.tests.conftest import * # noqa diff --git a/python/ray/data/tests/test_split.py b/python/ray/data/tests/test_split.py index a08e23f37..5df88703f 100644 --- a/python/ray/data/tests/test_split.py +++ b/python/ray/data/tests/test_split.py @@ -1,20 +1,19 @@ import math import random import time - from unittest.mock import patch + import numpy as np import pytest import ray - -from ray.tests.conftest import * # noqa -from ray.data.dataset import Dataset -from ray.data.block import BlockAccessor from ray.data._internal.block_list import BlockList -from ray.data._internal.stats import DatasetStats from ray.data._internal.plan import ExecutionPlan +from ray.data._internal.stats import DatasetStats +from ray.data.block import BlockAccessor +from ray.data.dataset import Dataset from ray.data.tests.conftest import * # noqa +from ray.tests.conftest import * # noqa @ray.remote @@ -324,7 +323,7 @@ def test_split_hints(ray_start_regular_shared): assert len(block_node_ids) == len(blocks) actors = [Actor.remote() for i in range(len(actor_node_ids))] with patch("ray.experimental.get_object_locations") as location_mock: - with patch("ray.state.actors") as state_mock: + with patch("ray._private.state.actors") as state_mock: block_locations = {} for i, node_id in enumerate(block_node_ids): if node_id: diff --git a/python/ray/exceptions.py b/python/ray/exceptions.py index e7d6bc4a8..d28d60f35 100644 --- a/python/ray/exceptions.py +++ b/python/ray/exceptions.py @@ -1,17 +1,24 @@ import os from traceback import format_exception +from typing import Optional, Union -from typing import Union, Optional - -import ray.cloudpickle as pickle -from ray.core.generated.common_pb2 import RayException, Language, PYTHON -from ray.core.generated.common_pb2 import Address, ActorDiedErrorContext -import ray.ray_constants as ray_constants -from ray._raylet import WorkerID, ActorID, TaskID import colorama import setproctitle +import ray._private.ray_constants as ray_constants +import ray.cloudpickle as pickle +from ray._raylet import ActorID, TaskID, WorkerID +from ray.core.generated.common_pb2 import ( + PYTHON, + ActorDiedErrorContext, + Address, + Language, + RayException, +) +from ray.util.annotations import DeveloperAPI, PublicAPI + +@PublicAPI class RayError(Exception): """Super class of all ray exception types.""" @@ -43,6 +50,7 @@ class RayError(Exception): return CrossLanguageError(ray_exception) +@PublicAPI class CrossLanguageError(RayError): """Raised from another language.""" @@ -55,6 +63,7 @@ class CrossLanguageError(RayError): ) +@PublicAPI class TaskCancelledError(RayError): """Raised when this task is cancelled. @@ -72,6 +81,7 @@ class TaskCancelledError(RayError): return "Task: " + str(self.task_id) + " was cancelled" +@PublicAPI class RayTaskError(RayError): """Indicates that a task threw an exception during execution. @@ -213,6 +223,7 @@ class RayTaskError(RayError): return "\n".join(out) +@PublicAPI class LocalRayletDiedError(RayError): """Indicates that the task's local raylet died.""" @@ -220,6 +231,7 @@ class LocalRayletDiedError(RayError): return "The task's local raylet died. Check raylet.out for more information." +@PublicAPI class WorkerCrashedError(RayError): """Indicates that the worker died unexpectedly while executing a task.""" @@ -230,6 +242,7 @@ class WorkerCrashedError(RayError): ) +@PublicAPI class RayActorError(RayError): """Indicates that the actor died unexpectedly before finishing a task. @@ -293,6 +306,7 @@ class RayActorError(RayError): return RayActorError(task_error) +@PublicAPI class RaySystemError(RayError): """Indicates that Ray encountered a system error. @@ -310,6 +324,7 @@ class RaySystemError(RayError): return error_msg +@PublicAPI class ObjectStoreFullError(RayError): """Indicates that the object store is full. @@ -326,6 +341,7 @@ class ObjectStoreFullError(RayError): ) +@PublicAPI class ObjectLostError(RayError): """Indicates that the object is lost from distributed memory, due to node failure or system error. @@ -366,6 +382,7 @@ class ObjectLostError(RayError): ) +@PublicAPI class ObjectFetchTimedOutError(ObjectLostError): """Indicates that an object fetch timed out. @@ -385,6 +402,7 @@ class ObjectFetchTimedOutError(ObjectLostError): ) +@DeveloperAPI class ReferenceCountingAssertionError(ObjectLostError, AssertionError): """Indicates that an object has been deleted while there was still a reference to it. @@ -404,6 +422,7 @@ class ReferenceCountingAssertionError(ObjectLostError, AssertionError): ) +@PublicAPI class OwnerDiedError(ObjectLostError): """Indicates that the owner of the object has died while there is still a reference to the object. @@ -442,6 +461,7 @@ class OwnerDiedError(ObjectLostError): ) +@PublicAPI class ObjectReconstructionFailedError(ObjectLostError): """Indicates that the object cannot be reconstructed. @@ -461,6 +481,7 @@ class ObjectReconstructionFailedError(ObjectLostError): ) +@PublicAPI class ObjectReconstructionFailedMaxAttemptsExceededError(ObjectLostError): """Indicates that the object cannot be reconstructed because the maximum number of task retries has been exceeded. @@ -482,6 +503,7 @@ class ObjectReconstructionFailedMaxAttemptsExceededError(ObjectLostError): ) +@PublicAPI class ObjectReconstructionFailedLineageEvictedError(ObjectLostError): """Indicates that the object cannot be reconstructed because its lineage was evicted due to memory pressure. @@ -503,24 +525,28 @@ class ObjectReconstructionFailedLineageEvictedError(ObjectLostError): ) +@PublicAPI class GetTimeoutError(RayError): """Indicates that a call to the worker timed out.""" pass +@PublicAPI class PlasmaObjectNotAvailable(RayError): """Called when an object was not available within the given timeout.""" pass +@PublicAPI class AsyncioActorExit(RayError): """Raised when an asyncio actor intentionally exits via exit_actor().""" pass +@PublicAPI class RuntimeEnvSetupError(RayError): """Raised when a runtime environment fails to be set up. @@ -539,6 +565,7 @@ class RuntimeEnvSetupError(RayError): return "\n".join(msgs) +@PublicAPI class TaskPlacementGroupRemoved(RayError): """Raised when the corresponding placement group was removed.""" @@ -546,6 +573,7 @@ class TaskPlacementGroupRemoved(RayError): return "The placement group corresponding to this task has been removed." +@PublicAPI class ActorPlacementGroupRemoved(RayError): """Raised when the corresponding placement group was removed.""" @@ -553,6 +581,7 @@ class ActorPlacementGroupRemoved(RayError): return "The placement group corresponding to this Actor has been removed." +@PublicAPI class PendingCallsLimitExceeded(RayError): """Raised when the pending actor calls exceeds `max_pending_calls` option. @@ -563,6 +592,7 @@ class PendingCallsLimitExceeded(RayError): pass +@PublicAPI class TaskUnschedulableError(RayError): """Raised when the task cannot be scheduled. @@ -577,6 +607,7 @@ class TaskUnschedulableError(RayError): return f"The task is not schedulable: {self.error_message}" +@PublicAPI class ActorUnschedulableError(RayError): """Raised when the actor cannot be scheduled. diff --git a/python/ray/experimental/job/example_job/demo_script.py b/python/ray/experimental/job/example_job/demo_script.py index 8d5a6d151..8bcc79611 100644 --- a/python/ray/experimental/job/example_job/demo_script.py +++ b/python/ray/experimental/job/example_job/demo_script.py @@ -15,12 +15,12 @@ class StepActor: self.current_step = 1 self.total_steps = total_steps - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker_id = worker.core_worker.get_actor_id() ray_kv._internal_kv_put(f"JOB:{worker_id}", self.current_step, overwrite=True) def run(self): - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker_id = worker.core_worker.get_actor_id() while self.current_step <= self.total_steps: diff --git a/python/ray/experimental/locations.py b/python/ray/experimental/locations.py index bfc692008..1f06eb4c0 100644 --- a/python/ray/experimental/locations.py +++ b/python/ray/experimental/locations.py @@ -1,6 +1,7 @@ +from typing import Any, Dict, List + import ray from ray._raylet import ObjectRef -from typing import Any, Dict, List def get_object_locations( @@ -34,6 +35,6 @@ def get_object_locations( """ if not ray.is_initialized(): raise RuntimeError("Ray hasn't been initialized.") - return ray.worker.global_worker.core_worker.get_object_locations( + return ray._private.worker.global_worker.core_worker.get_object_locations( obj_refs, timeout_ms ) diff --git a/python/ray/experimental/packaging/load_package.py b/python/ray/experimental/packaging/load_package.py index a7f2bf220..9dc708b12 100644 --- a/python/ray/experimental/packaging/load_package.py +++ b/python/ray/experimental/packaging/load_package.py @@ -10,19 +10,16 @@ runtime environment, which can include: You can run this file for an example of loading a "hello world" package. """ +import hashlib import importlib.util import os import re -import hashlib import subprocess import tempfile + import yaml import ray -from ray._private.runtime_env.packaging import ( - get_uri_for_directory, - upload_package_if_needed, -) def load_package(config_path: str) -> "_RuntimePackage": @@ -59,6 +56,11 @@ def load_package(config_path: str) -> "_RuntimePackage": >>> def f(): ... """ + from ray._private.runtime_env.packaging import ( + get_uri_for_directory, + upload_package_if_needed, + ) + config_path = _download_from_github_if_needed(config_path) if not os.path.exists(config_path): @@ -81,7 +83,7 @@ def load_package(config_path: str) -> "_RuntimePackage": if ray.is_initialized(): do_register_package() else: - ray.worker._post_init_hooks.append(do_register_package) + ray._private.worker._post_init_hooks.append(do_register_package) runtime_env["working_dir"] = pkg_uri # Autofill conda config. diff --git a/python/ray/experimental/raysort/main.py b/python/ray/experimental/raysort/main.py index 50c25c414..fa26442fb 100644 --- a/python/ray/experimental/raysort/main.py +++ b/python/ray/experimental/raysort/main.py @@ -9,19 +9,16 @@ import tempfile from typing import Callable, Dict, Iterable, List import numpy as np -import ray -from ray.experimental.raysort import constants -from ray.experimental.raysort import logging_utils -from ray.experimental.raysort import sortlib -from ray.experimental.raysort import tracing_utils +import ray +from ray.experimental.raysort import constants, logging_utils, sortlib, tracing_utils from ray.experimental.raysort.types import ( BlockInfo, ByteCount, - RecordCount, PartId, PartInfo, Path, + RecordCount, ) Args = argparse.Namespace @@ -131,7 +128,7 @@ def _get_mount_points(): def _part_info(args: Args, part_id: PartId, kind="input") -> PartInfo: - node = ray.worker.global_worker.node_ip_address + node = ray._private.worker.global_worker.node_ip_address mnt = random.choice(args.mount_points) filepath = _get_part_path(mnt, part_id, kind) return PartInfo(part_id, node, filepath) @@ -391,7 +388,7 @@ def sort_main(args: Args): writer = csv.writer(fout) writer.writerows(reducer_results) - logging.info(ray.internal.internal_api.memory_summary(stats_only=True)) + logging.info(ray._private.internal_api.memory_summary(stats_only=True)) # ------------------------------------------------------------ diff --git a/python/ray/experimental/shuffle.py b/python/ray/experimental/shuffle.py index a8f96f3bb..4aff5c48c 100644 --- a/python/ray/experimental/shuffle.py +++ b/python/ray/experimental/shuffle.py @@ -24,11 +24,11 @@ This will print out some statistics on the shuffle execution such as: """ import time -from typing import List, Iterable, Tuple, Callable, Any, Union +from typing import Any, Callable, Iterable, List, Tuple, Union import ray -from ray.cluster_utils import Cluster from ray import ObjectRef +from ray.cluster_utils import Cluster # TODO(ekl) why doesn't TypeVar() deserialize properly in Ray? # The type produced by the input reader function. @@ -239,9 +239,10 @@ def run( use_wait=False, tracker=None, ): - import numpy as np import time + import numpy as np + is_multi_node = num_nodes if ray_address: print("Connecting to a existing cluster...") @@ -310,7 +311,7 @@ def run( time.sleep(0.5) print() - print(ray.internal.internal_api.memory_summary(stats_only=True)) + print(ray._private.internal_api.memory_summary(stats_only=True)) print() print( "Shuffled", int(sum(output_sizes) / (1024 * 1024)), "MiB in", delta, "seconds" diff --git a/python/ray/experimental/state/api.py b/python/ray/experimental/state/api.py index ed4d9b49f..a8cad8b4c 100644 --- a/python/ray/experimental/state/api.py +++ b/python/ray/experimental/state/api.py @@ -1,22 +1,21 @@ +import urllib import warnings +from dataclasses import fields +from typing import Dict, Generator, List, Optional, Tuple, Union + import requests import ray -import urllib - -from typing import List, Tuple, Optional, Dict, Generator, Union -from dataclasses import fields - +from ray.dashboard.modules.dashboard_sdk import SubmissionClient from ray.experimental.state.common import ( DEFAULT_LIMIT, DEFAULT_RPC_TIMEOUT, GetLogOptions, ListApiOptions, - SupportedFilterType, StateResource, + SupportedFilterType, ) from ray.experimental.state.exception import RayStateApiException, ServerUnavailable -from ray.dashboard.modules.dashboard_sdk import SubmissionClient """ This file contains API client and methods for querying ray state. @@ -65,9 +64,12 @@ class StateApiClient(SubmissionClient): def _get_default_api_server_address(cls) -> str: assert ( ray.is_initialized() - and ray.worker.global_worker.node.address_info["webui_url"] is not None + and ray._private.worker.global_worker.node.address_info["webui_url"] + is not None + ) + return ( + f"http://{ray._private.worker.global_worker.node.address_info['webui_url']}" ) - return f"http://{ray.worker.global_worker.node.address_info['webui_url']}" def list( self, resource: StateResource, options: ListApiOptions, _explain: bool = False @@ -291,7 +293,7 @@ def get_log( if api_server_url is None: assert ray.is_initialized() api_server_url = ( - f"http://{ray.worker.global_worker.node.address_info['webui_url']}" + f"http://{ray._private.worker.global_worker.node.address_info['webui_url']}" ) media_type = "stream" if follow else "file" @@ -342,7 +344,7 @@ def list_logs( if api_server_url is None: assert ray.is_initialized() api_server_url = ( - f"http://{ray.worker.global_worker.node.address_info['webui_url']}" + f"http://{ray._private.worker.global_worker.node.address_info['webui_url']}" ) if not glob_filter: diff --git a/python/ray/experimental/state/state_cli.py b/python/ray/experimental/state/state_cli.py index 5e530bda1..41b28291e 100644 --- a/python/ray/experimental/state/state_cli.py +++ b/python/ray/experimental/state/state_cli.py @@ -1,15 +1,15 @@ -import click -import logging import json +import logging +from enum import Enum, unique +from typing import List, Tuple, Union + +import click import yaml -from enum import Enum, unique -from typing import Union, List, Tuple - import ray -import ray.ray_constants as ray_constants +import ray._private.ray_constants as ray_constants import ray._private.services as services - +from ray._private.gcs_utils import GcsClient from ray.experimental.state.api import StateApiClient from ray.experimental.state.common import ( DEFAULT_LIMIT, @@ -17,7 +17,6 @@ from ray.experimental.state.common import ( ListApiOptions, StateResource, ) -from ray._private.gcs_utils import GcsClient logger = logging.getLogger(__name__) diff --git a/python/ray/experimental/state/state_manager.py b/python/ray/experimental/state/state_manager.py index 4bf4811a4..dc51e6f05 100644 --- a/python/ray/experimental/state/state_manager.py +++ b/python/ray/experimental/state/state_manager.py @@ -1,48 +1,45 @@ -import logging import inspect - +import logging +from collections import defaultdict from functools import wraps +from typing import Dict, List, Optional import grpc -import ray - -from collections import defaultdict -from typing import Dict, List, Optional -from ray import ray_constants - from grpc.aio._call import UnaryStreamCall +import ray +import ray.dashboard.modules.log.log_consts as log_consts +from ray._private import ray_constants +from ray.core.generated import gcs_service_pb2_grpc from ray.core.generated.gcs_service_pb2 import ( - GetAllActorInfoRequest, GetAllActorInfoReply, - GetAllPlacementGroupRequest, - GetAllPlacementGroupReply, - GetAllNodeInfoRequest, + GetAllActorInfoRequest, GetAllNodeInfoReply, - GetAllWorkerInfoRequest, + GetAllNodeInfoRequest, + GetAllPlacementGroupReply, + GetAllPlacementGroupRequest, GetAllWorkerInfoReply, + GetAllWorkerInfoRequest, ) from ray.core.generated.node_manager_pb2 import ( - GetTasksInfoRequest, - GetTasksInfoReply, - GetNodeStatsRequest, GetNodeStatsReply, + GetNodeStatsRequest, + GetTasksInfoReply, + GetTasksInfoRequest, ) -from ray.core.generated.runtime_env_agent_pb2 import ( - GetRuntimeEnvsInfoRequest, - GetRuntimeEnvsInfoReply, -) +from ray.core.generated.node_manager_pb2_grpc import NodeManagerServiceStub from ray.core.generated.reporter_pb2 import ( ListLogsReply, - StreamLogRequest, ListLogsRequest, + StreamLogRequest, ) from ray.core.generated.reporter_pb2_grpc import LogServiceStub +from ray.core.generated.runtime_env_agent_pb2 import ( + GetRuntimeEnvsInfoReply, + GetRuntimeEnvsInfoRequest, +) from ray.core.generated.runtime_env_agent_pb2_grpc import RuntimeEnvServiceStub -from ray.core.generated import gcs_service_pb2_grpc -from ray.core.generated.node_manager_pb2_grpc import NodeManagerServiceStub -import ray.dashboard.modules.log.log_consts as log_consts -from ray.dashboard.modules.job.common import JobInfoStorageClient, JobInfo +from ray.dashboard.modules.job.common import JobInfo, JobInfoStorageClient from ray.experimental.state.exception import DataSourceUnavailable logger = logging.getLogger(__name__) diff --git a/python/ray/includes/function_descriptor.pxi b/python/ray/includes/function_descriptor.pxi index 0dc18195c..10a387538 100644 --- a/python/ray/includes/function_descriptor.pxi +++ b/python/ray/includes/function_descriptor.pxi @@ -14,7 +14,7 @@ import hashlib import cython import inspect import uuid -import ray.ray_constants as ray_constants +import ray._private.ray_constants as ray_constants ctypedef object (*FunctionDescriptor_from_cpp)(const CFunctionDescriptor &) diff --git a/python/ray/includes/object_ref.pxi b/python/ray/includes/object_ref.pxi index 59233ab92..7662b3065 100644 --- a/python/ray/includes/object_ref.pxi +++ b/python/ray/includes/object_ref.pxi @@ -48,7 +48,7 @@ cdef class ObjectRef(BaseID): self.in_core_worker = False self.call_site_data = call_site_data - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker # TODO(edoakes): We should be able to remove the in_core_worker flag. # But there are still some dummy object refs being created outside the # context of a core worker. @@ -60,7 +60,7 @@ cdef class ObjectRef(BaseID): def __dealloc__(self): if self.in_core_worker: try: - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker.core_worker.remove_object_ref_reference(self) except Exception as e: # There is a strange error in rllib that causes the above to @@ -151,6 +151,6 @@ cdef class ObjectRef(BaseID): The callback should take the result as the only argument. The result can be an exception object in case of task error. """ - core_worker = ray.worker.global_worker.core_worker + core_worker = ray._private.worker.global_worker.core_worker core_worker.set_get_async_callback(self, py_callback) return self diff --git a/python/ray/internal/__init__.py b/python/ray/internal/__init__.py index 52485fb32..36740dd18 100644 --- a/python/ray/internal/__init__.py +++ b/python/ray/internal/__init__.py @@ -1,3 +1,3 @@ -from ray.internal.internal_api import free +from ray._private.internal_api import free __all__ = ["free"] diff --git a/python/ray/job_config.py b/python/ray/job_config.py index 5139deaa8..a412a2e97 100644 --- a/python/ray/job_config.py +++ b/python/ray/job_config.py @@ -1,9 +1,11 @@ -from typing import Any, Dict, Optional, Union, List import uuid +from typing import Any, Dict, List, Optional, Union import ray._private.gcs_utils as gcs_utils +from ray.util.annotations import PublicAPI +@PublicAPI class JobConfig: """A class used to store the configurations of a job. @@ -96,7 +98,7 @@ class JobConfig: # TODO(edoakes): this is really unfortunate, but JobConfig is imported # all over the place so this causes circular imports. We should remove # this dependency and pass in a validated runtime_env instead. - from ray.utils import get_runtime_env_info + from ray._private.utils import get_runtime_env_info if self._cached_pb is None: pb = gcs_utils.JobConfig() diff --git a/python/ray/ray_operator/operator.py b/python/ray/ray_operator/operator.py index c1d14cef1..a9d8f691b 100644 --- a/python/ray/ray_operator/operator.py +++ b/python/ray/ray_operator/operator.py @@ -3,23 +3,20 @@ import logging import multiprocessing as mp import os import threading -from typing import Any -from typing import Callable -from typing import Dict -from typing import Tuple -from typing import Optional +from typing import Any, Callable, Dict, Optional, Tuple import kopf import yaml import ray.autoscaler._private.monitor as monitor -from ray._private import services +from ray._private import ray_constants, services from ray.autoscaler._private import commands from ray.ray_operator import operator_utils -from ray.ray_operator.operator_utils import STATUS_AUTOSCALING_EXCEPTION -from ray.ray_operator.operator_utils import STATUS_RUNNING -from ray.ray_operator.operator_utils import STATUS_UPDATING -from ray import ray_constants +from ray.ray_operator.operator_utils import ( + STATUS_AUTOSCALING_EXCEPTION, + STATUS_RUNNING, + STATUS_UPDATING, +) logger = logging.getLogger(__name__) diff --git a/python/ray/ray_operator/operator_utils.py b/python/ray/ray_operator/operator_utils.py index bb6c7fb5b..1ec79c607 100644 --- a/python/ray/ray_operator/operator_utils.py +++ b/python/ray/ray_operator/operator_utils.py @@ -3,15 +3,12 @@ import logging import os import re import time -from typing import Any -from typing import Dict -from typing import Iterator -from typing import List +from typing import Any, Dict, Iterator, List -from kubernetes.watch import Watch from kubernetes.client.rest import ApiException +from kubernetes.watch import Watch -from ray import ray_constants +from ray._private import ray_constants from ray.autoscaler._private._kubernetes import custom_objects_api from ray.autoscaler._private._kubernetes.node_provider import head_service_selector from ray.autoscaler._private.providers import _get_default_config diff --git a/python/ray/remote_function.py b/python/ray/remote_function.py index 8d67bb280..d69b8abc2 100644 --- a/python/ray/remote_function.py +++ b/python/ray/remote_function.py @@ -1,24 +1,27 @@ -from functools import wraps import inspect import logging -import uuid import os +import uuid +from functools import wraps -from ray import cloudpickle as pickle -from ray.util.annotations import DeveloperAPI -from ray.util.scheduling_strategies import PlacementGroupSchedulingStrategy -from ray._raylet import PythonFunctionDescriptor -from ray import cross_language, Language -from ray._private.client_mode_hook import client_mode_convert_function -from ray._private.client_mode_hook import client_mode_should_convert -from ray.util.placement_group import configure_placement_group_based_on_context import ray._private.signature -from ray.utils import get_runtime_env_info, parse_runtime_env -from ray.util.tracing.tracing_helper import ( - _tracing_task_invocation, - _inject_tracing_into_function, -) +from ray import Language +from ray import cloudpickle as pickle +from ray import cross_language from ray._private import ray_option_utils +from ray._private.client_mode_hook import ( + client_mode_convert_function, + client_mode_should_convert, +) +from ray._private.utils import get_runtime_env_info, parse_runtime_env +from ray._raylet import PythonFunctionDescriptor +from ray.util.annotations import DeveloperAPI, PublicAPI +from ray.util.placement_group import _configure_placement_group_based_on_context +from ray.util.scheduling_strategies import PlacementGroupSchedulingStrategy +from ray.util.tracing.tracing_helper import ( + _inject_tracing_into_function, + _tracing_task_invocation, +) logger = logging.getLogger(__name__) @@ -27,6 +30,7 @@ logger = logging.getLogger(__name__) _task_launch_hook = None +@PublicAPI class RemoteFunction: """A remote function. @@ -183,7 +187,7 @@ class RemoteFunction: if client_mode_should_convert(auto_init=True): return client_mode_convert_function(self, args, kwargs, **task_options) - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker.check_connected() # If this function was not exported in this session and job, we need to @@ -267,7 +271,7 @@ class RemoteFunction: placement_group_capture_child_tasks = ( worker.should_capture_child_tasks_in_placement_group ) - placement_group = configure_placement_group_based_on_context( + placement_group = _configure_placement_group_based_on_context( placement_group_capture_child_tasks, placement_group_bundle_index, resources, @@ -297,7 +301,7 @@ class RemoteFunction: def invocation(args, kwargs): if self._is_cross_language: - list_args = cross_language.format_args(worker, args, kwargs) + list_args = cross_language._format_args(worker, args, kwargs) elif not args and not kwargs and not self._function_signature: list_args = [] else: @@ -305,7 +309,7 @@ class RemoteFunction: self._function_signature, args, kwargs ) - if worker.mode == ray.worker.LOCAL_MODE: + if worker.mode == ray._private.worker.LOCAL_MODE: assert ( not self._is_cross_language ), "Cross language remote function cannot be executed locally." diff --git a/python/ray/runtime_context.py b/python/ray/runtime_context.py index e56473d1d..9dcddc6b3 100644 --- a/python/ray/runtime_context.py +++ b/python/ray/runtime_context.py @@ -1,5 +1,6 @@ -import ray.worker import logging + +import ray._private.worker from ray._private.client_mode_hook import client_mode_hook from ray.runtime_env import RuntimeEnv from ray.util.annotations import PublicAPI @@ -7,7 +8,7 @@ from ray.util.annotations import PublicAPI logger = logging.getLogger(__name__) -@PublicAPI(stability="beta") +@PublicAPI class RuntimeContext(object): """A class used for getting runtime context.""" @@ -26,7 +27,7 @@ class RuntimeContext(object): "node_id": self.node_id, "namespace": self.namespace, } - if self.worker.mode == ray.worker.WORKER_MODE: + if self.worker.mode == ray._private.worker.WORKER_MODE: if self.task_id is not None: context["task_id"] = self.task_id if self.actor_id is not None: @@ -92,7 +93,7 @@ class RuntimeContext(object): """ # only worker mode has actor_id assert ( - self.worker.mode == ray.worker.WORKER_MODE + self.worker.mode == ray._private.worker.WORKER_MODE ), f"This method is only available when the process is a\ worker. Current mode: {self.worker.mode}" task_id = self.worker.current_task_id @@ -110,7 +111,7 @@ class RuntimeContext(object): """ # only worker mode has actor_id assert ( - self.worker.mode == ray.worker.WORKER_MODE + self.worker.mode == ray._private.worker.WORKER_MODE ), f"This method is only available when the process is a\ worker. Current mode: {self.worker.mode}" actor_id = self.worker.actor_id @@ -135,7 +136,7 @@ class RuntimeContext(object): assert ( not self.actor_id.is_nil() ), "This method should't be called inside Ray tasks." - actor_info = ray.state.actors(self.actor_id.hex()) + actor_info = ray._private.state.actors(self.actor_id.hex()) return actor_info and actor_info["NumRestarts"] != 0 @property @@ -216,7 +217,7 @@ class RuntimeContext(object): _runtime_context = None -@PublicAPI(stability="beta") +@PublicAPI @client_mode_hook(auto_init=False) def get_runtime_context(): """Get the runtime context of the current driver/worker. @@ -232,6 +233,6 @@ def get_runtime_context(): """ global _runtime_context if _runtime_context is None: - _runtime_context = RuntimeContext(ray.worker.global_worker) + _runtime_context = RuntimeContext(ray._private.worker.global_worker) return _runtime_context diff --git a/python/ray/runtime_env.py b/python/ray/runtime_env.py index 0a9a6d12b..43cc0414d 100644 --- a/python/ray/runtime_env.py +++ b/python/ray/runtime_env.py @@ -1,26 +1,23 @@ -import os -import logging -from typing import Dict, List, Optional, Tuple, Any, Set, Union import json -from google.protobuf import json_format +import logging +import os from copy import deepcopy +from typing import Any, Dict, List, Optional, Set, Tuple, Union + +from google.protobuf import json_format import ray -from ray.core.generated.runtime_env_common_pb2 import ( - RuntimeEnv as ProtoRuntimeEnv, - RuntimeEnvConfig as ProtoRuntimeEnvConfig, -) +from ray._private.ray_constants import DEFAULT_RUNTIME_ENV_TIMEOUT_SECONDS +from ray._private.runtime_env.conda import get_uri as get_conda_uri +from ray._private.runtime_env.pip import get_uri as get_pip_uri from ray._private.runtime_env.plugin import RuntimeEnvPlugin from ray._private.runtime_env.validation import OPTION_TO_VALIDATION_FN from ray._private.utils import import_attr -from ray._private.runtime_env.conda import ( - get_uri as get_conda_uri, +from ray.core.generated.runtime_env_common_pb2 import RuntimeEnv as ProtoRuntimeEnv +from ray.core.generated.runtime_env_common_pb2 import ( + RuntimeEnvConfig as ProtoRuntimeEnvConfig, ) - -from ray._private.runtime_env.pip import get_uri as get_pip_uri from ray.util.annotations import PublicAPI -from ray.ray_constants import DEFAULT_RUNTIME_ENV_TIMEOUT_SECONDS - logger = logging.getLogger(__name__) @@ -246,7 +243,7 @@ class RuntimeEnv(dict): # Example for using container RuntimeEnv( container={"image": "anyscale/ray-ml:nightly-py38-cpu", - "worker_path": "/root/python/ray/workers/default_worker.py", + "worker_path": "/root/python/ray/_private/workers/default_worker.py", "run_options": ["--cap-drop SYS_ADMIN","--log-level=debug"]}) # Example for set env_vars diff --git a/python/ray/scripts/scripts.py b/python/ray/scripts/scripts.py index bab9ad504..daa186944 100644 --- a/python/ray/scripts/scripts.py +++ b/python/ray/scripts/scripts.py @@ -1,8 +1,4 @@ -from typing import Optional, Set - -import click import copy -from datetime import datetime import json import logging import os @@ -11,48 +7,49 @@ import sys import time import urllib import urllib.parse +from datetime import datetime +from distutils.dir_util import copy_tree +from typing import Optional, Set + +import click +import psutil import yaml import ray -import psutil -from ray._private.usage import usage_lib +import ray._private.ray_constants as ray_constants import ray._private.services as services -import ray.ray_constants as ray_constants import ray._private.utils -from ray.util.annotations import PublicAPI +from ray._private.internal_api import memory_summary +from ray._private.storage import _load_class +from ray._private.usage import usage_lib +from ray.autoscaler._private.cli_logger import add_click_logging_options, cf, cli_logger from ray.autoscaler._private.commands import ( + RUN_ENV_TYPES, attach_cluster, - exec_cluster, create_or_update_cluster, + debug_status, + exec_cluster, + get_cluster_dump_archive, + get_head_node_ip, + get_local_dump_archive, + get_worker_node_ips, + kill_node, monitor_cluster, rsync, teardown_cluster, - get_head_node_ip, - kill_node, - get_worker_node_ips, - get_local_dump_archive, - get_cluster_dump_archive, - debug_status, - RUN_ENV_TYPES, ) from ray.autoscaler._private.constants import RAY_PROCESSES from ray.autoscaler._private.fake_multi_node.node_provider import FAKE_HEAD_NODE_ID from ray.autoscaler._private.kuberay.run_autoscaler import run_kuberay_autoscaler -from ray.internal.internal_api import memory_summary -from ray.internal.storage import _load_class -from ray.autoscaler._private.cli_logger import add_click_logging_options, cli_logger, cf from ray.dashboard.modules.job.cli import job_cli_group -from ray.experimental.state.state_cli import list as cli_list -from ray.experimental.state.api import ( - get_log, - list_logs, -) +from ray.experimental.state.api import get_log, list_logs +from ray.experimental.state.common import DEFAULT_LIMIT from ray.experimental.state.state_cli import ( get_api_server_url, get_state_api_output_to_print, ) -from ray.experimental.state.common import DEFAULT_LIMIT -from distutils.dir_util import copy_tree +from ray.experimental.state.state_cli import list as cli_list +from ray.util.annotations import PublicAPI logger = logging.getLogger(__name__) @@ -178,7 +175,7 @@ def continue_debug_session(live_jobs: Set[str]): ) return host, port = session["pdb_address"].split(":") - ray.util.rpdb.connect_pdb_client(host, int(port)) + ray.util.rpdb._connect_pdb_client(host, int(port)) ray.experimental.internal_kv._internal_kv_del( key, namespace=ray_constants.KV_NAMESPACE_PDB ) @@ -209,7 +206,9 @@ def debug(address): ray.init(address=address, log_to_driver=False) while True: # Used to filter out and clean up entries from dead jobs. - live_jobs = {job["JobID"] for job in ray.state.jobs() if not job["IsDead"]} + live_jobs = { + job["JobID"] for job in ray._private.state.jobs() if not job["IsDead"] + } continue_debug_session(live_jobs) active_sessions = ray.experimental.internal_kv._internal_kv_list( @@ -264,7 +263,7 @@ def debug(address): ) ) host, port = session["pdb_address"].split(":") - ray.util.rpdb.connect_pdb_client(host, int(port)) + ray.util.rpdb._connect_pdb_client(host, int(port)) @cli.command() @@ -743,7 +742,7 @@ def start( " flag of `ray start` command." ) - node = ray.node.Node( + node = ray._private.node.Node( ray_params, head=True, shutdown_at_exit=block, spawn_reaper=block ) @@ -889,7 +888,7 @@ def start( cli_logger.labeled_value("Local node IP", ray_params.node_ip_address) - node = ray.node.Node( + node = ray._private.node.Node( ray_params, head=False, shutdown_at_exit=block, spawn_reaper=block ) @@ -2229,7 +2228,7 @@ def global_gc(address): address = services.canonicalize_bootstrap_address(address) logger.info(f"Connecting to Ray instance at {address}.") ray.init(address=address) - ray.internal.internal_api.global_gc() + ray._private.internal_api.global_gc() print("Triggered gc.collect() on all workers.") @@ -2312,7 +2311,7 @@ def healthcheck(address, redis_password, component): # If the status is too old, the service has probably already died. delta = cur_time - report_time - time_ok = delta < ray.ray_constants.HEALTHCHECK_EXPIRATION_S + time_ok = delta < ray._private.ray_constants.HEALTHCHECK_EXPIRATION_S if time_ok: sys.exit(0) diff --git a/python/ray/serve/__init__.py b/python/ray/serve/__init__.py index dc8b267ea..8aaf5c67a 100644 --- a/python/ray/serve/__init__.py +++ b/python/ray/serve/__init__.py @@ -1,13 +1,17 @@ +# Mute the warning because Serve sometimes intentionally calls +# ray.get inside async actors. +import ray._private.worker + try: from ray.serve.api import ( - start, - get_replica_context, - shutdown, - ingress, deployment, get_deployment, + get_replica_context, + ingress, list_deployments, run, + shutdown, + start, ) from ray.serve.batching import batch from ray.serve.config import HTTPOptions @@ -18,11 +22,8 @@ except ModuleNotFoundError as e: ) raise e -# Mute the warning because Serve sometimes intentionally calls -# ray.get inside async actors. -import ray.worker -ray.worker.blocking_get_inside_async_warned = True +ray._private.worker.blocking_get_inside_async_warned = True __all__ = [ "batch", diff --git a/python/ray/serve/api.py b/python/ray/serve/api.py index 6c3a083bb..1b764c92c 100644 --- a/python/ray/serve/api.py +++ b/python/ray/serve/api.py @@ -1,15 +1,7 @@ import collections import inspect import logging -from typing import ( - Any, - Callable, - Dict, - Optional, - Tuple, - Union, - overload, -) +from typing import Any, Callable, Dict, Optional, Tuple, Union, overload from fastapi import APIRouter, FastAPI from starlette.requests import Request @@ -20,51 +12,44 @@ import ray from ray import cloudpickle from ray.dag import DAGNode from ray._private.usage import usage_lib -from ray.util.annotations import PublicAPI from ray.serve.application import Application from ray.serve.client import ServeControllerClient -from ray.serve.config import ( - AutoscalingConfig, - DeploymentConfig, - HTTPOptions, -) +from ray.serve.config import AutoscalingConfig, DeploymentConfig, HTTPOptions from ray.serve.constants import ( + CONTROLLER_MAX_CONCURRENCY, DEFAULT_CHECKPOINT_PATH, + DEFAULT_HTTP_HOST, + DEFAULT_HTTP_PORT, HTTP_PROXY_TIMEOUT, SERVE_CONTROLLER_NAME, SERVE_NAMESPACE, - CONTROLLER_MAX_CONCURRENCY, - DEFAULT_HTTP_HOST, - DEFAULT_HTTP_PORT, ) from ray.serve.context import ( - set_global_client, + ReplicaContext, get_global_client, get_internal_replica_context, - ReplicaContext, + set_global_client, ) from ray.serve.controller import ServeController from ray.serve.deployment import Deployment from ray.serve.deployment_graph import ClassNode, FunctionNode +from ray.serve.deployment_graph_build import build as pipeline_build +from ray.serve.deployment_graph_build import get_and_validate_ingress_deployment from ray.serve.exceptions import RayServeException from ray.serve.handle import RayServeHandle from ray.serve.http_util import ASGIHTTPSender, make_fastapi_class_based_view from ray.serve.logging_utils import LoggingContext -from ray.serve.deployment_graph_build import ( - build as pipeline_build, - get_and_validate_ingress_deployment, -) from ray.serve.utils import ( + DEFAULT, ensure_serialization_context, format_actor_name, get_current_node_resource_key, get_random_letters, in_interactive_shell, - DEFAULT, install_serve_encoders_to_fastapi, ) - +from ray.util.annotations import PublicAPI logger = logging.getLogger(__file__) @@ -125,7 +110,7 @@ def start( f'{{"{key}": {kwargs[key]}}}) instead.' ) # Initialize ray if needed. - ray.worker.global_worker.filter_logs_by_job = False + ray._private.worker.global_worker.filter_logs_by_job = False if not ray.is_initialized(): ray.init(namespace=SERVE_NAMESPACE) diff --git a/python/ray/serve/context.py b/python/ray/serve/context.py index 26e18f137..273e3ae20 100644 --- a/python/ray/serve/context.py +++ b/python/ray/serve/context.py @@ -9,10 +9,10 @@ from typing import Callable import ray from ray.exceptions import RayActorError +from ray.serve.client import ServeControllerClient from ray.serve.common import ReplicaTag from ray.serve.constants import SERVE_CONTROLLER_NAME, SERVE_NAMESPACE from ray.serve.exceptions import RayServeException -from ray.serve.client import ServeControllerClient logger = logging.getLogger(__file__) @@ -92,8 +92,8 @@ def _connect() -> ServeControllerClient: RayServeException: if there is no running Serve controller actor. """ - # Initialize Ray if needed. - ray.worker.global_worker.filter_logs_by_job = False + # Initialize ray if needed. + ray._private.worker.global_worker.filter_logs_by_job = False if not ray.is_initialized(): ray.init(namespace=SERVE_NAMESPACE) diff --git a/python/ray/serve/deployment_graph_build.py b/python/ray/serve/deployment_graph_build.py index a7277d685..8d0b9d1b1 100644 --- a/python/ray/serve/deployment_graph_build.py +++ b/python/ray/serve/deployment_graph_build.py @@ -21,7 +21,7 @@ from ray.dag import ( ) from ray.dag.function_node import FunctionNode from ray.dag.input_node import InputNode -from ray.dag.utils import DAGNodeNameGenerator +from ray.dag.utils import _DAGNodeNameGenerator def build(ray_dag_root_node: DAGNode) -> List[Deployment]: @@ -76,7 +76,7 @@ def build(ray_dag_root_node: DAGNode) -> List[Deployment]: >>> deployments = build_app(ray_dag) # it can be method node >>> deployments = build_app(m1) # or just a regular node. """ - with DAGNodeNameGenerator() as node_name_generator: + with _DAGNodeNameGenerator() as node_name_generator: serve_root_dag = ray_dag_root_node.apply_recursive( lambda node: transform_ray_dag_to_serve_dag(node, node_name_generator) ) @@ -125,7 +125,7 @@ def get_and_validate_ingress_deployment( def transform_ray_dag_to_serve_dag( - dag_node: DAGNode, node_name_generator: DAGNodeNameGenerator + dag_node: DAGNode, node_name_generator: _DAGNodeNameGenerator ): """ Transform a Ray DAG to a Serve DAG. Map ClassNode to DeploymentNode with diff --git a/python/ray/serve/logging_utils.py b/python/ray/serve/logging_utils.py index 268e1efa6..fa7fcf1a1 100644 --- a/python/ray/serve/logging_utils.py +++ b/python/ray/serve/logging_utils.py @@ -47,7 +47,9 @@ def configure_component_logger( logger.addHandler(stream_handler) if log_to_file: - logs_dir = os.path.join(ray.worker._global_node.get_logs_dir_path(), "serve") + logs_dir = os.path.join( + ray._private.worker._global_node.get_logs_dir_path(), "serve" + ) os.makedirs(logs_dir, exist_ok=True) if component_type is not None: component_name = f"{component_type}_{component_name}" diff --git a/python/ray/serve/storage/kv_store.py b/python/ray/serve/storage/kv_store.py index a68f2b7bf..66e01372c 100644 --- a/python/ray/serve/storage/kv_store.py +++ b/python/ray/serve/storage/kv_store.py @@ -4,7 +4,7 @@ import sqlite3 from typing import Optional import ray -from ray import ray_constants +from ray._private import ray_constants from ray._private.gcs_utils import GcsClient from ray.serve.constants import RAY_SERVE_KV_TIMEOUT_S, SERVE_LOGGER_NAME from ray.serve.storage.kv_store_base import KVStoreBase diff --git a/python/ray/serve/tests/test_cluster.py b/python/ray/serve/tests/test_cluster.py index 1a00df10b..15bbd7e2a 100644 --- a/python/ray/serve/tests/test_cluster.py +++ b/python/ray/serve/tests/test_cluster.py @@ -1,16 +1,15 @@ -from collections import defaultdict import os import sys import time +from collections import defaultdict import pytest import requests import ray -from ray.cluster_utils import Cluster -from ray._private.test_utils import SignalActor, wait_for_condition - from ray import serve +from ray._private.test_utils import SignalActor, wait_for_condition +from ray.cluster_utils import Cluster from ray.serve.constants import SERVE_NAMESPACE from ray.serve.deployment_state import ReplicaStartupStatus, ReplicaState @@ -179,7 +178,7 @@ def test_intelligent_scale_down(ray_cluster): pass def get_actor_distributions(): - actors = ray.state.actors() + actors = ray._private.state.actors() node_to_actors = defaultdict(list) for actor in actors.values(): if "RayServeWrappedReplica" not in actor["ActorClassName"]: diff --git a/python/ray/serve/tests/test_deployment_graph_build.py b/python/ray/serve/tests/test_deployment_graph_build.py index f64b32786..f2e6fd4a9 100644 --- a/python/ray/serve/tests/test_deployment_graph_build.py +++ b/python/ray/serve/tests/test_deployment_graph_build.py @@ -23,7 +23,7 @@ from ray.serve.tests.resources.test_dags import ( get_multi_instantiation_class_nested_deployment_arg_dag, get_simple_func_dag, ) -from ray.dag.utils import DAGNodeNameGenerator +from ray.dag.utils import _DAGNodeNameGenerator def _validate_consistent_python_output( @@ -46,7 +46,7 @@ def _validate_consistent_python_output( ) def test_build_simple_func_dag(serve_instance): ray_dag, _ = get_simple_func_dag() - with DAGNodeNameGenerator() as node_name_generator: + with _DAGNodeNameGenerator() as node_name_generator: serve_root_dag = ray_dag.apply_recursive( lambda node: transform_ray_dag_to_serve_dag(node, node_name_generator) ) @@ -65,7 +65,7 @@ def test_build_simple_func_dag(serve_instance): def test_simple_single_class(serve_instance): ray_dag, _ = get_simple_class_with_class_method_dag() - with DAGNodeNameGenerator() as node_name_generator: + with _DAGNodeNameGenerator() as node_name_generator: serve_root_dag = ray_dag.apply_recursive( lambda node: transform_ray_dag_to_serve_dag(node, node_name_generator) ) @@ -82,7 +82,7 @@ def test_single_class_with_valid_ray_options(serve_instance): model = Model.options(num_cpus=1, memory=1000).bind(2, ratio=0.3) ray_dag = model.forward.bind(dag_input) - with DAGNodeNameGenerator() as node_name_generator: + with _DAGNodeNameGenerator() as node_name_generator: serve_root_dag = ray_dag.apply_recursive( lambda node: transform_ray_dag_to_serve_dag(node, node_name_generator) ) @@ -104,7 +104,7 @@ def test_single_class_with_invalid_deployment_options(serve_instance): model = Model.options(name="my_deployment").bind(2, ratio=0.3) ray_dag = model.forward.bind(dag_input) - with DAGNodeNameGenerator() as node_name_generator: + with _DAGNodeNameGenerator() as node_name_generator: serve_root_dag = ray_dag.apply_recursive( lambda node: transform_ray_dag_to_serve_dag(node, node_name_generator) ) @@ -119,7 +119,7 @@ def test_single_class_with_invalid_deployment_options(serve_instance): def test_func_class_with_class_method_dag(serve_instance): ray_dag, _ = get_func_class_with_class_method_dag() - with DAGNodeNameGenerator() as node_name_generator: + with _DAGNodeNameGenerator() as node_name_generator: serve_root_dag = ray_dag.apply_recursive( lambda node: transform_ray_dag_to_serve_dag(node, node_name_generator) ) @@ -143,7 +143,7 @@ def test_multi_instantiation_class_deployment_in_init_args(serve_instance): """ ray_dag, _ = get_multi_instantiation_class_deployment_in_init_args_dag() - with DAGNodeNameGenerator() as node_name_generator: + with _DAGNodeNameGenerator() as node_name_generator: serve_root_dag = ray_dag.apply_recursive( lambda node: transform_ray_dag_to_serve_dag(node, node_name_generator) ) @@ -165,7 +165,7 @@ def test_shared_deployment_handle(serve_instance): """ ray_dag, _ = get_shared_deployment_handle_dag() - with DAGNodeNameGenerator() as node_name_generator: + with _DAGNodeNameGenerator() as node_name_generator: serve_root_dag = ray_dag.apply_recursive( lambda node: transform_ray_dag_to_serve_dag(node, node_name_generator) ) @@ -188,7 +188,7 @@ def test_multi_instantiation_class_nested_deployment_arg(serve_instance): """ ray_dag, _ = get_multi_instantiation_class_nested_deployment_arg_dag() - with DAGNodeNameGenerator() as node_name_generator: + with _DAGNodeNameGenerator() as node_name_generator: serve_root_dag = ray_dag.apply_recursive( lambda node: transform_ray_dag_to_serve_dag(node, node_name_generator) ) @@ -216,7 +216,7 @@ def test_multi_instantiation_class_nested_deployment_arg(serve_instance): def test_get_pipeline_input_node(): # 1) No InputNode found ray_dag = combine.bind(1, 2) - with DAGNodeNameGenerator() as node_name_generator: + with _DAGNodeNameGenerator() as node_name_generator: serve_dag = ray_dag.apply_recursive( lambda node: transform_ray_dag_to_serve_dag(node, node_name_generator) ) @@ -234,7 +234,7 @@ def test_get_pipeline_input_node(): with pytest.raises( AssertionError, match="Each DAG should only have one unique InputNode" ): - with DAGNodeNameGenerator() as node_name_generator: + with _DAGNodeNameGenerator() as node_name_generator: serve_dag = ray_dag.apply_recursive( lambda node: transform_ray_dag_to_serve_dag(node, node_name_generator) ) @@ -243,7 +243,7 @@ def test_get_pipeline_input_node(): def test_unique_name_reset_upon_build(serve_instance): ray_dag, _ = get_multi_instantiation_class_deployment_in_init_args_dag() - with DAGNodeNameGenerator() as node_name_generator: + with _DAGNodeNameGenerator() as node_name_generator: serve_root_dag = ray_dag.apply_recursive( lambda node: transform_ray_dag_to_serve_dag(node, node_name_generator) ) @@ -251,7 +251,7 @@ def test_unique_name_reset_upon_build(serve_instance): assert deployments[0].name == "Model" assert deployments[1].name == "Model_1" - with DAGNodeNameGenerator() as node_name_generator: + with _DAGNodeNameGenerator() as node_name_generator: serve_root_dag = ray_dag.apply_recursive( lambda node: transform_ray_dag_to_serve_dag(node, node_name_generator) ) diff --git a/python/ray/serve/tests/test_json_serde.py b/python/ray/serve/tests/test_json_serde.py index 53c7e4f51..5573d3d00 100644 --- a/python/ray/serve/tests/test_json_serde.py +++ b/python/ray/serve/tests/test_json_serde.py @@ -6,7 +6,7 @@ import ray from ray.dag.dag_node import DAGNode from ray.dag.input_node import InputNode from ray import serve -from ray.dag.utils import DAGNodeNameGenerator +from ray.dag.utils import _DAGNodeNameGenerator from ray.serve.handle import ( RayServeSyncHandle, serve_handle_to_json_dict, @@ -235,7 +235,7 @@ def _test_deployment_json_serde_helper( 3) Deserialized serve dag can extract correct number and definition of serve deployments. """ - with DAGNodeNameGenerator() as node_name_generator: + with _DAGNodeNameGenerator() as node_name_generator: serve_root_dag = ray_dag.apply_recursive( lambda node: transform_ray_dag_to_serve_dag(node, node_name_generator) ) diff --git a/python/ray/serve/tests/test_persistence.py b/python/ray/serve/tests/test_persistence.py index 2a8be9f85..50c736d36 100644 --- a/python/ray/serve/tests/test_persistence.py +++ b/python/ray/serve/tests/test_persistence.py @@ -1,6 +1,6 @@ import ray -from ray._private.test_utils import run_string_as_driver from ray import serve +from ray._private.test_utils import run_string_as_driver def test_new_driver(serve_instance): @@ -16,7 +16,7 @@ def driver(): driver.deploy() """.format( - ray.worker._global_node.address + ray._private.worker._global_node.address ) run_string_as_driver(script) @@ -26,6 +26,7 @@ driver.deploy() if __name__ == "__main__": import sys + import pytest sys.exit(pytest.main(["-v", "-s", __file__])) diff --git a/python/ray/serve/tests/test_pipeline_dag_plot.py b/python/ray/serve/tests/test_pipeline_dag_plot.py index 117ae2c57..799a2fac0 100644 --- a/python/ray/serve/tests/test_pipeline_dag_plot.py +++ b/python/ray/serve/tests/test_pipeline_dag_plot.py @@ -6,7 +6,7 @@ import tempfile import ray from ray import serve from ray.dag import DAGNode -from ray.dag.utils import DAGNodeNameGenerator +from ray.dag.utils import _DAGNodeNameGenerator from ray.serve.deployment_graph import RayServeDAGHandle from ray.serve.deployment_graph import InputNode from ray.serve.drivers import DAGDriver @@ -116,7 +116,7 @@ async def json_resolver(request: starlette.requests.Request): def ray_dag_to_serve_dag(dag: DAGNode): - with DAGNodeNameGenerator() as deployment_name_generator: + with _DAGNodeNameGenerator() as deployment_name_generator: serve_dag = dag.apply_recursive( lambda node: transform_ray_dag_to_serve_dag(node, deployment_name_generator) ) @@ -133,7 +133,7 @@ def test_serve_pipeline_single_func_no_input_plot(): ray.dag.plot(serve_dag, to_file) assert os.path.isfile(to_file) - graph = ray.dag.vis_utils.dag_to_dot(serve_dag) + graph = ray.dag.vis_utils._dag_to_dot(serve_dag) to_string = graph.to_string() assert "fn_hello -> NoargDriver" in to_string @@ -149,7 +149,7 @@ def test_serve_pipeline_single_func_deployment_dag_plot(): ray.dag.plot(serve_dag, to_file) assert os.path.isfile(to_file) - graph = ray.dag.vis_utils.dag_to_dot(serve_dag) + graph = ray.dag.vis_utils._dag_to_dot(serve_dag) to_string = graph.to_string() assert "INPUT_NODE -> INPUT_ATTRIBUTE_NODE" in to_string assert "INPUT_NODE -> INPUT_ATTRIBUTE_NODE_1" in to_string @@ -178,7 +178,7 @@ def test_serve_pipeline_chained_function_plot(): ray.dag.plot(serve_dag, to_file) assert os.path.isfile(to_file) - graph = ray.dag.vis_utils.dag_to_dot(serve_dag) + graph = ray.dag.vis_utils._dag_to_dot(serve_dag) to_string = graph.to_string() assert "INPUT_NODE -> func_1" in to_string assert "INPUT_NODE -> func_2" in to_string @@ -198,7 +198,7 @@ def test_serve_pipeline_class_with_class_method_plot(): ray.dag.plot(serve_dag, to_file) assert os.path.isfile(to_file) - graph = ray.dag.vis_utils.dag_to_dot(serve_dag) + graph = ray.dag.vis_utils._dag_to_dot(serve_dag) to_string = graph.to_string() assert "Model -> forward" in to_string assert "INPUT_NODE -> forward" in to_string @@ -220,7 +220,7 @@ def test_serve_pipeline_func_class_with_class_method_plot(): ray.dag.plot(serve_dag, to_file) assert os.path.isfile(to_file) - graph = ray.dag.vis_utils.dag_to_dot(serve_dag) + graph = ray.dag.vis_utils._dag_to_dot(serve_dag) to_string = graph.to_string() assert "INPUT_NODE -> INPUT_ATTRIBUTE_NODE" in to_string assert "INPUT_NODE -> INPUT_ATTRIBUTE_NODE_1" in to_string @@ -249,7 +249,7 @@ def test_serve_pipeline_multi_instantiation_class_deployment_in_init_args_plot() ray.dag.plot(serve_dag, to_file) assert os.path.isfile(to_file) - graph = ray.dag.vis_utils.dag_to_dot(serve_dag) + graph = ray.dag.vis_utils._dag_to_dot(serve_dag) to_string = graph.to_string() assert "Model -> Combine" in to_string assert "Model_1 -> Combine" in to_string @@ -271,7 +271,7 @@ def test_serve_pipeline_test_shared_deployment_handle_plot(): ray.dag.plot(serve_dag, to_file) assert os.path.isfile(to_file) - graph = ray.dag.vis_utils.dag_to_dot(serve_dag) + graph = ray.dag.vis_utils._dag_to_dot(serve_dag) to_string = graph.to_string() assert "Model -> Combine" in to_string assert "Combine -> __call__" in to_string @@ -293,7 +293,7 @@ def test_serve_pipeline_multi_instantiation_class_nested_deployment_arg_dag_plot ray.dag.plot(serve_dag, to_file) assert os.path.isfile(to_file) - graph = ray.dag.vis_utils.dag_to_dot(serve_dag) + graph = ray.dag.vis_utils._dag_to_dot(serve_dag) to_string = graph.to_string() assert "Model -> Combine" in to_string assert "Model_1 -> Combine" in to_string @@ -314,7 +314,7 @@ def test_serve_pipeline_class_factory_plot(): ray.dag.plot(serve_dag, to_file) assert os.path.isfile(to_file) - graph = ray.dag.vis_utils.dag_to_dot(serve_dag) + graph = ray.dag.vis_utils._dag_to_dot(serve_dag) to_string = graph.to_string() assert "MyInlineClass -> get" in to_string assert "get -> NoargDriver" in to_string diff --git a/python/ray/serve/tests/test_standalone.py b/python/ray/serve/tests/test_standalone.py index 042bf5d8d..4f65cc1f2 100644 --- a/python/ray/serve/tests/test_standalone.py +++ b/python/ray/serve/tests/test_standalone.py @@ -9,37 +9,36 @@ import subprocess import sys from tempfile import mkstemp -import pytest import pydantic -from ray.serve.http_util import set_socket_reuse_port +import pytest import requests import ray import ray._private.gcs_utils as gcs_utils +from ray import serve from ray._private.services import new_port from ray._private.test_utils import ( + convert_actor_state, run_string_as_driver, wait_for_condition, - convert_actor_state, ) from ray.cluster_utils import Cluster, cluster_not_supported - -from ray import serve -from ray.serve.context import get_global_client from ray.serve.config import HTTPOptions from ray.serve.constants import ( SERVE_NAMESPACE, - SERVE_ROOT_URL_ENV_KEY, SERVE_PROXY_NAME, + SERVE_ROOT_URL_ENV_KEY, ) +from ray.serve.context import get_global_client from ray.serve.exceptions import RayServeException from ray.serve.generated.serve_pb2 import ActorNameList -from ray.serve.utils import block_until_http_ready, get_all_node_ids, format_actor_name +from ray.serve.http_util import set_socket_reuse_port +from ray.serve.utils import block_until_http_ready, format_actor_name, get_all_node_ids # Explicitly importing it here because it is a ray core tests utility ( # not in the tree) -from ray.tests.conftest import ray_start_with_dashboard # noqa: F401 from ray.tests.conftest import maybe_external_redis # noqa: F401 +from ray.tests.conftest import ray_start_with_dashboard # noqa: F401 @pytest.fixture @@ -205,7 +204,7 @@ def test_multiple_routers(ray_cluster): cluster.add_node(num_cpus=4) ray.init(head_node.address) - node_ids = ray.state.node_ids() + node_ids = ray._private.state.node_ids() assert len(node_ids) == 2 serve.start(http_options=dict(port=8005, location="EveryNode")) @@ -389,7 +388,7 @@ def test_no_http(ray_shutdown): # Only controller actor should exist live_actors = [ actor - for actor in ray.state.actors().values() + for actor in ray._private.state.actors().values() if actor["State"] == convert_actor_state(gcs_utils.ActorTableData.ALIVE) ] assert len(live_actors) == 1 @@ -413,17 +412,18 @@ def test_http_head_only(ray_cluster): cluster.add_node(num_cpus=4) ray.init(head_node.address) - node_ids = ray.state.node_ids() + node_ids = ray._private.state.node_ids() assert len(node_ids) == 2 serve.start(http_options={"port": new_port(), "location": "HeadOnly"}) # Only the controller and head node actor should be started - assert len(ray.state.actors()) == 2 + assert len(ray._private.state.actors()) == 2 # They should all be placed on the head node cpu_per_nodes = { - r["CPU"] for r in ray.state.state._available_resources_per_node().values() + r["CPU"] + for r in ray._private.state.state._available_resources_per_node().values() } assert cpu_per_nodes == {4, 4} @@ -442,7 +442,7 @@ def test_fixed_number_proxies(ray_cluster): cluster.add_node(num_cpus=4) ray.init(head_node.address) - node_ids = ray.state.node_ids() + node_ids = ray._private.state.node_ids() assert len(node_ids) == 3 with pytest.raises( diff --git a/python/ray/serve/tests/test_standalone2.py b/python/ray/serve/tests/test_standalone2.py index d3a0118c7..cd9e3bd71 100644 --- a/python/ray/serve/tests/test_standalone2.py +++ b/python/ray/serve/tests/test_standalone2.py @@ -10,7 +10,7 @@ import pytest import requests import ray -import ray.state +import ray._private.state from ray import serve from ray._private.test_utils import wait_for_condition from ray.cluster_utils import AutoscalingCluster @@ -680,7 +680,7 @@ def test_autoscaler_shutdown_node_http_everynode( assert ray.get(a.ready.remote()) == 1 # 2 proxies, 1 controller, and one placeholder. - wait_for_condition(lambda: len(ray.state.actors()) == 4) + wait_for_condition(lambda: len(ray._private.state.actors()) == 4) assert len(ray.nodes()) == 2 # Now make sure the placeholder actor exits. @@ -688,7 +688,12 @@ def test_autoscaler_shutdown_node_http_everynode( # The http proxy on worker node should exit as well. wait_for_condition( lambda: len( - list(filter(lambda a: a["State"] == "ALIVE", ray.state.actors().values())) + list( + filter( + lambda a: a["State"] == "ALIVE", + ray._private.state.actors().values(), + ) + ) ) == 2 ) diff --git a/python/ray/serve/utils.py b/python/ray/serve/utils.py index e8904d607..6019aaa13 100644 --- a/python/ray/serve/utils.py +++ b/python/ray/serve/utils.py @@ -1,6 +1,5 @@ import copy import importlib -from itertools import groupby import inspect import os import pickle @@ -10,15 +9,17 @@ import time import traceback from enum import Enum from functools import wraps +from itertools import groupby from typing import Dict, Iterable, List, Tuple import fastapi.encoders import numpy as np import pydantic import pydantic.json -import ray -import ray.serialization_addons import requests + +import ray +import ray.util.serialization_addons from ray.actor import ActorHandle from ray.exceptions import RayTaskError from ray.serve.constants import HTTP_PROXY_TIMEOUT @@ -158,7 +159,7 @@ def get_all_node_ids(): # We need to use the node_id and index here because we could # have multiple virtual nodes on the same host. In that case # they will have the same IP and therefore node_id. - for _, node_id_group in groupby(sorted(ray.state.node_ids())): + for _, node_id_group in groupby(sorted(ray._private.state.node_ids())): for index, node_id in enumerate(node_id_group): node_ids.append(("{}-{}".format(node_id, index), node_id)) @@ -179,7 +180,9 @@ def node_id_to_ip_addr(node_id: str): def get_node_id_for_actor(actor_handle): """Given an actor handle, return the node id it's placed on.""" - return ray.state.actors()[actor_handle._actor_id.hex()]["Address"]["NodeID"] + return ray._private.state.actors()[actor_handle._actor_id.hex()]["Address"][ + "NodeID" + ] def compute_iterable_delta(old: Iterable, new: Iterable) -> Tuple[set, set, set]: @@ -239,7 +242,7 @@ def ensure_serialization_context(): """Ensure the serialization addons on registered, even when Ray has not been started.""" ctx = StandaloneSerializationContext() - ray.serialization_addons.apply(ctx) + ray.util.serialization_addons.apply(ctx) def wrap_to_ray_error(function_name: str, exception: Exception) -> RayTaskError: @@ -254,7 +257,7 @@ def wrap_to_ray_error(function_name: str, exception: Exception) -> RayTaskError: def msgpack_serialize(obj): - ctx = ray.worker.global_worker.get_serialization_context() + ctx = ray._private.worker.global_worker.get_serialization_context() buffer = ctx.serialize(obj) serialized = buffer.to_bytes() return serialized diff --git a/python/ray/sgd/__init__.py b/python/ray/sgd/__init__.py deleted file mode 100644 index c34181ec9..000000000 --- a/python/ray/sgd/__init__.py +++ /dev/null @@ -1 +0,0 @@ -raise DeprecationWarning("ray.sgd has been moved to ray.train.") diff --git a/python/ray/streaming/__init__.py b/python/ray/streaming/__init__.py deleted file mode 100644 index cefe96736..000000000 --- a/python/ray/streaming/__init__.py +++ /dev/null @@ -1,28 +0,0 @@ -# flake8: noqa -# Ray should be imported before streaming -import ray - - -# Redirect ray.streaming.* to raystreaming, so user can keep -# their original habit. -def _update_modules(): - try: - import raystreaming - import raystreaming.context - import sys - - ray_streaming_module_name = raystreaming.__name__ - ray_streaming_modules = {} - for mod_name, module in sys.modules.items(): - if mod_name.startswith(ray_streaming_module_name): - ray_streaming_modules[ - mod_name.replace("raystreaming", "ray.streaming") - ] = module - sys.modules.update(ray_streaming_modules) - except Exception as e: - print("import raystreaming error: ", e) - - -_update_modules() - -__all__ = ["StreamingContext"] diff --git a/python/ray/tests/aws/test_autoscaler_aws.py b/python/ray/tests/aws/test_autoscaler_aws.py index a04c046a7..ff30367d9 100644 --- a/python/ray/tests/aws/test_autoscaler_aws.py +++ b/python/ray/tests/aws/test_autoscaler_aws.py @@ -1,36 +1,36 @@ import copy - -from click.exceptions import ClickException -import pytest from unittest.mock import Mock, patch +import pytest +from click.exceptions import ClickException + +import ray.tests.aws.utils.helpers as helpers +import ray.tests.aws.utils.stubs as stubs from ray.autoscaler._private.aws.config import ( + DEFAULT_AMI, _configure_subnet, _get_subnets_or_die, bootstrap_aws, log_to_cli, - DEFAULT_AMI, ) from ray.autoscaler._private.aws.node_provider import AWSNodeProvider from ray.autoscaler._private.providers import _get_node_provider -import ray.tests.aws.utils.stubs as stubs -import ray.tests.aws.utils.helpers as helpers from ray.tests.aws.utils.constants import ( - AUX_SUBNET, - DEFAULT_SUBNET, - DEFAULT_SG_AUX_SUBNET, - DEFAULT_SG, - DEFAULT_SG_DUAL_GROUP_RULES, - DEFAULT_SG_WITH_RULES_AUX_SUBNET, AUX_SG, - DEFAULT_SG_WITH_RULES, + AUX_SUBNET, + CUSTOM_IN_BOUND_RULES, + DEFAULT_CLUSTER_NAME, + DEFAULT_INSTANCE_PROFILE, + DEFAULT_KEY_PAIR, + DEFAULT_LT, + DEFAULT_SG, + DEFAULT_SG_AUX_SUBNET, + DEFAULT_SG_DUAL_GROUP_RULES, DEFAULT_SG_WITH_NAME, DEFAULT_SG_WITH_NAME_AND_RULES, - CUSTOM_IN_BOUND_RULES, - DEFAULT_KEY_PAIR, - DEFAULT_INSTANCE_PROFILE, - DEFAULT_CLUSTER_NAME, - DEFAULT_LT, + DEFAULT_SG_WITH_RULES, + DEFAULT_SG_WITH_RULES_AUX_SUBNET, + DEFAULT_SUBNET, ) diff --git a/python/ray/tests/conftest.py b/python/ray/tests/conftest.py index 4ec5df2f1..7da4f699e 100644 --- a/python/ray/tests/conftest.py +++ b/python/ray/tests/conftest.py @@ -18,7 +18,7 @@ from unittest import mock import pytest import ray -import ray.ray_constants as ray_constants +import ray._private.ray_constants as ray_constants import ray.util.client.server.server as ray_client_server from ray._private.runtime_env.pip import PipProcessor from ray._private.services import ( diff --git a/python/ray/tests/test_actor.py b/python/ray/tests/test_actor.py index ec71b225c..5a12a70a4 100644 --- a/python/ray/tests/test_actor.py +++ b/python/ray/tests/test_actor.py @@ -1,31 +1,32 @@ -import random -import pytest -import numpy as np +import datetime import os -from ray import cloudpickle as pickle -from ray import ray_constants -from ray.actor import ActorClassInheritanceException - -try: - import pytest_timeout -except ImportError: - pytest_timeout = None +import random import sys import tempfile -import datetime +import numpy as np +import pytest + +import ray +from ray import cloudpickle as pickle +from ray._private import ray_constants from ray._private.test_utils import ( client_test_enabled, wait_for_condition, wait_for_pid_to_exit, ) +from ray.actor import ActorClassInheritanceException from ray.tests.client_test_utils import create_remote_signal_actor -import ray # NOTE: We have to import setproctitle after ray because we bundle setproctitle # with ray. import setproctitle # noqa +try: + import pytest_timeout +except ImportError: + pytest_timeout = None + @pytest.mark.parametrize("set_enable_auto_connect", ["1", "0"], indirect=True) def test_caching_actors(shutdown_only, set_enable_auto_connect): @@ -289,8 +290,8 @@ def test_actor_method_metadata_cache(ray_start_regular): class Actor(object): pass - # The cache of ActorClassMethodMetadata. - cache = ray.actor.ActorClassMethodMetadata._cache + # The cache of _ActorClassMethodMetadata. + cache = ray.actor._ActorClassMethodMetadata._cache cache.clear() # Check cache hit during ActorHandle deserialization. @@ -300,7 +301,7 @@ def test_actor_method_metadata_cache(ray_start_regular): cached_data_id = [id(x) for x in list(cache.items())[0]] for x in range(10): a = pickle.loads(pickle.dumps(a)) - assert len(ray.actor.ActorClassMethodMetadata._cache) == 1 + assert len(ray.actor._ActorClassMethodMetadata._cache) == 1 assert [id(x) for x in list(cache.items())[0]] == cached_data_id @@ -312,7 +313,7 @@ def test_actor_class_name(ray_start_regular): pass Foo.remote() - g = ray.worker.global_worker.gcs_client + g = ray._private.worker.global_worker.gcs_client actor_keys = g.internal_kv_keys( b"ActorClass", ray_constants.KV_NAMESPACE_FUNCTION_TABLE ) diff --git a/python/ray/tests/test_actor_advanced.py b/python/ray/tests/test_actor_advanced.py index 4342a7fd1..104a64d05 100644 --- a/python/ray/tests/test_actor_advanced.py +++ b/python/ray/tests/test_actor_advanced.py @@ -1,28 +1,29 @@ -import numpy as np import os +import sys +import time + +import numpy as np import pytest +import ray +import ray._private.gcs_utils as gcs_utils +import ray.cluster_utils +from ray._private.test_utils import ( + SignalActor, + convert_actor_state, + get_non_head_nodes, + kill_actor_and_wait_for_failure, + make_global_state_accessor, + run_string_as_driver, + wait_for_condition, + wait_for_pid_to_exit, +) +from ray.experimental.internal_kv import _internal_kv_get, _internal_kv_put + try: import pytest_timeout except ImportError: pytest_timeout = None -import sys -import time - -import ray -import ray.cluster_utils -import ray._private.gcs_utils as gcs_utils -from ray._private.test_utils import ( - run_string_as_driver, - get_non_head_nodes, - kill_actor_and_wait_for_failure, - make_global_state_accessor, - SignalActor, - wait_for_condition, - wait_for_pid_to_exit, - convert_actor_state, -) -from ray.experimental.internal_kv import _internal_kv_get, _internal_kv_put def test_remote_functions_not_scheduled_on_actors(ray_start_regular): @@ -34,14 +35,14 @@ def test_remote_functions_not_scheduled_on_actors(ray_start_regular): pass def get_id(self): - return ray.worker.global_worker.worker_id + return ray._private.worker.global_worker.worker_id a = Actor.remote() actor_id = ray.get(a.get_id.remote()) @ray.remote def f(): - return ray.worker.global_worker.worker_id + return ray._private.worker.global_worker.worker_id resulting_ids = ray.get([f.remote() for _ in range(100)]) assert actor_id not in resulting_ids @@ -71,7 +72,7 @@ def test_actor_load_balancing(ray_start_cluster): pass def get_location(self): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id # Create a bunch of actors. num_actors = 30 @@ -172,7 +173,7 @@ def test_exception_raised_when_actor_node_dies(ray_start_cluster_head): self.x = 0 def node_id(self): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id def inc(self): self.x += 1 @@ -278,7 +279,7 @@ def setup_counter_actor( self.restored = False def node_id(self): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id def inc(self, *xs): self.x += 1 @@ -305,7 +306,7 @@ def setup_counter_actor( self.num_inc_calls = 0 self.restored = True - node_id = ray.worker.global_worker.node.unique_id + node_id = ray._private.worker.global_worker.node.unique_id # Create an actor that is not on the raylet. actor = Counter.remote(save_exception) @@ -503,7 +504,7 @@ def test_pickled_handle_consistency(setup_queue_actor): @ray.remote def fork(pickled_queue, key, num_items): - queue = ray.worker.pickle.loads(pickled_queue) + queue = ray._private.worker.pickle.loads(pickled_queue) x = None for item in range(num_items): x = queue.enqueue.remote(key, item) @@ -514,7 +515,7 @@ def test_pickled_handle_consistency(setup_queue_actor): num_items_per_fork = 100 # Submit some tasks on the pickled actor handle. - new_queue = ray.worker.pickle.dumps(queue) + new_queue = ray._private.worker.pickle.dumps(queue) forks = [fork.remote(new_queue, i, num_items_per_fork) for i in range(num_forks)] # Submit some more tasks on the original actor handle. for item in range(num_items_per_fork): @@ -752,13 +753,17 @@ def test_detached_actor_cleanup(ray_start_regular): detached_actor = ray.get_actor(dup_actor_name) ray.kill(detached_actor) # Wait until actor dies. - actor_status = ray.state.actors(actor_id=detached_actor._actor_id.hex()) + actor_status = ray._private.state.actors( + actor_id=detached_actor._actor_id.hex() + ) max_wait_time = 10 wait_time = 0 while actor_status["State"] != convert_actor_state( gcs_utils.ActorTableData.DEAD ): - actor_status = ray.state.actors(actor_id=detached_actor._actor_id.hex()) + actor_status = ray._private.state.actors( + actor_id=detached_actor._actor_id.hex() + ) time.sleep(1.0) wait_time += 1 if wait_time >= max_wait_time: @@ -790,11 +795,11 @@ detached_actor = DetachedActor.options(lifetime="detached", name="{}").remote() assert ray.get(detached_actor.ping.remote()) == "pong" ray.kill(detached_actor) # Wait until actor dies. -actor_status = ray.state.actors(actor_id=detached_actor._actor_id.hex()) +actor_status = ray._private.state.actors(actor_id=detached_actor._actor_id.hex()) max_wait_time = 10 wait_time = 0 while actor_status["State"] != convert_actor_state(gcs_utils.ActorTableData.DEAD): # noqa - actor_status = ray.state.actors(actor_id=detached_actor._actor_id.hex()) + actor_status = ray._private.state.actors(actor_id=detached_actor._actor_id.hex()) time.sleep(1.0) wait_time += 1 if wait_time >= max_wait_time: @@ -865,13 +870,13 @@ def test_detached_actor_cleanup_due_to_failure(ray_start_cluster): node_failure_actor_name = "node_failure_actor_name" def wait_until_actor_dead(handle): - actor_status = ray.state.actors(actor_id=handle._actor_id.hex()) + actor_status = ray._private.state.actors(actor_id=handle._actor_id.hex()) max_wait_time = 10 wait_time = 0 while actor_status["State"] != convert_actor_state( gcs_utils.ActorTableData.DEAD ): - actor_status = ray.state.actors(actor_id=handle._actor_id.hex()) + actor_status = ray._private.state.actors(actor_id=handle._actor_id.hex()) time.sleep(1.0) wait_time += 1 if wait_time >= max_wait_time: @@ -1003,7 +1008,7 @@ def test_pickling_actor_handle(ray_start_regular_shared): pass f = Foo.remote() - new_f = ray.worker.pickle.loads(ray.worker.pickle.dumps(f)) + new_f = ray._private.worker.pickle.loads(ray._private.worker.pickle.dumps(f)) # Verify that we can call a method on the unpickled handle. TODO(rkn): # we should also test this from a different driver. ray.get(new_f.method.remote()) @@ -1206,11 +1211,11 @@ def test_actor_timestamps(ray_start_regular): actor = Foo.remote() actor_id = ray.get(actor.get_id.remote()) - state_after_starting = ray.state.actors()[actor_id] + state_after_starting = ray._private.state.actors()[actor_id] time.sleep(1) del actor time.sleep(1) - state_after_ending = ray.state.actors()[actor_id] + state_after_ending = ray._private.state.actors()[actor_id] assert state_after_starting["StartTime"] == state_after_ending["StartTime"] @@ -1225,11 +1230,11 @@ def test_actor_timestamps(ray_start_regular): actor = Foo.remote() actor_id = ray.get(actor.get_id.remote()) - state_after_starting = ray.state.actors()[actor_id] + state_after_starting = ray._private.state.actors()[actor_id] time.sleep(1) actor.kill_self.remote() time.sleep(1) - state_after_ending = ray.state.actors()[actor_id] + state_after_ending = ray._private.state.actors()[actor_id] assert state_after_starting["StartTime"] == state_after_ending["StartTime"] @@ -1244,13 +1249,13 @@ def test_actor_timestamps(ray_start_regular): actor = Foo.options(max_restarts=1, max_task_retries=-1).remote() actor_id = ray.get(actor.get_id.remote()) - state_after_starting = ray.state.actors()[actor_id] + state_after_starting = ray._private.state.actors()[actor_id] time.sleep(1) actor.kill_self.remote() time.sleep(1) actor.kill_self.remote() time.sleep(1) - state_after_ending = ray.state.actors()[actor_id] + state_after_ending = ray._private.state.actors()[actor_id] assert state_after_starting["StartTime"] == state_after_ending["StartTime"] diff --git a/python/ray/tests/test_actor_group.py b/python/ray/tests/test_actor_group.py index 3f9757a1b..a74564235 100644 --- a/python/ray/tests/test_actor_group.py +++ b/python/ray/tests/test_actor_group.py @@ -1,6 +1,7 @@ -import pytest import time +import pytest + import ray from ray.util.actor_group import ActorGroup @@ -38,7 +39,7 @@ def test_actor_shutdown(ray_start_2_cpus): ag = ActorGroup(actor_cls=DummyActor, num_actors=2) time.sleep(1) assert "CPU" not in ray.available_resources() - assert len(ray.state.actors()) == 2 + assert len(ray._private.state.actors()) == 2 ag.shutdown() time.sleep(1) assert ray.available_resources()["CPU"] == 2 diff --git a/python/ray/tests/test_actor_resources.py b/python/ray/tests/test_actor_resources.py index 8e137a4d4..f0b99dede 100644 --- a/python/ray/tests/test_actor_resources.py +++ b/python/ray/tests/test_actor_resources.py @@ -1,16 +1,17 @@ import collections import os +import sys +import time + import pytest +import ray +import ray.cluster_utils + try: import pytest_timeout except ImportError: pytest_timeout = None -import sys -import time - -import ray -import ray.cluster_utils def test_actor_deletion_with_gpus(shutdown_only): @@ -96,7 +97,10 @@ def test_actor_gpus(ray_start_cluster): def get_location_and_ids(self): assert ray.get_gpu_ids() == self.gpu_ids - return (ray.worker.global_worker.node.unique_id, tuple(self.gpu_ids)) + return ( + ray._private.worker.global_worker.node.unique_id, + tuple(self.gpu_ids), + ) # Create one actor per GPU. actors = [Actor1.remote() for _ in range(num_nodes * num_gpus_per_raylet)] @@ -136,7 +140,10 @@ def test_actor_multiple_gpus(ray_start_cluster): def get_location_and_ids(self): assert ray.get_gpu_ids() == self.gpu_ids - return (ray.worker.global_worker.node.unique_id, tuple(self.gpu_ids)) + return ( + ray._private.worker.global_worker.node.unique_id, + tuple(self.gpu_ids), + ) # Create some actors. actors1 = [Actor1.remote() for _ in range(num_nodes * 2)] @@ -167,7 +174,10 @@ def test_actor_multiple_gpus(ray_start_cluster): self.gpu_ids = ray.get_gpu_ids() def get_location_and_ids(self): - return (ray.worker.global_worker.node.unique_id, tuple(self.gpu_ids)) + return ( + ray._private.worker.global_worker.node.unique_id, + tuple(self.gpu_ids), + ) # Create some actors. actors2 = [Actor2.remote() for _ in range(num_nodes)] @@ -206,7 +216,10 @@ def test_actor_different_numbers_of_gpus(ray_start_cluster): self.gpu_ids = ray.get_gpu_ids() def get_location_and_ids(self): - return (ray.worker.global_worker.node.unique_id, tuple(self.gpu_ids)) + return ( + ray._private.worker.global_worker.node.unique_id, + tuple(self.gpu_ids), + ) # Create some actors. actors = [Actor1.remote() for _ in range(0 + 5 + 10)] @@ -250,7 +263,10 @@ def test_actor_multiple_gpus_from_multiple_tasks(ray_start_cluster): self.gpu_ids = ray.get_gpu_ids() def get_location_and_ids(self): - return ((ray.worker.global_worker.node.unique_id), tuple(self.gpu_ids)) + return ( + (ray._private.worker.global_worker.node.unique_id), + tuple(self.gpu_ids), + ) def sleep(self): time.sleep(100) @@ -293,7 +309,10 @@ def test_actor_multiple_gpus_from_multiple_tasks(ray_start_cluster): self.gpu_ids = ray.get_gpu_ids() def get_location_and_ids(self): - return (ray.worker.global_worker.node.unique_id, tuple(self.gpu_ids)) + return ( + ray._private.worker.global_worker.node.unique_id, + tuple(self.gpu_ids), + ) # All the GPUs should be used up now. a = Actor.remote() @@ -336,7 +355,11 @@ def test_actors_and_tasks_with_gpus(ray_start_cluster): gpu_ids = ray.get_gpu_ids() assert len(gpu_ids) == 1 assert gpu_ids[0] in range(num_gpus_per_raylet) - return (ray.worker.global_worker.node.unique_id, tuple(gpu_ids), [t1, t2]) + return ( + ray._private.worker.global_worker.node.unique_id, + tuple(gpu_ids), + [t1, t2], + ) @ray.remote(num_gpus=2) def f2(): @@ -347,7 +370,11 @@ def test_actors_and_tasks_with_gpus(ray_start_cluster): assert len(gpu_ids) == 2 assert gpu_ids[0] in range(num_gpus_per_raylet) assert gpu_ids[1] in range(num_gpus_per_raylet) - return (ray.worker.global_worker.node.unique_id, tuple(gpu_ids), [t1, t2]) + return ( + ray._private.worker.global_worker.node.unique_id, + tuple(gpu_ids), + [t1, t2], + ) @ray.remote(num_gpus=1) class Actor1: @@ -358,7 +385,10 @@ def test_actors_and_tasks_with_gpus(ray_start_cluster): def get_location_and_ids(self): assert ray.get_gpu_ids() == self.gpu_ids - return (ray.worker.global_worker.node.unique_id, tuple(self.gpu_ids)) + return ( + ray._private.worker.global_worker.node.unique_id, + tuple(self.gpu_ids), + ) def locations_to_intervals_for_many_tasks(): # Launch a bunch of GPU tasks. @@ -568,12 +598,12 @@ def test_custom_label_placement(ray_start_cluster): @ray.remote(resources={"CustomResource1": 1}) class ResourceActor1: def get_location(self): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id @ray.remote(resources={"CustomResource2": 1}) class ResourceActor2: def get_location(self): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id # Create some actors. actors1 = [ResourceActor1.remote() for _ in range(2)] diff --git a/python/ray/tests/test_advanced.py b/python/ray/tests/test_advanced.py index 3a992b149..72abb123d 100644 --- a/python/ray/tests/test_advanced.py +++ b/python/ray/tests/test_advanced.py @@ -1,20 +1,16 @@ # coding: utf-8 import json import logging +import os import sys import time -import os import numpy as np import pytest -import ray.cluster_utils - import ray._private.profiling as profiling -from ray._private.test_utils import ( - client_test_enabled, - RayTestTimeoutException, -) +import ray.cluster_utils +from ray._private.test_utils import RayTestTimeoutException, client_test_enabled from ray.exceptions import ReferenceCountingAssertionError if client_test_enabled(): @@ -43,14 +39,14 @@ def test_internal_free(shutdown_only): # Free deletes from in-memory store. obj_ref = sampler.sample.remote() ray.get(obj_ref) - ray.internal.free(obj_ref) + ray._private.internal_api.free(obj_ref) with pytest.raises(ReferenceCountingAssertionError): ray.get(obj_ref) # Free deletes big objects from plasma store. big_id = sampler.sample_big.remote() ray.get(big_id) - ray.internal.free(big_id) + ray._private.internal_api.free(big_id) time.sleep(1) # wait for delete RPC to propagate with pytest.raises(ReferenceCountingAssertionError): ray.get(big_id) @@ -94,22 +90,22 @@ def test_caching_functions_to_run(shutdown_only): def f(worker_info): sys.path.append(1) - ray.worker.global_worker.run_function_on_all_workers(f) + ray._private.worker.global_worker.run_function_on_all_workers(f) def f(worker_info): sys.path.append(2) - ray.worker.global_worker.run_function_on_all_workers(f) + ray._private.worker.global_worker.run_function_on_all_workers(f) def g(worker_info): sys.path.append(3) - ray.worker.global_worker.run_function_on_all_workers(g) + ray._private.worker.global_worker.run_function_on_all_workers(g) def f(worker_info): sys.path.append(4) - ray.worker.global_worker.run_function_on_all_workers(f) + ray._private.worker.global_worker.run_function_on_all_workers(f) ray.init(num_cpus=1) @@ -130,7 +126,7 @@ def test_caching_functions_to_run(shutdown_only): sys.path.pop() sys.path.pop() - ray.worker.global_worker.run_function_on_all_workers(f) + ray._private.worker.global_worker.run_function_on_all_workers(f) @pytest.mark.skipif(client_test_enabled(), reason="internal api") @@ -138,7 +134,7 @@ def test_running_function_on_all_workers(ray_start_regular): def f(worker_info): sys.path.append("fake_directory") - ray.worker.global_worker.run_function_on_all_workers(f) + ray._private.worker.global_worker.run_function_on_all_workers(f) @ray.remote def get_path1(): @@ -154,7 +150,7 @@ def test_running_function_on_all_workers(ray_start_regular): def f(worker_info): sys.path.pop(-1) - ray.worker.global_worker.run_function_on_all_workers(f) + ray._private.worker.global_worker.run_function_on_all_workers(f) # Create a second remote function to guarantee that when we call # get_path2.remote(), the second function to run will have been run on @@ -284,7 +280,7 @@ def test_object_transfer_dump(ray_start_cluster_enabled): # The profiling information only flushes once every second. time.sleep(1.1) - transfer_dump = ray.state.object_transfer_timeline() + transfer_dump = ray._private.state.object_transfer_timeline() # Make sure the transfer dump can be serialized with JSON. json.loads(json.dumps(transfer_dump)) assert len(transfer_dump) >= num_nodes ** 2 diff --git a/python/ray/tests/test_advanced_2.py b/python/ray/tests/test_advanced_2.py index 3bc441107..630822953 100644 --- a/python/ray/tests/test_advanced_2.py +++ b/python/ray/tests/test_advanced_2.py @@ -9,10 +9,7 @@ import pytest import ray import ray.cluster_utils -from ray._private.test_utils import ( - RayTestTimeoutException, - wait_for_condition, -) +from ray._private.test_utils import RayTestTimeoutException, wait_for_condition logger = logging.getLogger(__name__) @@ -131,7 +128,7 @@ def test_zero_cpus_actor(ray_start_cluster): @ray.remote class Foo: def method(self): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id # Make sure tasks and actors run on the remote raylet. a = Foo.remote() @@ -208,32 +205,32 @@ def test_multiple_raylets(ray_start_cluster): # This must be run on the zeroth raylet. @ray.remote(num_cpus=11) def run_on_0(): - return ray.worker.global_worker.node.plasma_store_socket_name + return ray._private.worker.global_worker.node.plasma_store_socket_name # This must be run on the first raylet. @ray.remote(num_gpus=2) def run_on_1(): - return ray.worker.global_worker.node.plasma_store_socket_name + return ray._private.worker.global_worker.node.plasma_store_socket_name # This must be run on the second raylet. @ray.remote(num_cpus=6, num_gpus=1) def run_on_2(): - return ray.worker.global_worker.node.plasma_store_socket_name + return ray._private.worker.global_worker.node.plasma_store_socket_name # This can be run anywhere. @ray.remote(num_cpus=0, num_gpus=0) def run_on_0_1_2(): - return ray.worker.global_worker.node.plasma_store_socket_name + return ray._private.worker.global_worker.node.plasma_store_socket_name # This must be run on the first or second raylet. @ray.remote(num_gpus=1) def run_on_1_2(): - return ray.worker.global_worker.node.plasma_store_socket_name + return ray._private.worker.global_worker.node.plasma_store_socket_name # This must be run on the zeroth or second raylet. @ray.remote(num_cpus=8) def run_on_0_2(): - return ray.worker.global_worker.node.plasma_store_socket_name + return ray._private.worker.global_worker.node.plasma_store_socket_name def run_lots_of_tasks(): names = [] @@ -324,16 +321,16 @@ def test_custom_resources(ray_start_cluster): @ray.remote def f(): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id @ray.remote(resources={"CustomResource": 1}) def g(): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id @ray.remote(resources={"CustomResource": 1}) def h(): ray.get([f.remote() for _ in range(5)]) - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id # The g tasks should be scheduled only on the second raylet. raylet_ids = set(ray.get([g.remote() for _ in range(50)])) @@ -351,18 +348,18 @@ def test_node_id_resource(ray_start_cluster): cluster.add_node(num_cpus=3) ray.init(address=cluster.address) - local_node = ray.state.current_node_id() + local_node = ray._private.state.current_node_id() # Note that these will have the same IP in the test cluster - assert len(ray.state.node_ids()) == 2 - assert local_node in ray.state.node_ids() + assert len(ray._private.state.node_ids()) == 2 + assert local_node in ray._private.state.node_ids() @ray.remote(resources={local_node: 1}) def f(): - return ray.state.current_node_id() + return ray._private.state.current_node_id() # Check the node id resource is automatically usable for scheduling. - assert ray.get(f.remote()) == ray.state.current_node_id() + assert ray.get(f.remote()) == ray._private.state.current_node_id() def test_two_custom_resources(ray_start_cluster): @@ -378,7 +375,7 @@ def test_two_custom_resources(ray_start_cluster): # Sleep a while to emulate a slow operation. This is needed to make # sure tasks are scheduled to different nodes. time.sleep(0.1) - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id # Make sure each node has at least one idle worker. wait_for_condition(lambda: len(set(ray.get([foo.remote() for _ in range(6)]))) == 2) @@ -389,27 +386,27 @@ def test_two_custom_resources(ray_start_cluster): @ray.remote(resources={"CustomResource1": 1}) def f(): time.sleep(0.001) - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id @ray.remote(resources={"CustomResource2": 1}) def g(): time.sleep(0.001) - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id @ray.remote(resources={"CustomResource1": 1, "CustomResource2": 3}) def h(): time.sleep(0.001) - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id @ray.remote(resources={"CustomResource1": 4}) def j(): time.sleep(0.001) - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id @ray.remote(resources={"CustomResource3": 1}) def k(): time.sleep(0.001) - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id # The f and g tasks should be scheduled on both raylets. assert len(set(ray.get([f.remote() for _ in range(500)]))) == 2 diff --git a/python/ray/tests/test_advanced_3.py b/python/ray/tests/test_advanced_3.py index e288abd36..6f8b4e127 100644 --- a/python/ray/tests/test_advanced_3.py +++ b/python/ray/tests/test_advanced_3.py @@ -1,24 +1,21 @@ # coding: utf-8 import logging import os -import sys +import pickle import socket +import sys import time import numpy as np -import pickle import pytest import ray -import ray.util.accelerators import ray._private.utils import ray.cluster_utils -import setproctitle +import ray.util.accelerators +from ray._private.test_utils import check_call_ray, wait_for_num_actors -from ray._private.test_utils import ( - check_call_ray, - wait_for_num_actors, -) +import setproctitle logger = logging.getLogger(__name__) @@ -32,11 +29,11 @@ def test_global_state_api(shutdown_only): assert ray.cluster_resources()["CustomResource"] == 1 job_id = ray._private.utils.compute_job_id_from_driver( - ray.WorkerID(ray.worker.global_worker.worker_id) + ray.WorkerID(ray._private.worker.global_worker.worker_id) ) client_table = ray.nodes() - node_ip_address = ray.worker.global_worker.node_ip_address + node_ip_address = ray._private.worker.global_worker.node_ip_address assert len(client_table) == 1 assert client_table[0]["NodeManagerAddress"] == node_ip_address @@ -50,7 +47,7 @@ def test_global_state_api(shutdown_only): # Wait for actor to be created wait_for_num_actors(1) - actor_table = ray.state.actors() + actor_table = ray._private.state.actors() assert len(actor_table) == 1 (actor_info,) = actor_table.values() @@ -60,7 +57,7 @@ def test_global_state_api(shutdown_only): assert "IPAddress" in actor_info["OwnerAddress"] assert actor_info["Address"]["Port"] != actor_info["OwnerAddress"]["Port"] - job_table = ray.state.jobs() + job_table = ray._private.state.jobs() assert len(job_table) == 1 assert job_table[0]["JobID"] == job_id.hex() @@ -158,7 +155,7 @@ def test_workers(shutdown_only): @ray.remote def f(): - return id(ray.worker.global_worker), os.getpid() + return id(ray._private.worker.global_worker), os.getpid() # Wait until all of the workers have started. worker_ids = set() @@ -194,7 +191,7 @@ def test_wait_reconstruction(shutdown_only): x_id = f.remote() ray.wait([x_id]) ray.wait([f.remote()]) - assert not ray.worker.global_worker.core_worker.object_exists(x_id) + assert not ray._private.worker.global_worker.core_worker.object_exists(x_id) ready_ids, _ = ray.wait([x_id]) assert len(ready_ids) == 1 @@ -325,7 +322,7 @@ def test_put_pins_object(ray_start_object_store_memory): del x_id for _ in range(10): ray.put(np.zeros(10 * 1024 * 1024)) - assert not ray.worker.global_worker.core_worker.object_exists( + assert not ray._private.worker.global_worker.core_worker.object_exists( ray.ObjectRef(x_binary) ) diff --git a/python/ray/tests/test_advanced_4.py b/python/ray/tests/test_advanced_4.py index 83b8d334d..f23744c69 100644 --- a/python/ray/tests/test_advanced_4.py +++ b/python/ray/tests/test_advanced_4.py @@ -1,8 +1,9 @@ -import pytest -import ray import subprocess import sys +import pytest + +import ray from ray._private.gcs_utils import check_health from ray._private.test_utils import Semaphore, client_test_enabled, wait_for_condition from ray.experimental.internal_kv import _internal_kv_list @@ -41,7 +42,7 @@ def test_ray_memory(shutdown_only): def test_jemalloc_env_var_propagate(): """Test `propagate_jemalloc_env_var`""" - gcs_ptype = ray.ray_constants.PROCESS_TYPE_GCS_SERVER + gcs_ptype = ray._private.ray_constants.PROCESS_TYPE_GCS_SERVER """ If the shared library path is not specified, it should return an empty dict. @@ -54,7 +55,7 @@ def test_jemalloc_env_var_propagate(): actual = ray._private.services.propagate_jemalloc_env_var( jemalloc_path=None, jemalloc_conf="a,b,c", - jemalloc_comps=[ray.ray_constants.PROCESS_TYPE_GCS_SERVER], + jemalloc_comps=[ray._private.ray_constants.PROCESS_TYPE_GCS_SERVER], process_type=gcs_ptype, ) assert actual == expected @@ -66,7 +67,7 @@ def test_jemalloc_env_var_propagate(): actual = ray._private.services.propagate_jemalloc_env_var( jemalloc_path=library_path, jemalloc_conf="", - jemalloc_comps=[ray.ray_constants.PROCESS_TYPE_GCS_SERVER], + jemalloc_comps=[ray._private.ray_constants.PROCESS_TYPE_GCS_SERVER], process_type=gcs_ptype, ) assert actual == expected @@ -76,7 +77,7 @@ def test_jemalloc_env_var_propagate(): ray._private.services.propagate_jemalloc_env_var( jemalloc_path=library_path, jemalloc_conf="", - jemalloc_comps="ray.ray_constants.PROCESS_TYPE_GCS_SERVER,", + jemalloc_comps="ray._private.ray_constants.PROCESS_TYPE_GCS_SERVER,", process_type=gcs_ptype, ) @@ -85,7 +86,7 @@ def test_jemalloc_env_var_propagate(): actual = ray._private.services.propagate_jemalloc_env_var( jemalloc_path=library_path, jemalloc_conf="", - jemalloc_comps=[ray.ray_constants.PROCESS_TYPE_RAYLET], + jemalloc_comps=[ray._private.ray_constants.PROCESS_TYPE_RAYLET], process_type=gcs_ptype, ) """ @@ -97,7 +98,7 @@ def test_jemalloc_env_var_propagate(): actual = ray._private.services.propagate_jemalloc_env_var( jemalloc_path=library_path, jemalloc_conf=malloc_conf, - jemalloc_comps=[ray.ray_constants.PROCESS_TYPE_GCS_SERVER], + jemalloc_comps=[ray._private.ray_constants.PROCESS_TYPE_GCS_SERVER], process_type=gcs_ptype, ) assert actual == expected @@ -185,7 +186,7 @@ def get_gcs_memory_used(): def function_entry_num(job_id): - from ray.ray_constants import KV_NAMESPACE_FUNCTION_TABLE + from ray._private.ray_constants import KV_NAMESPACE_FUNCTION_TABLE return ( len( @@ -240,7 +241,7 @@ def test_function_table_gc(call_ray_start): # It's not working on win32. if sys.platform != "win32": assert get_gcs_memory_used() > 500 * 1024 * 1024 - job_id = ray.worker.global_worker.current_job_id.hex().encode() + job_id = ray._private.worker.global_worker.current_job_id.hex().encode() assert function_entry_num(job_id) > 0 ray.shutdown() @@ -264,7 +265,7 @@ def test_function_table_gc_actor(call_ray_start): # If there is a detached actor, the function won't be deleted. a = Actor.options(lifetime="detached", name="a").remote() ray.get(a.ready.remote()) - job_id = ray.worker.global_worker.current_job_id.hex().encode() + job_id = ray._private.worker.global_worker.current_job_id.hex().encode() ray.shutdown() ray.init(address="auto", namespace="b") @@ -277,7 +278,7 @@ def test_function_table_gc_actor(call_ray_start): # If there is not a detached actor, it'll be deleted when the job finishes. a = Actor.remote() ray.get(a.ready.remote()) - job_id = ray.worker.global_worker.current_job_id.hex().encode() + job_id = ray._private.worker.global_worker.current_job_id.hex().encode() ray.shutdown() ray.init(address="auto", namespace="c") wait_for_condition(lambda: function_entry_num(job_id) == 0) diff --git a/python/ray/tests/test_advanced_5.py b/python/ray/tests/test_advanced_5.py index 0bda22868..11f6f69ce 100644 --- a/python/ray/tests/test_advanced_5.py +++ b/python/ray/tests/test_advanced_5.py @@ -7,11 +7,7 @@ import numpy as np import pytest import ray.cluster_utils - -from ray._private.test_utils import ( - client_test_enabled, - SignalActor, -) +from ray._private.test_utils import SignalActor, client_test_enabled if client_test_enabled(): from ray.util.client import ray @@ -167,7 +163,7 @@ def test_actor_distribution_balance(ray_start_cluster_enabled, args): @ray.remote(memory=100 * 1024 ** 2, num_cpus=0.01, scheduling_strategy="SPREAD") class Foo: def method(self): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id actor_distribution = {} actor_list = [Foo.remote() for _ in range(actor_count)] @@ -217,7 +213,7 @@ def test_worker_lease_reply_with_resources(ray_start_cluster_enabled): @ray.remote(memory=800 * 1024 ** 2, num_cpus=0.01) class Foo: def method(self): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id foo1 = Foo.remote() o1 = foo1.method.remote() diff --git a/python/ray/tests/test_advanced_6.py b/python/ray/tests/test_advanced_6.py index d89e89c3a..8baf0afa5 100644 --- a/python/ray/tests/test_advanced_6.py +++ b/python/ray/tests/test_advanced_6.py @@ -1,20 +1,20 @@ # coding: utf-8 import logging import os +import platform +import signal import sys import time import psutil -import platform import pytest -import signal import ray import ray.cluster_utils from ray._private.test_utils import ( run_string_as_driver_nonblocking, - wait_for_pid_to_exit, wait_for_condition, + wait_for_pid_to_exit, ) logger = logging.getLogger(__name__) @@ -66,7 +66,7 @@ def test_local_mode_gpus(save_gpu_ids_shutdown_only): from importlib import reload - reload(ray.worker) + reload(ray._private.worker) ray.init(num_gpus=3, local_mode=True) diff --git a/python/ray/tests/test_advanced_7.py b/python/ray/tests/test_advanced_7.py index 162482049..60c7c635a 100644 --- a/python/ray/tests/test_advanced_7.py +++ b/python/ray/tests/test_advanced_7.py @@ -1,19 +1,16 @@ # coding: utf-8 -from concurrent.futures import ThreadPoolExecutor import logging import random import sys import threading import time +from concurrent.futures import ThreadPoolExecutor import numpy as np import pytest import ray.cluster_utils - -from ray._private.test_utils import ( - client_test_enabled, -) +from ray._private.test_utils import client_test_enabled if client_test_enabled(): from ray.util.client import ray @@ -176,16 +173,16 @@ def test_wait_makes_object_local(ray_start_cluster_enabled): # Test get makes the object local. x_id = a.method.remote() - assert not ray.worker.global_worker.core_worker.object_exists(x_id) + assert not ray._private.worker.global_worker.core_worker.object_exists(x_id) ray.get(x_id) - assert ray.worker.global_worker.core_worker.object_exists(x_id) + assert ray._private.worker.global_worker.core_worker.object_exists(x_id) # Test wait makes the object local. x_id = a.method.remote() - assert not ray.worker.global_worker.core_worker.object_exists(x_id) + assert not ray._private.worker.global_worker.core_worker.object_exists(x_id) ok, _ = ray.wait([x_id]) assert len(ok) == 1 - assert ray.worker.global_worker.core_worker.object_exists(x_id) + assert ray._private.worker.global_worker.core_worker.object_exists(x_id) @pytest.mark.skipif(client_test_enabled(), reason="internal api") @@ -215,7 +212,7 @@ def test_future_resolution_skip_plasma(ray_start_cluster_enabled): f_result = ray.get(f_ref) # borrowed_ref should be inlined on future resolution and shouldn't be # in Plasma. - assert ray.worker.global_worker.core_worker.object_exists( + assert ray._private.worker.global_worker.core_worker.object_exists( borrowed_ref, memory_store_only=True ) return f_result * 2 @@ -253,7 +250,7 @@ def test_task_output_inline_bytes_limit(ray_start_cluster_enabled): result = 0 for i, ref in enumerate(numbers): result += ray.get(ref) - inlined = ray.worker.global_worker.core_worker.object_exists( + inlined = ray._private.worker.global_worker.core_worker.object_exists( ref, memory_store_only=True ) if i < 2: diff --git a/python/ray/tests/test_advanced_8.py b/python/ray/tests/test_advanced_8.py index 92497e067..7f1d8aead 100644 --- a/python/ray/tests/test_advanced_8.py +++ b/python/ray/tests/test_advanced_8.py @@ -8,21 +8,18 @@ import time from unittest import mock import numpy as np -import pytest import psutil +import pytest import ray -from ray.dashboard import k8s_utils -import ray.ray_constants as ray_constants -import ray.util.accelerators -import ray._private.utils import ray._private.gcs_utils as gcs_utils -import ray.cluster_utils +import ray._private.ray_constants as ray_constants import ray._private.resource_spec as resource_spec - -from ray._private.test_utils import ( - wait_for_condition, -) +import ray._private.utils +import ray.cluster_utils +import ray.util.accelerators +from ray._private.test_utils import wait_for_condition +from ray.dashboard import k8s_utils from ray.runtime_env import RuntimeEnv logger = logging.getLogger(__name__) @@ -147,7 +144,7 @@ def test_ray_address_environment_variable(ray_start_cluster): # RAY_ADDRESS is set to the cluster address. os.environ["RAY_ADDRESS"] = address ray.init() - assert "CPU" not in ray.state.cluster_resources() + assert "CPU" not in ray._private.state.cluster_resources() ray.shutdown() del os.environ["RAY_ADDRESS"] @@ -155,7 +152,7 @@ def test_ray_address_environment_variable(ray_start_cluster): # RAY_ADDRESS is set to "auto". os.environ["RAY_ADDRESS"] = "auto" ray.init() - assert "CPU" not in ray.state.cluster_resources() + assert "CPU" not in ray._private.state.cluster_resources() ray.shutdown() del os.environ["RAY_ADDRESS"] @@ -163,13 +160,13 @@ def test_ray_address_environment_variable(ray_start_cluster): # when `address` is not `auto`. os.environ["RAY_ADDRESS"] = "test" ray.init(address=address) - assert "CPU" not in ray.state.cluster_resources() + assert "CPU" not in ray._private.state.cluster_resources() ray.shutdown() del os.environ["RAY_ADDRESS"] # Make sure we start a new cluster if RAY_ADDRESS is not set. ray.init() - assert "CPU" in ray.state.cluster_resources() + assert "CPU" in ray._private.state.cluster_resources() ray.shutdown() @@ -554,7 +551,7 @@ def test_sync_job_config(shutdown_only): ) # Check that the job config is synchronized at the driver side. - job_config = ray.worker.global_worker.core_worker.get_job_config() + job_config = ray._private.worker.global_worker.core_worker.get_job_config() job_runtime_env = RuntimeEnv.deserialize( job_config.runtime_env_info.serialized_runtime_env ) @@ -562,7 +559,7 @@ def test_sync_job_config(shutdown_only): @ray.remote def get_job_config(): - job_config = ray.worker.global_worker.core_worker.get_job_config() + job_config = ray._private.worker.global_worker.core_worker.get_job_config() return job_config.SerializeToString() # Check that the job config is synchronized at the worker side. diff --git a/python/ray/tests/test_advanced_9.py b/python/ray/tests/test_advanced_9.py index 596d355c1..3345b3390 100644 --- a/python/ray/tests/test_advanced_9.py +++ b/python/ray/tests/test_advanced_9.py @@ -1,7 +1,8 @@ -import pytest -import ray import sys +import pytest + +import ray from ray._private.test_utils import Semaphore, client_test_enabled, wait_for_condition from ray.experimental.internal_kv import _internal_kv_list @@ -88,7 +89,7 @@ def get_gcs_memory_used(): def function_entry_num(job_id): - from ray.ray_constants import KV_NAMESPACE_FUNCTION_TABLE + from ray._private.ray_constants import KV_NAMESPACE_FUNCTION_TABLE return ( len( @@ -143,7 +144,7 @@ def test_function_table_gc(call_ray_start): # It's not working on win32. if sys.platform != "win32": assert get_gcs_memory_used() > 500 * 1024 * 1024 - job_id = ray.worker.global_worker.current_job_id.hex().encode() + job_id = ray._private.worker.global_worker.current_job_id.hex().encode() assert function_entry_num(job_id) > 0 ray.shutdown() @@ -167,7 +168,7 @@ def test_function_table_gc_actor(call_ray_start): # If there is a detached actor, the function won't be deleted. a = Actor.options(lifetime="detached", name="a").remote() ray.get(a.ready.remote()) - job_id = ray.worker.global_worker.current_job_id.hex().encode() + job_id = ray._private.worker.global_worker.current_job_id.hex().encode() ray.shutdown() ray.init(address="auto", namespace="b") @@ -180,7 +181,7 @@ def test_function_table_gc_actor(call_ray_start): # If there is not a detached actor, it'll be deleted when the job finishes. a = Actor.remote() ray.get(a.ready.remote()) - job_id = ray.worker.global_worker.current_job_id.hex().encode() + job_id = ray._private.worker.global_worker.current_job_id.hex().encode() ray.shutdown() ray.init(address="auto", namespace="c") wait_for_condition(lambda: function_entry_num(job_id) == 0) diff --git a/python/ray/tests/test_asyncio.py b/python/ray/tests/test_asyncio.py index e5925d33f..17ce2b08c 100644 --- a/python/ray/tests/test_asyncio.py +++ b/python/ray/tests/test_asyncio.py @@ -240,7 +240,7 @@ async def test_asyncio_exit_actor(ray_start_regular_shared): @ray.remote def check_actor_gone_now(): def cond(): - return ray.state.actors()[a._ray_actor_id.hex()]["State"] != 2 + return ray._private.state.actors()[a._ray_actor_id.hex()]["State"] != 2 wait_for_condition(cond) @@ -313,7 +313,7 @@ async def test_async_obj_unhandled_errors(ray_start_regular_shared): num_exceptions += 1 # Test we report unhandled exceptions. - ray.worker._unhandled_error_handler = interceptor + ray._private.worker._unhandled_error_handler = interceptor x1 = f.remote() # NOTE: Unhandled exception is from waiting for the value of x1's ObjectID # in x1's destructor, and receiving an exception from f() instead. diff --git a/python/ray/tests/test_autoscaler.py b/python/ray/tests/test_autoscaler.py index 1f93c1f8a..51d67bae5 100644 --- a/python/ray/tests/test_autoscaler.py +++ b/python/ray/tests/test_autoscaler.py @@ -1,55 +1,55 @@ -from enum import Enum +import copy import json -import jsonschema import os import re import shutil -from subprocess import CalledProcessError import tempfile import threading import time import unittest -from unittest.mock import Mock -import yaml -import copy from collections import defaultdict -from ray.autoscaler._private.commands import get_or_create_head_node -from jsonschema.exceptions import ValidationError -from typing import Dict, Callable, List, Optional - -import ray -from ray.core.generated import gcs_service_pb2 -from ray.autoscaler._private.util import prepare_config, validate_config -from ray.autoscaler._private import commands -from ray.autoscaler.sdk import get_docker_host_mount_location -from ray.autoscaler._private.load_metrics import LoadMetrics -from ray.autoscaler._private.autoscaler import StandardAutoscaler, NonTerminatedNodes -from ray.autoscaler._private.prom_metrics import AutoscalerPrometheusMetrics -from ray.autoscaler._private.providers import ( - _NODE_PROVIDERS, - _clear_provider_cache, - _DEFAULT_CONFIGS, -) -from ray.autoscaler._private.readonly.node_provider import ReadOnlyNodeProvider -from ray.autoscaler.tags import ( - TAG_RAY_NODE_KIND, - TAG_RAY_NODE_STATUS, - STATUS_UP_TO_DATE, - STATUS_UPDATE_FAILED, - TAG_RAY_USER_NODE_TYPE, - NODE_TYPE_LEGACY_HEAD, - NODE_TYPE_LEGACY_WORKER, - NODE_KIND_HEAD, - NODE_KIND_WORKER, - STATUS_UNINITIALIZED, - TAG_RAY_CLUSTER_NAME, -) -from ray.autoscaler._private.constants import FOREGROUND_NODE_LAUNCH_KEY -from ray.autoscaler.node_provider import NodeProvider -from ray._private.test_utils import RayTestTimeoutException +from enum import Enum +from subprocess import CalledProcessError +from typing import Callable, Dict, List, Optional +from unittest.mock import Mock import grpc +import jsonschema import pytest +import yaml +from jsonschema.exceptions import ValidationError + +import ray +from ray._private.test_utils import RayTestTimeoutException +from ray.autoscaler._private import commands +from ray.autoscaler._private.autoscaler import NonTerminatedNodes, StandardAutoscaler +from ray.autoscaler._private.commands import get_or_create_head_node +from ray.autoscaler._private.constants import FOREGROUND_NODE_LAUNCH_KEY +from ray.autoscaler._private.load_metrics import LoadMetrics +from ray.autoscaler._private.prom_metrics import AutoscalerPrometheusMetrics +from ray.autoscaler._private.providers import ( + _DEFAULT_CONFIGS, + _NODE_PROVIDERS, + _clear_provider_cache, +) +from ray.autoscaler._private.readonly.node_provider import ReadOnlyNodeProvider +from ray.autoscaler._private.util import prepare_config, validate_config +from ray.autoscaler.node_provider import NodeProvider +from ray.autoscaler.sdk import get_docker_host_mount_location +from ray.autoscaler.tags import ( + NODE_KIND_HEAD, + NODE_KIND_WORKER, + NODE_TYPE_LEGACY_HEAD, + NODE_TYPE_LEGACY_WORKER, + STATUS_UNINITIALIZED, + STATUS_UP_TO_DATE, + STATUS_UPDATE_FAILED, + TAG_RAY_CLUSTER_NAME, + TAG_RAY_NODE_KIND, + TAG_RAY_NODE_STATUS, + TAG_RAY_USER_NODE_TYPE, +) +from ray.core.generated import gcs_service_pb2 WORKER_FILTER = {TAG_RAY_NODE_KIND: NODE_KIND_WORKER} diff --git a/python/ray/tests/test_autoscaler_drain_node_api.py b/python/ray/tests/test_autoscaler_drain_node_api.py index 2d3c27c78..4a006f77e 100644 --- a/python/ray/tests/test_autoscaler_drain_node_api.py +++ b/python/ray/tests/test_autoscaler_drain_node_api.py @@ -1,18 +1,18 @@ -import platform import logging +import platform import time import pytest import ray -from ray.autoscaler._private.fake_multi_node.node_provider import FakeMultiNodeProvider -from ray.cluster_utils import AutoscalingCluster -import ray.ray_constants as ray_constants +import ray._private.ray_constants as ray_constants from ray._private.test_utils import ( get_error_message, init_error_pubsub, wait_for_condition, ) +from ray.autoscaler._private.fake_multi_node.node_provider import FakeMultiNodeProvider +from ray.cluster_utils import AutoscalingCluster logger = logging.getLogger(__name__) diff --git a/python/ray/tests/test_autoscaler_fake_scaledown.py b/python/ray/tests/test_autoscaler_fake_scaledown.py index 74c1cae9b..c16d035c6 100644 --- a/python/ray/tests/test_autoscaler_fake_scaledown.py +++ b/python/ray/tests/test_autoscaler_fake_scaledown.py @@ -1,8 +1,9 @@ -import pytest import platform -import numpy as np import re +import numpy as np +import pytest + import ray from ray._private.test_utils import wait_for_condition from ray.cluster_utils import AutoscalingCluster @@ -70,7 +71,7 @@ def test_scaledown_shared_objects(shutdown_only): def check_memory(local_objs, num_spilled_objects=None, num_plasma_objects=None): def ok(): - s = ray.internal.internal_api.memory_summary() + s = ray._private.internal_api.memory_summary() print(f"\n\nMemory Summary:\n{s}\n") actual_objs = re.findall(r"LOCAL_REFERENCE[\s|\|]+([0-9a-f]+)", s) diff --git a/python/ray/tests/test_autoscaler_yaml.py b/python/ray/tests/test_autoscaler_yaml.py index 0bae18c5b..dc1ce057f 100644 --- a/python/ray/tests/test_autoscaler_yaml.py +++ b/python/ray/tests/test_autoscaler_yaml.py @@ -1,34 +1,34 @@ -import jsonschema +import copy import logging -import mock import os import sys import tempfile import unittest import urllib -import yaml -import copy from unittest.mock import MagicMock, Mock, patch + +import jsonschema import pytest +import yaml from click.exceptions import ClickException +import mock +from ray._private.test_utils import load_test_config, recursive_fnmatch from ray.autoscaler._private._azure.config import ( _configure_key_pair as _azure_configure_key_pair, ) +from ray.autoscaler._private._kubernetes.node_provider import KubernetesNodeProvider from ray.autoscaler._private.gcp import config as gcp_config +from ray.autoscaler._private.providers import _NODE_PROVIDERS from ray.autoscaler._private.util import ( + _get_default_config, + fill_node_type_min_max_workers, + merge_setup_commands, prepare_config, validate_config, - _get_default_config, - merge_setup_commands, - fill_node_type_min_max_workers, ) -from ray.autoscaler._private.providers import _NODE_PROVIDERS -from ray.autoscaler._private._kubernetes.node_provider import KubernetesNodeProvider from ray.autoscaler.tags import NODE_TYPE_LEGACY_HEAD, NODE_TYPE_LEGACY_WORKER -from ray._private.test_utils import load_test_config, recursive_fnmatch - RAY_PATH = os.path.abspath(os.path.dirname(os.path.dirname(__file__))) CONFIG_PATHS = recursive_fnmatch(os.path.join(RAY_PATH, "autoscaler"), "*.yaml") @@ -722,7 +722,6 @@ class AutoscalingConfigTest(unittest.TestCase): if __name__ == "__main__": - import pytest import sys if os.environ.get("PARALLEL_CI"): diff --git a/python/ray/tests/test_basic.py b/python/ray/tests/test_basic.py index 2f564fad5..e26e25566 100644 --- a/python/ray/tests/test_basic.py +++ b/python/ray/tests/test_basic.py @@ -8,16 +8,15 @@ import time import numpy as np import pytest +import ray import ray.cluster_utils from ray._private.test_utils import ( + SignalActor, client_test_enabled, get_error_message, - SignalActor, run_string_as_driver, ) -import ray - logger = logging.getLogger(__name__) @@ -249,6 +248,7 @@ def test_invalid_arguments(): def test_options(): """General test of option keywords in Ray.""" import re + from ray._private import ray_option_utils def f(): @@ -908,8 +908,8 @@ def test_failed_task(ray_start_shared_local_modes, error_pubsub): throw_exception_fct1.remote() throw_exception_fct1.remote() - if ray.worker.global_worker.mode != ray.worker.LOCAL_MODE: - msgs = get_error_message(p, 2, ray.ray_constants.TASK_PUSH_ERROR) + if ray._private.worker.global_worker.mode != ray._private.worker.LOCAL_MODE: + msgs = get_error_message(p, 2, ray._private.ray_constants.TASK_PUSH_ERROR) assert len(msgs) == 2 for msg in msgs: assert "Test function 1 intentionally failed." in msg.error_message diff --git a/python/ray/tests/test_basic_2.py b/python/ray/tests/test_basic_2.py index 8011fbbc4..b8b15318a 100644 --- a/python/ray/tests/test_basic_2.py +++ b/python/ray/tests/test_basic_2.py @@ -1,23 +1,21 @@ # coding: utf-8 -import os import logging +import os +import subprocess import sys +import tempfile import threading import time -import tempfile -import subprocess +from unittest.mock import MagicMock, patch import numpy as np import pytest -from unittest.mock import MagicMock, patch - -from ray.cluster_utils import Cluster, cluster_not_supported +from ray._private.ray_constants import KV_NAMESPACE_FUNCTION_TABLE from ray._private.test_utils import client_test_enabled +from ray.cluster_utils import Cluster, cluster_not_supported +from ray.exceptions import GetTimeoutError, RayTaskError from ray.tests.client_test_utils import create_remote_signal_actor -from ray.exceptions import GetTimeoutError -from ray.exceptions import RayTaskError -from ray.ray_constants import KV_NAMESPACE_FUNCTION_TABLE if client_test_enabled(): from ray.util.client import ray @@ -452,7 +450,7 @@ def test_skip_plasma(ray_start_regular_shared): a = Actor.remote() obj_ref = a.f.remote(1) # it is not stored in plasma - assert not ray.worker.global_worker.core_worker.object_exists(obj_ref) + assert not ray._private.worker.global_worker.core_worker.object_exists(obj_ref) assert ray.get(obj_ref) == 2 @@ -469,10 +467,10 @@ def test_actor_large_objects(ray_start_regular_shared): a = Actor.remote() obj_ref = a.f.remote() - assert not ray.worker.global_worker.core_worker.object_exists(obj_ref) + assert not ray._private.worker.global_worker.core_worker.object_exists(obj_ref) done, _ = ray.wait([obj_ref]) assert len(done) == 1 - assert ray.worker.global_worker.core_worker.object_exists(obj_ref) + assert ray._private.worker.global_worker.core_worker.object_exists(obj_ref) assert isinstance(ray.get(obj_ref), np.ndarray) @@ -654,7 +652,7 @@ def test_duplicate_args(ray_start_regular_shared): @pytest.mark.skipif(client_test_enabled(), reason="internal api") def test_get_correct_node_ip(): - with patch("ray.worker") as worker_mock: + with patch("ray._private.worker") as worker_mock: node_mock = MagicMock() node_mock.node_ip_address = "10.0.0.111" worker_mock._global_node = node_mock @@ -732,11 +730,11 @@ def test_use_dynamic_function_and_class(): # the same as in `FunctionActorManager.export`. key_func = ( b"RemoteFunction:" - + ray.worker.global_worker.current_job_id.hex().encode() + + ray._private.worker.global_worker.current_job_id.hex().encode() + b":" + f._function_descriptor.function_id.binary() ) - assert ray.worker.global_worker.gcs_client.internal_kv_exists( + assert ray._private.worker.global_worker.gcs_client.internal_kv_exists( key_func, KV_NAMESPACE_FUNCTION_TABLE ) foo_actor = Foo.remote() @@ -747,11 +745,11 @@ def test_use_dynamic_function_and_class(): # the same as in `FunctionActorManager.export_actor_class`. key_cls = ( b"ActorClass:" - + ray.worker.global_worker.current_job_id.hex().encode() + + ray._private.worker.global_worker.current_job_id.hex().encode() + b":" + foo_actor._ray_actor_creation_function_descriptor.function_id.binary() ) - assert ray.worker.global_worker.gcs_client.internal_kv_exists( + assert ray._private.worker.global_worker.gcs_client.internal_kv_exists( key_cls, namespace=KV_NAMESPACE_FUNCTION_TABLE ) diff --git a/python/ray/tests/test_basic_3.py b/python/ray/tests/test_basic_3.py index c2871f21c..d41e2a444 100644 --- a/python/ray/tests/test_basic_3.py +++ b/python/ray/tests/test_basic_3.py @@ -6,11 +6,10 @@ import time import numpy as np import pytest +import ray import ray.cluster_utils from ray._private.test_utils import dicts_equal -import ray - logger = logging.getLogger(__name__) @@ -72,7 +71,7 @@ def test_many_fractional_resources(shutdown_only): def f(block, accepted_resources): true_resources = { resource: value[0][1] - for resource, value in ray.worker.get_resource_ids().items() + for resource, value in ray._private.worker.get_resource_ids().items() } if block: ray.get(g.remote()) diff --git a/python/ray/tests/test_basic_4.py b/python/ray/tests/test_basic_4.py index 53d87008d..083ba0eaf 100644 --- a/python/ray/tests/test_basic_4.py +++ b/python/ray/tests/test_basic_4.py @@ -1,18 +1,18 @@ # coding: utf-8 import logging +import subprocess import sys import time -import subprocess +from pathlib import Path import numpy as np import pytest +import ray import ray.cluster_utils from ray._private.test_utils import wait_for_condition from ray.autoscaler._private.constants import RAY_PROCESSES -from pathlib import Path -import ray import psutil logger = logging.getLogger(__name__) @@ -64,7 +64,7 @@ def test_worker_startup_count(ray_start_cluster): slow_function.options(num_cpus=num_cpus).remote() # Check "debug_state.txt" to ensure no extra workers were started. - session_dir = ray.worker.global_worker.node.address_info["session_dir"] + session_dir = ray._private.worker.global_worker.node.address_info["session_dir"] session_path = Path(session_dir) debug_state_path = session_path / "logs" / "debug_state.txt" @@ -138,8 +138,8 @@ def test_fork_support(shutdown_only): @ray.remote def pool_factorial(): - import multiprocessing import math + import multiprocessing ctx = multiprocessing.get_context("fork") with ctx.Pool(processes=4) as pool: diff --git a/python/ray/tests/test_basic_5.py b/python/ray/tests/test_basic_5.py index 0257fd587..46d24980f 100644 --- a/python/ray/tests/test_basic_5.py +++ b/python/ray/tests/test_basic_5.py @@ -147,14 +147,14 @@ def init_func(worker_info): old.append(worker_info['worker'].worker_id) data_file.write_bytes(pickle.dumps(old)) -ray.worker.global_worker.run_function_on_all_workers(init_func) +ray._private.worker.global_worker.run_function_on_all_workers(init_func) ray.init(address='auto') @ray.remote def ready(): with FileLock(lock_file): worker_ids = pickle.loads(data_file.read_bytes()) - assert ray.worker.global_worker.worker_id in worker_ids + assert ray._private.worker.global_worker.worker_id in worker_ids ray.get(ready.remote()) """ diff --git a/python/ray/tests/test_client.py b/python/ray/tests/test_client.py index ab525d1bb..14bb582f6 100644 --- a/python/ray/tests/test_client.py +++ b/python/ray/tests/test_client.py @@ -1,25 +1,31 @@ -import os -import pytest -import time -import sys -import logging -import queue -import threading import _thread +import logging +import os +import queue +import sys +import threading +import time from unittest.mock import patch + import numpy as np -from ray.util.client.common import OBJECT_TRANSFER_CHUNK_SIZE +import pytest import ray.util.client.server.server as ray_client_server -from ray.tests.client_test_utils import create_remote_signal_actor -from ray.tests.client_test_utils import run_wrapped_actor_creation -from ray.util.client.common import ClientObjectRef -from ray.util.client.ray_client_helpers import connect_to_client_or_not -from ray.util.client.ray_client_helpers import ray_start_client_server -from ray._private.client_mode_hook import client_mode_should_convert -from ray._private.client_mode_hook import disable_client_hook -from ray._private.client_mode_hook import enable_client_mode +from ray._private.client_mode_hook import ( + client_mode_should_convert, + disable_client_hook, + enable_client_mode, +) from ray._private.test_utils import run_string_as_driver +from ray.tests.client_test_utils import ( + create_remote_signal_actor, + run_wrapped_actor_creation, +) +from ray.util.client.common import OBJECT_TRANSFER_CHUNK_SIZE, ClientObjectRef +from ray.util.client.ray_client_helpers import ( + connect_to_client_or_not, + ray_start_client_server, +) @pytest.mark.parametrize("connect_to_client", [False, True]) @@ -657,9 +663,10 @@ def test_client_gpu_ids(call_ray_stop_only): def test_client_serialize_addon(call_ray_stop_only): - import ray import pydantic + import ray + ray.init(num_cpus=0) class User(pydantic.BaseModel): diff --git a/python/ray/tests/test_client_builder.py b/python/ray/tests/test_client_builder.py index aafd05891..e1094af19 100644 --- a/python/ray/tests/test_client_builder.py +++ b/python/ray/tests/test_client_builder.py @@ -1,17 +1,18 @@ import os -import pytest import subprocess import sys import warnings -from unittest.mock import patch, Mock +from unittest.mock import Mock, patch + +import pytest import ray -import ray.util.client.server.server as ray_client_server import ray.client_builder as client_builder +import ray.util.client.server.server as ray_client_server from ray._private.test_utils import ( + run_string_as_driver, run_string_as_driver_nonblocking, wait_for_condition, - run_string_as_driver, ) @@ -104,7 +105,7 @@ print("Current namespace:", ray.get_runtime_context().namespace) def test_connect_to_cluster(ray_start_regular_shared): server = ray_client_server.serve("localhost:50055") with ray.client("localhost:50055").connect() as client_context: - assert client_context.dashboard_url == ray.worker.get_dashboard_url() + assert client_context.dashboard_url == ray._private.worker.get_dashboard_url() python_version = ".".join([str(x) for x in list(sys.version_info)[:3]]) assert client_context.python_version == python_version assert client_context.ray_version == ray.__version__ diff --git a/python/ray/tests/test_client_proxy.py b/python/ray/tests/test_client_proxy.py index 3e3c0f8ee..7bf30c0b8 100644 --- a/python/ray/tests/test_client_proxy.py +++ b/python/ray/tests/test_client_proxy.py @@ -1,26 +1,26 @@ -from glob import glob import json import os -import pytest import random import sys import time +from glob import glob from unittest.mock import patch import grpc +import pytest import ray -from ray.ray_constants import REDIS_DEFAULT_PASSWORD import ray.core.generated.ray_client_pb2 as ray_client_pb2 +import ray.util.client.server.proxier as proxier +from ray._private.ray_constants import REDIS_DEFAULT_PASSWORD +from ray._private.test_utils import run_string_as_driver from ray.cloudpickle.compat import pickle from ray.job_config import JobConfig -import ray.util.client.server.proxier as proxier -from ray._private.test_utils import run_string_as_driver def start_ray_and_proxy_manager(n_ports=2): ray_instance = ray.init(_redis_password=REDIS_DEFAULT_PASSWORD) - agent_port = ray.worker.global_worker.node.metrics_agent_port + agent_port = ray._private.worker.global_worker.node.metrics_agent_port pm = proxier.ProxyManager( ray_instance["address"], session_dir=ray_instance["session_dir"], diff --git a/python/ray/tests/test_client_references.py b/python/ray/tests/test_client_references.py index 917c3a6be..62f0be3ee 100644 --- a/python/ray/tests/test_client_references.py +++ b/python/ray/tests/test_client_references.py @@ -1,17 +1,18 @@ from concurrent.futures import Future import pytest + +import ray as real_ray +from ray._private.test_utils import object_memory_usage, wait_for_condition +from ray._raylet import ActorID, ObjectRef +from ray.core.generated.gcs_pb2 import ActorTableData from ray.util.client import _ClientContext from ray.util.client.common import ClientActorRef, ClientObjectRef -from ray.util.client.ray_client_helpers import ray_start_client_server from ray.util.client.ray_client_helpers import ( + ray_start_client_server, ray_start_client_server_pair, ray_start_cluster_client_server_pair, ) -from ray._private.test_utils import wait_for_condition, object_memory_usage -import ray as real_ray -from ray.core.generated.gcs_pb2 import ActorTableData -from ray._raylet import ActorID, ObjectRef def test_client_object_ref_basics(ray_start_regular): @@ -215,7 +216,7 @@ def test_delete_actor_on_disconnect(ray_start_cluster): def test_cond(): alive_actors = [ v - for v in real_ray.state.actors().values() + for v in real_ray._private.state.actors().values() if v["State"] != ActorTableData.DEAD ] return len(alive_actors) == 0 @@ -325,6 +326,7 @@ def test_named_actor_refcount(ray_start_regular): if __name__ == "__main__": import os import sys + import pytest if os.environ.get("PARALLEL_CI"): diff --git a/python/ray/tests/test_client_terminate.py b/python/ray/tests/test_client_terminate.py index 1f84b0191..cc220187a 100644 --- a/python/ray/tests/test_client_terminate.py +++ b/python/ray/tests/test_client_terminate.py @@ -1,15 +1,18 @@ -import pytest import sys import time -from ray.util.client.ray_client_helpers import ray_start_client_server +import pytest + +from ray._private.test_utils import convert_actor_state, wait_for_condition +from ray.exceptions import ( + GetTimeoutError, + ObjectLostError, + RayTaskError, + TaskCancelledError, + WorkerCrashedError, +) from ray.tests.client_test_utils import create_remote_signal_actor -from ray._private.test_utils import wait_for_condition, convert_actor_state -from ray.exceptions import TaskCancelledError -from ray.exceptions import RayTaskError -from ray.exceptions import WorkerCrashedError -from ray.exceptions import ObjectLostError -from ray.exceptions import GetTimeoutError +from ray.util.client.ray_client_helpers import ray_start_client_server def valid_exceptions(use_force): @@ -26,7 +29,7 @@ def _all_actors_dead(ray): def _all_actors_dead_internal(): return all( actor["State"] == convert_actor_state(gcs_utils.ActorTableData.DEAD) - for actor in list(real_ray.state.actors().values()) + for actor in list(real_ray._private.state.actors().values()) ) return _all_actors_dead_internal diff --git a/python/ray/tests/test_component_failures.py b/python/ray/tests/test_component_failures.py index 814a8ef6b..91216c271 100644 --- a/python/ray/tests/test_component_failures.py +++ b/python/ray/tests/test_component_failures.py @@ -2,12 +2,12 @@ import os import signal import sys import time -import numpy as np +import numpy as np import pytest import ray -from ray._private.test_utils import run_string_as_driver_nonblocking, SignalActor +from ray._private.test_utils import SignalActor, run_string_as_driver_nonblocking SIGKILL = signal.SIGKILL if sys.platform != "win32" else signal.SIGTERM @@ -53,7 +53,7 @@ def test_dying_worker_get(ray_start_2_cpus): # Seal the object so the store attempts to notify the worker that the # get has been fulfilled. obj = np.ones(200 * 1024, dtype=np.uint8) - ray.worker.global_worker.put_object(obj, x_id) + ray._private.worker.global_worker.put_object(obj, x_id) time.sleep(0.1) # Make sure that nothing has died. @@ -96,7 +96,7 @@ ray.get(ray.ObjectRef(ray._private.utils.hex_to_binary("{}"))) # Seal the object so the store attempts to notify the worker that the # get has been fulfilled. obj = np.ones(200 * 1024, dtype=np.uint8) - ray.worker.global_worker.put_object(obj, x_id) + ray._private.worker.global_worker.put_object(obj, x_id) time.sleep(0.1) # Make sure that nothing has died. @@ -134,7 +134,7 @@ def test_dying_worker_wait(ray_start_2_cpus): # Create the object. obj = np.ones(200 * 1024, dtype=np.uint8) - ray.worker.global_worker.put_object(obj, x_id) + ray._private.worker.global_worker.put_object(obj, x_id) time.sleep(0.1) # Make sure that nothing has died. @@ -177,7 +177,7 @@ ray.wait([ray.ObjectRef(ray._private.utils.hex_to_binary("{}"))]) # Seal the object so the store attempts to notify the worker that the # wait can return. obj = np.ones(200 * 1024, dtype=np.uint8) - ray.worker.global_worker.put_object(obj, x_id) + ray._private.worker.global_worker.put_object(obj, x_id) time.sleep(0.1) # Make sure that nothing has died. diff --git a/python/ray/tests/test_component_failures_2.py b/python/ray/tests/test_component_failures_2.py index acb577406..21bcf2032 100644 --- a/python/ray/tests/test_component_failures_2.py +++ b/python/ray/tests/test_component_failures_2.py @@ -5,12 +5,9 @@ import time import pytest import ray -import ray.ray_constants as ray_constants +import ray._private.ray_constants as ray_constants +from ray._private.test_utils import get_other_nodes, wait_for_condition from ray.cluster_utils import Cluster, cluster_not_supported -from ray._private.test_utils import ( - get_other_nodes, - wait_for_condition, -) SIGKILL = signal.SIGKILL if sys.platform != "win32" else signal.SIGTERM diff --git a/python/ray/tests/test_component_failures_3.py b/python/ray/tests/test_component_failures_3.py index 2773c60cb..0dd9c4160 100644 --- a/python/ray/tests/test_component_failures_3.py +++ b/python/ray/tests/test_component_failures_3.py @@ -5,7 +5,7 @@ import numpy as np import pytest import ray -import ray.ray_constants as ray_constants +import ray._private.ray_constants as ray_constants from ray._private.test_utils import get_other_nodes @@ -73,15 +73,15 @@ def test_actor_creation_node_failure(ray_start_cluster): def test_driver_lives_sequential(ray_start_regular): - ray.worker._global_node.kill_raylet() - ray.worker._global_node.kill_log_monitor() - ray.worker._global_node.kill_monitor() + ray._private.worker._global_node.kill_raylet() + ray._private.worker._global_node.kill_log_monitor() + ray._private.worker._global_node.kill_monitor() # If the driver can reach the tearDown method, then it is still alive. def test_driver_lives_parallel(ray_start_regular): - all_processes = ray.worker._global_node.all_processes + all_processes = ray._private.worker._global_node.all_processes process_infos = ( all_processes[ray_constants.PROCESS_TYPE_RAYLET] diff --git a/python/ray/tests/test_dashboard.py b/python/ray/tests/test_dashboard.py index c8f044a4e..a74c23d1a 100644 --- a/python/ray/tests/test_dashboard.py +++ b/python/ray/tests/test_dashboard.py @@ -7,13 +7,10 @@ import time import psutil import pytest import requests -from ray._private.test_utils import ( - run_string_as_driver, - wait_for_condition, -) import ray -from ray import ray_constants +from ray._private import ray_constants +from ray._private.test_utils import run_string_as_driver, wait_for_condition def search_agents(cluster): @@ -37,7 +34,10 @@ def search_agents(cluster): def test_ray_start_default_port_conflict(call_ray_stop_only, shutdown_only): subprocess.check_call(["ray", "start", "--head"]) ray.init(address="auto") - assert str(ray_constants.DEFAULT_DASHBOARD_PORT) in ray.worker.get_dashboard_url() + assert ( + str(ray_constants.DEFAULT_DASHBOARD_PORT) + in ray._private.worker.get_dashboard_url() + ) error_raised = False try: @@ -61,7 +61,7 @@ def test_ray_start_default_port_conflict(call_ray_stop_only, shutdown_only): def test_port_auto_increment(shutdown_only): ray.init() - url = ray.worker.get_dashboard_url() + url = ray._private.worker.get_dashboard_url() def dashboard_available(): try: @@ -78,7 +78,7 @@ import ray from ray._private.test_utils import wait_for_condition import requests ray.init() -url = ray.worker.get_dashboard_url() +url = ray._private.worker.get_dashboard_url() assert url != "{url}" def dashboard_available(): try: @@ -123,7 +123,7 @@ def test_port_conflict(listen_port, call_ray_stop_only, shutdown_only): def test_dashboard(shutdown_only): addresses = ray.init(include_dashboard=True, num_cpus=1) dashboard_url = addresses["webui_url"] - assert ray.worker.get_dashboard_url() == dashboard_url + assert ray._private.worker.get_dashboard_url() == dashboard_url assert re.match(r"^(localhost|\d+\.\d+\.\d+\.\d+):\d+$", dashboard_url) diff --git a/python/ray/tests/test_dataclient_disconnect.py b/python/ray/tests/test_dataclient_disconnect.py index 48934be90..467d1347e 100644 --- a/python/ray/tests/test_dataclient_disconnect.py +++ b/python/ray/tests/test_dataclient_disconnect.py @@ -1,8 +1,10 @@ -from ray.util.client.ray_client_helpers import ray_start_client_server -from unittest.mock import Mock, patch -import pytest import os import time +from unittest.mock import Mock, patch + +import pytest + +from ray.util.client.ray_client_helpers import ray_start_client_server def test_dataclient_disconnect_on_request(): diff --git a/python/ray/tests/test_failure.py b/python/ray/tests/test_failure.py index 3a9ae0f34..9d385fde6 100644 --- a/python/ray/tests/test_failure.py +++ b/python/ray/tests/test_failure.py @@ -7,18 +7,18 @@ import numpy as np import pytest import ray -import ray._private.utils import ray._private.gcs_utils as gcs_utils -import ray.ray_constants as ray_constants -from ray.exceptions import RayTaskError, RayActorError, GetTimeoutError +import ray._private.ray_constants as ray_constants +import ray._private.utils from ray._private.gcs_pubsub import GcsPublisher from ray._private.test_utils import ( - wait_for_condition, SignalActor, - init_error_pubsub, - get_error_message, convert_actor_state, + get_error_message, + init_error_pubsub, + wait_for_condition, ) +from ray.exceptions import GetTimeoutError, RayActorError, RayTaskError def test_unhandled_errors(ray_start_regular): @@ -39,7 +39,7 @@ def test_unhandled_errors(ray_start_regular): num_exceptions += 1 # Test we report unhandled exceptions. - ray.worker._unhandled_error_handler = interceptor + ray._private.worker._unhandled_error_handler = interceptor x1 = f.remote() x2 = a.f.remote() del x1 @@ -129,10 +129,10 @@ def test_failed_function_to_run(ray_start_2_cpus, error_pubsub): p = error_pubsub def f(worker): - if ray.worker.global_worker.mode == ray.WORKER_MODE: + if ray._private.worker.global_worker.mode == ray.WORKER_MODE: raise Exception("Function to run failed.") - ray.worker.global_worker.run_function_on_all_workers(f) + ray._private.worker.global_worker.run_function_on_all_workers(f) # Check that the error message is in the task info. errors = get_error_message(p, 2, ray_constants.FUNCTION_TO_RUN_PUSH_ERROR) assert len(errors) == 2 @@ -233,7 +233,7 @@ def test_worker_raising_exception(ray_start_regular, error_pubsub): def f(): # This is the only reasonable variable we can set here that makes the # execute_task function fail after the task got executed. - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker.function_actor_manager.increase_task_counter = None # Running this task should cause the worker to raise an exception after @@ -472,7 +472,7 @@ def test_version_mismatch(ray_start_cluster): def test_export_large_objects(ray_start_regular, error_pubsub): p = error_pubsub - import ray.ray_constants as ray_constants + import ray._private.ray_constants as ray_constants large_object = np.zeros( 2 * ray_constants.FUNCTION_SIZE_WARN_THRESHOLD, dtype=np.uint8 @@ -624,7 +624,7 @@ def test_actor_failover_with_bad_network(ray_start_cluster_head): # Wait for the actor to be alive again in a new worker process. def check_actor_restart(): - actors = list(ray.state.actors().values()) + actors = list(ray._private.state.actors().values()) assert len(actors) == 1 print(actors) return ( diff --git a/python/ray/tests/test_failure_2.py b/python/ray/tests/test_failure_2.py index 4b4ee19e1..46473c58c 100644 --- a/python/ray/tests/test_failure_2.py +++ b/python/ray/tests/test_failure_2.py @@ -8,19 +8,19 @@ import numpy as np import pytest import ray -from ray.experimental.internal_kv import _internal_kv_get -from ray.ray_constants import DEBUG_AUTOSCALING_ERROR +import ray._private.ray_constants as ray_constants import ray._private.utils -import ray.ray_constants as ray_constants -from ray.cluster_utils import cluster_not_supported +from ray._private.ray_constants import DEBUG_AUTOSCALING_ERROR from ray._private.test_utils import ( - init_error_pubsub, + Semaphore, get_error_message, get_log_batch, - Semaphore, - wait_for_condition, + init_error_pubsub, run_string_as_driver_nonblocking, + wait_for_condition, ) +from ray.cluster_utils import cluster_not_supported +from ray.experimental.internal_kv import _internal_kv_get def test_warning_for_too_many_actors(shutdown_only): @@ -136,7 +136,7 @@ def test_warning_for_dead_node(ray_start_cluster_2_nodes, error_pubsub): ) def test_warning_for_dead_autoscaler(ray_start_regular, error_pubsub): # Terminate the autoscaler process. - from ray.worker import _global_node + from ray._private.worker import _global_node autoscaler_process = _global_node.all_processes[ray_constants.PROCESS_TYPE_MONITOR][ 0 @@ -158,10 +158,10 @@ def test_raylet_crash_when_get(ray_start_regular): def sleep_to_kill_raylet(): # Don't kill raylet before default workers get connected. time.sleep(2) - ray.worker._global_node.kill_raylet() + ray._private.worker._global_node.kill_raylet() object_ref = ray.put(np.zeros(200 * 1024, dtype=np.uint8)) - ray.internal.free(object_ref) + ray._private.internal_api.free(object_ref) thread = threading.Thread(target=sleep_to_kill_raylet) thread.start() @@ -192,7 +192,7 @@ def test_eviction(ray_start_cluster): obj = large_object.remote() assert isinstance(ray.get(obj), np.ndarray) # Evict the object. - ray.internal.free([obj]) + ray._private.internal_api.free([obj]) # ray.get throws an exception. with pytest.raises(ray.exceptions.ReferenceCountingAssertionError): ray.get(obj) diff --git a/python/ray/tests/test_failure_4.py b/python/ray/tests/test_failure_4.py index ac8d6b564..98b39005b 100644 --- a/python/ray/tests/test_failure_4.py +++ b/python/ray/tests/test_failure_4.py @@ -1,32 +1,32 @@ +import subprocess import sys import time -import ray - -import pytest import grpc -from grpc._channel import _InactiveRpcError import numpy as np import psutil -import subprocess +import pytest +from grpc._channel import _InactiveRpcError -import ray.ray_constants as ray_constants - -from ray.cluster_utils import Cluster, cluster_not_supported +import ray +import ray._private.ray_constants as ray_constants +import ray.experimental.internal_kv as internal_kv from ray import NodeID -from ray.core.generated import node_manager_pb2 -from ray.core.generated import node_manager_pb2_grpc -from ray.core.generated import gcs_service_pb2 -from ray.core.generated import gcs_service_pb2_grpc from ray._private.test_utils import ( - init_error_pubsub, + SignalActor, get_error_message, + init_error_pubsub, run_string_as_driver, wait_for_condition, ) -from ray._private.test_utils import SignalActor +from ray.cluster_utils import Cluster, cluster_not_supported +from ray.core.generated import ( + gcs_service_pb2, + gcs_service_pb2_grpc, + node_manager_pb2, + node_manager_pb2_grpc, +) from ray.exceptions import LocalRayletDiedError -import ray.experimental.internal_kv as internal_kv def search_raylet(cluster): @@ -562,7 +562,7 @@ def test_locality_aware_scheduling_for_dead_nodes(shutdown_only): # This function requires obj1 and obj2. @ray.remote def func(obj1, obj2): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id # This function should be scheduled to node2. As node2 has both objects. assert ray.get(func.remote(obj1, obj2)) == node2.unique_id diff --git a/python/ray/tests/test_gcs_fault_tolerance.py b/python/ray/tests/test_gcs_fault_tolerance.py index 588415d34..89aa3b2b3 100644 --- a/python/ray/tests/test_gcs_fault_tolerance.py +++ b/python/ray/tests/test_gcs_fault_tolerance.py @@ -40,8 +40,8 @@ def test_gcs_server_restart(ray_start_regular_with_external_redis): result = ray.get(actor1.method.remote(1)) assert result == 3 - ray.worker._global_node.kill_gcs_server() - ray.worker._global_node.start_gcs_server() + ray._private.worker._global_node.kill_gcs_server() + ray._private.worker._global_node.start_gcs_server() actor2 = Increase.remote() result = ray.get(actor2.method.remote(2)) @@ -81,8 +81,8 @@ def test_gcs_server_restart_during_actor_creation( actor = Increase.remote() ids.append(actor.method.remote(1)) - ray.worker._global_node.kill_gcs_server() - ray.worker._global_node.start_gcs_server() + ray._private.worker._global_node.kill_gcs_server() + ray._private.worker._global_node.start_gcs_server() # The timeout seems too long. # TODO(mwtian): after fixing reconnection in GCS pubsub, try using a lower @@ -180,7 +180,7 @@ def test_actor_raylet_resubscription(ray_start_regular_with_external_redis): pid = ray.get(actor.ready.remote()) print("actor is ready and kill gcs") - ray.worker._global_node.kill_gcs_server() + ray._private.worker._global_node.kill_gcs_server() print("make actor exit") import psutil @@ -191,7 +191,7 @@ def test_actor_raylet_resubscription(ray_start_regular_with_external_redis): sleep(1) print("start gcs") - ray.worker._global_node.start_gcs_server() + ray._private.worker._global_node.start_gcs_server() print("try actor method again") with pytest.raises(ray.exceptions.RayActorError): @@ -212,14 +212,14 @@ def test_del_actor_after_gcs_server_restart(ray_start_regular_with_external_redi result = ray.get(actor.method.remote(1)) assert result == 3 - ray.worker._global_node.kill_gcs_server() - ray.worker._global_node.start_gcs_server() + ray._private.worker._global_node.kill_gcs_server() + ray._private.worker._global_node.start_gcs_server() actor_id = actor._actor_id.hex() del actor def condition(): - actor_status = ray.state.actors(actor_id=actor_id) + actor_status = ray._private.state.actors(actor_id=actor_id) if actor_status["State"] == convert_actor_state(gcs_utils.ActorTableData.DEAD): return True else: @@ -282,8 +282,8 @@ def test_worker_raylet_resubscription(tmp_path, ray_start_regular_with_external_ wait_for_condition(condition, timeout=5) # kill the gcs - ray.worker._global_node.kill_gcs_server() - ray.worker._global_node.start_gcs_server() + ray._private.worker._global_node.kill_gcs_server() + ray._private.worker._global_node.start_gcs_server() # make sure resubscription is done # TODO(iycheng): The current way of resubscription potentially will lose # worker failure message because we don't ask for the snapshot of worker @@ -328,11 +328,11 @@ def test_core_worker_resubscription(tmp_path, ray_start_regular_with_external_re a = Actor.remote() r = a.ready.remote() # Actor is not ready before GCS is down. - ray.worker._global_node.kill_gcs_server() + ray._private.worker._global_node.kill_gcs_server() lock.release() # Actor is ready after GCS starts - ray.worker._global_node.start_gcs_server() + ray._private.worker._global_node.start_gcs_server() # Test the resubscribe works: if not, it'll timeout because worker # will think the actor is not ready. ray.get(r, timeout=5) @@ -361,10 +361,10 @@ def test_detached_actor_restarts(ray_start_regular_with_external_redis): a = A.options(name="a", lifetime="detached", max_restarts=-1).remote() pid = ray.get(a.ready.remote()) - ray.worker._global_node.kill_gcs_server() + ray._private.worker._global_node.kill_gcs_server() p = psutil.Process(pid) p.kill() - ray.worker._global_node.start_gcs_server() + ray._private.worker._global_node.start_gcs_server() while True: try: @@ -376,7 +376,7 @@ def test_detached_actor_restarts(ray_start_regular_with_external_redis): @pytest.mark.parametrize("auto_reconnect", [True, False]) def test_gcs_client_reconnect(ray_start_regular_with_external_redis, auto_reconnect): - gcs_address = ray.worker.global_worker.gcs_client.address + gcs_address = ray._private.worker.global_worker.gcs_client.address gcs_client = ( gcs_utils.GcsClient(address=gcs_address) if auto_reconnect @@ -386,8 +386,8 @@ def test_gcs_client_reconnect(ray_start_regular_with_external_redis, auto_reconn gcs_client.internal_kv_put(b"a", b"b", True, None) gcs_client.internal_kv_get(b"a", None) == b"b" - ray.worker._global_node.kill_gcs_server() - ray.worker._global_node.start_gcs_server() + ray._private.worker._global_node.kill_gcs_server() + ray._private.worker._global_node.start_gcs_server() if auto_reconnect is False: # This may flake: when GCS server restarted quickly, there would be no # connection error when calling internal_kv_get(). @@ -426,7 +426,7 @@ def test_actor_workloads(ray_start_regular_with_external_redis): assert r == 10 print("GCS is killed") - ray.worker._global_node.kill_gcs_server() + ray._private.worker._global_node.kill_gcs_server() print("Start to create a new actor") cc = Counter.remote() @@ -434,7 +434,7 @@ def test_actor_workloads(ray_start_regular_with_external_redis): ray.get(cc.r.remote(10), timeout=5) assert ray.get(c.r.remote(10)) == 10 - ray.worker._global_node.start_gcs_server() + ray._private.worker._global_node.start_gcs_server() import threading @@ -449,13 +449,13 @@ def test_actor_workloads(ray_start_regular_with_external_redis): assert ray.get(c.r.remote(10)) == 10 - ray.worker._global_node.kill_gcs_server() + ray._private.worker._global_node.kill_gcs_server() sleep(2) assert ray.get(c.r.remote(10)) == 10 - ray.worker._global_node.start_gcs_server() + ray._private.worker._global_node.start_gcs_server() from ray._private.test_utils import run_string_as_driver diff --git a/python/ray/tests/test_gcs_utils.py b/python/ray/tests/test_gcs_utils.py index dbe70cf36..19fa8db6d 100644 --- a/python/ray/tests/test_gcs_utils.py +++ b/python/ray/tests/test_gcs_utils.py @@ -1,17 +1,19 @@ -import os -import sys import contextlib +import os import signal -import pytest +import sys + import grpc -import ray._private.gcs_utils as gcs_utils +import pytest + import ray +import ray._private.gcs_utils as gcs_utils @contextlib.contextmanager def stop_gcs_server(): - process = ray.worker._global_node.all_processes[ - ray.ray_constants.PROCESS_TYPE_GCS_SERVER + process = ray._private.worker._global_node.all_processes[ + ray._private.ray_constants.PROCESS_TYPE_GCS_SERVER ][0].process pid = process.pid os.kill(pid, signal.SIGSTOP) @@ -20,7 +22,7 @@ def stop_gcs_server(): def test_kv_basic(ray_start_regular): - gcs_address = ray.worker.global_worker.gcs_client.address + gcs_address = ray._private.worker.global_worker.gcs_client.address gcs_client = gcs_utils.GcsClient(address=gcs_address, nums_reconnect_retry=0) assert gcs_client.internal_kv_get(b"A", b"NS") is None @@ -43,7 +45,7 @@ def test_kv_basic(ray_start_regular): @pytest.mark.skipif(sys.platform == "win32", reason="Windows doesn't have signals.") def test_kv_timeout(ray_start_regular): - gcs_address = ray.worker.global_worker.gcs_client.address + gcs_address = ray._private.worker.global_worker.gcs_client.address gcs_client = gcs_utils.GcsClient(address=gcs_address, nums_reconnect_retry=0) assert gcs_client.internal_kv_put(b"A", b"", False, b"") == 1 @@ -65,7 +67,7 @@ def test_kv_timeout(ray_start_regular): @pytest.mark.asyncio async def test_kv_basic_aio(ray_start_regular): gcs_client = gcs_utils.GcsAioClient( - address=ray.worker.global_worker.gcs_client.address + address=ray._private.worker.global_worker.gcs_client.address ) assert await gcs_client.internal_kv_get(b"A", b"NS") is None @@ -92,7 +94,7 @@ async def test_kv_basic_aio(ray_start_regular): @pytest.mark.asyncio async def test_kv_timeout_aio(ray_start_regular): gcs_client = gcs_utils.GcsAioClient( - address=ray.worker.global_worker.gcs_client.address + address=ray._private.worker.global_worker.gcs_client.address ) # Make sure gcs_client is connected assert await gcs_client.internal_kv_put(b"A", b"", False, b"") == 1 @@ -112,7 +114,6 @@ async def test_kv_timeout_aio(ray_start_regular): if __name__ == "__main__": - import pytest import sys if os.environ.get("PARALLEL_CI"): diff --git a/python/ray/tests/test_global_gc.py b/python/ray/tests/test_global_gc.py index 445e35a19..709499645 100644 --- a/python/ray/tests/test_global_gc.py +++ b/python/ray/tests/test_global_gc.py @@ -4,13 +4,12 @@ import logging import weakref import numpy as np - import pytest import ray import ray.cluster_utils +from ray._private.internal_api import global_gc from ray._private.test_utils import wait_for_condition -from ray.internal.internal_api import global_gc logger = logging.getLogger(__name__) diff --git a/python/ray/tests/test_global_state.py b/python/ray/tests/test_global_state.py index e881ec663..9eb9d0a4e 100644 --- a/python/ray/tests/test_global_state.py +++ b/python/ray/tests/test_global_state.py @@ -1,20 +1,22 @@ import os +import time + import pytest +import ray +import ray._private.gcs_utils as gcs_utils +import ray._private.ray_constants +from ray._private.test_utils import ( + convert_actor_state, + make_global_state_accessor, + wait_for_condition, +) + try: import pytest_timeout except ImportError: pytest_timeout = None -import time -import ray -import ray.ray_constants -import ray._private.gcs_utils as gcs_utils -from ray._private.test_utils import ( - wait_for_condition, - convert_actor_state, - make_global_state_accessor, -) # TODO(rliaw): The proper way to do this is to have the pytest config setup. @@ -97,15 +99,15 @@ def test_global_state_actor_table(ray_start_regular): return os.getpid() # actor table should be empty at first - assert len(ray.state.actors()) == 0 + assert len(ray._private.state.actors()) == 0 # actor table should contain only one entry def get_actor_table_data(field): - return list(ray.state.actors().values())[0][field] + return list(ray._private.state.actors().values())[0][field] a = Actor.remote() pid = ray.get(a.ready.remote()) - assert len(ray.state.actors()) == 1 + assert len(ray._private.state.actors()) == 1 assert get_actor_table_data("Pid") == pid # actor table should contain only this entry @@ -124,7 +126,7 @@ def test_global_state_actor_table(ray_start_regular): def test_global_state_worker_table(ray_start_regular): # Get worker table from gcs. - workers_data = ray.state.workers() + workers_data = ray._private.state.workers() assert len(workers_data) == 1 @@ -136,23 +138,23 @@ def test_global_state_actor_entry(ray_start_regular): pass # actor table should be empty at first - assert len(ray.state.actors()) == 0 + assert len(ray._private.state.actors()) == 0 a = Actor.remote() b = Actor.remote() ray.get(a.ready.remote()) ray.get(b.ready.remote()) - assert len(ray.state.actors()) == 2 + assert len(ray._private.state.actors()) == 2 a_actor_id = a._actor_id.hex() b_actor_id = b._actor_id.hex() - assert ray.state.actors(actor_id=a_actor_id)["ActorID"] == a_actor_id - assert ray.state.actors(actor_id=a_actor_id)["State"] == convert_actor_state( - gcs_utils.ActorTableData.ALIVE - ) - assert ray.state.actors(actor_id=b_actor_id)["ActorID"] == b_actor_id - assert ray.state.actors(actor_id=b_actor_id)["State"] == convert_actor_state( - gcs_utils.ActorTableData.ALIVE - ) + assert ray._private.state.actors(actor_id=a_actor_id)["ActorID"] == a_actor_id + assert ray._private.state.actors(actor_id=a_actor_id)[ + "State" + ] == convert_actor_state(gcs_utils.ActorTableData.ALIVE) + assert ray._private.state.actors(actor_id=b_actor_id)["ActorID"] == b_actor_id + assert ray._private.state.actors(actor_id=b_actor_id)[ + "State" + ] == convert_actor_state(gcs_utils.ActorTableData.ALIVE) def test_node_name_cluster(ray_start_cluster): @@ -395,13 +397,12 @@ def test_heartbeat_ip(shutdown_only): def test_next_job_id(ray_start_regular): - job_id_1 = ray.state.next_job_id() - job_id_2 = ray.state.next_job_id() + job_id_1 = ray._private.state.next_job_id() + job_id_2 = ray._private.state.next_job_id() assert job_id_1.int() + 1 == job_id_2.int() if __name__ == "__main__": - import pytest import sys if os.environ.get("PARALLEL_CI"): diff --git a/python/ray/tests/test_healthcheck.py b/python/ray/tests/test_healthcheck.py index eb2278136..359d0f237 100644 --- a/python/ray/tests/test_healthcheck.py +++ b/python/ray/tests/test_healthcheck.py @@ -1,14 +1,15 @@ # coding: utf-8 import logging -import pytest import signal import subprocess import sys import time + import psutil +import pytest import ray -from ray import ray_constants +from ray._private import ray_constants logger = logging.getLogger(__name__) @@ -22,7 +23,7 @@ def test_healthcheck(): assert res.returncode == 0, res.stdout # Kill GCS to test ray health-check. - all_processes = ray.worker._global_node.all_processes + all_processes = ray._private.worker._global_node.all_processes assert ray_constants.PROCESS_TYPE_GCS_SERVER in all_processes gcs_proc_info = all_processes[ray_constants.PROCESS_TYPE_GCS_SERVER][0] gcs_proc = psutil.Process(gcs_proc_info.process.pid) @@ -57,7 +58,7 @@ def test_healthcheck_ray_client_server(): assert res.returncode == 0, res.stdout client_server_handle.send_signal(signal.SIGKILL) - time.sleep(ray.ray_constants.HEALTHCHECK_EXPIRATION_S) + time.sleep(ray._private.ray_constants.HEALTHCHECK_EXPIRATION_S) res = subprocess.run(["ray", "health-check", "--component", "ray_client_server"]) assert res.returncode != 0, res.stdout diff --git a/python/ray/tests/test_job.py b/python/ray/tests/test_job.py index a09dcdfcb..2b4b84244 100644 --- a/python/ray/tests/test_job.py +++ b/python/ray/tests/test_job.py @@ -119,10 +119,10 @@ _ = Actor.remote() # Wait for actor to be created wait_for_num_actors(1) - actor_table = ray.state.actors() + actor_table = ray._private.state.actors() assert len(actor_table) == 1 - job_table = ray.state.jobs() + job_table = ray._private.state.jobs() assert len(job_table) == 2 # dash # Kill the driver process. @@ -130,7 +130,7 @@ _ = Actor.remote() p.wait() def actor_finish(): - actor_table = ray.state.actors() + actor_table = ray._private.state.actors() if len(actor_table) == 0: return True else: @@ -167,10 +167,10 @@ ray.get(_.value.remote()) # Wait for actor to be created wait_for_num_actors(1, gcs_utils.ActorTableData.ALIVE) - actor_table = ray.state.actors() + actor_table = ray._private.state.actors() assert len(actor_table) == 1 - job_table = ray.state.jobs() + job_table = ray._private.state.jobs() assert len(job_table) == 2 # dash # Kill the driver process. @@ -216,7 +216,7 @@ ray.shutdown() while not os.path.exists(tmpfiles[1]): time.sleep(1) - jobs = list(ray.state.jobs()) + jobs = list(ray._private.state.jobs()) jobs.sort(key=lambda x: x["JobID"]) driver = jobs[0] @@ -244,7 +244,7 @@ ray.shutdown() # Give the second job time to clean itself up. time.sleep(1) - jobs = list(ray.state.jobs()) + jobs = list(ray._private.state.jobs()) jobs.sort(key=lambda x: x["JobID"]) # jobs[0] is the test case driver. @@ -268,7 +268,7 @@ def test_config_metadata(shutdown_only): ray.init(job_config=job_config) - from_worker = ray.worker.global_worker.core_worker.get_job_config() + from_worker = ray._private.worker.global_worker.core_worker.get_job_config() assert dict(from_worker.metadata) == job_config.metadata diff --git a/python/ray/tests/test_k8s_operator_unit_tests.py b/python/ray/tests/test_k8s_operator_unit_tests.py index 4a75e3e5f..b21f83dd1 100644 --- a/python/ray/tests/test_k8s_operator_unit_tests.py +++ b/python/ray/tests/test_k8s_operator_unit_tests.py @@ -12,7 +12,7 @@ import yaml from ray.autoscaler.tags import TAG_RAY_NODE_KIND, NODE_KIND_HEAD from ray.autoscaler.node_provider import NodeProvider -from ray.ray_constants import DEFAULT_PORT +from ray._private.ray_constants import DEFAULT_PORT from ray.ray_operator.operator_utils import cr_to_config from ray.ray_operator.operator_utils import check_redis_password_not_specified from ray.ray_operator.operator_utils import get_head_service diff --git a/python/ray/tests/test_kill_raylet_signal_log.py b/python/ray/tests/test_kill_raylet_signal_log.py index 67975a266..56abb06ef 100644 --- a/python/ray/tests/test_kill_raylet_signal_log.py +++ b/python/ray/tests/test_kill_raylet_signal_log.py @@ -1,11 +1,11 @@ import signal import sys -import pytest -import ray # Import psutil after ray so the packaged version is used. import psutil +import pytest +import ray from ray._private.test_utils import wait_for_condition @@ -20,7 +20,7 @@ def get_pid(name): def check_result(filename, num_signal, check_key): ray.init(num_cpus=1) - session_dir = ray.worker._global_node.get_session_dir_path() + session_dir = ray._private.worker._global_node.get_session_dir_path() raylet_out_path = filename.format(session_dir) pid = get_pid("raylet") assert pid > 0 diff --git a/python/ray/tests/test_logging.py b/python/ray/tests/test_logging.py index e3bcfc90b..a10090a1f 100644 --- a/python/ray/tests/test_logging.py +++ b/python/ray/tests/test_logging.py @@ -1,30 +1,30 @@ import os import re -import sys -import time - -from unittest.mock import MagicMock -from collections import defaultdict, Counter -from pathlib import Path import subprocess +import sys import tempfile +import time +from collections import Counter, defaultdict +from pathlib import Path +from unittest.mock import MagicMock + import pytest import ray -from ray.cross_language import java_actor_class -from ray import ray_constants -from ray._private.test_utils import ( - get_log_batch, - wait_for_condition, - init_log_pubsub, - get_log_message, - run_string_as_driver, -) +from ray._private import ray_constants from ray._private.log_monitor import ( - LogMonitor, LOG_NAME_UPDATE_INTERVAL_S, RAY_LOG_MONITOR_MANY_FILES_THRESHOLD, + LogMonitor, ) +from ray._private.test_utils import ( + get_log_batch, + get_log_message, + init_log_pubsub, + run_string_as_driver, + wait_for_condition, +) +from ray.cross_language import java_actor_class def set_logging_config(monkeypatch, max_bytes, backup_count): @@ -58,7 +58,7 @@ def test_log_rotation(shutdown_only, monkeypatch): backup_count = 3 set_logging_config(monkeypatch, max_bytes, backup_count) ray.init(num_cpus=1) - session_dir = ray.worker.global_worker.node.address_info["session_dir"] + session_dir = ray._private.worker.global_worker.node.address_info["session_dir"] session_path = Path(session_dir) log_dir_path = session_path / "logs" @@ -134,7 +134,7 @@ def test_periodic_event_stats(shutdown_only): num_cpus=1, _system_config={"event_stats_print_interval_ms": 100, "event_stats": True}, ) - session_dir = ray.worker.global_worker.node.address_info["session_dir"] + session_dir = ray._private.worker.global_worker.node.address_info["session_dir"] session_path = Path(session_dir) log_dir_path = session_path / "logs" @@ -171,7 +171,7 @@ def test_worker_id_names(shutdown_only): num_cpus=1, _system_config={"event_stats_print_interval_ms": 100, "event_stats": True}, ) - session_dir = ray.worker.global_worker.node.address_info["session_dir"] + session_dir = ray._private.worker.global_worker.node.address_info["session_dir"] session_path = Path(session_dir) log_dir_path = session_path / "logs" @@ -285,7 +285,7 @@ import ray os.environ["RAY_LOG_TO_STDERR"] = "1" ray.init() -session_dir = ray.worker.global_worker.node.address_info["session_dir"] +session_dir = ray._private.worker.global_worker.node.address_info["session_dir"] session_path = Path(session_dir) log_dir_path = session_path / "logs" diff --git a/python/ray/tests/test_memstat.py b/python/ray/tests/test_memstat.py index 39d86c62e..11043d3b2 100644 --- a/python/ray/tests/test_memstat.py +++ b/python/ray/tests/test_memstat.py @@ -1,16 +1,13 @@ -import numpy as np import os import time +import numpy as np import pytest -import ray -from ray.cluster_utils import Cluster, cluster_not_supported -from ray.internal.internal_api import memory_summary -from ray._private.test_utils import ( - wait_for_condition, - Semaphore, -) +import ray +from ray._private.internal_api import memory_summary +from ray._private.test_utils import Semaphore, wait_for_condition +from ray.cluster_utils import Cluster, cluster_not_supported # RayConfig to enable recording call sites during ObjectRej creations. ray_config = {"record_ref_creation_sites": True} @@ -105,7 +102,7 @@ def test_worker_task_refs(ray_start_regular): @ray.remote def f(y): - from ray.internal.internal_api import memory_summary + from ray._private.internal_api import memory_summary x_id = ray.put("HI") info = memory_summary(address) @@ -152,7 +149,7 @@ def test_actor_task_refs(ray_start_regular): self.refs = [] def f(self, x): - from ray.internal.internal_api import memory_summary + from ray._private.internal_api import memory_summary self.refs.append(x) return memory_summary(address) diff --git a/python/ray/tests/test_metrics.py b/python/ray/tests/test_metrics.py index 12371546f..1d94b4cb9 100644 --- a/python/ray/tests/test_metrics.py +++ b/python/ray/tests/test_metrics.py @@ -1,21 +1,19 @@ import os -import grpc -import requests -import time import platform +import time + +import grpc +import psutil # We must import psutil after ray because we bundle it with ray. import pytest +import requests import ray -from ray.core.generated import common_pb2 -from ray.core.generated import node_manager_pb2 -from ray.core.generated import node_manager_pb2_grpc from ray._private.test_utils import ( RayTestTimeoutException, wait_until_succeeded_without_exception, ) from ray._private.utils import init_grpc_channel - -import psutil # We must import psutil after ray because we bundle it with ray. +from ray.core.generated import common_pb2, node_manager_pb2, node_manager_pb2_grpc _WIN32 = os.name == "nt" @@ -57,7 +55,7 @@ def test_worker_stats(shutdown_only): @ray.remote def f(): - ray.worker.show_in_dashboard("test") + ray._private.worker.show_in_dashboard("test") return os.getpid() @ray.remote @@ -66,7 +64,7 @@ def test_worker_stats(shutdown_only): pass def f(self): - ray.worker.show_in_dashboard("test") + ray._private.worker.show_in_dashboard("test") return os.getpid() # Test show_in_dashboard for remote functions. @@ -168,7 +166,6 @@ def test_multi_node_metrics_export_port_discovery(ray_start_cluster): if __name__ == "__main__": - import pytest import sys if os.environ.get("PARALLEL_CI"): diff --git a/python/ray/tests/test_metrics_agent.py b/python/ray/tests/test_metrics_agent.py index b6b20cbed..f10648b66 100644 --- a/python/ray/tests/test_metrics_agent.py +++ b/python/ray/tests/test_metrics_agent.py @@ -1,22 +1,22 @@ import json +import os import pathlib from pprint import pformat -import os from unittest.mock import MagicMock import pytest import ray -from ray.autoscaler._private.constants import AUTOSCALER_METRIC_PORT -from ray.ray_constants import PROMETHEUS_SERVICE_DISCOVERY_FILE from ray._private.metrics_agent import PrometheusServiceDiscoveryWriter -from ray.util.metrics import Counter, Histogram, Gauge +from ray._private.ray_constants import PROMETHEUS_SERVICE_DISCOVERY_FILE from ray._private.test_utils import ( - wait_for_condition, SignalActor, fetch_prometheus, get_log_batch, + wait_for_condition, ) +from ray.autoscaler._private.constants import AUTOSCALER_METRIC_PORT +from ray.util.metrics import Counter, Gauge, Histogram os.environ["RAY_event_stats"] = "1" @@ -299,7 +299,7 @@ def test_prometheus_file_based_service_discovery(ray_start_cluster): def test_prome_file_discovery_run_by_dashboard(shutdown_only): ray.init(num_cpus=0) - global_node = ray.worker._global_node + global_node = ray._private.worker._global_node temp_dir = global_node.get_temp_dir_path() def is_service_discovery_exist(): diff --git a/python/ray/tests/test_multi_node.py b/python/ray/tests/test_multi_node.py index eb62413a9..57c8acbc4 100644 --- a/python/ray/tests/test_multi_node.py +++ b/python/ray/tests/test_multi_node.py @@ -1,18 +1,19 @@ import os -import pytest -import psutil import sys import time +import psutil +import pytest + import ray -from ray import ray_constants +from ray._private import ray_constants from ray._private.test_utils import ( RayTestTimeoutException, + get_error_message, + init_error_pubsub, + object_memory_usage, run_string_as_driver, run_string_as_driver_nonblocking, - init_error_pubsub, - get_error_message, - object_memory_usage, wait_for_condition, ) diff --git a/python/ray/tests/test_multi_node_2.py b/python/ray/tests/test_multi_node_2.py index f9569e82d..36c7391be 100644 --- a/python/ray/tests/test_multi_node_2.py +++ b/python/ray/tests/test_multi_node_2.py @@ -1,18 +1,19 @@ import logging -import pytest import time +import pytest + import ray -import ray.ray_constants as ray_constants -from ray.util.placement_group import placement_group, remove_placement_group -from ray.autoscaler.sdk import request_resources -from ray.autoscaler._private.monitor import Monitor -from ray.cluster_utils import Cluster +import ray._private.ray_constants as ray_constants from ray._private.test_utils import ( + SignalActor, generate_system_config_map, wait_for_condition, - SignalActor, ) +from ray.autoscaler._private.monitor import Monitor +from ray.autoscaler.sdk import request_resources +from ray.cluster_utils import Cluster +from ray.util.placement_group import placement_group, remove_placement_group logger = logging.getLogger(__name__) @@ -195,7 +196,7 @@ def test_heartbeats_single(ray_start_cluster_head): """ cluster = ray_start_cluster_head monitor = setup_monitor(cluster.gcs_address) - total_cpus = ray.state.cluster_resources()["CPU"] + total_cpus = ray._private.state.cluster_resources()["CPU"] verify_load_metrics(monitor, ({"CPU": 0.0}, {"CPU": total_cpus})) @ray.remote diff --git a/python/ray/tests/test_multi_node_3.py b/python/ray/tests/test_multi_node_3.py index e821ca38c..e34c21e0d 100644 --- a/python/ray/tests/test_multi_node_3.py +++ b/python/ray/tests/test_multi_node_3.py @@ -1,26 +1,26 @@ +import asyncio import os -import pytest import subprocess import sys -import asyncio - from pathlib import Path -import ray import psutil -import ray.ray_constants as ray_constants +import pytest + +import ray +import ray._private.ray_constants as ray_constants from ray._private.services import REDIS_EXECUTABLE, _start_redis_instance -from ray._private.utils import detect_fate_sharing_support from ray._private.test_utils import ( + Semaphore, check_call_ray, + check_call_subprocess, + kill_process_by_name, run_string_as_driver, run_string_as_driver_nonblocking, wait_for_children_of_pid, wait_for_children_of_pid_to_exit, - kill_process_by_name, - Semaphore, - check_call_subprocess, ) +from ray._private.utils import detect_fate_sharing_support def test_calling_start_ray_head(call_ray_stop_only): @@ -396,7 +396,7 @@ print("success") driver_script = driver_script_template.format(address, nonexistent_id.hex()) out = run_string_as_driver(driver_script) # Simulate the nonexistent dependency becoming available. - ray.worker.global_worker.put_object(None, nonexistent_id) + ray._private.worker.global_worker.put_object(None, nonexistent_id) # Make sure the first driver ran to completion. assert "success" in out @@ -420,7 +420,7 @@ print("success") driver_script = driver_script_template.format(address, nonexistent_id.hex()) out = run_string_as_driver(driver_script) # Simulate the nonexistent dependency becoming available. - ray.worker.global_worker.put_object(None, nonexistent_id) + ray._private.worker.global_worker.put_object(None, nonexistent_id) # Make sure the first driver ran to completion. assert "success" in out diff --git a/python/ray/tests/test_multinode_failures.py b/python/ray/tests/test_multinode_failures.py index e4fc6a0b2..f50949d4a 100644 --- a/python/ray/tests/test_multinode_failures.py +++ b/python/ray/tests/test_multinode_failures.py @@ -6,9 +6,9 @@ import time import pytest import ray -import ray.ray_constants as ray_constants +import ray._private.ray_constants as ray_constants +from ray._private.test_utils import Semaphore, get_other_nodes from ray.cluster_utils import Cluster, cluster_not_supported -from ray._private.test_utils import get_other_nodes, Semaphore SIGKILL = signal.SIGKILL if sys.platform != "win32" else signal.SIGTERM diff --git a/python/ray/tests/test_multinode_failures_2.py b/python/ray/tests/test_multinode_failures_2.py index 710c9aa44..003287b7e 100644 --- a/python/ray/tests/test_multinode_failures_2.py +++ b/python/ray/tests/test_multinode_failures_2.py @@ -5,8 +5,8 @@ import numpy as np import pytest import ray +import ray._private.ray_constants as ray_constants from ray._private.test_utils import get_other_nodes -import ray.ray_constants as ray_constants @pytest.mark.skip(reason="No reconstruction for objects placed in plasma yet") @@ -58,7 +58,7 @@ def test_object_reconstruction(ray_start_cluster): # node. for x in xs: ray.get(x) - ray.internal.free([x], local_only=True) + ray._private.internal_api.free([x], local_only=True) # Kill a component on one of the nodes. process.terminate() diff --git a/python/ray/tests/test_namespace.py b/python/ray/tests/test_namespace.py index 03a1bf9cc..847251a81 100644 --- a/python/ray/tests/test_namespace.py +++ b/python/ray/tests/test_namespace.py @@ -1,9 +1,10 @@ -import pytest import sys import time +import pytest + import ray -from ray import ray_constants +from ray._private import ray_constants from ray._private.test_utils import ( get_error_message, init_error_pubsub, diff --git a/python/ray/tests/test_object_manager.py b/python/ray/tests/test_object_manager.py index b1f71f0aa..e3ae8ac68 100644 --- a/python/ray/tests/test_object_manager.py +++ b/python/ray/tests/test_object_manager.py @@ -1,9 +1,10 @@ -from collections import defaultdict import multiprocessing -import numpy as np -import pytest import time import warnings +from collections import defaultdict + +import numpy as np +import pytest import ray from ray.cluster_utils import Cluster, cluster_not_supported @@ -106,7 +107,7 @@ def test_object_broadcast(ray_start_cluster_with_resource): # Wait for profiling information to be pushed to the profile table. time.sleep(1) - transfer_events = ray.state.object_transfer_timeline() + transfer_events = ray._private.state.object_transfer_timeline() # Make sure that each object was transferred a reasonable number of times. for x_id in object_refs: @@ -189,7 +190,7 @@ def test_actor_broadcast(ray_start_cluster_with_resource): # Wait for profiling information to be pushed to the profile table. time.sleep(1) # TODO(Sang): Re-enable it after event is introduced. - # transfer_events = ray.state.object_transfer_timeline() + # transfer_events = ray._private.state.object_transfer_timeline() # # Make sure that each object was transferred a reasonable number of times. # noqa # for x_id in object_refs: @@ -727,7 +728,6 @@ def test_maximize_concurrent_pull_race_condition(ray_start_cluster_head): if __name__ == "__main__": - import pytest import sys import os diff --git a/python/ray/tests/test_object_spilling.py b/python/ray/tests/test_object_spilling.py index 050e12032..f6e1068e2 100644 --- a/python/ray/tests/test_object_spilling.py +++ b/python/ray/tests/test_object_spilling.py @@ -7,16 +7,17 @@ from datetime import datetime, timedelta import numpy as np import pytest + import ray +from ray._private.external_storage import create_url_with_offset, parse_url_with_offset +from ray._private.internal_api import memory_summary +from ray._private.test_utils import wait_for_condition +from ray._raylet import GcsClientOptions from ray.tests.conftest import ( - file_system_object_spilling_config, buffer_object_spilling_config, + file_system_object_spilling_config, mock_distributed_fs_object_spilling_config, ) -from ray.external_storage import create_url_with_offset, parse_url_with_offset -from ray._private.test_utils import wait_for_condition -from ray.internal.internal_api import memory_summary -from ray._raylet import GcsClientOptions def run_basic_workload(): @@ -27,7 +28,9 @@ def run_basic_workload(): ray.get(ray.put(arr)) -def is_dir_empty(temp_folder, append_path=ray.ray_constants.DEFAULT_OBJECT_PREFIX): +def is_dir_empty( + temp_folder, append_path=ray._private.ray_constants.DEFAULT_OBJECT_PREFIX +): # append_path is used because the file based spilling will append # new directory path. num_files = 0 @@ -40,7 +43,7 @@ def is_dir_empty(temp_folder, append_path=ray.ray_constants.DEFAULT_OBJECT_PREFI def assert_no_thrashing(address): - state = ray.state.GlobalState() + state = ray._private.state.GlobalState() options = GcsClientOptions.from_gcs_address(address) state._initialize_global_state(options) summary = memory_summary(address=address, stats_only=True) @@ -103,9 +106,14 @@ def test_url_generation_and_parse(): def test_default_config(shutdown_only): ray.init(num_cpus=0, object_store_memory=75 * 1024 * 1024) # Make sure the object spilling configuration is properly set. - config = json.loads(ray.worker._global_node._config["object_spilling_config"]) + config = json.loads( + ray._private.worker._global_node._config["object_spilling_config"] + ) assert config["type"] == "filesystem" - assert config["params"]["directory_path"] == ray.worker._global_node._session_dir + assert ( + config["params"]["directory_path"] + == ray._private.worker._global_node._session_dir + ) # Make sure the basic workload can succeed. run_basic_workload() ray.shutdown() @@ -119,7 +127,7 @@ def test_default_config(shutdown_only): "object_store_full_delay_ms": 100, }, ) - assert "object_spilling_config" not in ray.worker._global_node._config + assert "object_spilling_config" not in ray._private.worker._global_node._config run_basic_workload() ray.shutdown() @@ -132,7 +140,9 @@ def test_default_config(shutdown_only): ) }, ) - config = json.loads(ray.worker._global_node._config["object_spilling_config"]) + config = json.loads( + ray._private.worker._global_node._config["object_spilling_config"] + ) assert config["type"] == "mock_distributed_fs" @@ -143,7 +153,9 @@ def test_default_config_buffering(shutdown_only): "object_spilling_config": (json.dumps(buffer_object_spilling_config)) }, ) - config = json.loads(ray.worker._global_node._config["object_spilling_config"]) + config = json.loads( + ray._private.worker._global_node._config["object_spilling_config"] + ) assert config["type"] == buffer_object_spilling_config["type"] assert ( config["params"]["buffer_size"] @@ -499,7 +511,7 @@ def test_spill_worker_failure(ray_start_regular): for proc in psutil.process_iter(): try: - name = ray.ray_constants.WORKER_PROCESS_TYPE_SPILL_WORKER_IDLE + name = ray._private.ray_constants.WORKER_PROCESS_TYPE_SPILL_WORKER_IDLE if name in proc.name(): return proc # for macOS diff --git a/python/ray/tests/test_object_spilling_2.py b/python/ray/tests/test_object_spilling_2.py index 76f6186ae..1e0c0e9aa 100644 --- a/python/ray/tests/test_object_spilling_2.py +++ b/python/ray/tests/test_object_spilling_2.py @@ -1,14 +1,15 @@ import os -import random import platform +import random import subprocess import sys import numpy as np import pytest + import ray -from ray._private.test_utils import wait_for_condition, run_string_as_driver -from ray.tests.test_object_spilling import is_dir_empty, assert_no_thrashing +from ray._private.test_utils import run_string_as_driver, wait_for_condition +from ray.tests.test_object_spilling import assert_no_thrashing, is_dir_empty def test_delete_objects(object_spilling_config, shutdown_only): @@ -255,7 +256,7 @@ def test_fusion_objects(fs_only_object_spilling_config, shutdown_only): is_test_passing = False # Since we'd like to see the temp directory that stores the files, # we need to append this directory. - temp_folder = temp_folder / ray.ray_constants.DEFAULT_OBJECT_PREFIX + temp_folder = temp_folder / ray._private.ray_constants.DEFAULT_OBJECT_PREFIX for path in temp_folder.iterdir(): file_size = path.stat().st_size # Make sure there are at least one diff --git a/python/ray/tests/test_object_spilling_3.py b/python/ray/tests/test_object_spilling_3.py index ad6d183d5..c2935619a 100644 --- a/python/ray/tests/test_object_spilling_3.py +++ b/python/ray/tests/test_object_spilling_3.py @@ -1,19 +1,20 @@ import json -import re import platform -import sys -import zlib -import shutil -import time -from collections import defaultdict import random +import re +import shutil +import sys +import time +import zlib +from collections import defaultdict import numpy as np import pytest + import ray from ray._private.test_utils import wait_for_condition -from ray.tests.test_object_spilling import is_dir_empty, assert_no_thrashing from ray.cluster_utils import Cluster, cluster_not_supported +from ray.tests.test_object_spilling import assert_no_thrashing, is_dir_empty @pytest.mark.skipif(platform.system() in ["Windows"], reason="Failing on Windows.") @@ -54,7 +55,7 @@ def test_multiple_directories(tmp_path, shutdown_only): num_files = defaultdict(int) for temp_dir in temp_dirs: - temp_folder = temp_dir / ray.ray_constants.DEFAULT_OBJECT_PREFIX + temp_folder = temp_dir / ray._private.ray_constants.DEFAULT_OBJECT_PREFIX for path in temp_folder.iterdir(): num_files[str(temp_folder)] += 1 @@ -85,7 +86,7 @@ def test_multiple_directories(tmp_path, shutdown_only): def _check_spilled(num_objects_spilled=0): def ok(): - s = ray.internal.internal_api.memory_summary(stats_only=True) + s = ray._private.internal_api.memory_summary(stats_only=True) if num_objects_spilled == 0: return "Spilled " not in s diff --git a/python/ray/tests/test_object_spilling_no_asan.py b/python/ray/tests/test_object_spilling_no_asan.py index 58e6fdd93..6021fe401 100644 --- a/python/ray/tests/test_object_spilling_no_asan.py +++ b/python/ray/tests/test_object_spilling_no_asan.py @@ -1,9 +1,10 @@ -import numpy as np -import platform -import pytest import os +import platform import sys +import numpy as np +import pytest + import ray @@ -34,7 +35,9 @@ def test_spill_fusion(fs_only_object_spilling_config, shutdown_only): # about 10 objects. xs = [ray.put(np.zeros(object_size // 8)) for _ in range(300)] # noqa: F841 - spill_dir = os.path.join(temp_folder, ray.ray_constants.DEFAULT_OBJECT_PREFIX) + spill_dir = os.path.join( + temp_folder, ray._private.ray_constants.DEFAULT_OBJECT_PREFIX + ) under_min, over_min = 0, 0 for filename in os.listdir(spill_dir): size = os.stat(os.path.join(spill_dir, filename)).st_size diff --git a/python/ray/tests/test_output.py b/python/ray/tests/test_output.py index 197f38770..0ee7a39c1 100644 --- a/python/ray/tests/test_output.py +++ b/python/ray/tests/test_output.py @@ -1,16 +1,16 @@ -import subprocess -import sys -import pytest +import os import re import signal +import subprocess +import sys import time -import os + +import pytest import ray - from ray._private.test_utils import ( - run_string_as_driver_nonblocking, run_string_as_driver, + run_string_as_driver_nonblocking, ) @@ -260,7 +260,7 @@ ray.init(local_mode=True) # In local mode this generates an ERROR level log. ray._private.utils.push_error_to_driver( - ray.worker.global_worker, "type", "Hello there") + ray._private.worker.global_worker, "type", "Hello there") """ proc = run_string_as_driver_nonblocking(script) @@ -572,7 +572,7 @@ os.environ["RAY_raylet_heartbeat_period_milliseconds"]=str(HEARTBEAT_PERIOD) ray.init(_node_name=\"{NODE_NAME}\") # This will kill raylet without letting it exit gracefully. -ray.worker._global_node.kill_raylet() +ray._private.worker._global_node.kill_raylet() time.sleep(NUM_HEARTBEATS * HEARTBEAT_PERIOD / 1000 + WAIT_BUFFER_SECONDS) ray.shutdown() """ diff --git a/python/ray/tests/test_placement_group.py b/python/ray/tests/test_placement_group.py index 28b23d798..547c7de69 100644 --- a/python/ray/tests/test_placement_group.py +++ b/python/ray/tests/test_placement_group.py @@ -1,11 +1,10 @@ -import pytest import sys +import pytest + import ray import ray.cluster_utils -from ray._private.test_utils import ( - placement_group_assert_no_leak, -) +from ray._private.test_utils import placement_group_assert_no_leak from ray.util.client.ray_client_helpers import connect_to_client_or_not @@ -167,7 +166,7 @@ def test_placement_group_pack( ray.get(actor_2.value.remote()) # Get all actors. - actor_infos = ray.state.actors() + actor_infos = ray._private.state.actors() # Make sure all actors in counter_list are collocated in one node. actor_info_1 = actor_infos.get(actor_1._actor_id.hex()) @@ -223,7 +222,7 @@ def test_placement_group_strict_pack(ray_start_cluster, connect_to_client): ray.get(actor_2.value.remote()) # Get all actors. - actor_infos = ray.state.actors() + actor_infos = ray._private.state.actors() # Make sure all actors in counter_list are collocated in one node. actor_info_1 = actor_infos.get(actor_1._actor_id.hex()) @@ -283,7 +282,7 @@ def test_placement_group_spread( [ray.get(actor.value.remote()) for actor in actors] # Get all actors. - actor_infos = ray.state.actors() + actor_infos = ray._private.state.actors() # Make sure all actors in counter_list are located in separate nodes. actor_info_objs = [actor_infos.get(actor._actor_id.hex()) for actor in actors] @@ -339,7 +338,7 @@ def test_placement_group_strict_spread( [ray.get(actor.value.remote()) for actor in actors] # Get all actors. - actor_infos = ray.state.actors() + actor_infos = ray._private.state.actors() # Make sure all actors in counter_list are located in separate nodes. actor_info_objs = [actor_infos.get(actor._actor_id.hex()) for actor in actors] @@ -371,7 +370,7 @@ def test_placement_group_actor_resource_ids(ray_start_cluster, connect_to_client @ray.remote(num_cpus=1) class F: def f(self): - return ray.worker.get_resource_ids() + return ray._private.worker.get_resource_ids() cluster = ray_start_cluster num_nodes = 1 @@ -392,7 +391,7 @@ def test_placement_group_actor_resource_ids(ray_start_cluster, connect_to_client def test_placement_group_task_resource_ids(ray_start_cluster, connect_to_client): @ray.remote(num_cpus=1) def f(): - return ray.worker.get_resource_ids() + return ray._private.worker.get_resource_ids() cluster = ray_start_cluster num_nodes = 1 @@ -424,7 +423,7 @@ def test_placement_group_task_resource_ids(ray_start_cluster, connect_to_client) def test_placement_group_hang(ray_start_cluster, connect_to_client): @ray.remote(num_cpus=1) def f(): - return ray.worker.get_resource_ids() + return ray._private.worker.get_resource_ids() cluster = ray_start_cluster num_nodes = 1 diff --git a/python/ray/tests/test_placement_group_2.py b/python/ray/tests/test_placement_group_2.py index 477fbfac4..9596698cc 100644 --- a/python/ray/tests/test_placement_group_2.py +++ b/python/ray/tests/test_placement_group_2.py @@ -1,29 +1,29 @@ -import pytest import sys import time +import pytest + +import ray +import ray._private.gcs_utils as gcs_utils +import ray.cluster_utils +from ray._private.test_utils import ( + convert_actor_state, + generate_system_config_map, + get_error_message, + get_other_nodes, + kill_actor_and_wait_for_failure, + placement_group_assert_no_leak, + run_string_as_driver, + wait_for_condition, +) +from ray.util.client.ray_client_helpers import connect_to_client_or_not +from ray.util.placement_group import get_current_placement_group + try: import pytest_timeout except ImportError: pytest_timeout = None -import ray -import ray.cluster_utils -import ray._private.gcs_utils as gcs_utils - -from ray._private.test_utils import ( - get_other_nodes, - generate_system_config_map, - kill_actor_and_wait_for_failure, - run_string_as_driver, - wait_for_condition, - get_error_message, - placement_group_assert_no_leak, - convert_actor_state, -) -from ray.util.placement_group import get_current_placement_group -from ray.util.client.ray_client_helpers import connect_to_client_or_not - @ray.remote class Increase: @@ -283,8 +283,8 @@ def test_mini_integration(ray_start_cluster, connect_to_client): @ray.remote(num_cpus=0, num_gpus=1) def random_tasks(): - import time import random + import time sleep_time = random.uniform(0.1, 0.2) time.sleep(sleep_time) @@ -390,7 +390,7 @@ def test_capture_child_actors(ray_start_cluster, connect_to_client): # Make sure all the actors are scheduled on the same node. # (why? The placement group has STRICT_PACK strategy). node_id_set = set() - for actor_info in ray.state.actors().values(): + for actor_info in ray._private.state.actors().values(): if actor_info["State"] == convert_actor_state( gcs_utils.ActorTableData.ALIVE ): @@ -415,7 +415,7 @@ def test_capture_child_actors(ray_start_cluster, connect_to_client): # It is because the child tasks are not scheduled on the same # placement group. node_id_set = set() - for actor_info in ray.state.actors().values(): + for actor_info in ray._private.state.actors().values(): if actor_info["State"] == convert_actor_state( gcs_utils.ActorTableData.ALIVE ): @@ -440,7 +440,7 @@ def test_capture_child_actors(ray_start_cluster, connect_to_client): # It is because the child tasks are not scheduled on the same # placement group. node_id_set = set() - for actor_info in ray.state.actors().values(): + for actor_info in ray._private.state.actors().values(): if actor_info["State"] == convert_actor_state( gcs_utils.ActorTableData.ALIVE ): @@ -534,7 +534,7 @@ def test_ready_warning_suppressed(ray_start_regular, error_pubsub): ray.get(pg.ready(), timeout=0.5) errors = get_error_message( - p, 1, ray.ray_constants.INFEASIBLE_TASK_ERROR, timeout=0.1 + p, 1, ray._private.ray_constants.INFEASIBLE_TASK_ERROR, timeout=0.1 ) assert len(errors) == 0 @@ -590,7 +590,7 @@ ray.shutdown() # Wait until the driver is reported as dead by GCS. def is_job_done(): - jobs = ray.state.jobs() + jobs = ray._private.state.jobs() for job in jobs: if job["IsDead"]: return True @@ -666,7 +666,7 @@ ray.shutdown() # Wait until the driver is reported as dead by GCS. def is_job_done(): - jobs = ray.state.jobs() + jobs = ray._private.state.jobs() for job in jobs: if job["IsDead"]: return True diff --git a/python/ray/tests/test_placement_group_3.py b/python/ray/tests/test_placement_group_3.py index e948135f2..9a86d8439 100644 --- a/python/ray/tests/test_placement_group_3.py +++ b/python/ray/tests/test_placement_group_3.py @@ -1,31 +1,31 @@ -import pytest import sys import time +import pytest + +import ray +import ray._private.gcs_utils as gcs_utils +import ray.cluster_utils +import ray.experimental.internal_kv as internal_kv +from ray._private.ray_constants import DEBUG_AUTOSCALING_ERROR, DEBUG_AUTOSCALING_STATUS +from ray._private.test_utils import ( + convert_actor_state, + generate_system_config_map, + is_placement_group_removed, + kill_actor_and_wait_for_failure, + run_string_as_driver, + wait_for_condition, +) +from ray.autoscaler._private.commands import debug_status +from ray.exceptions import RaySystemError +from ray.util.client.ray_client_helpers import connect_to_client_or_not +from ray.util.placement_group import placement_group, remove_placement_group + try: import pytest_timeout except ImportError: pytest_timeout = None -import ray -import ray.cluster_utils -import ray._private.gcs_utils as gcs_utils - -from ray.autoscaler._private.commands import debug_status -from ray._private.test_utils import ( - generate_system_config_map, - kill_actor_and_wait_for_failure, - run_string_as_driver, - wait_for_condition, - is_placement_group_removed, - convert_actor_state, -) -from ray.exceptions import RaySystemError -from ray.util.placement_group import placement_group, remove_placement_group -from ray.util.client.ray_client_helpers import connect_to_client_or_not -import ray.experimental.internal_kv as internal_kv -from ray.ray_constants import DEBUG_AUTOSCALING_ERROR, DEBUG_AUTOSCALING_STATUS - def get_ray_status_output(address): gcs_client = gcs_utils.GcsClient(address=address) @@ -169,7 +169,7 @@ ray.shutdown() # Wait until the driver is reported as dead by GCS. def is_job_done(): - jobs = ray.state.jobs() + jobs = ray._private.state.jobs() for job in jobs: if job["IsDead"]: return True @@ -184,7 +184,7 @@ ray.shutdown() def assert_alive_num_actor(expected_num_actor): alive_num_actor = 0 - for actor_info in ray.state.actors().values(): + for actor_info in ray._private.state.actors().values(): if actor_info["State"] == convert_actor_state( gcs_utils.ActorTableData.ALIVE ): @@ -274,7 +274,7 @@ ray.shutdown() # Wait until the driver is reported as dead by GCS. def is_job_done(): - jobs = ray.state.jobs() + jobs = ray._private.state.jobs() for job in jobs: if job["IsDead"]: return True diff --git a/python/ray/tests/test_placement_group_5.py b/python/ray/tests/test_placement_group_5.py index aacfe1dc5..57d5cec6e 100644 --- a/python/ray/tests/test_placement_group_5.py +++ b/python/ray/tests/test_placement_group_5.py @@ -1,9 +1,10 @@ -import pytest import sys -import ray -from ray.util.client.ray_client_helpers import connect_to_client_or_not +import pytest + +import ray from ray.tests.test_placement_group import are_pairwise_unique +from ray.util.client.ray_client_helpers import connect_to_client_or_not @pytest.mark.parametrize("connect_to_client", [False, True]) @@ -66,7 +67,7 @@ def test_placement_group_bin_packing_priority( [ray.get(actor.value.remote()) for actor in actors] # Get all actors. - actor_infos = ray.state.actors() + actor_infos = ray._private.state.actors() # Make sure all actors in counter_list are located in separate nodes. actor_info_objs = [actor_infos.get(actor._actor_id.hex()) for actor in actors] diff --git a/python/ray/tests/test_ray_debugger.py b/python/ray/tests/test_ray_debugger.py index 4e8308007..7b070a8df 100644 --- a/python/ray/tests/test_ray_debugger.py +++ b/python/ray/tests/test_ray_debugger.py @@ -9,10 +9,9 @@ import pexpect import pytest import ray -from ray.cluster_utils import Cluster, cluster_not_supported -from ray import ray_constants +from ray._private import ray_constants, services from ray._private.test_utils import run_string_as_driver, wait_for_condition -from ray._private import services +from ray.cluster_utils import Cluster, cluster_not_supported def test_ray_debugger_breakpoint(shutdown_only): diff --git a/python/ray/tests/test_ray_init.py b/python/ray/tests/test_ray_init.py index 27ce34957..b5c2fa1ff 100644 --- a/python/ray/tests/test_ray_init.py +++ b/python/ray/tests/test_ray_init.py @@ -1,18 +1,19 @@ +import logging import os import sys - -import logging -import pytest import unittest.mock + +import grpc +import pytest + import ray import ray._private.services -from ray.util.client.ray_client_helpers import ray_start_client_server +from ray._private.test_utils import run_string_as_driver from ray.client_builder import ClientContext from ray.cluster_utils import Cluster -from ray._private.test_utils import run_string_as_driver from ray.util.client.common import ClientObjectRef +from ray.util.client.ray_client_helpers import ray_start_client_server from ray.util.client.worker import Worker -import grpc def test_shutdown_and_reset_global_worker(shutdown_only): @@ -305,9 +306,10 @@ def test_ray_init_using_hostname(ray_start_cluster): def test_redis_connect_backoff(): - from ray import ray_constants import time + from ray._private import ray_constants + unreachable_address = "127.0.0.1:65535" redis_ip, redis_port = unreachable_address.split(":") wait_retries = ray_constants.START_REDIS_WAIT_RETRIES @@ -331,7 +333,6 @@ def test_redis_connect_backoff(): if __name__ == "__main__": - import pytest import sys if os.environ.get("PARALLEL_CI"): diff --git a/python/ray/tests/test_raylet_output.py b/python/ray/tests/test_raylet_output.py index 6cdef3c1f..025de0c2e 100644 --- a/python/ray/tests/test_raylet_output.py +++ b/python/ray/tests/test_raylet_output.py @@ -1,12 +1,11 @@ +import glob import os import sys -import glob import pytest + import ray -from ray._private.test_utils import ( - wait_for_condition, -) +from ray._private.test_utils import wait_for_condition def enable_export_loglevel(func): @@ -19,7 +18,7 @@ def enable_export_loglevel(func): @enable_export_loglevel def test_ray_log_redirected(ray_start_regular): - session_dir = ray.worker._global_node.get_session_dir_path() + session_dir = ray._private.worker._global_node.get_session_dir_path() assert os.path.exists(session_dir), "Session dir not found." raylet_out_path = "{}/logs/raylet.out".format(session_dir) raylet_err_path = "{}/logs/raylet.err".format(session_dir) diff --git a/python/ray/tests/test_reconstruction_2.py b/python/ray/tests/test_reconstruction_2.py index 31dd62f14..2dfa70530 100644 --- a/python/ray/tests/test_reconstruction_2.py +++ b/python/ray/tests/test_reconstruction_2.py @@ -1,18 +1,14 @@ +import os import sys import time -import os import numpy as np import pytest import ray -from ray._private.test_utils import ( - wait_for_condition, - SignalActor, - Semaphore, -) -from ray.internal.internal_api import memory_summary -import ray.ray_constants as ray_constants +import ray._private.ray_constants as ray_constants +from ray._private.internal_api import memory_summary +from ray._private.test_utils import Semaphore, SignalActor, wait_for_condition # Task status. WAITING_FOR_DEPENDENCIES = "WAITING_FOR_DEPENDENCIES" diff --git a/python/ray/tests/test_reference_counting.py b/python/ray/tests/test_reference_counting.py index f456d4e08..849117b3c 100644 --- a/python/ray/tests/test_reference_counting.py +++ b/python/ray/tests/test_reference_counting.py @@ -6,18 +6,17 @@ import sys import time import numpy as np - import pytest import ray -import ray.cluster_utils import ray._private.gcs_utils as gcs_utils +import ray.cluster_utils from ray._private.test_utils import ( SignalActor, + convert_actor_state, kill_actor_and_wait_for_failure, put_object, wait_for_condition, - convert_actor_state, ) logger = logging.getLogger(__name__) @@ -42,16 +41,16 @@ def _fill_object_store_and_get(obj, succeed=True, object_MiB=20, num_objects=5): if succeed: wait_for_condition( - lambda: ray.worker.global_worker.core_worker.object_exists(obj) + lambda: ray._private.worker.global_worker.core_worker.object_exists(obj) ) else: wait_for_condition( - lambda: not ray.worker.global_worker.core_worker.object_exists(obj) + lambda: not ray._private.worker.global_worker.core_worker.object_exists(obj) ) def _check_refcounts(expected): - actual = ray.worker.global_worker.core_worker.get_all_reference_counts() + actual = ray._private.worker.global_worker.core_worker.get_all_reference_counts() assert len(expected) == len(actual) for object_ref, (local, submitted) in expected.items(): hex_id = object_ref.hex().encode("ascii") @@ -253,18 +252,24 @@ def test_feature_flag(shutdown_only): del put_ref wait_for_condition( - lambda: not ray.worker.global_worker.core_worker.object_exists(ref) + lambda: not ray._private.worker.global_worker.core_worker.object_exists(ref) ) def test_out_of_band_serialized_object_ref(one_worker_100MiB): - assert len(ray.worker.global_worker.core_worker.get_all_reference_counts()) == 0 + assert ( + len(ray._private.worker.global_worker.core_worker.get_all_reference_counts()) + == 0 + ) obj_ref = ray.put("hello") _check_refcounts({obj_ref: (1, 0)}) obj_ref_str = ray.cloudpickle.dumps(obj_ref) _check_refcounts({obj_ref: (2, 0)}) del obj_ref - assert len(ray.worker.global_worker.core_worker.get_all_reference_counts()) == 1 + assert ( + len(ray._private.worker.global_worker.core_worker.get_all_reference_counts()) + == 1 + ) assert ray.get(ray.cloudpickle.loads(obj_ref_str)) == "hello" @@ -519,7 +524,7 @@ def test_basic_nested_ids(one_worker_100MiB): def _all_actors_dead(): return all( actor["State"] == convert_actor_state(gcs_utils.ActorTableData.DEAD) - for actor in list(ray.state.actors().values()) + for actor in list(ray._private.state.actors().values()) ) diff --git a/python/ray/tests/test_reference_counting_2.py b/python/ray/tests/test_reference_counting_2.py index f88ae2484..71cb799e4 100644 --- a/python/ray/tests/test_reference_counting_2.py +++ b/python/ray/tests/test_reference_counting_2.py @@ -8,12 +8,11 @@ import sys import time import numpy as np - import pytest import ray import ray.cluster_utils -from ray.internal.internal_api import memory_summary +from ray._private.internal_api import memory_summary from ray._private.test_utils import SignalActor, put_object, wait_for_condition SIGKILL = signal.SIGKILL if sys.platform != "win32" else signal.SIGTERM @@ -43,11 +42,11 @@ def _fill_object_store_and_get(obj, succeed=True, object_MiB=20, num_objects=5): if succeed: wait_for_condition( - lambda: ray.worker.global_worker.core_worker.object_exists(obj) + lambda: ray._private.worker.global_worker.core_worker.object_exists(obj) ) else: wait_for_condition( - lambda: not ray.worker.global_worker.core_worker.object_exists(obj) + lambda: not ray._private.worker.global_worker.core_worker.object_exists(obj) ) @@ -178,7 +177,7 @@ def test_pass_returned_object_ref(one_worker_100MiB, use_ray_put, failure): assert failure def ref_not_exists(): - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker inner_oid = ray.ObjectRef(inner_oid_binary) return not worker.core_worker.object_exists(inner_oid) diff --git a/python/ray/tests/test_resource_demand_scheduler.py b/python/ray/tests/test_resource_demand_scheduler.py index 555e068dc..68992d31f 100644 --- a/python/ray/tests/test_resource_demand_scheduler.py +++ b/python/ray/tests/test_resource_demand_scheduler.py @@ -1,64 +1,64 @@ -import pytest -from datetime import datetime -from dataclasses import asdict -import json -import time -import yaml -import tempfile -import shutil -import unittest -from unittest import mock import copy +import json +import shutil +import tempfile +import time +import unittest +from dataclasses import asdict +from datetime import datetime +from time import sleep +from unittest import mock + +import pytest +import yaml import ray -import ray.ray_constants +import ray._private.ray_constants +from ray._private.gcs_utils import PlacementGroupTableData +from ray._private.test_utils import same_elements +from ray.autoscaler._private.autoscaler import AutoscalerSummary +from ray.autoscaler._private.commands import get_or_create_head_node +from ray.autoscaler._private.constants import AUTOSCALER_MAX_RESOURCE_DEMAND_VECTOR_SIZE +from ray.autoscaler._private.load_metrics import LoadMetrics +from ray.autoscaler._private.providers import _NODE_PROVIDERS, _clear_provider_cache +from ray.autoscaler._private.resource_demand_scheduler import ( + ResourceDemandScheduler, + _add_min_workers_nodes, + _utilization_score, + get_bin_pack_residual, +) +from ray.autoscaler._private.resource_demand_scheduler import get_nodes_for as _get from ray.autoscaler._private.util import ( - prepare_config, + LoadMetricsSummary, format_info_string, is_placement_group_resource, - LoadMetricsSummary, + prepare_config, ) +from ray.autoscaler.tags import ( + NODE_KIND_HEAD, + NODE_KIND_WORKER, + NODE_TYPE_LEGACY_HEAD, + NODE_TYPE_LEGACY_WORKER, + STATUS_UNINITIALIZED, + STATUS_UP_TO_DATE, + STATUS_WAITING_FOR_SSH, + TAG_RAY_NODE_KIND, + TAG_RAY_NODE_STATUS, + TAG_RAY_USER_NODE_TYPE, +) +from ray.core.generated.common_pb2 import Bundle, PlacementStrategy from ray.tests.test_autoscaler import ( - SMALL_CLUSTER, MOCK_DEFAULT_CONFIG, MULTI_WORKER_CLUSTER, + SMALL_CLUSTER, TYPES_A, - MockProvider, - MockProcessRunner, - MockNodeInfoStub, - mock_raylet_id, - fill_in_raylet_ids, MockAutoscaler, + MockNodeInfoStub, + MockProcessRunner, + MockProvider, + fill_in_raylet_ids, + mock_raylet_id, ) -from ray.autoscaler._private.providers import _NODE_PROVIDERS, _clear_provider_cache -from ray.autoscaler._private.autoscaler import AutoscalerSummary -from ray.autoscaler._private.load_metrics import LoadMetrics -from ray.autoscaler._private.commands import get_or_create_head_node -from ray.autoscaler._private.resource_demand_scheduler import ( - _utilization_score, - _add_min_workers_nodes, - get_bin_pack_residual, - get_nodes_for as _get, - ResourceDemandScheduler, -) -from ray._private.gcs_utils import PlacementGroupTableData -from ray.core.generated.common_pb2 import Bundle, PlacementStrategy -from ray.autoscaler.tags import ( - TAG_RAY_USER_NODE_TYPE, - TAG_RAY_NODE_KIND, - NODE_KIND_WORKER, - TAG_RAY_NODE_STATUS, - STATUS_UP_TO_DATE, - STATUS_UNINITIALIZED, - STATUS_WAITING_FOR_SSH, - NODE_KIND_HEAD, - NODE_TYPE_LEGACY_WORKER, - NODE_TYPE_LEGACY_HEAD, -) -from ray._private.test_utils import same_elements -from ray.autoscaler._private.constants import AUTOSCALER_MAX_RESOURCE_DEMAND_VECTOR_SIZE - -from time import sleep GET_DEFAULT_METHOD = "ray.autoscaler._private.util._get_default_config" diff --git a/python/ray/tests/test_runtime_env.py b/python/ray/tests/test_runtime_env.py index c2415e32d..78a00d431 100644 --- a/python/ray/tests/test_runtime_env.py +++ b/python/ray/tests/test_runtime_env.py @@ -178,7 +178,7 @@ def test_no_spurious_worker_startup(shutdown_only, runtime_env_class): assert ray.get(a.get.remote()) == 0 # Check "debug_state.txt" to ensure no extra workers were started. - session_dir = ray.worker.global_worker.node.address_info["session_dir"] + session_dir = ray._private.worker.global_worker.node.address_info["session_dir"] session_path = Path(session_dir) debug_state_path = session_path / "logs" / "debug_state.txt" @@ -235,7 +235,7 @@ def test_runtime_env_no_spurious_resource_deadlock_msg( # Check no warning printed. ray.get(f.remote()) - errors = get_error_message(p, 5, ray.ray_constants.RESOURCE_DEADLOCK_ERROR) + errors = get_error_message(p, 5, ray._private.ray_constants.RESOURCE_DEADLOCK_ERROR) assert len(errors) == 0 @@ -639,7 +639,7 @@ def test_serialize_deserialize(option): elif option == "container": runtime_env["container"] = { "image": "anyscale/ray-ml:nightly-py38-cpu", - "worker_path": "/root/python/ray/workers/default_worker.py", + "worker_path": "/root/python/ray/_private/workers/default_worker.py", "run_options": ["--cap-drop SYS_ADMIN", "--log-level=debug"], } elif option == "plugins": @@ -811,7 +811,7 @@ def test_runtime_env_interface(): # Test the interface related to container container_init = { "image": "anyscale/ray-ml:nightly-py38-cpu", - "worker_path": "/root/python/ray/workers/default_worker.py", + "worker_path": "/root/python/ray/_private/workers/default_worker.py", "run_options": ["--cap-drop SYS_ADMIN", "--log-level=debug"], } update_container = {"image": "test_modify"} diff --git a/python/ray/tests/test_runtime_env_packaging.py b/python/ray/tests/test_runtime_env_packaging.py index 31a536aae..ec59f6af4 100644 --- a/python/ray/tests/test_runtime_env_packaging.py +++ b/python/ray/tests/test_runtime_env_packaging.py @@ -1,40 +1,41 @@ import os -from pathlib import Path import random -from shutil import copytree, rmtree, make_archive import shutil import socket import string import sys -from filecmp import dircmp import uuid +from filecmp import dircmp +from pathlib import Path +from shutil import copytree, make_archive, rmtree import pytest -from ray.ray_constants import KV_NAMESPACE_PACKAGE + +from ray._private.gcs_utils import GcsClient +from ray._private.ray_constants import KV_NAMESPACE_PACKAGE +from ray._private.runtime_env.packaging import ( + GCS_STORAGE_MAX_SIZE, + Protocol, + _dir_travel, + _get_excludes, + _store_package_in_gcs, + get_local_dir_from_uri, + get_top_level_dir_from_compressed_package, + get_uri_for_directory, + get_uri_for_package, + is_whl_uri, + is_zip_uri, + parse_uri, + remove_dir_from_filepaths, + unzip_package, + upload_package_if_needed, +) from ray.experimental.internal_kv import ( - _internal_kv_reset, _initialize_internal_kv, _internal_kv_del, _internal_kv_exists, _internal_kv_get, -) -from ray._private.gcs_utils import GcsClient -from ray._private.runtime_env.packaging import ( - _dir_travel, - _store_package_in_gcs, - get_local_dir_from_uri, - get_uri_for_directory, - _get_excludes, - get_uri_for_package, - upload_package_if_needed, - parse_uri, - is_zip_uri, - is_whl_uri, - Protocol, - get_top_level_dir_from_compressed_package, - remove_dir_from_filepaths, - unzip_package, - GCS_STORAGE_MAX_SIZE, + _internal_kv_reset, ) TOP_LEVEL_DIR_NAME = "top_level" diff --git a/python/ray/tests/test_runtime_env_working_dir_3.py b/python/ray/tests/test_runtime_env_working_dir_3.py index d027ac4f2..72997a526 100644 --- a/python/ray/tests/test_runtime_env_working_dir_3.py +++ b/python/ray/tests/test_runtime_env_working_dir_3.py @@ -1,22 +1,19 @@ -from pathlib import Path import os import sys -import time import tempfile +import time +from pathlib import Path +from unittest import mock import pytest from pytest_lazyfixture import lazy_fixture -from unittest import mock import ray import ray.experimental.internal_kv as kv -from ray.ray_constants import ( - RAY_RUNTIME_ENV_URI_PIN_EXPIRATION_S_ENV_VAR, -) -from ray._private.test_utils import wait_for_condition, chdir, check_local_files_gced +from ray._private.ray_constants import RAY_RUNTIME_ENV_URI_PIN_EXPIRATION_S_ENV_VAR +from ray._private.test_utils import chdir, check_local_files_gced, wait_for_condition from ray._private.utils import get_directory_size_bytes - # This test requires you have AWS credentials set up (any AWS credentials will # do, this test only accesses a public bucket). diff --git a/python/ray/tests/test_scheduling.py b/python/ray/tests/test_scheduling.py index 6d691453d..e23916d12 100644 --- a/python/ray/tests/test_scheduling.py +++ b/python/ray/tests/test_scheduling.py @@ -11,16 +11,15 @@ import numpy as np import pytest import ray -from ray.internal.internal_api import memory_summary -import ray.util.accelerators import ray.cluster_utils -from ray._private.test_utils import fetch_prometheus - +import ray.util.accelerators +from ray._private.internal_api import memory_summary from ray._private.test_utils import ( - wait_for_condition, Semaphore, - object_memory_usage, SignalActor, + fetch_prometheus, + object_memory_usage, + wait_for_condition, ) logger = logging.getLogger(__name__) @@ -54,7 +53,7 @@ def test_load_balancing(ray_start_cluster): @ray.remote def f(): time.sleep(0.10) - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id attempt_to_load_balance(f, [], 100, num_nodes, 10) attempt_to_load_balance(f, [], 1000, num_nodes, 100) @@ -83,7 +82,7 @@ def test_hybrid_policy(ray_start_cluster): def get_node(): ray.get(block_driver.release.remote()) ray.get(block_task.acquire.remote()) - return ray.worker.global_worker.current_node_id + return ray._private.worker.global_worker.current_node_id # Below the hybrid threshold we pack on the local node first. refs = [get_node.remote() for _ in range(5)] @@ -138,7 +137,7 @@ def test_legacy_spillback_distribution(ray_start_cluster): @ray.remote def task(): time.sleep(1) - return ray.worker.global_worker.current_node_id + return ray._private.worker.global_worker.current_node_id # Make sure tasks are spilled back non-deterministically. locations = ray.get([task.remote() for _ in range(8)]) @@ -155,7 +154,7 @@ def test_legacy_spillback_distribution(ray_start_cluster): pass def get_location(self): - return ray.worker.global_worker.current_node_id + return ray._private.worker.global_worker.current_node_id actors = [Actor1.remote() for _ in range(10)] locations = ray.get([actor.get_location.remote() for actor in actors]) @@ -181,10 +180,10 @@ def test_local_scheduling_first(ray_start_cluster): @ray.remote(num_cpus=1) def f(): time.sleep(0.01) - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id def local(): - return ray.get(f.remote()) == ray.worker.global_worker.node.unique_id + return ray.get(f.remote()) == ray._private.worker.global_worker.node.unique_id # Wait for a worker to get started. wait_for_condition(local) @@ -206,7 +205,7 @@ def test_load_balancing_with_dependencies(ray_start_cluster): @ray.remote def f(x): time.sleep(0.1) - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id # This object will be local to one of the raylets. Make sure # this doesn't prevent tasks from being scheduled on other raylets. @@ -278,11 +277,11 @@ def test_spread_scheduling_overrides_locality_aware_scheduling(ray_start_cluster @ray.remote(resources={"pin": 1}) def non_local(): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id @ray.remote(scheduling_strategy="SPREAD") def f(x): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id # Test that task f() runs on the local node as well # even though remote node has the dependencies. @@ -327,11 +326,11 @@ def test_locality_aware_leasing(ray_start_cluster): @ray.remote(resources={"pin": 1}) def non_local(): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id @ray.remote def f(x): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id # Test that task f() runs on the same node as non_local() # even though local node is lower critical resource utilization. @@ -359,15 +358,15 @@ def test_locality_aware_leasing_cached_objects(ray_start_cluster): @ray.remote def f(): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id @ray.remote def g(x): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id @ray.remote def h(x, y): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id # f_obj1 pinned on worker1. f_obj1 = f.options(resources={"pin_worker1": 1}).remote() @@ -401,7 +400,7 @@ def test_locality_aware_leasing_borrowed_objects(ray_start_cluster): @ray.remote def f(): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id @ray.remote def g(x): @@ -409,7 +408,7 @@ def test_locality_aware_leasing_borrowed_objects(ray_start_cluster): @ray.remote def h(x): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id # f will run on worker, f_obj will be pinned on worker. f_obj = f.options(resources={"pin_worker": 1}).remote() @@ -478,7 +477,7 @@ def test_many_args(ray_start_cluster): ( num_tasks_submitted_before, num_leases_requested_before, - ) = ray.worker.global_worker.core_worker.get_task_submission_stats() + ) = ray._private.worker.global_worker.core_worker.get_task_submission_stats() tasks = [] for i in range(100): args = [np.random.choice(xs) for _ in range(10)] @@ -488,7 +487,7 @@ def test_many_args(ray_start_cluster): ( num_tasks_submitted, num_leases_requested, - ) = ray.worker.global_worker.core_worker.get_task_submission_stats() + ) = ray._private.worker.global_worker.core_worker.get_task_submission_stats() num_tasks_submitted -= num_tasks_submitted_before num_leases_requested -= num_leases_requested_before print("submitted:", num_tasks_submitted, "leases requested:", num_leases_requested) @@ -553,12 +552,12 @@ def test_gpu(monkeypatch): pass def get_location(self): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id @ray.remote(num_cpus=1) def task_cpu(): time.sleep(10) - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id @ray.remote(num_returns=2, num_gpus=0.5) def launcher(): @@ -568,7 +567,7 @@ def test_gpu(monkeypatch): actor_results = [a.get_location.remote() for _ in range(n)] return ( ray.get(task_results + actor_results), - ray.worker.global_worker.node.unique_id, + ray._private.worker.global_worker.node.unique_id, ) r = launcher.remote() diff --git a/python/ray/tests/test_scheduling_2.py b/python/ray/tests/test_scheduling_2.py index 824b0eab8..6a7a5a63f 100644 --- a/python/ray/tests/test_scheduling_2.py +++ b/python/ray/tests/test_scheduling_2.py @@ -1,19 +1,20 @@ -import numpy as np +import os import platform -import pytest import sys import time -import os + +import numpy as np +import pytest import ray import ray._private.gcs_utils as gcs_utils -from ray.util.client.ray_client_helpers import connect_to_client_or_not import ray.experimental.internal_kv as internal_kv +from ray._private.test_utils import make_global_state_accessor, wait_for_condition +from ray.util.client.ray_client_helpers import connect_to_client_or_not from ray.util.scheduling_strategies import ( - PlacementGroupSchedulingStrategy, NodeAffinitySchedulingStrategy, + PlacementGroupSchedulingStrategy, ) -from ray._private.test_utils import wait_for_condition, make_global_state_accessor @pytest.mark.skipif( @@ -49,9 +50,9 @@ def test_load_balancing_under_constrained_memory( @ray.remote def f(i, x): - print(i, ray.worker.global_worker.node.unique_id) + print(i, ray._private.worker.global_worker.node.unique_id) time.sleep(0.1) - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id deps = [create_object.remote() for _ in range(num_tasks)] for i, dep in enumerate(deps): @@ -83,7 +84,7 @@ def test_critical_object_store_mem_resource_utilization(ray_start_cluster): @ray.remote def f(): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id # Wait for resource availabilities to propagate. time.sleep(1) @@ -112,7 +113,7 @@ def test_default_scheduling_strategy(ray_start_cluster, connect_to_client): @ray.remote(scheduling_strategy="DEFAULT") def get_node_id_1(): - return ray.worker.global_worker.current_node_id + return ray._private.worker.global_worker.current_node_id head_node_id = ray.get(get_node_id_1.options(resources={"head": 1}).remote()) worker_node_id = ray.get( @@ -126,7 +127,7 @@ def test_default_scheduling_strategy(ray_start_cluster, connect_to_client): scheduling_strategy=PlacementGroupSchedulingStrategy(placement_group=pg), ) def get_node_id_2(): - return ray.worker.global_worker.current_node_id + return ray._private.worker.global_worker.current_node_id assert ( ray.get(get_node_id_2.options(scheduling_strategy="DEFAULT").remote()) @@ -135,7 +136,7 @@ def test_default_scheduling_strategy(ray_start_cluster, connect_to_client): @ray.remote def get_node_id_3(): - return ray.worker.global_worker.current_node_id + return ray._private.worker.global_worker.current_node_id @ray.remote( num_cpus=1, @@ -146,7 +147,7 @@ def test_default_scheduling_strategy(ray_start_cluster, connect_to_client): class Actor1: def get_node_ids(self): return [ - ray.worker.global_worker.current_node_id, + ray._private.worker.global_worker.current_node_id, # Use parent's placement group ray.get(get_node_id_3.remote()), ray.get( @@ -177,7 +178,7 @@ def test_placement_group_scheduling_strategy(ray_start_cluster, connect_to_clien @ray.remote(scheduling_strategy="DEFAULT") def get_node_id_1(): - return ray.worker.global_worker.current_node_id + return ray._private.worker.global_worker.current_node_id worker_node_id = ray.get( get_node_id_1.options(resources={"worker": 1}).remote() @@ -200,7 +201,7 @@ def test_placement_group_scheduling_strategy(ray_start_cluster, connect_to_clien scheduling_strategy=PlacementGroupSchedulingStrategy(placement_group=pg), ) def get_node_id_2(): - return ray.worker.global_worker.current_node_id + return ray._private.worker.global_worker.current_node_id assert ray.get(get_node_id_2.remote()) == worker_node_id @@ -210,7 +211,7 @@ def test_placement_group_scheduling_strategy(ray_start_cluster, connect_to_clien ) class Actor1: def get_node_id(self): - return ray.worker.global_worker.current_node_id + return ray._private.worker.global_worker.current_node_id actor1 = Actor1.remote() assert ray.get(actor1.get_node_id.remote()) == worker_node_id @@ -218,7 +219,7 @@ def test_placement_group_scheduling_strategy(ray_start_cluster, connect_to_clien @ray.remote class Actor2: def get_node_id(self): - return ray.worker.global_worker.current_node_id + return ray._private.worker.global_worker.current_node_id actor2 = Actor2.options( scheduling_strategy=PlacementGroupSchedulingStrategy(placement_group=pg) diff --git a/python/ray/tests/test_scheduling_performance.py b/python/ray/tests/test_scheduling_performance.py index 4f8b09fba..01a006a92 100644 --- a/python/ray/tests/test_scheduling_performance.py +++ b/python/ray/tests/test_scheduling_performance.py @@ -5,10 +5,9 @@ import time import pytest -import ray.cluster_utils -import ray._private.test_utils - import ray +import ray._private.test_utils +import ray.cluster_utils logger = logging.getLogger(__name__) @@ -46,7 +45,7 @@ def test_actor_scheduling_latency(ray_start_cluster, args): self.start = time.time() def info(self): - return [ray.worker.global_worker.node.unique_id, self.start] + return [ray._private.worker.global_worker.node.unique_id, self.start] def create(self, num): ret_list = [] @@ -61,7 +60,7 @@ def test_actor_scheduling_latency(ray_start_cluster, args): self.start = time.time() def info(self): - return [ray.worker.global_worker.node.unique_id, self.start] + return [ray._private.worker.global_worker.node.unique_id, self.start] actor_distribution = {} actor_list = [] diff --git a/python/ray/tests/test_serialization.py b/python/ray/tests/test_serialization.py index 9358872ac..69ac4f509 100644 --- a/python/ray/tests/test_serialization.py +++ b/python/ray/tests/test_serialization.py @@ -8,8 +8,8 @@ import sys import weakref import numpy as np -from numpy import log import pytest +from numpy import log import ray import ray.cluster_utils @@ -333,6 +333,7 @@ def test_numpy_subclass_serialization_pickle(ray_start_regular): def test_inspect_serialization(enable_pickle_debug): import threading + from ray.cloudpickle import dumps_debug lock = threading.Lock() @@ -547,7 +548,7 @@ def test_reducer_override_no_reference_cycle(ray_start_shared_local_modes): wr = weakref.ref(f) bio = io.BytesIO() - from ray.cloudpickle import CloudPickler, loads, dumps + from ray.cloudpickle import CloudPickler, dumps, loads p = CloudPickler(bio, protocol=5) p.dump(f) @@ -655,7 +656,7 @@ def test_serialization_before_init(shutdown_only): works after initialization.""" # make sure ray is shutdown ray.shutdown() - assert ray.worker.global_worker.current_job_id.is_nil() + assert ray._private.worker.global_worker.current_job_id.is_nil() import threading diff --git a/python/ray/tests/test_state_api.py b/python/ray/tests/test_state_api.py index 0fc8ddfe6..5898a651a 100644 --- a/python/ray/tests/test_state_api.py +++ b/python/ray/tests/test_state_api.py @@ -10,7 +10,7 @@ from click.testing import CliRunner import ray import ray.dashboard.consts as dashboard_consts -import ray.ray_constants as ray_constants +import ray._private.ray_constants as ray_constants from ray._private.test_utils import wait_for_condition from ray.cluster_utils import cluster_not_supported from ray.core.generated.common_pb2 import ( @@ -766,7 +766,7 @@ async def test_state_data_source_client(ray_start_cluster): Test job """ job_client = JobSubmissionClient( - f"http://{ray.worker.global_worker.node.address_info['webui_url']}" + f"http://{ray._private.worker.global_worker.node.address_info['webui_url']}" ) job_id = job_client.submit_job( # noqa # Entrypoint shell command to execute @@ -903,7 +903,7 @@ def test_cli_apis_sanity_check(ray_start_cluster): runner = CliRunner() client = JobSubmissionClient( - f"http://{ray.worker.global_worker.node.address_info['webui_url']}" + f"http://{ray._private.worker.global_worker.node.address_info['webui_url']}" ) @ray.remote @@ -1017,7 +1017,7 @@ def test_list_nodes(shutdown_only): def test_list_jobs(shutdown_only): ray.init() client = JobSubmissionClient( - f"http://{ray.worker.global_worker.node.address_info['webui_url']}" + f"http://{ray._private.worker.global_worker.node.address_info['webui_url']}" ) job_id = client.submit_job( # noqa # Entrypoint shell command to execute @@ -1271,7 +1271,7 @@ def test_network_failure(shutdown_only): wait_for_condition(lambda: len(list_tasks()) == 4) # Kill raylet so that list_tasks will have network error on querying raylets. - ray.worker._global_node.kill_raylet() + ray._private.worker._global_node.kill_raylet() with pytest.raises(RayStateApiException): list_tasks(_explain=True) diff --git a/python/ray/tests/test_storage.py b/python/ray/tests/test_storage.py index 676784f67..3c070cabc 100644 --- a/python/ray/tests/test_storage.py +++ b/python/ray/tests/test_storage.py @@ -1,14 +1,15 @@ import os +import subprocess import urllib from pathlib import Path + import pyarrow.fs import pytest -import subprocess import ray -import ray.internal.storage as storage -from ray.tests.conftest import * # noqa +import ray._private.storage as storage from ray._private.test_utils import simulate_storage +from ray.tests.conftest import * # noqa def _custom_fs(uri): @@ -157,7 +158,7 @@ def test_connecting_to_cluster(shutdown_only, storage_type): try: subprocess.check_call(["ray", "start", "--head", "--storage", storage_uri]) ray.init(address="auto") - from ray.internal.storage import _storage_uri + from ray._private.storage import _storage_uri # make sure driver is using the same storage when connecting to a cluster assert _storage_uri == storage_uri diff --git a/python/ray/tests/test_stress_failure.py b/python/ray/tests/test_stress_failure.py index 14ed4a921..82eb09aa7 100644 --- a/python/ray/tests/test_stress_failure.py +++ b/python/ray/tests/test_stress_failure.py @@ -1,12 +1,13 @@ -import numpy as np -import pytest import sys import time +import numpy as np +import pytest + import ray -from ray.cluster_utils import Cluster, cluster_not_supported -import ray.ray_constants as ray_constants +import ray._private.ray_constants as ray_constants from ray._private.test_utils import get_error_message +from ray.cluster_utils import Cluster, cluster_not_supported @pytest.mark.xfail(cluster_not_supported, reason="cluster not supported") diff --git a/python/ray/tests/test_tempfile.py b/python/ray/tests/test_tempfile.py index 6201cd820..dfbfdbaa6 100644 --- a/python/ray/tests/test_tempfile.py +++ b/python/ray/tests/test_tempfile.py @@ -4,6 +4,7 @@ import sys import time import pytest + import ray from ray._private.test_utils import check_call_ray, wait_for_condition @@ -82,7 +83,7 @@ def test_raylet_tempfiles(shutdown_only): ) ray.init(num_cpus=0) - node = ray.worker._global_node + node = ray._private.worker._global_node top_levels = set(os.listdir(node.get_session_dir_path())) assert top_levels.issuperset({"sockets", "logs"}) log_files_expected = { @@ -115,7 +116,7 @@ def test_raylet_tempfiles(shutdown_only): ray.shutdown() ray.init(num_cpus=2) - node = ray.worker._global_node + node = ray._private.worker._global_node top_levels = set(os.listdir(node.get_session_dir_path())) assert top_levels.issuperset({"sockets", "logs"}) time.sleep(3) # wait workers to start @@ -135,7 +136,7 @@ def test_tempdir_privilege(shutdown_only): os.makedirs(tmp_dir, exist_ok=True) os.chmod(tmp_dir, 0o000) ray.init(num_cpus=1) - session_dir = ray.worker._global_node.get_session_dir_path() + session_dir = ray._private.worker._global_node.get_session_dir_path() assert os.path.exists(session_dir), "Specified socket path not found." @@ -143,7 +144,7 @@ def test_session_dir_uniqueness(): session_dirs = set() for i in range(2): ray.init(num_cpus=1) - session_dirs.add(ray.worker._global_node.get_session_dir_path) + session_dirs.add(ray._private.worker._global_node.get_session_dir_path) ray.shutdown() assert len(session_dirs) == 2 diff --git a/python/ray/tests/test_threaded_actor.py b/python/ray/tests/test_threaded_actor.py index d8e4af1dc..b3dadbc21 100644 --- a/python/ray/tests/test_threaded_actor.py +++ b/python/ray/tests/test_threaded_actor.py @@ -2,12 +2,12 @@ import sys import threading import time -import pytest import numpy as np -import ray +import pytest -from ray.state import available_resources +import ray import ray._private.test_utils as test_utils +from ray._private.state import available_resources def ensure_cpu_returned(expected_cpus): diff --git a/python/ray/tests/test_unhandled_error.py b/python/ray/tests/test_unhandled_error.py index 885c01fd2..276ae56c4 100644 --- a/python/ray/tests/test_unhandled_error.py +++ b/python/ray/tests/test_unhandled_error.py @@ -26,7 +26,7 @@ if __name__ == "__main__": ray.init(num_cpus=1) # Test we report unhandled exceptions. - ray.worker._unhandled_error_handler = interceptor + ray._private.worker._unhandled_error_handler = interceptor x1 = f.remote() start = time.time() diff --git a/python/ray/tests/test_usage_stats.py b/python/ray/tests/test_usage_stats.py index 278c95eb0..3cc404267 100644 --- a/python/ray/tests/test_usage_stats.py +++ b/python/ray/tests/test_usage_stats.py @@ -72,7 +72,7 @@ def read_file(temp_dir: Path, column: str): def print_dashboard_log(): - session_dir = ray.worker.global_worker.node.address_info["session_dir"] + session_dir = ray._private.worker.global_worker.node.address_info["session_dir"] session_path = Path(session_dir) log_dir_path = session_path / "logs" @@ -712,7 +712,7 @@ provider: Verify the usage_stats.json is updated. """ print("Verifying usage stats write.") - global_node = ray.worker._global_node + global_node = ray._private.worker._global_node temp_dir = pathlib.Path(global_node.get_session_dir_path()) wait_for_condition(lambda: file_exists(temp_dir), timeout=30) @@ -741,7 +741,7 @@ def test_first_usage_report_delayed(monkeypatch, ray_start_cluster, reset_lib_us # The first report should be delayed for 10s. time.sleep(5) - session_dir = ray.worker.global_worker.node.address_info["session_dir"] + session_dir = ray._private.worker.global_worker.node.address_info["session_dir"] session_path = Path(session_dir) assert not (session_path / usage_constants.USAGE_STATS_FILE).exists() @@ -765,7 +765,7 @@ def test_usage_report_disabled(monkeypatch, ray_start_cluster, reset_lib_usage): # Wait enough so that usage report should happen. time.sleep(5) - session_dir = ray.worker.global_worker.node.address_info["session_dir"] + session_dir = ray._private.worker.global_worker.node.address_info["session_dir"] session_path = Path(session_dir) log_dir_path = session_path / "logs" @@ -803,7 +803,7 @@ def test_usage_file_error_message(monkeypatch, ray_start_cluster, reset_lib_usag cluster.add_node(num_cpus=0) ray.init(address=cluster.address) - global_node = ray.worker._global_node + global_node = ray._private.worker._global_node temp_dir = pathlib.Path(global_node.get_session_dir_path()) try: wait_for_condition(lambda: file_exists(temp_dir), timeout=30) diff --git a/python/ray/train/_internal/backend_executor.py b/python/ray/train/_internal/backend_executor.py index 865ee470b..b8ccec69c 100644 --- a/python/ray/train/_internal/backend_executor.py +++ b/python/ray/train/_internal/backend_executor.py @@ -4,10 +4,10 @@ from collections import defaultdict from typing import Callable, Dict, List, Optional, Tuple, Type, TypeVar import ray -from ray.air.checkpoint import Checkpoint +from ray._private.ray_constants import env_integer from ray.exceptions import RayActorError -from ray.ray_constants import env_integer from ray.train._internal.dataset_spec import RayDatasetSpec +from ray.air.checkpoint import Checkpoint from ray.train._internal.session import ( TrainingResult, TrialInfo, @@ -147,8 +147,9 @@ class BackendExecutor: self._placement_group. """ current_placement_group = get_current_placement_group() + worker = ray._private.worker.global_worker should_capture_child_tasks_in_placement_group = ( - ray.worker.global_worker.should_capture_child_tasks_in_placement_group + worker.should_capture_child_tasks_in_placement_group ) should_create_placement_group = ( current_placement_group is None diff --git a/python/ray/train/callbacks/logging.py b/python/ray/train/callbacks/logging.py index 8bbfed43b..8893a388e 100644 --- a/python/ray/train/callbacks/logging.py +++ b/python/ray/train/callbacks/logging.py @@ -3,32 +3,32 @@ import logging import os import warnings from pathlib import Path -from typing import List, Optional, Dict, Set, Tuple, Union +from typing import Dict, List, Optional, Set, Tuple, Union import numpy as np -from ray.train.callbacks import TrainingCallback -from ray.train.callbacks.callback import _deprecation_msg from ray.train._internal.results_preprocessors import ( - IndexedResultsPreprocessor, ExcludedKeysResultsPreprocessor, + IndexedResultsPreprocessor, ) from ray.train._internal.results_preprocessors.preprocessor import ( SequentialResultsPreprocessor, ) +from ray.train.callbacks import TrainingCallback +from ray.train.callbacks.callback import _deprecation_msg from ray.train.constants import ( + PID, RESULT_FILE_JSON, - TRAINING_ITERATION, TIME_TOTAL_S, TIMESTAMP, - PID, TRAIN_CHECKPOINT_SUBDIR, + TRAINING_ITERATION, ) from ray.util.annotations import Deprecated from ray.util.debug import log_once from ray.util.ml_utils.dict import flatten_dict from ray.util.ml_utils.json import SafeFallbackEncoder -from ray.util.ml_utils.mlflow import MLflowLoggerUtil +from ray.util.ml_utils.mlflow import _MLflowLoggerUtil logger = logging.getLogger(__name__) @@ -211,7 +211,7 @@ class MLflowLoggerCallback(TrainingCallback): self.tags = tags self.save_artifact = save_artifact - self.mlflow_util = MLflowLoggerUtil() + self.mlflow_util = _MLflowLoggerUtil() def start_training(self, logdir: str, config: Dict, **info): self._logdir_manager.setup_logdir(default_logdir=logdir) diff --git a/python/ray/train/tests/test_backend.py b/python/ray/train/tests/test_backend.py index 56dadbd94..b9b173cbf 100644 --- a/python/ray/train/tests/test_backend.py +++ b/python/ray/train/tests/test_backend.py @@ -7,7 +7,9 @@ import pytest import ray import ray.train as train from ray.cluster_utils import Cluster -from ray.train.backend import Backend, BackendConfig + +# Trigger pytest hook to automatically zip test cluster logs to archive dir on failure +from ray.tests.conftest import pytest_runtest_makereport # noqa from ray.train._internal.backend_executor import ( BackendExecutor, InactiveWorkerGroupError, @@ -15,18 +17,16 @@ from ray.train._internal.backend_executor import ( TrainingWorkerError, ) from ray.train._internal.dataset_spec import RayDatasetSpec -from ray.train.tensorflow import TensorflowConfig -from ray.train.torch import TorchConfig +from ray.train._internal.worker_group import WorkerGroup +from ray.train.backend import Backend, BackendConfig from ray.train.constants import ( ENABLE_SHARE_CUDA_VISIBLE_DEVICES_ENV, TRAIN_ENABLE_WORKER_SPREAD_ENV, ) -from ray.train._internal.worker_group import WorkerGroup +from ray.train.tensorflow import TensorflowConfig +from ray.train.torch import TorchConfig from ray.util.placement_group import get_current_placement_group -# Trigger pytest hook to automatically zip test cluster logs to archive dir on failure -from ray.tests.conftest import pytest_runtest_makereport # noqa - @pytest.fixture def ray_start_2_cpus(): @@ -354,7 +354,7 @@ def test_cuda_visible_devices_multiple(ray_2_node_4_gpu, worker_results): def get_node_id_set(): node_id_set = set() - for actor_info in ray.state.actors().values(): + for actor_info in ray._private.state.actors().values(): node_id = actor_info["Address"]["NodeID"] node_id_set.add(node_id) return node_id_set @@ -413,7 +413,8 @@ def test_placement_group_parent(ray_4_node_4_cpu, placement_group_capture_child_ if __name__ == "__main__": - import pytest import sys + import pytest + sys.exit(pytest.main(["-v", "-x", __file__])) diff --git a/python/ray/train/tests/test_worker_group.py b/python/ray/train/tests/test_worker_group.py index bc307547e..249d2b3b4 100644 --- a/python/ray/train/tests/test_worker_group.py +++ b/python/ray/train/tests/test_worker_group.py @@ -1,6 +1,7 @@ -import pytest import time +import pytest + import ray from ray.train._internal.worker_group import WorkerGroup @@ -38,7 +39,7 @@ def test_worker_shutdown(ray_start_2_cpus): wg = WorkerGroup(num_workers=2) time.sleep(1) assert "CPU" not in ray.available_resources() - assert len(ray.state.actors()) == 2 + assert len(ray._private.state.actors()) == 2 wg.shutdown() time.sleep(1) assert ray.available_resources()["CPU"] == 2 @@ -121,7 +122,8 @@ def test_placement_group(ray_start_2_cpus): if __name__ == "__main__": - import pytest import sys + import pytest + sys.exit(pytest.main(["-v", "-x", __file__])) diff --git a/python/ray/tune/impl/out_of_band_serialize_dataset.py b/python/ray/tune/impl/out_of_band_serialize_dataset.py index 62971bb67..380869fa3 100644 --- a/python/ray/tune/impl/out_of_band_serialize_dataset.py +++ b/python/ray/tune/impl/out_of_band_serialize_dataset.py @@ -25,7 +25,7 @@ def _reduce(ds: ray.data.Dataset): @contextlib.contextmanager def out_of_band_serialize_dataset(): - context = ray.worker.global_worker.get_serialization_context() + context = ray._private.worker.global_worker.get_serialization_context() try: context._register_cloudpickle_reducer(ray.data.Dataset, _reduce) yield diff --git a/python/ray/tune/integration/mlflow.py b/python/ray/tune/integration/mlflow.py index 9f7d382da..42327f8fb 100644 --- a/python/ray/tune/integration/mlflow.py +++ b/python/ray/tune/integration/mlflow.py @@ -1,12 +1,12 @@ -from typing import Dict, Callable, Optional import logging +from typing import Callable, Dict, Optional import ray -from ray.tune.trainable import Trainable from ray.tune.logger import LoggerCallback -from ray.tune.result import TRAINING_ITERATION, TIMESTEPS_TOTAL +from ray.tune.result import TIMESTEPS_TOTAL, TRAINING_ITERATION +from ray.tune.trainable import Trainable from ray.tune.trial import Trial -from ray.util.ml_utils.mlflow import MLflowLoggerUtil +from ray.util.ml_utils.mlflow import _MLflowLoggerUtil logger = logging.getLogger(__name__) @@ -75,7 +75,7 @@ class MLflowLoggerCallback(LoggerCallback): self.tags = tags self.should_save_artifact = save_artifact - self.mlflow_util = MLflowLoggerUtil() + self.mlflow_util = _MLflowLoggerUtil() if ray.util.client.ray.is_connected(): logger.warning( @@ -247,7 +247,7 @@ def mlflow_mixin(func: Callable): class MLflowTrainableMixin: def __init__(self, config: Dict, *args, **kwargs): - self.mlflow_util = MLflowLoggerUtil() + self.mlflow_util = _MLflowLoggerUtil() if not isinstance(self, Trainable): raise ValueError( diff --git a/python/ray/tune/ray_trial_executor.py b/python/ray/tune/ray_trial_executor.py index 4a0df71eb..b497be166 100644 --- a/python/ray/tune/ray_trial_executor.py +++ b/python/ray/tune/ray_trial_executor.py @@ -1,44 +1,36 @@ # coding: utf-8 import copy import inspect -import random -from collections import deque -from enum import Enum -from functools import partial import logging import os +import random import time import traceback +from collections import deque from contextlib import contextmanager -from typing import ( - Callable, - Dict, - Iterable, - List, - Optional, - Set, - Union, -) +from enum import Enum +from functools import partial +from typing import Callable, Dict, Iterable, List, Optional, Set, Union import ray from ray.exceptions import GetTimeoutError, RayTaskError from ray.tune.error import ( - _AbortTrialExecution, TuneError, - _TuneStartTrialError, + _AbortTrialExecution, _TuneNoNextExecutorEventError, + _TuneStartTrialError, ) from ray.tune.logger import NoopLogger -from ray.tune.result import TRIAL_INFO, STDOUT_FILE, STDERR_FILE -from ray.tune.utils.placement_groups import _PlacementGroupManager, get_tune_pg_prefix -from ray.tune.utils.trainable import TrainableUtil +from ray.tune.result import STDERR_FILE, STDOUT_FILE, TRIAL_INFO from ray.tune.trial import Trial, _Location, _TrialInfo from ray.tune.utils import warn_if_slow +from ray.tune.utils.placement_groups import _PlacementGroupManager, get_tune_pg_prefix from ray.tune.utils.resource_updater import _ResourceUpdater +from ray.tune.utils.trainable import TrainableUtil from ray.util import log_once from ray.util.annotations import DeveloperAPI -from ray.util.ml_utils.checkpoint_manager import _TrackedCheckpoint, CheckpointStorage -from ray.util.placement_group import remove_placement_group, PlacementGroup +from ray.util.ml_utils.checkpoint_manager import CheckpointStorage, _TrackedCheckpoint +from ray.util.placement_group import PlacementGroup, remove_placement_group logger = logging.getLogger(__name__) @@ -144,7 +136,7 @@ class _TrialCleanup: def noop_logger_creator(config, logdir): # Set the working dir in the remote process, for user file writes os.makedirs(logdir, exist_ok=True) - if not ray.worker._mode() == ray.worker.LOCAL_MODE: + if not ray._private.worker._mode() == ray._private.worker.LOCAL_MODE: os.chdir(logdir) return NoopLogger(config, logdir) @@ -854,7 +846,7 @@ class RayTrialExecutor: For non-local mode it is no-op. """ - if ray.worker._mode() == ray.worker.LOCAL_MODE: + if ray._private.worker._mode() == ray._private.worker.LOCAL_MODE: old_dir = os.getcwd() try: os.chdir(trial.logdir) diff --git a/python/ray/tune/registry.py b/python/ray/tune/registry.py index f75808f9a..a663ceba4 100644 --- a/python/ray/tune/registry.py +++ b/python/ray/tune/registry.py @@ -1,19 +1,17 @@ import logging import uuid from functools import partial - from types import FunctionType -from typing import Optional, Type, Union +from typing import Callable, Optional, Type, Union import ray import ray.cloudpickle as pickle from ray.experimental.internal_kv import ( - _internal_kv_initialized, _internal_kv_get, + _internal_kv_initialized, _internal_kv_put, ) from ray.tune.error import TuneError -from typing import Callable TRAINABLE_CLASS = "trainable_class" ENV_CREATOR = "env_creator" @@ -81,8 +79,8 @@ def register_trainable(name: str, trainable: Union[Callable, Type], warn: bool = automatically converted into a class during registration. """ - from ray.tune.trainable import Trainable from ray.tune.function_runner import wrap_function + from ray.tune.trainable import Trainable if isinstance(trainable, type): logger.debug("Detected class for trainable.") @@ -208,7 +206,7 @@ class _Registry: _global_registry = _Registry(prefix="global") -ray.worker._post_init_hooks.append(_global_registry.flush_values) +ray._private.worker._post_init_hooks.append(_global_registry.flush_values) class _ParameterRegistry: diff --git a/python/ray/tune/tests/test_integration_mlflow.py b/python/ray/tune/tests/test_integration_mlflow.py index 2c12788bb..ef50993e4 100644 --- a/python/ray/tune/tests/test_integration_mlflow.py +++ b/python/ray/tune/tests/test_integration_mlflow.py @@ -4,15 +4,15 @@ import unittest from collections import namedtuple from unittest.mock import patch +from mlflow.tracking import MlflowClient + from ray.tune.function_runner import wrap_function from ray.tune.integration.mlflow import ( MLflowLoggerCallback, - mlflow_mixin, MLflowTrainableMixin, + mlflow_mixin, ) - -from mlflow.tracking import MlflowClient -from ray.util.ml_utils.mlflow import MLflowLoggerUtil +from ray.util.ml_utils.mlflow import _MLflowLoggerUtil class MockTrial( @@ -25,7 +25,7 @@ class MockTrial( return self.trial_name -class MockMLflowLoggerUtil(MLflowLoggerUtil): +class Mock_MLflowLoggerUtil(_MLflowLoggerUtil): def save_artifacts(self, dir, run_id): self.artifact_saved = True self.artifact_info = {"dir": dir, "run_id": run_id} @@ -136,7 +136,7 @@ class MLflowTest(unittest.TestCase): logger.setup() self.assertEqual(logger.tags, tags) - @patch("ray.tune.integration.mlflow.MLflowLoggerUtil", MockMLflowLoggerUtil) + @patch("ray.tune.integration.mlflow._MLflowLoggerUtil", Mock_MLflowLoggerUtil) def testMlFlowLoggerLogging(self): clear_env_vars() trial_config = {"par1": "a", "par2": "b"} @@ -226,7 +226,8 @@ class MLflowTest(unittest.TestCase): if __name__ == "__main__": - import pytest import sys + import pytest + sys.exit(pytest.main(["-v", __file__])) diff --git a/python/ray/tune/trainable.py b/python/ray/tune/trainable.py index 4c7500c8d..ca3d03522 100644 --- a/python/ray/tune/trainable.py +++ b/python/ray/tune/trainable.py @@ -252,7 +252,7 @@ class Trainable: def is_actor(self): try: - actor_id = ray.worker.global_worker.actor_id + actor_id = ray._private.worker.global_worker.actor_id return actor_id != actor_id.nil() except Exception: # If global_worker is not instantiated, we're not in an actor diff --git a/python/ray/tune/tune.py b/python/ray/tune/tune.py index 2cf45f394..ba2620634 100644 --- a/python/ray/tune/tune.py +++ b/python/ray/tune/tune.py @@ -1,66 +1,54 @@ -import threading -from typing import Any, Callable, Dict, Mapping, Optional, Sequence, Type, Union - import datetime import logging import os import signal import sys +import threading import time import warnings +from typing import Any, Callable, Dict, Mapping, Optional, Sequence, Type, Union import ray -from ray.util.annotations import PublicAPI -from ray.util.ml_utils.node import force_on_current_node -from ray.util.queue import Queue, Empty - from ray.tune.analysis import ExperimentAnalysis from ray.tune.callback import Callback from ray.tune.error import TuneError from ray.tune.experiment import Experiment, convert_to_experiment_list from ray.tune.progress_reporter import ( + ProgressReporter, RemoteReporterMixin, detect_reporter, - ProgressReporter, ) from ray.tune.ray_trial_executor import RayTrialExecutor from ray.tune.registry import get_trainable_cls, is_function_trainable + +# Must come last to avoid circular imports from ray.tune.schedulers import ( + FIFOScheduler, PopulationBasedTraining, PopulationBasedTrainingReplay, ResourceChangingScheduler, + TrialScheduler, +) +from ray.tune.schedulers.util import ( + set_search_properties_backwards_compatible as scheduler_set_search_props, ) from ray.tune.stopper import Stopper from ray.tune.suggest import BasicVariantGenerator, SearchAlgorithm, SearchGenerator from ray.tune.suggest.suggestion import ConcurrencyLimiter, Searcher - -# Turn off black here, as it will format the lines to be longer than 88 chars -# fmt: off from ray.tune.suggest.util import ( - set_search_properties_backwards_compatible - as searcher_set_search_properties_backwards_compatible, + set_search_properties_backwards_compatible as searcher_set_search_props, ) -from ray.tune.schedulers.util import ( - set_search_properties_backwards_compatible - as scheduler_set_search_properties_backwards_compatible, -) -# fmt: on - from ray.tune.suggest.variant_generator import has_unresolved_values -from ray.tune.syncer import ( - SyncConfig, - _validate_upload_dir, - SyncerCallback, -) +from ray.tune.syncer import SyncConfig, SyncerCallback, _validate_upload_dir from ray.tune.trainable import Trainable from ray.tune.trial import Trial from ray.tune.trial_runner import TrialRunner from ray.tune.utils.callback import create_default_callbacks from ray.tune.utils.log import Verbosity, has_verbosity, set_verbosity - -# Must come last to avoid circular imports -from ray.tune.schedulers import FIFOScheduler, TrialScheduler from ray.tune.utils.placement_groups import PlacementGroupFactory +from ray.util.annotations import PublicAPI +from ray.util.ml_utils.node import force_on_current_node +from ray.util.queue import Empty, Queue logger = logging.getLogger(__name__) @@ -539,7 +527,7 @@ def run( search_alg = create_searcher(search_alg) # if local_mode=True is set during ray.init(). - is_local_mode = ray.worker._mode() == ray.worker.LOCAL_MODE + is_local_mode = ray._private.worker._mode() == ray._private.worker.LOCAL_MODE if is_local_mode: max_concurrent_trials = 1 @@ -585,7 +573,7 @@ def run( if isinstance(search_alg, Searcher): search_alg = SearchGenerator(search_alg) - if config and not searcher_set_search_properties_backwards_compatible( + if config and not searcher_set_search_props( search_alg.set_search_properties, metric, mode, @@ -601,7 +589,7 @@ def run( "them in the search algorithm's search space if necessary." ) - if not scheduler_set_search_properties_backwards_compatible( + if not scheduler_set_search_props( scheduler.set_search_properties, metric, mode, **experiments[0].public_spec ): raise ValueError( diff --git a/python/ray/tune/utils/resource_updater.py b/python/ray/tune/utils/resource_updater.py index 86c3c61a9..0b416b83c 100644 --- a/python/ray/tune/utils/resource_updater.py +++ b/python/ray/tune/utils/resource_updater.py @@ -1,10 +1,10 @@ -from typing import Optional, Dict, Any import logging import os import time +from typing import Any, Dict, Optional import ray -from ray import ray_constants +from ray._private import ray_constants from ray._private.resource_spec import NODE_ID_PREFIX from ray.tune.resources import Resources diff --git a/python/ray/tune/utils/trainable.py b/python/ray/tune/utils/trainable.py index 08800a54f..587940a6e 100644 --- a/python/ray/tune/utils/trainable.py +++ b/python/ray/tune/utils/trainable.py @@ -3,9 +3,11 @@ import inspect import io import logging import os -import pandas as pd import shutil -from typing import Any, Dict, Union, Optional +from typing import Any, Dict, Optional, Union + +import pandas as pd +from six import string_types import ray import ray.cloudpickle as pickle @@ -13,7 +15,6 @@ from ray.tune.registry import _ParameterRegistry from ray.tune.utils import detect_checkpoint_function from ray.util import placement_group from ray.util.annotations import DeveloperAPI -from six import string_types logger = logging.getLogger(__name__) @@ -372,7 +373,7 @@ def with_parameters(trainable, **kwargs): ) parameter_registry = _ParameterRegistry() - ray.worker._post_init_hooks.append(parameter_registry.flush) + ray._private.worker._post_init_hooks.append(parameter_registry.flush) # Objects are moved into the object store prefix = f"{str(trainable)}_" diff --git a/python/ray/types.py b/python/ray/types.py index d4b6d0d15..79110f542 100644 --- a/python/ray/types.py +++ b/python/ray/types.py @@ -1,9 +1,12 @@ from typing import Generic, TypeVar +from ray.util.annotations import PublicAPI + T = TypeVar("T") # TODO(ekl) this is a dummy generic ref type for documentation purposes only. # We should try to make the Cython ray.ObjectRef properly generic. +@PublicAPI class ObjectRef(Generic[T]): pass diff --git a/python/ray/util/__init__.py b/python/ray/util/__init__.py index 2ccb9b4cf..15c0101c5 100644 --- a/python/ray/util/__init__.py +++ b/python/ray/util/__init__.py @@ -1,24 +1,23 @@ from typing import List import ray +from ray._private.client_mode_hook import client_mode_hook from ray._private.services import get_node_ip_address from ray.util import iter -from ray.util.annotations import PublicAPI +from ray.util import rpdb as pdb from ray.util.actor_pool import ActorPool +from ray.util.annotations import PublicAPI from ray.util.check_serialize import inspect_serializability -from ray.util.debug import log_once, disable_log_once_globally, enable_periodic_logging +from ray.util.client_connect import connect, disconnect +from ray.util.debug import disable_log_once_globally, enable_periodic_logging, log_once from ray.util.placement_group import ( + get_current_placement_group, + get_placement_group, placement_group, placement_group_table, remove_placement_group, - get_placement_group, - get_current_placement_group, ) -from ray.util import rpdb as pdb -from ray.util.serialization import register_serializer, deregister_serializer - -from ray.util.client_connect import connect, disconnect -from ray._private.client_mode_hook import client_mode_hook +from ray.util.serialization import deregister_serializer, register_serializer @PublicAPI(stability="beta") @@ -36,7 +35,7 @@ def list_named_actors(all_namespaces: bool = False) -> List[str]: returned regardless of namespace, and the returned entries will be of the form {"namespace": namespace, "name": name}. """ - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker.check_connected() actors = worker.core_worker.list_named_actors(all_namespaces) diff --git a/python/ray/util/check_serialize.py b/python/ray/util/check_serialize.py index 4536eff9e..c1c2ca7d1 100644 --- a/python/ray/util/check_serialize.py +++ b/python/ray/util/check_serialize.py @@ -1,15 +1,15 @@ """A utility for debugging serialization issues.""" -from typing import Any, Tuple, Set, Optional import inspect -import ray.cloudpickle as cp from contextlib import contextmanager +from typing import Any, Optional, Set, Tuple + +import colorama # Import ray first to use the bundled colorama import ray # noqa: F401 +import ray.cloudpickle as cp from ray.util.annotations import DeveloperAPI -import colorama - @contextmanager def _indent(printer): @@ -33,6 +33,7 @@ class _Printer: _printer = _Printer() +@DeveloperAPI class FailureTuple: """Represents the serialization 'frame'. diff --git a/python/ray/util/client/__init__.py b/python/ray/util/client/__init__.py index 413cb85f9..41018cdcd 100644 --- a/python/ray/util/client/__init__.py +++ b/python/ray/util/client/__init__.py @@ -1,16 +1,19 @@ -from typing import List, Tuple, Dict, Any, Optional -from ray.job_config import JobConfig +import logging +import os +import sys +import threading +from typing import Any, Dict, List, Optional, Tuple + +import grpc + +import ray._private.ray_constants as ray_constants from ray._private.client_mode_hook import ( _explicitly_disable_client_mode, _explicitly_enable_client_mode, ) -import os -import sys -import logging -import threading -import grpc -import ray.ray_constants as ray_constants from ray._private.ray_logging import setup_logger +from ray.job_config import JobConfig +from ray.util.annotations import DeveloperAPI logger = logging.getLogger(__name__) @@ -21,9 +24,9 @@ CURRENT_PROTOCOL_VERSION = "2022-05-13" class _ClientContext: def __init__(self): - from ray.util.client.api import ClientAPI + from ray.util.client.api import _ClientAPI - self.api = ClientAPI() + self.api = _ClientAPI() self.client_worker = None self._server = None self._connected_with_init = False @@ -106,11 +109,11 @@ class _ClientContext: The server side should have already registered the serializers via regular worker's serialization_context mechanism. """ - import ray.serialization_addons + import ray.util.serialization_addons from ray.util.serialization import StandaloneSerializationContext ctx = StandaloneSerializationContext() - ray.serialization_addons.apply(ctx) + ray.util.serialization_addons.apply(ctx) def _check_versions(self, conn_info: Dict[str, Any], ignore_version: bool) -> None: local_major_minor = f"{sys.version_info[0]}.{sys.version_info[1]}" @@ -138,11 +141,11 @@ class _ClientContext: def disconnect(self): """Disconnect the Ray Client.""" - from ray.util.client.api import ClientAPI + from ray.util.client.api import _ClientAPI if self.client_worker is not None: self.client_worker.close() - self.api = ClientAPI() + self.api = _ClientAPI() self.client_worker = None # remote can be called outside of a connection, which is why it @@ -207,6 +210,7 @@ _lock = threading.Lock() _default_context = _ClientContext() +@DeveloperAPI class RayAPIStub: """This class stands in as the replacement API for the `import ray` module. @@ -294,6 +298,7 @@ class RayAPIStub: ray = RayAPIStub() +@DeveloperAPI def num_connected_contexts(): """Return the number of client connections active.""" global _lock, _all_contexts diff --git a/python/ray/util/client/api.py b/python/ray/util/client/api.py index 5c9f9dbf5..9e6864c22 100644 --- a/python/ray/util/client/api.py +++ b/python/ray/util/client/api.py @@ -1,32 +1,30 @@ """This file defines the interface between the ray client worker and the overall ray module API. """ -from concurrent.futures import Future import json import logging +from concurrent.futures import Future +from typing import TYPE_CHECKING, Any, Callable, List, Optional -from ray.util.client.runtime_context import ClientWorkerPropertyAPI from ray._private import ray_option_utils -from typing import Any, Callable, List, Optional, TYPE_CHECKING +from ray.util.client.runtime_context import _ClientWorkerPropertyAPI if TYPE_CHECKING: from ray.actor import ActorClass - from ray.remote_function import RemoteFunction - from ray.util.client.common import ClientStub - from ray.util.client.common import ClientActorHandle - from ray.util.client.common import ClientObjectRef from ray.core.generated.ray_client_pb2 import DataResponse + from ray.remote_function import RemoteFunction + from ray.util.client.common import ClientActorHandle, ClientObjectRef, ClientStub logger = logging.getLogger(__name__) -def as_bytes(value): +def _as_bytes(value): if isinstance(value, str): return value.encode("utf-8") return value -class ClientAPI: +class _ClientAPI: """The Client-side methods corresponding to the ray API. Delegates to the Client Worker that contains the connection to the ClientServer. """ @@ -285,7 +283,7 @@ class ClientAPI: Returns: A RuntimeContext wrapping a client making get_cluster_info calls. """ - return ClientWorkerPropertyAPI(self.worker).build_runtime_context() + return _ClientWorkerPropertyAPI(self.worker).build_runtime_context() # Client process isn't assigned any GPUs. def get_gpu_ids(self) -> list: @@ -316,25 +314,25 @@ class ClientAPI: def _internal_kv_exists(self, key: bytes) -> bool: """Hook for internal_kv._internal_kv_exists.""" - return self.worker.internal_kv_exists(as_bytes(key)) + return self.worker.internal_kv_exists(_as_bytes(key)) def _internal_kv_get(self, key: bytes) -> bytes: """Hook for internal_kv._internal_kv_get.""" - return self.worker.internal_kv_get(as_bytes(key)) + return self.worker.internal_kv_get(_as_bytes(key)) def _internal_kv_put( self, key: bytes, value: bytes, overwrite: bool = False ) -> bool: """Hook for internal_kv._internal_kv_put.""" - return self.worker.internal_kv_put(as_bytes(key), as_bytes(value), overwrite) + return self.worker.internal_kv_put(_as_bytes(key), _as_bytes(value), overwrite) def _internal_kv_del(self, key: bytes) -> None: """Hook for internal_kv._internal_kv_del.""" - return self.worker.internal_kv_del(as_bytes(key)) + return self.worker.internal_kv_del(_as_bytes(key)) def _internal_kv_list(self, prefix: bytes) -> bytes: """Hook for internal_kv._internal_kv_list.""" - return self.worker.internal_kv_list(as_bytes(prefix)) + return self.worker.internal_kv_list(_as_bytes(prefix)) def _pin_runtime_env_uri(self, uri: str, expiration_s: int) -> None: """Hook for internal_kv._pin_runtime_env_uri.""" diff --git a/python/ray/util/client/common.py b/python/ray/util/client/common.py index 226853644..4f1112baa 100644 --- a/python/ray/util/client/common.py +++ b/python/ray/util/client/common.py @@ -1,34 +1,29 @@ +import inspect +import logging +import os +import pickle +import threading +import uuid +from collections import OrderedDict +from concurrent.futures import Future +from dataclasses import dataclass +from typing import Any, Callable, Dict, List, Optional, Tuple, Union + +import grpc + import ray._raylet as raylet import ray.core.generated.ray_client_pb2 as ray_client_pb2 import ray.core.generated.ray_client_pb2_grpc as ray_client_pb2_grpc -from ray.util.client import ray -from ray.util.client.options import validate_options -from ray._private.signature import get_signature, extract_signature -from ray._private.utils import check_oversized_function - -from concurrent.futures import Future -from dataclasses import dataclass -import grpc -import os -import uuid -import inspect -import pickle -from ray.util.inspect import ( - is_cython, +from ray._private.inspect_util import ( is_class_method, + is_cython, is_function_or_method, is_static_method, ) -import logging -import threading -from collections import OrderedDict -from typing import Any -from typing import List -from typing import Dict -from typing import Optional -from typing import Tuple -from typing import Union -from typing import Callable +from ray._private.signature import extract_signature, get_signature +from ray._private.utils import check_oversized_function +from ray.util.client import ray +from ray.util.client.options import validate_options logger = logging.getLogger(__name__) diff --git a/python/ray/util/client/runtime_context.py b/python/ray/util/client/runtime_context.py index 65e59a28c..054cd16ea 100644 --- a/python/ray/util/client/runtime_context.py +++ b/python/ray/util/client/runtime_context.py @@ -1,13 +1,12 @@ from typing import TYPE_CHECKING if TYPE_CHECKING: + from ray import JobID, NodeID from ray.runtime_context import RuntimeContext - from ray import JobID - from ray import NodeID -class ClientWorkerPropertyAPI: - """Emulates the properties of the ray.worker object for the client""" +class _ClientWorkerPropertyAPI: + """Emulates the properties of the ray._private.worker object for the client""" def __init__(self, worker): assert worker is not None @@ -29,7 +28,7 @@ class ClientWorkerPropertyAPI: @property def mode(self): - from ray.worker import SCRIPT_MODE + from ray._private.worker import SCRIPT_MODE return SCRIPT_MODE diff --git a/python/ray/util/client/server/logservicer.py b/python/ray/util/client/server/logservicer.py index 46a581b23..764e6c82c 100644 --- a/python/ray/util/client/server/logservicer.py +++ b/python/ray/util/client/server/logservicer.py @@ -2,17 +2,18 @@ with its handler. """ import io -import threading -import queue import logging -import grpc +import queue +import threading import uuid -from ray.worker import print_worker_logs -from ray.util.client.common import CLIENT_SERVER_MAX_THREADS -from ray._private.ray_logging import global_worker_stdstream_dispatcher +import grpc + import ray.core.generated.ray_client_pb2 as ray_client_pb2 import ray.core.generated.ray_client_pb2_grpc as ray_client_pb2_grpc +from ray._private.ray_logging import global_worker_stdstream_dispatcher +from ray._private.worker import print_worker_logs +from ray.util.client.common import CLIENT_SERVER_MAX_THREADS logger = logging.getLogger(__name__) diff --git a/python/ray/util/client/server/proxier.py b/python/ray/util/client/server/proxier.py index b928a8755..691271708 100644 --- a/python/ray/util/client/server/proxier.py +++ b/python/ray/util/client/server/proxier.py @@ -1,43 +1,44 @@ import atexit -from concurrent import futures -from dataclasses import dataclass -import grpc -import logging -from itertools import chain import json +import logging import socket import sys -from threading import Event, Lock, Thread, RLock import time import traceback +from concurrent import futures +from dataclasses import dataclass +from itertools import chain +from threading import Event, Lock, RLock, Thread from typing import Callable, Dict, List, Optional, Tuple +import grpc + +# Import psutil after ray so the packaged version is used. +import psutil + import ray -from ray.cloudpickle.compat import pickle -from ray.job_config import JobConfig import ray.core.generated.agent_manager_pb2 as agent_manager_pb2 import ray.core.generated.ray_client_pb2 as ray_client_pb2 import ray.core.generated.ray_client_pb2_grpc as ray_client_pb2_grpc import ray.core.generated.runtime_env_agent_pb2 as runtime_env_agent_pb2 import ray.core.generated.runtime_env_agent_pb2_grpc as runtime_env_agent_pb2_grpc # noqa: E501 -from ray.util.client.common import ( - _get_client_id_from_context, - ClientServerHandle, - CLIENT_SERVER_MAX_THREADS, - GRPC_OPTIONS, - _propagate_error_in_context, -) -from ray.util.client.server.dataservicer import _get_reconnecting_from_context from ray._private.client_mode_hook import disable_client_hook +from ray._private.gcs_utils import GcsClient from ray._private.parameter import RayParams from ray._private.runtime_env.context import RuntimeEnvContext from ray._private.services import ProcessInfo, start_ray_client_server from ray._private.tls_utils import add_port_to_grpc_server -from ray._private.gcs_utils import GcsClient from ray._private.utils import detect_fate_sharing_support - -# Import psutil after ray so the packaged version is used. -import psutil +from ray.cloudpickle.compat import pickle +from ray.job_config import JobConfig +from ray.util.client.common import ( + CLIENT_SERVER_MAX_THREADS, + GRPC_OPTIONS, + ClientServerHandle, + _get_client_id_from_context, + _propagate_error_in_context, +) +from ray.util.client.server.dataservicer import _get_reconnecting_from_context logger = logging.getLogger(__name__) @@ -139,7 +140,7 @@ class ProxyManager: self._check_thread.start() self.fate_share = bool(detect_fate_sharing_support()) - self._node: Optional[ray.node.Node] = None + self._node: Optional[ray._private.node.Node] = None atexit.register(self._cleanup) def _get_unused_port(self) -> int: @@ -175,7 +176,7 @@ class ProxyManager: return self._address @property - def node(self) -> ray.node.Node: + def node(self) -> ray._private.node.Node: """Gets a 'ray.Node' object for this node (the head node). If it does not already exist, one is created using the bootstrap address. @@ -184,7 +185,7 @@ class ProxyManager: return self._node ray_params = RayParams(gcs_address=self.address) - self._node = ray.node.Node( + self._node = ray._private.node.Node( ray_params, head=False, shutdown_at_exit=False, diff --git a/python/ray/util/client/server/server.py b/python/ray/util/client/server/server.py index 3d32b040e..62c14ac47 100644 --- a/python/ray/util/client/server/server.py +++ b/python/ray/util/client/server/server.py @@ -1,50 +1,45 @@ -import logging -from concurrent import futures -import gc -import grpc import base64 -from collections import defaultdict import functools -import math -import queue -import pickle - -import threading -from typing import Any, List -from typing import Dict -from typing import Set -from typing import Optional -from typing import Callable -from typing import Union -from ray import cloudpickle -from ray.job_config import JobConfig -import ray -import ray.state -import ray.core.generated.ray_client_pb2 as ray_client_pb2 -import ray.core.generated.ray_client_pb2_grpc as ray_client_pb2_grpc -import time +import gc import inspect import json -from ray.util.client.common import ( - ClientServerHandle, - GRPC_OPTIONS, - CLIENT_SERVER_MAX_THREADS, - OBJECT_TRANSFER_CHUNK_SIZE, - ResponseCache, -) -from ray import ray_constants -from ray.util.client.server.proxier import serve_proxier -from ray.util.client.server.server_pickler import dumps_from_server -from ray.util.client.server.server_pickler import loads_from_client -from ray.util.client.server.dataservicer import DataServicer -from ray.util.client.server.logservicer import LogstreamServicer -from ray.util.client.server.server_stubs import current_server -from ray.ray_constants import env_integer +import logging +import math +import pickle +import queue +import threading +import time +from collections import defaultdict +from concurrent import futures +from typing import Any, Callable, Dict, List, Optional, Set, Union + +import grpc + +import ray +import ray._private.state +import ray.core.generated.ray_client_pb2 as ray_client_pb2 +import ray.core.generated.ray_client_pb2_grpc as ray_client_pb2_grpc +from ray import cloudpickle +from ray._private import ray_constants from ray._private.client_mode_hook import disable_client_hook +from ray._private.gcs_utils import GcsClient +from ray._private.ray_constants import env_integer from ray._private.ray_logging import setup_logger from ray._private.services import canonicalize_bootstrap_address from ray._private.tls_utils import add_port_to_grpc_server -from ray._private.gcs_utils import GcsClient +from ray.job_config import JobConfig +from ray.util.client.common import ( + CLIENT_SERVER_MAX_THREADS, + GRPC_OPTIONS, + OBJECT_TRANSFER_CHUNK_SIZE, + ClientServerHandle, + ResponseCache, +) +from ray.util.client.server.dataservicer import DataServicer +from ray.util.client.server.logservicer import LogstreamServicer +from ray.util.client.server.proxier import serve_proxier +from ray.util.client.server.server_pickler import dumps_from_server, loads_from_client +from ray.util.client.server.server_stubs import current_server logger = logging.getLogger(__name__) @@ -129,7 +124,7 @@ class RayletServicer(ray_client_pb2_grpc.RayletDriverServicer): current_job_config = None with disable_client_hook(): if ray.is_initialized(): - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker current_job_config = worker.core_worker.get_job_config() else: extra_kwargs = json.loads(request.ray_init_kwargs or "{}") diff --git a/python/ray/util/client/worker.py b/python/ray/util/client/worker.py index cd458cb30..baad477e1 100644 --- a/python/ray/util/client/worker.py +++ b/python/ray/util/client/worker.py @@ -6,48 +6,45 @@ import base64 import json import logging import os +import tempfile import threading import time import uuid import warnings from collections import defaultdict from concurrent.futures import Future -import tempfile -from typing import Any, Callable, Dict, List, Optional, Tuple, TYPE_CHECKING, Union +from typing import TYPE_CHECKING, Any, Callable, Dict, List, Optional, Tuple, Union import grpc -from ray.job_config import JobConfig +import ray._private.utils import ray.cloudpickle as cloudpickle +import ray.core.generated.ray_client_pb2 as ray_client_pb2 +import ray.core.generated.ray_client_pb2_grpc as ray_client_pb2_grpc +from ray._private.ray_constants import DEFAULT_CLIENT_RECONNECT_GRACE_PERIOD +from ray._private.runtime_env.py_modules import upload_py_modules_if_needed +from ray._private.runtime_env.working_dir import upload_working_dir_if_needed # Use cloudpickle's version of pickle for UnpicklingError from ray.cloudpickle.compat import pickle -import ray.core.generated.ray_client_pb2 as ray_client_pb2 -import ray.core.generated.ray_client_pb2_grpc as ray_client_pb2_grpc from ray.exceptions import GetTimeoutError -from ray.ray_constants import DEFAULT_CLIENT_RECONNECT_GRACE_PERIOD -from ray.util.client.client_pickler import ( - dumps_from_client, - loads_from_server, -) +from ray.job_config import JobConfig +from ray.util.client.client_pickler import dumps_from_client, loads_from_server from ray.util.client.common import ( + GRPC_OPTIONS, + GRPC_UNRECOVERABLE_ERRORS, + INT32_MAX, + OBJECT_TRANSFER_WARNING_SIZE, ClientActorClass, ClientActorHandle, ClientActorRef, ClientObjectRef, ClientRemoteFunc, ClientStub, - GRPC_OPTIONS, - GRPC_UNRECOVERABLE_ERRORS, - INT32_MAX, - OBJECT_TRANSFER_WARNING_SIZE, ) from ray.util.client.dataclient import DataClient from ray.util.client.logsclient import LogstreamClient from ray.util.debug import log_once -import ray._private.utils -from ray._private.runtime_env.py_modules import upload_py_modules_if_needed -from ray._private.runtime_env.working_dir import upload_working_dir_if_needed if TYPE_CHECKING: from ray.actor import ActorClass diff --git a/python/ray/util/client_connect.py b/python/ray/util/client_connect.py index ea4963c23..a800611d6 100644 --- a/python/ray/util/client_connect.py +++ b/python/ray/util/client_connect.py @@ -1,13 +1,17 @@ -from ray.util.client import ray -from ray.job_config import JobConfig -from ray._private.client_mode_hook import _set_client_hook_status -from ray._private.client_mode_hook import _explicitly_enable_client_mode - -from typing import List, Tuple, Dict, Any, Optional +from typing import Any, Dict, List, Optional, Tuple import grpc +from ray._private.client_mode_hook import ( + _explicitly_enable_client_mode, + _set_client_hook_status, +) +from ray.job_config import JobConfig +from ray.util.annotations import Deprecated +from ray.util.client import ray + +@Deprecated def connect( conn_str: str, secure: bool = False, @@ -47,6 +51,7 @@ def connect( return conn +@Deprecated def disconnect(): """Disconnects from server; is idempotent.""" return ray.disconnect() diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index 4f46601bc..9399cdb88 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -4,6 +4,7 @@ import os from typing import List import numpy as np + import ray from ray.util.collective import types @@ -679,7 +680,7 @@ def _check_and_get_group(group_name): name = "info_" + group_name mgr = ray.get_actor(name=name) ids, world_size, rank, backend = ray.get(mgr.get_info.remote()) - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker id_ = worker.core_worker.get_actor_id() r = rank[ids.index(id_)] _group_mgr.create_collective_group(backend, world_size, r, group_name) @@ -732,7 +733,7 @@ def _check_backend_availability(backend: types.Backend): def _check_inside_actor(): """Check if currently it is inside a Ray actor/task.""" - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker if worker.mode == ray.WORKER_MODE: return else: diff --git a/python/ray/util/collective/collective_group/gloo_collective_group.py b/python/ray/util/collective/collective_group/gloo_collective_group.py index 3dba50024..309f59438 100644 --- a/python/ray/util/collective/collective_group/gloo_collective_group.py +++ b/python/ray/util/collective/collective_group/gloo_collective_group.py @@ -1,28 +1,28 @@ -import logging import datetime -import time +import logging import os import shutil +import time + +import numpy +import pygloo import ray -from ray import ray_constants -import pygloo -import numpy - +from ray._private import ray_constants from ray.util.collective.collective_group import gloo_util from ray.util.collective.collective_group.base_collective_group import BaseGroup +from ray.util.collective.const import get_store_name from ray.util.collective.types import ( - AllReduceOptions, - BarrierOptions, - Backend, - ReduceOptions, - BroadcastOptions, AllGatherOptions, + AllReduceOptions, + Backend, + BarrierOptions, + BroadcastOptions, + RecvOptions, + ReduceOptions, ReduceScatterOptions, SendOptions, - RecvOptions, ) -from ray.util.collective.const import get_store_name logger = logging.getLogger(__name__) @@ -43,7 +43,7 @@ class Rendezvous: def __init__(self, group_name, context, store_type, device_type): self._group_name = group_name self._context = context - redis_address = ray.worker._global_node.redis_address + redis_address = ray._private.worker._global_node.redis_address (self._redis_ip_address, self._redis_port) = ( redis_address.split(":") if store_type == "redis" else (None, None) ) @@ -68,7 +68,7 @@ class Rendezvous: redisStore = pygloo.rendezvous.RedisStore( self._redis_ip_address, int(self._redis_port) ) - redis_password = ray.worker._global_node.redis_password + redis_password = ray._private.worker._global_node.redis_password if redis_password is None or len(redis_password) == 0: redis_password = ray_constants.REDIS_DEFAULT_PASSWORD redisStore.authorize(redis_password) diff --git a/python/ray/util/collective/collective_group/gloo_util.py b/python/ray/util/collective/collective_group/gloo_util.py index 23fe4a58a..cb2715213 100644 --- a/python/ray/util/collective/collective_group/gloo_util.py +++ b/python/ray/util/collective/collective_group/gloo_util.py @@ -1,6 +1,14 @@ """Code to wrap some GLOO API calls.""" -import numpy import asyncio +import time + +import numpy + +import ray +import ray.experimental.internal_kv as internal_kv +from ray._private.gcs_utils import GcsClient +from ray.util.collective.types import ReduceOp, torch_available +from ray.util.queue import _QueueActor try: import pygloo @@ -9,14 +17,6 @@ except ImportError: "Can not import pygloo. Please run 'pip install pygloo' to install pygloo." ) -import time - -import ray -from ray.util.collective.types import ReduceOp, torch_available -from ray.util.queue import _QueueActor - -import ray.experimental.internal_kv as internal_kv -from ray._private.gcs_utils import GcsClient GLOO_REDUCE_OP_MAP = { ReduceOp.SUM: pygloo.ReduceOp.SUM, @@ -271,7 +271,7 @@ class RayInternalKvStore: def __init__(self, group_name: str): self._group_name = group_name self._job_id = ray.get_runtime_context().job_id - gcs_address = ray.worker._global_node.gcs_address + gcs_address = ray._private.worker._global_node.gcs_address self._gcs_client = GcsClient(address=gcs_address, nums_reconnect_retry=10) internal_kv._initialize_internal_kv(self._gcs_client) diff --git a/python/ray/util/dask/tests/test_dask_scheduler.py b/python/ray/util/dask/tests/test_dask_scheduler.py index d6422f2ce..edbf86d7b 100644 --- a/python/ray/util/dask/tests/test_dask_scheduler.py +++ b/python/ray/util/dask/tests/test_dask_scheduler.py @@ -4,15 +4,15 @@ import unittest import dask import dask.array as da import dask.dataframe as dd -import pytest import numpy as np import pandas as pd +import pytest import ray -from ray.util.client.common import ClientObjectRef -from ray.util.dask.callbacks import ProgressBarCallback -from ray.util.dask import ray_dask_get, enable_dask_on_ray, disable_dask_on_ray from ray.tests.conftest import * # noqa: F403, F401 +from ray.util.client.common import ClientObjectRef +from ray.util.dask import disable_dask_on_ray, enable_dask_on_ray, ray_dask_get +from ray.util.dask.callbacks import ProgressBarCallback @pytest.fixture @@ -80,7 +80,7 @@ def test_ray_dask_resources(ray_start_cluster, ray_enable_dask_on_ray): ray.init(address=cluster.address) def get_node_id(): - return ray.worker.global_worker.node.unique_id + return ray._private.worker.global_worker.node.unique_id # Test annotations on collection. with dask.annotate(ray_remote_args=dict(num_cpus=1, resources={"pin": 0.01})): diff --git a/python/ray/util/debug.py b/python/ray/util/debug.py index 189a8d808..2238783af 100644 --- a/python/ray/util/debug.py +++ b/python/ray/util/debug.py @@ -1,11 +1,14 @@ import time +from ray.util.annotations import DeveloperAPI + _logged = set() _disabled = False _periodic_log = False _last_logged = 0.0 +@DeveloperAPI def log_once(key): """Returns True if this is the "first" call for a given key. @@ -32,6 +35,7 @@ def log_once(key): return False +@DeveloperAPI def disable_log_once_globally(): """Make log_once() return False in this process.""" @@ -39,6 +43,7 @@ def disable_log_once_globally(): _disabled = True +@DeveloperAPI def enable_periodic_logging(): """Make log_once() periodically return True in this process.""" @@ -46,6 +51,7 @@ def enable_periodic_logging(): _periodic_log = True +@DeveloperAPI def reset_log_once(key): """Resets log_once for the provided key.""" diff --git a/python/ray/util/iter.py b/python/ray/util/iter.py index 2cb6f06c4..0e3502f1d 100644 --- a/python/ray/util/iter.py +++ b/python/ray/util/iter.py @@ -1,11 +1,12 @@ -from contextlib import contextmanager import collections import random import threading import time -from typing import TypeVar, Generic, Iterable, List, Callable, Any +from contextlib import contextmanager +from typing import Any, Callable, Generic, Iterable, List, TypeVar import ray +from ray.util.annotations import Deprecated from ray.util.iter_metrics import MetricsContext, SharedMetrics # The type of an iterator element. @@ -13,6 +14,7 @@ T = TypeVar("T") U = TypeVar("U") +@Deprecated def from_items( items: List[T], num_shards: int = 2, repeat: bool = False ) -> "ParallelIterator[T]": @@ -37,6 +39,7 @@ def from_items( return from_iterators(shards, repeat=repeat, name=name) +@Deprecated def from_range( n: int, num_shards: int = 2, repeat: bool = False ) -> "ParallelIterator[int]": @@ -68,6 +71,7 @@ def from_range( ) +@Deprecated def from_iterators( generators: List[Iterable[T]], repeat: bool = False, name=None ) -> "ParallelIterator[T]": @@ -106,6 +110,7 @@ def from_iterators( return from_actors(actors, name=name) +@Deprecated def from_actors( actors: List["ray.actor.ActorHandle"], name=None ) -> "ParallelIterator[T]": @@ -123,6 +128,7 @@ def from_actors( return ParallelIterator([_ActorSet(actors, [])], name, parent_iterators=[]) +@Deprecated class ParallelIterator(Generic[T]): """A parallel iterator over a set of remote actors. @@ -695,6 +701,7 @@ class ParallelIterator(Generic[T]): return LocalIterator(base_iterator, SharedMetrics(), name=name) +@Deprecated class LocalIterator(Generic[T]): """An iterator over a single shard of data. @@ -1125,6 +1132,7 @@ class LocalIterator(Generic[T]): ) +@Deprecated class ParallelIteratorWorker(object): """Worker actor for a ParallelIterator. diff --git a/python/ray/util/iter_metrics.py b/python/ray/util/iter_metrics.py index ccdf689f6..eb06a97c3 100644 --- a/python/ray/util/iter_metrics.py +++ b/python/ray/util/iter_metrics.py @@ -1,9 +1,11 @@ import collections from typing import List +from ray.util.annotations import Deprecated from ray.util.timer import _Timer +@Deprecated class MetricsContext: """Metrics context object for a local iterator. @@ -43,6 +45,7 @@ class MetricsContext: self.info = values["info"] +@Deprecated class SharedMetrics: """Holds an indirect reference to a (shared) metrics context. diff --git a/python/ray/util/joblib/ray_backend.py b/python/ray/util/joblib/ray_backend.py index d9dfce4b3..77b3882ad 100644 --- a/python/ray/util/joblib/ray_backend.py +++ b/python/ray/util/joblib/ray_backend.py @@ -1,11 +1,12 @@ +import logging from typing import Any, Dict, Optional + from joblib import Parallel from joblib._parallel_backends import MultiprocessingBackend from joblib.pool import PicklingPool -import logging -from ray.util.multiprocessing.pool import Pool import ray +from ray.util.multiprocessing.pool import Pool logger = logging.getLogger(__name__) @@ -69,7 +70,7 @@ class RayBackend(MultiprocessingBackend): else: logger.info("Starting local ray cluster") ray.init() - ray_cpus = int(ray.state.cluster_resources()["CPU"]) + ray_cpus = int(ray._private.state.cluster_resources()["CPU"]) n_jobs = ray_cpus eff_n_jobs = super(RayBackend, self).configure( @@ -87,6 +88,6 @@ class RayBackend(MultiprocessingBackend): def effective_n_jobs(self, n_jobs): eff_n_jobs = super(RayBackend, self).effective_n_jobs(n_jobs) if n_jobs == -1: - ray_cpus = int(ray.state.cluster_resources()["CPU"]) + ray_cpus = int(ray._private.state.cluster_resources()["CPU"]) eff_n_jobs = ray_cpus return eff_n_jobs diff --git a/python/ray/util/ml_utils/dict.py b/python/ray/util/ml_utils/dict.py index 1dc09e60b..3d102b329 100644 --- a/python/ray/util/ml_utils/dict.py +++ b/python/ray/util/ml_utils/dict.py @@ -1,11 +1,14 @@ -from typing import Dict, List, Union, Optional, TypeVar import copy from collections import deque from collections.abc import Mapping, Sequence +from typing import Dict, List, Optional, TypeVar, Union + +from ray.util.annotations import Deprecated T = TypeVar("T") +@Deprecated def merge_dicts(d1: dict, d2: dict) -> dict: """ Args: @@ -20,6 +23,7 @@ def merge_dicts(d1: dict, d2: dict) -> dict: return merged +@Deprecated def deep_update( original: dict, new_dict: dict, @@ -92,6 +96,7 @@ def deep_update( return original +@Deprecated def flatten_dict( dt: Dict, delimiter: str = "/", @@ -144,6 +149,7 @@ def flatten_dict( return dt +@Deprecated def unflatten_dict(dt: Dict[str, T], delimiter: str = "/") -> Dict[str, T]: """Unflatten dict. Does not support unflattening lists.""" dict_type = type(dt) @@ -164,6 +170,7 @@ def unflatten_dict(dt: Dict[str, T], delimiter: str = "/") -> Dict[str, T]: return out +@Deprecated def unflatten_list_dict(dt: Dict[str, T], delimiter: str = "/") -> Dict[str, T]: """Unflatten nested dict and list. @@ -212,6 +219,7 @@ def unflatten_list_dict(dt: Dict[str, T], delimiter: str = "/") -> Dict[str, T]: return out +@Deprecated def unflattened_lookup( flat_key: str, lookup: Union[Mapping, Sequence], delimiter: str = "/", **kwargs ) -> Union[Mapping, Sequence]: diff --git a/python/ray/util/ml_utils/filelock.py b/python/ray/util/ml_utils/filelock.py index ca0772827..ca05dd2cf 100644 --- a/python/ray/util/ml_utils/filelock.py +++ b/python/ray/util/ml_utils/filelock.py @@ -1,12 +1,16 @@ -from filelock import FileLock -from pathlib import Path import hashlib import os import tempfile +from pathlib import Path + +from filelock import FileLock + +from ray.util.annotations import Deprecated RAY_LOCKFILE_DIR = "_ray_lockfiles" +@Deprecated class TempFileLock: """FileLock wrapper that uses temporary file locks.""" diff --git a/python/ray/util/ml_utils/json.py b/python/ray/util/ml_utils/json.py index 9f84a7a55..a27a10816 100644 --- a/python/ray/util/ml_utils/json.py +++ b/python/ray/util/ml_utils/json.py @@ -1,8 +1,12 @@ import json -import numpy as np import numbers +import numpy as np +from ray.util.annotations import Deprecated + + +@Deprecated class SafeFallbackEncoder(json.JSONEncoder): def __init__(self, nan_str="null", **kwargs): super(SafeFallbackEncoder, self).__init__(**kwargs) diff --git a/python/ray/util/ml_utils/mlflow.py b/python/ray/util/ml_utils/mlflow.py index 4debac019..71bb84674 100644 --- a/python/ray/util/ml_utils/mlflow.py +++ b/python/ray/util/ml_utils/mlflow.py @@ -1,7 +1,7 @@ -from copy import deepcopy -import os import logging -from typing import Dict, Optional, TYPE_CHECKING +import os +from copy import deepcopy +from typing import TYPE_CHECKING, Dict, Optional if TYPE_CHECKING: from mlflow.entities import Run @@ -10,7 +10,7 @@ if TYPE_CHECKING: logger = logging.getLogger(__name__) -class MLflowLoggerUtil: +class _MLflowLoggerUtil: """Util class for setting up and logging to MLflow. Use this util for any library that needs MLflow logging/tracking logic @@ -28,7 +28,7 @@ class MLflowLoggerUtil: _mlflow = self._mlflow self.__dict__.pop("_mlflow") dict_copy = deepcopy(self.__dict__, memo) - copied_object = MLflowLoggerUtil() + copied_object = _MLflowLoggerUtil() copied_object.__dict__.update(dict_copy) self._mlflow = _mlflow copied_object._mlflow = _mlflow diff --git a/python/ray/util/ml_utils/node.py b/python/ray/util/ml_utils/node.py index 3490445e9..ef8e60f38 100644 --- a/python/ray/util/ml_utils/node.py +++ b/python/ray/util/ml_utils/node.py @@ -1,6 +1,8 @@ import ray +from ray.util.annotations import Deprecated +@Deprecated def get_current_node_resource_key() -> str: """Get the Ray resource key for current node. It can be used for actor placement. @@ -22,6 +24,7 @@ def get_current_node_resource_key() -> str: raise ValueError("Cannot found the node dictionary for current node.") +@Deprecated def force_on_current_node(task_or_actor=None): """Given a task or actor, place it on the current node. diff --git a/python/ray/util/ml_utils/tests/test_mlflow.py b/python/ray/util/ml_utils/tests/test_mlflow.py index bbc986a40..e54291f7b 100644 --- a/python/ray/util/ml_utils/tests/test_mlflow.py +++ b/python/ray/util/ml_utils/tests/test_mlflow.py @@ -3,7 +3,7 @@ import shutil import tempfile import unittest -from ray.util.ml_utils.mlflow import MLflowLoggerUtil +from ray.util.ml_utils.mlflow import _MLflowLoggerUtil class MLflowTest(unittest.TestCase): @@ -14,7 +14,7 @@ class MLflowTest(unittest.TestCase): mlflow.set_tracking_uri(self.dirpath) mlflow.create_experiment(name="existing_experiment") - self.mlflow_util = MLflowLoggerUtil() + self.mlflow_util = _MLflowLoggerUtil() self.tracking_uri = mlflow.get_tracking_uri() def tearDown(self): @@ -126,7 +126,8 @@ class MLflowTest(unittest.TestCase): if __name__ == "__main__": - import pytest import sys + import pytest + sys.exit(pytest.main(["-v", __file__])) diff --git a/python/ray/util/ml_utils/util.py b/python/ray/util/ml_utils/util.py index 4f99f8bf9..9f1b0fd21 100644 --- a/python/ray/util/ml_utils/util.py +++ b/python/ray/util/ml_utils/util.py @@ -1,8 +1,12 @@ -from contextlib import closing import socket +from contextlib import closing + import numpy as np +from ray.util.annotations import Deprecated + +@Deprecated def find_free_port(): with closing(socket.socket(socket.AF_INET, socket.SOCK_STREAM)) as s: s.bind(("", 0)) @@ -10,9 +14,11 @@ def find_free_port(): return s.getsockname()[1] +@Deprecated def is_nan(value): return np.isnan(value) +@Deprecated def is_nan_or_inf(value): return is_nan(value) or np.isinf(value) diff --git a/python/ray/util/multiprocessing/pool.py b/python/ray/util/multiprocessing/pool.py index b6671fa15..c3c908b32 100644 --- a/python/ray/util/multiprocessing/pool.py +++ b/python/ray/util/multiprocessing/pool.py @@ -1,26 +1,27 @@ -from typing import Callable, Iterable, List, Tuple, Optional, Any, Dict, Hashable -import logging -from multiprocessing import TimeoutError -import os -import time import collections -import threading -import queue import copy import gc -import sys import itertools +import logging +import os +import queue +import sys +import threading +import time +from multiprocessing import TimeoutError +from typing import Any, Callable, Dict, Hashable, Iterable, List, Optional, Tuple + +import ray +from ray.util import log_once try: - from joblib.parallel import BatchedCalls, parallel_backend from joblib._parallel_backends import SafeFunction + from joblib.parallel import BatchedCalls, parallel_backend except ImportError: BatchedCalls = None parallel_backend = None SafeFunction = None -import ray -from ray.util import log_once logger = logging.getLogger(__name__) @@ -617,7 +618,7 @@ class Pool: logger.info("Starting local ray cluster") ray.init(num_cpus=processes) - ray_cpus = int(ray.state.cluster_resources()["CPU"]) + ray_cpus = int(ray._private.state.cluster_resources()["CPU"]) if processes is None: processes = ray_cpus if processes <= 0: diff --git a/python/ray/util/placement_group.py b/python/ray/util/placement_group.py index 3d5de2422..a6567137d 100644 --- a/python/ray/util/placement_group.py +++ b/python/ray/util/placement_group.py @@ -1,12 +1,11 @@ -from typing import Dict, Union, List, Optional +from typing import Dict, List, Optional, Union import ray -from ray._raylet import PlacementGroupID +from ray._private.client_mode_hook import client_mode_should_convert, client_mode_wrap +from ray._private.ray_constants import to_memory_units from ray._private.utils import hex_to_binary -from ray.util.annotations import PublicAPI, DeveloperAPI -from ray.ray_constants import to_memory_units -from ray._private.client_mode_hook import client_mode_should_convert -from ray._private.client_mode_hook import client_mode_wrap +from ray._raylet import PlacementGroupID +from ray.util.annotations import DeveloperAPI, PublicAPI bundle_reservation_check = None BUNDLE_RESOURCE_LABEL = "bundle" @@ -106,7 +105,7 @@ class PlacementGroup: @client_mode_wrap def _call_placement_group_ready(pg_id: PlacementGroupID, timeout_seconds: int) -> bool: - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker.check_connected() return worker.core_worker.wait_placement_group_ready(pg_id, timeout_seconds) @@ -114,10 +113,12 @@ def _call_placement_group_ready(pg_id: PlacementGroupID, timeout_seconds: int) - @client_mode_wrap def _get_bundle_cache(pg_id: PlacementGroupID) -> List[Dict]: - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker.check_connected() - return list(ray.state.state.placement_group_table(pg_id)["bundles"].values()) + return list( + ray._private.state.state.placement_group_table(pg_id)["bundles"].values() + ) @PublicAPI @@ -155,7 +156,7 @@ def placement_group( Return: PlacementGroup: Placement group object. """ - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker.check_connected() if not isinstance(bundles, list): @@ -201,7 +202,7 @@ def remove_placement_group(placement_group: PlacementGroup) -> None: placement_group: The placement group to delete. """ assert placement_group is not None - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker.check_connected() worker.core_worker.remove_placement_group(placement_group.id) @@ -218,9 +219,9 @@ def get_placement_group(placement_group_name: str) -> PlacementGroup: """ if not placement_group_name: raise ValueError("Please supply a non-empty value to get_placement_group") - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker.check_connected() - placement_group_info = ray.state.state.get_placement_group_by_name( + placement_group_info = ray._private.state.state.get_placement_group_by_name( placement_group_name, worker.namespace ) if placement_group_info is None: @@ -240,10 +241,10 @@ def placement_group_table(placement_group: PlacementGroup = None) -> dict: placement_group: placement group to see states. """ - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker.check_connected() placement_group_id = placement_group.id if (placement_group is not None) else None - return ray.state.state.placement_group_table(placement_group_id) + return ray._private.state.state.placement_group_table(placement_group_id) @PublicAPI @@ -280,7 +281,7 @@ def get_current_placement_group() -> Optional[PlacementGroup]: if client_mode_should_convert(auto_init=True): # Client mode is only a driver. return None - worker = ray.worker.global_worker + worker = ray._private.worker.global_worker worker.check_connected() pg_id = worker.placement_group_id if pg_id.is_nil(): @@ -357,7 +358,7 @@ def _validate_resource_shape( ) -def configure_placement_group_based_on_context( +def _configure_placement_group_based_on_context( placement_group_capture_child_tasks: bool, bundle_index: int, resources: Dict, diff --git a/python/ray/util/rpdb.py b/python/ray/util/rpdb.py index 237bade66..7742e7c5d 100644 --- a/python/ray/util/rpdb.py +++ b/python/ray/util/rpdb.py @@ -12,13 +12,15 @@ import select import socket import sys import time +import traceback import uuid from pdb import Pdb -import setproctitle -import traceback from typing import Callable + +import setproctitle + import ray -from ray import ray_constants +from ray._private import ray_constants from ray.experimental.internal_kv import _internal_kv_del, _internal_kv_put from ray.util.annotations import DeveloperAPI @@ -26,12 +28,12 @@ PY3 = sys.version_info[0] == 3 log = logging.getLogger(__name__) -def cry(message, stderr=sys.__stderr__): +def _cry(message, stderr=sys.__stderr__): print(message, file=stderr) stderr.flush() -class LF2CRLF_FileWrapper(object): +class _LF2CRLF_FileWrapper(object): def __init__(self, connection): self.connection = connection self.stream = fh = connection.makefile("rw") @@ -62,7 +64,7 @@ class LF2CRLF_FileWrapper(object): self.write(line, nl_rex) -class PdbWrap(Pdb): +class _PdbWrap(Pdb): """Wrap PDB to run a custom exit hook on continue.""" def __init__(self, exit_hook: Callable[[], None]): @@ -76,7 +78,7 @@ class PdbWrap(Pdb): do_c = do_cont = do_continue -class RemotePdb(Pdb): +class _RemotePdb(Pdb): """ This will run pdb as a ephemeral telnet service. Once you connect no one else can connect. On construction this object will block execution till a @@ -108,7 +110,7 @@ class RemotePdb(Pdb): def listen(self): if not self._quiet: - cry( + _cry( "RemotePdb session open at %s:%s, " "use 'ray debug' to connect..." % (self._ip_address, self._listen_socket.getsockname()[1]) @@ -116,8 +118,8 @@ class RemotePdb(Pdb): self._listen_socket.listen(1) connection, address = self._listen_socket.accept() if not self._quiet: - cry("RemotePdb accepted connection from %s." % repr(address)) - self.handle = LF2CRLF_FileWrapper(connection) + _cry("RemotePdb accepted connection from %s." % repr(address)) + self.handle = _LF2CRLF_FileWrapper(connection) Pdb.__init__( self, completekey="tab", @@ -137,15 +139,15 @@ class RemotePdb(Pdb): ): self.backup.append((name, getattr(sys, name))) setattr(sys, name, self.handle) - RemotePdb.active_instance = self + _RemotePdb.active_instance = self def __restore(self): if self.backup and not self._quiet: - cry("Restoring streams: %s ..." % self.backup) + _cry("Restoring streams: %s ..." % self.backup) for name, fh in self.backup: setattr(sys, name, fh) self.handle.close() - RemotePdb.active_instance = None + _RemotePdb.active_instance = None def do_quit(self, arg): self.__restore() @@ -185,7 +187,7 @@ class RemotePdb(Pdb): Skip into the next remote call. """ # Tell the next task to drop into the debugger. - ray.worker.global_worker.debugger_breakpoint = self._breakpoint_uuid + ray._private.worker.global_worker.debugger_breakpoint = self._breakpoint_uuid # Tell the debug loop to connect to the next task. data = json.dumps( { @@ -205,13 +207,15 @@ class RemotePdb(Pdb): """get Skip to where the current task returns to. """ - ray.worker.global_worker.debugger_get_breakpoint = self._breakpoint_uuid + ray._private.worker.global_worker.debugger_get_breakpoint = ( + self._breakpoint_uuid + ) self.__restore() self.handle.connection.close() return Pdb.do_continue(self, arg) -def connect_ray_pdb( +def _connect_ray_pdb( host=None, port=None, patch_stdstreams=False, @@ -234,10 +238,10 @@ def connect_ray_pdb( if not breakpoint_uuid: breakpoint_uuid = uuid.uuid4().hex if debugger_external: - ip_address = ray.worker.global_worker.node_ip_address + ip_address = ray._private.worker.global_worker.node_ip_address else: ip_address = "localhost" - rdb = RemotePdb( + rdb = _RemotePdb( breakpoint_uuid=breakpoint_uuid, host=host, port=port, @@ -279,15 +283,15 @@ def set_trace(breakpoint_uuid=None): """ # If there is an active debugger already, we do not want to # start another one, so "set_trace" is just a no-op in that case. - if ray.worker.global_worker.debugger_breakpoint == b"": + if ray._private.worker.global_worker.debugger_breakpoint == b"": frame = sys._getframe().f_back - rdb = connect_ray_pdb( + rdb = _connect_ray_pdb( host=None, port=None, patch_stdstreams=False, quiet=None, breakpoint_uuid=breakpoint_uuid.decode() if breakpoint_uuid else None, - debugger_external=ray.worker.global_worker.ray_debugger_external, + debugger_external=ray._private.worker.global_worker.ray_debugger_external, ) rdb.set_trace(frame=frame) @@ -299,29 +303,29 @@ def _driver_set_trace(): spammed: https://github.com/ray-project/ray/issues/18172 """ print("*** Temporarily disabling Ray worker logs ***") - ray.worker._worker_logs_enabled = False + ray._private.worker._worker_logs_enabled = False def enable_logging(): print("*** Re-enabling Ray worker logs ***") - ray.worker._worker_logs_enabled = True + ray._private.worker._worker_logs_enabled = True - pdb = PdbWrap(enable_logging) + pdb = _PdbWrap(enable_logging) frame = sys._getframe().f_back pdb.set_trace(frame) -def post_mortem(): - rdb = connect_ray_pdb( +def _post_mortem(): + rdb = _connect_ray_pdb( host=None, port=None, patch_stdstreams=False, quiet=None, - debugger_external=ray.worker.global_worker.ray_debugger_external, + debugger_external=ray._private.worker.global_worker.ray_debugger_external, ) rdb.post_mortem() -def connect_pdb_client(host, port): +def _connect_pdb_client(host, port): s = socket.socket(socket.AF_INET, socket.SOCK_STREAM) s.connect((host, port)) diff --git a/python/ray/util/serialization.py b/python/ray/util/serialization.py index 72fcc8592..cd318a6dc 100644 --- a/python/ray/util/serialization.py +++ b/python/ray/util/serialization.py @@ -1,7 +1,9 @@ import ray import ray.cloudpickle as pickle +from ray.util.annotations import DeveloperAPI, PublicAPI +@PublicAPI def register_serializer(cls: type, *, serializer: callable, deserializer: callable): """Use the given serializer to serialize instances of type ``cls``, and use the deserializer to deserialize the serialized object. @@ -15,10 +17,11 @@ def register_serializer(cls: type, *, serializer: callable, deserializer: callab instance of type ``cls`` from the serialized object. This function itself must be serializable. """ - context = ray.worker.global_worker.get_serialization_context() + context = ray._private.worker.global_worker.get_serialization_context() context._register_cloudpickle_serializer(cls, serializer, deserializer) +@PublicAPI def deregister_serializer(cls: type): """Deregister the serializer associated with the type ``cls``. There is no effect if the serializer is unavailable. @@ -26,10 +29,11 @@ def deregister_serializer(cls: type): Args: cls: A Python class/type. """ - context = ray.worker.global_worker.get_serialization_context() + context = ray._private.worker.global_worker.get_serialization_context() context._unregister_cloudpickle_reducer(cls) +@DeveloperAPI class StandaloneSerializationContext: # NOTE(simon): Used for registering custom serializers. We cannot directly # use the SerializationContext because it requires Ray workers. Please diff --git a/python/ray/serialization_addons.py b/python/ray/util/serialization_addons.py similarity index 95% rename from python/ray/serialization_addons.py rename to python/ray/util/serialization_addons.py index 7fba3fcd7..5353c1366 100644 --- a/python/ray/serialization_addons.py +++ b/python/ray/util/serialization_addons.py @@ -3,7 +3,10 @@ This module is intended for implementing internal serializers for some site packages. """ +from ray.util.annotations import DeveloperAPI + +@DeveloperAPI def register_pydantic_serializer(serialization_context): try: import pydantic.fields @@ -32,6 +35,7 @@ def register_pydantic_serializer(serialization_context): ) +@DeveloperAPI def register_starlette_serializer(serialization_context): try: import starlette.datastructures @@ -47,6 +51,7 @@ def register_starlette_serializer(serialization_context): ) +@DeveloperAPI def apply(serialization_context): register_pydantic_serializer(serialization_context) register_starlette_serializer(serialization_context) diff --git a/python/ray/util/tracing/tracing_helper.py b/python/ray/util/tracing/tracing_helper.py index 812b01aa0..5c80961bf 100644 --- a/python/ray/util/tracing/tracing_helper.py +++ b/python/ray/util/tracing/tracing_helper.py @@ -1,13 +1,13 @@ -from contextlib import contextmanager -from functools import wraps import importlib import inspect import logging import os +from contextlib import contextmanager +from functools import wraps +from inspect import Parameter from types import ModuleType from typing import ( Any, - cast, Callable, Dict, Generator, @@ -16,12 +16,16 @@ from typing import ( Optional, Sequence, Union, + cast, ) -from inspect import Parameter +import ray._private.worker +from ray._private.inspect_util import ( + is_class_method, + is_function_or_method, + is_static_method, +) from ray.runtime_context import get_runtime_context -from ray.util.inspect import is_class_method, is_function_or_method, is_static_method -import ray.worker logger = logging.getLogger(__name__) @@ -87,7 +91,7 @@ _nameable = Union[str, Callable[..., Any]] _global_is_tracing_enabled = False -def sort_params_list(params_list: List[Parameter]): +def _sort_params_list(params_list: List[Parameter]): """Given a list of Parameters, if a kwargs Parameter exists, move it to the end of the list.""" for i, param in enumerate(params_list): @@ -97,29 +101,29 @@ def sort_params_list(params_list: List[Parameter]): return params_list -def add_param_to_signature(function: Callable, new_param: Parameter): +def _add_param_to_signature(function: Callable, new_param: Parameter): """Add additional Parameter to function signature.""" old_sig = inspect.signature(function) old_sig_list_repr = list(old_sig.parameters.values()) # If new_param is already in signature, do not add it again. if any(param.name == new_param.name for param in old_sig_list_repr): return old_sig - new_params = sort_params_list(old_sig_list_repr + [new_param]) + new_params = _sort_params_list(old_sig_list_repr + [new_param]) new_sig = old_sig.replace(parameters=new_params) return new_sig -def is_tracing_enabled() -> bool: +def _is_tracing_enabled() -> bool: """Checks environment variable feature flag to see if tracing is turned on. Tracing is off by default.""" return _global_is_tracing_enabled -class ImportFromStringError(Exception): +class _ImportFromStringError(Exception): pass -def import_from_string(import_str: Union[ModuleType, str]) -> ModuleType: +def _import_from_string(import_str: Union[ModuleType, str]) -> ModuleType: """Given a string that is in format ":", import the attribute.""" if not isinstance(import_str, str): @@ -130,7 +134,7 @@ def import_from_string(import_str: Union[ModuleType, str]) -> ModuleType: message = ( 'Import string "{import_str}" must be in format' '":".' ) - raise ImportFromStringError(message.format(import_str=import_str)) + raise _ImportFromStringError(message.format(import_str=import_str)) try: module = importlib.import_module(module_str) @@ -138,7 +142,7 @@ def import_from_string(import_str: Union[ModuleType, str]) -> ModuleType: if exc.name != module_str: raise exc from None message = 'Could not import module "{module_str}".' - raise ImportFromStringError(message.format(module_str=module_str)) + raise _ImportFromStringError(message.format(module_str=module_str)) instance = module try: @@ -146,14 +150,14 @@ def import_from_string(import_str: Union[ModuleType, str]) -> ModuleType: instance = getattr(instance, attr_str) except AttributeError: message = 'Attribute "{attrs_str}" not found in module "{module_str}".' - raise ImportFromStringError( + raise _ImportFromStringError( message.format(attrs_str=attrs_str, module_str=module_str) ) return instance -class DictPropagator: +class _DictPropagator: def inject_current_context() -> Dict[Any, Any]: """Inject trace context into otel propagator.""" context_dict: Dict[Any, Any] = {} @@ -168,7 +172,7 @@ class DictPropagator: @contextmanager -def use_context( +def _use_context( parent_context: "_opentelemetry.Context", ) -> Generator[None, None, None]: """Uses the Ray trace context for the span.""" @@ -197,14 +201,14 @@ def _function_hydrate_span_args(func: Callable[..., Any]): } # We only get task ID for workers - if ray.worker.global_worker.mode == ray.worker.WORKER_MODE: + if ray._private.worker.global_worker.mode == ray._private.worker.WORKER_MODE: task_id = ( runtime_context["task_id"].hex() if runtime_context.get("task_id") else None ) if task_id: span_args["ray.task_id"] = task_id - worker_id = getattr(ray.worker.global_worker, "worker_id", None) + worker_id = getattr(ray._private.worker.global_worker, "worker_id", None) if worker_id: span_args["ray.worker_id"] = worker_id.hex() @@ -248,7 +252,7 @@ def _actor_hydrate_span_args(class_: _nameable, method: _nameable): } # We only get actor ID for workers - if ray.worker.global_worker.mode == ray.worker.WORKER_MODE: + if ray._private.worker.global_worker.mode == ray._private.worker.WORKER_MODE: actor_id = ( runtime_context["actor_id"].hex() if runtime_context.get("actor_id") @@ -258,7 +262,7 @@ def _actor_hydrate_span_args(class_: _nameable, method: _nameable): if actor_id: span_args["ray.actor_id"] = actor_id - worker_id = getattr(ray.worker.global_worker, "worker_id", None) + worker_id = getattr(ray._private.worker.global_worker, "worker_id", None) if worker_id: span_args["ray.worker_id"] = worker_id.hex() @@ -297,7 +301,7 @@ def _tracing_task_invocation(method): ) -> Any: # If tracing feature flag is not on, perform a no-op. # Tracing doesn't work for cross lang yet. - if not is_tracing_enabled() or self._is_cross_language: + if not _is_tracing_enabled() or self._is_cross_language: if kwargs is not None: assert "_ray_trace_ctx" not in kwargs return method(self, args, kwargs, *_args, **_kwargs) @@ -311,7 +315,7 @@ def _tracing_task_invocation(method): attributes=_function_hydrate_span_args(self._function_name), ): # Inject a _ray_trace_ctx as a dictionary - kwargs["_ray_trace_ctx"] = DictPropagator.inject_current_context() + kwargs["_ray_trace_ctx"] = _DictPropagator.inject_current_context() return method(self, args, kwargs, *_args, **_kwargs) return _invocation_remote_span @@ -323,13 +327,13 @@ def _inject_tracing_into_function(function): Use the provided trace context from kwargs. """ # Add _ray_trace_ctx to function signature - if not is_tracing_enabled(): + if not _is_tracing_enabled(): return function setattr( function, "__signature__", - add_param_to_signature( + _add_param_to_signature( function, inspect.Parameter( "_ray_trace_ctx", inspect.Parameter.KEYWORD_ONLY, default=None @@ -350,8 +354,8 @@ def _inject_tracing_into_function(function): function_name = function.__module__ + "." + function.__name__ # Retrieves the context from the _ray_trace_ctx dictionary we injected - with use_context( - DictPropagator.extract(_ray_trace_ctx) + with _use_context( + _DictPropagator.extract(_ray_trace_ctx) ), tracer.start_as_current_span( _function_span_consumer_name(function_name), kind=_opentelemetry.trace.SpanKind.CONSUMER, @@ -378,7 +382,7 @@ def _tracing_actor_creation(method): kwargs = {} # If tracing feature flag is not on, perform a no-op - if not is_tracing_enabled(): + if not _is_tracing_enabled(): assert "_ray_trace_ctx" not in kwargs return method(self, args, kwargs, *_args, **_kwargs) @@ -392,7 +396,7 @@ def _tracing_actor_creation(method): attributes=_actor_hydrate_span_args(class_name, method_name), ) as span: # Inject a _ray_trace_ctx as a dictionary - kwargs["_ray_trace_ctx"] = DictPropagator.inject_current_context() + kwargs["_ray_trace_ctx"] = _DictPropagator.inject_current_context() result = method(self, args, kwargs, *_args, **_kwargs) @@ -415,7 +419,7 @@ def _tracing_actor_method_invocation(method): **_kwargs: Any, ) -> Any: # If tracing feature flag is not on, perform a no-op - if not is_tracing_enabled() or self._actor_ref()._ray_is_cross_language: + if not _is_tracing_enabled() or self._actor_ref()._ray_is_cross_language: if kwargs is not None: assert "_ray_trace_ctx" not in kwargs return method(self, args, kwargs, *_args, **_kwargs) @@ -433,7 +437,7 @@ def _tracing_actor_method_invocation(method): attributes=_actor_hydrate_span_args(class_name, method_name), ) as span: # Inject a _ray_trace_ctx as a dictionary - kwargs["_ray_trace_ctx"] = DictPropagator.inject_current_context() + kwargs["_ray_trace_ctx"] = _DictPropagator.inject_current_context() span.set_attribute("ray.actor_id", self._actor_ref()._ray_actor_id.hex()) @@ -458,7 +462,7 @@ def _inject_tracing_into_class(_cls): will extract the trace context """ # If tracing feature flag is not on, perform a no-op - if not is_tracing_enabled() or _ray_trace_ctx is None: + if not _is_tracing_enabled() or _ray_trace_ctx is None: return method(self, *_args, **_kwargs) tracer: _opentelemetry.trace.Tracer = _opentelemetry.trace.get_tracer( @@ -467,8 +471,8 @@ def _inject_tracing_into_class(_cls): # Retrieves the context from the _ray_trace_ctx dictionary we # injected. - with use_context( - DictPropagator.extract(_ray_trace_ctx) + with _use_context( + _DictPropagator.extract(_ray_trace_ctx) ), tracer.start_as_current_span( _actor_span_consumer_name(self.__class__.__name__, method), kind=_opentelemetry.trace.SpanKind.CONSUMER, @@ -490,15 +494,15 @@ def _inject_tracing_into_class(_cls): will extract the trace context """ # If tracing feature flag is not on, perform a no-op - if not is_tracing_enabled() or _ray_trace_ctx is None: + if not _is_tracing_enabled() or _ray_trace_ctx is None: return await method(self, *_args, **_kwargs) tracer = _opentelemetry.trace.get_tracer(__name__) # Retrieves the context from the _ray_trace_ctx dictionary we # injected, or starts a new context - with use_context( - DictPropagator.extract(_ray_trace_ctx) + with _use_context( + _DictPropagator.extract(_ray_trace_ctx) ), tracer.start_as_current_span( _actor_span_consumer_name(self.__class__.__name__, method.__name__), kind=_opentelemetry.trace.SpanKind.CONSUMER, @@ -522,7 +526,7 @@ def _inject_tracing_into_class(_cls): setattr( method, "__signature__", - add_param_to_signature( + _add_param_to_signature( method, inspect.Parameter( "_ray_trace_ctx", inspect.Parameter.KEYWORD_ONLY, default=None diff --git a/python/ray/utils.py b/python/ray/utils.py deleted file mode 100644 index 92b836562..000000000 --- a/python/ray/utils.py +++ /dev/null @@ -1,87 +0,0 @@ -from typing import Dict, Union, Optional -from google.protobuf import json_format - -import ray._private.utils as private_utils -from ray.runtime_env import RuntimeEnv, RuntimeEnvConfig -from ray.core.generated.runtime_env_common_pb2 import ( - RuntimeEnvInfo as ProtoRuntimeEnvInfo, -) - -deprecated = private_utils.deprecated( - "If you need to use this function, open a feature request issue on GitHub.", - removal_release="1.4", - warn_once=True, -) - -get_system_memory = deprecated(private_utils.get_system_memory) - - -def get_runtime_env_info( - runtime_env: RuntimeEnv, - *, - is_job_runtime_env: bool = False, - serialize: bool = False, -): - """Create runtime env info from runtime env. - - In the user interface, the argument `runtime_env` contains some fields - which not contained in `ProtoRuntimeEnv` but in `ProtoRuntimeEnvInfo`, - such as `eager_install`. This function will extract those fields from - `RuntimeEnv` and create a new `ProtoRuntimeEnvInfo`, and serialize it. - """ - proto_runtime_env_info = ProtoRuntimeEnvInfo() - - proto_runtime_env_info.uris[:] = runtime_env.get_uris() - - # Normally, `RuntimeEnv` should guarantee the accuracy of field eager_install, - # but so far, the internal code has not completely prohibited direct - # modification of fields in RuntimeEnv, so we should check it for insurance. - # TODO(Catch-Bull): overload `__setitem__` for `RuntimeEnv`, change the - # runtime_env of all internal code from dict to RuntimeEnv. - - eager_install = runtime_env.get("eager_install") - if is_job_runtime_env or eager_install is not None: - if eager_install is None: - eager_install = True - elif not isinstance(eager_install, bool): - raise TypeError( - f"eager_install must be a boolean. got {type(eager_install)}" - ) - proto_runtime_env_info.runtime_env_eager_install = eager_install - - runtime_env_config = runtime_env.get("config") - if runtime_env_config is None: - runtime_env_config = RuntimeEnvConfig.default_config() - else: - runtime_env_config = RuntimeEnvConfig.parse_and_validate_runtime_env_config( - runtime_env_config - ) - - proto_runtime_env_info.runtime_env_config.CopyFrom( - runtime_env_config.build_proto_runtime_env_config() - ) - - proto_runtime_env_info.serialized_runtime_env = runtime_env.serialize() - - if not serialize: - return proto_runtime_env_info - - return json_format.MessageToJson(proto_runtime_env_info) - - -def parse_runtime_env(runtime_env: Optional[Union[Dict, RuntimeEnv]]): - # Parse local pip/conda config files here. If we instead did it in - # .remote(), it would get run in the Ray Client server, which runs on - # a remote node where the files aren't available. - if runtime_env: - if isinstance(runtime_env, dict): - return RuntimeEnv(**(runtime_env or {})) - raise TypeError( - "runtime_env must be dict or RuntimeEnv, ", - f"but got: {type(runtime_env)}", - ) - else: - # Keep the new_runtime_env as None. In .remote(), we need to know - # if runtime_env is None to know whether or not to fall back to the - # runtime_env specified in the @ray.remote decorator. - return None diff --git a/python/ray/workflow/tests/test_events.py b/python/ray/workflow/tests/test_events.py index 4eedfdf0f..05d615fb1 100644 --- a/python/ray/workflow/tests/test_events.py +++ b/python/ray/workflow/tests/test_events.py @@ -1,13 +1,14 @@ -from ray.tests.conftest import * # noqa -import pytest - import asyncio -import ray -from ray import workflow -from ray.workflow.tests import utils import subprocess import time +import pytest + +import ray +from ray import workflow +from ray.tests.conftest import * # noqa +from ray.workflow.tests import utils + def test_sleep(workflow_start_regular_shared): @ray.remote @@ -165,7 +166,7 @@ def test_crash_during_event_checkpointing(workflow_start_regular_shared): """Ensure that if the cluster dies while the event is being checkpointed, we properly re-poll for the event.""" - from ray.internal import storage + from ray._private import storage storage_uri = storage._storage_uri @@ -229,7 +230,7 @@ def test_crash_after_commit(workflow_start_regular_shared): checkpointing. """ - from ray.internal import storage + from ray._private import storage storage_uri = storage._storage_uri diff --git a/python/ray/workflow/tests/test_inplace_workflows.py b/python/ray/workflow/tests/test_inplace_workflows.py index 2f79984f3..4dd601924 100644 --- a/python/ray/workflow/tests/test_inplace_workflows.py +++ b/python/ray/workflow/tests/test_inplace_workflows.py @@ -1,13 +1,13 @@ -import ray -from ray.tests.conftest import * # noqa - import pytest + +import ray from ray import workflow +from ray.tests.conftest import * # noqa @ray.remote def check_and_update(x, worker_id): - from ray.worker import global_worker + from ray._private.worker import global_worker _worker_id = global_worker.worker_id if worker_id == _worker_id: @@ -17,7 +17,7 @@ def check_and_update(x, worker_id): @ray.remote def inplace_test(): - from ray.worker import global_worker + from ray._private.worker import global_worker worker_id = global_worker.worker_id x = check_and_update.options(**workflow.options(allow_inplace=True)).bind( @@ -32,7 +32,7 @@ def inplace_test(): @ray.remote def exp_inplace(k, n, worker_id=None): - from ray.worker import global_worker + from ray._private.worker import global_worker _worker_id = global_worker.worker_id if worker_id is not None: @@ -51,7 +51,7 @@ def exp_inplace(k, n, worker_id=None): @ray.remote def exp_remote(k, n, worker_id=None): - from ray.worker import global_worker + from ray._private.worker import global_worker _worker_id = global_worker.worker_id if worker_id is not None: diff --git a/python/ray/workflow/tests/test_serialization.py b/python/ray/workflow/tests/test_serialization.py index 3e1608a4d..00e4c4318 100644 --- a/python/ray/workflow/tests/test_serialization.py +++ b/python/ray/workflow/tests/test_serialization.py @@ -1,14 +1,15 @@ -from filelock import FileLock -import pytest -import ray -from ray import workflow -from ray.workflow import serialization -from ray.workflow import workflow_storage -from ray._private.test_utils import run_string_as_driver_nonblocking -from ray.tests.conftest import * # noqa import subprocess import time +import pytest +from filelock import FileLock + +import ray +from ray import workflow +from ray._private.test_utils import run_string_as_driver_nonblocking +from ray.tests.conftest import * # noqa +from ray.workflow import serialization, workflow_storage + @ray.remote def identity(x): @@ -173,7 +174,7 @@ def test_embedded_objectrefs(workflow_start_regular): def __init__(self, refs): self.refs = refs - from ray.internal.storage import _storage_uri + from ray._private.storage import _storage_uri wrapped = ObjectRefsWrapper([ray.put(1), ray.put(2)]) diff --git a/python/ray/workflow/tests/test_storage.py b/python/ray/workflow/tests/test_storage.py index 5222e4de0..e00185611 100644 --- a/python/ray/workflow/tests/test_storage.py +++ b/python/ray/workflow/tests/test_storage.py @@ -1,17 +1,19 @@ +import subprocess +import time + import pytest + import ray +from ray import workflow from ray._private import signature from ray.tests.conftest import * # noqa -from ray import workflow from ray.workflow import workflow_storage from ray.workflow.common import ( StepType, - WorkflowStepRuntimeOptions, WorkflowNotFoundError, + WorkflowStepRuntimeOptions, ) from ray.workflow.tests import utils -import subprocess -import time def some_func(x): @@ -23,7 +25,7 @@ def some_func2(x): def test_delete(workflow_start_regular): - from ray.internal.storage import _storage_uri + from ray._private.storage import _storage_uri # Try deleting a random workflow that never existed. with pytest.raises(WorkflowNotFoundError): @@ -149,7 +151,7 @@ def test_workflow_storage(workflow_start_regular): # test s3 path # here we hardcode the path to make sure s3 path is parsed correctly - from ray.internal.storage import _storage_uri + from ray._private.storage import _storage_uri if _storage_uri.startswith("s3://"): assert wf_storage._get("steps/outputs.json", True) == root_output_metadata diff --git a/python/ray/workflow/workflow_context.py b/python/ray/workflow/workflow_context.py index ad91a9f12..6136a25b8 100644 --- a/python/ray/workflow/workflow_context.py +++ b/python/ray/workflow/workflow_context.py @@ -1,16 +1,17 @@ import copy -from dataclasses import dataclass, field import logging -from typing import Optional, List, TYPE_CHECKING from contextlib import contextmanager +from dataclasses import dataclass, field +from typing import TYPE_CHECKING, List, Optional + import ray +from ray._private.ray_logging import configure_log_file, get_worker_log_file_name from ray.workflow.common import WorkflowStatus -from ray._private.ray_logging import get_worker_log_file_name, configure_log_file logger = logging.getLogger(__name__) if TYPE_CHECKING: - from ray.workflow.common import StepID, CheckpointModeType + from ray.workflow.common import CheckpointModeType, StepID @dataclass @@ -206,7 +207,7 @@ def workflow_logging_context(job_id) -> None: Args: job_id: The ID of the job that submits the workflow execution. """ - node = ray.worker._global_node + node = ray._private.worker._global_node original_out_file, original_err_file = node.get_log_file_handles( get_worker_log_file_name("WORKER") ) diff --git a/python/ray/workflow/workflow_storage.py b/python/ray/workflow/workflow_storage.py index b057084e0..fc5e5fa08 100644 --- a/python/ray/workflow/workflow_storage.py +++ b/python/ray/workflow/workflow_storage.py @@ -4,28 +4,25 @@ workflows. """ import json -import os -from typing import Dict, List, Optional, Any, Callable, Tuple, Union, Set -from dataclasses import dataclass import logging +import os +from dataclasses import dataclass +from typing import Any, Callable, Dict, List, Optional, Set, Tuple, Union import ray from ray import cloudpickle -from ray._private import signature -from ray.internal import storage +from ray._private import signature, storage +from ray.types import ObjectRef +from ray.workflow import serialization, serialization_context, workflow_context from ray.workflow.common import ( - Workflow, StepID, - WorkflowStatus, - WorkflowRef, + Workflow, WorkflowNotFoundError, + WorkflowRef, + WorkflowStatus, WorkflowStepRuntimeOptions, ) -from ray.workflow import workflow_context -from ray.workflow import serialization -from ray.workflow import serialization_context from ray.workflow.storage import DataLoadError, DataSaveError, KeyNotFoundError -from ray.types import ObjectRef logger = logging.getLogger(__name__) diff --git a/release/nightly_tests/chaos_test/test_chaos_basic.py b/release/nightly_tests/chaos_test/test_chaos_basic.py index 53eb35a5e..5c53e5959 100644 --- a/release/nightly_tests/chaos_test/test_chaos_basic.py +++ b/release/nightly_tests/chaos_test/test_chaos_basic.py @@ -1,16 +1,16 @@ import argparse +import json +import logging import os import random import string import time -import json -import logging import numpy as np -import ray -from ray.data._internal.progress_bar import ProgressBar +import ray from ray._private.test_utils import monitor_memory_usage, wait_for_condition +from ray.data._internal.progress_bar import ProgressBar def run_task_workload(total_num_cpus, smoke): @@ -87,7 +87,7 @@ def run_actor_workload(total_num_cpus, smoke): if smoke: multiplier = 1 TOTAL_TASKS = int(300 * multiplier) - current_node_ip = ray.worker.global_worker.node_ip_address + current_node_ip = ray._private.worker.global_worker.node_ip_address db_actors = [ DBActor.options(resources={f"node:{current_node_ip}": 0.001}).remote() for _ in range(NUM_CPUS) diff --git a/release/nightly_tests/dask_on_ray/dask_on_ray_sort.py b/release/nightly_tests/dask_on_ray/dask_on_ray_sort.py index d46e4368b..8dd932438 100644 --- a/release/nightly_tests/dask_on_ray/dask_on_ray_sort.py +++ b/release/nightly_tests/dask_on_ray/dask_on_ray_sort.py @@ -1,16 +1,15 @@ +import csv +import json +import os.path +import time + +import boto3 import dask import dask.dataframe as dd -import json -import pandas as pd import numpy as np -import os.path -import csv -import boto3 - +import pandas as pd from dask.distributed import Client -import time - def load_dataset(client, data_dir, s3_bucket, nbytes, npartitions): num_bytes_per_partition = nbytes // npartitions @@ -185,7 +184,7 @@ if __name__ == "__main__": import ray ray.init(address="auto") - from ray.util.dask import ray_dask_get, dataframe_optimize + from ray.util.dask import dataframe_optimize, ray_dask_get dask.config.set(scheduler=ray_dask_get, dataframe_optimize=dataframe_optimize) client = None @@ -236,7 +235,7 @@ if __name__ == "__main__": ) ) - print(ray.internal.internal_api.memory_summary(stats_only=True)) + print(ray._private.internal_api.memory_summary(stats_only=True)) duration = np.mean(output) with open(os.environ["TEST_OUTPUT_JSON"], "w") as f: diff --git a/release/nightly_tests/dask_on_ray/large_scale_test.py b/release/nightly_tests/dask_on_ray/large_scale_test.py index 4466fa8af..41bfce49d 100644 --- a/release/nightly_tests/dask_on_ray/large_scale_test.py +++ b/release/nightly_tests/dask_on_ray/large_scale_test.py @@ -1,22 +1,23 @@ """ @author jennakwon06 """ -from copy import copy, deepcopy - -import time -import random import argparse -import ray -import logging -import os -from typing import List, Tuple -import numpy as np -import dask.array -import xarray -from ray.util.dask import ray_dask_get -from ray._private.test_utils import monitor_memory_usage -import math import json +import logging +import math +import os +import random +import time +from copy import copy, deepcopy +from typing import List, Tuple + +import dask.array +import numpy as np +import xarray + +import ray +from ray._private.test_utils import monitor_memory_usage +from ray.util.dask import ray_dask_get """ We simulate a real-life usecase where we process a time-series @@ -461,7 +462,7 @@ def main(): used_gb, usage = ray.get(monitor_actor.get_peak_memory_info.remote()) print(f"Peak memory usage: {round(used_gb, 2)}GB") print(f"Peak memory usage per processes:\n {usage}") - print(ray.internal.internal_api.memory_summary(stats_only=True)) + print(ray._private.internal_api.memory_summary(stats_only=True)) with open(os.environ["TEST_OUTPUT_JSON"], "w") as f: f.write( json.dumps( diff --git a/release/nightly_tests/dataset/sort.py b/release/nightly_tests/dataset/sort.py index f3bd6c1af..82302ff97 100644 --- a/release/nightly_tests/dataset/sort.py +++ b/release/nightly_tests/dataset/sort.py @@ -8,12 +8,12 @@ import numpy as np import psutil import ray +from ray._private.internal_api import memory_summary from ray.data._internal.arrow_block import ArrowRow from ray.data._internal.util import _check_pyarrow_version from ray.data.block import Block, BlockMetadata from ray.data.context import DatasetContext from ray.data.datasource import Datasource, ReadTask -from ray.internal.internal_api import memory_summary class RandomIntRowDatasource(Datasource[ArrowRow]): diff --git a/release/nightly_tests/stress_tests/test_many_tasks.py b/release/nightly_tests/stress_tests/test_many_tasks.py index d370d674b..9e0fbca4c 100644 --- a/release/nightly_tests/stress_tests/test_many_tasks.py +++ b/release/nightly_tests/stress_tests/test_many_tasks.py @@ -1,12 +1,13 @@ #!/usr/bin/env python import argparse -from collections import defaultdict -import numpy as np import json import logging import os import time +from collections import defaultdict + +import numpy as np import ray @@ -138,7 +139,7 @@ def stage4(): start = time.perf_counter() time.sleep(1) end = time.perf_counter() - return start, end, ray.worker.global_worker.node.unique_id + return start, end, ray._private.worker.global_worker.node.unique_id results = ray.get([func.remote(i) for i in range(num_tasks)]) diff --git a/rllib/evaluation/rollout_worker.py b/rllib/evaluation/rollout_worker.py index 73a45f302..b2d9e9aab 100644 --- a/rllib/evaluation/rollout_worker.py +++ b/rllib/evaluation/rollout_worker.py @@ -1,12 +1,9 @@ import copy -import gym -from gym.spaces import Discrete, MultiDiscrete, Space import logging -import numpy as np -import platform import os -import tree # pip install dm_tree +import platform from typing import ( + TYPE_CHECKING, Any, Callable, Container, @@ -16,49 +13,50 @@ from typing import ( Set, Tuple, Type, - TYPE_CHECKING, Union, ) +import gym +import numpy as np +import tree # pip install dm_tree +from gym.spaces import Discrete, MultiDiscrete, Space + import ray from ray import ObjectRef from ray import cloudpickle as pickle from ray.rllib.env.base_env import BaseEnv, convert_to_base_env from ray.rllib.env.env_context import EnvContext -from ray.rllib.env.multi_agent_env import MultiAgentEnv from ray.rllib.env.external_multi_agent_env import ExternalMultiAgentEnv -from ray.rllib.env.wrappers.atari_wrappers import wrap_deepmind, is_atari -from ray.rllib.evaluation.sampler import AsyncSampler, SyncSampler +from ray.rllib.env.multi_agent_env import MultiAgentEnv +from ray.rllib.env.wrappers.atari_wrappers import is_atari, wrap_deepmind from ray.rllib.evaluation.metrics import RolloutMetrics +from ray.rllib.evaluation.sampler import AsyncSampler, SyncSampler from ray.rllib.models import ModelCatalog from ray.rllib.models.preprocessors import Preprocessor -from ray.rllib.offline import NoopOutput, IOContext, OutputWriter, InputReader +from ray.rllib.offline import InputReader, IOContext, NoopOutput, OutputWriter from ray.rllib.offline.estimators import ( - OffPolicyEstimate, - OffPolicyEstimator, - ImportanceSampling, - WeightedImportanceSampling, DirectMethod, DoublyRobust, + ImportanceSampling, + OffPolicyEstimate, + OffPolicyEstimator, + WeightedImportanceSampling, ) -from ray.rllib.policy.sample_batch import MultiAgentBatch, DEFAULT_POLICY_ID from ray.rllib.policy.policy import Policy, PolicySpec from ray.rllib.policy.policy_map import PolicyMap +from ray.rllib.policy.sample_batch import DEFAULT_POLICY_ID, MultiAgentBatch from ray.rllib.policy.torch_policy import TorchPolicy from ray.rllib.policy.torch_policy_v2 import TorchPolicyV2 -from ray.rllib.utils import force_list, merge_dicts, check_env +from ray.rllib.utils import check_env, force_list, merge_dicts from ray.rllib.utils.annotations import DeveloperAPI, ExperimentalAPI from ray.rllib.utils.debug import summarize, update_global_seed_if_necessary -from ray.rllib.utils.deprecation import ( - Deprecated, - deprecation_warning, -) +from ray.rllib.utils.deprecation import Deprecated, deprecation_warning from ray.rllib.utils.error import ERR_MSG_NO_GPUS, HOWTO_CHANGE_CONFIG -from ray.rllib.utils.filter import get_filter, Filter +from ray.rllib.utils.filter import Filter, get_filter from ray.rllib.utils.framework import try_import_tf, try_import_torch from ray.rllib.utils.sgd import do_minibatch_sgd -from ray.rllib.utils.tf_utils import get_gpu_devices as get_tf_gpu_devices from ray.rllib.utils.tf_run_builder import _TFRunBuilder +from ray.rllib.utils.tf_utils import get_gpu_devices as get_tf_gpu_devices from ray.rllib.utils.typing import ( AgentID, EnvConfigDict, @@ -74,13 +72,13 @@ from ray.rllib.utils.typing import ( SampleBatchType, T, ) -from ray.util.debug import log_once, disable_log_once_globally, enable_periodic_logging +from ray.util.debug import disable_log_once_globally, enable_periodic_logging, log_once from ray.util.iter import ParallelIteratorWorker if TYPE_CHECKING: + from ray.rllib.algorithms.callbacks import DefaultCallbacks # noqa from ray.rllib.evaluation.episode import Episode from ray.rllib.evaluation.observation_function import ObservationFunction - from ray.rllib.algorithms.callbacks import DefaultCallbacks # noqa tf1, tf, tfv = try_import_tf() torch, _ = try_import_torch() @@ -605,7 +603,7 @@ class RolloutWorker(ParallelIteratorWorker): # Error if we don't find enough GPUs. if ( ray.is_initialized() - and ray.worker._mode() != ray.worker.LOCAL_MODE + and ray._private.worker._mode() != ray._private.worker.LOCAL_MODE and not policy_config.get("_fake_gpus") ): @@ -623,7 +621,7 @@ class RolloutWorker(ParallelIteratorWorker): # requested. elif ( ray.is_initialized() - and ray.worker._mode() == ray.worker.LOCAL_MODE + and ray._private.worker._mode() == ray._private.worker.LOCAL_MODE and num_gpus > 0 and not policy_config.get("_fake_gpus") ): diff --git a/rllib/policy/tf_policy.py b/rllib/policy/tf_policy.py index 2717f7c68..1a1c58a23 100644 --- a/rllib/policy/tf_policy.py +++ b/rllib/policy/tf_policy.py @@ -152,7 +152,10 @@ class TFPolicy(Policy): # Get devices to build the graph on. worker_idx = config.get("worker_index", 0) - if not config["_fake_gpus"] and ray.worker._mode() == ray.worker.LOCAL_MODE: + if ( + not config["_fake_gpus"] + and ray._private.worker._mode() == ray._private.worker.LOCAL_MODE + ): num_gpus = 0 elif worker_idx == 0: num_gpus = config["num_gpus"] @@ -186,7 +189,7 @@ class TFPolicy(Policy): # We are a remote worker (WORKER_MODE=1): # GPUs should be assigned to us by ray. - if ray.worker._mode() == ray.worker.WORKER_MODE: + if ray._private.worker._mode() == ray._private.worker.WORKER_MODE: gpu_ids = ray.get_gpu_ids() if len(gpu_ids) < num_gpus: diff --git a/rllib/policy/torch_policy.py b/rllib/policy/torch_policy.py index d329d4c66..5331b9428 100644 --- a/rllib/policy/torch_policy.py +++ b/rllib/policy/torch_policy.py @@ -1,14 +1,12 @@ import copy import functools -import gym import logging import math -import numpy as np import os import threading import time -import tree # pip install dm_tree from typing import ( + TYPE_CHECKING, Any, Callable, Dict, @@ -18,9 +16,12 @@ from typing import ( Tuple, Type, Union, - TYPE_CHECKING, ) +import gym +import numpy as np +import tree # pip install dm_tree + import ray from ray.rllib.models.catalog import ModelCatalog from ray.rllib.models.modelv2 import ModelV2 @@ -29,7 +30,7 @@ from ray.rllib.models.torch.torch_modelv2 import TorchModelV2 from ray.rllib.policy.policy import Policy from ray.rllib.policy.rnn_sequencing import pad_batch_to_sequences_of_same_size from ray.rllib.policy.sample_batch import SampleBatch -from ray.rllib.utils import force_list, NullContextManager +from ray.rllib.utils import NullContextManager, force_list from ray.rllib.utils.annotations import DeveloperAPI, override from ray.rllib.utils.framework import try_import_torch from ray.rllib.utils.metrics import NUM_AGENT_STEPS_TRAINED @@ -39,12 +40,12 @@ from ray.rllib.utils.spaces.space_utils import normalize_action from ray.rllib.utils.threading import with_lock from ray.rllib.utils.torch_utils import convert_to_torch_tensor from ray.rllib.utils.typing import ( + AlgorithmConfigDict, GradInfoDict, ModelGradients, ModelWeights, - TensorType, TensorStructType, - AlgorithmConfigDict, + TensorType, ) if TYPE_CHECKING: @@ -173,7 +174,10 @@ class TorchPolicy(Policy): # Get devices to build the graph on. worker_idx = self.config.get("worker_index", 0) - if not config["_fake_gpus"] and ray.worker._mode() == ray.worker.LOCAL_MODE: + if ( + not config["_fake_gpus"] + and ray._private.worker._mode() == ray._private.worker.LOCAL_MODE + ): num_gpus = 0 elif worker_idx == 0: num_gpus = config["num_gpus"] @@ -216,7 +220,7 @@ class TorchPolicy(Policy): ) # We are a remote worker (WORKER_MODE=1): # GPUs should be assigned to us by ray. - if ray.worker._mode() == ray.worker.WORKER_MODE: + if ray._private.worker._mode() == ray._private.worker.WORKER_MODE: gpu_ids = ray.get_gpu_ids() if len(gpu_ids) < num_gpus: diff --git a/rllib/policy/torch_policy_v2.py b/rllib/policy/torch_policy_v2.py index 10eddbe2f..30fb12a46 100644 --- a/rllib/policy/torch_policy_v2.py +++ b/rllib/policy/torch_policy_v2.py @@ -94,7 +94,10 @@ class TorchPolicyV2(Policy): # Get devices to build the graph on. worker_idx = self.config.get("worker_index", 0) - if not config["_fake_gpus"] and ray.worker._mode() == ray.worker.LOCAL_MODE: + if ( + not config["_fake_gpus"] + and ray._private.worker._mode() == ray._private.worker.LOCAL_MODE + ): num_gpus = 0 elif worker_idx == 0: num_gpus = config["num_gpus"] @@ -137,7 +140,7 @@ class TorchPolicyV2(Policy): ) # We are a remote worker (WORKER_MODE=1): # GPUs should be assigned to us by ray. - if ray.worker._mode() == ray.worker.WORKER_MODE: + if ray._private.worker._mode() == ray._private.worker.WORKER_MODE: gpu_ids = ray.get_gpu_ids() if len(gpu_ids) < num_gpus: diff --git a/rllib/utils/torch_utils.py b/rllib/utils/torch_utils.py index 71d4b84f8..d99b45aa2 100644 --- a/rllib/utils/torch_utils.py +++ b/rllib/utils/torch_utils.py @@ -304,7 +304,10 @@ def get_device(config): # Figure out the number of GPUs to use on the local side (index=0) or on # the remote workers (index > 0). worker_idx = config.get("worker_index", 0) - if not config["_fake_gpus"] and ray.worker._mode() == ray.worker.LOCAL_MODE: + if ( + not config["_fake_gpus"] + and ray._private.worker._mode() == ray._private.worker.LOCAL_MODE + ): num_gpus = 0 elif worker_idx == 0: num_gpus = config["num_gpus"] @@ -327,7 +330,7 @@ def get_device(config): else: # We are a remote worker (WORKER_MODE=1): # GPUs should be assigned to us by ray. - if ray.worker._mode() == ray.worker.WORKER_MODE: + if ray._private.worker._mode() == ray._private.worker.WORKER_MODE: gpu_ids = ray.get_gpu_ids() if len(gpu_ids) < num_gpus: diff --git a/src/ray/common/constants.h b/src/ray/common/constants.h index 68442bfd3..b55b0ee4b 100644 --- a/src/ray/common/constants.h +++ b/src/ray/common/constants.h @@ -48,7 +48,7 @@ constexpr char kEnvVarKeyRayletPid[] = "RAY_RAYLET_PID"; constexpr int kMessagePackOffset = 9; /// Filename of "shim process" that sets up Python worker environment. -/// Should be kept in sync with SETUP_WORKER_FILENAME in ray.ray_constants. +/// Should be kept in sync with SETUP_WORKER_FILENAME in ray_constants.py constexpr char kSetupWorkerFilename[] = "setup_worker.py"; /// The version of Ray