@@ -194,7 +194,7 @@ from ray.util.scheduling_strategies import (
194194import ray._private.ray_constants as ray_constants
195195import ray.cloudpickle as ray_pickle
196196from ray.core.generated.common_pb2 import ActorDiedErrorContext
197- from ray.core.generated.gcs_pb2 import JobTableData
197+ from ray.core.generated.gcs_pb2 import JobTableData, GcsNodeInfo
198198from ray.core.generated.gcs_service_pb2 import GetAllResourceUsageReply
199199from ray._private.async_compat import (
200200 sync_to_async,
@@ -2880,27 +2880,26 @@ cdef class GcsClient:
28802880 c_uri, expiration_s, timeout_ms))
28812881
28822882 @_auto_reconnect
2883- def get_all_node_info (self , timeout = None ):
2883+ def get_all_node_info (self , timeout = None ) -> Dict[ NodeID , GcsNodeInfo] :
28842884 cdef:
28852885 int64_t timeout_ms = round (1000 * timeout) if timeout else - 1
2886- CGcsNodeInfo node_info
2887- c_vector[CGcsNodeInfo] node_infos
2886+ CGcsNodeInfo c_node_info
2887+ c_vector[CGcsNodeInfo] c_node_infos
2888+ c_vector[c_string] serialized_node_infos
28882889 with nogil:
2889- check_status(self .inner.get().GetAllNodeInfo(timeout_ms, node_infos))
2890+ check_status(self.inner.get().GetAllNodeInfo(timeout_ms , c_node_infos ))
2891+ for c_node_info in c_node_infos:
2892+ serialized_node_infos.push_back(c_node_info.SerializeAsString())
28902893
28912894 result = {}
2892- for node_info in node_infos:
2893- c_resources = PythonGetResourcesTotal(node_info)
2894- result[node_info.node_id()] = {
2895- " node_name" : node_info.node_name(),
2896- " state" : node_info.state(),
2897- " labels" : PythonGetNodeLabels(node_info),
2898- " resources" : {key.decode(): value for key, value in c_resources}
2899- }
2895+ for serialized in serialized_node_infos:
2896+ node_info = GcsNodeInfo()
2897+ node_info.ParseFromString(serialized )
2898+ result[NodeID.from_binary(node_info.node_id )] = node_info
29002899 return result
29012900
29022901 @_auto_reconnect
2903- def get_all_job_info (self , timeout = None ) -> Dict[bytes , JobTableData]:
2902+ def get_all_job_info(self , timeout = None ) -> Dict[JobID , JobTableData]:
29042903 # Ideally we should use json_format.MessageToDict(job_info ),
29052904 # but `job_info` is a cpp pb message not a python one.
29062905 # Manually converting each and every protobuf field is out of question ,
@@ -2918,7 +2917,7 @@ cdef class GcsClient:
29182917 for serialized in serialized_job_infos:
29192918 job_info = JobTableData()
29202919 job_info.ParseFromString(serialized )
2921- result[job_info.job_id] = job_info
2920+ result[JobID.from_binary( job_info.job_id ) ] = job_info
29222921 return result
29232922
29242923 @_auto_reconnect
0 commit comments