2022-04-22 10:45:03 +09:00
|
|
|
import asyncio
|
2022-04-19 13:15:43 +09:00
|
|
|
import logging
|
2022-06-22 22:21:50 +09:00
|
|
|
|
2022-06-21 14:49:29 +09:00
|
|
|
from dataclasses import asdict, fields
|
2022-04-23 07:59:46 +09:00
|
|
|
from itertools import islice
|
2022-06-04 09:17:30 +09:00
|
|
|
from typing import List, Tuple
|
2022-04-22 10:45:03 +09:00
|
|
|
|
2022-07-24 19:55:36 -07:00
|
|
|
from ray._private.ray_constants import env_integer
|
|
|
|
|
2022-04-22 10:45:03 +09:00
|
|
|
import ray.dashboard.memory_utils as memory_utils
|
2022-06-21 14:49:29 +09:00
|
|
|
import ray.dashboard.utils as dashboard_utils
|
|
|
|
from ray._private.utils import binary_to_hex
|
|
|
|
from ray.core.generated.common_pb2 import TaskStatus
|
2022-04-19 13:15:43 +09:00
|
|
|
from ray.experimental.state.common import (
|
|
|
|
ActorState,
|
2022-06-21 14:49:29 +09:00
|
|
|
ListApiOptions,
|
|
|
|
ListApiResponse,
|
2022-04-19 13:15:43 +09:00
|
|
|
NodeState,
|
2022-04-22 10:45:03 +09:00
|
|
|
ObjectState,
|
2022-06-21 14:49:29 +09:00
|
|
|
PlacementGroupState,
|
2022-05-03 06:01:00 +09:00
|
|
|
RuntimeEnvState,
|
2022-06-22 22:21:50 +09:00
|
|
|
SummaryApiResponse,
|
2022-07-25 23:31:49 -07:00
|
|
|
RAY_MAX_LIMIT_FROM_API_SERVER,
|
2022-06-22 22:21:50 +09:00
|
|
|
SummaryApiOptions,
|
|
|
|
TaskSummaries,
|
2022-06-21 14:49:29 +09:00
|
|
|
StateSchema,
|
|
|
|
SupportedFilterType,
|
|
|
|
TaskState,
|
|
|
|
WorkerState,
|
2022-06-22 22:21:50 +09:00
|
|
|
StateSummary,
|
|
|
|
ActorSummaries,
|
|
|
|
ObjectSummaries,
|
2022-06-28 23:56:44 +09:00
|
|
|
filter_fields,
|
2022-06-28 21:42:19 +09:00
|
|
|
PredicateType,
|
2022-05-24 19:56:49 +09:00
|
|
|
)
|
|
|
|
from ray.experimental.state.state_manager import (
|
|
|
|
DataSourceUnavailable,
|
2022-06-21 14:49:29 +09:00
|
|
|
StateDataSourceClient,
|
2022-04-19 13:15:43 +09:00
|
|
|
)
|
2022-05-03 06:01:00 +09:00
|
|
|
from ray.runtime_env import RuntimeEnv
|
2022-06-28 23:56:44 +09:00
|
|
|
from ray.experimental.state.util import convert_string_to_type
|
2022-04-19 13:15:43 +09:00
|
|
|
|
|
|
|
logger = logging.getLogger(__name__)
|
|
|
|
|
2022-05-24 19:56:49 +09:00
|
|
|
GCS_QUERY_FAILURE_WARNING = (
|
|
|
|
"Failed to query data from GCS. It is due to "
|
|
|
|
"(1) GCS is unexpectedly failed. "
|
|
|
|
"(2) GCS is overloaded. "
|
|
|
|
"(3) There's an unexpected network issue. "
|
|
|
|
"Please check the gcs_server.out log to find the root cause."
|
|
|
|
)
|
|
|
|
NODE_QUERY_FAILURE_WARNING = (
|
|
|
|
"Failed to query data from {type}. "
|
2022-07-25 23:31:49 -07:00
|
|
|
"Queried {total} {type} "
|
2022-05-24 19:56:49 +09:00
|
|
|
"and {network_failures} {type} failed to reply. It is due to "
|
|
|
|
"(1) {type} is unexpectedly failed. "
|
|
|
|
"(2) {type} is overloaded. "
|
|
|
|
"(3) There's an unexpected network issue. Please check the "
|
|
|
|
"{log_command} to find the root cause."
|
|
|
|
)
|
|
|
|
|
2022-04-19 13:15:43 +09:00
|
|
|
|
2022-06-04 09:17:30 +09:00
|
|
|
def _convert_filters_type(
|
2022-06-28 21:42:19 +09:00
|
|
|
filter: List[Tuple[str, PredicateType, SupportedFilterType]],
|
|
|
|
schema: StateSchema,
|
2022-06-04 09:17:30 +09:00
|
|
|
) -> List[Tuple[str, SupportedFilterType]]:
|
|
|
|
"""Convert the given filter's type to SupportedFilterType.
|
|
|
|
|
|
|
|
This method is necessary because click can only accept a single type
|
|
|
|
for its tuple (which is string in this case).
|
|
|
|
|
|
|
|
Args:
|
|
|
|
filter: A list of filter which is a tuple of (key, val).
|
|
|
|
schema: The state schema. It is used to infer the type of the column for filter.
|
|
|
|
|
|
|
|
Returns:
|
2022-07-26 21:49:28 +09:00
|
|
|
A new list of filters with correct types that match the schema.
|
2022-06-04 09:17:30 +09:00
|
|
|
"""
|
|
|
|
new_filter = []
|
|
|
|
schema = {field.name: field.type for field in fields(schema)}
|
|
|
|
|
2022-06-28 21:42:19 +09:00
|
|
|
for col, predicate, val in filter:
|
2022-06-04 09:17:30 +09:00
|
|
|
if col in schema:
|
|
|
|
column_type = schema[col]
|
2022-07-26 21:49:28 +09:00
|
|
|
try:
|
|
|
|
isinstance(val, column_type)
|
|
|
|
except TypeError:
|
|
|
|
# Calling `isinstance` to the Literal type raises a TypeError.
|
|
|
|
# Ignore this case.
|
2022-06-04 09:17:30 +09:00
|
|
|
pass
|
2022-07-26 21:49:28 +09:00
|
|
|
else:
|
|
|
|
if isinstance(val, column_type):
|
|
|
|
# Do nothing.
|
|
|
|
pass
|
|
|
|
elif column_type is int:
|
|
|
|
try:
|
|
|
|
val = convert_string_to_type(val, int)
|
|
|
|
except ValueError:
|
|
|
|
raise ValueError(
|
|
|
|
f"Invalid filter `--filter {col} {val}` for a int type "
|
|
|
|
"column. Please provide an integer filter "
|
|
|
|
f"`--filter {col} [int]`"
|
|
|
|
)
|
|
|
|
elif column_type is float:
|
|
|
|
try:
|
|
|
|
val = convert_string_to_type(val, float)
|
|
|
|
except ValueError:
|
|
|
|
raise ValueError(
|
|
|
|
f"Invalid filter `--filter {col} {val}` for a float "
|
|
|
|
"type column. Please provide an integer filter "
|
|
|
|
f"`--filter {col} [float]`"
|
|
|
|
)
|
|
|
|
elif column_type is bool:
|
|
|
|
try:
|
|
|
|
val = convert_string_to_type(val, bool)
|
|
|
|
except ValueError:
|
|
|
|
raise ValueError(
|
|
|
|
f"Invalid filter `--filter {col} {val}` for a boolean "
|
|
|
|
"type column. Please provide "
|
|
|
|
f"`--filter {col} [True|true|1]` for True or "
|
|
|
|
f"`--filter {col} [False|false|0]` for False."
|
|
|
|
)
|
2022-06-28 21:42:19 +09:00
|
|
|
new_filter.append((col, predicate, val))
|
2022-06-04 09:17:30 +09:00
|
|
|
return new_filter
|
|
|
|
|
|
|
|
|
2022-04-22 10:45:03 +09:00
|
|
|
# TODO(sang): Move the class to state/state_manager.py.
|
2022-05-22 16:07:03 +09:00
|
|
|
# TODO(sang): Remove *State and replaces with Pydantic or protobuf.
|
2022-04-22 10:45:03 +09:00
|
|
|
# (depending on API interface standardization).
|
|
|
|
class StateAPIManager:
|
|
|
|
"""A class to query states from data source, caches, and post-processes
|
|
|
|
the entries.
|
|
|
|
"""
|
|
|
|
|
|
|
|
def __init__(self, state_data_source_client: StateDataSourceClient):
|
|
|
|
self._client = state_data_source_client
|
|
|
|
|
|
|
|
@property
|
|
|
|
def data_source_client(self):
|
|
|
|
return self._client
|
2022-04-19 13:15:43 +09:00
|
|
|
|
2022-06-04 09:17:30 +09:00
|
|
|
def _filter(
|
|
|
|
self,
|
|
|
|
data: List[dict],
|
|
|
|
filters: List[Tuple[str, SupportedFilterType]],
|
|
|
|
state_dataclass: StateSchema,
|
2022-06-28 23:56:44 +09:00
|
|
|
detail: bool,
|
2022-06-04 09:17:30 +09:00
|
|
|
) -> List[dict]:
|
|
|
|
"""Return the filtered data given filters.
|
|
|
|
|
|
|
|
Args:
|
|
|
|
data: A list of state data.
|
|
|
|
filters: A list of KV tuple to filter data (key, val). The data is filtered
|
|
|
|
if data[key] != val.
|
|
|
|
state_dataclass: The state schema.
|
|
|
|
|
|
|
|
Returns:
|
|
|
|
A list of filtered state data in dictionary. Each state data's
|
2022-07-20 16:38:15 -07:00
|
|
|
unnecessary columns are filtered by the given state_dataclass schema.
|
2022-06-04 09:17:30 +09:00
|
|
|
"""
|
|
|
|
filters = _convert_filters_type(filters, state_dataclass)
|
|
|
|
result = []
|
|
|
|
for datum in data:
|
|
|
|
match = True
|
2022-06-28 21:42:19 +09:00
|
|
|
for filter_column, filter_predicate, filter_value in filters:
|
2022-06-04 09:17:30 +09:00
|
|
|
filterable_columns = state_dataclass.filterable_columns()
|
2022-07-19 00:54:16 -07:00
|
|
|
filter_column = filter_column.lower()
|
2022-06-04 09:17:30 +09:00
|
|
|
if filter_column not in filterable_columns:
|
|
|
|
raise ValueError(
|
|
|
|
f"The given filter column {filter_column} is not supported. "
|
|
|
|
f"Supported filter columns: {filterable_columns}"
|
|
|
|
)
|
|
|
|
|
2022-07-23 21:33:07 -07:00
|
|
|
if filter_column not in datum:
|
|
|
|
match = False
|
|
|
|
elif filter_predicate == "=":
|
2022-06-28 21:42:19 +09:00
|
|
|
match = datum[filter_column] == filter_value
|
|
|
|
elif filter_predicate == "!=":
|
|
|
|
match = datum[filter_column] != filter_value
|
|
|
|
else:
|
|
|
|
raise ValueError(
|
|
|
|
f"Unsupported filter predicate {filter_predicate} is given. "
|
|
|
|
"Available predicates: =, !=."
|
|
|
|
)
|
|
|
|
|
|
|
|
if not match:
|
2022-06-04 09:17:30 +09:00
|
|
|
break
|
|
|
|
|
|
|
|
if match:
|
2022-06-28 23:56:44 +09:00
|
|
|
result.append(filter_fields(datum, state_dataclass, detail))
|
2022-06-04 09:17:30 +09:00
|
|
|
return result
|
|
|
|
|
2022-05-24 19:56:49 +09:00
|
|
|
async def list_actors(self, *, option: ListApiOptions) -> ListApiResponse:
|
2022-04-22 10:45:03 +09:00
|
|
|
"""List all actor information from the cluster.
|
|
|
|
|
|
|
|
Returns:
|
|
|
|
{actor_id -> actor_data_in_dict}
|
|
|
|
actor_data_in_dict's schema is in ActorState
|
2022-05-24 19:56:49 +09:00
|
|
|
|
2022-04-22 10:45:03 +09:00
|
|
|
"""
|
2022-05-24 19:56:49 +09:00
|
|
|
try:
|
|
|
|
reply = await self._client.get_all_actor_info(timeout=option.timeout)
|
|
|
|
except DataSourceUnavailable:
|
|
|
|
raise DataSourceUnavailable(GCS_QUERY_FAILURE_WARNING)
|
|
|
|
|
2022-04-23 07:59:46 +09:00
|
|
|
result = []
|
2022-04-19 13:15:43 +09:00
|
|
|
for message in reply.actor_table_data:
|
2022-07-24 19:55:36 -07:00
|
|
|
data = self._message_to_dict(
|
|
|
|
message=message, fields_to_decode=["actor_id", "owner_id"]
|
|
|
|
)
|
2022-04-23 07:59:46 +09:00
|
|
|
result.append(data)
|
2022-07-25 23:31:49 -07:00
|
|
|
num_after_truncation = len(result)
|
2022-06-28 23:56:44 +09:00
|
|
|
result = self._filter(result, option.filters, ActorState, option.detail)
|
2022-07-25 23:31:49 -07:00
|
|
|
num_filtered = len(result)
|
|
|
|
|
2022-04-23 07:59:46 +09:00
|
|
|
# Sort to make the output deterministic.
|
|
|
|
result.sort(key=lambda entry: entry["actor_id"])
|
2022-06-29 10:33:57 +09:00
|
|
|
result = list(islice(result, option.limit))
|
|
|
|
return ListApiResponse(
|
|
|
|
result=result,
|
|
|
|
total=reply.total,
|
2022-07-25 23:31:49 -07:00
|
|
|
num_after_truncation=num_after_truncation,
|
|
|
|
num_filtered=num_filtered,
|
2022-06-29 10:33:57 +09:00
|
|
|
)
|
2022-04-19 13:15:43 +09:00
|
|
|
|
2022-05-24 19:56:49 +09:00
|
|
|
async def list_placement_groups(self, *, option: ListApiOptions) -> ListApiResponse:
|
2022-04-22 10:45:03 +09:00
|
|
|
"""List all placement group information from the cluster.
|
|
|
|
|
|
|
|
Returns:
|
|
|
|
{pg_id -> pg_data_in_dict}
|
|
|
|
pg_data_in_dict's schema is in PlacementGroupState
|
|
|
|
"""
|
2022-05-24 19:56:49 +09:00
|
|
|
try:
|
|
|
|
reply = await self._client.get_all_placement_group_info(
|
|
|
|
timeout=option.timeout
|
|
|
|
)
|
|
|
|
except DataSourceUnavailable:
|
|
|
|
raise DataSourceUnavailable(GCS_QUERY_FAILURE_WARNING)
|
|
|
|
|
2022-04-23 07:59:46 +09:00
|
|
|
result = []
|
2022-04-19 13:15:43 +09:00
|
|
|
for message in reply.placement_group_table_data:
|
2022-04-23 07:59:46 +09:00
|
|
|
|
2022-04-19 13:15:43 +09:00
|
|
|
data = self._message_to_dict(
|
|
|
|
message=message,
|
2022-07-14 06:44:07 -07:00
|
|
|
fields_to_decode=["placement_group_id", "node_id"],
|
2022-04-19 13:15:43 +09:00
|
|
|
)
|
2022-04-23 07:59:46 +09:00
|
|
|
result.append(data)
|
2022-07-25 23:31:49 -07:00
|
|
|
num_after_truncation = len(result)
|
2022-04-23 07:59:46 +09:00
|
|
|
|
2022-06-28 23:56:44 +09:00
|
|
|
result = self._filter(
|
|
|
|
result, option.filters, PlacementGroupState, option.detail
|
|
|
|
)
|
2022-07-25 23:31:49 -07:00
|
|
|
num_filtered = len(result)
|
2022-04-23 07:59:46 +09:00
|
|
|
# Sort to make the output deterministic.
|
|
|
|
result.sort(key=lambda entry: entry["placement_group_id"])
|
2022-06-29 10:33:57 +09:00
|
|
|
return ListApiResponse(
|
|
|
|
result=list(islice(result, option.limit)),
|
|
|
|
total=reply.total,
|
2022-07-25 23:31:49 -07:00
|
|
|
num_after_truncation=num_after_truncation,
|
|
|
|
num_filtered=num_filtered,
|
2022-06-29 10:33:57 +09:00
|
|
|
)
|
2022-04-19 13:15:43 +09:00
|
|
|
|
2022-05-24 19:56:49 +09:00
|
|
|
async def list_nodes(self, *, option: ListApiOptions) -> ListApiResponse:
|
2022-04-22 10:45:03 +09:00
|
|
|
"""List all node information from the cluster.
|
|
|
|
|
|
|
|
Returns:
|
|
|
|
{node_id -> node_data_in_dict}
|
|
|
|
node_data_in_dict's schema is in NodeState
|
|
|
|
"""
|
2022-05-24 19:56:49 +09:00
|
|
|
try:
|
|
|
|
reply = await self._client.get_all_node_info(timeout=option.timeout)
|
|
|
|
except DataSourceUnavailable:
|
|
|
|
raise DataSourceUnavailable(GCS_QUERY_FAILURE_WARNING)
|
|
|
|
|
2022-04-23 07:59:46 +09:00
|
|
|
result = []
|
2022-04-19 13:15:43 +09:00
|
|
|
for message in reply.node_info_list:
|
|
|
|
data = self._message_to_dict(message=message, fields_to_decode=["node_id"])
|
2022-06-13 21:52:57 +09:00
|
|
|
data["node_ip"] = data["node_manager_address"]
|
2022-04-23 07:59:46 +09:00
|
|
|
result.append(data)
|
2022-04-19 13:15:43 +09:00
|
|
|
|
2022-07-25 23:31:49 -07:00
|
|
|
total_nodes = len(result)
|
|
|
|
# No reason to truncate node because they are usually small.
|
|
|
|
num_after_truncation = len(result)
|
|
|
|
|
2022-06-28 23:56:44 +09:00
|
|
|
result = self._filter(result, option.filters, NodeState, option.detail)
|
2022-07-25 23:31:49 -07:00
|
|
|
num_filtered = len(result)
|
|
|
|
|
2022-04-23 07:59:46 +09:00
|
|
|
# Sort to make the output deterministic.
|
|
|
|
result.sort(key=lambda entry: entry["node_id"])
|
2022-06-29 10:33:57 +09:00
|
|
|
result = list(islice(result, option.limit))
|
|
|
|
return ListApiResponse(
|
|
|
|
result=result,
|
|
|
|
total=total_nodes,
|
2022-07-25 23:31:49 -07:00
|
|
|
num_after_truncation=num_after_truncation,
|
|
|
|
num_filtered=num_filtered,
|
2022-06-29 10:33:57 +09:00
|
|
|
)
|
2022-04-23 07:59:46 +09:00
|
|
|
|
2022-05-24 19:56:49 +09:00
|
|
|
async def list_workers(self, *, option: ListApiOptions) -> ListApiResponse:
|
2022-04-22 10:45:03 +09:00
|
|
|
"""List all worker information from the cluster.
|
|
|
|
|
|
|
|
Returns:
|
|
|
|
{worker_id -> worker_data_in_dict}
|
|
|
|
worker_data_in_dict's schema is in WorkerState
|
|
|
|
"""
|
2022-05-24 19:56:49 +09:00
|
|
|
try:
|
|
|
|
reply = await self._client.get_all_worker_info(timeout=option.timeout)
|
|
|
|
except DataSourceUnavailable:
|
|
|
|
raise DataSourceUnavailable(GCS_QUERY_FAILURE_WARNING)
|
|
|
|
|
2022-04-23 07:59:46 +09:00
|
|
|
result = []
|
2022-04-19 13:15:43 +09:00
|
|
|
for message in reply.worker_table_data:
|
|
|
|
data = self._message_to_dict(
|
2022-06-28 23:56:44 +09:00
|
|
|
message=message, fields_to_decode=["worker_id", "raylet_id"]
|
2022-04-19 13:15:43 +09:00
|
|
|
)
|
|
|
|
data["worker_id"] = data["worker_address"]["worker_id"]
|
2022-06-28 23:56:44 +09:00
|
|
|
data["node_id"] = data["worker_address"]["raylet_id"]
|
|
|
|
data["ip"] = data["worker_address"]["ip_address"]
|
2022-04-23 07:59:46 +09:00
|
|
|
result.append(data)
|
|
|
|
|
2022-07-25 23:31:49 -07:00
|
|
|
num_after_truncation = len(result)
|
2022-06-28 23:56:44 +09:00
|
|
|
result = self._filter(result, option.filters, WorkerState, option.detail)
|
2022-07-25 23:31:49 -07:00
|
|
|
num_filtered = len(result)
|
2022-04-23 07:59:46 +09:00
|
|
|
# Sort to make the output deterministic.
|
|
|
|
result.sort(key=lambda entry: entry["worker_id"])
|
2022-06-29 10:33:57 +09:00
|
|
|
result = list(islice(result, option.limit))
|
|
|
|
return ListApiResponse(
|
|
|
|
result=result,
|
|
|
|
total=reply.total,
|
2022-07-25 23:31:49 -07:00
|
|
|
num_after_truncation=num_after_truncation,
|
|
|
|
num_filtered=num_filtered,
|
2022-06-29 10:33:57 +09:00
|
|
|
)
|
2022-04-23 07:59:46 +09:00
|
|
|
|
2022-08-05 19:33:49 -07:00
|
|
|
async def list_jobs(self, *, option: ListApiOptions) -> ListApiResponse:
|
2022-04-23 07:59:46 +09:00
|
|
|
# TODO(sang): Support limit & timeout & async calls.
|
2022-05-24 19:56:49 +09:00
|
|
|
try:
|
2022-06-21 14:49:29 +09:00
|
|
|
result = []
|
2022-08-05 19:33:49 -07:00
|
|
|
job_info = await self._client.get_job_info()
|
2022-06-21 14:49:29 +09:00
|
|
|
for job_id, data in job_info.items():
|
|
|
|
data = asdict(data)
|
|
|
|
data["job_id"] = job_id
|
|
|
|
result.append(data)
|
2022-05-24 19:56:49 +09:00
|
|
|
except DataSourceUnavailable:
|
|
|
|
raise DataSourceUnavailable(GCS_QUERY_FAILURE_WARNING)
|
2022-06-29 10:33:57 +09:00
|
|
|
return ListApiResponse(
|
|
|
|
result=result,
|
|
|
|
# TODO(sang): Support this.
|
|
|
|
total=len(result),
|
2022-07-25 23:31:49 -07:00
|
|
|
num_after_truncation=len(result),
|
|
|
|
num_filtered=len(result),
|
2022-06-29 10:33:57 +09:00
|
|
|
)
|
2022-04-19 13:15:43 +09:00
|
|
|
|
2022-05-24 19:56:49 +09:00
|
|
|
async def list_tasks(self, *, option: ListApiOptions) -> ListApiResponse:
|
2022-04-22 10:45:03 +09:00
|
|
|
"""List all task information from the cluster.
|
|
|
|
|
|
|
|
Returns:
|
|
|
|
{task_id -> task_data_in_dict}
|
|
|
|
task_data_in_dict's schema is in TaskState
|
|
|
|
"""
|
2022-05-24 19:56:49 +09:00
|
|
|
raylet_ids = self._client.get_all_registered_raylet_ids()
|
2022-04-22 10:45:03 +09:00
|
|
|
replies = await asyncio.gather(
|
|
|
|
*[
|
2022-04-23 07:59:46 +09:00
|
|
|
self._client.get_task_info(node_id, timeout=option.timeout)
|
2022-05-24 19:56:49 +09:00
|
|
|
for node_id in raylet_ids
|
|
|
|
],
|
|
|
|
return_exceptions=True,
|
2022-04-22 10:45:03 +09:00
|
|
|
)
|
|
|
|
|
2022-05-24 19:56:49 +09:00
|
|
|
unresponsive_nodes = 0
|
2022-05-16 21:39:05 +09:00
|
|
|
running_task_id = set()
|
2022-05-24 19:56:49 +09:00
|
|
|
successful_replies = []
|
2022-06-29 10:33:57 +09:00
|
|
|
total_tasks = 0
|
2022-05-16 21:39:05 +09:00
|
|
|
for reply in replies:
|
2022-05-24 19:56:49 +09:00
|
|
|
if isinstance(reply, DataSourceUnavailable):
|
|
|
|
unresponsive_nodes += 1
|
|
|
|
continue
|
|
|
|
elif isinstance(reply, Exception):
|
|
|
|
raise reply
|
|
|
|
|
|
|
|
successful_replies.append(reply)
|
2022-06-29 10:33:57 +09:00
|
|
|
total_tasks += reply.total
|
2022-05-16 21:39:05 +09:00
|
|
|
for task_id in reply.running_task_ids:
|
|
|
|
running_task_id.add(binary_to_hex(task_id))
|
|
|
|
|
2022-05-24 19:56:49 +09:00
|
|
|
partial_failure_warning = None
|
|
|
|
if len(raylet_ids) > 0 and unresponsive_nodes > 0:
|
|
|
|
warning_msg = NODE_QUERY_FAILURE_WARNING.format(
|
|
|
|
type="raylet",
|
|
|
|
total=len(raylet_ids),
|
|
|
|
network_failures=unresponsive_nodes,
|
|
|
|
log_command="raylet.out",
|
|
|
|
)
|
|
|
|
if unresponsive_nodes == len(raylet_ids):
|
|
|
|
raise DataSourceUnavailable(warning_msg)
|
|
|
|
partial_failure_warning = (
|
|
|
|
f"The returned data may contain incomplete result. {warning_msg}"
|
|
|
|
)
|
|
|
|
|
2022-04-23 07:59:46 +09:00
|
|
|
result = []
|
2022-05-24 19:56:49 +09:00
|
|
|
for reply in successful_replies:
|
|
|
|
assert not isinstance(reply, Exception)
|
2022-05-16 21:39:05 +09:00
|
|
|
tasks = reply.owned_task_info_entries
|
2022-04-22 10:45:03 +09:00
|
|
|
for task in tasks:
|
|
|
|
data = self._message_to_dict(
|
|
|
|
message=task,
|
|
|
|
fields_to_decode=["task_id"],
|
|
|
|
)
|
2022-05-16 21:39:05 +09:00
|
|
|
if data["task_id"] in running_task_id:
|
|
|
|
data["scheduling_state"] = TaskStatus.DESCRIPTOR.values_by_number[
|
|
|
|
TaskStatus.RUNNING
|
|
|
|
].name
|
2022-04-23 07:59:46 +09:00
|
|
|
result.append(data)
|
2022-07-25 23:31:49 -07:00
|
|
|
num_after_truncation = len(result)
|
2022-06-28 23:56:44 +09:00
|
|
|
result = self._filter(result, option.filters, TaskState, option.detail)
|
2022-07-25 23:31:49 -07:00
|
|
|
num_filtered = len(result)
|
2022-04-23 07:59:46 +09:00
|
|
|
# Sort to make the output deterministic.
|
|
|
|
result.sort(key=lambda entry: entry["task_id"])
|
2022-06-29 10:33:57 +09:00
|
|
|
result = list(islice(result, option.limit))
|
2022-05-24 19:56:49 +09:00
|
|
|
return ListApiResponse(
|
2022-06-29 10:33:57 +09:00
|
|
|
result=result,
|
2022-05-24 19:56:49 +09:00
|
|
|
partial_failure_warning=partial_failure_warning,
|
2022-06-29 10:33:57 +09:00
|
|
|
total=total_tasks,
|
2022-07-25 23:31:49 -07:00
|
|
|
num_after_truncation=num_after_truncation,
|
|
|
|
num_filtered=num_filtered,
|
2022-05-24 19:56:49 +09:00
|
|
|
)
|
2022-04-23 07:59:46 +09:00
|
|
|
|
2022-05-24 19:56:49 +09:00
|
|
|
async def list_objects(self, *, option: ListApiOptions) -> ListApiResponse:
|
2022-04-22 10:45:03 +09:00
|
|
|
"""List all object information from the cluster.
|
|
|
|
|
|
|
|
Returns:
|
|
|
|
{object_id -> object_data_in_dict}
|
|
|
|
object_data_in_dict's schema is in ObjectState
|
|
|
|
"""
|
2022-05-24 19:56:49 +09:00
|
|
|
raylet_ids = self._client.get_all_registered_raylet_ids()
|
2022-04-22 10:45:03 +09:00
|
|
|
replies = await asyncio.gather(
|
|
|
|
*[
|
2022-04-23 07:59:46 +09:00
|
|
|
self._client.get_object_info(node_id, timeout=option.timeout)
|
2022-05-24 19:56:49 +09:00
|
|
|
for node_id in raylet_ids
|
|
|
|
],
|
|
|
|
return_exceptions=True,
|
2022-04-22 10:45:03 +09:00
|
|
|
)
|
|
|
|
|
2022-05-24 19:56:49 +09:00
|
|
|
unresponsive_nodes = 0
|
2022-04-22 10:45:03 +09:00
|
|
|
worker_stats = []
|
2022-06-29 10:33:57 +09:00
|
|
|
total_objects = 0
|
2022-06-22 22:21:50 +09:00
|
|
|
for reply, _ in zip(replies, raylet_ids):
|
2022-05-24 19:56:49 +09:00
|
|
|
if isinstance(reply, DataSourceUnavailable):
|
|
|
|
unresponsive_nodes += 1
|
|
|
|
continue
|
|
|
|
elif isinstance(reply, Exception):
|
|
|
|
raise reply
|
|
|
|
|
2022-06-29 10:33:57 +09:00
|
|
|
total_objects += reply.total
|
2022-04-22 10:45:03 +09:00
|
|
|
for core_worker_stat in reply.core_workers_stats:
|
|
|
|
# NOTE: Set preserving_proto_field_name=False here because
|
|
|
|
# `construct_memory_table` requires a dictionary that has
|
|
|
|
# modified protobuf name
|
|
|
|
# (e.g., workerId instead of worker_id) as a key.
|
|
|
|
worker_stats.append(
|
|
|
|
self._message_to_dict(
|
|
|
|
message=core_worker_stat,
|
|
|
|
fields_to_decode=["object_id"],
|
|
|
|
preserving_proto_field_name=False,
|
|
|
|
)
|
|
|
|
)
|
2022-04-23 07:59:46 +09:00
|
|
|
|
2022-05-24 19:56:49 +09:00
|
|
|
partial_failure_warning = None
|
|
|
|
if len(raylet_ids) > 0 and unresponsive_nodes > 0:
|
|
|
|
warning_msg = NODE_QUERY_FAILURE_WARNING.format(
|
|
|
|
type="raylet",
|
|
|
|
total=len(raylet_ids),
|
|
|
|
network_failures=unresponsive_nodes,
|
|
|
|
log_command="raylet.out",
|
|
|
|
)
|
|
|
|
if unresponsive_nodes == len(raylet_ids):
|
|
|
|
raise DataSourceUnavailable(warning_msg)
|
|
|
|
partial_failure_warning = (
|
|
|
|
f"The returned data may contain incomplete result. {warning_msg}"
|
|
|
|
)
|
|
|
|
|
2022-04-23 07:59:46 +09:00
|
|
|
result = []
|
2022-04-22 10:45:03 +09:00
|
|
|
memory_table = memory_utils.construct_memory_table(worker_stats)
|
|
|
|
for entry in memory_table.table:
|
|
|
|
data = entry.as_dict()
|
|
|
|
# `construct_memory_table` returns object_ref field which is indeed
|
|
|
|
# object_id. We do transformation here.
|
|
|
|
# TODO(sang): Refactor `construct_memory_table`.
|
|
|
|
data["object_id"] = data["object_ref"]
|
|
|
|
del data["object_ref"]
|
2022-06-28 23:56:44 +09:00
|
|
|
data["ip"] = data["node_ip_address"]
|
|
|
|
del data["node_ip_address"]
|
2022-04-23 07:59:46 +09:00
|
|
|
result.append(data)
|
|
|
|
|
2022-07-24 19:55:36 -07:00
|
|
|
# Add callsite warnings if it is not configured.
|
|
|
|
callsite_warning = []
|
|
|
|
callsite_enabled = env_integer("RAY_record_ref_creation_sites", 0)
|
|
|
|
if not callsite_enabled:
|
|
|
|
callsite_warning.append(
|
|
|
|
"Callsite is not being recorded. "
|
|
|
|
"To record callsite information for each ObjectRef created, set "
|
|
|
|
"env variable RAY_record_ref_creation_sites=1 during `ray start` "
|
|
|
|
"and `ray.init`."
|
|
|
|
)
|
|
|
|
|
2022-07-25 23:31:49 -07:00
|
|
|
num_after_truncation = len(result)
|
2022-06-28 23:56:44 +09:00
|
|
|
result = self._filter(result, option.filters, ObjectState, option.detail)
|
2022-07-25 23:31:49 -07:00
|
|
|
num_filtered = len(result)
|
2022-04-23 07:59:46 +09:00
|
|
|
# Sort to make the output deterministic.
|
|
|
|
result.sort(key=lambda entry: entry["object_id"])
|
2022-06-29 10:33:57 +09:00
|
|
|
result = list(islice(result, option.limit))
|
2022-05-24 19:56:49 +09:00
|
|
|
return ListApiResponse(
|
2022-06-29 10:33:57 +09:00
|
|
|
result=result,
|
2022-05-24 19:56:49 +09:00
|
|
|
partial_failure_warning=partial_failure_warning,
|
2022-06-29 10:33:57 +09:00
|
|
|
total=total_objects,
|
2022-07-25 23:31:49 -07:00
|
|
|
num_after_truncation=num_after_truncation,
|
|
|
|
num_filtered=num_filtered,
|
2022-07-24 19:55:36 -07:00
|
|
|
warnings=callsite_warning,
|
2022-05-24 19:56:49 +09:00
|
|
|
)
|
2022-04-22 10:45:03 +09:00
|
|
|
|
2022-05-24 19:56:49 +09:00
|
|
|
async def list_runtime_envs(self, *, option: ListApiOptions) -> ListApiResponse:
|
2022-05-03 06:01:00 +09:00
|
|
|
"""List all runtime env information from the cluster.
|
|
|
|
|
|
|
|
Returns:
|
|
|
|
A list of runtime env information in the cluster.
|
|
|
|
The schema of returned "dict" is equivalent to the
|
|
|
|
`RuntimeEnvState` protobuf message.
|
|
|
|
We don't have id -> data mapping like other API because runtime env
|
|
|
|
doesn't have unique ids.
|
|
|
|
"""
|
2022-05-24 19:56:49 +09:00
|
|
|
agent_ids = self._client.get_all_registered_agent_ids()
|
2022-05-03 06:01:00 +09:00
|
|
|
replies = await asyncio.gather(
|
|
|
|
*[
|
|
|
|
self._client.get_runtime_envs_info(node_id, timeout=option.timeout)
|
2022-05-24 19:56:49 +09:00
|
|
|
for node_id in agent_ids
|
|
|
|
],
|
|
|
|
return_exceptions=True,
|
2022-05-03 06:01:00 +09:00
|
|
|
)
|
2022-05-24 19:56:49 +09:00
|
|
|
|
2022-05-03 06:01:00 +09:00
|
|
|
result = []
|
2022-05-24 19:56:49 +09:00
|
|
|
unresponsive_nodes = 0
|
2022-06-29 10:33:57 +09:00
|
|
|
total_runtime_envs = 0
|
2022-05-03 06:01:00 +09:00
|
|
|
for node_id, reply in zip(self._client.get_all_registered_agent_ids(), replies):
|
2022-05-24 19:56:49 +09:00
|
|
|
if isinstance(reply, DataSourceUnavailable):
|
|
|
|
unresponsive_nodes += 1
|
|
|
|
continue
|
|
|
|
elif isinstance(reply, Exception):
|
|
|
|
raise reply
|
|
|
|
|
2022-06-29 10:33:57 +09:00
|
|
|
total_runtime_envs += reply.total
|
2022-05-03 06:01:00 +09:00
|
|
|
states = reply.runtime_env_states
|
|
|
|
for state in states:
|
|
|
|
data = self._message_to_dict(message=state, fields_to_decode=[])
|
2022-07-25 23:31:49 -07:00
|
|
|
# Need to deserialize this field.
|
2022-05-03 06:01:00 +09:00
|
|
|
data["runtime_env"] = RuntimeEnv.deserialize(
|
|
|
|
data["runtime_env"]
|
|
|
|
).to_dict()
|
|
|
|
data["node_id"] = node_id
|
|
|
|
result.append(data)
|
|
|
|
|
2022-05-24 19:56:49 +09:00
|
|
|
partial_failure_warning = None
|
|
|
|
if len(agent_ids) > 0 and unresponsive_nodes > 0:
|
|
|
|
warning_msg = NODE_QUERY_FAILURE_WARNING.format(
|
|
|
|
type="agent",
|
|
|
|
total=len(agent_ids),
|
|
|
|
network_failures=unresponsive_nodes,
|
|
|
|
log_command="dashboard_agent.log",
|
|
|
|
)
|
|
|
|
if unresponsive_nodes == len(agent_ids):
|
|
|
|
raise DataSourceUnavailable(warning_msg)
|
|
|
|
partial_failure_warning = (
|
|
|
|
f"The returned data may contain incomplete result. {warning_msg}"
|
|
|
|
)
|
2022-07-25 23:31:49 -07:00
|
|
|
num_after_truncation = len(result)
|
2022-06-28 23:56:44 +09:00
|
|
|
result = self._filter(result, option.filters, RuntimeEnvState, option.detail)
|
2022-07-25 23:31:49 -07:00
|
|
|
num_filtered = len(result)
|
2022-06-04 09:17:30 +09:00
|
|
|
|
2022-05-03 06:01:00 +09:00
|
|
|
# Sort to make the output deterministic.
|
|
|
|
def sort_func(entry):
|
|
|
|
# If creation time is not there yet (runtime env is failed
|
|
|
|
# to be created or not created yet, they are the highest priority.
|
|
|
|
# Otherwise, "bigger" creation time is coming first.
|
|
|
|
if "creation_time_ms" not in entry:
|
|
|
|
return float("inf")
|
|
|
|
elif entry["creation_time_ms"] is None:
|
|
|
|
return float("inf")
|
|
|
|
else:
|
|
|
|
return float(entry["creation_time_ms"])
|
|
|
|
|
|
|
|
result.sort(key=sort_func, reverse=True)
|
2022-06-29 10:33:57 +09:00
|
|
|
result = list(islice(result, option.limit))
|
2022-05-24 19:56:49 +09:00
|
|
|
return ListApiResponse(
|
2022-06-29 10:33:57 +09:00
|
|
|
result=result,
|
2022-05-24 19:56:49 +09:00
|
|
|
partial_failure_warning=partial_failure_warning,
|
2022-06-29 10:33:57 +09:00
|
|
|
total=total_runtime_envs,
|
2022-07-25 23:31:49 -07:00
|
|
|
num_after_truncation=num_after_truncation,
|
|
|
|
num_filtered=num_filtered,
|
2022-05-24 19:56:49 +09:00
|
|
|
)
|
2022-05-03 06:01:00 +09:00
|
|
|
|
2022-06-22 22:21:50 +09:00
|
|
|
async def summarize_tasks(self, option: SummaryApiOptions) -> SummaryApiResponse:
|
2022-06-29 10:33:57 +09:00
|
|
|
# For summary, try getting as many entries as possible to minimze data loss.
|
2022-06-22 22:21:50 +09:00
|
|
|
result = await self.list_tasks(
|
2022-07-25 23:31:49 -07:00
|
|
|
option=ListApiOptions(
|
|
|
|
timeout=option.timeout, limit=RAY_MAX_LIMIT_FROM_API_SERVER, filters=[]
|
|
|
|
)
|
2022-06-22 22:21:50 +09:00
|
|
|
)
|
|
|
|
summary = StateSummary(
|
|
|
|
node_id_to_summary={
|
|
|
|
"cluster": TaskSummaries.to_summary(tasks=result.result)
|
|
|
|
}
|
|
|
|
)
|
|
|
|
return SummaryApiResponse(
|
2022-07-24 19:55:36 -07:00
|
|
|
total=result.total,
|
|
|
|
result=summary,
|
|
|
|
partial_failure_warning=result.partial_failure_warning,
|
|
|
|
warnings=result.warnings,
|
2022-07-25 23:31:49 -07:00
|
|
|
num_after_truncation=result.num_after_truncation,
|
2022-07-27 22:56:30 +09:00
|
|
|
# Currently, there's no filtering support for summary,
|
|
|
|
# so we don't calculate this separately.
|
|
|
|
num_filtered=len(result.result),
|
2022-06-22 22:21:50 +09:00
|
|
|
)
|
|
|
|
|
|
|
|
async def summarize_actors(self, option: SummaryApiOptions) -> SummaryApiResponse:
|
2022-06-29 10:33:57 +09:00
|
|
|
# For summary, try getting as many entries as possible to minimze data loss.
|
2022-06-22 22:21:50 +09:00
|
|
|
result = await self.list_actors(
|
2022-07-25 23:31:49 -07:00
|
|
|
option=ListApiOptions(
|
|
|
|
timeout=option.timeout, limit=RAY_MAX_LIMIT_FROM_API_SERVER, filters=[]
|
|
|
|
)
|
2022-06-22 22:21:50 +09:00
|
|
|
)
|
|
|
|
summary = StateSummary(
|
|
|
|
node_id_to_summary={
|
|
|
|
"cluster": ActorSummaries.to_summary(actors=result.result)
|
|
|
|
}
|
|
|
|
)
|
|
|
|
return SummaryApiResponse(
|
2022-07-24 19:55:36 -07:00
|
|
|
total=result.total,
|
|
|
|
result=summary,
|
|
|
|
partial_failure_warning=result.partial_failure_warning,
|
|
|
|
warnings=result.warnings,
|
2022-07-25 23:31:49 -07:00
|
|
|
num_after_truncation=result.num_after_truncation,
|
2022-07-27 22:56:30 +09:00
|
|
|
# Currently, there's no filtering support for summary,
|
|
|
|
# so we don't calculate this separately.
|
|
|
|
num_filtered=len(result.result),
|
2022-06-22 22:21:50 +09:00
|
|
|
)
|
|
|
|
|
|
|
|
async def summarize_objects(self, option: SummaryApiOptions) -> SummaryApiResponse:
|
2022-07-25 23:31:49 -07:00
|
|
|
# For summary, try getting as many entries as possible to minimize data loss.
|
2022-06-22 22:21:50 +09:00
|
|
|
result = await self.list_objects(
|
2022-07-25 23:31:49 -07:00
|
|
|
option=ListApiOptions(
|
|
|
|
timeout=option.timeout, limit=RAY_MAX_LIMIT_FROM_API_SERVER, filters=[]
|
|
|
|
)
|
2022-06-22 22:21:50 +09:00
|
|
|
)
|
|
|
|
summary = StateSummary(
|
|
|
|
node_id_to_summary={
|
|
|
|
"cluster": ObjectSummaries.to_summary(objects=result.result)
|
|
|
|
}
|
|
|
|
)
|
|
|
|
return SummaryApiResponse(
|
2022-07-24 19:55:36 -07:00
|
|
|
total=result.total,
|
|
|
|
result=summary,
|
|
|
|
partial_failure_warning=result.partial_failure_warning,
|
|
|
|
warnings=result.warnings,
|
2022-07-25 23:31:49 -07:00
|
|
|
num_after_truncation=result.num_after_truncation,
|
2022-07-27 22:56:30 +09:00
|
|
|
# Currently, there's no filtering support for summary,
|
|
|
|
# so we don't calculate this separately.
|
|
|
|
num_filtered=len(result.result),
|
2022-06-22 22:21:50 +09:00
|
|
|
)
|
|
|
|
|
2022-04-22 10:45:03 +09:00
|
|
|
def _message_to_dict(
|
|
|
|
self,
|
|
|
|
*,
|
|
|
|
message,
|
|
|
|
fields_to_decode: List[str],
|
|
|
|
preserving_proto_field_name: bool = True,
|
|
|
|
) -> dict:
|
2022-04-19 13:15:43 +09:00
|
|
|
return dashboard_utils.message_to_dict(
|
|
|
|
message,
|
|
|
|
fields_to_decode,
|
|
|
|
including_default_value_fields=True,
|
2022-04-22 10:45:03 +09:00
|
|
|
preserving_proto_field_name=preserving_proto_field_name,
|
2022-04-19 13:15:43 +09:00
|
|
|
)
|