Monitoring with the CLI or SDK#

Monitoring and debugging capabilities in Ray are available through a CLI or SDK.

CLI command ray status#

You can monitor node status and resource usage by running the CLI command, ray status, on the head node. It displays

  • Node Status: Nodes that are running and autoscaling up or down. Addresses of running nodes. Information about pending nodes and failed nodes.

  • Resource Usage: The Ray resource usage of the cluster. For example, requested CPUs from all Ray Tasks and Actors. Number of GPUs that are used.

Following is an example output:

$ ray status
======== Autoscaler status: 2021-10-12 13:10:21.035674 ========
Node status
---------------------------------------------------------------
Healthy:
 1 ray.head.default
 2 ray.worker.cpu
Pending:
 (no pending nodes)
Recent failures:
 (no failures)

Resources
---------------------------------------------------------------
Usage:
 0.0/10.0 CPU
 0.00/70.437 GiB memory
 0.00/10.306 GiB object_store_memory

Demands:
 (no resource demands)

When you need more verbose info about each node, run ray status -v. This is helpful when you need to investigate why particular nodes don’t autoscale down.

Ray State CLI and SDK#

Tip

Provide feedback on using Ray state APIs - feedback form!

Use Ray State APIs to access the current state (snapshot) of Ray through the CLI or Python SDK (developer APIs).

Note

This feature requires a full installation of Ray using pip install "ray[default]". This feature also requires that the dashboard component is available. The dashboard component needs to be included when starting the Ray Cluster, which is the default behavior for ray start and ray.init().

Note

State API CLI commands are stable, while Python SDKs are DeveloperAPI. CLI usage is recommended over Python SDKs.

Get started#

This example uses the following script that runs two Tasks and creates two Actors.

import ray
import time

ray.init(num_cpus=4)

@ray.remote
def task_running_300_seconds():
    time.sleep(300)

@ray.remote
class Actor:
    def __init__(self):
        pass

# Create 2 tasks
tasks = [task_running_300_seconds.remote() for _ in range(2)]

# Create 2 actors
actors = [Actor.remote() for _ in range(2)]

See the summarized states of tasks. If it doesn’t return the output immediately, retry the command.

ray summary tasks
======== Tasks Summary: 2022-07-22 08:54:38.332537 ========
Stats:
------------------------------------
total_actor_scheduled: 2
total_actor_tasks: 0
total_tasks: 2


Table (group by func_name):
------------------------------------
    FUNC_OR_CLASS_NAME        STATE_COUNTS    TYPE
0   task_running_300_seconds  RUNNING: 2      NORMAL_TASK
1   Actor.__init__            FINISHED: 2     ACTOR_CREATION_TASK
from ray.util.state import summarize_tasks
print(summarize_tasks())
{'cluster': {'summary': {'task_running_300_seconds': {'func_or_class_name': 'task_running_300_seconds', 'type': 'NORMAL_TASK', 'state_counts': {'RUNNING': 2}}, 'Actor.__init__': {'func_or_class_name': 'Actor.__init__', 'type': 'ACTOR_CREATION_TASK', 'state_counts': {'FINISHED': 2}}}, 'total_tasks': 2, 'total_actor_tasks': 0, 'total_actor_scheduled': 2, 'summary_by': 'func_name'}}

List all Actors.

ray list actors
======== List: 2022-07-23 21:29:39.323925 ========
Stats:
------------------------------
Total: 2

Table:
------------------------------
    ACTOR_ID                          CLASS_NAME    NAME      PID  STATE
0  31405554844820381c2f0f8501000000  Actor                 96956  ALIVE
1  f36758a9f8871a9ca993b1d201000000  Actor                 96955  ALIVE
from ray.util.state import list_actors
print(list_actors())
[ActorState(actor_id='...', class_name='Actor', state='ALIVE', job_id='01000000', name='', node_id='...', pid=..., ray_namespace='...', serialized_runtime_env=None, required_resources=None, death_cause=None, is_detached=None, placement_group_id=None, repr_name=None), ActorState(actor_id='...', class_name='Actor', state='ALIVE', job_id='01000000', name='', node_id='...', pid=..., ray_namespace='...', serialized_runtime_env=None, required_resources=None, death_cause=None, is_detached=None, placement_group_id=None, repr_name=None)]

Get the state of a single Task using the get API.

# In this case, 31405554844820381c2f0f8501000000
ray get actors <ACTOR_ID>
---
actor_id: 31405554844820381c2f0f8501000000
class_name: Actor
death_cause: null
is_detached: false
name: ''
pid: 96956
resource_mapping: []
serialized_runtime_env: '{}'
state: ALIVE
from ray.util.state import get_actor
# In this case, 31405554844820381c2f0f8501000000
print(get_actor(id=<ACTOR_ID>))

