Skip to content

Commit

Permalink
[Dashboard] Add API support for the logical view and machine view in …
Browse files Browse the repository at this point in the history
…new backend (ray-project#11012)

* Add API support for the logical view and machine view, which lean on datacenter in common.

* Update dashboard/datacenter.py

Co-authored-by: fyrestone <fyrestone@outlook.com>

* Update dashboard/modules/logical_view/logical_view_head.py

Co-authored-by: fyrestone <fyrestone@outlook.com>

* Address PR comments

* lint

* Add dashboard tests to CI build

* Fix integration issues

* lint

Co-authored-by: Max Fitton <max@semprehealth.com>
Co-authored-by: fyrestone <fyrestone@outlook.com>
  • Loading branch information
3 people authored Oct 3, 2020
1 parent ee71fec commit ff6d412
Show file tree
Hide file tree
Showing 10 changed files with 460 additions and 19 deletions.
12 changes: 6 additions & 6 deletions .travis.yml
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ matrix:
- RAY_CYTHON_EXAMPLES=1
- RAY_USE_RANDOM_PORTS=1
install:
- . ./ci/travis/ci.sh init RAY_CI_SERVE_AFFECTED,RAY_CI_TUNE_AFFECTED,RAY_CI_PYTHON_AFFECTED
- . ./ci/travis/ci.sh init RAY_CI_SERVE_AFFECTED,RAY_CI_TUNE_AFFECTED,RAY_CI_PYTHON_AFFECTED,RAY_CI_DASHBOARD_AFFECTED
before_script:
- . ./ci/travis/ci.sh build

Expand All @@ -40,7 +40,7 @@ matrix:
- RAY_CYTHON_EXAMPLES=1
- RAY_USE_RANDOM_PORTS=1
install:
- . ./ci/travis/ci.sh init RAY_CI_SERVE_AFFECTED,RAY_CI_TUNE_AFFECTED,RAY_CI_PYTHON_AFFECTED
- . ./ci/travis/ci.sh init RAY_CI_SERVE_AFFECTED,RAY_CI_TUNE_AFFECTED,RAY_CI_PYTHON_AFFECTED,RAY_CI_DASHBOARD_AFFECTED
before_script:
- . ./ci/travis/ci.sh build
script:
Expand All @@ -55,7 +55,7 @@ matrix:
- RAY_CYTHON_EXAMPLES=1
- RAY_USE_RANDOM_PORTS=1
install:
- . ./ci/travis/ci.sh init RAY_CI_SERVE_AFFECTED,RAY_CI_TUNE_AFFECTED,RAY_CI_PYTHON_AFFECTED
- . ./ci/travis/ci.sh init RAY_CI_SERVE_AFFECTED,RAY_CI_TUNE_AFFECTED,RAY_CI_PYTHON_AFFECTED,RAY_CI_DASHBOARD_AFFECTED
before_script:
- . ./ci/travis/ci.sh build
script:
Expand Down Expand Up @@ -84,7 +84,7 @@ matrix:
- RAY_CYTHON_EXAMPLES=1
- RAY_USE_RANDOM_PORTS=1
install:
- . ./ci/travis/ci.sh init RAY_CI_SERVE_AFFECTED,RAY_CI_TUNE_AFFECTED,RAY_CI_PYTHON_AFFECTED
- . ./ci/travis/ci.sh init RAY_CI_SERVE_AFFECTED,RAY_CI_TUNE_AFFECTED,RAY_CI_PYTHON_AFFECTED,RAY_CI_DASHBOARD_AFFECTED
before_script:
- . ./ci/travis/ci.sh build

Expand All @@ -97,7 +97,7 @@ matrix:
- RAY_CYTHON_EXAMPLES=1
- RAY_USE_RANDOM_PORTS=1
install:
- . ./ci/travis/ci.sh init RAY_CI_SERVE_AFFECTED,RAY_CI_TUNE_AFFECTED,RAY_CI_PYTHON_AFFECTED
- . ./ci/travis/ci.sh init RAY_CI_SERVE_AFFECTED,RAY_CI_TUNE_AFFECTED,RAY_CI_PYTHON_AFFECTED,RAY_CI_DASHBOARD_AFFECTED
before_script:
- . ./ci/travis/ci.sh build
script:
Expand All @@ -113,7 +113,7 @@ matrix:
- RAY_CYTHON_EXAMPLES=1
- RAY_USE_RANDOM_PORTS=1
install:
- . ./ci/travis/ci.sh init RAY_CI_SERVE_AFFECTED,RAY_CI_TUNE_AFFECTED,RAY_CI_PYTHON_AFFECTED
- . ./ci/travis/ci.sh init RAY_CI_SERVE_AFFECTED,RAY_CI_TUNE_AFFECTED,RAY_CI_PYTHON_AFFECTED,RAY_CI_DASHBOARD_AFFECTED
before_script:
- . ./ci/travis/ci.sh build
script:
Expand Down
121 changes: 115 additions & 6 deletions dashboard/datacenter.py
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
import logging

import ray.new_dashboard.consts as dashboard_consts
import ray.new_dashboard.memory_utils as memory_utils
from ray.new_dashboard.actor_utils import actor_classname_from_task_spec
from ray.new_dashboard.utils import Dict, Signal

logger = logging.getLogger(__name__)
Expand All @@ -27,6 +28,12 @@ class DataSource:
node_id_to_ip = Dict()
# {node id hex(str): hostname(str)}
node_id_to_hostname = Dict()
# {node ip (str): log entries by pid
# (dict from pid to list of latest log entries)}
ip_and_pid_to_logs = Dict()
# {node ip (str): error entries by pid
# (dict from pid to list of latest err entries)}
ip_and_pid_to_errors = Dict()


class DataOrganizer:
Expand Down Expand Up @@ -54,12 +61,26 @@ async def purge():
@classmethod
async def get_node_actors(cls, node_id):
node_stats = DataSource.node_stats.get(node_id, {})
node_worker_id_set = set()
worker_id_to_info = {}
for worker_stats in node_stats.get("workersStats", []):
node_worker_id_set.add(worker_stats["workerId"])
worker_id_to_info[worker_stats["workerId"]] = worker_stats

node_actors = {}
for actor_id, actor_table_data in DataSource.actors.items():
if actor_table_data["address"]["workerId"] in node_worker_id_set:
if actor_table_data["address"]["workerId"] in worker_id_to_info:
worker_stats = worker_id_to_info[actor_table_data["address"][
"workerId"]]

actor_constructor = worker_stats.get("coreWorkerStats", {})\
.get("actorTitle", "Unknown actor constructor")

actor_table_data["actorConstructor"] = actor_constructor

actor_class = actor_classname_from_task_spec(
actor_table_data.get("taskSpec", {}))

actor_table_data["actorClass"] = actor_class
actor_table_data.update(worker_stats["coreWorkerStats"])
node_actors[actor_id] = actor_table_data
return node_actors

Expand All @@ -69,6 +90,18 @@ async def get_node_info(cls, node_id):
node_stats = DataSource.node_stats.get(node_id, {})
node = DataSource.nodes.get(node_id, {})

# Merge node log count information into the payload
log_info = DataSource.ip_and_pid_to_logs.get(node_physical_stats["ip"],
{})
node_log_count = 0
for entries in log_info.values():
node_log_count += len(entries)
error_info = DataSource.ip_and_pid_to_errors.get(
node_physical_stats["ip"], {})
node_err_count = 0
for entries in error_info.values():
node_err_count += len(entries)

# Merge coreWorkerStats (node stats) to workers (node physical stats)
workers_stats = node_stats.pop("workersStats", {})
pid_to_worker_stats = {}
Expand All @@ -88,15 +121,24 @@ async def get_node_info(cls, node_id):
worker["coreWorkerStats"] = list(worker_stats.values())
worker["language"] = pid_to_language.get(worker["pid"], "")
worker["jobId"] = pid_to_job_id.get(worker["pid"], "ffff")
worker["logCount"] = len(log_info.get(str(worker["pid"]), []))
worker["errorCount"] = len(error_info.get(str(worker["pid"]), []))

ray_stats = _extract_view_data(
node_stats["viewData"],
{"object_store_used_memory", "object_store_available_memory"})

node_info = node_physical_stats
# Merge node stats to node physical stats
# Merge node stats to node physical stats under raylet
node_info["raylet"] = node_stats
node_info["raylet"].update(ray_stats)

# Merge GcsNodeInfo to node physical stats
node_info["raylet"].update(node)
# Merge actors to node physical stats
node_info["actors"] = await cls.get_node_actors(node_id)

node_info["logCount"] = node_log_count
node_info["errorCount"] = node_err_count
await GlobalSignals.node_info_fetched.send(node_info)

return node_info
Expand All @@ -112,3 +154,70 @@ async def get_all_node_summary(cls):
node_info["raylet"].pop("viewData", None)
all_nodes_summary.append(node_info)
return all_nodes_summary

@classmethod
async def get_all_node_details(cls):
node_details = []
for node_id in DataSource.nodes.keys():
node_details.append(await cls.get_node_info(node_id))
return node_details

@classmethod
async def get_all_actors(cls):
all_actors = {}
for node_id in DataSource.nodes.keys():
all_actors.update(await cls.get_node_actors(node_id))
return all_actors

@classmethod
async def get_actor_creation_tasks(cls):
infeasible_tasks = sum(
(node_stats.get("infeasibleTasks", [])
for node_stats in DataSource.node_stats.values()), [])
for task in infeasible_tasks:
task["actorClass"] = actor_classname_from_task_spec(task)
task["state"] = "INFEASIBLE"

resource_pending_tasks = sum(
(data.get("readyTasks", [])
for data in DataSource.node_stats.values()), [])
for task in resource_pending_tasks:
task["actorClass"] = actor_classname_from_task_spec(task)
task["state"] = "PENDING_RESOURCES"

results = {
task["actorCreationTaskSpec"]["actorId"]: task
for task in resource_pending_tasks + infeasible_tasks
}
return results

@classmethod
async def get_memory_table(cls,
sort_by=memory_utils.SortingType.OBJECT_SIZE,
group_by=memory_utils.GroupByType.STACK_TRACE):
all_worker_stats = []
for node_stats in DataSource.node_stats.values():
all_worker_stats.extend(node_stats.get("workersStats", []))
memory_information = memory_utils.construct_memory_table(
all_worker_stats, group_by=group_by, sort_by=sort_by)
return memory_information


def _extract_view_data(views, data_keys):
view_data = {}
for view in views:
view_name = view["viewName"]
if view_name in data_keys:
if not view.get("measures"):
view_data[view_name] = 0
continue
measure = view["measures"][0]
if "doubleValue" in measure:
measure_value = measure["doubleValue"]
elif "intValue" in measure:
measure_value = measure["intValue"]
else:
measure_value = 0
view_data[view_name] = measure_value

return view_data
4 changes: 2 additions & 2 deletions dashboard/memory_utils.py
Original file line number Diff line number Diff line change
Expand Up @@ -269,10 +269,10 @@ def as_dict(self):
}

