[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:
SangBin Cho 2022-01-26 21:03:54 +09:00 committed by GitHub
parent 0c33ff718d
commit e62c0052a0
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
26 changed files with 282 additions and 149 deletions

View file

@ -244,6 +244,12 @@
python/ray/tests/test_output python/ray/tests/test_output
- bazel test --test_output=streamed --config=ci --test_env=RAY_MINIMAL=1 $(./scripts/bazel_export_options) - bazel test --test_output=streamed --config=ci --test_env=RAY_MINIMAL=1 $(./scripts/bazel_export_options)
python/ray/tests/test_runtime_env_ray_minimal 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) - bazel test --test_output=streamed --config=ci --test_env=RAY_MINIMAL=1 $(./scripts/bazel_export_options)
python/ray/tests/test_serve_ray_minimal python/ray/tests/test_serve_ray_minimal

View file

@ -12,7 +12,6 @@ from typing import List
DEFAULT_BLACKLIST = [ DEFAULT_BLACKLIST = [
"aiohttp", "aiohttp",
"aiohttp_cors", "aiohttp_cors",
"aioredis",
"colorful", "colorful",
"py-spy", "py-spy",
# "requests", # "requests",

View file

@ -13,13 +13,11 @@ try:
from grpc import aio as aiogrpc from grpc import aio as aiogrpc
except ImportError: except ImportError:
from grpc.experimental import aio as aiogrpc from grpc.experimental import aio as aiogrpc
from distutils.version import LooseVersion
import ray import ray
import ray.experimental.internal_kv as internal_kv import ray.experimental.internal_kv as internal_kv
import ray.dashboard.consts as dashboard_consts import ray.dashboard.consts as dashboard_consts
import ray.dashboard.utils as dashboard_utils 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.ray_constants as ray_constants
import ray._private.services import ray._private.services
import ray._private.utils 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.core.generated import agent_manager_pb2_grpc
from ray._private.ray_logging import setup_component_logger 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 after ray so the packaged version is used.
import psutil import psutil
@ -44,7 +37,6 @@ except AttributeError:
create_task = asyncio.ensure_future create_task = asyncio.ensure_future
logger = logging.getLogger(__name__) logger = logging.getLogger(__name__)
routes = dashboard_optional_utils.ClassMethodRouteTable
aiogrpc.init_grpc_aio() aiogrpc.init_grpc_aio()
@ -55,6 +47,7 @@ class DashboardAgent(object):
redis_address, redis_address,
dashboard_agent_port, dashboard_agent_port,
gcs_address, gcs_address,
minimal,
redis_password=None, redis_password=None,
temp_dir=None, temp_dir=None,
session_dir=None, session_dir=None,
@ -69,6 +62,7 @@ class DashboardAgent(object):
"""Initialize the DashboardAgent object.""" """Initialize the DashboardAgent object."""
# Public attributes are accessible for all agent modules. # Public attributes are accessible for all agent modules.
self.ip = node_ip_address self.ip = node_ip_address
self.minimal = minimal
if use_gcs_for_bootstrap(): if use_gcs_for_bootstrap():
assert gcs_address is not None assert gcs_address is not None
@ -107,7 +101,16 @@ class DashboardAgent(object):
options = (("grpc.enable_http_proxy", 0), ) options = (("grpc.enable_http_proxy", 0), )
self.aiogrpc_raylet_channel = ray._private.utils.init_grpc_channel( self.aiogrpc_raylet_channel = ray._private.utils.init_grpc_channel(
f"{self.ip}:{self.node_manager_port}", options, asynchronous=True) 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): def _load_modules(self):
"""Load dashboard agent modules.""" """Load dashboard agent modules."""
@ -118,11 +121,15 @@ class DashboardAgent(object):
logger.info("Loading %s: %s", logger.info("Loading %s: %s",
dashboard_utils.DashboardAgentModule.__name__, cls) dashboard_utils.DashboardAgentModule.__name__, cls)
c = cls(self) c = cls(self)
dashboard_optional_utils.ClassMethodRouteTable.bind(c)
modules.append(c) modules.append(c)
logger.info("Loaded %d modules.", len(modules)) logger.info("Loaded %d modules.", len(modules))
return 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 run(self):
async def _check_parent(): async def _check_parent():
"""Check if raylet is dead and fate-share if it is.""" """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) "Failed to connect to redis at %s", self.redis_address)
sys.exit(-1) 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. # Start a grpc asyncio server.
await self.server.start() await self.server.start()
@ -177,44 +176,19 @@ class DashboardAgent(object):
self.gcs_client = GcsClient(address=self.gcs_address) self.gcs_client = GcsClient(address=self.gcs_address)
modules = self._load_modules() modules = self._load_modules()
# Http server should be initialized after all modules loaded. # Setup http server if necessary.
app = aiohttp.web.Application() if not self.minimal:
app.add_routes(routes=routes.bound_routes()) # If the agent is not minimal it should start the http server
# to communicate with the dashboard in a head node.
# Enable CORS on all routes. # Http server is not started in the minimal version because
cors = aiohttp_cors.setup( # it requires additional dependencies that are not
app, # included in the minimal ray package.
defaults={ self.http_server = await self._configure_http_server(modules)
"*": 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))
# Write the dashboard agent port to redis. # Write the dashboard agent port to redis.
# TODO: Use async version if performance is an issue # 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( internal_kv._internal_kv_put(
f"{dashboard_consts.DASHBOARD_AGENT_PORT_PREFIX}{self.node_id}", f"{dashboard_consts.DASHBOARD_AGENT_PORT_PREFIX}{self.node_id}",
json.dumps([http_port, self.grpc_port]), json.dumps([http_port, self.grpc_port]),
@ -236,8 +210,9 @@ class DashboardAgent(object):
await asyncio.gather(*tasks) await asyncio.gather(*tasks)
await self.server.wait_for_termination() 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__": if __name__ == "__main__":
@ -356,6 +331,13 @@ if __name__ == "__main__":
type=str, type=str,
default=None, default=None,
help="Specify the path of the resource directory used by runtime_env.") 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() args = parser.parse_args()
try: try:
@ -373,6 +355,7 @@ if __name__ == "__main__":
args.redis_address, args.redis_address,
args.dashboard_agent_port, args.dashboard_agent_port,
args.gcs_address, args.gcs_address,
args.minimal,
redis_password=args.redis_password, redis_password=args.redis_password,
temp_dir=args.temp_dir, temp_dir=args.temp_dir,
session_dir=args.session_dir, session_dir=args.session_dir,

View file

@ -203,6 +203,13 @@ if __name__ == "__main__":
type=str, type=str,
default=None, default=None,
help="Specify the path of the temporary directory use by Ray process.") 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() args = parser.parse_args()

View 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()

View file

@ -191,7 +191,7 @@ class ActorHead(dashboard_utils.DashboardHeadModule):
@routes.get("/logical/actors") @routes.get("/logical/actors")
@dashboard_optional_utils.aiohttp_cache @dashboard_optional_utils.aiohttp_cache
async def get_all_actors(self, req) -> aiohttp.web.Response: async def get_all_actors(self, req) -> aiohttp.web.Response:
return dashboard_optional_utils.rest_response( return rest_response(
success=True, success=True,
message="All actors fetched.", message="All actors fetched.",
actors=DataSource.actors) actors=DataSource.actors)
@ -230,3 +230,7 @@ class ActorHead(dashboard_utils.DashboardHeadModule):
gcs_service_pb2_grpc.ActorInfoGcsServiceStub(gcs_channel) gcs_service_pb2_grpc.ActorInfoGcsServiceStub(gcs_channel)
await asyncio.gather(self._update_actors()) await asyncio.gather(self._update_actors())
@staticmethod
def is_minimal_module():
return False

View file

@ -7,7 +7,6 @@ import ray.experimental.internal_kv as internal_kv
import ray.ray_constants as ray_constants import ray.ray_constants as ray_constants
import ray._private.utils as utils import ray._private.utils as utils
import ray.dashboard.utils as dashboard_utils import ray.dashboard.utils as dashboard_utils
import ray.dashboard.optional_utils as dashboard_optional_utils
import ray.dashboard.consts as dashboard_consts import ray.dashboard.consts as dashboard_consts
from ray.dashboard.utils import async_loop_forever, create_task from ray.dashboard.utils import async_loop_forever, create_task
from ray.dashboard.modules.event import event_consts 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 from ray.core.generated import event_pb2_grpc
logger = logging.getLogger(__name__) logger = logging.getLogger(__name__)
routes = dashboard_optional_utils.ClassMethodRouteTable
class EventAgent(dashboard_utils.DashboardAgentModule): class EventAgent(dashboard_utils.DashboardAgentModule):
@ -91,3 +89,7 @@ class EventAgent(dashboard_utils.DashboardAgentModule):
source_types=event_consts.EVENT_AGENT_MONITOR_SOURCE_TYPES) source_types=event_consts.EVENT_AGENT_MONITOR_SOURCE_TYPES)
# Start reporting events. # Start reporting events.
await self.report_events() await self.report_events()
@staticmethod
def is_minimal_module():
return False

View file

@ -85,3 +85,7 @@ class EventHead(dashboard_utils.DashboardHeadModule,
self._event_dir, self._event_dir,
lambda data: self._update_events(parse_event_strings(data)), lambda data: self._update_events(parse_event_strings(data)),
source_types=event_consts.EVENT_HEAD_MONITOR_SOURCE_TYPES) source_types=event_consts.EVENT_HEAD_MONITOR_SOURCE_TYPES)
@staticmethod
def is_minimal_module():
return False

View file

@ -239,3 +239,7 @@ class JobHead(dashboard_utils.DashboardHeadModule):
async def run(self, server): async def run(self, server):
if not self._job_manager: if not self._job_manager:
self._job_manager = JobManager() self._job_manager = JobManager()
@staticmethod
def is_minimal_module():
return False

View file

@ -90,14 +90,12 @@ class TestSubmit:
["submit", "--address=arg_addr", "--", "echo hello"]) ["submit", "--address=arg_addr", "--", "echo hello"])
assert mock_sdk_client.called_with("arg_addr") assert mock_sdk_client.called_with("arg_addr")
assert result.exit_code == 0 assert result.exit_code == 0
# Test passing address via env var. # Test passing address via env var.
with set_env_var("RAY_ADDRESS", "env_addr"): with set_env_var("RAY_ADDRESS", "env_addr"):
result = runner.invoke(job_cli_group, result = runner.invoke(job_cli_group,
["submit", "--", "echo hello"]) ["submit", "--", "echo hello"])
assert result.exit_code == 0 assert result.exit_code == 0
assert mock_sdk_client.called_with("env_addr") assert mock_sdk_client.called_with("env_addr")
# Test passing no address. # Test passing no address.
result = runner.invoke(job_cli_group, ["submit", "--", "echo hello"]) result = runner.invoke(job_cli_group, ["submit", "--", "echo hello"])
assert result.exit_code == 1 assert result.exit_code == 1

View file

@ -16,3 +16,7 @@ class LogAgent(dashboard_utils.DashboardAgentModule):
async def run(self, server): async def run(self, server):
pass pass
@staticmethod
def is_minimal_module():
return False

View file

@ -94,3 +94,7 @@ class LogHead(dashboard_utils.DashboardHeadModule):
async def run(self, server): async def run(self, server):
pass pass
@staticmethod
def is_minimal_module():
return False

View file

@ -342,3 +342,7 @@ class NodeHead(dashboard_utils.DashboardHeadModule):
await asyncio.gather(self._update_nodes(), self._update_node_stats(), await asyncio.gather(self._update_nodes(), self._update_node_stats(),
self._update_log_info(), self._update_log_info(),
self._update_error_info()) self._update_error_info())
@staticmethod
def is_minimal_module():
return False

View file

@ -589,3 +589,7 @@ class ReporterAgent(dashboard_utils.DashboardAgentModule,
await aioredis_client.publish(key, data) await aioredis_client.publish(key, data)
await self._perform_iteration(publish) await self._perform_iteration(publish)
@staticmethod
def is_minimal_module():
return False

View file

@ -167,3 +167,7 @@ class ReportHead(dashboard_utils.DashboardHeadModule):
except Exception: except Exception:
logger.exception("Error receiving node physical stats " logger.exception("Error receiving node physical stats "
"from reporter agent.") "from reporter agent.")
@staticmethod
def is_minimal_module():
return False

View file

@ -253,3 +253,7 @@ class RuntimeEnvAgent(dashboard_utils.DashboardAgentModule,
async def run(self, server): async def run(self, server):
runtime_env_agent_pb2_grpc.add_RuntimeEnvServiceServicer_to_server( runtime_env_agent_pb2_grpc.add_RuntimeEnvServiceServicer_to_server(
self, server) self, server)
@staticmethod
def is_minimal_module():
return True

View file

@ -212,3 +212,7 @@ class APIHead(dashboard_utils.DashboardHeadModule):
self._gcs_actor_info_stub = \ self._gcs_actor_info_stub = \
gcs_service_pb2_grpc.ActorInfoGcsServiceStub( gcs_service_pb2_grpc.ActorInfoGcsServiceStub(
self._dashboard_head.aiogrpc_gcs_channel) self._dashboard_head.aiogrpc_gcs_channel)
@staticmethod
def is_minimal_module():
return False

View file

@ -18,6 +18,10 @@ class TestAgent(dashboard_utils.DashboardAgentModule):
def __init__(self, dashboard_agent): def __init__(self, dashboard_agent):
super().__init__(dashboard_agent) super().__init__(dashboard_agent)
@staticmethod
def is_minimal_module():
return False
@routes.get("/test/http_get_from_agent") @routes.get("/test/http_get_from_agent")
async def get_url(self, req) -> aiohttp.web.Response: async def get_url(self, req) -> aiohttp.web.Response:
url = req.query.get("url") url = req.query.get("url")

View file

@ -30,6 +30,10 @@ class TestHead(dashboard_utils.DashboardHeadModule):
ip, ports = change.new ip, ports = change.new
self._notified_agents[ip] = ports self._notified_agents[ip] = ports
@staticmethod
def is_minimal_module():
return False
@routes.get("/test/route_get") @routes.get("/test/route_get")
async def route_get(self, req) -> aiohttp.web.Response: async def route_get(self, req) -> aiohttp.web.Response:
pass pass

View file

@ -230,3 +230,7 @@ class TuneController(dashboard_utils.DashboardHeadModule):
async def run(self, server): async def run(self, server):
# Forever loop the collection process # Forever loop the collection process
await self.collect() await self.collect()
@staticmethod
def is_minimal_module():
return False

View file

@ -5,12 +5,8 @@ import opencensus # noqa: F401
import prometheus_client # noqa: F401 import prometheus_client # noqa: F401
import aiohttp # noqa: F401 import aiohttp # noqa: F401
import aiosignal # noqa: F401
import aiohttp.web # noqa: F401 import aiohttp.web # noqa: F401
import aiohttp_cors # noqa: F401 import aiohttp_cors # noqa: F401
from aiohttp import hdrs # noqa: F401 from aiohttp import hdrs # noqa: F401
from frozenlist import FrozenList # noqa: F401
from aiohttp.typedefs import PathLike # noqa: F401 from aiohttp.typedefs import PathLike # noqa: F401
from aiohttp.web import RouteDef # noqa: F401 from aiohttp.web import RouteDef # noqa: F401
import aioredis # noqa: F401

View file

@ -18,7 +18,8 @@ import aiosignal # noqa: F401
from google.protobuf.json_format import MessageToDict from google.protobuf.json_format import MessageToDict
from frozenlist import FrozenList # noqa: F401 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: try:
create_task = asyncio.create_task create_task = asyncio.create_task
@ -44,6 +45,15 @@ class DashboardAgentModule(abc.ABC):
:param server: Asyncio GRPC server. :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): class DashboardHeadModule(abc.ABC):
def __init__(self, dashboard_head): def __init__(self, dashboard_head):
@ -61,6 +71,15 @@ class DashboardHeadModule(abc.ABC):
:param server: Asyncio GRPC server. :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): def dashboard_module(enable):
"""A decorator for dashboard module.""" """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__}") logger.info(f"Get all modules by type: {module_type.__name__}")
import ray.dashboard.modules import ray.dashboard.modules
should_only_load_minimal_modules = (
not check_dashboard_dependencies_installed())
for module_loader, name, ispkg in pkgutil.walk_packages( for module_loader, name, ispkg in pkgutil.walk_packages(
ray.dashboard.modules.__path__, ray.dashboard.modules.__path__,
ray.dashboard.modules.__name__ + "."): ray.dashboard.modules.__name__ + "."):
importlib.import_module(name) try:
return [ importlib.import_module(name)
m for m in module_type.__subclasses__() except ModuleNotFoundError as e:
if getattr(m, "__ray_dashboard_module_enable__", True) 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): def to_posix_time(dt):

View file

@ -1309,9 +1309,7 @@ def start_dashboard(require_dashboard,
raise e raise e
# Make sure the process can start. # Make sure the process can start.
try: if not ray._private.utils.check_dashboard_dependencies_installed():
import ray.dashboard.optional_deps # noqa: F401
except ImportError:
if require_dashboard: if require_dashboard:
logger.exception("dashboard dependency error") logger.exception("dashboard dependency error")
raise ImportError(DASHBOARD_DEPENDENCY_ERROR_MESSAGE) raise ImportError(DASHBOARD_DEPENDENCY_ERROR_MESSAGE)
@ -1650,41 +1648,43 @@ def start_raylet(redis_address,
if max_worker_port is None: if max_worker_port is None:
max_worker_port = 0 max_worker_port = 0
if not ray._private.utils.check_dashboard_dependencies_installed(): agent_command = [
# An empty agent command will cause the raylet not to start it. sys.executable,
agent_command = [] "-u",
else: os.path.join(RAY_PATH, "dashboard", "agent.py"),
agent_command = [ f"--node-ip-address={node_ip_address}",
sys.executable, f"--redis-address={redis_address}",
"-u", f"--metrics-export-port={metrics_export_port}",
os.path.join(RAY_PATH, "dashboard", "agent.py"), f"--dashboard-agent-port={metrics_agent_port}",
f"--node-ip-address={node_ip_address}", f"--listen-port={dashboard_agent_listen_port}",
f"--redis-address={redis_address}", "--node-manager-port=RAY_NODE_MANAGER_PORT_PLACEHOLDER",
f"--metrics-export-port={metrics_export_port}", f"--object-store-name={plasma_store_name}",
f"--dashboard-agent-port={metrics_agent_port}", f"--raylet-name={raylet_name}",
f"--listen-port={dashboard_agent_listen_port}", f"--temp-dir={temp_dir}",
"--node-manager-port=RAY_NODE_MANAGER_PORT_PLACEHOLDER", f"--session-dir={session_dir}",
f"--object-store-name={plasma_store_name}", f"--runtime-env-dir={resource_dir}",
f"--raylet-name={raylet_name}", f"--log-dir={log_dir}",
f"--temp-dir={temp_dir}", f"--logging-rotate-bytes={max_bytes}",
f"--session-dir={session_dir}", f"--logging-rotate-backup-count={backup_count}",
f"--runtime-env-dir={resource_dir}", f"--gcs-address={gcs_address}",
f"--log-dir={log_dir}", ]
f"--logging-rotate-bytes={max_bytes}", if stdout_file is None and stderr_file is None:
f"--logging-rotate-backup-count={backup_count}", # If not redirecting logging to files, unset log filename.
f"--gcs-address={gcs_address}", # This will cause log records to go to stderr.
] agent_command.append("--logging-filename=")
if stdout_file is None and stderr_file is None: # Use stderr log format with the component name as a message prefix.
# If not redirecting logging to files, unset log filename. logging_format = ray_constants.LOGGER_FORMAT_STDERR.format(
# This will cause log records to go to stderr. component=ray_constants.PROCESS_TYPE_DASHBOARD_AGENT)
agent_command.append("--logging-filename=") agent_command.append(f"--logging-format={logging_format}")
# 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: if redis_password is not None and len(redis_password) != 0:
agent_command.append("--redis-password={}".format(redis_password)) 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 = [ command = [
RAYLET_EXECUTABLE, RAYLET_EXECUTABLE,

View file

@ -14,32 +14,23 @@ import sys
import pytest import pytest
import ray import ray
from ray.exceptions import RuntimeEnvSetupError
from ray._private.test_utils import wait_for_condition
def _test_task_and_actor(capsys): def _test_task_and_actor(capsys):
@ray.remote @ray.remote
def f(): def f():
pass return 1
with pytest.raises(RuntimeEnvSetupError): # with pytest.raises(RuntimeEnvSetupError):
ray.get(f.options(runtime_env={"pip": ["requests"]}).remote()) assert ray.get(f.options(runtime_env={"pip": ["requests"]}).remote()) == 1
def stderr_checker():
captured = capsys.readouterr()
return "ray[default]" in captured.err
wait_for_condition(stderr_checker)
@ray.remote @ray.remote
class A: class A:
def task(self): def task(self):
pass return 1
A.options(runtime_env={"pip": ["requests"]}).remote() a = A.options(runtime_env={"pip": ["requests"]}).remote()
assert ray.get(a.task.remote()) == 1
wait_for_condition(stderr_checker)
@pytest.mark.skipif( @pytest.mark.skipif(
@ -72,20 +63,13 @@ def test_task_actor(shutdown_only, capsys):
os.environ.get("RAY_MINIMAL") != "1", os.environ.get("RAY_MINIMAL") != "1",
reason="This test is only run in CI with a minimal Ray installation.") reason="This test is only run in CI with a minimal Ray installation.")
def test_ray_init(shutdown_only, capsys): 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 @ray.remote
def f(): def f():
pass return 1
ray.get(f.remote()) assert ray.get(f.remote()) == 1
def stderr_checker():
captured = capsys.readouterr()
return "ray[default]" in captured.err
wait_for_condition(stderr_checker)
@pytest.mark.skipif( @pytest.mark.skipif(
@ -98,9 +82,7 @@ def test_ray_init(shutdown_only, capsys):
["ray start --head --ray-client-server-port 25552 --port 0"], ["ray start --head --ray-client-server-port 25552 --port 0"],
indirect=True) indirect=True)
def test_ray_client_init(call_ray_start): def test_ray_client_init(call_ray_start):
with pytest.raises(ConnectionAbortedError) as excinfo: ray.init("ray://localhost:25552", runtime_env={"pip": ["requests"]})
ray.init("ray://localhost:25552", runtime_env={"pip": ["requests"]})
assert "ray[default]" in str(excinfo.value)
if __name__ == "__main__": if __name__ == "__main__":

View file

@ -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.services import ProcessInfo, start_ray_client_server
from ray._private.tls_utils import add_port_to_grpc_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.gcs_utils import GcsClient, use_gcs_for_bootstrap
from ray._private.utils import (detect_fate_sharing_support, from ray._private.utils import (detect_fate_sharing_support)
check_dashboard_dependencies_installed)
# Import psutil after ray so the packaged version is used. # Import psutil after ray so the packaged version is used.
import psutil import psutil
@ -212,12 +211,6 @@ class ProxyManager():
Includes retry logic to handle the case when the agent is Includes retry logic to handle the case when the agent is
temporarily unreachable (e.g., hasn't been started up yet). 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( create_env_request = runtime_env_agent_pb2.CreateRuntimeEnvRequest(
serialized_runtime_env=serialized_runtime_env, serialized_runtime_env=serialized_runtime_env,
job_id=f"ray_client_server_{specific_server.port}".encode("utf-8")) job_id=f"ray_client_server_{specific_server.port}".encode("utf-8"))

View file

@ -199,11 +199,8 @@ if setup_spec.type == SetupType.RAY:
], ],
"default": [ "default": [
"aiohttp >= 3.7", "aiohttp >= 3.7",
"aiosignal",
"aiohttp_cors", "aiohttp_cors",
"aioredis < 2",
"colorful", "colorful",
"frozenlist",
"py-spy >= 0.2.0", "py-spy >= 0.2.0",
"requests", "requests",
"gpustat >= 1.0.0b1", # for windows "gpustat >= 1.0.0b1", # for windows
@ -270,6 +267,9 @@ if setup_spec.type == SetupType.RAY:
"protobuf >= 3.15.3", "protobuf >= 3.15.3",
"pyyaml", "pyyaml",
"redis >= 3.5.0", "redis >= 3.5.0",
"aioredis < 2",
"aiosignal",
"frozenlist",
] ]