Access logs through the ray logs API.

ray list actors
# In this case, ACTOR_ID is 31405554844820381c2f0f8501000000
ray logs actor --id <ACTOR_ID>
--- Log has been truncated to last 1000 lines. Use `--tail` flag to toggle. ---

:actor_name:Actor
Actor created
from ray.util.state import get_log

# In this case, ACTOR_ID is 31405554844820381c2f0f8501000000
for line in get_log(actor_id=<ACTOR_ID>):
    print(line)

Key Concepts#

Ray State APIs allow you to access states of resources through summary, list, and get APIs. It also supports logs API to access logs.

  • states: The state of the cluster of corresponding resources. States consist of immutable metadata (e.g., Actor’s name) and mutable states (e.g., Actor’s scheduling state or pid).

  • resources: Resources created by Ray. E.g., actors, tasks, objects, placement groups, and etc.

  • summary: API to return the summarized view of resources.

  • list: API to return every individual entity of resources.

  • get: API to return a single entity of resources in detail.

  • logs: API to access the log of Actors, Tasks, Workers, or system log files.

User guides#

Getting a summary of states of entities by type#

Return the summarized information of the given Ray entity (Objects, Actors, Tasks). It is recommended to start monitoring states through summary APIs first. When you find anomalies (e.g., Actors running for a long time, Tasks that are not scheduled for a long time), you can use list or get APIs to get more details for an individual abnormal entity.

Summarize all actors

ray summary actors
from ray.util.state import summarize_actors
print(summarize_actors())
{'cluster': {'summary': {'Actor': {'class_name': 'Actor', 'state_counts': {'ALIVE': 2}}}, 'total_actors': 2, 'summary_by': 'class'}}

Summarize all tasks

ray summary tasks
from ray.util.state import summarize_tasks
print(summarize_tasks())
{'cluster': {'summary': {'task_running_300_seconds': {'func_or_class_name': 'task_running_300_seconds', 'type': 'NORMAL_TASK', 'state_counts': {'RUNNING': 2}}, 'Actor.__init__': {'func_or_class_name': 'Actor.__init__', 'type': 'ACTOR_CREATION_TASK', 'state_counts': {'FINISHED': 2}}}, 'total_tasks': 2, 'total_actor_tasks': 0, 'total_actor_scheduled': 2, 'summary_by': 'func_name'}}

Summarize all objects

Note

By default, objects are summarized by callsite. However, callsite is not recorded by Ray by default. To get callsite info, set env variable RAY_record_ref_creation_sites=1 when starting the Ray Cluster RAY_record_ref_creation_sites=1 ray start –head

ray summary objects
from ray.util.state import summarize_objects
print(summarize_objects())
{'cluster': {'summary': {'disabled': {'total_objects': 6, 'total_size_mb': 0.0, 'total_num_workers': 3, 'total_num_nodes': 1, 'task_state_counts': {'SUBMITTED_TO_WORKER': 2, 'FINISHED': 2, 'NIL': 2}, 'ref_type_counts': {'LOCAL_REFERENCE': 2, 'ACTOR_HANDLE': 4}}}, 'total_objects': 6, 'total_size_mb': 0.0, 'callsite_enabled': False, 'summary_by': 'callsite'}}

See state CLI reference for more details about ray summary command.

List the states of all entities of certain type#

Get a list of resources. Possible resources include:

List all nodes

ray list nodes
from ray.util.state import list_nodes
list_nodes()

List all placement groups

ray list placement-groups
from ray.util.state import list_placement_groups
list_placement_groups()

List local referenced objects created by a process

Tip

You can list resources with one or multiple filters: using --filter or -f

ray list objects -f pid=<PID> -f reference_type=LOCAL_REFERENCE
from ray.util.state import list_objects
list_objects(filters=[("pid", "=", 1234), ("reference_type", "=", "LOCAL_REFERENCE")])

List alive actors

ray list actors -f state=ALIVE
from ray.util.state import list_actors
list_actors(filters=[("state", "=", "ALIVE")])

List running tasks

ray list tasks -f state=RUNNING
from ray.util.state import list_tasks
list_tasks(filters=[("state", "=", "RUNNING")])

List non-running tasks

ray list tasks -f state!=RUNNING
from ray.util.state import list_tasks
list_tasks(filters=[("state", "!=", "RUNNING")])

List running tasks that have a name func

ray list tasks -f state=RUNNING -f name="task_running_300_seconds()"
from ray.util.state import list_tasks
list_tasks(filters=[("state", "=", "RUNNING"), ("name", "=", "task_running_300_seconds()")])

List tasks with more details

Tip

When --detail is specified, the API can query more data sources to obtain state information in details.