def get_entries(self) -> List[dict]:
return [entry.__dict__() for entry in self.table]
return [entry.as_dict() for entry in self.table]

def __repr__(self):
return str(self.__dict__())
return str(self.as_dict())

def __str__(self):
return self.__repr__()
Expand Down
Empty file.
58 changes: 58 additions & 0 deletions dashboard/modules/logical_view/logical_view_head.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
import logging
import aiohttp.web
import ray.utils
import ray.new_dashboard.utils as dashboard_utils
import ray.new_dashboard.actor_utils as actor_utils
from ray.new_dashboard.utils import rest_response
from ray.new_dashboard.datacenter import DataOrganizer
from ray.core.generated import core_worker_pb2
from ray.core.generated import core_worker_pb2_grpc

from grpc.experimental import aio as aiogrpc

logger = logging.getLogger(__name__)
routes = dashboard_utils.ClassMethodRouteTable


class LogicalViewHead(dashboard_utils.DashboardHeadModule):
@routes.get("/logical/actor_groups")
async def get_actor_groups(self, req) -> aiohttp.web.Response:
actors = await DataOrganizer.get_all_actors()
actor_creation_tasks = await DataOrganizer.get_actor_creation_tasks()
# actor_creation_tasks have some common interface with actors,
# and they get processed and shown in tandem in the logical view
# hence we merge them together before constructing actor groups.
actors.update(actor_creation_tasks)
actor_groups = actor_utils.construct_actor_groups(actors)
return await rest_response(
success=True,
message="Fetched actor groups.",
actor_groups=actor_groups)

