Skip to content

Commit

Permalink
[core][dashboard] Add Pg Id and Format Required Resources in /logical…
Browse files Browse the repository at this point in the history
…/actors Response (ray-project#47754)

## Problem to Solve
In issue ray-project#45658, we found that the placement group id and the required
resources are not displayed correctly in the actor UI tab in the
dashboard. To be specific:
* The "Placement group ID" column content is missing
* The "Required resources" column shows resources with pg ids and bundle
entries as well
<img width="2532" alt="Screenshot 2024-09-18 at 9 49 27 AM"
src="https://github.com/user-attachments/assets/1bdd3bd3-6216-459d-87ba-b28de8f3367e">

## Root Cause
With investigation, we found the root cause of the issue is that:
* The dashboard calls "/logical/actors" to get the actor information to
display
* The response content of the above REST API doesn't contain the
placement group id field and the required resources are not formatted
correctly
* Therefore, the dashboard fails to the display the corresponding
content correctly

## Changes in the PR
* Include `placementGroupId` in the keys to be translated/preserved in
the `actor_table_data_to_dict` function so that the pg id can be
included in the content of the REST API response
* Leverage the `parse_pg_formatted_resources_to_original` function to
reformatted the required resources before sending the REST API response
* Test logic added in `test_actors` tests to verify the logic of setting
the placement group id as well as the correctly formatted required
resources

Signed-off-by: Mengjin Yan <mengjinyan3@gmail.com>
Signed-off-by: ujjawal-khare <ujjawal.khare@dream11.com>
  • Loading branch information
MengjinYan authored and ujjawal-khare committed Oct 15, 2024
1 parent ae8a86a commit f13ce61
Show file tree
Hide file tree
Showing 3 changed files with 55 additions and 3 deletions.
11 changes: 10 additions & 1 deletion python/ray/dashboard/datacenter.py
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,10 @@
from typing import Any, List, Optional

import ray.dashboard.consts as dashboard_consts
from ray._private.utils import get_or_create_event_loop
from ray._private.utils import (
get_or_create_event_loop,
parse_pg_formatted_resources_to_original,
)
from ray.dashboard.utils import (
Dict,
MutableNotificationDict,
Expand Down Expand Up @@ -276,4 +279,10 @@ async def _get_actor_info(actor):
actor["gpus"] = actor_process_gpu_stats
actor["processStats"] = actor_process_stats
actor["mem"] = node_physical_stats.get("mem", [])

required_resources = parse_pg_formatted_resources_to_original(
actor["requiredResources"]
)
actor["requiredResources"] = required_resources

return actor
2 changes: 2 additions & 0 deletions python/ray/dashboard/modules/actor/actor_head.py
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@ def actor_table_data_to_dict(message):
"taskId",
"parentTaskId",
"sourceActorId",
"placementGroupId",
},
always_print_fields_with_no_presence=True,
)
Expand All @@ -71,6 +72,7 @@ def actor_table_data_to_dict(message):
"startTime",
"endTime",
"reprName",
"placementGroupId",
}
light_message = {k: v for (k, v) in orig_message.items() if k in fields}
light_message["actorClass"] = orig_message["className"]
Expand Down
45 changes: 43 additions & 2 deletions python/ray/dashboard/modules/actor/tests/test_actor.py
Original file line number Diff line number Diff line change
Expand Up @@ -11,11 +11,21 @@
from ray._private.test_utils import format_web_url, wait_until_server_available
from ray.dashboard.modules.actor import actor_consts
from ray.dashboard.tests.conftest import * # noqa
from ray.util.placement_group import placement_group
from ray.util.scheduling_strategies import PlacementGroupSchedulingStrategy

logger = logging.getLogger(__name__)


def test_actors(disable_aiohttp_cache, ray_start_with_dashboard):
"""
Tests the REST API dashboard calls on:
- alive actors
- infeasible actors
- dead actors
- pg acrors (with pg_id set and required_resources formatted)
"""

@ray.remote
class Foo:
def __init__(self, num):
Expand All @@ -39,17 +49,39 @@ def __repr__(self) -> str:
class InfeasibleActor:
pass

pg = placement_group([{"CPU": 1}])

@ray.remote(num_cpus=1)
class PgActor:
def __init__(self):
pass

def do_task(self):
return 1

def get_placement_group_id(self):
return ray.get_runtime_context().get_placement_group_id()

foo_actors = [Foo.options(name="first").remote(4), Foo.remote(5)]
infeasible_actor = InfeasibleActor.options(name="infeasible").remote() # noqa
dead_actor = Foo.options(name="dead").remote(1)
pg_actor = PgActor.options(
name="pg",
scheduling_strategy=PlacementGroupSchedulingStrategy(
placement_group=pg,
),
).remote()

ray.kill(dead_actor)
results = [actor.do_task.remote() for actor in foo_actors] # noqa
[actor.do_task.remote() for actor in foo_actors]
pg_actor.do_task.remote()
webui_url = ray_start_with_dashboard["webui_url"]
assert wait_until_server_available(webui_url)
webui_url = format_web_url(webui_url)
job_id = ray.get_runtime_context().get_job_id()
node_id = ray.get(foo_actors[0].get_node_id.remote())
pid = ray.get(foo_actors[0].get_pid.remote())
placement_group_id = ray.get(pg_actor.get_placement_group_id.remote())

timeout_seconds = 5
start_time = time.time()
Expand All @@ -61,7 +93,8 @@ class InfeasibleActor:
resp_json = resp.json()
resp_data = resp_json["data"]
actors = resp_data["actors"]
assert len(actors) == 4
assert len(actors) == 5

for a in actors.values():
if a["name"] == "first":
actor_response = a
Expand Down Expand Up @@ -104,6 +137,14 @@ class InfeasibleActor:
all_pids = {entry["pid"] for entry in actors.values()}
assert 0 in all_pids # The infeasible actor
assert len(all_pids) > 1

# Check the pg actor metadata.
for a in actors.values():
if a["name"] == "pg":
pg_actor_response = a
assert pg_actor_response["placementGroupId"] == placement_group_id
assert pg_actor_response["requiredResources"] == {"CPU": 1.0}

break
except Exception as ex:
last_ex = ex
Expand Down

0 comments on commit f13ce61

Please sign in to comment.