ray list tasks --detail
from ray.util.state import list_tasks
list_tasks(detail=True)

See state CLI reference for more details about ray list command.

Get the states of a particular entity (task, actor, etc.)#

Get a task’s states

ray get tasks <TASK_ID>
from ray.util.state import get_task
get_task(id=<TASK_ID>)

Get a node’s states

ray get nodes <NODE_ID>
from ray.util.state import get_node
get_node(id=<NODE_ID>)

See state CLI reference for more details about ray get command.

Fetch the logs of a particular entity (task, actor, etc.)#

State API also allows you to access Ray logs. Note that you cannot access the logs from a dead node. By default, the API prints logs from a head node.

Get all retrievable log file names from a head node in a cluster

ray logs cluster
# You could get the node ID / node IP from `ray list nodes`
from ray.util.state import list_logs
# `ray logs` by default print logs from a head node.
# To list the same logs, you should provide the head node ID.
# Get the node ID / node IP from `ray list nodes`
list_logs(node_id=<HEAD_NODE_ID>)

Get a particular log file from a node

# Get the node ID / node IP from `ray list nodes`
ray logs cluster gcs_server.out --node-id <NODE_ID>
# `ray logs cluster` is alias to `ray logs` when querying with globs.
ray logs gcs_server.out --node-id <NODE_ID>
from ray.util.state import get_log

# Node IP can be retrieved from list_nodes() or ray.nodes()
for line in get_log(filename="gcs_server.out", node_id=<NODE_ID>):
    print(line)

Stream a log file from a node

# Get the node ID / node IP from `ray list nodes`
ray logs raylet.out --node-ip <NODE_IP> --follow
# Or,
ray logs cluster raylet.out --node-ip <NODE_IP> --follow
from ray.util.state import get_log

# Retrieve the Node IP from list_nodes() or ray.nodes()
# The loop blocks with `follow=True`
for line in get_log(filename="raylet.out", node_ip=<NODE_IP>, follow=True):
    print(line)

Stream log from an actor with actor id

ray logs actor --id=<ACTOR_ID> --follow
from ray.util.state import get_log

# Get the Actor's ID from the output of `ray list actors`.
# The loop blocks with `follow=True`
for line in get_log(actor_id=<ACTOR_ID>, follow=True):
    print(line)

Stream log from a pid

ray logs worker --pid=<PID> --follow
from ray.util.state import get_log

# Retrieve the node IP from list_nodes() or ray.nodes()
# get the PID of the worker running the Actor easily when output
# of worker is directed to the driver (default)
# The loop blocks with `follow=True`
for line in get_log(pid=<PID>, node_ip=<NODE_IP>, follow=True):
    print(line)

See state CLI reference for more details about ray logs command.

Failure Semantics#

The State APIs don’t guarantee to return a consistent or complete snapshot of the cluster all the time. By default, all Python SDKs raise an exception when output is missing from the API. The CLI returns a partial result and provides warning messages. Here are cases where there can be missing output from the API.

Query Failures

State APIs query “data sources” (e.g., GCS, raylets, etc.) to obtain and build the snapshot of the Cluster. However, data sources are sometimes unavailable (e.g., the source is down or overloaded). In this case, APIs return a partial (incomplete) snapshot of the Cluster, and users are informed that the output is incomplete through a warning message. All warnings are printed through Python’s warnings library, and they can be suppressed.

Data Truncation

When the returned number of entities (number of rows) is too large (> 100K), state APIs truncate the output data to ensure system stability (when this happens, there’s no way to choose truncated data). When truncation happens it is informed through Python’s warnings module.

Garbage Collected Resources

Depending on the lifecycle of the resources, some “finished” resources are not accessible through the APIs because they are already garbage collected.

Note

Do not to rely on this API to obtain correct information on finished resources. For example, Ray periodically garbage collects DEAD state Actor data to reduce memory usage. Or it cleans up the FINISHED state of Tasks when its lineage goes out of scope.

API Reference#

Using Ray CLI tools from outside the cluster#

These CLI commands have to be run on a node in the Ray Cluster. Examples for executing these commands from a machine outside the Ray Cluster are provided below.

Execute a command on the cluster using ray exec:

$ ray exec <cluster config file> "ray status"

Execute a command on the cluster using kubectl exec and the configured RayCluster name. Ray uses the Service targeting the Ray head pod to execute a CLI command on the cluster.

# First, find the name of the Ray head service.
$ kubectl get pod | grep <RayCluster name>-head
# NAME                                             READY   STATUS    RESTARTS   AGE
# <RayCluster name>-head-xxxxx                     2/2     Running   0          XXs

# Then, use the name of the Ray head service to run `ray status`.
$ kubectl exec <RayCluster name>-head-xxxxx -- ray status