@routes.get("/logical/kill_actor")
async def kill_actor(self, req) -> aiohttp.web.Response:
try:
actor_id = req.query["actorId"]
ip_address = req.query["ipAddress"]
port = req.query["port"]
except KeyError:
return await rest_response(success=False, message="Bad Request")
try:
channel = aiogrpc.insecure_channel(f"{ip_address}:{port}")
stub = core_worker_pb2_grpc.CoreWorkerServiceStub(channel)

await stub.KillActor(
core_worker_pb2.KillActorRequest(
intended_actor_id=ray.utils.hex_to_binary(actor_id)))

except aiogrpc.AioRpcError:
# This always throws an exception because the worker
# is killed and the channel is closed on the worker side
# before this handler, however it deletes the actor correctly.
pass

return await rest_response(
success=True, message=f"Killed actor with id {actor_id}")

async def run(self, server):
pass
79 changes: 79 additions & 0 deletions dashboard/modules/logical_view/test_logical_view_head.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,79 @@
import os
import sys
import logging
import requests
import time
import traceback
import pytest
import ray
from ray.new_dashboard.tests.conftest import * # noqa
from ray.test_utils import (
format_web_url,
wait_until_server_available,
)

os.environ["RAY_USE_NEW_DASHBOARD"] = "1"

