Skip to content

Commit 0651d3b

Browse files
richardliawrobertnishihara
authored andcommitted
[tune/core] Use Global State API for resources (#3004)
1 parent faa31ae commit 0651d3b

File tree

2 files changed

+17
-8
lines changed

2 files changed

+17
-8
lines changed

python/ray/experimental/state.py

Lines changed: 13 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -1310,9 +1310,19 @@ def cluster_resources(self):
13101310

13111311
return dict(resources)
13121312

1313+
def _live_client_ids(self):
1314+
"""Returns a set of client IDs corresponding to clients still alive."""
1315+
return {
1316+
client["ClientID"]
1317+
for client in self.client_table() if client["IsInsertion"]
1318+
}
1319+
13131320
def available_resources(self):
13141321
"""Get the current available cluster resources.
13151322
1323+
This is different from `cluster_resources` in that this will return
1324+
idle (available) resources rather than total resources.
1325+
13161326
Note that this information can grow stale as tasks start and finish.
13171327
13181328
Returns:
@@ -1364,6 +1374,7 @@ def available_resources(self):
13641374
if local_scheduler_id not in local_scheduler_ids:
13651375
del available_resources_by_id[local_scheduler_id]
13661376
else:
1377+
# TODO(rliaw): Is this a fair assumption?
13671378
# Assumes the number of Redis clients does not change
13681379
subscribe_clients = [
13691380
redis_client.pubsub(ignore_subscribe_messages=True)
@@ -1373,7 +1384,7 @@ def available_resources(self):
13731384
subscribe_client.subscribe(
13741385
ray.gcs_utils.XRAY_HEARTBEAT_CHANNEL)
13751386

1376-
client_ids = {client["ClientID"] for client in self.client_table()}
1387+
client_ids = self._live_client_ids()
13771388

13781389
while set(available_resources_by_id.keys()) != client_ids:
13791390
for subscribe_client in subscribe_clients:
@@ -1403,10 +1414,7 @@ def available_resources(self):
14031414
available_resources_by_id[client_id] = dynamic_resources
14041415

14051416
# Update clients in cluster
1406-
client_ids = {
1407-
client["ClientID"]
1408-
for client in self.client_table()
1409-
}
1417+
client_ids = self._live_client_ids()
14101418

14111419
# Remove disconnected clients
14121420
for client_id in available_resources_by_id.keys():

python/ray/tune/ray_trial_executor.py

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -213,12 +213,13 @@ def _return_resources(self, resources):
213213
assert self._committed_resources.gpu >= 0
214214

215215
def _update_avail_resources(self):
216-
clients = ray.global_state.client_table()
217216
if ray.worker.global_worker.use_raylet:
218217
# TODO(rliaw): Remove once raylet flag is swapped
219-
num_cpus = sum(cl['Resources']['CPU'] for cl in clients)
220-
num_gpus = sum(cl['Resources'].get('GPU', 0) for cl in clients)
218+
resources = ray.global_state.cluster_resources()
219+
num_cpus = resources["CPU"]
220+
num_gpus = resources["GPU"]
221221
else:
222+
clients = ray.global_state.client_table()
222223
local_schedulers = [
223224
entry for client in clients.values() for entry in client
224225
if (entry['ClientType'] == 'local_scheduler'

0 commit comments

Comments
 (0)