mirror of
https://github.com/vale981/ray
synced 2025-03-05 10:01:43 -05:00
[Dashboard] Agent in minimal ray installation (#21817)
This is the second part of https://docs.google.com/document/d/12qP3x5uaqZSKS-A_kK0ylPOp0E02_l-deAbmm8YtdFw/edit#. After this PR, dashboard agents will fully work with minimal ray installation. Note that this PR requires to introduce "aioredis", "frozenlist", and "aiosignal" to the minimal installation. These dependencies are very small (or will be removed soon), and including them to minimal makes thing very easy. Please see the below for the reasoning.
This commit is contained in:
parent
0c33ff718d
commit
e62c0052a0
26 changed files with 282 additions and 149 deletions
|
@ -244,6 +244,12 @@
|
|||
python/ray/tests/test_output
|
||||
- bazel test --test_output=streamed --config=ci --test_env=RAY_MINIMAL=1 $(./scripts/bazel_export_options)
|
||||
python/ray/tests/test_runtime_env_ray_minimal
|
||||
- bazel test --test_output=streamed --config=ci $(./scripts/bazel_export_options)
|
||||
python/ray/tests/test_runtime_env
|
||||
- bazel test --test_output=streamed --config=ci $(./scripts/bazel_export_options)
|
||||
python/ray/tests/test_runtime_env_complicated
|
||||
- bazel test --test_output=streamed --config=ci $(./scripts/bazel_export_options)
|
||||
python/ray/tests/test_runtime_env_validation
|
||||
- bazel test --test_output=streamed --config=ci --test_env=RAY_MINIMAL=1 $(./scripts/bazel_export_options)
|
||||
python/ray/tests/test_serve_ray_minimal
|
||||
|
||||
|
|
|
@ -12,7 +12,6 @@ from typing import List
|
|||
DEFAULT_BLACKLIST = [
|
||||
"aiohttp",
|
||||
"aiohttp_cors",
|
||||
"aioredis",
|
||||
"colorful",
|
||||
"py-spy",
|
||||
# "requests",
|
||||
|
|
|
@ -13,13 +13,11 @@ try:
|
|||
from grpc import aio as aiogrpc
|
||||
except ImportError:
|
||||
from grpc.experimental import aio as aiogrpc
|
||||
from distutils.version import LooseVersion
|
||||
|
||||
import ray
|
||||
import ray.experimental.internal_kv as internal_kv
|
||||
import ray.dashboard.consts as dashboard_consts
|
||||
import ray.dashboard.utils as dashboard_utils
|
||||
import ray.dashboard.optional_utils as dashboard_optional_utils
|
||||
import ray.ray_constants as ray_constants
|
||||
import ray._private.services
|
||||
import ray._private.utils
|
||||
|
@ -30,11 +28,6 @@ from ray.core.generated import agent_manager_pb2
|
|||
from ray.core.generated import agent_manager_pb2_grpc
|
||||
from ray._private.ray_logging import setup_component_logger
|
||||
|
||||
# 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 aiohttp, aiohttp_cors, hdrs
|
||||
|
||||
# Import psutil after ray so the packaged version is used.
|
||||
import psutil
|
||||
|
||||
|
@ -44,7 +37,6 @@ except AttributeError:
|
|||
create_task = asyncio.ensure_future
|
||||
|
||||
logger = logging.getLogger(__name__)
|
||||
routes = dashboard_optional_utils.ClassMethodRouteTable
|
||||
|
||||
aiogrpc.init_grpc_aio()
|
||||
|
||||
|
@ -55,6 +47,7 @@ class DashboardAgent(object):
|
|||
redis_address,
|
||||
dashboard_agent_port,
|
||||
gcs_address,
|
||||
minimal,
|
||||
redis_password=None,
|
||||
temp_dir=None,
|
||||
session_dir=None,
|
||||
|
@ -69,6 +62,7 @@ class DashboardAgent(object):
|
|||
"""Initialize the DashboardAgent object."""
|
||||
# Public attributes are accessible for all agent modules.
|
||||
self.ip = node_ip_address
|
||||
self.minimal = minimal
|
||||
|
||||
if use_gcs_for_bootstrap():
|
||||
assert gcs_address is not None
|
||||
|
@ -107,7 +101,16 @@ class DashboardAgent(object):
|
|||
options = (("grpc.enable_http_proxy", 0), )
|
||||
self.aiogrpc_raylet_channel = ray._private.utils.init_grpc_channel(
|
||||
f"{self.ip}:{self.node_manager_port}", options, asynchronous=True)
|
||||
self.http_session = None
|
||||
|
||||
# If the agent is started as non-minimal version, http server should
|
||||
# be configured to communicate with the dashboard in a head node.
|
||||
self.http_server = None
|
||||
|
||||
async def _configure_http_server(self, modules):
|
||||
from ray.dashboard.http_server_agent import HttpServerAgent
|
||||
http_server = HttpServerAgent(self.ip, self.listen_port)
|
||||
await http_server.start(modules)
|
||||
return http_server
|
||||
|
||||
def _load_modules(self):
|
||||
"""Load dashboard agent modules."""
|
||||
|
@ -118,11 +121,15 @@ class DashboardAgent(object):
|
|||
logger.info("Loading %s: %s",
|
||||
dashboard_utils.DashboardAgentModule.__name__, cls)
|
||||
c = cls(self)
|
||||
dashboard_optional_utils.ClassMethodRouteTable.bind(c)
|
||||
modules.append(c)
|
||||
logger.info("Loaded %d modules.", len(modules))
|
||||
return modules
|
||||
|
||||
@property
|
||||
def http_session(self):
|
||||
assert self.http_server, "Accessing unsupported API in a minimal ray."
|
||||
return self.http_server.http_session
|
||||
|
||||
async def run(self):
|
||||
async def _check_parent():
|
||||
"""Check if raylet is dead and fate-share if it is."""
|
||||
|
@ -158,14 +165,6 @@ class DashboardAgent(object):
|
|||
"Failed to connect to redis at %s", self.redis_address)
|
||||
sys.exit(-1)
|
||||
|
||||
# Create a http session for all modules.
|
||||
# aiohttp<4.0.0 uses a 'loop' variable, aiohttp>=4.0.0 doesn't anymore
|
||||
if LooseVersion(aiohttp.__version__) < LooseVersion("4.0.0"):
|
||||
self.http_session = aiohttp.ClientSession(
|
||||
loop=asyncio.get_event_loop())
|
||||
else:
|
||||
self.http_session = aiohttp.ClientSession()
|
||||
|
||||
# Start a grpc asyncio server.
|
||||
await self.server.start()
|
||||
|
||||
|
@ -177,44 +176,19 @@ class DashboardAgent(object):
|
|||
self.gcs_client = GcsClient(address=self.gcs_address)
|
||||
modules = self._load_modules()
|
||||
|
||||
# Http server should be initialized after all modules loaded.
|
||||
app = aiohttp.web.Application()
|
||||
app.add_routes(routes=routes.bound_routes())
|
||||
|
||||
# Enable CORS on all routes.
|
||||
cors = aiohttp_cors.setup(
|
||||
app,
|
||||
defaults={
|
||||
"*": aiohttp_cors.ResourceOptions(
|
||||
allow_credentials=True,
|
||||
expose_headers="*",
|
||||
allow_methods="*",
|
||||
allow_headers=("Content-Type", "X-Header"),
|
||||
)
|
||||
})
|
||||
for route in list(app.router.routes()):
|
||||
cors.add(route)
|
||||
|
||||
runner = aiohttp.web.AppRunner(app)
|
||||
await runner.setup()
|
||||
site = aiohttp.web.TCPSite(
|
||||
runner, "127.0.0.1"
|
||||
if self.ip == "127.0.0.1" else "0.0.0.0", self.listen_port)
|
||||
await site.start()
|
||||
http_host, http_port, *_ = site._server.sockets[0].getsockname()
|
||||
logger.info("Dashboard agent http address: %s:%s", http_host,
|
||||
http_port)
|
||||
|
||||
# Dump registered http routes.
|
||||
dump_routes = [
|
||||
r for r in app.router.routes() if r.method != hdrs.METH_HEAD
|
||||
]
|
||||
for r in dump_routes:
|
||||
logger.info(r)
|
||||
logger.info("Registered %s routes.", len(dump_routes))
|
||||
# Setup http server if necessary.
|
||||
if not self.minimal:
|
||||
# If the agent is not minimal it should start the http server
|
||||
# to communicate with the dashboard in a head node.
|
||||
# Http server is not started in the minimal version because
|
||||
# it requires additional dependencies that are not
|
||||
# included in the minimal ray package.
|
||||
self.http_server = await self._configure_http_server(modules)
|
||||
|
||||
# Write the dashboard agent port to redis.
|
||||
# TODO: Use async version if performance is an issue
|
||||
# -1 should indicate that http server is not started.
|
||||
http_port = -1 if not self.http_server else self.http_server.http_port
|
||||
internal_kv._internal_kv_put(
|
||||
f"{dashboard_consts.DASHBOARD_AGENT_PORT_PREFIX}{self.node_id}",
|
||||
json.dumps([http_port, self.grpc_port]),
|
||||
|
@ -236,8 +210,9 @@ class DashboardAgent(object):
|
|||
await asyncio.gather(*tasks)
|
||||
|
||||
await self.server.wait_for_termination()
|
||||
# Wait for finish signal.
|
||||
await runner.cleanup()
|
||||
|
||||
if self.http_server:
|
||||
await self.http_server.cleanup()
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
|
@ -356,6 +331,13 @@ if __name__ == "__main__":
|
|||
type=str,
|
||||
default=None,
|
||||
help="Specify the path of the resource directory used by runtime_env.")
|
||||
parser.add_argument(
|
||||
"--minimal",
|
||||
action="store_true",
|
||||
help=(
|
||||
"Minimal agent only contains a subset of features that don't "
|
||||
"require additional dependencies installed when ray is installed "
|
||||
"by `pip install ray[default]`."))
|
||||
|
||||
args = parser.parse_args()
|
||||
try:
|
||||
|
@ -373,6 +355,7 @@ if __name__ == "__main__":
|
|||
args.redis_address,
|
||||
args.dashboard_agent_port,
|
||||
args.gcs_address,
|
||||
args.minimal,
|
||||
redis_password=args.redis_password,
|
||||
temp_dir=args.temp_dir,
|
||||
session_dir=args.session_dir,
|
||||
|
|
|
@ -203,6 +203,13 @@ if __name__ == "__main__":
|
|||
type=str,
|
||||
default=None,
|
||||
help="Specify the path of the temporary directory use by Ray process.")
|
||||
parser.add_argument(
|
||||
"--minimal",
|
||||
action="store_true",
|
||||
help=(
|
||||
"Minimal dashboard only contains a subset of features that don't "
|
||||
"require additional dependencies installed when ray is installed "
|
||||
"by `pip install ray[default]`."))
|
||||
|
||||
args = parser.parse_args()
|
||||
|
||||
|
|
75
dashboard/http_server_agent.py
Normal file
75
dashboard/http_server_agent.py
Normal file
|
@ -0,0 +1,75 @@
|
|||
import asyncio
|
||||
import logging
|
||||
|
||||
from distutils.version import LooseVersion
|
||||
|
||||
import ray.dashboard.optional_utils as dashboard_optional_utils
|
||||
|
||||
from ray.dashboard.optional_deps import aiohttp, aiohttp_cors, hdrs
|
||||
|
||||
logger = logging.getLogger(__name__)
|
||||
routes = dashboard_optional_utils.ClassMethodRouteTable
|
||||
|
||||
|
||||
class HttpServerAgent:
|
||||
def __init__(self, ip, listen_port):
|
||||
self.ip = ip
|
||||
self.listen_port = listen_port
|
||||
self.http_host = None
|
||||
self.http_port = None
|
||||
self.http_session = None
|
||||
self.runner = None
|
||||
|
||||
# Create a http session for all modules.
|
||||
# aiohttp<4.0.0 uses a 'loop' variable, aiohttp>=4.0.0 doesn't anymore
|
||||
if LooseVersion(aiohttp.__version__) < LooseVersion("4.0.0"):
|
||||
self.http_session = aiohttp.ClientSession(
|
||||
loop=asyncio.get_event_loop())
|
||||
else:
|
||||
self.http_session = aiohttp.ClientSession()
|
||||
|
||||
async def start(self, modules):
|
||||
# Bind routes for every module so that each module
|
||||
# can use decorator-style routes.
|
||||
for c in modules:
|
||||
dashboard_optional_utils.ClassMethodRouteTable.bind(c)
|
||||
|
||||
app = aiohttp.web.Application()
|
||||
app.add_routes(routes=routes.bound_routes())
|
||||
|
||||
# Enable CORS on all routes.
|
||||
cors = aiohttp_cors.setup(
|
||||
app,
|
||||
defaults={
|
||||
"*": aiohttp_cors.ResourceOptions(
|
||||
allow_credentials=True,
|
||||
expose_headers="*",
|
||||
allow_methods="*",
|
||||
allow_headers=("Content-Type", "X-Header"),
|
||||
)
|
||||
})
|
||||
for route in list(app.router.routes()):
|
||||
cors.add(route)
|
||||
|
||||
self.runner = aiohttp.web.AppRunner(app)
|
||||
await self.runner.setup()
|
||||
site = aiohttp.web.TCPSite(
|
||||
self.runner, "127.0.0.1"
|
||||
if self.ip == "127.0.0.1" else "0.0.0.0", self.listen_port)
|
||||
await site.start()
|
||||
self.http_host, self.http_port, *_ = (
|
||||
site._server.sockets[0].getsockname())
|
||||
logger.info("Dashboard agent http address: %s:%s", self.http_host,
|
||||
self.http_port)
|
||||
|
||||
# Dump registered http routes.
|
||||
dump_routes = [
|
||||
r for r in app.router.routes() if r.method != hdrs.METH_HEAD
|
||||
]
|
||||
for r in dump_routes:
|
||||
logger.info(r)
|
||||
logger.info("Registered %s routes.", len(dump_routes))
|
||||
|
||||
async def cleanup(self):
|
||||
# Wait for finish signal.
|
||||
await self.runner.cleanup()
|
|
@ -191,7 +191,7 @@ class ActorHead(dashboard_utils.DashboardHeadModule):
|
|||
@routes.get("/logical/actors")
|
||||
@dashboard_optional_utils.aiohttp_cache
|
||||
async def get_all_actors(self, req) -> aiohttp.web.Response:
|
||||
return dashboard_optional_utils.rest_response(
|
||||
return rest_response(
|
||||
success=True,
|
||||
message="All actors fetched.",
|
||||
actors=DataSource.actors)
|
||||
|
@ -230,3 +230,7 @@ class ActorHead(dashboard_utils.DashboardHeadModule):
|
|||
gcs_service_pb2_grpc.ActorInfoGcsServiceStub(gcs_channel)
|
||||
|
||||
await asyncio.gather(self._update_actors())
|
||||
|
||||
@staticmethod
|
||||
def is_minimal_module():
|
||||
return False
|
||||
|
|
|
@ -7,7 +7,6 @@ import ray.experimental.internal_kv as internal_kv
|
|||
import ray.ray_constants as ray_constants
|
||||
import ray._private.utils as utils
|
||||
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, create_task
|
||||
from ray.dashboard.modules.event import event_consts
|
||||
|
@ -16,7 +15,6 @@ from ray.core.generated import event_pb2
|
|||
from ray.core.generated import event_pb2_grpc
|
||||
|
||||
logger = logging.getLogger(__name__)
|
||||
routes = dashboard_optional_utils.ClassMethodRouteTable
|
||||
|
||||
|
||||
class EventAgent(dashboard_utils.DashboardAgentModule):
|
||||
|
@ -91,3 +89,7 @@ class EventAgent(dashboard_utils.DashboardAgentModule):
|
|||
source_types=event_consts.EVENT_AGENT_MONITOR_SOURCE_TYPES)
|
||||
# Start reporting events.
|
||||
await self.report_events()
|
||||
|
||||
@staticmethod
|
||||
def is_minimal_module():
|
||||
return False
|
||||
|
|
|
@ -85,3 +85,7 @@ class EventHead(dashboard_utils.DashboardHeadModule,
|
|||
self._event_dir,
|
||||
lambda data: self._update_events(parse_event_strings(data)),
|
||||
source_types=event_consts.EVENT_HEAD_MONITOR_SOURCE_TYPES)
|
||||
|
||||
@staticmethod
|
||||
def is_minimal_module():
|
||||
return False
|
||||
|
|
|
@ -239,3 +239,7 @@ class JobHead(dashboard_utils.DashboardHeadModule):
|
|||
async def run(self, server):
|
||||
if not self._job_manager:
|
||||
self._job_manager = JobManager()
|
||||
|
||||
@staticmethod
|
||||
def is_minimal_module():
|
||||
return False
|
||||
|
|
|
@ -90,14 +90,12 @@ class TestSubmit:
|
|||
["submit", "--address=arg_addr", "--", "echo hello"])
|
||||
assert mock_sdk_client.called_with("arg_addr")
|
||||
assert result.exit_code == 0
|
||||
|
||||
# Test passing address via env var.
|
||||
with set_env_var("RAY_ADDRESS", "env_addr"):
|
||||
result = runner.invoke(job_cli_group,
|
||||
["submit", "--", "echo hello"])
|
||||
assert result.exit_code == 0
|
||||
assert mock_sdk_client.called_with("env_addr")
|
||||
|
||||
# Test passing no address.
|
||||
result = runner.invoke(job_cli_group, ["submit", "--", "echo hello"])
|
||||
assert result.exit_code == 1
|
||||
|
|
|
@ -16,3 +16,7 @@ class LogAgent(dashboard_utils.DashboardAgentModule):
|
|||
|
||||
async def run(self, server):
|
||||
pass
|
||||
|
||||
@staticmethod
|
||||
def is_minimal_module():
|
||||
return False
|
||||
|
|
|
@ -94,3 +94,7 @@ class LogHead(dashboard_utils.DashboardHeadModule):
|
|||
|
||||
async def run(self, server):
|
||||
pass
|
||||
|
||||
@staticmethod
|
||||
def is_minimal_module():
|
||||
return False
|
||||
|
|
|
@ -342,3 +342,7 @@ class NodeHead(dashboard_utils.DashboardHeadModule):
|
|||
await asyncio.gather(self._update_nodes(), self._update_node_stats(),
|
||||
self._update_log_info(),
|
||||
self._update_error_info())
|
||||
|
||||
@staticmethod
|
||||
def is_minimal_module():
|
||||
return False
|
||||
|
|
|
@ -589,3 +589,7 @@ class ReporterAgent(dashboard_utils.DashboardAgentModule,
|
|||
await aioredis_client.publish(key, data)
|
||||
|
||||
await self._perform_iteration(publish)
|
||||
|
||||
@staticmethod
|
||||
def is_minimal_module():
|
||||
return False
|
||||
|
|
|
@ -167,3 +167,7 @@ class ReportHead(dashboard_utils.DashboardHeadModule):
|
|||
except Exception:
|
||||
logger.exception("Error receiving node physical stats "
|
||||
"from reporter agent.")
|
||||
|
||||
@staticmethod
|
||||
def is_minimal_module():
|
||||
return False
|
||||
|
|
|
@ -253,3 +253,7 @@ class RuntimeEnvAgent(dashboard_utils.DashboardAgentModule,
|
|||
async def run(self, server):
|
||||
runtime_env_agent_pb2_grpc.add_RuntimeEnvServiceServicer_to_server(
|
||||
self, server)
|
||||
|
||||
@staticmethod
|
||||
def is_minimal_module():
|
||||
return True
|
||||
|
|
|
@ -212,3 +212,7 @@ class APIHead(dashboard_utils.DashboardHeadModule):
|
|||
self._gcs_actor_info_stub = \
|
||||
gcs_service_pb2_grpc.ActorInfoGcsServiceStub(
|
||||
self._dashboard_head.aiogrpc_gcs_channel)
|
||||
|
||||
@staticmethod
|
||||
def is_minimal_module():
|
||||
return False
|
||||
|
|
|
@ -18,6 +18,10 @@ class TestAgent(dashboard_utils.DashboardAgentModule):
|
|||
def __init__(self, dashboard_agent):
|
||||
super().__init__(dashboard_agent)
|
||||
|
||||
@staticmethod
|
||||
def is_minimal_module():
|
||||
return False
|
||||
|
||||
@routes.get("/test/http_get_from_agent")
|
||||
async def get_url(self, req) -> aiohttp.web.Response:
|
||||
url = req.query.get("url")
|
||||
|
|
|
@ -30,6 +30,10 @@ class TestHead(dashboard_utils.DashboardHeadModule):
|
|||
ip, ports = change.new
|
||||
self._notified_agents[ip] = ports
|
||||
|
||||
@staticmethod
|
||||
def is_minimal_module():
|
||||
return False
|
||||
|
||||
@routes.get("/test/route_get")
|
||||
async def route_get(self, req) -> aiohttp.web.Response:
|
||||
pass
|
||||
|
|
|
@ -230,3 +230,7 @@ class TuneController(dashboard_utils.DashboardHeadModule):
|
|||
async def run(self, server):
|
||||
# Forever loop the collection process
|
||||
await self.collect()
|
||||
|
||||
@staticmethod
|
||||
def is_minimal_module():
|
||||
return False
|
||||
|
|
|
@ -5,12 +5,8 @@ import opencensus # noqa: F401
|
|||
import prometheus_client # noqa: F401
|
||||
|
||||
import aiohttp # noqa: F401
|
||||
import aiosignal # noqa: F401
|
||||
import aiohttp.web # noqa: F401
|
||||
import aiohttp_cors # noqa: F401
|
||||
from aiohttp import hdrs # noqa: F401
|
||||
from frozenlist import FrozenList # noqa: F401
|
||||
from aiohttp.typedefs import PathLike # noqa: F401
|
||||
from aiohttp.web import RouteDef # noqa: F401
|
||||
|
||||
import aioredis # noqa: F401
|
||||
|
|
|
@ -18,7 +18,8 @@ import aiosignal # noqa: F401
|
|||
from google.protobuf.json_format import MessageToDict
|
||||
from frozenlist import FrozenList # noqa: F401
|
||||
|
||||
from ray._private.utils import binary_to_hex
|
||||
from ray._private.utils import (binary_to_hex,
|
||||
check_dashboard_dependencies_installed)
|
||||
|
||||
try:
|
||||
create_task = asyncio.create_task
|
||||
|
@ -44,6 +45,15 @@ class DashboardAgentModule(abc.ABC):
|
|||
:param server: Asyncio GRPC server.
|
||||
"""
|
||||
|
||||
@staticmethod
|
||||
@abc.abstractclassmethod
|
||||
def is_minimal_module():
|
||||
"""
|
||||
Return True if the module is minimal, meaning it
|
||||
should work with `pip install ray` that doesn't requires additonal
|
||||
dependencies.
|
||||
"""
|
||||
|
||||
|
||||
class DashboardHeadModule(abc.ABC):
|
||||
def __init__(self, dashboard_head):
|
||||
|
@ -61,6 +71,15 @@ class DashboardHeadModule(abc.ABC):
|
|||
:param server: Asyncio GRPC server.
|
||||
"""
|
||||
|
||||
@staticmethod
|
||||
@abc.abstractclassmethod
|
||||
def is_minimal_module():
|
||||
"""
|
||||
Return True if the module is minimal, meaning it
|
||||
should work with `pip install ray` that doesn't requires additonal
|
||||
dependencies.
|
||||
"""
|
||||
|
||||
|
||||
def dashboard_module(enable):
|
||||
"""A decorator for dashboard module."""
|
||||
|
@ -78,15 +97,36 @@ def get_all_modules(module_type):
|
|||
"""
|
||||
logger.info(f"Get all modules by type: {module_type.__name__}")
|
||||
import ray.dashboard.modules
|
||||
should_only_load_minimal_modules = (
|
||||
not check_dashboard_dependencies_installed())
|
||||
|
||||
for module_loader, name, ispkg in pkgutil.walk_packages(
|
||||
ray.dashboard.modules.__path__,
|
||||
ray.dashboard.modules.__name__ + "."):
|
||||
importlib.import_module(name)
|
||||
return [
|
||||
m for m in module_type.__subclasses__()
|
||||
if getattr(m, "__ray_dashboard_module_enable__", True)
|
||||
]
|
||||
try:
|
||||
importlib.import_module(name)
|
||||
except ModuleNotFoundError as e:
|
||||
logger.info(f"Module {name} cannot be loaded because "
|
||||
"we cannot import all dependencies. Download "
|
||||
"`pip install ray[default]` for the full "
|
||||
f"dashboard functionality. Error: {e}")
|
||||
if not should_only_load_minimal_modules:
|
||||
logger.info(
|
||||
"Although `pip install ray[default] is downloaded, "
|
||||
"module couldn't be imported`")
|
||||
raise e
|
||||
|
||||
imported_modules = []
|
||||
# module_type.__subclasses__() should contain modules that
|
||||
# we could successfully import.
|
||||
for m in module_type.__subclasses__():
|
||||
if not getattr(m, "__ray_dashboard_module_enable__", True):
|
||||
continue
|
||||
if should_only_load_minimal_modules and not m.is_minimal_module():
|
||||
continue
|
||||
imported_modules.append(m)
|
||||
logger.info(f"Available modules: {imported_modules}")
|
||||
return imported_modules
|
||||
|
||||
|
||||
def to_posix_time(dt):
|
||||
|
|
|
@ -1309,9 +1309,7 @@ def start_dashboard(require_dashboard,
|
|||
raise e
|
||||
|
||||
# Make sure the process can start.
|
||||
try:
|
||||
import ray.dashboard.optional_deps # noqa: F401
|
||||
except ImportError:
|
||||
if not ray._private.utils.check_dashboard_dependencies_installed():
|
||||
if require_dashboard:
|
||||
logger.exception("dashboard dependency error")
|
||||
raise ImportError(DASHBOARD_DEPENDENCY_ERROR_MESSAGE)
|
||||
|
@ -1650,41 +1648,43 @@ def start_raylet(redis_address,
|
|||
if max_worker_port is None:
|
||||
max_worker_port = 0
|
||||
|
||||
if not ray._private.utils.check_dashboard_dependencies_installed():
|
||||
# An empty agent command will cause the raylet not to start it.
|
||||
agent_command = []
|
||||
else:
|
||||
agent_command = [
|
||||
sys.executable,
|
||||
"-u",
|
||||
os.path.join(RAY_PATH, "dashboard", "agent.py"),
|
||||
f"--node-ip-address={node_ip_address}",
|
||||
f"--redis-address={redis_address}",
|
||||
f"--metrics-export-port={metrics_export_port}",
|
||||
f"--dashboard-agent-port={metrics_agent_port}",
|
||||
f"--listen-port={dashboard_agent_listen_port}",
|
||||
"--node-manager-port=RAY_NODE_MANAGER_PORT_PLACEHOLDER",
|
||||
f"--object-store-name={plasma_store_name}",
|
||||
f"--raylet-name={raylet_name}",
|
||||
f"--temp-dir={temp_dir}",
|
||||
f"--session-dir={session_dir}",
|
||||
f"--runtime-env-dir={resource_dir}",
|
||||
f"--log-dir={log_dir}",
|
||||
f"--logging-rotate-bytes={max_bytes}",
|
||||
f"--logging-rotate-backup-count={backup_count}",
|
||||
f"--gcs-address={gcs_address}",
|
||||
]
|
||||
if stdout_file is None and stderr_file is None:
|
||||
# If not redirecting logging to files, unset log filename.
|
||||
# This will cause log records to go to stderr.
|
||||
agent_command.append("--logging-filename=")
|
||||
# Use stderr log format with the component name as a message prefix.
|
||||
logging_format = ray_constants.LOGGER_FORMAT_STDERR.format(
|
||||
component=ray_constants.PROCESS_TYPE_DASHBOARD_AGENT)
|
||||
agent_command.append(f"--logging-format={logging_format}")
|
||||
agent_command = [
|
||||
sys.executable,
|
||||
"-u",
|
||||
os.path.join(RAY_PATH, "dashboard", "agent.py"),
|
||||
f"--node-ip-address={node_ip_address}",
|
||||
f"--redis-address={redis_address}",
|
||||
f"--metrics-export-port={metrics_export_port}",
|
||||
f"--dashboard-agent-port={metrics_agent_port}",
|
||||
f"--listen-port={dashboard_agent_listen_port}",
|
||||
"--node-manager-port=RAY_NODE_MANAGER_PORT_PLACEHOLDER",
|
||||
f"--object-store-name={plasma_store_name}",
|
||||
f"--raylet-name={raylet_name}",
|
||||
f"--temp-dir={temp_dir}",
|
||||
f"--session-dir={session_dir}",
|
||||
f"--runtime-env-dir={resource_dir}",
|
||||
f"--log-dir={log_dir}",
|
||||
f"--logging-rotate-bytes={max_bytes}",
|
||||
f"--logging-rotate-backup-count={backup_count}",
|
||||
f"--gcs-address={gcs_address}",
|
||||
]
|
||||
if stdout_file is None and stderr_file is None:
|
||||
# If not redirecting logging to files, unset log filename.
|
||||
# This will cause log records to go to stderr.
|
||||
agent_command.append("--logging-filename=")
|
||||
# Use stderr log format with the component name as a message prefix.
|
||||
logging_format = ray_constants.LOGGER_FORMAT_STDERR.format(
|
||||
component=ray_constants.PROCESS_TYPE_DASHBOARD_AGENT)
|
||||
agent_command.append(f"--logging-format={logging_format}")
|
||||
|
||||
if redis_password is not None and len(redis_password) != 0:
|
||||
agent_command.append("--redis-password={}".format(redis_password))
|
||||
if redis_password is not None and len(redis_password) != 0:
|
||||
agent_command.append("--redis-password={}".format(redis_password))
|
||||
|
||||
if not ray._private.utils.check_dashboard_dependencies_installed():
|
||||
# If dependencies are not installed, it is the minimally packaged
|
||||
# ray. We should restrict the features within dashboard agent
|
||||
# that requires additional dependencies to be downloaded.
|
||||
agent_command.append("--minimal")
|
||||
|
||||
command = [
|
||||
RAYLET_EXECUTABLE,
|
||||
|
|
|
@ -14,32 +14,23 @@ import sys
|
|||
import pytest
|
||||
|
||||
import ray
|
||||
from ray.exceptions import RuntimeEnvSetupError
|
||||
from ray._private.test_utils import wait_for_condition
|
||||
|
||||
|
||||
def _test_task_and_actor(capsys):
|
||||
@ray.remote
|
||||
def f():
|
||||
pass
|
||||
return 1
|
||||
|
||||
with pytest.raises(RuntimeEnvSetupError):
|
||||
ray.get(f.options(runtime_env={"pip": ["requests"]}).remote())
|
||||
|
||||
def stderr_checker():
|
||||
captured = capsys.readouterr()
|
||||
return "ray[default]" in captured.err
|
||||
|
||||
wait_for_condition(stderr_checker)
|
||||
# with pytest.raises(RuntimeEnvSetupError):
|
||||
assert ray.get(f.options(runtime_env={"pip": ["requests"]}).remote()) == 1
|
||||
|
||||
@ray.remote
|
||||
class A:
|
||||
def task(self):
|
||||
pass
|
||||
return 1
|
||||
|
||||
A.options(runtime_env={"pip": ["requests"]}).remote()
|
||||
|
||||
wait_for_condition(stderr_checker)
|
||||
a = A.options(runtime_env={"pip": ["requests"]}).remote()
|
||||
assert ray.get(a.task.remote()) == 1
|
||||
|
||||
|
||||
@pytest.mark.skipif(
|
||||
|
@ -72,20 +63,13 @@ def test_task_actor(shutdown_only, capsys):
|
|||
os.environ.get("RAY_MINIMAL") != "1",
|
||||
reason="This test is only run in CI with a minimal Ray installation.")
|
||||
def test_ray_init(shutdown_only, capsys):
|
||||
with pytest.raises(RuntimeEnvSetupError):
|
||||
ray.init(runtime_env={"pip": ["requests"]})
|
||||
ray.init(runtime_env={"pip": ["requests"]})
|
||||
|
||||
@ray.remote
|
||||
def f():
|
||||
pass
|
||||
@ray.remote
|
||||
def f():
|
||||
return 1
|
||||
|
||||
ray.get(f.remote())
|
||||
|
||||
def stderr_checker():
|
||||
captured = capsys.readouterr()
|
||||
return "ray[default]" in captured.err
|
||||
|
||||
wait_for_condition(stderr_checker)
|
||||
assert ray.get(f.remote()) == 1
|
||||
|
||||
|
||||
@pytest.mark.skipif(
|
||||
|
@ -98,9 +82,7 @@ def test_ray_init(shutdown_only, capsys):
|
|||
["ray start --head --ray-client-server-port 25552 --port 0"],
|
||||
indirect=True)
|
||||
def test_ray_client_init(call_ray_start):
|
||||
with pytest.raises(ConnectionAbortedError) as excinfo:
|
||||
ray.init("ray://localhost:25552", runtime_env={"pip": ["requests"]})
|
||||
assert "ray[default]" in str(excinfo.value)
|
||||
ray.init("ray://localhost:25552", runtime_env={"pip": ["requests"]})
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
|
|
|
@ -30,8 +30,7 @@ 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, use_gcs_for_bootstrap
|
||||
from ray._private.utils import (detect_fate_sharing_support,
|
||||
check_dashboard_dependencies_installed)
|
||||
from ray._private.utils import (detect_fate_sharing_support)
|
||||
|
||||
# Import psutil after ray so the packaged version is used.
|
||||
import psutil
|
||||
|
@ -212,12 +211,6 @@ class ProxyManager():
|
|||
Includes retry logic to handle the case when the agent is
|
||||
temporarily unreachable (e.g., hasn't been started up yet).
|
||||
"""
|
||||
if not check_dashboard_dependencies_installed():
|
||||
raise RuntimeError("Not all required Ray dependencies for the "
|
||||
"runtime_env feature were found on the "
|
||||
"cluster. To install the required "
|
||||
"dependencies, please run `pip install "
|
||||
"\"ray[default]\"` on all cluster nodes.")
|
||||
create_env_request = runtime_env_agent_pb2.CreateRuntimeEnvRequest(
|
||||
serialized_runtime_env=serialized_runtime_env,
|
||||
job_id=f"ray_client_server_{specific_server.port}".encode("utf-8"))
|
||||
|
|
|
@ -199,11 +199,8 @@ if setup_spec.type == SetupType.RAY:
|
|||
],
|
||||
"default": [
|
||||
"aiohttp >= 3.7",
|
||||
"aiosignal",
|
||||
"aiohttp_cors",
|
||||
"aioredis < 2",
|
||||
"colorful",
|
||||
"frozenlist",
|
||||
"py-spy >= 0.2.0",
|
||||
"requests",
|
||||
"gpustat >= 1.0.0b1", # for windows
|
||||
|
@ -270,6 +267,9 @@ if setup_spec.type == SetupType.RAY:
|
|||
"protobuf >= 3.15.3",
|
||||
"pyyaml",
|
||||
"redis >= 3.5.0",
|
||||
"aioredis < 2",
|
||||
"aiosignal",
|
||||
"frozenlist",
|
||||
]
|
||||
|
||||
|
||||
|
|
Loading…
Add table
Reference in a new issue