logger = logging.getLogger(__name__)


def test_actor_groups(ray_start_with_dashboard):
@ray.remote
class Foo:
def __init__(self, num):
self.num = num

def do_task(self):
return self.num

@ray.remote(num_gpus=1)
class InfeasibleActor:
pass

foo_actors = [Foo.remote(4), Foo.remote(5)]
infeasible_actor = InfeasibleActor.remote() # noqa
results = [actor.do_task.remote() for actor in foo_actors] # noqa
assert (wait_until_server_available(ray_start_with_dashboard["webui_url"])
is True)
webui_url = ray_start_with_dashboard["webui_url"]
webui_url = format_web_url(webui_url)

timeout_seconds = 5
start_time = time.time()
last_ex = None
while True:
time.sleep(1)
try:
response = requests.get(webui_url + "/logical/actor_groups")
response.raise_for_status()
actor_groups_resp = response.json()
assert actor_groups_resp["result"] is True, actor_groups_resp[
"msg"]
actor_groups = actor_groups_resp["data"]["actorGroups"]
assert "Foo" in actor_groups
summary = actor_groups["Foo"]["summary"]
# 2 __init__ tasks and 2 do_task tasks
assert summary["numExecutedTasks"] == 4
assert summary["stateToCount"]["ALIVE"] == 2

entries = actor_groups["Foo"]["entries"]
assert len(entries) == 2
assert "InfeasibleActor" in actor_groups

entries = actor_groups["InfeasibleActor"]["entries"]
assert "requiredResources" in entries[0]
assert "GPU" in entries[0]["requiredResources"]
break
except Exception as ex:
last_ex = ex
finally:
if time.time() > start_time + timeout_seconds:
ex_stack = traceback.format_exception(
type(last_ex), last_ex,
last_ex.__traceback__) if last_ex else []
ex_stack = "".join(ex_stack)
raise Exception(f"Timed out while testing, {ex_stack}")


if __name__ == "__main__":
sys.exit(pytest.main(["-v", __file__]))
2 changes: 1 addition & 1 deletion dashboard/modules/reporter/test_reporter.py
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ def getpid(self):
launch_profiling = None
while True:
# Sometimes some startup time is required
if time.time() - start_time > 10:
if time.time() - start_time > 15:
raise RayTestTimeoutException(
"Timed out while collecting profiling stats, "
f"launch_profiling: {launch_profiling}")
Expand Down
2 changes: 2 additions & 0 deletions dashboard/modules/stats_collector/stats_collector_consts.py
Original file line number Diff line number Diff line change
@@ -1,3 +1,5 @@
NODE_STATS_UPDATE_INTERVAL_SECONDS = 1
RETRY_GET_ALL_ACTOR_INFO_INTERVAL_SECONDS = 1
ACTOR_CHANNEL = "ACTOR"
ERROR_INFO_UPDATE_INTERVAL_SECONDS = 5
LOG_INFO_UPDATE_INTERVAL_SECONDS = 5
Loading

0 comments on commit ff6d412

Please sign in to comment.