Skip to content

Commit

Permalink
Rename ClientTableData to GcsNodeInfo (ray-project#5251)
Browse files Browse the repository at this point in the history
  • Loading branch information
micafan authored and raulchen committed Jul 30, 2019
1 parent 3ba8680 commit b3bcf59
Show file tree
Hide file tree
Showing 21 changed files with 333 additions and 339 deletions.
32 changes: 16 additions & 16 deletions java/runtime/src/main/java/org/ray/runtime/gcs/GcsClient.java
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
import org.ray.api.runtimecontext.NodeInfo;
import org.ray.runtime.generated.Gcs;
import org.ray.runtime.generated.Gcs.ActorCheckpointIdData;
import org.ray.runtime.generated.Gcs.ClientTableData;
import org.ray.runtime.generated.Gcs.GcsNodeInfo;
import org.ray.runtime.generated.Gcs.TablePrefix;
import org.ray.runtime.util.IdUtil;
import org.slf4j.Logger;
Expand Down Expand Up @@ -60,40 +60,40 @@ public List<NodeInfo> getAllNodeInfo() {
return new ArrayList<>();
}

// This map is used for deduplication of client entries.
Map<UniqueId, NodeInfo> clients = new HashMap<>();
// This map is used for deduplication of node entries.
Map<UniqueId, NodeInfo> nodes = new HashMap<>();
for (byte[] result : results) {
Preconditions.checkNotNull(result);
ClientTableData data = null;
GcsNodeInfo data = null;
try {
data = ClientTableData.parseFrom(result);
data = GcsNodeInfo.parseFrom(result);
} catch (InvalidProtocolBufferException e) {
throw new RuntimeException("Received invalid protobuf data from GCS.");
}
final UniqueId clientId = UniqueId
.fromByteBuffer(data.getClientId().asReadOnlyByteBuffer());
final UniqueId nodeId = UniqueId
.fromByteBuffer(data.getNodeId().asReadOnlyByteBuffer());

if (data.getIsInsertion()) {
if (data.getState() == GcsNodeInfo.GcsNodeState.ALIVE) {
//Code path of node insertion.
NodeInfo nodeInfo = new NodeInfo(
clientId, data.getNodeManagerAddress(), true, new HashMap<>());
clients.put(clientId, nodeInfo);
nodeId, data.getNodeManagerAddress(), true, new HashMap<>());
nodes.put(nodeId, nodeInfo);
} else {
// Code path of node deletion.
NodeInfo nodeInfo = new NodeInfo(clientId, clients.get(clientId).nodeAddress,
NodeInfo nodeInfo = new NodeInfo(nodeId, nodes.get(nodeId).nodeAddress,
false, new HashMap<>());
clients.put(clientId, nodeInfo);
nodes.put(nodeId, nodeInfo);
}
}

// Fill resources.
for (Map.Entry<UniqueId, NodeInfo> client : clients.entrySet()) {
if (client.getValue().isAlive) {
client.getValue().resources.putAll(getResourcesForClient(client.getKey()));
for (Map.Entry<UniqueId, NodeInfo> node : nodes.entrySet()) {
if (node.getValue().isAlive) {
node.getValue().resources.putAll(getResourcesForClient(node.getKey()));
}
}

return new ArrayList<>(clients.values());
return new ArrayList<>(nodes.values());
}

private Map<String, Double> getResourcesForClient(UniqueId clientId) {
Expand Down
4 changes: 2 additions & 2 deletions python/ray/gcs_utils.py
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,7 @@

from ray.core.generated.gcs_pb2 import (
ActorCheckpointIdData,
ClientTableData,
GcsNodeInfo,
JobTableData,
ErrorTableData,
ErrorType,
Expand All @@ -21,7 +21,7 @@

__all__ = [
"ActorCheckpointIdData",
"ClientTableData",
"GcsNodeInfo",
"JobTableData",
"ErrorTableData",
"ErrorType",
Expand Down
5 changes: 2 additions & 3 deletions python/ray/monitor.py
Original file line number Diff line number Diff line change
Expand Up @@ -279,13 +279,12 @@ def update_raylet_map(self, _append_port=False):
all_raylet_nodes = ray.nodes()
self.raylet_id_to_ip_map = {}
for raylet_info in all_raylet_nodes:
client_id = (raylet_info.get("DBClientID")
or raylet_info["ClientID"])
node_id = (raylet_info.get("DBClientID") or raylet_info["NodeID"])
ip_address = (raylet_info.get("AuxAddress")
or raylet_info["NodeManagerAddress"]).split(":")[0]
if _append_port:
ip_address += ":" + str(raylet_info["NodeManagerPort"])
self.raylet_id_to_ip_map[client_id] = ip_address
self.raylet_id_to_ip_map[node_id] = ip_address

def _maybe_flush_gcs(self):
"""Experimental: issue a flush request to the GCS.
Expand Down
78 changes: 39 additions & 39 deletions python/ray/state.py
Original file line number Diff line number Diff line change
Expand Up @@ -43,49 +43,49 @@ def _parse_client_table(redis_client):
node_info = {}
gcs_entry = gcs_utils.GcsEntry.FromString(message)

ordered_client_ids = []
ordered_node_ids = []

# Since GCS entries are append-only, we override so that
# only the latest entries are kept.
for entry in gcs_entry.entries:
client = gcs_utils.ClientTableData.FromString(entry)

client_id = ray.utils.binary_to_hex(client.client_id)

if client.is_insertion:
ordered_client_ids.append(client_id)
node_info[client_id] = {
"ClientID": client_id,
"IsInsertion": client.is_insertion,
"NodeManagerAddress": client.node_manager_address,
"NodeManagerPort": client.node_manager_port,
"ObjectManagerPort": client.object_manager_port,
"ObjectStoreSocketName": client.object_store_socket_name,
"RayletSocketName": client.raylet_socket_name
item = gcs_utils.GcsNodeInfo.FromString(entry)

node_id = ray.utils.binary_to_hex(item.node_id)

if item.state == gcs_utils.GcsNodeInfo.GcsNodeState.Value("ALIVE"):
ordered_node_ids.append(node_id)
node_info[node_id] = {
"NodeID": node_id,
"Alive": True,
"NodeManagerAddress": item.node_manager_address,
"NodeManagerPort": item.node_manager_port,
"ObjectManagerPort": item.object_manager_port,
"ObjectStoreSocketName": item.object_store_socket_name,
"RayletSocketName": item.raylet_socket_name
}

# If this client is being removed, then it must
# If this node is being removed, then it must
# have previously been inserted, and
# it cannot have previously been removed.
else:
assert client_id in node_info, "Client not found!"
assert node_info[client_id]["IsInsertion"], (
"Unexpected duplicate removal of client.")
node_info[client_id]["IsInsertion"] = client.is_insertion
assert node_id in node_info, "node not found!"
assert node_info[node_id]["Alive"], (
"Unexpected duplicate removal of node.")
node_info[node_id]["Alive"] = False
# Fill resource info.
for client_id in ordered_client_ids:
if node_info[client_id]["IsInsertion"]:
resources = _parse_resource_table(redis_client, client_id)
for node_id in ordered_node_ids:
if node_info[node_id]["Alive"]:
resources = _parse_resource_table(redis_client, node_id)
else:
resources = {}
node_info[client_id]["Resources"] = resources
node_info[node_id]["Resources"] = resources
# NOTE: We return the list comprehension below instead of simply doing
# 'list(node_info.values())' in order to have the nodes appear in the order
# that they joined the cluster. Python dictionaries do not preserve
# insertion order. We could use an OrderedDict, but then we'd have to be
# sure to only insert a given node a single time (clients that die appear
# twice in the GCS log).
return [node_info[client_id] for client_id in ordered_client_ids]
return [node_info[node_id] for node_id in ordered_node_ids]


def _parse_resource_table(redis_client, client_id):
Expand Down Expand Up @@ -402,7 +402,7 @@ def client_table(self):

for client in client_table:
# These are equivalent and is better for application developers.
client["alive"] = client["IsInsertion"]
client["alive"] = client["Alive"]
return client_table

def _job_table(self, job_id):
Expand Down Expand Up @@ -690,11 +690,11 @@ def chrome_tracing_object_transfer_dump(self, filename=None):
"""
self._check_connected()

client_id_to_address = {}
for client_info in self.client_table():
client_id_to_address[client_info["ClientID"]] = "{}:{}".format(
client_info["NodeManagerAddress"],
client_info["ObjectManagerPort"])
node_id_to_address = {}
for node_info in self.client_table():
node_id_to_address[node_info["NodeID"]] = "{}:{}".format(
node_info["NodeManagerAddress"],
node_info["ObjectManagerPort"])

all_events = []

Expand All @@ -705,13 +705,13 @@ def chrome_tracing_object_transfer_dump(self, filename=None):

for event in items:
if event["event_type"] == "transfer_send":
object_id, remote_client_id, _, _ = event["extra_data"]
object_id, remote_node_id, _, _ = event["extra_data"]

elif event["event_type"] == "transfer_receive":
object_id, remote_client_id, _, _ = event["extra_data"]
object_id, remote_node_id, _, _ = event["extra_data"]

elif event["event_type"] == "receive_pull_request":
object_id, remote_client_id = event["extra_data"]
object_id, remote_node_id = event["extra_data"]

else:
assert False, "This should be unreachable."
Expand All @@ -729,9 +729,9 @@ def chrome_tracing_object_transfer_dump(self, filename=None):
"name": event["event_type"],
# The identifier for the group of rows that the event
# appears in.
"pid": client_id_to_address[key],
"pid": node_id_to_address[key],
# The identifier for the row that the event appears in.
"tid": client_id_to_address[remote_client_id],
"tid": node_id_to_address[remote_node_id],
# The start time in microseconds.
"ts": self._seconds_to_microseconds(event["start_time"]),
# The duration in microseconds.
Expand Down Expand Up @@ -825,16 +825,16 @@ def cluster_resources(self):
clients = self.client_table()
for client in clients:
# Only count resources from latest entries of live clients.
if client["IsInsertion"]:
if client["Alive"]:
for key, value in client["Resources"].items():
resources[key] += value
return dict(resources)

def _live_client_ids(self):
"""Returns a set of client IDs corresponding to clients still alive."""
return {
client["ClientID"]
for client in self.client_table() if (client["IsInsertion"])
client["NodeID"]
for client in self.client_table() if (client["Alive"])
}

def available_resources(self):
Expand Down
4 changes: 1 addition & 3 deletions python/ray/tests/cluster_utils.py
Original file line number Diff line number Diff line change
Expand Up @@ -174,9 +174,7 @@ def wait_for_nodes(self, timeout=30):
start_time = time.time()
while time.time() - start_time < timeout:
clients = ray.state._parse_client_table(redis_client)
live_clients = [
client for client in clients if client["IsInsertion"]
]
live_clients = [client for client in clients if client["Alive"]]

expected = len(self.list_all_nodes())
if len(live_clients) == expected:
Expand Down
Loading

0 comments on commit b3bcf59

Please sign in to comment.