mirror of
https://github.com/vale981/ray
synced 2025-03-05 10:01:43 -05:00
[dashboard] Don't start dashboard agent when missing dependencies (#17966)
This commit is contained in:
parent
65823088c6
commit
7916500c43
7 changed files with 98 additions and 86 deletions
|
@ -10,21 +10,6 @@ import json
|
|||
import time
|
||||
import traceback
|
||||
|
||||
try:
|
||||
import aiohttp
|
||||
import aiohttp.web
|
||||
import aiohttp_cors
|
||||
from aiohttp import hdrs
|
||||
|
||||
import aioredis # noqa: F401
|
||||
except ImportError:
|
||||
print("Not all Ray Dashboard dependencies were found. "
|
||||
"In Ray 1.4+, the Ray CLI, autoscaler, and dashboard will "
|
||||
"only be usable via `pip install 'ray[default]'`. Please "
|
||||
"update your install command.")
|
||||
# Set an exit code different from throwing an exception.
|
||||
sys.exit(2)
|
||||
|
||||
from grpc.experimental import aio as aiogrpc
|
||||
|
||||
import ray
|
||||
|
@ -37,6 +22,11 @@ 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.new_dashboard.optional_deps import aiohttp, aiohttp_cors, hdrs
|
||||
|
||||
# Import psutil after ray so the packaged version is used.
|
||||
import psutil
|
||||
|
||||
|
|
|
@ -1,21 +1,5 @@
|
|||
import sys
|
||||
|
||||
try:
|
||||
import opencensus # noqa: F401
|
||||
import prometheus_client # noqa: F401
|
||||
|
||||
import aiohttp.web
|
||||
import aiohttp
|
||||
import aiohttp_cors # noqa: F401
|
||||
import aioredis # noqa: F401
|
||||
except (ModuleNotFoundError, ImportError):
|
||||
print("Not all Ray Dashboard dependencies were found. "
|
||||
"In Ray 1.4+, the Ray CLI, autoscaler, and dashboard will "
|
||||
"only be usable via `pip install 'ray[default]'`. Please "
|
||||
"update your install command.")
|
||||
# Set an exit code different from throwing an exception.
|
||||
sys.exit(2)
|
||||
|
||||
import argparse
|
||||
import asyncio
|
||||
import errno
|
||||
|
@ -34,6 +18,11 @@ import ray._private.utils
|
|||
from ray._private.ray_logging import setup_component_logger
|
||||
from ray._private.metrics_agent import PrometheusServiceDiscoveryWriter
|
||||
|
||||
# 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.new_dashboard.optional_deps import aiohttp
|
||||
|
||||
# Logger for this module. It should be configured at the entry point
|
||||
# into the program using Ray. Ray provides a default configuration at
|
||||
# entry/init points.
|
||||
|
|
|
@ -5,9 +5,6 @@ import asyncio
|
|||
import logging
|
||||
import ipaddress
|
||||
|
||||
import aiohttp
|
||||
import aiohttp.web
|
||||
from aiohttp import hdrs
|
||||
from grpc.experimental import aio as aiogrpc
|
||||
|
||||
import ray._private.services
|
||||
|
@ -19,6 +16,11 @@ from ray.core.generated import gcs_service_pb2_grpc
|
|||
from ray.new_dashboard.datacenter import DataOrganizer
|
||||
from ray.new_dashboard.utils import async_loop_forever
|
||||
|
||||
# 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.new_dashboard.optional_deps import aiohttp, hdrs
|
||||
|
||||
logger = logging.getLogger(__name__)
|
||||
routes = dashboard_utils.ClassMethodRouteTable
|
||||
|
||||
|
|
16
dashboard/optional_deps.py
Normal file
16
dashboard/optional_deps.py
Normal file
|
@ -0,0 +1,16 @@
|
|||
# These checks have to come first because aiohttp looks
|
||||
# for opencensus, too, and raises a different error otherwise.
|
||||
import opencensus # noqa: F401
|
||||
|
||||
import prometheus_client # noqa: F401
|
||||
|
||||
import aiohttp # noqa: F401
|
||||
import aiohttp.signals
|
||||
import aiohttp.web # noqa: F401
|
||||
import aiohttp_cors # noqa: F401
|
||||
from aiohttp import hdrs # noqa: F401
|
||||
from aiohttp.frozenlist import FrozenList # noqa: F401
|
||||
from aiohttp.typedefs import PathLike # noqa: F401
|
||||
from aiohttp.web import RouteDef # noqa: F401
|
||||
|
||||
import aioredis # noqa: F401
|
|
@ -10,6 +10,7 @@ import logging
|
|||
import os
|
||||
import pkgutil
|
||||
import socket
|
||||
import time
|
||||
import traceback
|
||||
from abc import ABCMeta, abstractmethod
|
||||
from base64 import b64decode
|
||||
|
@ -17,20 +18,18 @@ from collections import namedtuple
|
|||
from collections.abc import MutableMapping, Mapping, Sequence
|
||||
from typing import Any
|
||||
|
||||
import aiohttp.signals
|
||||
import aiohttp.web
|
||||
import aioredis
|
||||
import time
|
||||
from aiohttp import hdrs
|
||||
from aiohttp.frozenlist import FrozenList
|
||||
from aiohttp.typedefs import PathLike
|
||||
from aiohttp.web import RouteDef
|
||||
from google.protobuf.json_format import MessageToDict
|
||||
|
||||
import ray.new_dashboard.consts as dashboard_consts
|
||||
from ray.ray_constants import env_bool
|
||||
from ray._private.utils import binary_to_hex
|
||||
|
||||
# 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.new_dashboard.optional_deps import (aiohttp, aioredis, hdrs,
|
||||
FrozenList, PathLike, RouteDef)
|
||||
|
||||
try:
|
||||
create_task = asyncio.create_task
|
||||
except AttributeError:
|
||||
|
|
|
@ -55,6 +55,12 @@ GCS_SERVER_EXECUTABLE = os.path.join(
|
|||
DEFAULT_WORKER_EXECUTABLE = os.path.join(
|
||||
RAY_PATH, "core/src/ray/cpp/default_worker" + EXE_SUFFIX)
|
||||
|
||||
DASHBOARD_DEPENDENCY_ERROR_MESSAGE = (
|
||||
"Not all Ray Dashboard dependencies were "
|
||||
"found. To use the dashboard please "
|
||||
"install Ray using `pip install "
|
||||
"ray[default]`.")
|
||||
|
||||
# Logger for this module. It should be configured at the entry point
|
||||
# into the program using Ray. Ray provides a default configuration at
|
||||
# entry/init points.
|
||||
|
@ -222,6 +228,16 @@ def get_ray_address_to_use_or_die():
|
|||
find_redis_address_or_die())
|
||||
|
||||
|
||||
def _log_dashboard_dependency_warning_once():
|
||||
if log_once("dashboard_failed_import"
|
||||
) and not os.getenv("RAY_DISABLE_IMPORT_WARNING") == "1":
|
||||
warning_message = DASHBOARD_DEPENDENCY_ERROR_MESSAGE
|
||||
warning_message += " To disable this message, set " \
|
||||
"RAY_DISABLE_IMPORT_WARNING " \
|
||||
"env var to '1'."
|
||||
warnings.warn(warning_message)
|
||||
|
||||
|
||||
def find_redis_address_or_die():
|
||||
|
||||
redis_addresses = find_redis_address()
|
||||
|
@ -1174,29 +1190,12 @@ def start_dashboard(require_dashboard,
|
|||
|
||||
# Make sure the process can start.
|
||||
try:
|
||||
# These checks have to come first because aiohttp looks
|
||||
# for opencensus, too, and raises a different error otherwise.
|
||||
import opencensus # noqa: F401
|
||||
import prometheus_client # noqa: F401
|
||||
|
||||
import aiohttp # noqa: F401
|
||||
import aioredis # noqa: F401
|
||||
import aiohttp_cors # noqa: F401
|
||||
import grpc # noqa: F401
|
||||
import ray.new_dashboard.optional_deps # noqa: F401
|
||||
except ImportError:
|
||||
warning_message = (
|
||||
"Not all Ray Dashboard dependencies were found. "
|
||||
"To use the dashboard please install Ray like so: `pip "
|
||||
"install ray[default]`.")
|
||||
if require_dashboard:
|
||||
raise ImportError(warning_message)
|
||||
raise ImportError(DASHBOARD_DEPENDENCY_ERROR_MESSAGE)
|
||||
else:
|
||||
if log_once("dashboard_failed_import") and not os.getenv(
|
||||
"RAY_DISABLE_IMPORT_WARNING") == "1":
|
||||
warning_message += " To disable this message, set " \
|
||||
"RAY_DISABLE_IMPORT_WARNING " \
|
||||
"env var to '1'."
|
||||
warnings.warn(warning_message)
|
||||
_log_dashboard_dependency_warning_once()
|
||||
return None, None
|
||||
|
||||
# Start the dashboard process.
|
||||
|
@ -1500,30 +1499,47 @@ def start_raylet(redis_address,
|
|||
if max_worker_port is None:
|
||||
max_worker_port = 0
|
||||
|
||||
# Create agent command
|
||||
agent_command = [
|
||||
sys.executable,
|
||||
"-u",
|
||||
os.path.join(RAY_PATH, "new_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"--runtime-env-setup-hook={runtime_env_setup_hook}",
|
||||
f"--log-dir={log_dir}",
|
||||
f"--logging-rotate-bytes={max_bytes}",
|
||||
f"--logging-rotate-backup-count={backup_count}",
|
||||
]
|
||||
# Check to see if we should start the dashboard agent or not based on the
|
||||
# Ray installation version the user has installed (ray vs. ray[default]).
|
||||
# Unfortunately there doesn't seem to be a cleaner way to detect this other
|
||||
# than just blindly importing the relevant packages.
|
||||
def check_should_start_agent():
|
||||
try:
|
||||
import ray.new_dashboard.optional_deps # noqa: F401
|
||||
|
||||
if redis_password is not None and len(redis_password) != 0:
|
||||
agent_command.append("--redis-password={}".format(redis_password))
|
||||
return True
|
||||
except ImportError:
|
||||
_log_dashboard_dependency_warning_once()
|
||||
|
||||
return False
|
||||
|
||||
if not check_should_start_agent():
|
||||
# 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, "new_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"--runtime-env-setup-hook={runtime_env_setup_hook}",
|
||||
f"--log-dir={log_dir}",
|
||||
f"--logging-rotate-bytes={max_bytes}",
|
||||
f"--logging-rotate-backup-count={backup_count}",
|
||||
]
|
||||
|
||||
if redis_password is not None and len(redis_password) != 0:
|
||||
agent_command.append("--redis-password={}".format(redis_password))
|
||||
|
||||
command = [
|
||||
RAYLET_EXECUTABLE,
|
||||
|
|
|
@ -194,7 +194,7 @@ int main(int argc, char *argv[]) {
|
|||
if (!agent_command.empty()) {
|
||||
node_manager_config.agent_command = agent_command;
|
||||
} else {
|
||||
RAY_LOG(DEBUG) << "Agent command is empty.";
|
||||
RAY_LOG(DEBUG) << "Agent command is empty. Not starting agent.";
|
||||
}
|
||||
|
||||
node_manager_config.report_resources_period_ms =
|
||||
|
|
Loading…
Add table
Reference in a new issue