mirror of
https://github.com/vale981/ray
synced 2025-03-06 10:31:39 -05:00
Rename fields/variables from client id to node id (#12457)
This commit is contained in:
parent
3964defbe1
commit
b85c6abc3e
43 changed files with 670 additions and 682 deletions
|
@ -1,20 +1,20 @@
|
||||||
import ray
|
import ray
|
||||||
|
|
||||||
|
|
||||||
def set_resource(resource_name, capacity, client_id=None):
|
def set_resource(resource_name, capacity, node_id=None):
|
||||||
""" Set a resource to a specified capacity.
|
""" Set a resource to a specified capacity.
|
||||||
|
|
||||||
This creates, updates or deletes a custom resource for a target NodeID.
|
This creates, updates or deletes a custom resource for a target NodeID.
|
||||||
If the resource already exists, it's capacity is updated to the new value.
|
If the resource already exists, it's capacity is updated to the new value.
|
||||||
If the capacity is set to 0, the resource is deleted.
|
If the capacity is set to 0, the resource is deleted.
|
||||||
If NodeID is not specified or set to None,
|
If NodeID is not specified or set to None,
|
||||||
the resource is created on the local client where the actor is running.
|
the resource is created on the local node where the actor is running.
|
||||||
|
|
||||||
Args:
|
Args:
|
||||||
resource_name (str): Name of the resource to be created
|
resource_name (str): Name of the resource to be created
|
||||||
capacity (int): Capacity of the new resource. Resource is deleted if
|
capacity (int): Capacity of the new resource. Resource is deleted if
|
||||||
capacity is 0.
|
capacity is 0.
|
||||||
client_id (str): The NodeID of the node where the resource is to be
|
node_id (str): The NodeID of the node where the resource is to be
|
||||||
set.
|
set.
|
||||||
|
|
||||||
Returns:
|
Returns:
|
||||||
|
@ -24,12 +24,12 @@ def set_resource(resource_name, capacity, client_id=None):
|
||||||
ValueError: This exception is raised when a non-negative capacity is
|
ValueError: This exception is raised when a non-negative capacity is
|
||||||
specified.
|
specified.
|
||||||
"""
|
"""
|
||||||
if client_id is not None:
|
if node_id is not None:
|
||||||
client_id_obj = ray.NodeID(ray.utils.hex_to_binary(client_id))
|
node_id_obj = ray.NodeID(ray.utils.hex_to_binary(node_id))
|
||||||
else:
|
else:
|
||||||
client_id_obj = ray.NodeID.nil()
|
node_id_obj = ray.NodeID.nil()
|
||||||
if (capacity < 0) or (capacity != int(capacity)):
|
if (capacity < 0) or (capacity != int(capacity)):
|
||||||
raise ValueError(
|
raise ValueError(
|
||||||
"Capacity {} must be a non-negative integer.".format(capacity))
|
"Capacity {} must be a non-negative integer.".format(capacity))
|
||||||
return ray.worker.global_worker.core_worker.set_resource(
|
return ray.worker.global_worker.core_worker.set_resource(
|
||||||
resource_name, capacity, client_id_obj)
|
resource_name, capacity, node_id_obj)
|
||||||
|
|
|
@ -155,8 +155,8 @@ class Monitor:
|
||||||
heartbeat_batch_data.placement_group_load.placement_group_data)
|
heartbeat_batch_data.placement_group_load.placement_group_data)
|
||||||
|
|
||||||
# Update the load metrics for this raylet.
|
# Update the load metrics for this raylet.
|
||||||
client_id = ray.utils.binary_to_hex(heartbeat_message.client_id)
|
node_id = ray.utils.binary_to_hex(heartbeat_message.node_id)
|
||||||
ip = self.raylet_id_to_ip_map.get(client_id)
|
ip = self.raylet_id_to_ip_map.get(node_id)
|
||||||
if ip:
|
if ip:
|
||||||
self.load_metrics.update(ip, total_resources,
|
self.load_metrics.update(ip, total_resources,
|
||||||
available_resources, resource_load,
|
available_resources, resource_load,
|
||||||
|
@ -164,7 +164,7 @@ class Monitor:
|
||||||
pending_placement_groups)
|
pending_placement_groups)
|
||||||
else:
|
else:
|
||||||
logger.warning(
|
logger.warning(
|
||||||
f"Monitor: could not find ip for client {client_id}")
|
f"Monitor: could not find ip for node {node_id}")
|
||||||
|
|
||||||
def autoscaler_resource_request_handler(self, _, data):
|
def autoscaler_resource_request_handler(self, _, data):
|
||||||
"""Handle a notification of a resource request for the autoscaler.
|
"""Handle a notification of a resource request for the autoscaler.
|
||||||
|
|
|
@ -766,19 +766,19 @@ class GlobalState:
|
||||||
self._check_connected()
|
self._check_connected()
|
||||||
|
|
||||||
resources = defaultdict(int)
|
resources = defaultdict(int)
|
||||||
clients = self.node_table()
|
nodes = self.node_table()
|
||||||
for client in clients:
|
for node in nodes:
|
||||||
# Only count resources from latest entries of live clients.
|
# Only count resources from latest entries of live nodes.
|
||||||
if client["Alive"]:
|
if node["Alive"]:
|
||||||
for key, value in client["Resources"].items():
|
for key, value in node["Resources"].items():
|
||||||
resources[key] += value
|
resources[key] += value
|
||||||
return dict(resources)
|
return dict(resources)
|
||||||
|
|
||||||
def _live_client_ids(self):
|
def _live_node_ids(self):
|
||||||
"""Returns a set of client IDs corresponding to clients still alive."""
|
"""Returns a set of node IDs corresponding to nodes still alive."""
|
||||||
return {
|
return {
|
||||||
client["NodeID"]
|
node["NodeID"]
|
||||||
for client in self.node_table() if (client["Alive"])
|
for node in self.node_table() if (node["Alive"])
|
||||||
}
|
}
|
||||||
|
|
||||||
def _available_resources_per_node(self):
|
def _available_resources_per_node(self):
|
||||||
|
@ -800,7 +800,7 @@ class GlobalState:
|
||||||
available_resources_by_id[node_id] = dynamic_resources
|
available_resources_by_id[node_id] = dynamic_resources
|
||||||
|
|
||||||
# Update nodes in cluster.
|
# Update nodes in cluster.
|
||||||
node_ids = self._live_client_ids()
|
node_ids = self._live_node_ids()
|
||||||
# Remove disconnected nodes.
|
# Remove disconnected nodes.
|
||||||
for node_id in available_resources_by_id.keys():
|
for node_id in available_resources_by_id.keys():
|
||||||
if node_id not in node_ids:
|
if node_id not in node_ids:
|
||||||
|
|
|
@ -10,7 +10,7 @@ logger = logging.getLogger(__name__)
|
||||||
|
|
||||||
|
|
||||||
def test_dynamic_res_creation(ray_start_regular):
|
def test_dynamic_res_creation(ray_start_regular):
|
||||||
# This test creates a resource locally (without specifying the client_id)
|
# This test creates a resource locally (without specifying the node_id)
|
||||||
res_name = "test_res"
|
res_name = "test_res"
|
||||||
res_capacity = 1.0
|
res_capacity = 1.0
|
||||||
|
|
||||||
|
@ -30,7 +30,7 @@ def test_dynamic_res_creation(ray_start_regular):
|
||||||
|
|
||||||
|
|
||||||
def test_dynamic_res_deletion(shutdown_only):
|
def test_dynamic_res_deletion(shutdown_only):
|
||||||
# This test deletes a resource locally (without specifying the client_id)
|
# This test deletes a resource locally (without specifying the node_id)
|
||||||
res_name = "test_res"
|
res_name = "test_res"
|
||||||
res_capacity = 1.0
|
res_capacity = 1.0
|
||||||
|
|
||||||
|
@ -79,7 +79,7 @@ def test_dynamic_res_infeasible_rescheduling(ray_start_regular):
|
||||||
assert successful # The task completed
|
assert successful # The task completed
|
||||||
|
|
||||||
|
|
||||||
def test_dynamic_res_updation_clientid(ray_start_cluster):
|
def test_dynamic_res_updation_nodeid(ray_start_cluster):
|
||||||
# This test does a simple resource capacity update
|
# This test does a simple resource capacity update
|
||||||
cluster = ray_start_cluster
|
cluster = ray_start_cluster
|
||||||
|
|
||||||
|
@ -94,9 +94,9 @@ def test_dynamic_res_updation_clientid(ray_start_cluster):
|
||||||
target_node_id = ray.nodes()[1]["NodeID"]
|
target_node_id = ray.nodes()[1]["NodeID"]
|
||||||
|
|
||||||
@ray.remote
|
@ray.remote
|
||||||
def set_res(resource_name, resource_capacity, client_id):
|
def set_res(resource_name, resource_capacity, node_id):
|
||||||
ray.experimental.set_resource(
|
ray.experimental.set_resource(
|
||||||
resource_name, resource_capacity, client_id=client_id)
|
resource_name, resource_capacity, node_id=node_id)
|
||||||
|
|
||||||
# Create resource
|
# Create resource
|
||||||
ray.get(set_res.remote(res_name, res_capacity, target_node_id))
|
ray.get(set_res.remote(res_name, res_capacity, target_node_id))
|
||||||
|
@ -115,8 +115,8 @@ def test_dynamic_res_updation_clientid(ray_start_cluster):
|
||||||
wait_for_condition(check_resources)
|
wait_for_condition(check_resources)
|
||||||
|
|
||||||
|
|
||||||
def test_dynamic_res_creation_clientid(ray_start_cluster):
|
def test_dynamic_res_creation_nodeid(ray_start_cluster):
|
||||||
# Creates a resource on a specific client and verifies creation.
|
# Creates a resource on a specific node and verifies creation.
|
||||||
cluster = ray_start_cluster
|
cluster = ray_start_cluster
|
||||||
|
|
||||||
res_name = "test_res"
|
res_name = "test_res"
|
||||||
|
@ -130,9 +130,9 @@ def test_dynamic_res_creation_clientid(ray_start_cluster):
|
||||||
target_node_id = ray.nodes()[1]["NodeID"]
|
target_node_id = ray.nodes()[1]["NodeID"]
|
||||||
|
|
||||||
@ray.remote
|
@ray.remote
|
||||||
def set_res(resource_name, resource_capacity, res_client_id):
|
def set_res(resource_name, resource_capacity, res_node_id):
|
||||||
ray.experimental.set_resource(
|
ray.experimental.set_resource(
|
||||||
resource_name, resource_capacity, client_id=res_client_id)
|
resource_name, resource_capacity, node_id=res_node_id)
|
||||||
|
|
||||||
ray.get(set_res.remote(res_name, res_capacity, target_node_id))
|
ray.get(set_res.remote(res_name, res_capacity, target_node_id))
|
||||||
|
|
||||||
|
@ -146,8 +146,8 @@ def test_dynamic_res_creation_clientid(ray_start_cluster):
|
||||||
wait_for_condition(check_resources)
|
wait_for_condition(check_resources)
|
||||||
|
|
||||||
|
|
||||||
def test_dynamic_res_creation_clientid_multiple(ray_start_cluster):
|
def test_dynamic_res_creation_nodeid_multiple(ray_start_cluster):
|
||||||
# This test creates resources on multiple clients using the clientid
|
# This test creates resources on multiple nodes using the nodeid
|
||||||
# specifier
|
# specifier
|
||||||
cluster = ray_start_cluster
|
cluster = ray_start_cluster
|
||||||
|
|
||||||
|
@ -162,9 +162,9 @@ def test_dynamic_res_creation_clientid_multiple(ray_start_cluster):
|
||||||
target_node_ids = [node["NodeID"] for node in ray.nodes()]
|
target_node_ids = [node["NodeID"] for node in ray.nodes()]
|
||||||
|
|
||||||
@ray.remote
|
@ray.remote
|
||||||
def set_res(resource_name, resource_capacity, res_client_id):
|
def set_res(resource_name, resource_capacity, res_node_id):
|
||||||
ray.experimental.set_resource(
|
ray.experimental.set_resource(
|
||||||
resource_name, resource_capacity, client_id=res_client_id)
|
resource_name, resource_capacity, node_id=res_node_id)
|
||||||
|
|
||||||
results = []
|
results = []
|
||||||
for nid in target_node_ids:
|
for nid in target_node_ids:
|
||||||
|
@ -184,8 +184,8 @@ def test_dynamic_res_creation_clientid_multiple(ray_start_cluster):
|
||||||
wait_for_condition(check_resources)
|
wait_for_condition(check_resources)
|
||||||
|
|
||||||
|
|
||||||
def test_dynamic_res_deletion_clientid(ray_start_cluster):
|
def test_dynamic_res_deletion_nodeid(ray_start_cluster):
|
||||||
# This test deletes a resource on a given client id
|
# This test deletes a resource on a given node id
|
||||||
cluster = ray_start_cluster
|
cluster = ray_start_cluster
|
||||||
|
|
||||||
res_name = "test_res"
|
res_name = "test_res"
|
||||||
|
@ -203,9 +203,8 @@ def test_dynamic_res_deletion_clientid(ray_start_cluster):
|
||||||
|
|
||||||
# Launch the delete task
|
# Launch the delete task
|
||||||
@ray.remote
|
@ray.remote
|
||||||
def delete_res(resource_name, res_client_id):
|
def delete_res(resource_name, res_node_id):
|
||||||
ray.experimental.set_resource(
|
ray.experimental.set_resource(resource_name, 0, node_id=res_node_id)
|
||||||
resource_name, 0, client_id=res_client_id)
|
|
||||||
|
|
||||||
ray.get(delete_res.remote(res_name, target_node_id))
|
ray.get(delete_res.remote(res_name, target_node_id))
|
||||||
|
|
||||||
|
@ -236,9 +235,9 @@ def test_dynamic_res_creation_scheduler_consistency(ray_start_cluster):
|
||||||
node_ids = [node["NodeID"] for node in ray.nodes()]
|
node_ids = [node["NodeID"] for node in ray.nodes()]
|
||||||
|
|
||||||
@ray.remote
|
@ray.remote
|
||||||
def set_res(resource_name, resource_capacity, res_client_id):
|
def set_res(resource_name, resource_capacity, res_node_id):
|
||||||
ray.experimental.set_resource(
|
ray.experimental.set_resource(
|
||||||
resource_name, resource_capacity, client_id=res_client_id)
|
resource_name, resource_capacity, node_id=res_node_id)
|
||||||
|
|
||||||
# Create the resource on node1
|
# Create the resource on node1
|
||||||
target_node_id = node_ids[1]
|
target_node_id = node_ids[1]
|
||||||
|
@ -273,14 +272,13 @@ def test_dynamic_res_deletion_scheduler_consistency(ray_start_cluster):
|
||||||
node_ids = [node["NodeID"] for node in ray.nodes()]
|
node_ids = [node["NodeID"] for node in ray.nodes()]
|
||||||
|
|
||||||
@ray.remote
|
@ray.remote
|
||||||
def delete_res(resource_name, res_client_id):
|
def delete_res(resource_name, res_node_id):
|
||||||
ray.experimental.set_resource(
|
ray.experimental.set_resource(resource_name, 0, node_id=res_node_id)
|
||||||
resource_name, 0, client_id=res_client_id)
|
|
||||||
|
|
||||||
@ray.remote
|
@ray.remote
|
||||||
def set_res(resource_name, resource_capacity, res_client_id):
|
def set_res(resource_name, resource_capacity, res_node_id):
|
||||||
ray.experimental.set_resource(
|
ray.experimental.set_resource(
|
||||||
resource_name, resource_capacity, client_id=res_client_id)
|
resource_name, resource_capacity, node_id=res_node_id)
|
||||||
|
|
||||||
# Create the resource on node1
|
# Create the resource on node1
|
||||||
target_node_id = node_ids[1]
|
target_node_id = node_ids[1]
|
||||||
|
@ -326,9 +324,9 @@ def test_dynamic_res_concurrent_res_increment(ray_start_cluster):
|
||||||
target_node_id = node_ids[1]
|
target_node_id = node_ids[1]
|
||||||
|
|
||||||
@ray.remote
|
@ray.remote
|
||||||
def set_res(resource_name, resource_capacity, res_client_id):
|
def set_res(resource_name, resource_capacity, res_node_id):
|
||||||
ray.experimental.set_resource(
|
ray.experimental.set_resource(
|
||||||
resource_name, resource_capacity, client_id=res_client_id)
|
resource_name, resource_capacity, node_id=res_node_id)
|
||||||
|
|
||||||
# Create the resource on node 1
|
# Create the resource on node 1
|
||||||
ray.get(set_res.remote(res_name, res_capacity, target_node_id))
|
ray.get(set_res.remote(res_name, res_capacity, target_node_id))
|
||||||
|
@ -416,9 +414,9 @@ def test_dynamic_res_concurrent_res_decrement(ray_start_cluster):
|
||||||
target_node_id = node_ids[1]
|
target_node_id = node_ids[1]
|
||||||
|
|
||||||
@ray.remote
|
@ray.remote
|
||||||
def set_res(resource_name, resource_capacity, res_client_id):
|
def set_res(resource_name, resource_capacity, res_node_id):
|
||||||
ray.experimental.set_resource(
|
ray.experimental.set_resource(
|
||||||
resource_name, resource_capacity, client_id=res_client_id)
|
resource_name, resource_capacity, node_id=res_node_id)
|
||||||
|
|
||||||
# Create the resource on node 1
|
# Create the resource on node 1
|
||||||
ray.get(set_res.remote(res_name, res_capacity, target_node_id))
|
ray.get(set_res.remote(res_name, res_capacity, target_node_id))
|
||||||
|
@ -504,14 +502,13 @@ def test_dynamic_res_concurrent_res_delete(ray_start_cluster):
|
||||||
target_node_id = node_ids[1]
|
target_node_id = node_ids[1]
|
||||||
|
|
||||||
@ray.remote
|
@ray.remote
|
||||||
def set_res(resource_name, resource_capacity, res_client_id):
|
def set_res(resource_name, resource_capacity, res_node_id):
|
||||||
ray.experimental.set_resource(
|
ray.experimental.set_resource(
|
||||||
resource_name, resource_capacity, client_id=res_client_id)
|
resource_name, resource_capacity, node_id=res_node_id)
|
||||||
|
|
||||||
@ray.remote
|
@ray.remote
|
||||||
def delete_res(resource_name, res_client_id):
|
def delete_res(resource_name, res_node_id):
|
||||||
ray.experimental.set_resource(
|
ray.experimental.set_resource(resource_name, 0, node_id=res_node_id)
|
||||||
resource_name, 0, client_id=res_client_id)
|
|
||||||
|
|
||||||
# Create the resource on node 1
|
# Create the resource on node 1
|
||||||
ray.get(set_res.remote(res_name, res_capacity, target_node_id))
|
ray.get(set_res.remote(res_name, res_capacity, target_node_id))
|
||||||
|
@ -572,7 +569,7 @@ def test_dynamic_res_concurrent_res_delete(ray_start_cluster):
|
||||||
|
|
||||||
def test_dynamic_res_creation_stress(ray_start_cluster):
|
def test_dynamic_res_creation_stress(ray_start_cluster):
|
||||||
# This stress tests creates many resources simultaneously on the same
|
# This stress tests creates many resources simultaneously on the same
|
||||||
# client and then checks if the final state is consistent
|
# node and then checks if the final state is consistent
|
||||||
|
|
||||||
cluster = ray_start_cluster
|
cluster = ray_start_cluster
|
||||||
|
|
||||||
|
@ -590,14 +587,13 @@ def test_dynamic_res_creation_stress(ray_start_cluster):
|
||||||
target_node_id = node_ids[1]
|
target_node_id = node_ids[1]
|
||||||
|
|
||||||
@ray.remote
|
@ray.remote
|
||||||
def set_res(resource_name, resource_capacity, res_client_id):
|
def set_res(resource_name, resource_capacity, res_node_id):
|
||||||
ray.experimental.set_resource(
|
ray.experimental.set_resource(
|
||||||
resource_name, resource_capacity, client_id=res_client_id)
|
resource_name, resource_capacity, node_id=res_node_id)
|
||||||
|
|
||||||
@ray.remote
|
@ray.remote
|
||||||
def delete_res(resource_name, res_client_id):
|
def delete_res(resource_name, res_node_id):
|
||||||
ray.experimental.set_resource(
|
ray.experimental.set_resource(resource_name, 0, node_id=res_node_id)
|
||||||
resource_name, 0, client_id=res_client_id)
|
|
||||||
|
|
||||||
results = [
|
results = [
|
||||||
set_res.remote(str(i), res_capacity, target_node_id)
|
set_res.remote(str(i), res_capacity, target_node_id)
|
||||||
|
|
|
@ -1182,8 +1182,8 @@ Status CoreWorker::NotifyActorResumedFromCheckpoint(
|
||||||
}
|
}
|
||||||
|
|
||||||
Status CoreWorker::SetResource(const std::string &resource_name, const double capacity,
|
Status CoreWorker::SetResource(const std::string &resource_name, const double capacity,
|
||||||
const NodeID &client_id) {
|
const NodeID &node_id) {
|
||||||
return local_raylet_client_->SetResource(resource_name, capacity, client_id);
|
return local_raylet_client_->SetResource(resource_name, capacity, node_id);
|
||||||
}
|
}
|
||||||
|
|
||||||
void CoreWorker::SpillOwnedObject(const ObjectID &object_id,
|
void CoreWorker::SpillOwnedObject(const ObjectID &object_id,
|
||||||
|
@ -2124,7 +2124,7 @@ void CoreWorker::HandleAddObjectLocationOwner(
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
reference_counter_->AddObjectLocation(ObjectID::FromBinary(request.object_id()),
|
reference_counter_->AddObjectLocation(ObjectID::FromBinary(request.object_id()),
|
||||||
NodeID::FromBinary(request.client_id()));
|
NodeID::FromBinary(request.node_id()));
|
||||||
send_reply_callback(Status::OK(), nullptr, nullptr);
|
send_reply_callback(Status::OK(), nullptr, nullptr);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2137,7 +2137,7 @@ void CoreWorker::HandleRemoveObjectLocationOwner(
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
reference_counter_->RemoveObjectLocation(ObjectID::FromBinary(request.object_id()),
|
reference_counter_->RemoveObjectLocation(ObjectID::FromBinary(request.object_id()),
|
||||||
NodeID::FromBinary(request.client_id()));
|
NodeID::FromBinary(request.node_id()));
|
||||||
send_reply_callback(Status::OK(), nullptr, nullptr);
|
send_reply_callback(Status::OK(), nullptr, nullptr);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2149,10 +2149,10 @@ void CoreWorker::HandleGetObjectLocationsOwner(
|
||||||
send_reply_callback)) {
|
send_reply_callback)) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
std::unordered_set<NodeID> client_ids =
|
std::unordered_set<NodeID> node_ids =
|
||||||
reference_counter_->GetObjectLocations(ObjectID::FromBinary(request.object_id()));
|
reference_counter_->GetObjectLocations(ObjectID::FromBinary(request.object_id()));
|
||||||
for (const auto &client_id : client_ids) {
|
for (const auto &node_id : node_ids) {
|
||||||
reply->add_client_ids(client_id.Binary());
|
reply->add_node_ids(node_id.Binary());
|
||||||
}
|
}
|
||||||
send_reply_callback(Status::OK(), nullptr, nullptr);
|
send_reply_callback(Status::OK(), nullptr, nullptr);
|
||||||
}
|
}
|
||||||
|
|
|
@ -619,13 +619,13 @@ class CoreWorker : public rpc::CoreWorkerServiceHandler {
|
||||||
Status NotifyActorResumedFromCheckpoint(const ActorID &actor_id,
|
Status NotifyActorResumedFromCheckpoint(const ActorID &actor_id,
|
||||||
const ActorCheckpointID &checkpoint_id);
|
const ActorCheckpointID &checkpoint_id);
|
||||||
|
|
||||||
/// Sets a resource with the specified capacity and client id
|
/// Sets a resource with the specified capacity and node id
|
||||||
/// \param[in] resource_name Name of the resource to be set.
|
/// \param[in] resource_name Name of the resource to be set.
|
||||||
/// \param[in] capacity Capacity of the resource.
|
/// \param[in] capacity Capacity of the resource.
|
||||||
/// \param[in] client_Id NodeID where the resource is to be set.
|
/// \param[in] node_id NodeID where the resource is to be set.
|
||||||
/// \return Status
|
/// \return Status
|
||||||
Status SetResource(const std::string &resource_name, const double capacity,
|
Status SetResource(const std::string &resource_name, const double capacity,
|
||||||
const NodeID &client_id);
|
const NodeID &node_id);
|
||||||
|
|
||||||
/// Request an object to be spilled to external storage.
|
/// Request an object to be spilled to external storage.
|
||||||
/// \param[in] object_ids The objects to be spilled.
|
/// \param[in] object_ids The objects to be spilled.
|
||||||
|
|
|
@ -204,7 +204,7 @@ TEST_F(GlobalStateAccessorTest, TestGetAllHeartbeat) {
|
||||||
// Report heartbeat first time.
|
// Report heartbeat first time.
|
||||||
std::promise<bool> promise1;
|
std::promise<bool> promise1;
|
||||||
auto heartbeat1 = std::make_shared<rpc::HeartbeatTableData>();
|
auto heartbeat1 = std::make_shared<rpc::HeartbeatTableData>();
|
||||||
heartbeat1->set_client_id(node_table_data->node_id());
|
heartbeat1->set_node_id(node_table_data->node_id());
|
||||||
RAY_CHECK_OK(gcs_client_->Nodes().AsyncReportHeartbeat(
|
RAY_CHECK_OK(gcs_client_->Nodes().AsyncReportHeartbeat(
|
||||||
heartbeat1, [&promise1](Status status) { promise1.set_value(status.ok()); }));
|
heartbeat1, [&promise1](Status status) { promise1.set_value(status.ok()); }));
|
||||||
WaitReady(promise1.get_future(), timeout_ms_);
|
WaitReady(promise1.get_future(), timeout_ms_);
|
||||||
|
@ -216,7 +216,7 @@ TEST_F(GlobalStateAccessorTest, TestGetAllHeartbeat) {
|
||||||
// Report heartbeat with resources changed.
|
// Report heartbeat with resources changed.
|
||||||
std::promise<bool> promise2;
|
std::promise<bool> promise2;
|
||||||
auto heartbeat2 = std::make_shared<rpc::HeartbeatTableData>();
|
auto heartbeat2 = std::make_shared<rpc::HeartbeatTableData>();
|
||||||
heartbeat2->set_client_id(node_table_data->node_id());
|
heartbeat2->set_node_id(node_table_data->node_id());
|
||||||
(*heartbeat2->mutable_resources_total())["CPU"] = 1;
|
(*heartbeat2->mutable_resources_total())["CPU"] = 1;
|
||||||
(*heartbeat2->mutable_resources_total())["GPU"] = 10;
|
(*heartbeat2->mutable_resources_total())["GPU"] = 10;
|
||||||
heartbeat2->set_resources_available_changed(true);
|
heartbeat2->set_resources_available_changed(true);
|
||||||
|
@ -238,7 +238,7 @@ TEST_F(GlobalStateAccessorTest, TestGetAllHeartbeat) {
|
||||||
// Report heartbeat with resources unchanged. (Only works with light heartbeat enabled)
|
// Report heartbeat with resources unchanged. (Only works with light heartbeat enabled)
|
||||||
std::promise<bool> promise3;
|
std::promise<bool> promise3;
|
||||||
auto heartbeat3 = std::make_shared<rpc::HeartbeatTableData>();
|
auto heartbeat3 = std::make_shared<rpc::HeartbeatTableData>();
|
||||||
heartbeat3->set_client_id(node_table_data->node_id());
|
heartbeat3->set_node_id(node_table_data->node_id());
|
||||||
(*heartbeat3->mutable_resources_available())["CPU"] = 1;
|
(*heartbeat3->mutable_resources_available())["CPU"] = 1;
|
||||||
(*heartbeat3->mutable_resources_available())["GPU"] = 6;
|
(*heartbeat3->mutable_resources_available())["GPU"] = 6;
|
||||||
RAY_CHECK_OK(gcs_client_->Nodes().AsyncReportHeartbeat(
|
RAY_CHECK_OK(gcs_client_->Nodes().AsyncReportHeartbeat(
|
||||||
|
@ -260,8 +260,8 @@ TEST_F(GlobalStateAccessorTest, TestProfileTable) {
|
||||||
int profile_count = RayConfig::instance().maximum_profile_table_rows_count() + 1;
|
int profile_count = RayConfig::instance().maximum_profile_table_rows_count() + 1;
|
||||||
ASSERT_EQ(global_state_->GetAllProfileInfo().size(), 0);
|
ASSERT_EQ(global_state_->GetAllProfileInfo().size(), 0);
|
||||||
for (int index = 0; index < profile_count; ++index) {
|
for (int index = 0; index < profile_count; ++index) {
|
||||||
auto client_id = NodeID::FromRandom();
|
auto node_id = NodeID::FromRandom();
|
||||||
auto profile_table_data = Mocker::GenProfileTableData(client_id);
|
auto profile_table_data = Mocker::GenProfileTableData(node_id);
|
||||||
std::promise<bool> promise;
|
std::promise<bool> promise;
|
||||||
RAY_CHECK_OK(gcs_client_->Stats().AsyncAddProfileData(
|
RAY_CHECK_OK(gcs_client_->Stats().AsyncAddProfileData(
|
||||||
profile_table_data,
|
profile_table_data,
|
||||||
|
|
|
@ -796,7 +796,7 @@ TEST_F(ServiceBasedGcsClientTest, TestNodeHeartbeat) {
|
||||||
// Report heartbeat of a node to GCS.
|
// Report heartbeat of a node to GCS.
|
||||||
NodeID node_id = NodeID::FromBinary(node_info->node_id());
|
NodeID node_id = NodeID::FromBinary(node_info->node_id());
|
||||||
auto heartbeat = std::make_shared<rpc::HeartbeatTableData>();
|
auto heartbeat = std::make_shared<rpc::HeartbeatTableData>();
|
||||||
heartbeat->set_client_id(node_id.Binary());
|
heartbeat->set_node_id(node_id.Binary());
|
||||||
// Set this flag because GCS won't publish unchanged heartbeat.
|
// Set this flag because GCS won't publish unchanged heartbeat.
|
||||||
heartbeat->set_should_global_gc(true);
|
heartbeat->set_should_global_gc(true);
|
||||||
ASSERT_TRUE(ReportHeartbeat(heartbeat));
|
ASSERT_TRUE(ReportHeartbeat(heartbeat));
|
||||||
|
@ -819,13 +819,13 @@ TEST_F(ServiceBasedGcsClientTest, TestNodeHeartbeatWithLightHeartbeat) {
|
||||||
// Report unchanged heartbeat of a node to GCS.
|
// Report unchanged heartbeat of a node to GCS.
|
||||||
NodeID node_id = NodeID::FromBinary(node_info->node_id());
|
NodeID node_id = NodeID::FromBinary(node_info->node_id());
|
||||||
auto heartbeat = std::make_shared<rpc::HeartbeatTableData>();
|
auto heartbeat = std::make_shared<rpc::HeartbeatTableData>();
|
||||||
heartbeat->set_client_id(node_id.Binary());
|
heartbeat->set_node_id(node_id.Binary());
|
||||||
ASSERT_TRUE(ReportHeartbeat(heartbeat));
|
ASSERT_TRUE(ReportHeartbeat(heartbeat));
|
||||||
WaitForExpectedCount(heartbeat_batch_count, 0);
|
WaitForExpectedCount(heartbeat_batch_count, 0);
|
||||||
|
|
||||||
// Report changed heartbeat of a node to GCS.
|
// Report changed heartbeat of a node to GCS.
|
||||||
auto heartbeat1 = std::make_shared<rpc::HeartbeatTableData>();
|
auto heartbeat1 = std::make_shared<rpc::HeartbeatTableData>();
|
||||||
heartbeat1->set_client_id(node_id.Binary());
|
heartbeat1->set_node_id(node_id.Binary());
|
||||||
heartbeat1->set_resources_available_changed(true);
|
heartbeat1->set_resources_available_changed(true);
|
||||||
ASSERT_TRUE(ReportHeartbeat(heartbeat1));
|
ASSERT_TRUE(ReportHeartbeat(heartbeat1));
|
||||||
WaitForExpectedCount(heartbeat_batch_count, 1);
|
WaitForExpectedCount(heartbeat_batch_count, 1);
|
||||||
|
@ -847,7 +847,7 @@ TEST_F(ServiceBasedGcsClientTest, TestGetAllAvailableResources) {
|
||||||
// Report heartbeat of a node to GCS.
|
// Report heartbeat of a node to GCS.
|
||||||
NodeID node_id = NodeID::FromBinary(node_info->node_id());
|
NodeID node_id = NodeID::FromBinary(node_info->node_id());
|
||||||
auto heartbeat = std::make_shared<rpc::HeartbeatTableData>();
|
auto heartbeat = std::make_shared<rpc::HeartbeatTableData>();
|
||||||
heartbeat->set_client_id(node_id.Binary());
|
heartbeat->set_node_id(node_id.Binary());
|
||||||
// Set this flag to indicate resources has changed.
|
// Set this flag to indicate resources has changed.
|
||||||
heartbeat->set_resources_available_changed(true);
|
heartbeat->set_resources_available_changed(true);
|
||||||
(*heartbeat->mutable_resources_available())["CPU"] = 1.0;
|
(*heartbeat->mutable_resources_available())["CPU"] = 1.0;
|
||||||
|
@ -879,7 +879,7 @@ TEST_F(ServiceBasedGcsClientTest, TestGetAllAvailableResourcesWithLightHeartbeat
|
||||||
// Report heartbeat of a node to GCS.
|
// Report heartbeat of a node to GCS.
|
||||||
NodeID node_id = NodeID::FromBinary(node_info->node_id());
|
NodeID node_id = NodeID::FromBinary(node_info->node_id());
|
||||||
auto heartbeat = std::make_shared<rpc::HeartbeatTableData>();
|
auto heartbeat = std::make_shared<rpc::HeartbeatTableData>();
|
||||||
heartbeat->set_client_id(node_id.Binary());
|
heartbeat->set_node_id(node_id.Binary());
|
||||||
heartbeat->set_resources_available_changed(true);
|
heartbeat->set_resources_available_changed(true);
|
||||||
(*heartbeat->mutable_resources_available())["CPU"] = 1.0;
|
(*heartbeat->mutable_resources_available())["CPU"] = 1.0;
|
||||||
(*heartbeat->mutable_resources_available())["GPU"] = 10.0;
|
(*heartbeat->mutable_resources_available())["GPU"] = 10.0;
|
||||||
|
@ -895,7 +895,7 @@ TEST_F(ServiceBasedGcsClientTest, TestGetAllAvailableResourcesWithLightHeartbeat
|
||||||
|
|
||||||
// Report unchanged heartbeat of a node to GCS.
|
// Report unchanged heartbeat of a node to GCS.
|
||||||
auto heartbeat1 = std::make_shared<rpc::HeartbeatTableData>();
|
auto heartbeat1 = std::make_shared<rpc::HeartbeatTableData>();
|
||||||
heartbeat1->set_client_id(node_id.Binary());
|
heartbeat1->set_node_id(node_id.Binary());
|
||||||
(*heartbeat1->mutable_resources_available())["GPU"] = 8.0;
|
(*heartbeat1->mutable_resources_available())["GPU"] = 8.0;
|
||||||
ASSERT_TRUE(ReportHeartbeat(heartbeat1));
|
ASSERT_TRUE(ReportHeartbeat(heartbeat1));
|
||||||
WaitForExpectedCount(heartbeat_batch_count, 1);
|
WaitForExpectedCount(heartbeat_batch_count, 1);
|
||||||
|
@ -1223,7 +1223,7 @@ TEST_F(ServiceBasedGcsClientTest, TestNodeTableResubscribe) {
|
||||||
std::string key = "CPU";
|
std::string key = "CPU";
|
||||||
ASSERT_TRUE(UpdateResources(node_id, key));
|
ASSERT_TRUE(UpdateResources(node_id, key));
|
||||||
auto heartbeat = std::make_shared<rpc::HeartbeatTableData>();
|
auto heartbeat = std::make_shared<rpc::HeartbeatTableData>();
|
||||||
heartbeat->set_client_id(node_info->node_id());
|
heartbeat->set_node_id(node_info->node_id());
|
||||||
// Set this flag because GCS won't publish unchanged heartbeat.
|
// Set this flag because GCS won't publish unchanged heartbeat.
|
||||||
heartbeat->set_should_global_gc(true);
|
heartbeat->set_should_global_gc(true);
|
||||||
ASSERT_TRUE(ReportHeartbeat(heartbeat));
|
ASSERT_TRUE(ReportHeartbeat(heartbeat));
|
||||||
|
@ -1235,7 +1235,7 @@ TEST_F(ServiceBasedGcsClientTest, TestNodeTableResubscribe) {
|
||||||
ASSERT_TRUE(RegisterNode(*node_info));
|
ASSERT_TRUE(RegisterNode(*node_info));
|
||||||
node_id = NodeID::FromBinary(node_info->node_id());
|
node_id = NodeID::FromBinary(node_info->node_id());
|
||||||
ASSERT_TRUE(UpdateResources(node_id, key));
|
ASSERT_TRUE(UpdateResources(node_id, key));
|
||||||
heartbeat->set_client_id(node_info->node_id());
|
heartbeat->set_node_id(node_info->node_id());
|
||||||
ASSERT_TRUE(ReportHeartbeat(heartbeat));
|
ASSERT_TRUE(ReportHeartbeat(heartbeat));
|
||||||
|
|
||||||
WaitForExpectedCount(node_change_count, 2);
|
WaitForExpectedCount(node_change_count, 2);
|
||||||
|
|
|
@ -41,7 +41,7 @@ void GcsNodeManager::NodeFailureDetector::Start() {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void GcsNodeManager::NodeFailureDetector::AddNode(const ray::NodeID &node_id) {
|
void GcsNodeManager::NodeFailureDetector::AddNode(const NodeID &node_id) {
|
||||||
heartbeats_.emplace(node_id, num_heartbeats_timeout_);
|
heartbeats_.emplace(node_id, num_heartbeats_timeout_);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -195,7 +195,7 @@ void GcsNodeManager::HandleGetAllNodeInfo(const rpc::GetAllNodeInfoRequest &requ
|
||||||
void GcsNodeManager::HandleReportHeartbeat(const rpc::ReportHeartbeatRequest &request,
|
void GcsNodeManager::HandleReportHeartbeat(const rpc::ReportHeartbeatRequest &request,
|
||||||
rpc::ReportHeartbeatReply *reply,
|
rpc::ReportHeartbeatReply *reply,
|
||||||
rpc::SendReplyCallback send_reply_callback) {
|
rpc::SendReplyCallback send_reply_callback) {
|
||||||
NodeID node_id = NodeID::FromBinary(request.heartbeat().client_id());
|
NodeID node_id = NodeID::FromBinary(request.heartbeat().node_id());
|
||||||
auto heartbeat_data = std::make_shared<rpc::HeartbeatTableData>();
|
auto heartbeat_data = std::make_shared<rpc::HeartbeatTableData>();
|
||||||
heartbeat_data->CopyFrom(request.heartbeat());
|
heartbeat_data->CopyFrom(request.heartbeat());
|
||||||
|
|
||||||
|
|
|
@ -249,7 +249,7 @@ class GcsNodeTable : public GcsTable<NodeID, GcsNodeInfo> {
|
||||||
public:
|
public:
|
||||||
explicit GcsNodeTable(std::shared_ptr<StoreClient> &store_client)
|
explicit GcsNodeTable(std::shared_ptr<StoreClient> &store_client)
|
||||||
: GcsTable(store_client) {
|
: GcsTable(store_client) {
|
||||||
table_name_ = TablePrefix_Name(TablePrefix::CLIENT);
|
table_name_ = TablePrefix_Name(TablePrefix::NODE);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
|
@ -100,7 +100,7 @@ class GcsPlacementGroupSchedulerTest : public ::testing::Test {
|
||||||
void AddNode(const std::shared_ptr<rpc::GcsNodeInfo> &node, int cpu_num = 10) {
|
void AddNode(const std::shared_ptr<rpc::GcsNodeInfo> &node, int cpu_num = 10) {
|
||||||
gcs_node_manager_->AddNode(node);
|
gcs_node_manager_->AddNode(node);
|
||||||
rpc::HeartbeatTableData heartbeat;
|
rpc::HeartbeatTableData heartbeat;
|
||||||
heartbeat.set_client_id(node->node_id());
|
heartbeat.set_node_id(node->node_id());
|
||||||
(*heartbeat.mutable_resources_available())["CPU"] = cpu_num;
|
(*heartbeat.mutable_resources_available())["CPU"] = cpu_num;
|
||||||
gcs_node_manager_->UpdateNodeRealtimeResources(NodeID::FromBinary(node->node_id()),
|
gcs_node_manager_->UpdateNodeRealtimeResources(NodeID::FromBinary(node->node_id()),
|
||||||
heartbeat);
|
heartbeat);
|
||||||
|
|
|
@ -585,7 +585,7 @@ TEST_F(GcsServerTest, TestNodeInfo) {
|
||||||
|
|
||||||
// Report heartbeat
|
// Report heartbeat
|
||||||
rpc::ReportHeartbeatRequest report_heartbeat_request;
|
rpc::ReportHeartbeatRequest report_heartbeat_request;
|
||||||
report_heartbeat_request.mutable_heartbeat()->set_client_id(gcs_node_info->node_id());
|
report_heartbeat_request.mutable_heartbeat()->set_node_id(gcs_node_info->node_id());
|
||||||
ASSERT_TRUE(ReportHeartbeat(report_heartbeat_request));
|
ASSERT_TRUE(ReportHeartbeat(report_heartbeat_request));
|
||||||
|
|
||||||
// Update node resources
|
// Update node resources
|
||||||
|
|
|
@ -499,8 +499,8 @@ RedisNodeInfoAccessor::RedisNodeInfoAccessor(RedisGcsClient *client_impl)
|
||||||
|
|
||||||
Status RedisNodeInfoAccessor::RegisterSelf(const GcsNodeInfo &local_node_info,
|
Status RedisNodeInfoAccessor::RegisterSelf(const GcsNodeInfo &local_node_info,
|
||||||
const StatusCallback &callback) {
|
const StatusCallback &callback) {
|
||||||
ClientTable &client_table = client_impl_->client_table();
|
NodeTable &node_table = client_impl_->node_table();
|
||||||
Status status = client_table.Connect(local_node_info);
|
Status status = node_table.Connect(local_node_info);
|
||||||
if (callback != nullptr) {
|
if (callback != nullptr) {
|
||||||
callback(Status::OK());
|
callback(Status::OK());
|
||||||
}
|
}
|
||||||
|
@ -508,47 +508,47 @@ Status RedisNodeInfoAccessor::RegisterSelf(const GcsNodeInfo &local_node_info,
|
||||||
}
|
}
|
||||||
|
|
||||||
Status RedisNodeInfoAccessor::UnregisterSelf() {
|
Status RedisNodeInfoAccessor::UnregisterSelf() {
|
||||||
ClientTable &client_table = client_impl_->client_table();
|
NodeTable &node_table = client_impl_->node_table();
|
||||||
return client_table.Disconnect();
|
return node_table.Disconnect();
|
||||||
}
|
}
|
||||||
|
|
||||||
const NodeID &RedisNodeInfoAccessor::GetSelfId() const {
|
const NodeID &RedisNodeInfoAccessor::GetSelfId() const {
|
||||||
ClientTable &client_table = client_impl_->client_table();
|
NodeTable &node_table = client_impl_->node_table();
|
||||||
return client_table.GetLocalClientId();
|
return node_table.GetLocalNodeId();
|
||||||
}
|
}
|
||||||
|
|
||||||
const GcsNodeInfo &RedisNodeInfoAccessor::GetSelfInfo() const {
|
const GcsNodeInfo &RedisNodeInfoAccessor::GetSelfInfo() const {
|
||||||
ClientTable &client_table = client_impl_->client_table();
|
NodeTable &node_table = client_impl_->node_table();
|
||||||
return client_table.GetLocalClient();
|
return node_table.GetLocalNode();
|
||||||
}
|
}
|
||||||
|
|
||||||
Status RedisNodeInfoAccessor::AsyncRegister(const GcsNodeInfo &node_info,
|
Status RedisNodeInfoAccessor::AsyncRegister(const GcsNodeInfo &node_info,
|
||||||
const StatusCallback &callback) {
|
const StatusCallback &callback) {
|
||||||
ClientTable::WriteCallback on_done = nullptr;
|
NodeTable::WriteCallback on_done = nullptr;
|
||||||
if (callback != nullptr) {
|
if (callback != nullptr) {
|
||||||
on_done = [callback](RedisGcsClient *client, const NodeID &id,
|
on_done = [callback](RedisGcsClient *client, const NodeID &id,
|
||||||
const GcsNodeInfo &data) { callback(Status::OK()); };
|
const GcsNodeInfo &data) { callback(Status::OK()); };
|
||||||
}
|
}
|
||||||
ClientTable &client_table = client_impl_->client_table();
|
NodeTable &node_table = client_impl_->node_table();
|
||||||
return client_table.MarkConnected(node_info, on_done);
|
return node_table.MarkConnected(node_info, on_done);
|
||||||
}
|
}
|
||||||
|
|
||||||
Status RedisNodeInfoAccessor::AsyncUnregister(const NodeID &node_id,
|
Status RedisNodeInfoAccessor::AsyncUnregister(const NodeID &node_id,
|
||||||
const StatusCallback &callback) {
|
const StatusCallback &callback) {
|
||||||
ClientTable::WriteCallback on_done = nullptr;
|
NodeTable::WriteCallback on_done = nullptr;
|
||||||
if (callback != nullptr) {
|
if (callback != nullptr) {
|
||||||
on_done = [callback](RedisGcsClient *client, const NodeID &id,
|
on_done = [callback](RedisGcsClient *client, const NodeID &id,
|
||||||
const GcsNodeInfo &data) { callback(Status::OK()); };
|
const GcsNodeInfo &data) { callback(Status::OK()); };
|
||||||
}
|
}
|
||||||
ClientTable &client_table = client_impl_->client_table();
|
NodeTable &node_table = client_impl_->node_table();
|
||||||
return client_table.MarkDisconnected(node_id, on_done);
|
return node_table.MarkDisconnected(node_id, on_done);
|
||||||
}
|
}
|
||||||
|
|
||||||
Status RedisNodeInfoAccessor::AsyncSubscribeToNodeChange(
|
Status RedisNodeInfoAccessor::AsyncSubscribeToNodeChange(
|
||||||
const SubscribeCallback<NodeID, GcsNodeInfo> &subscribe, const StatusCallback &done) {
|
const SubscribeCallback<NodeID, GcsNodeInfo> &subscribe, const StatusCallback &done) {
|
||||||
RAY_CHECK(subscribe != nullptr);
|
RAY_CHECK(subscribe != nullptr);
|
||||||
ClientTable &client_table = client_impl_->client_table();
|
NodeTable &node_table = client_impl_->node_table();
|
||||||
return client_table.SubscribeToNodeChange(subscribe, done);
|
return node_table.SubscribeToNodeChange(subscribe, done);
|
||||||
}
|
}
|
||||||
|
|
||||||
Status RedisNodeInfoAccessor::AsyncGetAll(
|
Status RedisNodeInfoAccessor::AsyncGetAll(
|
||||||
|
@ -565,15 +565,15 @@ Status RedisNodeInfoAccessor::AsyncGetAll(
|
||||||
}
|
}
|
||||||
callback(Status::OK(), result);
|
callback(Status::OK(), result);
|
||||||
};
|
};
|
||||||
ClientTable &client_table = client_impl_->client_table();
|
NodeTable &node_table = client_impl_->node_table();
|
||||||
return client_table.Lookup(on_done);
|
return node_table.Lookup(on_done);
|
||||||
}
|
}
|
||||||
|
|
||||||
boost::optional<GcsNodeInfo> RedisNodeInfoAccessor::Get(const NodeID &node_id,
|
boost::optional<GcsNodeInfo> RedisNodeInfoAccessor::Get(const NodeID &node_id,
|
||||||
bool filter_dead_nodes) const {
|
bool filter_dead_nodes) const {
|
||||||
GcsNodeInfo node_info;
|
GcsNodeInfo node_info;
|
||||||
ClientTable &client_table = client_impl_->client_table();
|
NodeTable &node_table = client_impl_->node_table();
|
||||||
bool found = client_table.GetClient(node_id, &node_info);
|
bool found = node_table.GetNode(node_id, &node_info);
|
||||||
boost::optional<GcsNodeInfo> optional_node;
|
boost::optional<GcsNodeInfo> optional_node;
|
||||||
if (found) {
|
if (found) {
|
||||||
optional_node = std::move(node_info);
|
optional_node = std::move(node_info);
|
||||||
|
@ -582,13 +582,13 @@ boost::optional<GcsNodeInfo> RedisNodeInfoAccessor::Get(const NodeID &node_id,
|
||||||
}
|
}
|
||||||
|
|
||||||
const std::unordered_map<NodeID, GcsNodeInfo> &RedisNodeInfoAccessor::GetAll() const {
|
const std::unordered_map<NodeID, GcsNodeInfo> &RedisNodeInfoAccessor::GetAll() const {
|
||||||
ClientTable &client_table = client_impl_->client_table();
|
NodeTable &node_table = client_impl_->node_table();
|
||||||
return client_table.GetAllClients();
|
return node_table.GetAllNodes();
|
||||||
}
|
}
|
||||||
|
|
||||||
bool RedisNodeInfoAccessor::IsRemoved(const NodeID &node_id) const {
|
bool RedisNodeInfoAccessor::IsRemoved(const NodeID &node_id) const {
|
||||||
ClientTable &client_table = client_impl_->client_table();
|
NodeTable &node_table = client_impl_->node_table();
|
||||||
return client_table.IsRemoved(node_id);
|
return node_table.IsRemoved(node_id);
|
||||||
}
|
}
|
||||||
Status RedisNodeInfoAccessor::AsyncReportHeartbeat(
|
Status RedisNodeInfoAccessor::AsyncReportHeartbeat(
|
||||||
const std::shared_ptr<HeartbeatTableData> &data_ptr, const StatusCallback &callback) {
|
const std::shared_ptr<HeartbeatTableData> &data_ptr, const StatusCallback &callback) {
|
||||||
|
@ -598,7 +598,7 @@ Status RedisNodeInfoAccessor::AsyncReportHeartbeat(
|
||||||
const HeartbeatTableData &data) { callback(Status::OK()); };
|
const HeartbeatTableData &data) { callback(Status::OK()); };
|
||||||
}
|
}
|
||||||
|
|
||||||
NodeID node_id = NodeID::FromBinary(data_ptr->client_id());
|
NodeID node_id = NodeID::FromBinary(data_ptr->node_id());
|
||||||
HeartbeatTable &heartbeat_table = client_impl_->heartbeat_table();
|
HeartbeatTable &heartbeat_table = client_impl_->heartbeat_table();
|
||||||
return heartbeat_table.Add(JobID::Nil(), node_id, data_ptr, on_done);
|
return heartbeat_table.Add(JobID::Nil(), node_id, data_ptr, on_done);
|
||||||
}
|
}
|
||||||
|
|
|
@ -431,7 +431,7 @@ Status RedisContext::RunArgvAsync(const std::vector<std::string> &args,
|
||||||
return status;
|
return status;
|
||||||
}
|
}
|
||||||
|
|
||||||
Status RedisContext::SubscribeAsync(const NodeID &client_id,
|
Status RedisContext::SubscribeAsync(const NodeID &node_id,
|
||||||
const TablePubsub pubsub_channel,
|
const TablePubsub pubsub_channel,
|
||||||
const RedisCallback &redisCallback,
|
const RedisCallback &redisCallback,
|
||||||
int64_t *out_callback_index) {
|
int64_t *out_callback_index) {
|
||||||
|
@ -444,7 +444,7 @@ Status RedisContext::SubscribeAsync(const NodeID &client_id,
|
||||||
RAY_CHECK(out_callback_index != nullptr);
|
RAY_CHECK(out_callback_index != nullptr);
|
||||||
*out_callback_index = callback_index;
|
*out_callback_index = callback_index;
|
||||||
Status status = Status::OK();
|
Status status = Status::OK();
|
||||||
if (client_id.IsNil()) {
|
if (node_id.IsNil()) {
|
||||||
// Subscribe to all messages.
|
// Subscribe to all messages.
|
||||||
std::string redis_command = "SUBSCRIBE %d";
|
std::string redis_command = "SUBSCRIBE %d";
|
||||||
status = async_redis_subscribe_context_->RedisAsyncCommand(
|
status = async_redis_subscribe_context_->RedisAsyncCommand(
|
||||||
|
@ -456,7 +456,7 @@ Status RedisContext::SubscribeAsync(const NodeID &client_id,
|
||||||
status = async_redis_subscribe_context_->RedisAsyncCommand(
|
status = async_redis_subscribe_context_->RedisAsyncCommand(
|
||||||
reinterpret_cast<redisCallbackFn *>(&GlobalRedisCallback),
|
reinterpret_cast<redisCallbackFn *>(&GlobalRedisCallback),
|
||||||
reinterpret_cast<void *>(callback_index), redis_command.c_str(), pubsub_channel,
|
reinterpret_cast<void *>(callback_index), redis_command.c_str(), pubsub_channel,
|
||||||
client_id.Data(), client_id.Size());
|
node_id.Data(), node_id.Size());
|
||||||
}
|
}
|
||||||
|
|
||||||
return status;
|
return status;
|
||||||
|
|
|
@ -238,12 +238,12 @@ class RedisContext {
|
||||||
|
|
||||||
/// Subscribe to a specific Pub-Sub channel.
|
/// Subscribe to a specific Pub-Sub channel.
|
||||||
///
|
///
|
||||||
/// \param client_id The client ID that subscribe this message.
|
/// \param node_id The node ID that subscribe this message.
|
||||||
/// \param pubsub_channel The Pub-Sub channel to subscribe to.
|
/// \param pubsub_channel The Pub-Sub channel to subscribe to.
|
||||||
/// \param redisCallback The callback function that the notification calls.
|
/// \param redisCallback The callback function that the notification calls.
|
||||||
/// \param out_callback_index The output pointer to callback index.
|
/// \param out_callback_index The output pointer to callback index.
|
||||||
/// \return Status.
|
/// \return Status.
|
||||||
Status SubscribeAsync(const NodeID &client_id, const TablePubsub pubsub_channel,
|
Status SubscribeAsync(const NodeID &node_id, const TablePubsub pubsub_channel,
|
||||||
const RedisCallback &redisCallback, int64_t *out_callback_index);
|
const RedisCallback &redisCallback, int64_t *out_callback_index);
|
||||||
|
|
||||||
/// Subscribes the client to the given channel.
|
/// Subscribes the client to the given channel.
|
||||||
|
|
|
@ -48,11 +48,11 @@ Status RedisGcsClient::Connect(boost::asio::io_service &io_service) {
|
||||||
log_based_actor_table_.reset(new LogBasedActorTable({primary_context}, this));
|
log_based_actor_table_.reset(new LogBasedActorTable({primary_context}, this));
|
||||||
actor_table_.reset(new ActorTable({primary_context}, this));
|
actor_table_.reset(new ActorTable({primary_context}, this));
|
||||||
|
|
||||||
// TODO(micafan) Modify ClientTable' Constructor(remove NodeID) in future.
|
// TODO(micafan) Modify NodeTable' Constructor(remove NodeID) in future.
|
||||||
// We will use NodeID instead of NodeID.
|
// We will use NodeID instead of NodeID.
|
||||||
// For worker/driver, it might not have this field(NodeID).
|
// For worker/driver, it might not have this field(NodeID).
|
||||||
// For raylet, NodeID should be initialized in raylet layer(not here).
|
// For raylet, NodeID should be initialized in raylet layer(not here).
|
||||||
client_table_.reset(new ClientTable({primary_context}, this));
|
node_table_.reset(new NodeTable({primary_context}, this));
|
||||||
|
|
||||||
job_table_.reset(new JobTable({primary_context}, this));
|
job_table_.reset(new JobTable({primary_context}, this));
|
||||||
heartbeat_batch_table_.reset(new HeartbeatBatchTable({primary_context}, this));
|
heartbeat_batch_table_.reset(new HeartbeatBatchTable({primary_context}, this));
|
||||||
|
@ -103,7 +103,7 @@ std::string RedisGcsClient::DebugString() const {
|
||||||
result << "\n- TaskLeaseTable: " << task_lease_table_->DebugString();
|
result << "\n- TaskLeaseTable: " << task_lease_table_->DebugString();
|
||||||
result << "\n- HeartbeatTable: " << heartbeat_table_->DebugString();
|
result << "\n- HeartbeatTable: " << heartbeat_table_->DebugString();
|
||||||
result << "\n- ProfileTable: " << profile_table_->DebugString();
|
result << "\n- ProfileTable: " << profile_table_->DebugString();
|
||||||
result << "\n- ClientTable: " << client_table_->DebugString();
|
result << "\n- NodeTable: " << node_table_->DebugString();
|
||||||
result << "\n- JobTable: " << job_table_->DebugString();
|
result << "\n- JobTable: " << job_table_->DebugString();
|
||||||
return result.str();
|
return result.str();
|
||||||
}
|
}
|
||||||
|
@ -126,7 +126,7 @@ TaskReconstructionLog &RedisGcsClient::task_reconstruction_log() {
|
||||||
|
|
||||||
TaskLeaseTable &RedisGcsClient::task_lease_table() { return *task_lease_table_; }
|
TaskLeaseTable &RedisGcsClient::task_lease_table() { return *task_lease_table_; }
|
||||||
|
|
||||||
ClientTable &RedisGcsClient::client_table() { return *client_table_; }
|
NodeTable &RedisGcsClient::node_table() { return *node_table_; }
|
||||||
|
|
||||||
HeartbeatTable &RedisGcsClient::heartbeat_table() { return *heartbeat_table_; }
|
HeartbeatTable &RedisGcsClient::heartbeat_table() { return *heartbeat_table_; }
|
||||||
|
|
||||||
|
|
|
@ -93,7 +93,7 @@ class RAY_EXPORT RedisGcsClient : public GcsClient {
|
||||||
/// Implements the Objects() interface.
|
/// Implements the Objects() interface.
|
||||||
ObjectTable &object_table();
|
ObjectTable &object_table();
|
||||||
/// Implements the Nodes() interface.
|
/// Implements the Nodes() interface.
|
||||||
ClientTable &client_table();
|
NodeTable &node_table();
|
||||||
HeartbeatTable &heartbeat_table();
|
HeartbeatTable &heartbeat_table();
|
||||||
HeartbeatBatchTable &heartbeat_batch_table();
|
HeartbeatBatchTable &heartbeat_batch_table();
|
||||||
DynamicResourceTable &resource_table();
|
DynamicResourceTable &resource_table();
|
||||||
|
@ -122,7 +122,7 @@ class RAY_EXPORT RedisGcsClient : public GcsClient {
|
||||||
std::unique_ptr<HeartbeatTable> heartbeat_table_;
|
std::unique_ptr<HeartbeatTable> heartbeat_table_;
|
||||||
std::unique_ptr<HeartbeatBatchTable> heartbeat_batch_table_;
|
std::unique_ptr<HeartbeatBatchTable> heartbeat_batch_table_;
|
||||||
std::unique_ptr<ProfileTable> profile_table_;
|
std::unique_ptr<ProfileTable> profile_table_;
|
||||||
std::unique_ptr<ClientTable> client_table_;
|
std::unique_ptr<NodeTable> node_table_;
|
||||||
std::unique_ptr<ActorCheckpointTable> actor_checkpoint_table_;
|
std::unique_ptr<ActorCheckpointTable> actor_checkpoint_table_;
|
||||||
std::unique_ptr<ActorCheckpointIdTable> actor_checkpoint_id_table_;
|
std::unique_ptr<ActorCheckpointIdTable> actor_checkpoint_id_table_;
|
||||||
std::unique_ptr<DynamicResourceTable> resource_table_;
|
std::unique_ptr<DynamicResourceTable> resource_table_;
|
||||||
|
|
|
@ -20,7 +20,7 @@ namespace gcs {
|
||||||
|
|
||||||
template <typename ID, typename Data, typename Table>
|
template <typename ID, typename Data, typename Table>
|
||||||
Status SubscriptionExecutor<ID, Data, Table>::AsyncSubscribeAll(
|
Status SubscriptionExecutor<ID, Data, Table>::AsyncSubscribeAll(
|
||||||
const NodeID &client_id, const SubscribeCallback<ID, Data> &subscribe,
|
const NodeID &node_id, const SubscribeCallback<ID, Data> &subscribe,
|
||||||
const StatusCallback &done) {
|
const StatusCallback &done) {
|
||||||
// TODO(micafan) Optimize the lock when necessary.
|
// TODO(micafan) Optimize the lock when necessary.
|
||||||
// Consider avoiding locking in single-threaded processes.
|
// Consider avoiding locking in single-threaded processes.
|
||||||
|
@ -99,7 +99,7 @@ Status SubscriptionExecutor<ID, Data, Table>::AsyncSubscribeAll(
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
Status status = table_.Subscribe(JobID::Nil(), client_id, on_subscribe, on_done);
|
Status status = table_.Subscribe(JobID::Nil(), node_id, on_subscribe, on_done);
|
||||||
if (status.ok()) {
|
if (status.ok()) {
|
||||||
registration_status_ = RegistrationStatus::kRegistering;
|
registration_status_ = RegistrationStatus::kRegistering;
|
||||||
subscribe_all_callback_ = subscribe;
|
subscribe_all_callback_ = subscribe;
|
||||||
|
@ -110,15 +110,15 @@ Status SubscriptionExecutor<ID, Data, Table>::AsyncSubscribeAll(
|
||||||
|
|
||||||
template <typename ID, typename Data, typename Table>
|
template <typename ID, typename Data, typename Table>
|
||||||
Status SubscriptionExecutor<ID, Data, Table>::AsyncSubscribe(
|
Status SubscriptionExecutor<ID, Data, Table>::AsyncSubscribe(
|
||||||
const NodeID &client_id, const ID &id, const SubscribeCallback<ID, Data> &subscribe,
|
const NodeID &node_id, const ID &id, const SubscribeCallback<ID, Data> &subscribe,
|
||||||
const StatusCallback &done) {
|
const StatusCallback &done) {
|
||||||
RAY_CHECK(client_id != NodeID::Nil());
|
RAY_CHECK(node_id != NodeID::Nil());
|
||||||
|
|
||||||
// NOTE(zhijunfu): `Subscribe` and other operations use different redis contexts,
|
// NOTE(zhijunfu): `Subscribe` and other operations use different redis contexts,
|
||||||
// thus we need to call `RequestNotifications` in the Subscribe callback to ensure
|
// thus we need to call `RequestNotifications` in the Subscribe callback to ensure
|
||||||
// it's processed after the `Subscribe` request. Otherwise if `RequestNotifications`
|
// it's processed after the `Subscribe` request. Otherwise if `RequestNotifications`
|
||||||
// is processed first we will miss the initial notification.
|
// is processed first we will miss the initial notification.
|
||||||
auto on_subscribe_done = [this, client_id, id, subscribe, done](Status status) {
|
auto on_subscribe_done = [this, node_id, id, subscribe, done](Status status) {
|
||||||
auto on_request_notification_done = [this, done, id](Status status) {
|
auto on_request_notification_done = [this, done, id](Status status) {
|
||||||
if (!status.ok()) {
|
if (!status.ok()) {
|
||||||
std::unique_lock<std::mutex> lock(mutex_);
|
std::unique_lock<std::mutex> lock(mutex_);
|
||||||
|
@ -131,7 +131,7 @@ Status SubscriptionExecutor<ID, Data, Table>::AsyncSubscribe(
|
||||||
|
|
||||||
{
|
{
|
||||||
std::unique_lock<std::mutex> lock(mutex_);
|
std::unique_lock<std::mutex> lock(mutex_);
|
||||||
status = table_.RequestNotifications(JobID::Nil(), id, client_id,
|
status = table_.RequestNotifications(JobID::Nil(), id, node_id,
|
||||||
on_request_notification_done);
|
on_request_notification_done);
|
||||||
if (!status.ok()) {
|
if (!status.ok()) {
|
||||||
id_to_callback_map_.erase(id);
|
id_to_callback_map_.erase(id);
|
||||||
|
@ -143,14 +143,13 @@ Status SubscriptionExecutor<ID, Data, Table>::AsyncSubscribe(
|
||||||
std::unique_lock<std::mutex> lock(mutex_);
|
std::unique_lock<std::mutex> lock(mutex_);
|
||||||
const auto it = id_to_callback_map_.find(id);
|
const auto it = id_to_callback_map_.find(id);
|
||||||
if (it != id_to_callback_map_.end()) {
|
if (it != id_to_callback_map_.end()) {
|
||||||
RAY_LOG(DEBUG) << "Duplicate subscription to id " << id << " client_id "
|
RAY_LOG(DEBUG) << "Duplicate subscription to id " << id << " node_id " << node_id;
|
||||||
<< client_id;
|
|
||||||
return Status::Invalid("Duplicate subscription to element!");
|
return Status::Invalid("Duplicate subscription to element!");
|
||||||
}
|
}
|
||||||
id_to_callback_map_[id] = subscribe;
|
id_to_callback_map_[id] = subscribe;
|
||||||
}
|
}
|
||||||
|
|
||||||
auto status = AsyncSubscribeAll(client_id, nullptr, on_subscribe_done);
|
auto status = AsyncSubscribeAll(node_id, nullptr, on_subscribe_done);
|
||||||
if (!status.ok()) {
|
if (!status.ok()) {
|
||||||
std::unique_lock<std::mutex> lock(mutex_);
|
std::unique_lock<std::mutex> lock(mutex_);
|
||||||
id_to_callback_map_.erase(id);
|
id_to_callback_map_.erase(id);
|
||||||
|
@ -160,13 +159,13 @@ Status SubscriptionExecutor<ID, Data, Table>::AsyncSubscribe(
|
||||||
|
|
||||||
template <typename ID, typename Data, typename Table>
|
template <typename ID, typename Data, typename Table>
|
||||||
Status SubscriptionExecutor<ID, Data, Table>::AsyncUnsubscribe(
|
Status SubscriptionExecutor<ID, Data, Table>::AsyncUnsubscribe(
|
||||||
const NodeID &client_id, const ID &id, const StatusCallback &done) {
|
const NodeID &node_id, const ID &id, const StatusCallback &done) {
|
||||||
SubscribeCallback<ID, Data> subscribe = nullptr;
|
SubscribeCallback<ID, Data> subscribe = nullptr;
|
||||||
{
|
{
|
||||||
std::unique_lock<std::mutex> lock(mutex_);
|
std::unique_lock<std::mutex> lock(mutex_);
|
||||||
const auto it = id_to_callback_map_.find(id);
|
const auto it = id_to_callback_map_.find(id);
|
||||||
if (it == id_to_callback_map_.end()) {
|
if (it == id_to_callback_map_.end()) {
|
||||||
RAY_LOG(DEBUG) << "Invalid Unsubscribe! id " << id << " client_id " << client_id;
|
RAY_LOG(DEBUG) << "Invalid Unsubscribe! id " << id << " node_id " << node_id;
|
||||||
return Status::Invalid("Invalid Unsubscribe, no existing subscription found.");
|
return Status::Invalid("Invalid Unsubscribe, no existing subscription found.");
|
||||||
}
|
}
|
||||||
subscribe = std::move(it->second);
|
subscribe = std::move(it->second);
|
||||||
|
@ -195,7 +194,7 @@ Status SubscriptionExecutor<ID, Data, Table>::AsyncUnsubscribe(
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
return table_.CancelNotifications(JobID::Nil(), id, client_id, on_done);
|
return table_.CancelNotifications(JobID::Nil(), id, node_id, on_done);
|
||||||
}
|
}
|
||||||
|
|
||||||
template class SubscriptionExecutor<ActorID, ActorTableData, LogBasedActorTable>;
|
template class SubscriptionExecutor<ActorID, ActorTableData, LogBasedActorTable>;
|
||||||
|
|
|
@ -39,42 +39,42 @@ class SubscriptionExecutor {
|
||||||
/// Subscribe to operations of all elements.
|
/// Subscribe to operations of all elements.
|
||||||
/// Repeated subscription will return a failure.
|
/// Repeated subscription will return a failure.
|
||||||
///
|
///
|
||||||
/// \param client_id The type of update to listen to. If this is nil, then a
|
/// \param node_id The type of update to listen to. If this is nil, then a
|
||||||
/// message for each update will be received. Else, only
|
/// message for each update will be received. Else, only
|
||||||
/// messages for the given client will be received.
|
/// messages for the given node will be received.
|
||||||
/// \param subscribe Callback that will be called each time when an element
|
/// \param subscribe Callback that will be called each time when an element
|
||||||
/// is registered or updated.
|
/// is registered or updated.
|
||||||
/// \param done Callback that will be called when subscription is complete.
|
/// \param done Callback that will be called when subscription is complete.
|
||||||
/// \return Status
|
/// \return Status
|
||||||
Status AsyncSubscribeAll(const NodeID &client_id,
|
Status AsyncSubscribeAll(const NodeID &node_id,
|
||||||
const SubscribeCallback<ID, Data> &subscribe,
|
const SubscribeCallback<ID, Data> &subscribe,
|
||||||
const StatusCallback &done);
|
const StatusCallback &done);
|
||||||
|
|
||||||
/// Subscribe to operations of an element.
|
/// Subscribe to operations of an element.
|
||||||
/// Repeated subscription to an element will return a failure.
|
/// Repeated subscription to an element will return a failure.
|
||||||
///
|
///
|
||||||
/// \param client_id The type of update to listen to. If this is nil, then a
|
/// \param node_id The type of update to listen to. If this is nil, then a
|
||||||
/// message for each update will be received. Else, only
|
/// message for each update will be received. Else, only
|
||||||
/// messages for the given client will be received.
|
/// messages for the given node will be received.
|
||||||
/// \param id The id of the element to be subscribe to.
|
/// \param id The id of the element to be subscribe to.
|
||||||
/// \param subscribe Callback that will be called each time when the element
|
/// \param subscribe Callback that will be called each time when the element
|
||||||
/// is registered or updated.
|
/// is registered or updated.
|
||||||
/// \param done Callback that will be called when subscription is complete.
|
/// \param done Callback that will be called when subscription is complete.
|
||||||
/// \return Status
|
/// \return Status
|
||||||
Status AsyncSubscribe(const NodeID &client_id, const ID &id,
|
Status AsyncSubscribe(const NodeID &node_id, const ID &id,
|
||||||
const SubscribeCallback<ID, Data> &subscribe,
|
const SubscribeCallback<ID, Data> &subscribe,
|
||||||
const StatusCallback &done);
|
const StatusCallback &done);
|
||||||
|
|
||||||
/// Cancel subscription to an element.
|
/// Cancel subscription to an element.
|
||||||
/// Unsubscribing can only be called after the subscription request is completed.
|
/// Unsubscribing can only be called after the subscription request is completed.
|
||||||
///
|
///
|
||||||
/// \param client_id The type of update to listen to. If this is nil, then a
|
/// \param node_id The type of update to listen to. If this is nil, then a
|
||||||
/// message for each update will be received. Else, only
|
/// message for each update will be received. Else, only
|
||||||
/// messages for the given client will be received.
|
/// messages for the given node will be received.
|
||||||
/// \param id The id of the element to be unsubscribed to.
|
/// \param id The id of the element to be unsubscribed to.
|
||||||
/// \param done Callback that will be called when cancel subscription is complete.
|
/// \param done Callback that will be called when cancel subscription is complete.
|
||||||
/// \return Status
|
/// \return Status
|
||||||
Status AsyncUnsubscribe(const NodeID &client_id, const ID &id,
|
Status AsyncUnsubscribe(const NodeID &node_id, const ID &id,
|
||||||
const StatusCallback &done);
|
const StatusCallback &done);
|
||||||
|
|
||||||
private:
|
private:
|
||||||
|
|
|
@ -137,7 +137,7 @@ Status Log<ID, Data>::Lookup(const JobID &job_id, const ID &id, const Callback &
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename ID, typename Data>
|
template <typename ID, typename Data>
|
||||||
Status Log<ID, Data>::Subscribe(const JobID &job_id, const NodeID &client_id,
|
Status Log<ID, Data>::Subscribe(const JobID &job_id, const NodeID &node_id,
|
||||||
const Callback &subscribe,
|
const Callback &subscribe,
|
||||||
const SubscriptionCallback &done) {
|
const SubscriptionCallback &done) {
|
||||||
auto subscribe_wrapper = [subscribe](RedisGcsClient *client, const ID &id,
|
auto subscribe_wrapper = [subscribe](RedisGcsClient *client, const ID &id,
|
||||||
|
@ -146,11 +146,11 @@ Status Log<ID, Data>::Subscribe(const JobID &job_id, const NodeID &client_id,
|
||||||
RAY_CHECK(change_mode != GcsChangeMode::REMOVE);
|
RAY_CHECK(change_mode != GcsChangeMode::REMOVE);
|
||||||
subscribe(client, id, data);
|
subscribe(client, id, data);
|
||||||
};
|
};
|
||||||
return Subscribe(job_id, client_id, subscribe_wrapper, done);
|
return Subscribe(job_id, node_id, subscribe_wrapper, done);
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename ID, typename Data>
|
template <typename ID, typename Data>
|
||||||
Status Log<ID, Data>::Subscribe(const JobID &job_id, const NodeID &client_id,
|
Status Log<ID, Data>::Subscribe(const JobID &job_id, const NodeID &node_id,
|
||||||
const NotificationCallback &subscribe,
|
const NotificationCallback &subscribe,
|
||||||
const SubscriptionCallback &done) {
|
const SubscriptionCallback &done) {
|
||||||
RAY_CHECK(subscribe_callback_index_ == -1)
|
RAY_CHECK(subscribe_callback_index_ == -1)
|
||||||
|
@ -184,7 +184,7 @@ Status Log<ID, Data>::Subscribe(const JobID &job_id, const NodeID &client_id,
|
||||||
|
|
||||||
subscribe_callback_index_ = 1;
|
subscribe_callback_index_ = 1;
|
||||||
for (auto &context : shard_contexts_) {
|
for (auto &context : shard_contexts_) {
|
||||||
RAY_RETURN_NOT_OK(context->SubscribeAsync(client_id, pubsub_channel_, callback,
|
RAY_RETURN_NOT_OK(context->SubscribeAsync(node_id, pubsub_channel_, callback,
|
||||||
&subscribe_callback_index_));
|
&subscribe_callback_index_));
|
||||||
}
|
}
|
||||||
return Status::OK();
|
return Status::OK();
|
||||||
|
@ -192,7 +192,7 @@ Status Log<ID, Data>::Subscribe(const JobID &job_id, const NodeID &client_id,
|
||||||
|
|
||||||
template <typename ID, typename Data>
|
template <typename ID, typename Data>
|
||||||
Status Log<ID, Data>::RequestNotifications(const JobID &job_id, const ID &id,
|
Status Log<ID, Data>::RequestNotifications(const JobID &job_id, const ID &id,
|
||||||
const NodeID &client_id,
|
const NodeID &node_id,
|
||||||
const StatusCallback &done) {
|
const StatusCallback &done) {
|
||||||
RAY_CHECK(subscribe_callback_index_ >= 0)
|
RAY_CHECK(subscribe_callback_index_ >= 0)
|
||||||
<< "Client requested notifications on a key before Subscribe completed";
|
<< "Client requested notifications on a key before Subscribe completed";
|
||||||
|
@ -208,13 +208,13 @@ Status Log<ID, Data>::RequestNotifications(const JobID &job_id, const ID &id,
|
||||||
}
|
}
|
||||||
|
|
||||||
return GetRedisContext(id)->RunAsync("RAY.TABLE_REQUEST_NOTIFICATIONS", id,
|
return GetRedisContext(id)->RunAsync("RAY.TABLE_REQUEST_NOTIFICATIONS", id,
|
||||||
client_id.Data(), client_id.Size(), prefix_,
|
node_id.Data(), node_id.Size(), prefix_,
|
||||||
pubsub_channel_, callback);
|
pubsub_channel_, callback);
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename ID, typename Data>
|
template <typename ID, typename Data>
|
||||||
Status Log<ID, Data>::CancelNotifications(const JobID &job_id, const ID &id,
|
Status Log<ID, Data>::CancelNotifications(const JobID &job_id, const ID &id,
|
||||||
const NodeID &client_id,
|
const NodeID &node_id,
|
||||||
const StatusCallback &done) {
|
const StatusCallback &done) {
|
||||||
RAY_CHECK(subscribe_callback_index_ >= 0)
|
RAY_CHECK(subscribe_callback_index_ >= 0)
|
||||||
<< "Client canceled notifications on a key before Subscribe completed";
|
<< "Client canceled notifications on a key before Subscribe completed";
|
||||||
|
@ -228,7 +228,7 @@ Status Log<ID, Data>::CancelNotifications(const JobID &job_id, const ID &id,
|
||||||
}
|
}
|
||||||
|
|
||||||
return GetRedisContext(id)->RunAsync("RAY.TABLE_CANCEL_NOTIFICATIONS", id,
|
return GetRedisContext(id)->RunAsync("RAY.TABLE_CANCEL_NOTIFICATIONS", id,
|
||||||
client_id.Data(), client_id.Size(), prefix_,
|
node_id.Data(), node_id.Size(), prefix_,
|
||||||
pubsub_channel_, callback);
|
pubsub_channel_, callback);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -315,12 +315,12 @@ Status Table<ID, Data>::Lookup(const JobID &job_id, const ID &id, const Callback
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename ID, typename Data>
|
template <typename ID, typename Data>
|
||||||
Status Table<ID, Data>::Subscribe(const JobID &job_id, const NodeID &client_id,
|
Status Table<ID, Data>::Subscribe(const JobID &job_id, const NodeID &node_id,
|
||||||
const Callback &subscribe,
|
const Callback &subscribe,
|
||||||
const FailureCallback &failure,
|
const FailureCallback &failure,
|
||||||
const SubscriptionCallback &done) {
|
const SubscriptionCallback &done) {
|
||||||
return Log<ID, Data>::Subscribe(
|
return Log<ID, Data>::Subscribe(
|
||||||
job_id, client_id,
|
job_id, node_id,
|
||||||
[subscribe, failure](RedisGcsClient *client, const ID &id,
|
[subscribe, failure](RedisGcsClient *client, const ID &id,
|
||||||
const std::vector<Data> &data) {
|
const std::vector<Data> &data) {
|
||||||
RAY_CHECK(data.empty() || data.size() == 1);
|
RAY_CHECK(data.empty() || data.size() == 1);
|
||||||
|
@ -336,10 +336,10 @@ Status Table<ID, Data>::Subscribe(const JobID &job_id, const NodeID &client_id,
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename ID, typename Data>
|
template <typename ID, typename Data>
|
||||||
Status Table<ID, Data>::Subscribe(const JobID &job_id, const NodeID &client_id,
|
Status Table<ID, Data>::Subscribe(const JobID &job_id, const NodeID &node_id,
|
||||||
const Callback &subscribe,
|
const Callback &subscribe,
|
||||||
const SubscriptionCallback &done) {
|
const SubscriptionCallback &done) {
|
||||||
return Subscribe(job_id, client_id, subscribe, /*failure*/ nullptr, done);
|
return Subscribe(job_id, node_id, subscribe, /*failure*/ nullptr, done);
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename ID, typename Data>
|
template <typename ID, typename Data>
|
||||||
|
@ -379,7 +379,7 @@ Status Set<ID, Data>::Remove(const JobID &job_id, const ID &id,
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename ID, typename Data>
|
template <typename ID, typename Data>
|
||||||
Status Set<ID, Data>::Subscribe(const JobID &job_id, const NodeID &client_id,
|
Status Set<ID, Data>::Subscribe(const JobID &job_id, const NodeID &node_id,
|
||||||
const NotificationCallback &subscribe,
|
const NotificationCallback &subscribe,
|
||||||
const SubscriptionCallback &done) {
|
const SubscriptionCallback &done) {
|
||||||
auto on_subscribe = [subscribe](RedisGcsClient *client, const ID &id,
|
auto on_subscribe = [subscribe](RedisGcsClient *client, const ID &id,
|
||||||
|
@ -390,7 +390,7 @@ Status Set<ID, Data>::Subscribe(const JobID &job_id, const NodeID &client_id,
|
||||||
notification_vec.emplace_back(std::move(change_notification));
|
notification_vec.emplace_back(std::move(change_notification));
|
||||||
subscribe(client, id, notification_vec);
|
subscribe(client, id, notification_vec);
|
||||||
};
|
};
|
||||||
return Log<ID, Data>::Subscribe(job_id, client_id, on_subscribe, done);
|
return Log<ID, Data>::Subscribe(job_id, node_id, on_subscribe, done);
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename ID, typename Data>
|
template <typename ID, typename Data>
|
||||||
|
@ -481,7 +481,7 @@ Status Hash<ID, Data>::Lookup(const JobID &job_id, const ID &id,
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename ID, typename Data>
|
template <typename ID, typename Data>
|
||||||
Status Hash<ID, Data>::Subscribe(const JobID &job_id, const NodeID &client_id,
|
Status Hash<ID, Data>::Subscribe(const JobID &job_id, const NodeID &node_id,
|
||||||
const HashNotificationCallback &subscribe,
|
const HashNotificationCallback &subscribe,
|
||||||
const SubscriptionCallback &done) {
|
const SubscriptionCallback &done) {
|
||||||
RAY_CHECK(subscribe_callback_index_ == -1)
|
RAY_CHECK(subscribe_callback_index_ == -1)
|
||||||
|
@ -526,7 +526,7 @@ Status Hash<ID, Data>::Subscribe(const JobID &job_id, const NodeID &client_id,
|
||||||
|
|
||||||
subscribe_callback_index_ = 1;
|
subscribe_callback_index_ = 1;
|
||||||
for (auto &context : shard_contexts_) {
|
for (auto &context : shard_contexts_) {
|
||||||
RAY_RETURN_NOT_OK(context->SubscribeAsync(client_id, pubsub_channel_, callback,
|
RAY_RETURN_NOT_OK(context->SubscribeAsync(node_id, pubsub_channel_, callback,
|
||||||
&subscribe_callback_index_));
|
&subscribe_callback_index_));
|
||||||
}
|
}
|
||||||
return Status::OK();
|
return Status::OK();
|
||||||
|
@ -536,7 +536,7 @@ std::string ProfileTable::DebugString() const {
|
||||||
return Log<UniqueID, ProfileTableData>::DebugString();
|
return Log<UniqueID, ProfileTableData>::DebugString();
|
||||||
}
|
}
|
||||||
|
|
||||||
void ClientTable::RegisterNodeChangeCallback(const NodeChangeCallback &callback) {
|
void NodeTable::RegisterNodeChangeCallback(const NodeChangeCallback &callback) {
|
||||||
RAY_CHECK(node_change_callback_ == nullptr);
|
RAY_CHECK(node_change_callback_ == nullptr);
|
||||||
node_change_callback_ = callback;
|
node_change_callback_ = callback;
|
||||||
// Call the callback for any added clients that are cached.
|
// Call the callback for any added clients that are cached.
|
||||||
|
@ -549,8 +549,7 @@ void ClientTable::RegisterNodeChangeCallback(const NodeChangeCallback &callback)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void ClientTable::HandleNotification(RedisGcsClient *client,
|
void NodeTable::HandleNotification(RedisGcsClient *client, const GcsNodeInfo &node_info) {
|
||||||
const GcsNodeInfo &node_info) {
|
|
||||||
NodeID node_id = NodeID::FromBinary(node_info.node_id());
|
NodeID node_id = NodeID::FromBinary(node_info.node_id());
|
||||||
bool is_alive = (node_info.state() == GcsNodeInfo::ALIVE);
|
bool is_alive = (node_info.state() == GcsNodeInfo::ALIVE);
|
||||||
// It's possible to get duplicate notifications from the client table, so
|
// It's possible to get duplicate notifications from the client table, so
|
||||||
|
@ -565,20 +564,20 @@ void ClientTable::HandleNotification(RedisGcsClient *client,
|
||||||
// was alive and is now dead or resources have been updated.
|
// was alive and is now dead or resources have been updated.
|
||||||
bool was_alive = (entry->second.state() == GcsNodeInfo::ALIVE);
|
bool was_alive = (entry->second.state() == GcsNodeInfo::ALIVE);
|
||||||
is_notif_new = was_alive && !is_alive;
|
is_notif_new = was_alive && !is_alive;
|
||||||
// Once a client with a given ID has been removed, it should never be added
|
// Once a node with a given ID has been removed, it should never be added
|
||||||
// again. If the entry was in the cache and the client was deleted, check
|
// again. If the entry was in the cache and the node was deleted, check
|
||||||
// that this new notification is not an insertion.
|
// that this new notification is not an insertion.
|
||||||
if (!was_alive) {
|
if (!was_alive) {
|
||||||
RAY_CHECK(!is_alive)
|
RAY_CHECK(!is_alive)
|
||||||
<< "Notification for addition of a client that was already removed:" << node_id;
|
<< "Notification for addition of a node that was already removed:" << node_id;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Add the notification to our cache. Notifications are idempotent.
|
// Add the notification to our cache. Notifications are idempotent.
|
||||||
RAY_LOG(DEBUG) << "[ClientTableNotification] ClientTable Insertion/Deletion "
|
RAY_LOG(DEBUG) << "[NodeTableNotification] NodeTable Insertion/Deletion "
|
||||||
"notification for client id "
|
"notification for node id "
|
||||||
<< node_id << ". IsAlive: " << is_alive
|
<< node_id << ". IsAlive: " << is_alive
|
||||||
<< ". Setting the client cache to data.";
|
<< ". Setting the node cache to data.";
|
||||||
node_cache_[node_id] = node_info;
|
node_cache_[node_id] = node_info;
|
||||||
|
|
||||||
// If the notification is new, call any registered callbacks.
|
// If the notification is new, call any registered callbacks.
|
||||||
|
@ -598,24 +597,24 @@ void ClientTable::HandleNotification(RedisGcsClient *client,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
const NodeID &ClientTable::GetLocalClientId() const {
|
const NodeID &NodeTable::GetLocalNodeId() const {
|
||||||
RAY_CHECK(!local_node_id_.IsNil());
|
RAY_CHECK(!local_node_id_.IsNil());
|
||||||
return local_node_id_;
|
return local_node_id_;
|
||||||
}
|
}
|
||||||
|
|
||||||
const GcsNodeInfo &ClientTable::GetLocalClient() const { return local_node_info_; }
|
const GcsNodeInfo &NodeTable::GetLocalNode() const { return local_node_info_; }
|
||||||
|
|
||||||
bool ClientTable::IsRemoved(const NodeID &node_id) const {
|
bool NodeTable::IsRemoved(const NodeID &node_id) const {
|
||||||
return removed_nodes_.count(node_id) == 1;
|
return removed_nodes_.count(node_id) == 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
Status ClientTable::Connect(const GcsNodeInfo &local_node_info) {
|
Status NodeTable::Connect(const GcsNodeInfo &local_node_info) {
|
||||||
RAY_CHECK(!disconnected_) << "Tried to reconnect a disconnected node.";
|
RAY_CHECK(!disconnected_) << "Tried to reconnect a disconnected node.";
|
||||||
RAY_CHECK(local_node_id_.IsNil()) << "This node is already connected.";
|
RAY_CHECK(local_node_id_.IsNil()) << "This node is already connected.";
|
||||||
RAY_CHECK(local_node_info.state() == GcsNodeInfo::ALIVE);
|
RAY_CHECK(local_node_info.state() == GcsNodeInfo::ALIVE);
|
||||||
|
|
||||||
auto node_info_ptr = std::make_shared<GcsNodeInfo>(local_node_info);
|
auto node_info_ptr = std::make_shared<GcsNodeInfo>(local_node_info);
|
||||||
Status status = SyncAppend(JobID::Nil(), client_log_key_, node_info_ptr);
|
Status status = SyncAppend(JobID::Nil(), node_log_key_, node_info_ptr);
|
||||||
if (status.ok()) {
|
if (status.ok()) {
|
||||||
local_node_id_ = NodeID::FromBinary(local_node_info.node_id());
|
local_node_id_ = NodeID::FromBinary(local_node_info.node_id());
|
||||||
local_node_info_ = local_node_info;
|
local_node_info_ = local_node_info;
|
||||||
|
@ -623,10 +622,10 @@ Status ClientTable::Connect(const GcsNodeInfo &local_node_info) {
|
||||||
return status;
|
return status;
|
||||||
}
|
}
|
||||||
|
|
||||||
Status ClientTable::Disconnect() {
|
Status NodeTable::Disconnect() {
|
||||||
local_node_info_.set_state(GcsNodeInfo::DEAD);
|
local_node_info_.set_state(GcsNodeInfo::DEAD);
|
||||||
auto node_info_ptr = std::make_shared<GcsNodeInfo>(local_node_info_);
|
auto node_info_ptr = std::make_shared<GcsNodeInfo>(local_node_info_);
|
||||||
Status status = SyncAppend(JobID::Nil(), client_log_key_, node_info_ptr);
|
Status status = SyncAppend(JobID::Nil(), node_log_key_, node_info_ptr);
|
||||||
|
|
||||||
if (status.ok()) {
|
if (status.ok()) {
|
||||||
// We successfully added the deletion entry. Mark ourselves as disconnected.
|
// We successfully added the deletion entry. Mark ourselves as disconnected.
|
||||||
|
@ -635,27 +634,27 @@ Status ClientTable::Disconnect() {
|
||||||
return status;
|
return status;
|
||||||
}
|
}
|
||||||
|
|
||||||
ray::Status ClientTable::MarkConnected(const GcsNodeInfo &node_info,
|
ray::Status NodeTable::MarkConnected(const GcsNodeInfo &node_info,
|
||||||
const WriteCallback &done) {
|
const WriteCallback &done) {
|
||||||
RAY_CHECK(node_info.state() == GcsNodeInfo::ALIVE);
|
RAY_CHECK(node_info.state() == GcsNodeInfo::ALIVE);
|
||||||
auto node_info_ptr = std::make_shared<GcsNodeInfo>(node_info);
|
auto node_info_ptr = std::make_shared<GcsNodeInfo>(node_info);
|
||||||
return Append(JobID::Nil(), client_log_key_, node_info_ptr, done);
|
return Append(JobID::Nil(), node_log_key_, node_info_ptr, done);
|
||||||
}
|
}
|
||||||
|
|
||||||
ray::Status ClientTable::MarkDisconnected(const NodeID &dead_node_id,
|
ray::Status NodeTable::MarkDisconnected(const NodeID &dead_node_id,
|
||||||
const WriteCallback &done) {
|
const WriteCallback &done) {
|
||||||
auto node_info = std::make_shared<GcsNodeInfo>();
|
auto node_info = std::make_shared<GcsNodeInfo>();
|
||||||
node_info->set_node_id(dead_node_id.Binary());
|
node_info->set_node_id(dead_node_id.Binary());
|
||||||
node_info->set_state(GcsNodeInfo::DEAD);
|
node_info->set_state(GcsNodeInfo::DEAD);
|
||||||
return Append(JobID::Nil(), client_log_key_, node_info, done);
|
return Append(JobID::Nil(), node_log_key_, node_info, done);
|
||||||
}
|
}
|
||||||
|
|
||||||
ray::Status ClientTable::SubscribeToNodeChange(
|
ray::Status NodeTable::SubscribeToNodeChange(
|
||||||
const SubscribeCallback<NodeID, GcsNodeInfo> &subscribe, const StatusCallback &done) {
|
const SubscribeCallback<NodeID, GcsNodeInfo> &subscribe, const StatusCallback &done) {
|
||||||
// Callback for a notification from the client table.
|
// Callback for a notification from the client table.
|
||||||
auto on_subscribe = [this](RedisGcsClient *client, const UniqueID &log_key,
|
auto on_subscribe = [this](RedisGcsClient *client, const UniqueID &log_key,
|
||||||
const std::vector<GcsNodeInfo> ¬ifications) {
|
const std::vector<GcsNodeInfo> ¬ifications) {
|
||||||
RAY_CHECK(log_key == client_log_key_);
|
RAY_CHECK(log_key == node_log_key_);
|
||||||
std::unordered_map<std::string, GcsNodeInfo> connected_nodes;
|
std::unordered_map<std::string, GcsNodeInfo> connected_nodes;
|
||||||
std::unordered_map<std::string, GcsNodeInfo> disconnected_nodes;
|
std::unordered_map<std::string, GcsNodeInfo> disconnected_nodes;
|
||||||
for (auto ¬ification : notifications) {
|
for (auto ¬ification : notifications) {
|
||||||
|
@ -690,7 +689,7 @@ ray::Status ClientTable::SubscribeToNodeChange(
|
||||||
// Register node change callbacks after RequestNotification finishes.
|
// Register node change callbacks after RequestNotification finishes.
|
||||||
RegisterNodeChangeCallback(subscribe);
|
RegisterNodeChangeCallback(subscribe);
|
||||||
};
|
};
|
||||||
RAY_CHECK_OK(RequestNotifications(JobID::Nil(), client_log_key_, subscribe_id_,
|
RAY_CHECK_OK(RequestNotifications(JobID::Nil(), node_log_key_, subscribe_id_,
|
||||||
on_request_notification_done));
|
on_request_notification_done));
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -698,7 +697,7 @@ ray::Status ClientTable::SubscribeToNodeChange(
|
||||||
return Subscribe(JobID::Nil(), subscribe_id_, on_subscribe, on_done);
|
return Subscribe(JobID::Nil(), subscribe_id_, on_subscribe, on_done);
|
||||||
}
|
}
|
||||||
|
|
||||||
bool ClientTable::GetClient(const NodeID &node_id, GcsNodeInfo *node_info) const {
|
bool NodeTable::GetNode(const NodeID &node_id, GcsNodeInfo *node_info) const {
|
||||||
RAY_CHECK(!node_id.IsNil());
|
RAY_CHECK(!node_id.IsNil());
|
||||||
auto entry = node_cache_.find(node_id);
|
auto entry = node_cache_.find(node_id);
|
||||||
auto found = (entry != node_cache_.end());
|
auto found = (entry != node_cache_.end());
|
||||||
|
@ -708,16 +707,16 @@ bool ClientTable::GetClient(const NodeID &node_id, GcsNodeInfo *node_info) const
|
||||||
return found;
|
return found;
|
||||||
}
|
}
|
||||||
|
|
||||||
const std::unordered_map<NodeID, GcsNodeInfo> &ClientTable::GetAllClients() const {
|
const std::unordered_map<NodeID, GcsNodeInfo> &NodeTable::GetAllNodes() const {
|
||||||
return node_cache_;
|
return node_cache_;
|
||||||
}
|
}
|
||||||
|
|
||||||
Status ClientTable::Lookup(const Callback &lookup) {
|
Status NodeTable::Lookup(const Callback &lookup) {
|
||||||
RAY_CHECK(lookup != nullptr);
|
RAY_CHECK(lookup != nullptr);
|
||||||
return Log::Lookup(JobID::Nil(), client_log_key_, lookup);
|
return Log::Lookup(JobID::Nil(), node_log_key_, lookup);
|
||||||
}
|
}
|
||||||
|
|
||||||
std::string ClientTable::DebugString() const {
|
std::string NodeTable::DebugString() const {
|
||||||
std::stringstream result;
|
std::stringstream result;
|
||||||
result << Log<NodeID, GcsNodeInfo>::DebugString();
|
result << Log<NodeID, GcsNodeInfo>::DebugString();
|
||||||
result << ", cache size: " << node_cache_.size()
|
result << ", cache size: " << node_cache_.size()
|
||||||
|
@ -725,7 +724,7 @@ std::string ClientTable::DebugString() const {
|
||||||
return result.str();
|
return result.str();
|
||||||
}
|
}
|
||||||
|
|
||||||
Status TaskLeaseTable::Subscribe(const JobID &job_id, const NodeID &client_id,
|
Status TaskLeaseTable::Subscribe(const JobID &job_id, const NodeID &node_id,
|
||||||
const Callback &subscribe,
|
const Callback &subscribe,
|
||||||
const SubscriptionCallback &done) {
|
const SubscriptionCallback &done) {
|
||||||
auto on_subscribe = [subscribe](RedisGcsClient *client, const TaskID &task_id,
|
auto on_subscribe = [subscribe](RedisGcsClient *client, const TaskID &task_id,
|
||||||
|
@ -741,7 +740,7 @@ Status TaskLeaseTable::Subscribe(const JobID &job_id, const NodeID &client_id,
|
||||||
}
|
}
|
||||||
subscribe(client, task_id, result);
|
subscribe(client, task_id, result);
|
||||||
};
|
};
|
||||||
return Table<TaskID, TaskLeaseData>::Subscribe(job_id, client_id, on_subscribe, done);
|
return Table<TaskID, TaskLeaseData>::Subscribe(job_id, node_id, on_subscribe, done);
|
||||||
}
|
}
|
||||||
|
|
||||||
std::vector<ActorID> SyncGetAllActorID(redisContext *redis_context,
|
std::vector<ActorID> SyncGetAllActorID(redisContext *redis_context,
|
||||||
|
|
|
@ -71,10 +71,10 @@ template <typename ID>
|
||||||
class PubsubInterface {
|
class PubsubInterface {
|
||||||
public:
|
public:
|
||||||
virtual Status RequestNotifications(const JobID &job_id, const ID &id,
|
virtual Status RequestNotifications(const JobID &job_id, const ID &id,
|
||||||
const NodeID &client_id,
|
const NodeID &node_id,
|
||||||
const StatusCallback &done) = 0;
|
const StatusCallback &done) = 0;
|
||||||
virtual Status CancelNotifications(const JobID &job_id, const ID &id,
|
virtual Status CancelNotifications(const JobID &job_id, const ID &id,
|
||||||
const NodeID &client_id,
|
const NodeID &node_id,
|
||||||
const StatusCallback &done) = 0;
|
const StatusCallback &done) = 0;
|
||||||
virtual ~PubsubInterface(){};
|
virtual ~PubsubInterface(){};
|
||||||
};
|
};
|
||||||
|
@ -100,7 +100,7 @@ class LogInterface {
|
||||||
/// pubsub_channel_ member if pubsub is required.
|
/// pubsub_channel_ member if pubsub is required.
|
||||||
///
|
///
|
||||||
/// Example tables backed by Log:
|
/// Example tables backed by Log:
|
||||||
/// ClientTable: Stores a log of which GCS clients have been added or deleted
|
/// NodeTable: Stores a log of which GCS clients have been added or deleted
|
||||||
/// from the system.
|
/// from the system.
|
||||||
template <typename ID, typename Data>
|
template <typename ID, typename Data>
|
||||||
class Log : public LogInterface<ID, Data>, virtual public PubsubInterface<ID> {
|
class Log : public LogInterface<ID, Data>, virtual public PubsubInterface<ID> {
|
||||||
|
@ -185,18 +185,18 @@ class Log : public LogInterface<ID, Data>, virtual public PubsubInterface<ID> {
|
||||||
/// requests notifications for. This may only be called once per Log
|
/// requests notifications for. This may only be called once per Log
|
||||||
///
|
///
|
||||||
/// \param job_id The ID of the job.
|
/// \param job_id The ID of the job.
|
||||||
/// \param client_id The type of update to listen to. If this is nil, then a
|
/// \param node_id The type of update to listen to. If this is nil, then a
|
||||||
/// message for each Add to the table will be received. Else, only
|
/// message for each Add to the table will be received. Else, only
|
||||||
/// messages for the given client will be received. In the latter
|
/// messages for the given node will be received. In the latter
|
||||||
/// case, the client may request notifications on specific keys in the
|
/// case, the node may request notifications on specific keys in the
|
||||||
/// table via `RequestNotifications`.
|
/// table via `RequestNotifications`.
|
||||||
/// \param subscribe Callback that is called on each received message. If the
|
/// \param subscribe Callback that is called on each received message. If the
|
||||||
/// callback is called with an empty vector, then there was no data at the key.
|
/// callback is called with an empty vector, then there was no data at the key.
|
||||||
/// \param done Callback that is called when subscription is complete and we
|
/// \param done Callback that is called when subscription is complete and we
|
||||||
/// are ready to receive messages.
|
/// are ready to receive messages.
|
||||||
/// \return Status
|
/// \return Status
|
||||||
Status Subscribe(const JobID &job_id, const NodeID &client_id,
|
Status Subscribe(const JobID &job_id, const NodeID &node_id, const Callback &subscribe,
|
||||||
const Callback &subscribe, const SubscriptionCallback &done);
|
const SubscriptionCallback &done);
|
||||||
|
|
||||||
/// Request notifications about a key in this table.
|
/// Request notifications about a key in this table.
|
||||||
///
|
///
|
||||||
|
@ -205,26 +205,26 @@ class Log : public LogInterface<ID, Data>, virtual public PubsubInterface<ID> {
|
||||||
/// the current values at the key, if any, and a subsequent notification will
|
/// the current values at the key, if any, and a subsequent notification will
|
||||||
/// be published for every following `Append` to the key. Before
|
/// be published for every following `Append` to the key. Before
|
||||||
/// notifications can be requested, the caller must first call `Subscribe`,
|
/// notifications can be requested, the caller must first call `Subscribe`,
|
||||||
/// with the same `client_id`.
|
/// with the same `node_id`.
|
||||||
///
|
///
|
||||||
/// \param job_id The ID of the job.
|
/// \param job_id The ID of the job.
|
||||||
/// \param id The ID of the key to request notifications for.
|
/// \param id The ID of the key to request notifications for.
|
||||||
/// \param client_id The client who is requesting notifications. Before
|
/// \param node_id The node who is requesting notifications.
|
||||||
/// \param done Callback that is called when request notifications is complete.
|
/// \param done Callback that is called when request notifications is complete.
|
||||||
/// notifications can be requested, a call to `Subscribe` to this
|
/// notifications can be requested, a call to `Subscribe` to this
|
||||||
/// table with the same `client_id` must complete successfully.
|
/// table with the same `node_id` must complete successfully.
|
||||||
/// \return Status
|
/// \return Status
|
||||||
Status RequestNotifications(const JobID &job_id, const ID &id, const NodeID &client_id,
|
Status RequestNotifications(const JobID &job_id, const ID &id, const NodeID &node_id,
|
||||||
const StatusCallback &done);
|
const StatusCallback &done);
|
||||||
|
|
||||||
/// Cancel notifications about a key in this table.
|
/// Cancel notifications about a key in this table.
|
||||||
///
|
///
|
||||||
/// \param job_id The ID of the job.
|
/// \param job_id The ID of the job.
|
||||||
/// \param id The ID of the key to request notifications for.
|
/// \param id The ID of the key to request notifications for.
|
||||||
/// \param client_id The client who originally requested notifications.
|
/// \param node_id The node who originally requested notifications.
|
||||||
/// \param done Callback that is called when cancel notifications is complete.
|
/// \param done Callback that is called when cancel notifications is complete.
|
||||||
/// \return Status
|
/// \return Status
|
||||||
Status CancelNotifications(const JobID &job_id, const ID &id, const NodeID &client_id,
|
Status CancelNotifications(const JobID &job_id, const ID &id, const NodeID &node_id,
|
||||||
const StatusCallback &done);
|
const StatusCallback &done);
|
||||||
|
|
||||||
/// Subscribe to any modifications to the key. The caller may choose
|
/// Subscribe to any modifications to the key. The caller may choose
|
||||||
|
@ -235,17 +235,17 @@ class Log : public LogInterface<ID, Data>, virtual public PubsubInterface<ID> {
|
||||||
/// function supports notifications of remove operations.
|
/// function supports notifications of remove operations.
|
||||||
///
|
///
|
||||||
/// \param job_id The ID of the job.
|
/// \param job_id The ID of the job.
|
||||||
/// \param client_id The type of update to listen to. If this is nil, then a
|
/// \param node_id The type of update to listen to. If this is nil, then a
|
||||||
/// message for each Add to the table will be received. Else, only
|
/// message for each Add to the table will be received. Else, only
|
||||||
/// messages for the given client will be received. In the latter
|
/// messages for the given node will be received. In the latter
|
||||||
/// case, the client may request notifications on specific keys in the
|
/// case, the node may request notifications on specific keys in the
|
||||||
/// table via `RequestNotifications`.
|
/// table via `RequestNotifications`.
|
||||||
/// \param subscribe Callback that is called on each received message. If the
|
/// \param subscribe Callback that is called on each received message. If the
|
||||||
/// callback is called with an empty vector, then there was no data at the key.
|
/// callback is called with an empty vector, then there was no data at the key.
|
||||||
/// \param done Callback that is called when subscription is complete and we
|
/// \param done Callback that is called when subscription is complete and we
|
||||||
/// are ready to receive messages.
|
/// are ready to receive messages.
|
||||||
/// \return Status
|
/// \return Status
|
||||||
Status Subscribe(const JobID &job_id, const NodeID &client_id,
|
Status Subscribe(const JobID &job_id, const NodeID &node_id,
|
||||||
const NotificationCallback &subscribe,
|
const NotificationCallback &subscribe,
|
||||||
const SubscriptionCallback &done);
|
const SubscriptionCallback &done);
|
||||||
|
|
||||||
|
@ -368,10 +368,10 @@ class Table : private Log<ID, Data>,
|
||||||
/// notifications for. This may only be called once per Table instance.
|
/// notifications for. This may only be called once per Table instance.
|
||||||
///
|
///
|
||||||
/// \param job_id The ID of the job.
|
/// \param job_id The ID of the job.
|
||||||
/// \param client_id The type of update to listen to. If this is nil, then a
|
/// \param node_id The type of update to listen to. If this is nil, then a
|
||||||
/// message for each Add to the table will be received. Else, only
|
/// message for each Add to the table will be received. Else, only
|
||||||
/// messages for the given client will be received. In the latter
|
/// messages for the given node will be received. In the latter
|
||||||
/// case, the client may request notifications on specific keys in the
|
/// case, the node may request notifications on specific keys in the
|
||||||
/// table via `RequestNotifications`.
|
/// table via `RequestNotifications`.
|
||||||
/// \param subscribe Callback that is called on each received message. If the
|
/// \param subscribe Callback that is called on each received message. If the
|
||||||
/// callback is called with an empty vector, then there was no data at the key.
|
/// callback is called with an empty vector, then there was no data at the key.
|
||||||
|
@ -380,27 +380,26 @@ class Table : private Log<ID, Data>,
|
||||||
/// \param done Callback that is called when subscription is complete and we
|
/// \param done Callback that is called when subscription is complete and we
|
||||||
/// are ready to receive messages.
|
/// are ready to receive messages.
|
||||||
/// \return Status
|
/// \return Status
|
||||||
Status Subscribe(const JobID &job_id, const NodeID &client_id,
|
Status Subscribe(const JobID &job_id, const NodeID &node_id, const Callback &subscribe,
|
||||||
const Callback &subscribe, const FailureCallback &failure,
|
const FailureCallback &failure, const SubscriptionCallback &done);
|
||||||
const SubscriptionCallback &done);
|
|
||||||
|
|
||||||
/// Subscribe to any Add operations to this table. The caller may choose to
|
/// Subscribe to any Add operations to this table. The caller may choose to
|
||||||
/// subscribe to all Adds, or to subscribe only to keys that it requests
|
/// subscribe to all Adds, or to subscribe only to keys that it requests
|
||||||
/// notifications for. This may only be called once per Table instance.
|
/// notifications for. This may only be called once per Table instance.
|
||||||
///
|
///
|
||||||
/// \param job_id The ID of the job.
|
/// \param job_id The ID of the job.
|
||||||
/// \param client_id The type of update to listen to. If this is nil, then a
|
/// \param node_id The type of update to listen to. If this is nil, then a
|
||||||
/// message for each Add to the table will be received. Else, only
|
/// message for each Add to the table will be received. Else, only
|
||||||
/// messages for the given client will be received. In the latter
|
/// messages for the given node will be received. In the latter
|
||||||
/// case, the client may request notifications on specific keys in the
|
/// case, the node may request notifications on specific keys in the
|
||||||
/// table via `RequestNotifications`.
|
/// table via `RequestNotifications`.
|
||||||
/// \param subscribe Callback that is called on each received message. If the
|
/// \param subscribe Callback that is called on each received message. If the
|
||||||
/// callback is called with an empty vector, then there was no data at the key.
|
/// callback is called with an empty vector, then there was no data at the key.
|
||||||
/// \param done Callback that is called when subscription is complete and we
|
/// \param done Callback that is called when subscription is complete and we
|
||||||
/// are ready to receive messages.
|
/// are ready to receive messages.
|
||||||
/// \return Status
|
/// \return Status
|
||||||
Status Subscribe(const JobID &job_id, const NodeID &client_id,
|
Status Subscribe(const JobID &job_id, const NodeID &node_id, const Callback &subscribe,
|
||||||
const Callback &subscribe, const SubscriptionCallback &done);
|
const SubscriptionCallback &done);
|
||||||
|
|
||||||
void Delete(const JobID &job_id, const ID &id) { Log<ID, Data>::Delete(job_id, id); }
|
void Delete(const JobID &job_id, const ID &id) { Log<ID, Data>::Delete(job_id, id); }
|
||||||
|
|
||||||
|
@ -490,16 +489,16 @@ class Set : private Log<ID, Data>,
|
||||||
/// Subscribe to any add or remove operations to this table.
|
/// Subscribe to any add or remove operations to this table.
|
||||||
///
|
///
|
||||||
/// \param job_id The ID of the job.
|
/// \param job_id The ID of the job.
|
||||||
/// \param client_id The type of update to listen to. If this is nil, then a
|
/// \param node_id The type of update to listen to. If this is nil, then a
|
||||||
/// message for each add or remove to the table will be received. Else, only
|
/// message for each add or remove to the table will be received. Else, only
|
||||||
/// messages for the given client will be received. In the latter
|
/// messages for the given node will be received. In the latter
|
||||||
/// case, the client may request notifications on specific keys in the
|
/// case, the node may request notifications on specific keys in the
|
||||||
/// table via `RequestNotifications`.
|
/// table via `RequestNotifications`.
|
||||||
/// \param subscribe Callback that is called on each received message.
|
/// \param subscribe Callback that is called on each received message.
|
||||||
/// \param done Callback that is called when subscription is complete and we
|
/// \param done Callback that is called when subscription is complete and we
|
||||||
/// are ready to receive messages.
|
/// are ready to receive messages.
|
||||||
/// \return Status
|
/// \return Status
|
||||||
Status Subscribe(const JobID &job_id, const NodeID &client_id,
|
Status Subscribe(const JobID &job_id, const NodeID &node_id,
|
||||||
const NotificationCallback &subscribe,
|
const NotificationCallback &subscribe,
|
||||||
const SubscriptionCallback &done);
|
const SubscriptionCallback &done);
|
||||||
|
|
||||||
|
@ -591,16 +590,16 @@ class HashInterface {
|
||||||
/// Subscribe to any Update or Remove operations to this hash table.
|
/// Subscribe to any Update or Remove operations to this hash table.
|
||||||
///
|
///
|
||||||
/// \param job_id The ID of the job.
|
/// \param job_id The ID of the job.
|
||||||
/// \param client_id The type of update to listen to. If this is nil, then a
|
/// \param node_id The type of update to listen to. If this is nil, then a
|
||||||
/// message for each Update to the table will be received. Else, only
|
/// message for each Update to the table will be received. Else, only
|
||||||
/// messages for the given client will be received. In the latter
|
/// messages for the given node will be received. In the latter
|
||||||
/// case, the client may request notifications on specific keys in the
|
/// case, the node may request notifications on specific keys in the
|
||||||
/// table via `RequestNotifications`.
|
/// table via `RequestNotifications`.
|
||||||
/// \param subscribe HashNotificationCallback that is called on each received message.
|
/// \param subscribe HashNotificationCallback that is called on each received message.
|
||||||
/// \param done SubscriptionCallback that is called when subscription is complete and
|
/// \param done SubscriptionCallback that is called when subscription is complete and
|
||||||
/// we are ready to receive messages.
|
/// we are ready to receive messages.
|
||||||
/// \return Status
|
/// \return Status
|
||||||
virtual Status Subscribe(const JobID &job_id, const NodeID &client_id,
|
virtual Status Subscribe(const JobID &job_id, const NodeID &node_id,
|
||||||
const HashNotificationCallback &subscribe,
|
const HashNotificationCallback &subscribe,
|
||||||
const SubscriptionCallback &done) = 0;
|
const SubscriptionCallback &done) = 0;
|
||||||
|
|
||||||
|
@ -628,7 +627,7 @@ class Hash : private Log<ID, Data>,
|
||||||
Status Update(const JobID &job_id, const ID &id, const DataMap &pairs,
|
Status Update(const JobID &job_id, const ID &id, const DataMap &pairs,
|
||||||
const HashCallback &done) override;
|
const HashCallback &done) override;
|
||||||
|
|
||||||
Status Subscribe(const JobID &job_id, const NodeID &client_id,
|
Status Subscribe(const JobID &job_id, const NodeID &node_id,
|
||||||
const HashNotificationCallback &subscribe,
|
const HashNotificationCallback &subscribe,
|
||||||
const SubscriptionCallback &done) override;
|
const SubscriptionCallback &done) override;
|
||||||
|
|
||||||
|
@ -807,8 +806,8 @@ class TaskLeaseTable : public Table<TaskID, TaskLeaseData> {
|
||||||
|
|
||||||
/// Implement this method for the subscription tools class SubscriptionExecutor.
|
/// Implement this method for the subscription tools class SubscriptionExecutor.
|
||||||
/// In this way TaskLeaseTable() can also reuse class SubscriptionExecutor.
|
/// In this way TaskLeaseTable() can also reuse class SubscriptionExecutor.
|
||||||
Status Subscribe(const JobID &job_id, const NodeID &client_id,
|
Status Subscribe(const JobID &job_id, const NodeID &node_id, const Callback &subscribe,
|
||||||
const Callback &subscribe, const SubscriptionCallback &done);
|
const SubscriptionCallback &done);
|
||||||
};
|
};
|
||||||
|
|
||||||
class ActorCheckpointTable : public Table<ActorCheckpointID, ActorCheckpointData> {
|
class ActorCheckpointTable : public Table<ActorCheckpointID, ActorCheckpointData> {
|
||||||
|
@ -874,33 +873,33 @@ class ProfileTable : public Log<UniqueID, ProfileTableData> {
|
||||||
std::string DebugString() const;
|
std::string DebugString() const;
|
||||||
};
|
};
|
||||||
|
|
||||||
/// \class ClientTable
|
/// \class NodeTable
|
||||||
///
|
///
|
||||||
/// The ClientTable stores information about active and inactive clients. It is
|
/// The NodeTable stores information about active and inactive nodes. It is
|
||||||
/// structured as a single log stored at a key known to all clients. When a
|
/// structured as a single log stored at a key known to all nodes. When a
|
||||||
/// client connects, it appends an entry to the log indicating that it is
|
/// node connects, it appends an entry to the log indicating that it is
|
||||||
/// alive. When a client disconnects, or if another client detects its failure,
|
/// alive. When a node disconnects, or if another node detects its failure,
|
||||||
/// it should append an entry to the log indicating that it is dead. A client
|
/// it should append an entry to the log indicating that it is dead. A node
|
||||||
/// that is marked as dead should never again be marked as alive; if it needs
|
/// that is marked as dead should never again be marked as alive; if it needs
|
||||||
/// to reconnect, it must connect with a different NodeID.
|
/// to reconnect, it must connect with a different NodeID.
|
||||||
class ClientTable : public Log<NodeID, GcsNodeInfo> {
|
class NodeTable : public Log<NodeID, GcsNodeInfo> {
|
||||||
public:
|
public:
|
||||||
ClientTable(const std::vector<std::shared_ptr<RedisContext>> &contexts,
|
NodeTable(const std::vector<std::shared_ptr<RedisContext>> &contexts,
|
||||||
RedisGcsClient *client)
|
RedisGcsClient *client)
|
||||||
: Log(contexts, client) {
|
: Log(contexts, client) {
|
||||||
pubsub_channel_ = TablePubsub::CLIENT_PUBSUB;
|
pubsub_channel_ = TablePubsub::NODE_PUBSUB;
|
||||||
prefix_ = TablePrefix::CLIENT;
|
prefix_ = TablePrefix::NODE;
|
||||||
};
|
};
|
||||||
|
|
||||||
/// Connect as a client to the GCS. This registers us in the client table
|
/// Connect as a NODE to the GCS. This registers us in the NODE table
|
||||||
/// and begins subscription to client table notifications.
|
/// and begins subscription to NODE table notifications.
|
||||||
///
|
///
|
||||||
/// \param local_node_info Information about the connecting client. This must have the
|
/// \param local_node_info Information about the connecting NODE. This must have the
|
||||||
/// same id as the one set in the client table.
|
/// same id as the one set in the NODE table.
|
||||||
/// \return Status
|
/// \return Status
|
||||||
ray::Status Connect(const GcsNodeInfo &local_node_info);
|
ray::Status Connect(const GcsNodeInfo &local_node_info);
|
||||||
|
|
||||||
/// Disconnect the client from the GCS. The client ID assigned during
|
/// Disconnect the NODE from the GCS. The NODE ID assigned during
|
||||||
/// registration should never be reused after disconnecting.
|
/// registration should never be reused after disconnecting.
|
||||||
///
|
///
|
||||||
/// \return Status
|
/// \return Status
|
||||||
|
@ -913,7 +912,7 @@ class ClientTable : public Log<NodeID, GcsNodeInfo> {
|
||||||
/// \return Status
|
/// \return Status
|
||||||
ray::Status MarkConnected(const GcsNodeInfo &node_info, const WriteCallback &done);
|
ray::Status MarkConnected(const GcsNodeInfo &node_info, const WriteCallback &done);
|
||||||
|
|
||||||
/// Mark a different node as disconnected. The client ID should never be
|
/// Mark a different node as disconnected. The NODE ID should never be
|
||||||
/// reused for a new node.
|
/// reused for a new node.
|
||||||
///
|
///
|
||||||
/// \param dead_node_id The ID of the node to mark as dead.
|
/// \param dead_node_id The ID of the node to mark as dead.
|
||||||
|
@ -926,38 +925,38 @@ class ClientTable : public Log<NodeID, GcsNodeInfo> {
|
||||||
const SubscribeCallback<NodeID, GcsNodeInfo> &subscribe,
|
const SubscribeCallback<NodeID, GcsNodeInfo> &subscribe,
|
||||||
const StatusCallback &done);
|
const StatusCallback &done);
|
||||||
|
|
||||||
/// Get a client's information from the cache. The cache only contains
|
/// Get a node's information from the cache. The cache only contains
|
||||||
/// information for clients that we've heard a notification for.
|
/// information for nodes that we've heard a notification for.
|
||||||
///
|
///
|
||||||
/// \param client The client to get information about.
|
/// \param node The node to get information about.
|
||||||
/// \param node_info The client information will be copied here if
|
/// \param node_info The node information will be copied here if
|
||||||
/// we have the client in the cache.
|
/// we have the node in the cache.
|
||||||
/// a nil client ID.
|
/// a nil node ID.
|
||||||
/// \return Whether teh client is in the cache.
|
/// \return Whether the node is in the cache.
|
||||||
bool GetClient(const NodeID &client, GcsNodeInfo *node_info) const;
|
bool GetNode(const NodeID &node, GcsNodeInfo *node_info) const;
|
||||||
|
|
||||||
/// Get the local client's ID.
|
/// Get the local node's ID.
|
||||||
///
|
///
|
||||||
/// \return The local client's ID.
|
/// \return The local node's ID.
|
||||||
const NodeID &GetLocalClientId() const;
|
const NodeID &GetLocalNodeId() const;
|
||||||
|
|
||||||
/// Get the local client's information.
|
/// Get the local node's information.
|
||||||
///
|
///
|
||||||
/// \return The local client's information.
|
/// \return The local node's information.
|
||||||
const GcsNodeInfo &GetLocalClient() const;
|
const GcsNodeInfo &GetLocalNode() const;
|
||||||
|
|
||||||
/// Check whether the given client is removed.
|
/// Check whether the given node is removed.
|
||||||
///
|
///
|
||||||
/// \param node_id The ID of the client to check.
|
/// \param node_id The ID of the node to check.
|
||||||
/// \return Whether the client with ID client_id is removed.
|
/// \return Whether the node with specified ID is removed.
|
||||||
bool IsRemoved(const NodeID &node_id) const;
|
bool IsRemoved(const NodeID &node_id) const;
|
||||||
|
|
||||||
/// Get the information of all clients.
|
/// Get the information of all nodes.
|
||||||
///
|
///
|
||||||
/// \return The client ID to client information map.
|
/// \return The node ID to node information map.
|
||||||
const std::unordered_map<NodeID, GcsNodeInfo> &GetAllClients() const;
|
const std::unordered_map<NodeID, GcsNodeInfo> &GetAllNodes() const;
|
||||||
|
|
||||||
/// Lookup the client data in the client table.
|
/// Lookup the node data in the node table.
|
||||||
///
|
///
|
||||||
/// \param lookup Callback that is called after lookup. If the callback is
|
/// \param lookup Callback that is called after lookup. If the callback is
|
||||||
/// called with an empty vector, then there was no data at the key.
|
/// called with an empty vector, then there was no data at the key.
|
||||||
|
@ -969,10 +968,10 @@ class ClientTable : public Log<NodeID, GcsNodeInfo> {
|
||||||
/// \return string.
|
/// \return string.
|
||||||
std::string DebugString() const;
|
std::string DebugString() const;
|
||||||
|
|
||||||
/// The key at which the log of client information is stored. This key must
|
/// The key at which the log of node information is stored. This key must
|
||||||
/// be kept the same across all instances of the ClientTable, so that all
|
/// be kept the same across all instances of the NodeTable, so that all
|
||||||
/// clients append and read from the same key.
|
/// nodes append and read from the same key.
|
||||||
NodeID client_log_key_;
|
NodeID node_log_key_;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
using NodeChangeCallback =
|
using NodeChangeCallback =
|
||||||
|
@ -983,10 +982,10 @@ class ClientTable : public Log<NodeID, GcsNodeInfo> {
|
||||||
/// \param callback The callback to register.
|
/// \param callback The callback to register.
|
||||||
void RegisterNodeChangeCallback(const NodeChangeCallback &callback);
|
void RegisterNodeChangeCallback(const NodeChangeCallback &callback);
|
||||||
|
|
||||||
/// Handle a client table notification.
|
/// Handle a node table notification.
|
||||||
void HandleNotification(RedisGcsClient *client, const GcsNodeInfo &node_info);
|
void HandleNotification(RedisGcsClient *client, const GcsNodeInfo &node_info);
|
||||||
|
|
||||||
/// Whether this client has called Disconnect().
|
/// Whether this node has called Disconnect().
|
||||||
bool disconnected_{false};
|
bool disconnected_{false};
|
||||||
/// This node's ID. It will be initialized when we call method `Connect(...)`.
|
/// This node's ID. It will be initialized when we call method `Connect(...)`.
|
||||||
NodeID local_node_id_;
|
NodeID local_node_id_;
|
||||||
|
|
|
@ -70,7 +70,7 @@ class TestGcs : public ::testing::Test {
|
||||||
};
|
};
|
||||||
|
|
||||||
TestGcs *test;
|
TestGcs *test;
|
||||||
NodeID local_client_id = NodeID::FromRandom();
|
NodeID local_node_id = NodeID::FromRandom();
|
||||||
|
|
||||||
class TestGcsWithAsio : public TestGcs {
|
class TestGcsWithAsio : public TestGcs {
|
||||||
public:
|
public:
|
||||||
|
@ -272,7 +272,7 @@ class TaskTableTestHelper {
|
||||||
num_modifications](gcs::RedisGcsClient *client) {
|
num_modifications](gcs::RedisGcsClient *client) {
|
||||||
// Request notifications for one of the keys.
|
// Request notifications for one of the keys.
|
||||||
RAY_CHECK_OK(client->raylet_task_table().RequestNotifications(
|
RAY_CHECK_OK(client->raylet_task_table().RequestNotifications(
|
||||||
job_id, task_id2, local_client_id, nullptr));
|
job_id, task_id2, local_node_id, nullptr));
|
||||||
// Write both keys. We should only receive notifications for the key that
|
// Write both keys. We should only receive notifications for the key that
|
||||||
// we requested them for.
|
// we requested them for.
|
||||||
for (uint64_t i = 0; i < num_modifications; i++) {
|
for (uint64_t i = 0; i < num_modifications; i++) {
|
||||||
|
@ -288,7 +288,7 @@ class TaskTableTestHelper {
|
||||||
// Subscribe to notifications for this client. This allows us to request and
|
// Subscribe to notifications for this client. This allows us to request and
|
||||||
// receive notifications for specific keys.
|
// receive notifications for specific keys.
|
||||||
RAY_CHECK_OK(client->raylet_task_table().Subscribe(
|
RAY_CHECK_OK(client->raylet_task_table().Subscribe(
|
||||||
job_id, local_client_id, notification_callback, failure_callback,
|
job_id, local_node_id, notification_callback, failure_callback,
|
||||||
subscribe_callback));
|
subscribe_callback));
|
||||||
// Run the event loop. The loop will only stop if the registered subscription
|
// Run the event loop. The loop will only stop if the registered subscription
|
||||||
// callback is called for the requested key.
|
// callback is called for the requested key.
|
||||||
|
@ -342,9 +342,9 @@ class TaskTableTestHelper {
|
||||||
// Request notifications, then cancel immediately. We should receive a
|
// Request notifications, then cancel immediately. We should receive a
|
||||||
// notification for the current value at the key.
|
// notification for the current value at the key.
|
||||||
RAY_CHECK_OK(client->raylet_task_table().RequestNotifications(
|
RAY_CHECK_OK(client->raylet_task_table().RequestNotifications(
|
||||||
job_id, task_id, local_client_id, nullptr));
|
job_id, task_id, local_node_id, nullptr));
|
||||||
RAY_CHECK_OK(client->raylet_task_table().CancelNotifications(
|
RAY_CHECK_OK(client->raylet_task_table().CancelNotifications(
|
||||||
job_id, task_id, local_client_id, nullptr));
|
job_id, task_id, local_node_id, nullptr));
|
||||||
// Write to the key. Since we canceled notifications, we should not receive
|
// Write to the key. Since we canceled notifications, we should not receive
|
||||||
// a notification for these writes.
|
// a notification for these writes.
|
||||||
for (uint64_t i = 1; i < num_modifications; i++) {
|
for (uint64_t i = 1; i < num_modifications; i++) {
|
||||||
|
@ -354,13 +354,13 @@ class TaskTableTestHelper {
|
||||||
// Request notifications again. We should receive a notification for the
|
// Request notifications again. We should receive a notification for the
|
||||||
// current value at the key.
|
// current value at the key.
|
||||||
RAY_CHECK_OK(client->raylet_task_table().RequestNotifications(
|
RAY_CHECK_OK(client->raylet_task_table().RequestNotifications(
|
||||||
job_id, task_id, local_client_id, nullptr));
|
job_id, task_id, local_node_id, nullptr));
|
||||||
};
|
};
|
||||||
|
|
||||||
// Subscribe to notifications for this client. This allows us to request and
|
// Subscribe to notifications for this client. This allows us to request and
|
||||||
// receive notifications for specific keys.
|
// receive notifications for specific keys.
|
||||||
RAY_CHECK_OK(client->raylet_task_table().Subscribe(
|
RAY_CHECK_OK(client->raylet_task_table().Subscribe(
|
||||||
job_id, local_client_id, notification_callback, failure_callback,
|
job_id, local_node_id, notification_callback, failure_callback,
|
||||||
subscribe_callback));
|
subscribe_callback));
|
||||||
// Run the event loop. The loop will only stop if the registered subscription
|
// Run the event loop. The loop will only stop if the registered subscription
|
||||||
// callback is called for the requested key.
|
// callback is called for the requested key.
|
||||||
|
@ -716,7 +716,7 @@ class SetTestHelper {
|
||||||
managers2](gcs::RedisGcsClient *client) {
|
managers2](gcs::RedisGcsClient *client) {
|
||||||
// Request notifications for one of the keys.
|
// Request notifications for one of the keys.
|
||||||
RAY_CHECK_OK(client->object_table().RequestNotifications(job_id, object_id2,
|
RAY_CHECK_OK(client->object_table().RequestNotifications(job_id, object_id2,
|
||||||
local_client_id, nullptr));
|
local_node_id, nullptr));
|
||||||
// Write both keys. We should only receive notifications for the key that
|
// Write both keys. We should only receive notifications for the key that
|
||||||
// we requested them for.
|
// we requested them for.
|
||||||
auto remaining = std::vector<std::string>(++managers1.begin(), managers1.end());
|
auto remaining = std::vector<std::string>(++managers1.begin(), managers1.end());
|
||||||
|
@ -736,7 +736,7 @@ class SetTestHelper {
|
||||||
// Subscribe to notifications for this client. This allows us to request and
|
// Subscribe to notifications for this client. This allows us to request and
|
||||||
// receive notifications for specific keys.
|
// receive notifications for specific keys.
|
||||||
RAY_CHECK_OK(client->object_table().Subscribe(
|
RAY_CHECK_OK(client->object_table().Subscribe(
|
||||||
job_id, local_client_id, notification_callback, subscribe_callback));
|
job_id, local_node_id, notification_callback, subscribe_callback));
|
||||||
// Run the event loop. The loop will only stop if the registered subscription
|
// Run the event loop. The loop will only stop if the registered subscription
|
||||||
// callback is called for the requested key.
|
// callback is called for the requested key.
|
||||||
test->Start();
|
test->Start();
|
||||||
|
@ -793,9 +793,9 @@ class SetTestHelper {
|
||||||
// Request notifications, then cancel immediately. We should receive a
|
// Request notifications, then cancel immediately. We should receive a
|
||||||
// notification for the current value at the key.
|
// notification for the current value at the key.
|
||||||
RAY_CHECK_OK(client->object_table().RequestNotifications(job_id, object_id,
|
RAY_CHECK_OK(client->object_table().RequestNotifications(job_id, object_id,
|
||||||
local_client_id, nullptr));
|
local_node_id, nullptr));
|
||||||
RAY_CHECK_OK(client->object_table().CancelNotifications(job_id, object_id,
|
RAY_CHECK_OK(client->object_table().CancelNotifications(job_id, object_id,
|
||||||
local_client_id, nullptr));
|
local_node_id, nullptr));
|
||||||
// Add to the key. Since we canceled notifications, we should not
|
// Add to the key. Since we canceled notifications, we should not
|
||||||
// receive a notification for these writes.
|
// receive a notification for these writes.
|
||||||
auto remaining = std::vector<std::string>(++managers.begin(), managers.end());
|
auto remaining = std::vector<std::string>(++managers.begin(), managers.end());
|
||||||
|
@ -807,13 +807,13 @@ class SetTestHelper {
|
||||||
// Request notifications again. We should receive a notification for the
|
// Request notifications again. We should receive a notification for the
|
||||||
// current values at the key.
|
// current values at the key.
|
||||||
RAY_CHECK_OK(client->object_table().RequestNotifications(job_id, object_id,
|
RAY_CHECK_OK(client->object_table().RequestNotifications(job_id, object_id,
|
||||||
local_client_id, nullptr));
|
local_node_id, nullptr));
|
||||||
};
|
};
|
||||||
|
|
||||||
// Subscribe to notifications for this client. This allows us to request and
|
// Subscribe to notifications for this client. This allows us to request and
|
||||||
// receive notifications for specific keys.
|
// receive notifications for specific keys.
|
||||||
RAY_CHECK_OK(client->object_table().Subscribe(
|
RAY_CHECK_OK(client->object_table().Subscribe(
|
||||||
job_id, local_client_id, notification_callback, subscribe_callback));
|
job_id, local_node_id, notification_callback, subscribe_callback));
|
||||||
// Run the event loop. The loop will only stop if the registered subscription
|
// Run the event loop. The loop will only stop if the registered subscription
|
||||||
// callback is called for the requested key.
|
// callback is called for the requested key.
|
||||||
test->Start();
|
test->Start();
|
||||||
|
@ -1053,7 +1053,7 @@ class LogSubscribeTestHelper {
|
||||||
job_ids2](gcs::RedisGcsClient *client) {
|
job_ids2](gcs::RedisGcsClient *client) {
|
||||||
// Request notifications for one of the keys.
|
// Request notifications for one of the keys.
|
||||||
RAY_CHECK_OK(client->job_table().RequestNotifications(job_id, job_id2,
|
RAY_CHECK_OK(client->job_table().RequestNotifications(job_id, job_id2,
|
||||||
local_client_id, nullptr));
|
local_node_id, nullptr));
|
||||||
// Write both keys. We should only receive notifications for the key that
|
// Write both keys. We should only receive notifications for the key that
|
||||||
// we requested them for.
|
// we requested them for.
|
||||||
auto remaining = std::vector<std::string>(++job_ids1.begin(), job_ids1.end());
|
auto remaining = std::vector<std::string>(++job_ids1.begin(), job_ids1.end());
|
||||||
|
@ -1073,7 +1073,7 @@ class LogSubscribeTestHelper {
|
||||||
// Subscribe to notifications for this client. This allows us to request and
|
// Subscribe to notifications for this client. This allows us to request and
|
||||||
// receive notifications for specific keys.
|
// receive notifications for specific keys.
|
||||||
RAY_CHECK_OK(client->job_table().Subscribe(
|
RAY_CHECK_OK(client->job_table().Subscribe(
|
||||||
job_id, local_client_id, notification_callback, subscribe_callback));
|
job_id, local_node_id, notification_callback, subscribe_callback));
|
||||||
// Run the event loop. The loop will only stop if the registered subscription
|
// Run the event loop. The loop will only stop if the registered subscription
|
||||||
// callback is called for the requested key.
|
// callback is called for the requested key.
|
||||||
test->Start();
|
test->Start();
|
||||||
|
@ -1118,9 +1118,9 @@ class LogSubscribeTestHelper {
|
||||||
// Request notifications, then cancel immediately. We should receive a
|
// Request notifications, then cancel immediately. We should receive a
|
||||||
// notification for the current value at the key.
|
// notification for the current value at the key.
|
||||||
RAY_CHECK_OK(client->job_table().RequestNotifications(job_id, random_job_id,
|
RAY_CHECK_OK(client->job_table().RequestNotifications(job_id, random_job_id,
|
||||||
local_client_id, nullptr));
|
local_node_id, nullptr));
|
||||||
RAY_CHECK_OK(client->job_table().CancelNotifications(job_id, random_job_id,
|
RAY_CHECK_OK(client->job_table().CancelNotifications(job_id, random_job_id,
|
||||||
local_client_id, nullptr));
|
local_node_id, nullptr));
|
||||||
// Append to the key. Since we canceled notifications, we should not
|
// Append to the key. Since we canceled notifications, we should not
|
||||||
// receive a notification for these writes.
|
// receive a notification for these writes.
|
||||||
auto remaining = std::vector<std::string>(++job_ids.begin(), job_ids.end());
|
auto remaining = std::vector<std::string>(++job_ids.begin(), job_ids.end());
|
||||||
|
@ -1132,13 +1132,13 @@ class LogSubscribeTestHelper {
|
||||||
// Request notifications again. We should receive a notification for the
|
// Request notifications again. We should receive a notification for the
|
||||||
// current values at the key.
|
// current values at the key.
|
||||||
RAY_CHECK_OK(client->job_table().RequestNotifications(job_id, random_job_id,
|
RAY_CHECK_OK(client->job_table().RequestNotifications(job_id, random_job_id,
|
||||||
local_client_id, nullptr));
|
local_node_id, nullptr));
|
||||||
};
|
};
|
||||||
|
|
||||||
// Subscribe to notifications for this client. This allows us to request and
|
// Subscribe to notifications for this client. This allows us to request and
|
||||||
// receive notifications for specific keys.
|
// receive notifications for specific keys.
|
||||||
RAY_CHECK_OK(client->job_table().Subscribe(
|
RAY_CHECK_OK(client->job_table().Subscribe(
|
||||||
job_id, local_client_id, notification_callback, subscribe_callback));
|
job_id, local_node_id, notification_callback, subscribe_callback));
|
||||||
// Run the event loop. The loop will only stop if the registered subscription
|
// Run the event loop. The loop will only stop if the registered subscription
|
||||||
// callback is called for the requested key.
|
// callback is called for the requested key.
|
||||||
test->Start();
|
test->Start();
|
||||||
|
@ -1183,122 +1183,122 @@ TEST_F(TestGcsWithAsio, TestSetSubscribeCancel) {
|
||||||
SetTestHelper::TestSetSubscribeCancel(job_id_, client_);
|
SetTestHelper::TestSetSubscribeCancel(job_id_, client_);
|
||||||
}
|
}
|
||||||
|
|
||||||
/// A helper class for ClientTable testing.
|
/// A helper class for NodeTable testing.
|
||||||
class ClientTableTestHelper {
|
class NodeTableTestHelper {
|
||||||
public:
|
public:
|
||||||
static void ClientTableNotification(std::shared_ptr<gcs::RedisGcsClient> client,
|
static void NodeTableNotification(std::shared_ptr<gcs::RedisGcsClient> client,
|
||||||
const NodeID &client_id, const GcsNodeInfo &data,
|
const NodeID &node_id, const GcsNodeInfo &data,
|
||||||
bool is_alive) {
|
bool is_alive) {
|
||||||
NodeID added_id = local_client_id;
|
NodeID added_id = local_node_id;
|
||||||
ASSERT_EQ(client_id, added_id);
|
ASSERT_EQ(node_id, added_id);
|
||||||
ASSERT_EQ(NodeID::FromBinary(data.node_id()), added_id);
|
ASSERT_EQ(NodeID::FromBinary(data.node_id()), added_id);
|
||||||
ASSERT_EQ(data.state() == GcsNodeInfo::ALIVE, is_alive);
|
ASSERT_EQ(data.state() == GcsNodeInfo::ALIVE, is_alive);
|
||||||
|
|
||||||
GcsNodeInfo cached_client;
|
GcsNodeInfo cached_node;
|
||||||
ASSERT_TRUE(client->client_table().GetClient(added_id, &cached_client));
|
ASSERT_TRUE(client->node_table().GetNode(added_id, &cached_node));
|
||||||
ASSERT_EQ(NodeID::FromBinary(cached_client.node_id()), added_id);
|
ASSERT_EQ(NodeID::FromBinary(cached_node.node_id()), added_id);
|
||||||
ASSERT_EQ(cached_client.state() == GcsNodeInfo::ALIVE, is_alive);
|
ASSERT_EQ(cached_node.state() == GcsNodeInfo::ALIVE, is_alive);
|
||||||
}
|
}
|
||||||
|
|
||||||
static void TestClientTableConnect(const JobID &job_id,
|
static void TestNodeTableConnect(const JobID &job_id,
|
||||||
std::shared_ptr<gcs::RedisGcsClient> client) {
|
std::shared_ptr<gcs::RedisGcsClient> client) {
|
||||||
// Subscribe to a node gets added and removed. The latter
|
// Subscribe to a node gets added and removed. The latter
|
||||||
// event will stop the event loop.
|
// event will stop the event loop.
|
||||||
RAY_CHECK_OK(client->client_table().SubscribeToNodeChange(
|
RAY_CHECK_OK(client->node_table().SubscribeToNodeChange(
|
||||||
[client](const NodeID &id, const GcsNodeInfo &data) {
|
[client](const NodeID &id, const GcsNodeInfo &data) {
|
||||||
// TODO(micafan)
|
// TODO(micafan)
|
||||||
RAY_LOG(INFO) << "Test alive=" << data.state() << " id=" << id;
|
RAY_LOG(INFO) << "Test alive=" << data.state() << " id=" << id;
|
||||||
if (data.state() == GcsNodeInfo::ALIVE) {
|
if (data.state() == GcsNodeInfo::ALIVE) {
|
||||||
ClientTableNotification(client, id, data, true);
|
NodeTableNotification(client, id, data, true);
|
||||||
test->Stop();
|
test->Stop();
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
nullptr));
|
nullptr));
|
||||||
|
|
||||||
// Connect and disconnect to client table. We should receive notifications
|
// Connect and disconnect to node table. We should receive notifications
|
||||||
// for the addition and removal of our own entry.
|
// for the addition and removal of our own entry.
|
||||||
GcsNodeInfo local_node_info;
|
GcsNodeInfo local_node_info;
|
||||||
local_node_info.set_node_id(local_client_id.Binary());
|
local_node_info.set_node_id(local_node_id.Binary());
|
||||||
local_node_info.set_node_manager_address("127.0.0.1");
|
local_node_info.set_node_manager_address("127.0.0.1");
|
||||||
local_node_info.set_node_manager_port(0);
|
local_node_info.set_node_manager_port(0);
|
||||||
local_node_info.set_object_manager_port(0);
|
local_node_info.set_object_manager_port(0);
|
||||||
RAY_CHECK_OK(client->client_table().Connect(local_node_info));
|
RAY_CHECK_OK(client->node_table().Connect(local_node_info));
|
||||||
test->Start();
|
test->Start();
|
||||||
}
|
}
|
||||||
|
|
||||||
static void TestClientTableDisconnect(const JobID &job_id,
|
static void TestNodeTableDisconnect(const JobID &job_id,
|
||||||
std::shared_ptr<gcs::RedisGcsClient> client) {
|
std::shared_ptr<gcs::RedisGcsClient> client) {
|
||||||
// Register callbacks for when a client gets added and removed. The latter
|
// Register callbacks for when a node gets added and removed. The latter
|
||||||
// event will stop the event loop.
|
// event will stop the event loop.
|
||||||
RAY_CHECK_OK(client->client_table().SubscribeToNodeChange(
|
RAY_CHECK_OK(client->node_table().SubscribeToNodeChange(
|
||||||
[client](const NodeID &id, const GcsNodeInfo &data) {
|
[client](const NodeID &id, const GcsNodeInfo &data) {
|
||||||
if (data.state() == GcsNodeInfo::ALIVE) {
|
if (data.state() == GcsNodeInfo::ALIVE) {
|
||||||
ClientTableNotification(client, id, data, /*is_insertion=*/true);
|
NodeTableNotification(client, id, data, /*is_insertion=*/true);
|
||||||
// Disconnect from the client table. We should receive a notification
|
// Disconnect from the node table. We should receive a notification
|
||||||
// for the removal of our own entry.
|
// for the removal of our own entry.
|
||||||
RAY_CHECK_OK(client->client_table().Disconnect());
|
RAY_CHECK_OK(client->node_table().Disconnect());
|
||||||
} else {
|
} else {
|
||||||
ClientTableNotification(client, id, data, /*is_insertion=*/false);
|
NodeTableNotification(client, id, data, /*is_insertion=*/false);
|
||||||
test->Stop();
|
test->Stop();
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
nullptr));
|
nullptr));
|
||||||
|
|
||||||
// Connect to the client table. We should receive notification for the
|
// Connect to the node table. We should receive notification for the
|
||||||
// addition of our own entry.
|
// addition of our own entry.
|
||||||
GcsNodeInfo local_node_info;
|
GcsNodeInfo local_node_info;
|
||||||
local_node_info.set_node_id(local_client_id.Binary());
|
local_node_info.set_node_id(local_node_id.Binary());
|
||||||
local_node_info.set_node_manager_address("127.0.0.1");
|
local_node_info.set_node_manager_address("127.0.0.1");
|
||||||
local_node_info.set_node_manager_port(0);
|
local_node_info.set_node_manager_port(0);
|
||||||
local_node_info.set_object_manager_port(0);
|
local_node_info.set_object_manager_port(0);
|
||||||
RAY_CHECK_OK(client->client_table().Connect(local_node_info));
|
RAY_CHECK_OK(client->node_table().Connect(local_node_info));
|
||||||
test->Start();
|
test->Start();
|
||||||
}
|
}
|
||||||
|
|
||||||
static void TestClientTableImmediateDisconnect(
|
static void TestNodeTableImmediateDisconnect(
|
||||||
const JobID &job_id, std::shared_ptr<gcs::RedisGcsClient> client) {
|
const JobID &job_id, std::shared_ptr<gcs::RedisGcsClient> client) {
|
||||||
// Register callbacks for when a client gets added and removed. The latter
|
// Register callbacks for when a node gets added and removed. The latter
|
||||||
// event will stop the event loop.
|
// event will stop the event loop.
|
||||||
RAY_CHECK_OK(client->client_table().SubscribeToNodeChange(
|
RAY_CHECK_OK(client->node_table().SubscribeToNodeChange(
|
||||||
[client](const NodeID &id, const GcsNodeInfo &data) {
|
[client](const NodeID &id, const GcsNodeInfo &data) {
|
||||||
if (data.state() == GcsNodeInfo::ALIVE) {
|
if (data.state() == GcsNodeInfo::ALIVE) {
|
||||||
ClientTableNotification(client, id, data, true);
|
NodeTableNotification(client, id, data, true);
|
||||||
} else {
|
} else {
|
||||||
ClientTableNotification(client, id, data, false);
|
NodeTableNotification(client, id, data, false);
|
||||||
test->Stop();
|
test->Stop();
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
nullptr));
|
nullptr));
|
||||||
// Connect to then immediately disconnect from the client table. We should
|
// Connect to then immediately disconnect from the node table. We should
|
||||||
// receive notifications for the addition and removal of our own entry.
|
// receive notifications for the addition and removal of our own entry.
|
||||||
GcsNodeInfo local_node_info;
|
GcsNodeInfo local_node_info;
|
||||||
local_node_info.set_node_id(local_client_id.Binary());
|
local_node_info.set_node_id(local_node_id.Binary());
|
||||||
local_node_info.set_node_manager_address("127.0.0.1");
|
local_node_info.set_node_manager_address("127.0.0.1");
|
||||||
local_node_info.set_node_manager_port(0);
|
local_node_info.set_node_manager_port(0);
|
||||||
local_node_info.set_object_manager_port(0);
|
local_node_info.set_object_manager_port(0);
|
||||||
RAY_CHECK_OK(client->client_table().Connect(local_node_info));
|
RAY_CHECK_OK(client->node_table().Connect(local_node_info));
|
||||||
RAY_CHECK_OK(client->client_table().Disconnect());
|
RAY_CHECK_OK(client->node_table().Disconnect());
|
||||||
test->Start();
|
test->Start();
|
||||||
}
|
}
|
||||||
|
|
||||||
static void TestClientTableMarkDisconnected(
|
static void TestNodeTableMarkDisconnected(const JobID &job_id,
|
||||||
const JobID &job_id, std::shared_ptr<gcs::RedisGcsClient> client) {
|
std::shared_ptr<gcs::RedisGcsClient> client) {
|
||||||
GcsNodeInfo local_node_info;
|
GcsNodeInfo local_node_info;
|
||||||
local_node_info.set_node_id(local_client_id.Binary());
|
local_node_info.set_node_id(local_node_id.Binary());
|
||||||
local_node_info.set_node_manager_address("127.0.0.1");
|
local_node_info.set_node_manager_address("127.0.0.1");
|
||||||
local_node_info.set_node_manager_port(0);
|
local_node_info.set_node_manager_port(0);
|
||||||
local_node_info.set_object_manager_port(0);
|
local_node_info.set_object_manager_port(0);
|
||||||
// Connect to the client table to start receiving notifications.
|
// Connect to the node table to start receiving notifications.
|
||||||
RAY_CHECK_OK(client->client_table().Connect(local_node_info));
|
RAY_CHECK_OK(client->node_table().Connect(local_node_info));
|
||||||
// Mark a different client as dead.
|
// Mark a different node as dead.
|
||||||
NodeID dead_client_id = NodeID::FromRandom();
|
NodeID dead_node_id = NodeID::FromRandom();
|
||||||
RAY_CHECK_OK(client->client_table().MarkDisconnected(dead_client_id, nullptr));
|
RAY_CHECK_OK(client->node_table().MarkDisconnected(dead_node_id, nullptr));
|
||||||
// Make sure we only get a notification for the removal of the client we
|
// Make sure we only get a notification for the removal of the node we
|
||||||
// marked as dead.
|
// marked as dead.
|
||||||
RAY_CHECK_OK(client->client_table().SubscribeToNodeChange(
|
RAY_CHECK_OK(client->node_table().SubscribeToNodeChange(
|
||||||
[dead_client_id](const UniqueID &id, const GcsNodeInfo &data) {
|
[dead_node_id](const UniqueID &id, const GcsNodeInfo &data) {
|
||||||
if (data.state() == GcsNodeInfo::DEAD) {
|
if (data.state() == GcsNodeInfo::DEAD) {
|
||||||
ASSERT_EQ(NodeID::FromBinary(data.node_id()), dead_client_id);
|
ASSERT_EQ(NodeID::FromBinary(data.node_id()), dead_node_id);
|
||||||
test->Stop();
|
test->Stop();
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
|
@ -1307,24 +1307,24 @@ class ClientTableTestHelper {
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
TEST_F(TestGcsWithAsio, TestClientTableConnect) {
|
TEST_F(TestGcsWithAsio, TestNodeTableConnect) {
|
||||||
test = this;
|
test = this;
|
||||||
ClientTableTestHelper::TestClientTableConnect(job_id_, client_);
|
NodeTableTestHelper::TestNodeTableConnect(job_id_, client_);
|
||||||
}
|
}
|
||||||
|
|
||||||
TEST_F(TestGcsWithAsio, TestClientTableDisconnect) {
|
TEST_F(TestGcsWithAsio, TestNodeTableDisconnect) {
|
||||||
test = this;
|
test = this;
|
||||||
ClientTableTestHelper::TestClientTableDisconnect(job_id_, client_);
|
NodeTableTestHelper::TestNodeTableDisconnect(job_id_, client_);
|
||||||
}
|
}
|
||||||
|
|
||||||
TEST_F(TestGcsWithAsio, TestClientTableImmediateDisconnect) {
|
TEST_F(TestGcsWithAsio, TestNodeTableImmediateDisconnect) {
|
||||||
test = this;
|
test = this;
|
||||||
ClientTableTestHelper::TestClientTableImmediateDisconnect(job_id_, client_);
|
NodeTableTestHelper::TestNodeTableImmediateDisconnect(job_id_, client_);
|
||||||
}
|
}
|
||||||
|
|
||||||
TEST_F(TestGcsWithAsio, TestClientTableMarkDisconnected) {
|
TEST_F(TestGcsWithAsio, TestNodeTableMarkDisconnected) {
|
||||||
test = this;
|
test = this;
|
||||||
ClientTableTestHelper::TestClientTableMarkDisconnected(job_id_, client_);
|
NodeTableTestHelper::TestNodeTableMarkDisconnected(job_id_, client_);
|
||||||
}
|
}
|
||||||
|
|
||||||
class HashTableTestHelper {
|
class HashTableTestHelper {
|
||||||
|
@ -1332,7 +1332,7 @@ class HashTableTestHelper {
|
||||||
static void TestHashTable(const JobID &job_id,
|
static void TestHashTable(const JobID &job_id,
|
||||||
std::shared_ptr<gcs::RedisGcsClient> client) {
|
std::shared_ptr<gcs::RedisGcsClient> client) {
|
||||||
uint64_t expected_count = 14;
|
uint64_t expected_count = 14;
|
||||||
NodeID client_id = NodeID::FromRandom();
|
NodeID node_id = NodeID::FromRandom();
|
||||||
// Prepare the first resource map: data_map1.
|
// Prepare the first resource map: data_map1.
|
||||||
DynamicResourceTable::DataMap data_map1;
|
DynamicResourceTable::DataMap data_map1;
|
||||||
auto cpu_data = std::make_shared<ResourceTableData>();
|
auto cpu_data = std::make_shared<ResourceTableData>();
|
||||||
|
@ -1400,8 +1400,8 @@ class HashTableTestHelper {
|
||||||
// Step 0: Subscribe the change of the hash table.
|
// Step 0: Subscribe the change of the hash table.
|
||||||
RAY_CHECK_OK(client->resource_table().Subscribe(
|
RAY_CHECK_OK(client->resource_table().Subscribe(
|
||||||
job_id, NodeID::Nil(), notification_callback, subscribe_callback));
|
job_id, NodeID::Nil(), notification_callback, subscribe_callback));
|
||||||
RAY_CHECK_OK(client->resource_table().RequestNotifications(job_id, client_id,
|
RAY_CHECK_OK(client->resource_table().RequestNotifications(job_id, node_id,
|
||||||
local_client_id, nullptr));
|
local_node_id, nullptr));
|
||||||
|
|
||||||
// Step 1: Add elements to the hash table.
|
// Step 1: Add elements to the hash table.
|
||||||
auto update_callback1 = [data_map1, compare_test](
|
auto update_callback1 = [data_map1, compare_test](
|
||||||
|
@ -1411,24 +1411,24 @@ class HashTableTestHelper {
|
||||||
test->IncrementNumCallbacks();
|
test->IncrementNumCallbacks();
|
||||||
};
|
};
|
||||||
RAY_CHECK_OK(
|
RAY_CHECK_OK(
|
||||||
client->resource_table().Update(job_id, client_id, data_map1, update_callback1));
|
client->resource_table().Update(job_id, node_id, data_map1, update_callback1));
|
||||||
auto lookup_callback1 = [data_map1, compare_test](
|
auto lookup_callback1 = [data_map1, compare_test](
|
||||||
RedisGcsClient *client, const NodeID &id,
|
RedisGcsClient *client, const NodeID &id,
|
||||||
const DynamicResourceTable::DataMap &callback_data) {
|
const DynamicResourceTable::DataMap &callback_data) {
|
||||||
compare_test(data_map1, callback_data);
|
compare_test(data_map1, callback_data);
|
||||||
test->IncrementNumCallbacks();
|
test->IncrementNumCallbacks();
|
||||||
};
|
};
|
||||||
RAY_CHECK_OK(client->resource_table().Lookup(job_id, client_id, lookup_callback1));
|
RAY_CHECK_OK(client->resource_table().Lookup(job_id, node_id, lookup_callback1));
|
||||||
|
|
||||||
// Step 2: Decrease one element, increase one and add a new one.
|
// Step 2: Decrease one element, increase one and add a new one.
|
||||||
RAY_CHECK_OK(client->resource_table().Update(job_id, client_id, data_map2, nullptr));
|
RAY_CHECK_OK(client->resource_table().Update(job_id, node_id, data_map2, nullptr));
|
||||||
auto lookup_callback2 = [data_map2, compare_test](
|
auto lookup_callback2 = [data_map2, compare_test](
|
||||||
RedisGcsClient *client, const NodeID &id,
|
RedisGcsClient *client, const NodeID &id,
|
||||||
const DynamicResourceTable::DataMap &callback_data) {
|
const DynamicResourceTable::DataMap &callback_data) {
|
||||||
compare_test(data_map2, callback_data);
|
compare_test(data_map2, callback_data);
|
||||||
test->IncrementNumCallbacks();
|
test->IncrementNumCallbacks();
|
||||||
};
|
};
|
||||||
RAY_CHECK_OK(client->resource_table().Lookup(job_id, client_id, lookup_callback2));
|
RAY_CHECK_OK(client->resource_table().Lookup(job_id, node_id, lookup_callback2));
|
||||||
std::vector<std::string> delete_keys({"GPU", "CUSTOM", "None-Existent"});
|
std::vector<std::string> delete_keys({"GPU", "CUSTOM", "None-Existent"});
|
||||||
auto remove_callback = [delete_keys](RedisGcsClient *client, const NodeID &id,
|
auto remove_callback = [delete_keys](RedisGcsClient *client, const NodeID &id,
|
||||||
const std::vector<std::string> &callback_data) {
|
const std::vector<std::string> &callback_data) {
|
||||||
|
@ -1438,7 +1438,7 @@ class HashTableTestHelper {
|
||||||
}
|
}
|
||||||
test->IncrementNumCallbacks();
|
test->IncrementNumCallbacks();
|
||||||
};
|
};
|
||||||
RAY_CHECK_OK(client->resource_table().RemoveEntries(job_id, client_id, delete_keys,
|
RAY_CHECK_OK(client->resource_table().RemoveEntries(job_id, node_id, delete_keys,
|
||||||
remove_callback));
|
remove_callback));
|
||||||
DynamicResourceTable::DataMap data_map3(data_map2);
|
DynamicResourceTable::DataMap data_map3(data_map2);
|
||||||
data_map3.erase("GPU");
|
data_map3.erase("GPU");
|
||||||
|
@ -1449,22 +1449,22 @@ class HashTableTestHelper {
|
||||||
compare_test(data_map3, callback_data);
|
compare_test(data_map3, callback_data);
|
||||||
test->IncrementNumCallbacks();
|
test->IncrementNumCallbacks();
|
||||||
};
|
};
|
||||||
RAY_CHECK_OK(client->resource_table().Lookup(job_id, client_id, lookup_callback3));
|
RAY_CHECK_OK(client->resource_table().Lookup(job_id, node_id, lookup_callback3));
|
||||||
|
|
||||||
// Step 3: Reset the the resources to data_map1.
|
// Step 3: Reset the the resources to data_map1.
|
||||||
RAY_CHECK_OK(
|
RAY_CHECK_OK(
|
||||||
client->resource_table().Update(job_id, client_id, data_map1, update_callback1));
|
client->resource_table().Update(job_id, node_id, data_map1, update_callback1));
|
||||||
auto lookup_callback4 = [data_map1, compare_test](
|
auto lookup_callback4 = [data_map1, compare_test](
|
||||||
RedisGcsClient *client, const NodeID &id,
|
RedisGcsClient *client, const NodeID &id,
|
||||||
const DynamicResourceTable::DataMap &callback_data) {
|
const DynamicResourceTable::DataMap &callback_data) {
|
||||||
compare_test(data_map1, callback_data);
|
compare_test(data_map1, callback_data);
|
||||||
test->IncrementNumCallbacks();
|
test->IncrementNumCallbacks();
|
||||||
};
|
};
|
||||||
RAY_CHECK_OK(client->resource_table().Lookup(job_id, client_id, lookup_callback4));
|
RAY_CHECK_OK(client->resource_table().Lookup(job_id, node_id, lookup_callback4));
|
||||||
|
|
||||||
// Step 4: Removing all elements will remove the home Hash table from GCS.
|
// Step 4: Removing all elements will remove the home Hash table from GCS.
|
||||||
RAY_CHECK_OK(client->resource_table().RemoveEntries(
|
RAY_CHECK_OK(client->resource_table().RemoveEntries(
|
||||||
job_id, client_id, {"GPU", "CPU", "CUSTOM", "None-Existent"}, nullptr));
|
job_id, node_id, {"GPU", "CPU", "CUSTOM", "None-Existent"}, nullptr));
|
||||||
auto lookup_callback5 = [expected_count](
|
auto lookup_callback5 = [expected_count](
|
||||||
RedisGcsClient *client, const NodeID &id,
|
RedisGcsClient *client, const NodeID &id,
|
||||||
const DynamicResourceTable::DataMap &callback_data) {
|
const DynamicResourceTable::DataMap &callback_data) {
|
||||||
|
@ -1475,7 +1475,7 @@ class HashTableTestHelper {
|
||||||
test->Stop();
|
test->Stop();
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
RAY_CHECK_OK(client->resource_table().Lookup(job_id, client_id, lookup_callback5));
|
RAY_CHECK_OK(client->resource_table().Lookup(job_id, node_id, lookup_callback5));
|
||||||
test->Start();
|
test->Start();
|
||||||
ASSERT_EQ(test->NumCallbacks(), expected_count);
|
ASSERT_EQ(test->NumCallbacks(), expected_count);
|
||||||
}
|
}
|
||||||
|
|
|
@ -56,7 +56,7 @@ bool UpdateObjectLocations(const std::vector<rpc::ObjectLocationChange> &locatio
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
// Filter out the removed clients from the object locations.
|
// Filter out the removed nodes from the object locations.
|
||||||
for (auto it = node_ids->begin(); it != node_ids->end();) {
|
for (auto it = node_ids->begin(); it != node_ids->end();) {
|
||||||
if (gcs_client->Nodes().IsRemoved(*it)) {
|
if (gcs_client->Nodes().IsRemoved(*it)) {
|
||||||
it = node_ids->erase(it);
|
it = node_ids->erase(it);
|
||||||
|
@ -71,29 +71,29 @@ bool UpdateObjectLocations(const std::vector<rpc::ObjectLocationChange> &locatio
|
||||||
} // namespace
|
} // namespace
|
||||||
|
|
||||||
ray::Status ObjectDirectory::ReportObjectAdded(
|
ray::Status ObjectDirectory::ReportObjectAdded(
|
||||||
const ObjectID &object_id, const NodeID &client_id,
|
const ObjectID &object_id, const NodeID &node_id,
|
||||||
const object_manager::protocol::ObjectInfoT &object_info) {
|
const object_manager::protocol::ObjectInfoT &object_info) {
|
||||||
RAY_LOG(DEBUG) << "Reporting object added to GCS " << object_id;
|
RAY_LOG(DEBUG) << "Reporting object added to GCS " << object_id;
|
||||||
ray::Status status =
|
ray::Status status =
|
||||||
gcs_client_->Objects().AsyncAddLocation(object_id, client_id, nullptr);
|
gcs_client_->Objects().AsyncAddLocation(object_id, node_id, nullptr);
|
||||||
return status;
|
return status;
|
||||||
}
|
}
|
||||||
|
|
||||||
ray::Status ObjectDirectory::ReportObjectRemoved(
|
ray::Status ObjectDirectory::ReportObjectRemoved(
|
||||||
const ObjectID &object_id, const NodeID &client_id,
|
const ObjectID &object_id, const NodeID &node_id,
|
||||||
const object_manager::protocol::ObjectInfoT &object_info) {
|
const object_manager::protocol::ObjectInfoT &object_info) {
|
||||||
RAY_LOG(DEBUG) << "Reporting object removed to GCS " << object_id;
|
RAY_LOG(DEBUG) << "Reporting object removed to GCS " << object_id;
|
||||||
ray::Status status =
|
ray::Status status =
|
||||||
gcs_client_->Objects().AsyncRemoveLocation(object_id, client_id, nullptr);
|
gcs_client_->Objects().AsyncRemoveLocation(object_id, node_id, nullptr);
|
||||||
return status;
|
return status;
|
||||||
};
|
};
|
||||||
|
|
||||||
void ObjectDirectory::LookupRemoteConnectionInfo(
|
void ObjectDirectory::LookupRemoteConnectionInfo(
|
||||||
RemoteConnectionInfo &connection_info) const {
|
RemoteConnectionInfo &connection_info) const {
|
||||||
auto node_info = gcs_client_->Nodes().Get(connection_info.client_id);
|
auto node_info = gcs_client_->Nodes().Get(connection_info.node_id);
|
||||||
if (node_info) {
|
if (node_info) {
|
||||||
NodeID result_node_id = NodeID::FromBinary(node_info->node_id());
|
NodeID result_node_id = NodeID::FromBinary(node_info->node_id());
|
||||||
RAY_CHECK(result_node_id == connection_info.client_id);
|
RAY_CHECK(result_node_id == connection_info.node_id);
|
||||||
connection_info.ip = node_info->node_manager_address();
|
connection_info.ip = node_info->node_manager_address();
|
||||||
connection_info.port = static_cast<uint16_t>(node_info->object_manager_port());
|
connection_info.port = static_cast<uint16_t>(node_info->object_manager_port());
|
||||||
}
|
}
|
||||||
|
@ -105,18 +105,18 @@ std::vector<RemoteConnectionInfo> ObjectDirectory::LookupAllRemoteConnections()
|
||||||
for (const auto &item : node_map) {
|
for (const auto &item : node_map) {
|
||||||
RemoteConnectionInfo info(item.first);
|
RemoteConnectionInfo info(item.first);
|
||||||
LookupRemoteConnectionInfo(info);
|
LookupRemoteConnectionInfo(info);
|
||||||
if (info.Connected() && info.client_id != gcs_client_->Nodes().GetSelfId()) {
|
if (info.Connected() && info.node_id != gcs_client_->Nodes().GetSelfId()) {
|
||||||
remote_connections.push_back(info);
|
remote_connections.push_back(info);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return remote_connections;
|
return remote_connections;
|
||||||
}
|
}
|
||||||
|
|
||||||
void ObjectDirectory::HandleClientRemoved(const NodeID &client_id) {
|
void ObjectDirectory::HandleNodeRemoved(const NodeID &node_id) {
|
||||||
for (auto &listener : listeners_) {
|
for (auto &listener : listeners_) {
|
||||||
const ObjectID &object_id = listener.first;
|
const ObjectID &object_id = listener.first;
|
||||||
if (listener.second.current_object_locations.count(client_id) > 0) {
|
if (listener.second.current_object_locations.count(node_id) > 0) {
|
||||||
// If the subscribed object has the removed client as a location, update
|
// If the subscribed object has the removed node as a location, update
|
||||||
// its locations with an empty update so that the location will be removed.
|
// its locations with an empty update so that the location will be removed.
|
||||||
UpdateObjectLocations({}, gcs_client_, &listener.second.current_object_locations,
|
UpdateObjectLocations({}, gcs_client_, &listener.second.current_object_locations,
|
||||||
&listener.second.spilled_url);
|
&listener.second.spilled_url);
|
||||||
|
|
|
@ -29,13 +29,13 @@ namespace ray {
|
||||||
|
|
||||||
/// Connection information for remote object managers.
|
/// Connection information for remote object managers.
|
||||||
struct RemoteConnectionInfo {
|
struct RemoteConnectionInfo {
|
||||||
RemoteConnectionInfo(const NodeID &id) : client_id(id) {}
|
RemoteConnectionInfo(const NodeID &id) : node_id(id) {}
|
||||||
|
|
||||||
// Returns whether there is enough information to connect to the remote
|
// Returns whether there is enough information to connect to the remote
|
||||||
// object manager.
|
// object manager.
|
||||||
bool Connected() const { return !ip.empty(); }
|
bool Connected() const { return !ip.empty(); }
|
||||||
|
|
||||||
NodeID client_id;
|
NodeID node_id;
|
||||||
std::string ip;
|
std::string ip;
|
||||||
uint16_t port;
|
uint16_t port;
|
||||||
};
|
};
|
||||||
|
@ -47,8 +47,8 @@ class ObjectDirectoryInterface {
|
||||||
/// Lookup how to connect to a remote object manager.
|
/// Lookup how to connect to a remote object manager.
|
||||||
///
|
///
|
||||||
/// \param connection_info The connection information to fill out. This
|
/// \param connection_info The connection information to fill out. This
|
||||||
/// should be pre-populated with the requested client ID. If the directory
|
/// should be pre-populated with the requested node ID. If the directory
|
||||||
/// has information about the requested client, then the rest of the fields
|
/// has information about the requested node, then the rest of the fields
|
||||||
/// in this struct will be populated accordingly.
|
/// in this struct will be populated accordingly.
|
||||||
virtual void LookupRemoteConnectionInfo(
|
virtual void LookupRemoteConnectionInfo(
|
||||||
RemoteConnectionInfo &connection_info) const = 0;
|
RemoteConnectionInfo &connection_info) const = 0;
|
||||||
|
@ -63,21 +63,21 @@ class ObjectDirectoryInterface {
|
||||||
std::function<void(const ray::ObjectID &object_id,
|
std::function<void(const ray::ObjectID &object_id,
|
||||||
const std::unordered_set<ray::NodeID> &, const std::string &)>;
|
const std::unordered_set<ray::NodeID> &, const std::string &)>;
|
||||||
|
|
||||||
/// Lookup object locations. Callback may be invoked with empty list of client ids.
|
/// Lookup object locations. Callback may be invoked with empty list of node ids.
|
||||||
///
|
///
|
||||||
/// \param object_id The object's ObjectID.
|
/// \param object_id The object's ObjectID.
|
||||||
/// \param callback Invoked with (possibly empty) list of client ids and object_id.
|
/// \param callback Invoked with (possibly empty) list of node ids and object_id.
|
||||||
/// \return Status of whether async call to backend succeeded.
|
/// \return Status of whether async call to backend succeeded.
|
||||||
virtual ray::Status LookupLocations(const ObjectID &object_id,
|
virtual ray::Status LookupLocations(const ObjectID &object_id,
|
||||||
const rpc::Address &owner_address,
|
const rpc::Address &owner_address,
|
||||||
const OnLocationsFound &callback) = 0;
|
const OnLocationsFound &callback) = 0;
|
||||||
|
|
||||||
/// Handle the removal of an object manager client. This updates the
|
/// Handle the removal of an object manager node. This updates the
|
||||||
/// locations of all subscribed objects that have the removed client as a
|
/// locations of all subscribed objects that have the removed node as a
|
||||||
/// location, and fires the subscribed callbacks for those objects.
|
/// location, and fires the subscribed callbacks for those objects.
|
||||||
///
|
///
|
||||||
/// \param client_id The object manager client that was removed.
|
/// \param node_id The object manager node that was removed.
|
||||||
virtual void HandleClientRemoved(const NodeID &client_id) = 0;
|
virtual void HandleNodeRemoved(const NodeID &node_id) = 0;
|
||||||
|
|
||||||
/// Subscribe to be notified of locations (NodeID) of the given object.
|
/// Subscribe to be notified of locations (NodeID) of the given object.
|
||||||
/// The callback will be invoked with the complete list of known locations
|
/// The callback will be invoked with the complete list of known locations
|
||||||
|
@ -90,7 +90,7 @@ class ObjectDirectoryInterface {
|
||||||
/// \param callback_id The id associated with the specified callback. This is
|
/// \param callback_id The id associated with the specified callback. This is
|
||||||
/// needed when UnsubscribeObjectLocations is called.
|
/// needed when UnsubscribeObjectLocations is called.
|
||||||
/// \param object_id The required object's ObjectID.
|
/// \param object_id The required object's ObjectID.
|
||||||
/// \param success_cb Invoked with non-empty list of client ids and object_id.
|
/// \param success_cb Invoked with non-empty list of node ids and object_id.
|
||||||
/// \return Status of whether subscription succeeded.
|
/// \return Status of whether subscription succeeded.
|
||||||
virtual ray::Status SubscribeObjectLocations(const UniqueID &callback_id,
|
virtual ray::Status SubscribeObjectLocations(const UniqueID &callback_id,
|
||||||
const ObjectID &object_id,
|
const ObjectID &object_id,
|
||||||
|
@ -110,21 +110,21 @@ class ObjectDirectoryInterface {
|
||||||
/// Report objects added to this node's store to the object directory.
|
/// Report objects added to this node's store to the object directory.
|
||||||
///
|
///
|
||||||
/// \param object_id The object id that was put into the store.
|
/// \param object_id The object id that was put into the store.
|
||||||
/// \param client_id The client id corresponding to this node.
|
/// \param node_id The node id corresponding to this node.
|
||||||
/// \param object_info Additional information about the object.
|
/// \param object_info Additional information about the object.
|
||||||
/// \return Status of whether this method succeeded.
|
/// \return Status of whether this method succeeded.
|
||||||
virtual ray::Status ReportObjectAdded(
|
virtual ray::Status ReportObjectAdded(
|
||||||
const ObjectID &object_id, const NodeID &client_id,
|
const ObjectID &object_id, const NodeID &node_id,
|
||||||
const object_manager::protocol::ObjectInfoT &object_info) = 0;
|
const object_manager::protocol::ObjectInfoT &object_info) = 0;
|
||||||
|
|
||||||
/// Report objects removed from this client's store to the object directory.
|
/// Report objects removed from this node's store to the object directory.
|
||||||
///
|
///
|
||||||
/// \param object_id The object id that was removed from the store.
|
/// \param object_id The object id that was removed from the store.
|
||||||
/// \param client_id The client id corresponding to this node.
|
/// \param node_id The node id corresponding to this node.
|
||||||
/// \param object_info Additional information about the object.
|
/// \param object_info Additional information about the object.
|
||||||
/// \return Status of whether this method succeeded.
|
/// \return Status of whether this method succeeded.
|
||||||
virtual ray::Status ReportObjectRemoved(
|
virtual ray::Status ReportObjectRemoved(
|
||||||
const ObjectID &object_id, const NodeID &client_id,
|
const ObjectID &object_id, const NodeID &node_id,
|
||||||
const object_manager::protocol::ObjectInfoT &object_info) = 0;
|
const object_manager::protocol::ObjectInfoT &object_info) = 0;
|
||||||
|
|
||||||
/// Returns debug string for class.
|
/// Returns debug string for class.
|
||||||
|
@ -140,7 +140,7 @@ class ObjectDirectory : public ObjectDirectoryInterface {
|
||||||
///
|
///
|
||||||
/// \param io_service The event loop to dispatch callbacks to. This should
|
/// \param io_service The event loop to dispatch callbacks to. This should
|
||||||
/// usually be the same event loop that the given gcs_client runs on.
|
/// usually be the same event loop that the given gcs_client runs on.
|
||||||
/// \param gcs_client A Ray GCS client to request object and client
|
/// \param gcs_client A Ray GCS client to request object and node
|
||||||
/// information from.
|
/// information from.
|
||||||
ObjectDirectory(boost::asio::io_service &io_service,
|
ObjectDirectory(boost::asio::io_service &io_service,
|
||||||
std::shared_ptr<gcs::GcsClient> &gcs_client);
|
std::shared_ptr<gcs::GcsClient> &gcs_client);
|
||||||
|
@ -155,7 +155,7 @@ class ObjectDirectory : public ObjectDirectoryInterface {
|
||||||
const rpc::Address &owner_address,
|
const rpc::Address &owner_address,
|
||||||
const OnLocationsFound &callback) override;
|
const OnLocationsFound &callback) override;
|
||||||
|
|
||||||
void HandleClientRemoved(const NodeID &client_id) override;
|
void HandleNodeRemoved(const NodeID &node_id) override;
|
||||||
|
|
||||||
ray::Status SubscribeObjectLocations(const UniqueID &callback_id,
|
ray::Status SubscribeObjectLocations(const UniqueID &callback_id,
|
||||||
const ObjectID &object_id,
|
const ObjectID &object_id,
|
||||||
|
@ -165,10 +165,10 @@ class ObjectDirectory : public ObjectDirectoryInterface {
|
||||||
const ObjectID &object_id) override;
|
const ObjectID &object_id) override;
|
||||||
|
|
||||||
ray::Status ReportObjectAdded(
|
ray::Status ReportObjectAdded(
|
||||||
const ObjectID &object_id, const NodeID &client_id,
|
const ObjectID &object_id, const NodeID &node_id,
|
||||||
const object_manager::protocol::ObjectInfoT &object_info) override;
|
const object_manager::protocol::ObjectInfoT &object_info) override;
|
||||||
ray::Status ReportObjectRemoved(
|
ray::Status ReportObjectRemoved(
|
||||||
const ObjectID &object_id, const NodeID &client_id,
|
const ObjectID &object_id, const NodeID &node_id,
|
||||||
const object_manager::protocol::ObjectInfoT &object_info) override;
|
const object_manager::protocol::ObjectInfoT &object_info) override;
|
||||||
|
|
||||||
std::string DebugString() const override;
|
std::string DebugString() const override;
|
||||||
|
|
|
@ -136,8 +136,8 @@ void ObjectManager::HandleObjectAdded(
|
||||||
auto iter = unfulfilled_push_requests_.find(object_id);
|
auto iter = unfulfilled_push_requests_.find(object_id);
|
||||||
if (iter != unfulfilled_push_requests_.end()) {
|
if (iter != unfulfilled_push_requests_.end()) {
|
||||||
for (auto &pair : iter->second) {
|
for (auto &pair : iter->second) {
|
||||||
auto &client_id = pair.first;
|
auto &node_id = pair.first;
|
||||||
main_service_->post([this, object_id, client_id]() { Push(object_id, client_id); });
|
main_service_->post([this, object_id, node_id]() { Push(object_id, node_id); });
|
||||||
// When push timeout is set to -1, there will be an empty timer in pair.second.
|
// When push timeout is set to -1, there will be an empty timer in pair.second.
|
||||||
if (pair.second != nullptr) {
|
if (pair.second != nullptr) {
|
||||||
pair.second->cancel();
|
pair.second->cancel();
|
||||||
|
@ -189,24 +189,23 @@ ray::Status ObjectManager::Pull(const ObjectID &object_id,
|
||||||
|
|
||||||
pull_requests_.emplace(object_id, PullRequest());
|
pull_requests_.emplace(object_id, PullRequest());
|
||||||
// Subscribe to object notifications. A notification will be received every
|
// Subscribe to object notifications. A notification will be received every
|
||||||
// time the set of client IDs for the object changes. Notifications will also
|
// time the set of node IDs for the object changes. Notifications will also
|
||||||
// be received if the list of locations is empty. The set of client IDs has
|
// be received if the list of locations is empty. The set of node IDs has
|
||||||
// no ordering guarantee between notifications.
|
// no ordering guarantee between notifications.
|
||||||
return object_directory_->SubscribeObjectLocations(
|
return object_directory_->SubscribeObjectLocations(
|
||||||
object_directory_pull_callback_id_, object_id, owner_address,
|
object_directory_pull_callback_id_, object_id, owner_address,
|
||||||
[this](const ObjectID &object_id, const std::unordered_set<NodeID> &client_ids,
|
[this](const ObjectID &object_id, const std::unordered_set<NodeID> &node_ids,
|
||||||
const std::string &spilled_url) {
|
const std::string &spilled_url) {
|
||||||
// Exit if the Pull request has already been fulfilled or canceled.
|
// Exit if the Pull request has already been fulfilled or canceled.
|
||||||
auto it = pull_requests_.find(object_id);
|
auto it = pull_requests_.find(object_id);
|
||||||
if (it == pull_requests_.end()) {
|
if (it == pull_requests_.end()) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
// Reset the list of clients that are now expected to have the object.
|
// Reset the list of nodes that are now expected to have the object.
|
||||||
// NOTE(swang): Since we are overwriting the previous list of clients,
|
// NOTE(swang): Since we are overwriting the previous list of nodes,
|
||||||
// we may end up sending a duplicate request to the same client as
|
// we may end up sending a duplicate request to the same node as
|
||||||
// before.
|
// before.
|
||||||
it->second.client_locations =
|
it->second.node_locations = std::vector<NodeID>(node_ids.begin(), node_ids.end());
|
||||||
std::vector<NodeID>(client_ids.begin(), client_ids.end());
|
|
||||||
if (!spilled_url.empty()) {
|
if (!spilled_url.empty()) {
|
||||||
// Try to restore the spilled object.
|
// Try to restore the spilled object.
|
||||||
restore_spilled_object_(object_id, spilled_url,
|
restore_spilled_object_(object_id, spilled_url,
|
||||||
|
@ -216,17 +215,17 @@ ray::Status ObjectManager::Pull(const ObjectID &object_id,
|
||||||
TryPull(object_id);
|
TryPull(object_id);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
} else if (it->second.client_locations.empty()) {
|
} else if (it->second.node_locations.empty()) {
|
||||||
// The object locations are now empty, so we should wait for the next
|
// The object locations are now empty, so we should wait for the next
|
||||||
// notification about a new object location. Cancel the timer until
|
// notification about a new object location. Cancel the timer until
|
||||||
// the next Pull attempt since there are no more clients to try.
|
// the next Pull attempt since there are no more nodes to try.
|
||||||
if (it->second.retry_timer != nullptr) {
|
if (it->second.retry_timer != nullptr) {
|
||||||
it->second.retry_timer->cancel();
|
it->second.retry_timer->cancel();
|
||||||
it->second.timer_set = false;
|
it->second.timer_set = false;
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
// New object locations were found, so begin trying to pull from a
|
// New object locations were found, so begin trying to pull from a
|
||||||
// client. This will be called every time a new client location
|
// node. This will be called every time a new node location
|
||||||
// appears.
|
// appears.
|
||||||
TryPull(object_id);
|
TryPull(object_id);
|
||||||
}
|
}
|
||||||
|
@ -239,15 +238,15 @@ void ObjectManager::TryPull(const ObjectID &object_id) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
auto &node_vector = it->second.client_locations;
|
auto &node_vector = it->second.node_locations;
|
||||||
|
|
||||||
// The timer should never fire if there are no expected client locations.
|
// The timer should never fire if there are no expected node locations.
|
||||||
if (node_vector.empty()) {
|
if (node_vector.empty()) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
RAY_CHECK(local_objects_.count(object_id) == 0);
|
RAY_CHECK(local_objects_.count(object_id) == 0);
|
||||||
// Make sure that there is at least one client which is not the local client.
|
// Make sure that there is at least one node which is not the local node.
|
||||||
// TODO(rkn): It may actually be possible for this check to fail.
|
// TODO(rkn): It may actually be possible for this check to fail.
|
||||||
if (node_vector.size() == 1 && node_vector[0] == self_node_id_) {
|
if (node_vector.size() == 1 && node_vector[0] == self_node_id_) {
|
||||||
RAY_LOG(WARNING) << "The object manager with ID " << self_node_id_
|
RAY_LOG(WARNING) << "The object manager with ID " << self_node_id_
|
||||||
|
@ -260,7 +259,7 @@ void ObjectManager::TryPull(const ObjectID &object_id) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
// Choose a random client to pull the object from.
|
// Choose a random node to pull the object from.
|
||||||
// Generate a random index.
|
// Generate a random index.
|
||||||
std::uniform_int_distribution<int> distribution(0, node_vector.size() - 1);
|
std::uniform_int_distribution<int> distribution(0, node_vector.size() - 1);
|
||||||
int node_index = distribution(gen_);
|
int node_index = distribution(gen_);
|
||||||
|
@ -284,7 +283,7 @@ void ObjectManager::TryPull(const ObjectID &object_id) {
|
||||||
|
|
||||||
auto rpc_client = GetRpcClient(node_id);
|
auto rpc_client = GetRpcClient(node_id);
|
||||||
if (rpc_client) {
|
if (rpc_client) {
|
||||||
// Try pulling from the client.
|
// Try pulling from the node.
|
||||||
rpc_service_.post([this, object_id, node_id, rpc_client]() {
|
rpc_service_.post([this, object_id, node_id, rpc_client]() {
|
||||||
SendPullRequest(object_id, node_id, rpc_client);
|
SendPullRequest(object_id, node_id, rpc_client);
|
||||||
});
|
});
|
||||||
|
@ -294,9 +293,9 @@ void ObjectManager::TryPull(const ObjectID &object_id) {
|
||||||
<< " , setup rpc connection failed.";
|
<< " , setup rpc connection failed.";
|
||||||
}
|
}
|
||||||
|
|
||||||
// If there are more clients to try, try them in succession, with a timeout
|
// If there are more nodes to try, try them in succession, with a timeout
|
||||||
// in between each try.
|
// in between each try.
|
||||||
if (!it->second.client_locations.empty()) {
|
if (!it->second.node_locations.empty()) {
|
||||||
if (it->second.retry_timer == nullptr) {
|
if (it->second.retry_timer == nullptr) {
|
||||||
// Set the timer if we haven't already.
|
// Set the timer if we haven't already.
|
||||||
it->second.retry_timer = std::unique_ptr<boost::asio::deadline_timer>(
|
it->second.retry_timer = std::unique_ptr<boost::asio::deadline_timer>(
|
||||||
|
@ -305,13 +304,13 @@ void ObjectManager::TryPull(const ObjectID &object_id) {
|
||||||
|
|
||||||
// Wait for a timeout. If we receive the object or a caller Cancels the
|
// Wait for a timeout. If we receive the object or a caller Cancels the
|
||||||
// Pull within the timeout, then nothing will happen. Otherwise, the timer
|
// Pull within the timeout, then nothing will happen. Otherwise, the timer
|
||||||
// will fire and the next client in the list will be tried.
|
// will fire and the next node in the list will be tried.
|
||||||
boost::posix_time::milliseconds retry_timeout(config_.pull_timeout_ms);
|
boost::posix_time::milliseconds retry_timeout(config_.pull_timeout_ms);
|
||||||
it->second.retry_timer->expires_from_now(retry_timeout);
|
it->second.retry_timer->expires_from_now(retry_timeout);
|
||||||
it->second.retry_timer->async_wait(
|
it->second.retry_timer->async_wait(
|
||||||
[this, object_id](const boost::system::error_code &error) {
|
[this, object_id](const boost::system::error_code &error) {
|
||||||
if (!error) {
|
if (!error) {
|
||||||
// Try the Pull from the next client.
|
// Try the Pull from the next node.
|
||||||
TryPull(object_id);
|
TryPull(object_id);
|
||||||
} else {
|
} else {
|
||||||
// Check that the error was due to the timer being canceled.
|
// Check that the error was due to the timer being canceled.
|
||||||
|
@ -321,7 +320,7 @@ void ObjectManager::TryPull(const ObjectID &object_id) {
|
||||||
// Record that we set the timer until the next attempt.
|
// Record that we set the timer until the next attempt.
|
||||||
it->second.timer_set = true;
|
it->second.timer_set = true;
|
||||||
} else {
|
} else {
|
||||||
// The timer is not reset since there are no more clients to try. Go back
|
// The timer is not reset since there are no more nodes to try. Go back
|
||||||
// to waiting for more notifications. Once we receive a new object location
|
// to waiting for more notifications. Once we receive a new object location
|
||||||
// from the object directory, then the Pull will be retried.
|
// from the object directory, then the Pull will be retried.
|
||||||
it->second.timer_set = false;
|
it->second.timer_set = false;
|
||||||
|
@ -329,38 +328,38 @@ void ObjectManager::TryPull(const ObjectID &object_id) {
|
||||||
};
|
};
|
||||||
|
|
||||||
void ObjectManager::SendPullRequest(
|
void ObjectManager::SendPullRequest(
|
||||||
const ObjectID &object_id, const NodeID &client_id,
|
const ObjectID &object_id, const NodeID &node_id,
|
||||||
std::shared_ptr<rpc::ObjectManagerClient> rpc_client) {
|
std::shared_ptr<rpc::ObjectManagerClient> rpc_client) {
|
||||||
rpc::PullRequest pull_request;
|
rpc::PullRequest pull_request;
|
||||||
pull_request.set_object_id(object_id.Binary());
|
pull_request.set_object_id(object_id.Binary());
|
||||||
pull_request.set_client_id(self_node_id_.Binary());
|
pull_request.set_node_id(self_node_id_.Binary());
|
||||||
|
|
||||||
rpc_client->Pull(pull_request, [object_id, client_id](const Status &status,
|
rpc_client->Pull(pull_request, [object_id, node_id](const Status &status,
|
||||||
const rpc::PullReply &reply) {
|
const rpc::PullReply &reply) {
|
||||||
if (!status.ok()) {
|
if (!status.ok()) {
|
||||||
RAY_LOG(WARNING) << "Send pull " << object_id << " request to client " << client_id
|
RAY_LOG(WARNING) << "Send pull " << object_id << " request to node " << node_id
|
||||||
<< " failed due to" << status.message();
|
<< " failed due to" << status.message();
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
void ObjectManager::HandlePushTaskTimeout(const ObjectID &object_id,
|
void ObjectManager::HandlePushTaskTimeout(const ObjectID &object_id,
|
||||||
const NodeID &client_id) {
|
const NodeID &node_id) {
|
||||||
RAY_LOG(WARNING) << "Invalid Push request ObjectID: " << object_id
|
RAY_LOG(WARNING) << "Invalid Push request ObjectID: " << object_id
|
||||||
<< " after waiting for " << config_.push_timeout_ms << " ms.";
|
<< " after waiting for " << config_.push_timeout_ms << " ms.";
|
||||||
auto iter = unfulfilled_push_requests_.find(object_id);
|
auto iter = unfulfilled_push_requests_.find(object_id);
|
||||||
RAY_CHECK(iter != unfulfilled_push_requests_.end());
|
RAY_CHECK(iter != unfulfilled_push_requests_.end());
|
||||||
size_t num_erased = iter->second.erase(client_id);
|
size_t num_erased = iter->second.erase(node_id);
|
||||||
RAY_CHECK(num_erased == 1);
|
RAY_CHECK(num_erased == 1);
|
||||||
if (iter->second.size() == 0) {
|
if (iter->second.size() == 0) {
|
||||||
unfulfilled_push_requests_.erase(iter);
|
unfulfilled_push_requests_.erase(iter);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void ObjectManager::HandleSendFinished(const ObjectID &object_id, const NodeID &client_id,
|
void ObjectManager::HandleSendFinished(const ObjectID &object_id, const NodeID &node_id,
|
||||||
uint64_t chunk_index, double start_time,
|
uint64_t chunk_index, double start_time,
|
||||||
double end_time, ray::Status status) {
|
double end_time, ray::Status status) {
|
||||||
RAY_LOG(DEBUG) << "HandleSendFinished on " << self_node_id_ << " to " << client_id
|
RAY_LOG(DEBUG) << "HandleSendFinished on " << self_node_id_ << " to " << node_id
|
||||||
<< " of object " << object_id << " chunk " << chunk_index
|
<< " of object " << object_id << " chunk " << chunk_index
|
||||||
<< ", status: " << status.ToString();
|
<< ", status: " << status.ToString();
|
||||||
if (!status.ok()) {
|
if (!status.ok()) {
|
||||||
|
@ -371,18 +370,18 @@ void ObjectManager::HandleSendFinished(const ObjectID &object_id, const NodeID &
|
||||||
profile_event.set_event_type("transfer_send");
|
profile_event.set_event_type("transfer_send");
|
||||||
profile_event.set_start_time(start_time);
|
profile_event.set_start_time(start_time);
|
||||||
profile_event.set_end_time(end_time);
|
profile_event.set_end_time(end_time);
|
||||||
// Encode the object ID, client ID, chunk index, and status as a json list,
|
// Encode the object ID, node ID, chunk index, and status as a json list,
|
||||||
// which will be parsed by the reader of the profile table.
|
// which will be parsed by the reader of the profile table.
|
||||||
profile_event.set_extra_data("[\"" + object_id.Hex() + "\",\"" + client_id.Hex() +
|
profile_event.set_extra_data("[\"" + object_id.Hex() + "\",\"" + node_id.Hex() + "\"," +
|
||||||
"\"," + std::to_string(chunk_index) + ",\"" +
|
std::to_string(chunk_index) + ",\"" + status.ToString() +
|
||||||
status.ToString() + "\"]");
|
"\"]");
|
||||||
|
|
||||||
std::lock_guard<std::mutex> lock(profile_mutex_);
|
std::lock_guard<std::mutex> lock(profile_mutex_);
|
||||||
profile_events_.push_back(profile_event);
|
profile_events_.push_back(profile_event);
|
||||||
}
|
}
|
||||||
|
|
||||||
void ObjectManager::HandleReceiveFinished(const ObjectID &object_id,
|
void ObjectManager::HandleReceiveFinished(const ObjectID &object_id,
|
||||||
const NodeID &client_id, uint64_t chunk_index,
|
const NodeID &node_id, uint64_t chunk_index,
|
||||||
double start_time, double end_time,
|
double start_time, double end_time,
|
||||||
ray::Status status) {
|
ray::Status status) {
|
||||||
if (!status.ok()) {
|
if (!status.ok()) {
|
||||||
|
@ -393,24 +392,24 @@ void ObjectManager::HandleReceiveFinished(const ObjectID &object_id,
|
||||||
profile_event.set_event_type("transfer_receive");
|
profile_event.set_event_type("transfer_receive");
|
||||||
profile_event.set_start_time(start_time);
|
profile_event.set_start_time(start_time);
|
||||||
profile_event.set_end_time(end_time);
|
profile_event.set_end_time(end_time);
|
||||||
// Encode the object ID, client ID, chunk index, and status as a json list,
|
// Encode the object ID, node ID, chunk index, and status as a json list,
|
||||||
// which will be parsed by the reader of the profile table.
|
// which will be parsed by the reader of the profile table.
|
||||||
|
|
||||||
profile_event.set_extra_data("[\"" + object_id.Hex() + "\",\"" + client_id.Hex() +
|
profile_event.set_extra_data("[\"" + object_id.Hex() + "\",\"" + node_id.Hex() + "\"," +
|
||||||
"\"," + std::to_string(chunk_index) + ",\"" +
|
std::to_string(chunk_index) + ",\"" + status.ToString() +
|
||||||
status.ToString() + "\"]");
|
"\"]");
|
||||||
|
|
||||||
std::lock_guard<std::mutex> lock(profile_mutex_);
|
std::lock_guard<std::mutex> lock(profile_mutex_);
|
||||||
profile_events_.push_back(profile_event);
|
profile_events_.push_back(profile_event);
|
||||||
}
|
}
|
||||||
|
|
||||||
void ObjectManager::Push(const ObjectID &object_id, const NodeID &client_id) {
|
void ObjectManager::Push(const ObjectID &object_id, const NodeID &node_id) {
|
||||||
RAY_LOG(DEBUG) << "Push on " << self_node_id_ << " to " << client_id << " of object "
|
RAY_LOG(DEBUG) << "Push on " << self_node_id_ << " to " << node_id << " of object "
|
||||||
<< object_id;
|
<< object_id;
|
||||||
if (local_objects_.count(object_id) == 0) {
|
if (local_objects_.count(object_id) == 0) {
|
||||||
// Avoid setting duplicated timer for the same object and client pair.
|
// Avoid setting duplicated timer for the same object and node pair.
|
||||||
auto &clients = unfulfilled_push_requests_[object_id];
|
auto &nodes = unfulfilled_push_requests_[object_id];
|
||||||
if (clients.count(client_id) == 0) {
|
if (nodes.count(node_id) == 0) {
|
||||||
// If config_.push_timeout_ms < 0, we give an empty timer
|
// If config_.push_timeout_ms < 0, we give an empty timer
|
||||||
// and the task will be kept infinitely.
|
// and the task will be kept infinitely.
|
||||||
auto timer = std::unique_ptr<boost::asio::deadline_timer>();
|
auto timer = std::unique_ptr<boost::asio::deadline_timer>();
|
||||||
|
@ -424,22 +423,22 @@ void ObjectManager::Push(const ObjectID &object_id, const NodeID &client_id) {
|
||||||
auto clean_push_period = boost::posix_time::milliseconds(config_.push_timeout_ms);
|
auto clean_push_period = boost::posix_time::milliseconds(config_.push_timeout_ms);
|
||||||
timer->expires_from_now(clean_push_period);
|
timer->expires_from_now(clean_push_period);
|
||||||
timer->async_wait(
|
timer->async_wait(
|
||||||
[this, object_id, client_id](const boost::system::error_code &error) {
|
[this, object_id, node_id](const boost::system::error_code &error) {
|
||||||
// Timer killing will receive the boost::asio::error::operation_aborted,
|
// Timer killing will receive the boost::asio::error::operation_aborted,
|
||||||
// we only handle the timeout event.
|
// we only handle the timeout event.
|
||||||
if (!error) {
|
if (!error) {
|
||||||
HandlePushTaskTimeout(object_id, client_id);
|
HandlePushTaskTimeout(object_id, node_id);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
if (config_.push_timeout_ms != 0) {
|
if (config_.push_timeout_ms != 0) {
|
||||||
clients.emplace(client_id, std::move(timer));
|
nodes.emplace(node_id, std::move(timer));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
auto rpc_client = GetRpcClient(client_id);
|
auto rpc_client = GetRpcClient(node_id);
|
||||||
if (rpc_client) {
|
if (rpc_client) {
|
||||||
const object_manager::protocol::ObjectInfoT &object_info =
|
const object_manager::protocol::ObjectInfoT &object_info =
|
||||||
local_objects_[object_id].object_info;
|
local_objects_[object_id].object_info;
|
||||||
|
@ -454,15 +453,15 @@ void ObjectManager::Push(const ObjectID &object_id, const NodeID &client_id) {
|
||||||
owner_address.set_port(object_info.owner_port);
|
owner_address.set_port(object_info.owner_port);
|
||||||
owner_address.set_worker_id(object_info.owner_worker_id);
|
owner_address.set_worker_id(object_info.owner_worker_id);
|
||||||
|
|
||||||
RAY_LOG(DEBUG) << "Sending object chunks of " << object_id << " to client "
|
RAY_LOG(DEBUG) << "Sending object chunks of " << object_id << " to node " << node_id
|
||||||
<< client_id << ", number of chunks: " << num_chunks
|
<< ", number of chunks: " << num_chunks
|
||||||
<< ", total data size: " << data_size;
|
<< ", total data size: " << data_size;
|
||||||
|
|
||||||
UniqueID push_id = UniqueID::FromRandom();
|
UniqueID push_id = UniqueID::FromRandom();
|
||||||
push_manager_->StartPush(client_id, object_id, num_chunks, [=](int64_t chunk_id) {
|
push_manager_->StartPush(node_id, object_id, num_chunks, [=](int64_t chunk_id) {
|
||||||
SendObjectChunk(push_id, object_id, owner_address, client_id, data_size,
|
SendObjectChunk(push_id, object_id, owner_address, node_id, data_size,
|
||||||
metadata_size, chunk_id, rpc_client, [=](const Status &status) {
|
metadata_size, chunk_id, rpc_client, [=](const Status &status) {
|
||||||
push_manager_->OnChunkComplete(client_id, object_id);
|
push_manager_->OnChunkComplete(node_id, object_id);
|
||||||
});
|
});
|
||||||
});
|
});
|
||||||
} else {
|
} else {
|
||||||
|
@ -474,7 +473,7 @@ void ObjectManager::Push(const ObjectID &object_id, const NodeID &client_id) {
|
||||||
|
|
||||||
void ObjectManager::SendObjectChunk(const UniqueID &push_id, const ObjectID &object_id,
|
void ObjectManager::SendObjectChunk(const UniqueID &push_id, const ObjectID &object_id,
|
||||||
const rpc::Address &owner_address,
|
const rpc::Address &owner_address,
|
||||||
const NodeID &client_id, uint64_t data_size,
|
const NodeID &node_id, uint64_t data_size,
|
||||||
uint64_t metadata_size, uint64_t chunk_index,
|
uint64_t metadata_size, uint64_t chunk_index,
|
||||||
std::shared_ptr<rpc::ObjectManagerClient> rpc_client,
|
std::shared_ptr<rpc::ObjectManagerClient> rpc_client,
|
||||||
std::function<void(const Status &)> on_complete) {
|
std::function<void(const Status &)> on_complete) {
|
||||||
|
@ -484,7 +483,7 @@ void ObjectManager::SendObjectChunk(const UniqueID &push_id, const ObjectID &obj
|
||||||
push_request.set_push_id(push_id.Binary());
|
push_request.set_push_id(push_id.Binary());
|
||||||
push_request.set_object_id(object_id.Binary());
|
push_request.set_object_id(object_id.Binary());
|
||||||
push_request.mutable_owner_address()->CopyFrom(owner_address);
|
push_request.mutable_owner_address()->CopyFrom(owner_address);
|
||||||
push_request.set_client_id(self_node_id_.Binary());
|
push_request.set_node_id(self_node_id_.Binary());
|
||||||
push_request.set_data_size(data_size);
|
push_request.set_data_size(data_size);
|
||||||
push_request.set_metadata_size(metadata_size);
|
push_request.set_metadata_size(metadata_size);
|
||||||
push_request.set_chunk_index(chunk_index);
|
push_request.set_chunk_index(chunk_index);
|
||||||
|
@ -508,17 +507,16 @@ void ObjectManager::SendObjectChunk(const UniqueID &push_id, const ObjectID &obj
|
||||||
|
|
||||||
// record the time cost between send chunk and receive reply
|
// record the time cost between send chunk and receive reply
|
||||||
rpc::ClientCallback<rpc::PushReply> callback =
|
rpc::ClientCallback<rpc::PushReply> callback =
|
||||||
[this, start_time, object_id, client_id, chunk_index, owner_address, rpc_client,
|
[this, start_time, object_id, node_id, chunk_index, owner_address, rpc_client,
|
||||||
on_complete](const Status &status, const rpc::PushReply &reply) {
|
on_complete](const Status &status, const rpc::PushReply &reply) {
|
||||||
// TODO: Just print warning here, should we try to resend this chunk?
|
// TODO: Just print warning here, should we try to resend this chunk?
|
||||||
if (!status.ok()) {
|
if (!status.ok()) {
|
||||||
RAY_LOG(WARNING) << "Send object " << object_id << " chunk to client "
|
RAY_LOG(WARNING) << "Send object " << object_id << " chunk to node " << node_id
|
||||||
<< client_id << " failed due to" << status.message()
|
<< " failed due to" << status.message()
|
||||||
<< ", chunk index: " << chunk_index;
|
<< ", chunk index: " << chunk_index;
|
||||||
}
|
}
|
||||||
double end_time = absl::GetCurrentTimeNanos() / 1e9;
|
double end_time = absl::GetCurrentTimeNanos() / 1e9;
|
||||||
HandleSendFinished(object_id, client_id, chunk_index, start_time, end_time,
|
HandleSendFinished(object_id, node_id, chunk_index, start_time, end_time, status);
|
||||||
status);
|
|
||||||
on_complete(status);
|
on_complete(status);
|
||||||
};
|
};
|
||||||
rpc_client->Push(push_request, callback);
|
rpc_client->Push(push_request, callback);
|
||||||
|
@ -600,18 +598,18 @@ ray::Status ObjectManager::LookupRemainingWaitObjects(const UniqueID &wait_id) {
|
||||||
RAY_RETURN_NOT_OK(object_directory_->LookupLocations(
|
RAY_RETURN_NOT_OK(object_directory_->LookupLocations(
|
||||||
object_id, wait_state.owner_addresses[object_id],
|
object_id, wait_state.owner_addresses[object_id],
|
||||||
[this, wait_id](const ObjectID &lookup_object_id,
|
[this, wait_id](const ObjectID &lookup_object_id,
|
||||||
const std::unordered_set<NodeID> &client_ids,
|
const std::unordered_set<NodeID> &node_ids,
|
||||||
const std::string &spilled_url) {
|
const std::string &spilled_url) {
|
||||||
auto &wait_state = active_wait_requests_.find(wait_id)->second;
|
auto &wait_state = active_wait_requests_.find(wait_id)->second;
|
||||||
// Note that the object is guaranteed to be added to local_objects_ before
|
// Note that the object is guaranteed to be added to local_objects_ before
|
||||||
// the notification is triggered.
|
// the notification is triggered.
|
||||||
bool remote_object_ready = !client_ids.empty() || !spilled_url.empty();
|
bool remote_object_ready = !node_ids.empty() || !spilled_url.empty();
|
||||||
if (local_objects_.count(lookup_object_id) > 0 ||
|
if (local_objects_.count(lookup_object_id) > 0 ||
|
||||||
(!wait_state.wait_local && remote_object_ready)) {
|
(!wait_state.wait_local && remote_object_ready)) {
|
||||||
wait_state.remaining.erase(lookup_object_id);
|
wait_state.remaining.erase(lookup_object_id);
|
||||||
wait_state.found.insert(lookup_object_id);
|
wait_state.found.insert(lookup_object_id);
|
||||||
}
|
}
|
||||||
RAY_LOG(DEBUG) << "Wait request " << wait_id << ": " << client_ids.size()
|
RAY_LOG(DEBUG) << "Wait request " << wait_id << ": " << node_ids.size()
|
||||||
<< " locations found for object " << lookup_object_id;
|
<< " locations found for object " << lookup_object_id;
|
||||||
wait_state.requested_objects.erase(lookup_object_id);
|
wait_state.requested_objects.erase(lookup_object_id);
|
||||||
if (wait_state.requested_objects.empty()) {
|
if (wait_state.requested_objects.empty()) {
|
||||||
|
@ -643,7 +641,7 @@ void ObjectManager::SubscribeRemainingWaitObjects(const UniqueID &wait_id) {
|
||||||
RAY_CHECK_OK(object_directory_->SubscribeObjectLocations(
|
RAY_CHECK_OK(object_directory_->SubscribeObjectLocations(
|
||||||
wait_id, object_id, wait_state.owner_addresses[object_id],
|
wait_id, object_id, wait_state.owner_addresses[object_id],
|
||||||
[this, wait_id](const ObjectID &subscribe_object_id,
|
[this, wait_id](const ObjectID &subscribe_object_id,
|
||||||
const std::unordered_set<NodeID> &client_ids,
|
const std::unordered_set<NodeID> &node_ids,
|
||||||
const std::string &spilled_url) {
|
const std::string &spilled_url) {
|
||||||
auto object_id_wait_state = active_wait_requests_.find(wait_id);
|
auto object_id_wait_state = active_wait_requests_.find(wait_id);
|
||||||
if (object_id_wait_state == active_wait_requests_.end()) {
|
if (object_id_wait_state == active_wait_requests_.end()) {
|
||||||
|
@ -656,7 +654,7 @@ void ObjectManager::SubscribeRemainingWaitObjects(const UniqueID &wait_id) {
|
||||||
auto &wait_state = object_id_wait_state->second;
|
auto &wait_state = object_id_wait_state->second;
|
||||||
// Note that the object is guaranteed to be added to local_objects_ before
|
// Note that the object is guaranteed to be added to local_objects_ before
|
||||||
// the notification is triggered.
|
// the notification is triggered.
|
||||||
bool remote_object_ready = !client_ids.empty() || !spilled_url.empty();
|
bool remote_object_ready = !node_ids.empty() || !spilled_url.empty();
|
||||||
if (local_objects_.count(subscribe_object_id) > 0 ||
|
if (local_objects_.count(subscribe_object_id) > 0 ||
|
||||||
(!wait_state.wait_local && remote_object_ready)) {
|
(!wait_state.wait_local && remote_object_ready)) {
|
||||||
RAY_LOG(DEBUG) << "Wait request " << wait_id
|
RAY_LOG(DEBUG) << "Wait request " << wait_id
|
||||||
|
@ -736,7 +734,7 @@ void ObjectManager::WaitComplete(const UniqueID &wait_id) {
|
||||||
void ObjectManager::HandlePush(const rpc::PushRequest &request, rpc::PushReply *reply,
|
void ObjectManager::HandlePush(const rpc::PushRequest &request, rpc::PushReply *reply,
|
||||||
rpc::SendReplyCallback send_reply_callback) {
|
rpc::SendReplyCallback send_reply_callback) {
|
||||||
ObjectID object_id = ObjectID::FromBinary(request.object_id());
|
ObjectID object_id = ObjectID::FromBinary(request.object_id());
|
||||||
NodeID client_id = NodeID::FromBinary(request.client_id());
|
NodeID node_id = NodeID::FromBinary(request.node_id());
|
||||||
|
|
||||||
// Serialize.
|
// Serialize.
|
||||||
uint64_t chunk_index = request.chunk_index();
|
uint64_t chunk_index = request.chunk_index();
|
||||||
|
@ -746,21 +744,21 @@ void ObjectManager::HandlePush(const rpc::PushRequest &request, rpc::PushReply *
|
||||||
const std::string &data = request.data();
|
const std::string &data = request.data();
|
||||||
|
|
||||||
double start_time = absl::GetCurrentTimeNanos() / 1e9;
|
double start_time = absl::GetCurrentTimeNanos() / 1e9;
|
||||||
auto status = ReceiveObjectChunk(client_id, object_id, owner_address, data_size,
|
auto status = ReceiveObjectChunk(node_id, object_id, owner_address, data_size,
|
||||||
metadata_size, chunk_index, data);
|
metadata_size, chunk_index, data);
|
||||||
double end_time = absl::GetCurrentTimeNanos() / 1e9;
|
double end_time = absl::GetCurrentTimeNanos() / 1e9;
|
||||||
|
|
||||||
HandleReceiveFinished(object_id, client_id, chunk_index, start_time, end_time, status);
|
HandleReceiveFinished(object_id, node_id, chunk_index, start_time, end_time, status);
|
||||||
send_reply_callback(status, nullptr, nullptr);
|
send_reply_callback(status, nullptr, nullptr);
|
||||||
}
|
}
|
||||||
|
|
||||||
ray::Status ObjectManager::ReceiveObjectChunk(const NodeID &client_id,
|
ray::Status ObjectManager::ReceiveObjectChunk(const NodeID &node_id,
|
||||||
const ObjectID &object_id,
|
const ObjectID &object_id,
|
||||||
const rpc::Address &owner_address,
|
const rpc::Address &owner_address,
|
||||||
uint64_t data_size, uint64_t metadata_size,
|
uint64_t data_size, uint64_t metadata_size,
|
||||||
uint64_t chunk_index,
|
uint64_t chunk_index,
|
||||||
const std::string &data) {
|
const std::string &data) {
|
||||||
RAY_LOG(DEBUG) << "ReceiveObjectChunk on " << self_node_id_ << " from " << client_id
|
RAY_LOG(DEBUG) << "ReceiveObjectChunk on " << self_node_id_ << " from " << node_id
|
||||||
<< " of object " << object_id << " chunk index: " << chunk_index
|
<< " of object " << object_id << " chunk index: " << chunk_index
|
||||||
<< ", chunk data size: " << data.size()
|
<< ", chunk data size: " << data.size()
|
||||||
<< ", object size: " << data_size;
|
<< ", object size: " << data_size;
|
||||||
|
@ -788,22 +786,21 @@ ray::Status ObjectManager::ReceiveObjectChunk(const NodeID &client_id,
|
||||||
void ObjectManager::HandlePull(const rpc::PullRequest &request, rpc::PullReply *reply,
|
void ObjectManager::HandlePull(const rpc::PullRequest &request, rpc::PullReply *reply,
|
||||||
rpc::SendReplyCallback send_reply_callback) {
|
rpc::SendReplyCallback send_reply_callback) {
|
||||||
ObjectID object_id = ObjectID::FromBinary(request.object_id());
|
ObjectID object_id = ObjectID::FromBinary(request.object_id());
|
||||||
NodeID client_id = NodeID::FromBinary(request.client_id());
|
NodeID node_id = NodeID::FromBinary(request.node_id());
|
||||||
RAY_LOG(DEBUG) << "Received pull request from client " << client_id << " for object ["
|
RAY_LOG(DEBUG) << "Received pull request from node " << node_id << " for object ["
|
||||||
<< object_id << "].";
|
<< object_id << "].";
|
||||||
|
|
||||||
rpc::ProfileTableData::ProfileEvent profile_event;
|
rpc::ProfileTableData::ProfileEvent profile_event;
|
||||||
profile_event.set_event_type("receive_pull_request");
|
profile_event.set_event_type("receive_pull_request");
|
||||||
profile_event.set_start_time(absl::GetCurrentTimeNanos() / 1e9);
|
profile_event.set_start_time(absl::GetCurrentTimeNanos() / 1e9);
|
||||||
profile_event.set_end_time(profile_event.start_time());
|
profile_event.set_end_time(profile_event.start_time());
|
||||||
profile_event.set_extra_data("[\"" + object_id.Hex() + "\",\"" + client_id.Hex() +
|
profile_event.set_extra_data("[\"" + object_id.Hex() + "\",\"" + node_id.Hex() + "\"]");
|
||||||
"\"]");
|
|
||||||
{
|
{
|
||||||
std::lock_guard<std::mutex> lock(profile_mutex_);
|
std::lock_guard<std::mutex> lock(profile_mutex_);
|
||||||
profile_events_.emplace_back(profile_event);
|
profile_events_.emplace_back(profile_event);
|
||||||
}
|
}
|
||||||
|
|
||||||
main_service_->post([this, object_id, client_id]() { Push(object_id, client_id); });
|
main_service_->post([this, object_id, node_id]() { Push(object_id, node_id); });
|
||||||
send_reply_callback(Status::OK(), nullptr, nullptr);
|
send_reply_callback(Status::OK(), nullptr, nullptr);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -825,7 +822,7 @@ void ObjectManager::FreeObjects(const std::vector<ObjectID> &object_ids,
|
||||||
const auto remote_connections = object_directory_->LookupAllRemoteConnections();
|
const auto remote_connections = object_directory_->LookupAllRemoteConnections();
|
||||||
std::vector<std::shared_ptr<rpc::ObjectManagerClient>> rpc_clients;
|
std::vector<std::shared_ptr<rpc::ObjectManagerClient>> rpc_clients;
|
||||||
for (const auto &connection_info : remote_connections) {
|
for (const auto &connection_info : remote_connections) {
|
||||||
auto rpc_client = GetRpcClient(connection_info.client_id);
|
auto rpc_client = GetRpcClient(connection_info.node_id);
|
||||||
if (rpc_client != nullptr) {
|
if (rpc_client != nullptr) {
|
||||||
rpc_clients.push_back(rpc_client);
|
rpc_clients.push_back(rpc_client);
|
||||||
}
|
}
|
||||||
|
@ -856,10 +853,10 @@ void ObjectManager::SpreadFreeObjectsRequest(
|
||||||
}
|
}
|
||||||
|
|
||||||
std::shared_ptr<rpc::ObjectManagerClient> ObjectManager::GetRpcClient(
|
std::shared_ptr<rpc::ObjectManagerClient> ObjectManager::GetRpcClient(
|
||||||
const NodeID &client_id) {
|
const NodeID &node_id) {
|
||||||
auto it = remote_object_manager_clients_.find(client_id);
|
auto it = remote_object_manager_clients_.find(node_id);
|
||||||
if (it == remote_object_manager_clients_.end()) {
|
if (it == remote_object_manager_clients_.end()) {
|
||||||
RemoteConnectionInfo connection_info(client_id);
|
RemoteConnectionInfo connection_info(node_id);
|
||||||
object_directory_->LookupRemoteConnectionInfo(connection_info);
|
object_directory_->LookupRemoteConnectionInfo(connection_info);
|
||||||
if (!connection_info.Connected()) {
|
if (!connection_info.Connected()) {
|
||||||
return nullptr;
|
return nullptr;
|
||||||
|
@ -871,8 +868,7 @@ std::shared_ptr<rpc::ObjectManagerClient> ObjectManager::GetRpcClient(
|
||||||
<< ", port: " << connection_info.port
|
<< ", port: " << connection_info.port
|
||||||
<< ", local port: " << GetServerPort();
|
<< ", local port: " << GetServerPort();
|
||||||
|
|
||||||
it = remote_object_manager_clients_
|
it = remote_object_manager_clients_.emplace(node_id, std::move(object_manager_client))
|
||||||
.emplace(client_id, std::move(object_manager_client))
|
|
||||||
.first;
|
.first;
|
||||||
}
|
}
|
||||||
return it->second;
|
return it->second;
|
||||||
|
|
|
@ -50,7 +50,7 @@ struct ObjectManagerConfig {
|
||||||
/// its own port.
|
/// its own port.
|
||||||
int object_manager_port;
|
int object_manager_port;
|
||||||
/// The time in milliseconds to wait before retrying a pull
|
/// The time in milliseconds to wait before retrying a pull
|
||||||
/// that fails due to client id lookup.
|
/// that fails due to node id lookup.
|
||||||
unsigned int pull_timeout_ms;
|
unsigned int pull_timeout_ms;
|
||||||
/// Object chunk size, in bytes
|
/// Object chunk size, in bytes
|
||||||
uint64_t object_chunk_size;
|
uint64_t object_chunk_size;
|
||||||
|
@ -141,28 +141,28 @@ class ObjectManager : public ObjectManagerInterface,
|
||||||
/// \param push_id Unique push id to indicate this push request
|
/// \param push_id Unique push id to indicate this push request
|
||||||
/// \param object_id Object id
|
/// \param object_id Object id
|
||||||
/// \param owner_address The address of the object's owner
|
/// \param owner_address The address of the object's owner
|
||||||
/// \param client_id The id of the receiver.
|
/// \param node_id The id of the receiver.
|
||||||
/// \param data_size Data size
|
/// \param data_size Data size
|
||||||
/// \param metadata_size Metadata size
|
/// \param metadata_size Metadata size
|
||||||
/// \param chunk_index Chunk index of this object chunk, start with 0
|
/// \param chunk_index Chunk index of this object chunk, start with 0
|
||||||
/// \param rpc_client Rpc client used to send message to remote object manager
|
/// \param rpc_client Rpc client used to send message to remote object manager
|
||||||
/// \param on_complete Callback to run on completion.
|
/// \param on_complete Callback to run on completion.
|
||||||
void SendObjectChunk(const UniqueID &push_id, const ObjectID &object_id,
|
void SendObjectChunk(const UniqueID &push_id, const ObjectID &object_id,
|
||||||
const rpc::Address &owner_address, const NodeID &client_id,
|
const rpc::Address &owner_address, const NodeID &node_id,
|
||||||
uint64_t data_size, uint64_t metadata_size, uint64_t chunk_index,
|
uint64_t data_size, uint64_t metadata_size, uint64_t chunk_index,
|
||||||
std::shared_ptr<rpc::ObjectManagerClient> rpc_client,
|
std::shared_ptr<rpc::ObjectManagerClient> rpc_client,
|
||||||
std::function<void(const Status &)> on_complete);
|
std::function<void(const Status &)> on_complete);
|
||||||
|
|
||||||
/// Receive object chunk from remote object manager, small object may contain one chunk
|
/// Receive object chunk from remote object manager, small object may contain one chunk
|
||||||
///
|
///
|
||||||
/// \param client_id Client id of remote object manager which sends this chunk
|
/// \param node_id Node id of remote object manager which sends this chunk
|
||||||
/// \param object_id Object id
|
/// \param object_id Object id
|
||||||
/// \param owner_address The address of the object's owner
|
/// \param owner_address The address of the object's owner
|
||||||
/// \param data_size Data size
|
/// \param data_size Data size
|
||||||
/// \param metadata_size Metadata size
|
/// \param metadata_size Metadata size
|
||||||
/// \param chunk_index Chunk index
|
/// \param chunk_index Chunk index
|
||||||
/// \param data Chunk data
|
/// \param data Chunk data
|
||||||
ray::Status ReceiveObjectChunk(const NodeID &client_id, const ObjectID &object_id,
|
ray::Status ReceiveObjectChunk(const NodeID &node_id, const ObjectID &object_id,
|
||||||
const rpc::Address &owner_address, uint64_t data_size,
|
const rpc::Address &owner_address, uint64_t data_size,
|
||||||
uint64_t metadata_size, uint64_t chunk_index,
|
uint64_t metadata_size, uint64_t chunk_index,
|
||||||
const std::string &data);
|
const std::string &data);
|
||||||
|
@ -170,14 +170,14 @@ class ObjectManager : public ObjectManagerInterface,
|
||||||
/// Send pull request
|
/// Send pull request
|
||||||
///
|
///
|
||||||
/// \param object_id Object id
|
/// \param object_id Object id
|
||||||
/// \param client_id Remote server client id
|
/// \param node_id Remote server node id
|
||||||
void SendPullRequest(const ObjectID &object_id, const NodeID &client_id,
|
void SendPullRequest(const ObjectID &object_id, const NodeID &node_id,
|
||||||
std::shared_ptr<rpc::ObjectManagerClient> rpc_client);
|
std::shared_ptr<rpc::ObjectManagerClient> rpc_client);
|
||||||
|
|
||||||
/// Get the rpc client according to the client ID
|
/// Get the rpc client according to the node ID
|
||||||
///
|
///
|
||||||
/// \param client_id Remote client id, will send rpc request to it
|
/// \param node_id Remote node id, will send rpc request to it
|
||||||
std::shared_ptr<rpc::ObjectManagerClient> GetRpcClient(const NodeID &client_id);
|
std::shared_ptr<rpc::ObjectManagerClient> GetRpcClient(const NodeID &node_id);
|
||||||
|
|
||||||
/// Get the port of the object manager rpc server.
|
/// Get the port of the object manager rpc server.
|
||||||
int GetServerPort() const { return object_manager_server_.GetPort(); }
|
int GetServerPort() const { return object_manager_server_.GetPort(); }
|
||||||
|
@ -223,9 +223,9 @@ class ObjectManager : public ObjectManagerInterface,
|
||||||
/// on the same object, the second one might be ignored).
|
/// on the same object, the second one might be ignored).
|
||||||
///
|
///
|
||||||
/// \param object_id The object's object id.
|
/// \param object_id The object's object id.
|
||||||
/// \param client_id The remote node's client id.
|
/// \param node_id The remote node's id.
|
||||||
/// \return Void.
|
/// \return Void.
|
||||||
void Push(const ObjectID &object_id, const NodeID &client_id);
|
void Push(const ObjectID &object_id, const NodeID &node_id);
|
||||||
|
|
||||||
/// Pull an object from NodeID.
|
/// Pull an object from NodeID.
|
||||||
///
|
///
|
||||||
|
@ -233,9 +233,9 @@ class ObjectManager : public ObjectManagerInterface,
|
||||||
/// \return Status of whether the pull request successfully initiated.
|
/// \return Status of whether the pull request successfully initiated.
|
||||||
ray::Status Pull(const ObjectID &object_id, const rpc::Address &owner_address) override;
|
ray::Status Pull(const ObjectID &object_id, const rpc::Address &owner_address) override;
|
||||||
|
|
||||||
/// Try to Pull an object from one of its expected client locations. If there
|
/// Try to Pull an object from one of its expected node locations. If there
|
||||||
/// are more client locations to try after this attempt, then this method
|
/// are more node locations to try after this attempt, then this method
|
||||||
/// will try each of the other clients in succession, with a timeout between
|
/// will try each of the other nodes in succession, with a timeout between
|
||||||
/// each attempt. If the object is received or if the Pull is Canceled before
|
/// each attempt. If the object is received or if the Pull is Canceled before
|
||||||
/// the timeout, then no more Pull requests for this object will be sent
|
/// the timeout, then no more Pull requests for this object will be sent
|
||||||
/// to other node managers until TryPull is called again.
|
/// to other node managers until TryPull is called again.
|
||||||
|
@ -295,10 +295,10 @@ class ObjectManager : public ObjectManagerInterface,
|
||||||
friend class TestObjectManager;
|
friend class TestObjectManager;
|
||||||
|
|
||||||
struct PullRequest {
|
struct PullRequest {
|
||||||
PullRequest() : retry_timer(nullptr), timer_set(false), client_locations() {}
|
PullRequest() : retry_timer(nullptr), timer_set(false), node_locations() {}
|
||||||
std::unique_ptr<boost::asio::deadline_timer> retry_timer;
|
std::unique_ptr<boost::asio::deadline_timer> retry_timer;
|
||||||
bool timer_set;
|
bool timer_set;
|
||||||
std::vector<NodeID> client_locations;
|
std::vector<NodeID> node_locations;
|
||||||
};
|
};
|
||||||
|
|
||||||
struct WaitState {
|
struct WaitState {
|
||||||
|
@ -373,7 +373,7 @@ class ObjectManager : public ObjectManagerInterface,
|
||||||
/// completed.
|
/// completed.
|
||||||
///
|
///
|
||||||
/// \param object_id The ID of the object that was sent.
|
/// \param object_id The ID of the object that was sent.
|
||||||
/// \param client_id The ID of the client that the chunk was sent to.
|
/// \param node_id The ID of the node that the chunk was sent to.
|
||||||
/// \param chunk_index The index of the chunk.
|
/// \param chunk_index The index of the chunk.
|
||||||
/// \param start_time_us The time when the object manager began sending the
|
/// \param start_time_us The time when the object manager began sending the
|
||||||
/// chunk.
|
/// chunk.
|
||||||
|
@ -381,7 +381,7 @@ class ObjectManager : public ObjectManagerInterface,
|
||||||
/// chunk.
|
/// chunk.
|
||||||
/// \param status The status of the send (e.g., did it succeed or fail).
|
/// \param status The status of the send (e.g., did it succeed or fail).
|
||||||
/// \return Void.
|
/// \return Void.
|
||||||
void HandleSendFinished(const ObjectID &object_id, const NodeID &client_id,
|
void HandleSendFinished(const ObjectID &object_id, const NodeID &node_id,
|
||||||
uint64_t chunk_index, double start_time_us, double end_time_us,
|
uint64_t chunk_index, double start_time_us, double end_time_us,
|
||||||
ray::Status status);
|
ray::Status status);
|
||||||
|
|
||||||
|
@ -389,7 +389,7 @@ class ObjectManager : public ObjectManagerInterface,
|
||||||
/// completed.
|
/// completed.
|
||||||
///
|
///
|
||||||
/// \param object_id The ID of the object that was received.
|
/// \param object_id The ID of the object that was received.
|
||||||
/// \param client_id The ID of the client that the chunk was received from.
|
/// \param node_id The ID of the node that the chunk was received from.
|
||||||
/// \param chunk_index The index of the chunk.
|
/// \param chunk_index The index of the chunk.
|
||||||
/// \param start_time_us The time when the object manager began receiving the
|
/// \param start_time_us The time when the object manager began receiving the
|
||||||
/// chunk.
|
/// chunk.
|
||||||
|
@ -397,12 +397,12 @@ class ObjectManager : public ObjectManagerInterface,
|
||||||
/// chunk.
|
/// chunk.
|
||||||
/// \param status The status of the receive (e.g., did it succeed or fail).
|
/// \param status The status of the receive (e.g., did it succeed or fail).
|
||||||
/// \return Void.
|
/// \return Void.
|
||||||
void HandleReceiveFinished(const ObjectID &object_id, const NodeID &client_id,
|
void HandleReceiveFinished(const ObjectID &object_id, const NodeID &node_id,
|
||||||
uint64_t chunk_index, double start_time_us,
|
uint64_t chunk_index, double start_time_us,
|
||||||
double end_time_us, ray::Status status);
|
double end_time_us, ray::Status status);
|
||||||
|
|
||||||
/// Handle Push task timeout.
|
/// Handle Push task timeout.
|
||||||
void HandlePushTaskTimeout(const ObjectID &object_id, const NodeID &client_id);
|
void HandlePushTaskTimeout(const ObjectID &object_id, const NodeID &node_id);
|
||||||
|
|
||||||
NodeID self_node_id_;
|
NodeID self_node_id_;
|
||||||
const ObjectManagerConfig config_;
|
const ObjectManagerConfig config_;
|
||||||
|
|
|
@ -22,9 +22,9 @@ OwnershipBasedObjectDirectory::OwnershipBasedObjectDirectory(
|
||||||
|
|
||||||
namespace {
|
namespace {
|
||||||
|
|
||||||
/// Filter out the removed clients from the object locations.
|
/// Filter out the removed nodes from the object locations.
|
||||||
void FilterRemovedClients(std::shared_ptr<gcs::GcsClient> gcs_client,
|
void FilterRemovedNodes(std::shared_ptr<gcs::GcsClient> gcs_client,
|
||||||
std::unordered_set<NodeID> *node_ids) {
|
std::unordered_set<NodeID> *node_ids) {
|
||||||
for (auto it = node_ids->begin(); it != node_ids->end();) {
|
for (auto it = node_ids->begin(); it != node_ids->end();) {
|
||||||
if (gcs_client->Nodes().IsRemoved(*it)) {
|
if (gcs_client->Nodes().IsRemoved(*it)) {
|
||||||
it = node_ids->erase(it);
|
it = node_ids->erase(it);
|
||||||
|
@ -64,7 +64,7 @@ std::shared_ptr<rpc::CoreWorkerClient> OwnershipBasedObjectDirectory::GetClient(
|
||||||
}
|
}
|
||||||
|
|
||||||
ray::Status OwnershipBasedObjectDirectory::ReportObjectAdded(
|
ray::Status OwnershipBasedObjectDirectory::ReportObjectAdded(
|
||||||
const ObjectID &object_id, const NodeID &client_id,
|
const ObjectID &object_id, const NodeID &node_id,
|
||||||
const object_manager::protocol::ObjectInfoT &object_info) {
|
const object_manager::protocol::ObjectInfoT &object_info) {
|
||||||
WorkerID worker_id = WorkerID::FromBinary(object_info.owner_worker_id);
|
WorkerID worker_id = WorkerID::FromBinary(object_info.owner_worker_id);
|
||||||
rpc::Address owner_address = GetOwnerAddressFromObjectInfo(object_info);
|
rpc::Address owner_address = GetOwnerAddressFromObjectInfo(object_info);
|
||||||
|
@ -77,7 +77,7 @@ ray::Status OwnershipBasedObjectDirectory::ReportObjectAdded(
|
||||||
rpc::AddObjectLocationOwnerRequest request;
|
rpc::AddObjectLocationOwnerRequest request;
|
||||||
request.set_intended_worker_id(object_info.owner_worker_id);
|
request.set_intended_worker_id(object_info.owner_worker_id);
|
||||||
request.set_object_id(object_id.Binary());
|
request.set_object_id(object_id.Binary());
|
||||||
request.set_client_id(client_id.Binary());
|
request.set_node_id(node_id.Binary());
|
||||||
|
|
||||||
rpc_client->AddObjectLocationOwner(
|
rpc_client->AddObjectLocationOwner(
|
||||||
request, [worker_id, object_id](Status status,
|
request, [worker_id, object_id](Status status,
|
||||||
|
@ -91,7 +91,7 @@ ray::Status OwnershipBasedObjectDirectory::ReportObjectAdded(
|
||||||
}
|
}
|
||||||
|
|
||||||
ray::Status OwnershipBasedObjectDirectory::ReportObjectRemoved(
|
ray::Status OwnershipBasedObjectDirectory::ReportObjectRemoved(
|
||||||
const ObjectID &object_id, const NodeID &client_id,
|
const ObjectID &object_id, const NodeID &node_id,
|
||||||
const object_manager::protocol::ObjectInfoT &object_info) {
|
const object_manager::protocol::ObjectInfoT &object_info) {
|
||||||
WorkerID worker_id = WorkerID::FromBinary(object_info.owner_worker_id);
|
WorkerID worker_id = WorkerID::FromBinary(object_info.owner_worker_id);
|
||||||
rpc::Address owner_address = GetOwnerAddressFromObjectInfo(object_info);
|
rpc::Address owner_address = GetOwnerAddressFromObjectInfo(object_info);
|
||||||
|
@ -105,7 +105,7 @@ ray::Status OwnershipBasedObjectDirectory::ReportObjectRemoved(
|
||||||
rpc::RemoveObjectLocationOwnerRequest request;
|
rpc::RemoveObjectLocationOwnerRequest request;
|
||||||
request.set_intended_worker_id(object_info.owner_worker_id);
|
request.set_intended_worker_id(object_info.owner_worker_id);
|
||||||
request.set_object_id(object_id.Binary());
|
request.set_object_id(object_id.Binary());
|
||||||
request.set_client_id(client_id.Binary());
|
request.set_node_id(node_id.Binary());
|
||||||
|
|
||||||
rpc_client->RemoveObjectLocationOwner(
|
rpc_client->RemoveObjectLocationOwner(
|
||||||
request, [worker_id, object_id](Status status,
|
request, [worker_id, object_id](Status status,
|
||||||
|
@ -126,13 +126,13 @@ void OwnershipBasedObjectDirectory::SubscriptionCallback(
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
std::unordered_set<NodeID> client_ids;
|
std::unordered_set<NodeID> node_ids;
|
||||||
for (auto const &client_id : reply.client_ids()) {
|
for (auto const &node_id : reply.node_ids()) {
|
||||||
client_ids.emplace(NodeID::FromBinary(client_id));
|
node_ids.emplace(NodeID::FromBinary(node_id));
|
||||||
}
|
}
|
||||||
FilterRemovedClients(gcs_client_, &client_ids);
|
FilterRemovedNodes(gcs_client_, &node_ids);
|
||||||
if (client_ids != it->second.current_object_locations) {
|
if (node_ids != it->second.current_object_locations) {
|
||||||
it->second.current_object_locations = std::move(client_ids);
|
it->second.current_object_locations = std::move(node_ids);
|
||||||
auto callbacks = it->second.callbacks;
|
auto callbacks = it->second.callbacks;
|
||||||
// Call all callbacks associated with the object id locations we have
|
// Call all callbacks associated with the object id locations we have
|
||||||
// received. This notifies the client even if the list of locations is
|
// received. This notifies the client even if the list of locations is
|
||||||
|
@ -224,12 +224,12 @@ ray::Status OwnershipBasedObjectDirectory::LookupLocations(
|
||||||
RAY_LOG(ERROR) << "Worker " << worker_id << " failed to get the location for "
|
RAY_LOG(ERROR) << "Worker " << worker_id << " failed to get the location for "
|
||||||
<< object_id;
|
<< object_id;
|
||||||
}
|
}
|
||||||
std::unordered_set<NodeID> client_ids;
|
std::unordered_set<NodeID> node_ids;
|
||||||
for (auto const &client_id : reply.client_ids()) {
|
for (auto const &node_id : reply.node_ids()) {
|
||||||
client_ids.emplace(NodeID::FromBinary(client_id));
|
node_ids.emplace(NodeID::FromBinary(node_id));
|
||||||
}
|
}
|
||||||
FilterRemovedClients(gcs_client_, &client_ids);
|
FilterRemovedNodes(gcs_client_, &node_ids);
|
||||||
callback(object_id, client_ids, "");
|
callback(object_id, node_ids, "");
|
||||||
});
|
});
|
||||||
return Status::OK();
|
return Status::OK();
|
||||||
}
|
}
|
||||||
|
|
|
@ -37,7 +37,7 @@ class OwnershipBasedObjectDirectory : public ObjectDirectory {
|
||||||
///
|
///
|
||||||
/// \param io_service The event loop to dispatch callbacks to. This should
|
/// \param io_service The event loop to dispatch callbacks to. This should
|
||||||
/// usually be the same event loop that the given gcs_client runs on.
|
/// usually be the same event loop that the given gcs_client runs on.
|
||||||
/// \param gcs_client A Ray GCS client to request object and client
|
/// \param gcs_client A Ray GCS client to request object and node
|
||||||
/// information from.
|
/// information from.
|
||||||
OwnershipBasedObjectDirectory(boost::asio::io_service &io_service,
|
OwnershipBasedObjectDirectory(boost::asio::io_service &io_service,
|
||||||
std::shared_ptr<gcs::GcsClient> &gcs_client);
|
std::shared_ptr<gcs::GcsClient> &gcs_client);
|
||||||
|
@ -56,10 +56,10 @@ class OwnershipBasedObjectDirectory : public ObjectDirectory {
|
||||||
const ObjectID &object_id) override;
|
const ObjectID &object_id) override;
|
||||||
|
|
||||||
ray::Status ReportObjectAdded(
|
ray::Status ReportObjectAdded(
|
||||||
const ObjectID &object_id, const NodeID &client_id,
|
const ObjectID &object_id, const NodeID &node_id,
|
||||||
const object_manager::protocol::ObjectInfoT &object_info) override;
|
const object_manager::protocol::ObjectInfoT &object_info) override;
|
||||||
ray::Status ReportObjectRemoved(
|
ray::Status ReportObjectRemoved(
|
||||||
const ObjectID &object_id, const NodeID &client_id,
|
const ObjectID &object_id, const NodeID &node_id,
|
||||||
const object_manager::protocol::ObjectInfoT &object_info) override;
|
const object_manager::protocol::ObjectInfoT &object_info) override;
|
||||||
|
|
||||||
std::string DebugString() const override;
|
std::string DebugString() const override;
|
||||||
|
|
|
@ -22,15 +22,15 @@ namespace ray {
|
||||||
TEST(TestPushManager, TestSingleTransfer) {
|
TEST(TestPushManager, TestSingleTransfer) {
|
||||||
std::vector<int> results;
|
std::vector<int> results;
|
||||||
results.reserve(10);
|
results.reserve(10);
|
||||||
auto client_id = NodeID::FromRandom();
|
auto node_id = NodeID::FromRandom();
|
||||||
auto obj_id = ObjectID::FromRandom();
|
auto obj_id = ObjectID::FromRandom();
|
||||||
PushManager pm(5);
|
PushManager pm(5);
|
||||||
pm.StartPush(client_id, obj_id, 10, [&](int64_t chunk_id) { results[chunk_id] = 1; });
|
pm.StartPush(node_id, obj_id, 10, [&](int64_t chunk_id) { results[chunk_id] = 1; });
|
||||||
ASSERT_EQ(pm.NumChunksInFlight(), 5);
|
ASSERT_EQ(pm.NumChunksInFlight(), 5);
|
||||||
ASSERT_EQ(pm.NumChunksRemaining(), 10);
|
ASSERT_EQ(pm.NumChunksRemaining(), 10);
|
||||||
ASSERT_EQ(pm.NumPushesInFlight(), 1);
|
ASSERT_EQ(pm.NumPushesInFlight(), 1);
|
||||||
for (int i = 0; i < 10; i++) {
|
for (int i = 0; i < 10; i++) {
|
||||||
pm.OnChunkComplete(client_id, obj_id);
|
pm.OnChunkComplete(node_id, obj_id);
|
||||||
}
|
}
|
||||||
ASSERT_EQ(pm.NumChunksInFlight(), 0);
|
ASSERT_EQ(pm.NumChunksInFlight(), 0);
|
||||||
ASSERT_EQ(pm.NumChunksRemaining(), 0);
|
ASSERT_EQ(pm.NumChunksRemaining(), 0);
|
||||||
|
@ -43,20 +43,20 @@ TEST(TestPushManager, TestSingleTransfer) {
|
||||||
TEST(TestPushManager, TestSuppressDuplicates) {
|
TEST(TestPushManager, TestSuppressDuplicates) {
|
||||||
std::vector<int> results;
|
std::vector<int> results;
|
||||||
results.reserve(10);
|
results.reserve(10);
|
||||||
auto client_id = NodeID::FromRandom();
|
auto node_id = NodeID::FromRandom();
|
||||||
auto obj_id = ObjectID::FromRandom();
|
auto obj_id = ObjectID::FromRandom();
|
||||||
PushManager pm(5);
|
PushManager pm(5);
|
||||||
|
|
||||||
// First send.
|
// First send.
|
||||||
pm.StartPush(client_id, obj_id, 10, [&](int64_t chunk_id) { results[chunk_id] = 1; });
|
pm.StartPush(node_id, obj_id, 10, [&](int64_t chunk_id) { results[chunk_id] = 1; });
|
||||||
// Duplicates are all ignored.
|
// Duplicates are all ignored.
|
||||||
pm.StartPush(client_id, obj_id, 10, [&](int64_t chunk_id) { results[chunk_id] = 2; });
|
pm.StartPush(node_id, obj_id, 10, [&](int64_t chunk_id) { results[chunk_id] = 2; });
|
||||||
ASSERT_EQ(pm.NumChunksInFlight(), 5);
|
ASSERT_EQ(pm.NumChunksInFlight(), 5);
|
||||||
ASSERT_EQ(pm.NumChunksRemaining(), 10);
|
ASSERT_EQ(pm.NumChunksRemaining(), 10);
|
||||||
ASSERT_EQ(pm.NumPushesInFlight(), 1);
|
ASSERT_EQ(pm.NumPushesInFlight(), 1);
|
||||||
for (int i = 0; i < 10; i++) {
|
for (int i = 0; i < 10; i++) {
|
||||||
pm.StartPush(client_id, obj_id, 10, [&](int64_t chunk_id) { results[chunk_id] = 2; });
|
pm.StartPush(node_id, obj_id, 10, [&](int64_t chunk_id) { results[chunk_id] = 2; });
|
||||||
pm.OnChunkComplete(client_id, obj_id);
|
pm.OnChunkComplete(node_id, obj_id);
|
||||||
}
|
}
|
||||||
ASSERT_EQ(pm.NumChunksInFlight(), 0);
|
ASSERT_EQ(pm.NumChunksInFlight(), 0);
|
||||||
ASSERT_EQ(pm.NumChunksRemaining(), 0);
|
ASSERT_EQ(pm.NumChunksRemaining(), 0);
|
||||||
|
@ -66,9 +66,9 @@ TEST(TestPushManager, TestSuppressDuplicates) {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Second allowed send.
|
// Second allowed send.
|
||||||
pm.StartPush(client_id, obj_id, 10, [&](int64_t chunk_id) { results[chunk_id] = 3; });
|
pm.StartPush(node_id, obj_id, 10, [&](int64_t chunk_id) { results[chunk_id] = 3; });
|
||||||
for (int i = 0; i < 10; i++) {
|
for (int i = 0; i < 10; i++) {
|
||||||
pm.OnChunkComplete(client_id, obj_id);
|
pm.OnChunkComplete(node_id, obj_id);
|
||||||
}
|
}
|
||||||
ASSERT_EQ(pm.NumChunksInFlight(), 0);
|
ASSERT_EQ(pm.NumChunksInFlight(), 0);
|
||||||
ASSERT_EQ(pm.NumChunksRemaining(), 0);
|
ASSERT_EQ(pm.NumChunksRemaining(), 0);
|
||||||
|
@ -83,17 +83,17 @@ TEST(TestPushManager, TestMultipleTransfers) {
|
||||||
results1.reserve(10);
|
results1.reserve(10);
|
||||||
std::vector<int> results2;
|
std::vector<int> results2;
|
||||||
results2.reserve(10);
|
results2.reserve(10);
|
||||||
auto client1 = NodeID::FromRandom();
|
auto node1 = NodeID::FromRandom();
|
||||||
auto client2 = NodeID::FromRandom();
|
auto node2 = NodeID::FromRandom();
|
||||||
auto obj_id = ObjectID::FromRandom();
|
auto obj_id = ObjectID::FromRandom();
|
||||||
int num_active1 = 0;
|
int num_active1 = 0;
|
||||||
int num_active2 = 0;
|
int num_active2 = 0;
|
||||||
PushManager pm(5);
|
PushManager pm(5);
|
||||||
pm.StartPush(client1, obj_id, 10, [&](int64_t chunk_id) {
|
pm.StartPush(node1, obj_id, 10, [&](int64_t chunk_id) {
|
||||||
results1[chunk_id] = 1;
|
results1[chunk_id] = 1;
|
||||||
num_active1++;
|
num_active1++;
|
||||||
});
|
});
|
||||||
pm.StartPush(client2, obj_id, 10, [&](int64_t chunk_id) {
|
pm.StartPush(node2, obj_id, 10, [&](int64_t chunk_id) {
|
||||||
results2[chunk_id] = 2;
|
results2[chunk_id] = 2;
|
||||||
num_active2++;
|
num_active2++;
|
||||||
});
|
});
|
||||||
|
@ -102,10 +102,10 @@ TEST(TestPushManager, TestMultipleTransfers) {
|
||||||
ASSERT_EQ(pm.NumPushesInFlight(), 2);
|
ASSERT_EQ(pm.NumPushesInFlight(), 2);
|
||||||
for (int i = 0; i < 20; i++) {
|
for (int i = 0; i < 20; i++) {
|
||||||
if (num_active1 > 0) {
|
if (num_active1 > 0) {
|
||||||
pm.OnChunkComplete(client1, obj_id);
|
pm.OnChunkComplete(node1, obj_id);
|
||||||
num_active1--;
|
num_active1--;
|
||||||
} else if (num_active2 > 0) {
|
} else if (num_active2 > 0) {
|
||||||
pm.OnChunkComplete(client2, obj_id);
|
pm.OnChunkComplete(node2, obj_id);
|
||||||
num_active2--;
|
num_active2--;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -164,7 +164,7 @@ message WaitForObjectEvictionReply {
|
||||||
message AddObjectLocationOwnerRequest {
|
message AddObjectLocationOwnerRequest {
|
||||||
bytes intended_worker_id = 1;
|
bytes intended_worker_id = 1;
|
||||||
bytes object_id = 2;
|
bytes object_id = 2;
|
||||||
bytes client_id = 3;
|
bytes node_id = 3;
|
||||||
}
|
}
|
||||||
|
|
||||||
message AddObjectLocationOwnerReply {
|
message AddObjectLocationOwnerReply {
|
||||||
|
@ -173,7 +173,7 @@ message AddObjectLocationOwnerReply {
|
||||||
message RemoveObjectLocationOwnerRequest {
|
message RemoveObjectLocationOwnerRequest {
|
||||||
bytes intended_worker_id = 1;
|
bytes intended_worker_id = 1;
|
||||||
bytes object_id = 2;
|
bytes object_id = 2;
|
||||||
bytes client_id = 3;
|
bytes node_id = 3;
|
||||||
}
|
}
|
||||||
|
|
||||||
message RemoveObjectLocationOwnerReply {
|
message RemoveObjectLocationOwnerReply {
|
||||||
|
@ -185,7 +185,7 @@ message GetObjectLocationsOwnerRequest {
|
||||||
}
|
}
|
||||||
|
|
||||||
message GetObjectLocationsOwnerReply {
|
message GetObjectLocationsOwnerReply {
|
||||||
repeated bytes client_ids = 1;
|
repeated bytes node_ids = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
message KillActorRequest {
|
message KillActorRequest {
|
||||||
|
|
|
@ -26,7 +26,7 @@ enum TablePrefix {
|
||||||
UNUSED = 1;
|
UNUSED = 1;
|
||||||
TASK = 2;
|
TASK = 2;
|
||||||
RAYLET_TASK = 3;
|
RAYLET_TASK = 3;
|
||||||
CLIENT = 4;
|
NODE = 4;
|
||||||
OBJECT = 5;
|
OBJECT = 5;
|
||||||
ACTOR = 6;
|
ACTOR = 6;
|
||||||
FUNCTION = 7;
|
FUNCTION = 7;
|
||||||
|
@ -54,7 +54,7 @@ enum TablePubsub {
|
||||||
NO_PUBLISH = 1;
|
NO_PUBLISH = 1;
|
||||||
TASK_PUBSUB = 2;
|
TASK_PUBSUB = 2;
|
||||||
RAYLET_TASK_PUBSUB = 3;
|
RAYLET_TASK_PUBSUB = 3;
|
||||||
CLIENT_PUBSUB = 4;
|
NODE_PUBSUB = 4;
|
||||||
OBJECT_PUBSUB = 5;
|
OBJECT_PUBSUB = 5;
|
||||||
ACTOR_PUBSUB = 6;
|
ACTOR_PUBSUB = 6;
|
||||||
HEARTBEAT_PUBSUB = 7;
|
HEARTBEAT_PUBSUB = 7;
|
||||||
|
@ -231,7 +231,7 @@ message ResourceTableData {
|
||||||
}
|
}
|
||||||
|
|
||||||
message AvailableResources {
|
message AvailableResources {
|
||||||
// Node manager client id.
|
// Node id.
|
||||||
bytes node_id = 1;
|
bytes node_id = 1;
|
||||||
// Resource capacity currently available on this node manager.
|
// Resource capacity currently available on this node manager.
|
||||||
map<string, double> resources_available = 2;
|
map<string, double> resources_available = 2;
|
||||||
|
@ -302,8 +302,8 @@ message PlacementGroupLoad {
|
||||||
}
|
}
|
||||||
|
|
||||||
message HeartbeatTableData {
|
message HeartbeatTableData {
|
||||||
// Node manager client id
|
// Node id.
|
||||||
bytes client_id = 1;
|
bytes node_id = 1;
|
||||||
// Resource capacity currently available on this node manager.
|
// Resource capacity currently available on this node manager.
|
||||||
map<string, double> resources_available = 2;
|
map<string, double> resources_available = 2;
|
||||||
// Indicates whether avaialbe resources is changed. Only used when
|
// Indicates whether avaialbe resources is changed. Only used when
|
||||||
|
@ -323,8 +323,8 @@ message HeartbeatTableData {
|
||||||
}
|
}
|
||||||
|
|
||||||
message ResourcesData {
|
message ResourcesData {
|
||||||
// Node manager client id
|
// Node id.
|
||||||
bytes client_id = 1;
|
bytes node_id = 1;
|
||||||
// Resource capacity currently available on this node manager.
|
// Resource capacity currently available on this node manager.
|
||||||
map<string, double> resources_available = 2;
|
map<string, double> resources_available = 2;
|
||||||
// Indicates whether avaialbe resources is changed. Only used when
|
// Indicates whether avaialbe resources is changed. Only used when
|
||||||
|
@ -356,7 +356,7 @@ message HeartbeatBatchTableData {
|
||||||
message TaskLeaseData {
|
message TaskLeaseData {
|
||||||
// The task ID.
|
// The task ID.
|
||||||
bytes task_id = 1;
|
bytes task_id = 1;
|
||||||
// Node manager client ID.
|
// Node manager ID.
|
||||||
bytes node_manager_id = 2;
|
bytes node_manager_id = 2;
|
||||||
// The time that the lease was last acquired at. NOTE(swang): This is the
|
// The time that the lease was last acquired at. NOTE(swang): This is the
|
||||||
// system clock time according to the node that added the entry and is not
|
// system clock time according to the node that added the entry and is not
|
||||||
|
|
|
@ -24,8 +24,8 @@ message PushRequest {
|
||||||
bytes push_id = 1;
|
bytes push_id = 1;
|
||||||
// The object ID being transferred.
|
// The object ID being transferred.
|
||||||
bytes object_id = 2;
|
bytes object_id = 2;
|
||||||
// The client ID of client sending this object
|
// The node ID of client sending this object
|
||||||
bytes client_id = 3;
|
bytes node_id = 3;
|
||||||
// The owner address
|
// The owner address
|
||||||
Address owner_address = 4;
|
Address owner_address = 4;
|
||||||
// The index of the chunk being transferred.
|
// The index of the chunk being transferred.
|
||||||
|
@ -39,8 +39,8 @@ message PushRequest {
|
||||||
}
|
}
|
||||||
|
|
||||||
message PullRequest {
|
message PullRequest {
|
||||||
// ID of the requesting client.
|
// Node ID of the requesting client.
|
||||||
bytes client_id = 1;
|
bytes node_id = 1;
|
||||||
// Requested ObjectID.
|
// Requested ObjectID.
|
||||||
bytes object_id = 2;
|
bytes object_id = 2;
|
||||||
}
|
}
|
||||||
|
|
|
@ -289,8 +289,8 @@ table SetResourceRequest {
|
||||||
resource_name: string;
|
resource_name: string;
|
||||||
// Capacity of the resource to be set.
|
// Capacity of the resource to be set.
|
||||||
capacity: double;
|
capacity: double;
|
||||||
// Client ID where this resource will be set.
|
// Node ID where this resource will be set.
|
||||||
client_id: string;
|
node_id: string;
|
||||||
}
|
}
|
||||||
|
|
||||||
table SubscribePlasmaReady {
|
table SubscribePlasmaReady {
|
||||||
|
|
|
@ -415,7 +415,7 @@ void NodeManager::Heartbeat() {
|
||||||
|
|
||||||
auto heartbeat_data = std::make_shared<HeartbeatTableData>();
|
auto heartbeat_data = std::make_shared<HeartbeatTableData>();
|
||||||
SchedulingResources &local_resources = cluster_resource_map_[self_node_id_];
|
SchedulingResources &local_resources = cluster_resource_map_[self_node_id_];
|
||||||
heartbeat_data->set_client_id(self_node_id_.Binary());
|
heartbeat_data->set_node_id(self_node_id_.Binary());
|
||||||
|
|
||||||
if (new_scheduler_enabled_) {
|
if (new_scheduler_enabled_) {
|
||||||
new_resource_scheduler_->Heartbeat(light_heartbeat_enabled_, heartbeat_data);
|
new_resource_scheduler_->Heartbeat(light_heartbeat_enabled_, heartbeat_data);
|
||||||
|
@ -711,7 +711,7 @@ void NodeManager::GetObjectManagerProfileInfo() {
|
||||||
void NodeManager::NodeAdded(const GcsNodeInfo &node_info) {
|
void NodeManager::NodeAdded(const GcsNodeInfo &node_info) {
|
||||||
const NodeID node_id = NodeID::FromBinary(node_info.node_id());
|
const NodeID node_id = NodeID::FromBinary(node_info.node_id());
|
||||||
|
|
||||||
RAY_LOG(DEBUG) << "[NodeAdded] Received callback from client id " << node_id;
|
RAY_LOG(DEBUG) << "[NodeAdded] Received callback from node id " << node_id;
|
||||||
if (1 == cluster_resource_map_.count(node_id)) {
|
if (1 == cluster_resource_map_.count(node_id)) {
|
||||||
RAY_LOG(DEBUG) << "Received notification of a new node that already exists: "
|
RAY_LOG(DEBUG) << "Received notification of a new node that already exists: "
|
||||||
<< node_id;
|
<< node_id;
|
||||||
|
@ -729,7 +729,7 @@ void NodeManager::NodeAdded(const GcsNodeInfo &node_info) {
|
||||||
remote_node_manager_addresses_[node_id] =
|
remote_node_manager_addresses_[node_id] =
|
||||||
std::make_pair(node_info.node_manager_address(), node_info.node_manager_port());
|
std::make_pair(node_info.node_manager_address(), node_info.node_manager_port());
|
||||||
|
|
||||||
// Fetch resource info for the remote client and update cluster resource map.
|
// Fetch resource info for the remote node and update cluster resource map.
|
||||||
RAY_CHECK_OK(gcs_client_->Nodes().AsyncGetResources(
|
RAY_CHECK_OK(gcs_client_->Nodes().AsyncGetResources(
|
||||||
node_id,
|
node_id,
|
||||||
[this, node_id](Status status,
|
[this, node_id](Status status,
|
||||||
|
@ -749,7 +749,7 @@ void NodeManager::NodeRemoved(const GcsNodeInfo &node_info) {
|
||||||
// TODO(swang): If we receive a notification for our own death, clean up and
|
// TODO(swang): If we receive a notification for our own death, clean up and
|
||||||
// exit immediately.
|
// exit immediately.
|
||||||
const NodeID node_id = NodeID::FromBinary(node_info.node_id());
|
const NodeID node_id = NodeID::FromBinary(node_info.node_id());
|
||||||
RAY_LOG(DEBUG) << "[NodeRemoved] Received callback from client id " << node_id;
|
RAY_LOG(DEBUG) << "[NodeRemoved] Received callback from node id " << node_id;
|
||||||
|
|
||||||
RAY_CHECK(node_id != self_node_id_)
|
RAY_CHECK(node_id != self_node_id_)
|
||||||
<< "Exiting because this node manager has mistakenly been marked dead by the "
|
<< "Exiting because this node manager has mistakenly been marked dead by the "
|
||||||
|
@ -762,14 +762,14 @@ void NodeManager::NodeRemoved(const GcsNodeInfo &node_info) {
|
||||||
// check that it is actually removed, or log a warning otherwise, but that may
|
// check that it is actually removed, or log a warning otherwise, but that may
|
||||||
// not be necessary.
|
// not be necessary.
|
||||||
|
|
||||||
// Remove the client from the resource map.
|
// Remove the node from the resource map.
|
||||||
if (0 == cluster_resource_map_.erase(node_id)) {
|
if (0 == cluster_resource_map_.erase(node_id)) {
|
||||||
RAY_LOG(DEBUG) << "Received NodeRemoved callback for an unknown node: " << node_id
|
RAY_LOG(DEBUG) << "Received NodeRemoved callback for an unknown node: " << node_id
|
||||||
<< ".";
|
<< ".";
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
// Remove the client from the resource map.
|
// Remove the node from the resource map.
|
||||||
if (new_scheduler_enabled_) {
|
if (new_scheduler_enabled_) {
|
||||||
if (!new_resource_scheduler_->RemoveNode(node_id.Binary())) {
|
if (!new_resource_scheduler_->RemoveNode(node_id.Binary())) {
|
||||||
RAY_LOG(DEBUG) << "Received NodeRemoved callback for an unknown node: " << node_id
|
RAY_LOG(DEBUG) << "Received NodeRemoved callback for an unknown node: " << node_id
|
||||||
|
@ -779,14 +779,14 @@ void NodeManager::NodeRemoved(const GcsNodeInfo &node_info) {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Remove the node manager address.
|
// Remove the node manager address.
|
||||||
const auto client_entry = remote_node_manager_addresses_.find(node_id);
|
const auto node_entry = remote_node_manager_addresses_.find(node_id);
|
||||||
if (client_entry != remote_node_manager_addresses_.end()) {
|
if (node_entry != remote_node_manager_addresses_.end()) {
|
||||||
remote_node_manager_addresses_.erase(client_entry);
|
remote_node_manager_addresses_.erase(node_entry);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Notify the object directory that the client has been removed so that it
|
// Notify the object directory that the node has been removed so that it
|
||||||
// can remove it from any cached locations.
|
// can remove it from any cached locations.
|
||||||
object_directory_->HandleClientRemoved(node_id);
|
object_directory_->HandleNodeRemoved(node_id);
|
||||||
|
|
||||||
// Clean up workers that were owned by processes that were on the failed
|
// Clean up workers that were owned by processes that were on the failed
|
||||||
// node.
|
// node.
|
||||||
|
@ -839,13 +839,13 @@ void NodeManager::HandleUnexpectedWorkerFailure(const rpc::Address &address) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void NodeManager::ResourceCreateUpdated(const NodeID &client_id,
|
void NodeManager::ResourceCreateUpdated(const NodeID &node_id,
|
||||||
const ResourceSet &createUpdatedResources) {
|
const ResourceSet &createUpdatedResources) {
|
||||||
RAY_LOG(DEBUG) << "[ResourceCreateUpdated] received callback from client id "
|
RAY_LOG(DEBUG) << "[ResourceCreateUpdated] received callback from node id " << node_id
|
||||||
<< client_id << " with created or updated resources: "
|
<< " with created or updated resources: "
|
||||||
<< createUpdatedResources.ToString() << ". Updating resource map.";
|
<< createUpdatedResources.ToString() << ". Updating resource map.";
|
||||||
|
|
||||||
SchedulingResources &cluster_schedres = cluster_resource_map_[client_id];
|
SchedulingResources &cluster_schedres = cluster_resource_map_[node_id];
|
||||||
|
|
||||||
// Update local_available_resources_ and SchedulingResources
|
// Update local_available_resources_ and SchedulingResources
|
||||||
for (const auto &resource_pair : createUpdatedResources.GetResourceMap()) {
|
for (const auto &resource_pair : createUpdatedResources.GetResourceMap()) {
|
||||||
|
@ -853,46 +853,46 @@ void NodeManager::ResourceCreateUpdated(const NodeID &client_id,
|
||||||
const double &new_resource_capacity = resource_pair.second;
|
const double &new_resource_capacity = resource_pair.second;
|
||||||
|
|
||||||
cluster_schedres.UpdateResourceCapacity(resource_label, new_resource_capacity);
|
cluster_schedres.UpdateResourceCapacity(resource_label, new_resource_capacity);
|
||||||
if (client_id == self_node_id_) {
|
if (node_id == self_node_id_) {
|
||||||
local_available_resources_.AddOrUpdateResource(resource_label,
|
local_available_resources_.AddOrUpdateResource(resource_label,
|
||||||
new_resource_capacity);
|
new_resource_capacity);
|
||||||
}
|
}
|
||||||
if (new_scheduler_enabled_) {
|
if (new_scheduler_enabled_) {
|
||||||
new_resource_scheduler_->UpdateResourceCapacity(client_id.Binary(), resource_label,
|
new_resource_scheduler_->UpdateResourceCapacity(node_id.Binary(), resource_label,
|
||||||
new_resource_capacity);
|
new_resource_capacity);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
RAY_LOG(DEBUG) << "[ResourceCreateUpdated] Updated cluster_resource_map.";
|
RAY_LOG(DEBUG) << "[ResourceCreateUpdated] Updated cluster_resource_map.";
|
||||||
|
|
||||||
if (client_id == self_node_id_) {
|
if (node_id == self_node_id_) {
|
||||||
// The resource update is on the local node, check if we can reschedule tasks.
|
// The resource update is on the local node, check if we can reschedule tasks.
|
||||||
TryLocalInfeasibleTaskScheduling();
|
TryLocalInfeasibleTaskScheduling();
|
||||||
}
|
}
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
void NodeManager::ResourceDeleted(const NodeID &client_id,
|
void NodeManager::ResourceDeleted(const NodeID &node_id,
|
||||||
const std::vector<std::string> &resource_names) {
|
const std::vector<std::string> &resource_names) {
|
||||||
if (RAY_LOG_ENABLED(DEBUG)) {
|
if (RAY_LOG_ENABLED(DEBUG)) {
|
||||||
std::ostringstream oss;
|
std::ostringstream oss;
|
||||||
for (auto &resource_name : resource_names) {
|
for (auto &resource_name : resource_names) {
|
||||||
oss << resource_name << ", ";
|
oss << resource_name << ", ";
|
||||||
}
|
}
|
||||||
RAY_LOG(DEBUG) << "[ResourceDeleted] received callback from client id " << client_id
|
RAY_LOG(DEBUG) << "[ResourceDeleted] received callback from node id " << node_id
|
||||||
<< " with deleted resources: " << oss.str()
|
<< " with deleted resources: " << oss.str()
|
||||||
<< ". Updating resource map.";
|
<< ". Updating resource map.";
|
||||||
}
|
}
|
||||||
|
|
||||||
SchedulingResources &cluster_schedres = cluster_resource_map_[client_id];
|
SchedulingResources &cluster_schedres = cluster_resource_map_[node_id];
|
||||||
|
|
||||||
// Update local_available_resources_ and SchedulingResources
|
// Update local_available_resources_ and SchedulingResources
|
||||||
for (const auto &resource_label : resource_names) {
|
for (const auto &resource_label : resource_names) {
|
||||||
cluster_schedres.DeleteResource(resource_label);
|
cluster_schedres.DeleteResource(resource_label);
|
||||||
if (client_id == self_node_id_) {
|
if (node_id == self_node_id_) {
|
||||||
local_available_resources_.DeleteResource(resource_label);
|
local_available_resources_.DeleteResource(resource_label);
|
||||||
}
|
}
|
||||||
if (new_scheduler_enabled_) {
|
if (new_scheduler_enabled_) {
|
||||||
new_resource_scheduler_->DeleteResource(client_id.Binary(), resource_label);
|
new_resource_scheduler_->DeleteResource(node_id.Binary(), resource_label);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return;
|
return;
|
||||||
|
@ -918,15 +918,15 @@ void NodeManager::TryLocalInfeasibleTaskScheduling() {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void NodeManager::HeartbeatAdded(const NodeID &client_id,
|
void NodeManager::HeartbeatAdded(const NodeID &node_id,
|
||||||
const HeartbeatTableData &heartbeat_data) {
|
const HeartbeatTableData &heartbeat_data) {
|
||||||
// Locate the client id in remote client table and update available resources based on
|
// Locate the node id in remote node table and update available resources based on
|
||||||
// the received heartbeat information.
|
// the received heartbeat information.
|
||||||
auto it = cluster_resource_map_.find(client_id);
|
auto it = cluster_resource_map_.find(node_id);
|
||||||
if (it == cluster_resource_map_.end()) {
|
if (it == cluster_resource_map_.end()) {
|
||||||
// Haven't received the client registration for this client yet, skip this heartbeat.
|
// Haven't received the node registration for this node yet, skip this heartbeat.
|
||||||
RAY_LOG(INFO) << "[HeartbeatAdded]: received heartbeat from unknown client id "
|
RAY_LOG(INFO) << "[HeartbeatAdded]: received heartbeat from unknown node id "
|
||||||
<< client_id;
|
<< node_id;
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
// Trigger local GC at the next heartbeat interval.
|
// Trigger local GC at the next heartbeat interval.
|
||||||
|
@ -963,9 +963,9 @@ void NodeManager::HeartbeatAdded(const NodeID &client_id,
|
||||||
remote_resources.SetLoadResources(std::move(remote_load));
|
remote_resources.SetLoadResources(std::move(remote_load));
|
||||||
}
|
}
|
||||||
|
|
||||||
if (new_scheduler_enabled_ && client_id != self_node_id_) {
|
if (new_scheduler_enabled_ && node_id != self_node_id_) {
|
||||||
new_resource_scheduler_->AddOrUpdateNode(
|
new_resource_scheduler_->AddOrUpdateNode(
|
||||||
client_id.Binary(), remote_resources.GetTotalResources().GetResourceMap(),
|
node_id.Binary(), remote_resources.GetTotalResources().GetResourceMap(),
|
||||||
remote_resources.GetAvailableResources().GetResourceMap());
|
remote_resources.GetAvailableResources().GetResourceMap());
|
||||||
// TODO(swang): We could probably call this once per batch instead of once
|
// TODO(swang): We could probably call this once per batch instead of once
|
||||||
// per node in the batch.
|
// per node in the batch.
|
||||||
|
@ -993,19 +993,19 @@ void NodeManager::HeartbeatAdded(const NodeID &client_id,
|
||||||
}
|
}
|
||||||
// Attempt to forward the task. If this fails to forward the task,
|
// Attempt to forward the task. If this fails to forward the task,
|
||||||
// the task will be resubmit locally.
|
// the task will be resubmit locally.
|
||||||
ForwardTaskOrResubmit(task, client_id);
|
ForwardTaskOrResubmit(task, node_id);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void NodeManager::HeartbeatBatchAdded(const HeartbeatBatchTableData &heartbeat_batch) {
|
void NodeManager::HeartbeatBatchAdded(const HeartbeatBatchTableData &heartbeat_batch) {
|
||||||
// Update load information provided by each heartbeat.
|
// Update load information provided by each heartbeat.
|
||||||
for (const auto &heartbeat_data : heartbeat_batch.batch()) {
|
for (const auto &heartbeat_data : heartbeat_batch.batch()) {
|
||||||
const NodeID &client_id = NodeID::FromBinary(heartbeat_data.client_id());
|
const NodeID &node_id = NodeID::FromBinary(heartbeat_data.node_id());
|
||||||
if (client_id == self_node_id_) {
|
if (node_id == self_node_id_) {
|
||||||
// Skip heartbeats from self.
|
// Skip heartbeats from self.
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
HeartbeatAdded(client_id, heartbeat_data);
|
HeartbeatAdded(node_id, heartbeat_data);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1975,7 +1975,7 @@ void NodeManager::ProcessSetResourceRequest(
|
||||||
double const &capacity = message->capacity();
|
double const &capacity = message->capacity();
|
||||||
bool is_deletion = capacity <= 0;
|
bool is_deletion = capacity <= 0;
|
||||||
|
|
||||||
NodeID node_id = from_flatbuf<NodeID>(*message->client_id());
|
NodeID node_id = from_flatbuf<NodeID>(*message->node_id());
|
||||||
|
|
||||||
// If the python arg was null, set node_id to the local node id.
|
// If the python arg was null, set node_id to the local node id.
|
||||||
if (node_id.IsNil()) {
|
if (node_id.IsNil()) {
|
||||||
|
|
|
@ -176,7 +176,7 @@ class NodeManager : public rpc::NodeManagerServiceHandler {
|
||||||
LocalObjectManager &GetLocalObjectManager() { return local_object_manager_; }
|
LocalObjectManager &GetLocalObjectManager() { return local_object_manager_; }
|
||||||
|
|
||||||
private:
|
private:
|
||||||
/// Methods for handling clients.
|
/// Methods for handling nodes.
|
||||||
|
|
||||||
/// Handle an unexpected failure notification from GCS pubsub.
|
/// Handle an unexpected failure notification from GCS pubsub.
|
||||||
///
|
///
|
||||||
|
@ -195,21 +195,21 @@ class NodeManager : public rpc::NodeManagerServiceHandler {
|
||||||
void NodeRemoved(const GcsNodeInfo &node_info);
|
void NodeRemoved(const GcsNodeInfo &node_info);
|
||||||
|
|
||||||
/// Handler for the addition or updation of a resource in the GCS
|
/// Handler for the addition or updation of a resource in the GCS
|
||||||
/// \param client_id ID of the node that created or updated resources.
|
/// \param node_id ID of the node that created or updated resources.
|
||||||
/// \param createUpdatedResources Created or updated resources.
|
/// \param createUpdatedResources Created or updated resources.
|
||||||
/// \return Void.
|
/// \return Void.
|
||||||
void ResourceCreateUpdated(const NodeID &client_id,
|
void ResourceCreateUpdated(const NodeID &node_id,
|
||||||
const ResourceSet &createUpdatedResources);
|
const ResourceSet &createUpdatedResources);
|
||||||
|
|
||||||
/// Handler for the deletion of a resource in the GCS
|
/// Handler for the deletion of a resource in the GCS
|
||||||
/// \param client_id ID of the node that deleted resources.
|
/// \param node_id ID of the node that deleted resources.
|
||||||
/// \param resource_names Names of deleted resources.
|
/// \param resource_names Names of deleted resources.
|
||||||
/// \return Void.
|
/// \return Void.
|
||||||
void ResourceDeleted(const NodeID &client_id,
|
void ResourceDeleted(const NodeID &node_id,
|
||||||
const std::vector<std::string> &resource_names);
|
const std::vector<std::string> &resource_names);
|
||||||
|
|
||||||
/// Evaluates the local infeasible queue to check if any tasks can be scheduled.
|
/// Evaluates the local infeasible queue to check if any tasks can be scheduled.
|
||||||
/// This is called whenever there's an update to the resources on the local client.
|
/// This is called whenever there's an update to the resources on the local node.
|
||||||
/// \return Void.
|
/// \return Void.
|
||||||
void TryLocalInfeasibleTaskScheduling();
|
void TryLocalInfeasibleTaskScheduling();
|
||||||
|
|
||||||
|
@ -814,7 +814,7 @@ class NodeManager : public rpc::NodeManagerServiceHandler {
|
||||||
// TODO(swang): Evict entries from these caches.
|
// TODO(swang): Evict entries from these caches.
|
||||||
/// Cache for the WorkerTable in the GCS.
|
/// Cache for the WorkerTable in the GCS.
|
||||||
absl::flat_hash_set<WorkerID> failed_workers_cache_;
|
absl::flat_hash_set<WorkerID> failed_workers_cache_;
|
||||||
/// Cache for the ClientTable in the GCS.
|
/// Cache for the NodeTable in the GCS.
|
||||||
absl::flat_hash_set<NodeID> failed_nodes_cache_;
|
absl::flat_hash_set<NodeID> failed_nodes_cache_;
|
||||||
|
|
||||||
/// Concurrency for the following map
|
/// Concurrency for the following map
|
||||||
|
|
|
@ -23,13 +23,13 @@ namespace raylet {
|
||||||
ReconstructionPolicy::ReconstructionPolicy(
|
ReconstructionPolicy::ReconstructionPolicy(
|
||||||
boost::asio::io_service &io_service,
|
boost::asio::io_service &io_service,
|
||||||
std::function<void(const TaskID &, const ObjectID &)> reconstruction_handler,
|
std::function<void(const TaskID &, const ObjectID &)> reconstruction_handler,
|
||||||
int64_t initial_reconstruction_timeout_ms, const NodeID &client_id,
|
int64_t initial_reconstruction_timeout_ms, const NodeID &node_id,
|
||||||
std::shared_ptr<gcs::GcsClient> gcs_client,
|
std::shared_ptr<gcs::GcsClient> gcs_client,
|
||||||
std::shared_ptr<ObjectDirectoryInterface> object_directory)
|
std::shared_ptr<ObjectDirectoryInterface> object_directory)
|
||||||
: io_service_(io_service),
|
: io_service_(io_service),
|
||||||
reconstruction_handler_(reconstruction_handler),
|
reconstruction_handler_(reconstruction_handler),
|
||||||
initial_reconstruction_timeout_ms_(initial_reconstruction_timeout_ms),
|
initial_reconstruction_timeout_ms_(initial_reconstruction_timeout_ms),
|
||||||
client_id_(client_id),
|
node_id_(node_id),
|
||||||
gcs_client_(gcs_client),
|
gcs_client_(gcs_client),
|
||||||
object_directory_(std::move(object_directory)) {}
|
object_directory_(std::move(object_directory)) {}
|
||||||
|
|
||||||
|
@ -148,7 +148,7 @@ void ReconstructionPolicy::AttemptReconstruction(const TaskID &task_id,
|
||||||
auto reconstruction_entry = std::make_shared<TaskReconstructionData>();
|
auto reconstruction_entry = std::make_shared<TaskReconstructionData>();
|
||||||
reconstruction_entry->set_task_id(task_id.Binary());
|
reconstruction_entry->set_task_id(task_id.Binary());
|
||||||
reconstruction_entry->set_num_reconstructions(reconstruction_attempt);
|
reconstruction_entry->set_num_reconstructions(reconstruction_attempt);
|
||||||
reconstruction_entry->set_node_manager_id(client_id_.Binary());
|
reconstruction_entry->set_node_manager_id(node_id_.Binary());
|
||||||
RAY_CHECK_OK(gcs_client_->Tasks().AttemptTaskReconstruction(
|
RAY_CHECK_OK(gcs_client_->Tasks().AttemptTaskReconstruction(
|
||||||
reconstruction_entry,
|
reconstruction_entry,
|
||||||
/*done=*/
|
/*done=*/
|
||||||
|
@ -178,10 +178,9 @@ void ReconstructionPolicy::HandleTaskLeaseExpired(const TaskID &task_id) {
|
||||||
RAY_CHECK_OK(object_directory_->LookupLocations(
|
RAY_CHECK_OK(object_directory_->LookupLocations(
|
||||||
created_object_id, it->second.owner_addresses[created_object_id],
|
created_object_id, it->second.owner_addresses[created_object_id],
|
||||||
[this, task_id, reconstruction_attempt](
|
[this, task_id, reconstruction_attempt](
|
||||||
const ray::ObjectID &object_id,
|
const ray::ObjectID &object_id, const std::unordered_set<ray::NodeID> &nodes,
|
||||||
const std::unordered_set<ray::NodeID> &clients,
|
|
||||||
const std::string &spilled_url) {
|
const std::string &spilled_url) {
|
||||||
if (clients.empty() && spilled_url.empty()) {
|
if (nodes.empty() && spilled_url.empty()) {
|
||||||
// The required object no longer exists on any live nodes. Attempt
|
// The required object no longer exists on any live nodes. Attempt
|
||||||
// reconstruction.
|
// reconstruction.
|
||||||
AttemptReconstruction(task_id, object_id, reconstruction_attempt);
|
AttemptReconstruction(task_id, object_id, reconstruction_attempt);
|
||||||
|
|
|
@ -47,14 +47,14 @@ class ReconstructionPolicy : public ReconstructionPolicyInterface {
|
||||||
/// \param initial_reconstruction_timeout_ms The initial timeout within which
|
/// \param initial_reconstruction_timeout_ms The initial timeout within which
|
||||||
/// a task lease notification must be received. Otherwise, reconstruction
|
/// a task lease notification must be received. Otherwise, reconstruction
|
||||||
/// will be triggered.
|
/// will be triggered.
|
||||||
/// \param client_id The client ID to use when requesting notifications from
|
/// \param node_id The node ID to use when requesting notifications from
|
||||||
/// the GCS.
|
/// the GCS.
|
||||||
/// \param gcs_client The Client of GCS.
|
/// \param gcs_client The Client of GCS.
|
||||||
/// lease notifications from.
|
/// lease notifications from.
|
||||||
ReconstructionPolicy(
|
ReconstructionPolicy(
|
||||||
boost::asio::io_service &io_service,
|
boost::asio::io_service &io_service,
|
||||||
std::function<void(const TaskID &, const ObjectID &)> reconstruction_handler,
|
std::function<void(const TaskID &, const ObjectID &)> reconstruction_handler,
|
||||||
int64_t initial_reconstruction_timeout_ms, const NodeID &client_id,
|
int64_t initial_reconstruction_timeout_ms, const NodeID &node_id,
|
||||||
std::shared_ptr<gcs::GcsClient> gcs_client,
|
std::shared_ptr<gcs::GcsClient> gcs_client,
|
||||||
std::shared_ptr<ObjectDirectoryInterface> object_directory);
|
std::shared_ptr<ObjectDirectoryInterface> object_directory);
|
||||||
|
|
||||||
|
@ -151,8 +151,8 @@ class ReconstructionPolicy : public ReconstructionPolicyInterface {
|
||||||
/// The initial timeout within which a task lease notification must be
|
/// The initial timeout within which a task lease notification must be
|
||||||
/// received. Otherwise, reconstruction will be triggered.
|
/// received. Otherwise, reconstruction will be triggered.
|
||||||
const int64_t initial_reconstruction_timeout_ms_;
|
const int64_t initial_reconstruction_timeout_ms_;
|
||||||
/// The client ID to use when requesting notifications from the GCS.
|
/// The node ID to use when requesting notifications from the GCS.
|
||||||
const NodeID client_id_;
|
const NodeID node_id_;
|
||||||
/// A client connection to the GCS.
|
/// A client connection to the GCS.
|
||||||
std::shared_ptr<gcs::GcsClient> gcs_client_;
|
std::shared_ptr<gcs::GcsClient> gcs_client_;
|
||||||
/// The object directory used to lookup object locations.
|
/// The object directory used to lookup object locations.
|
||||||
|
|
|
@ -69,15 +69,15 @@ class MockObjectDirectory : public ObjectDirectoryInterface {
|
||||||
locations_[object_id] = locations;
|
locations_[object_id] = locations;
|
||||||
}
|
}
|
||||||
|
|
||||||
void HandleClientRemoved(const NodeID &client_id) override {
|
void HandleNodeRemoved(const NodeID &node_id) override {
|
||||||
for (auto &locations : locations_) {
|
for (auto &locations : locations_) {
|
||||||
locations.second.erase(client_id);
|
locations.second.erase(node_id);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
std::string DebugString() const override { return ""; }
|
std::string DebugString() const override { return ""; }
|
||||||
|
|
||||||
MOCK_METHOD0(GetLocalClientID, ray::NodeID());
|
MOCK_METHOD0(GetLocalNodeID, ray::NodeID());
|
||||||
MOCK_CONST_METHOD1(LookupRemoteConnectionInfo, void(RemoteConnectionInfo &));
|
MOCK_CONST_METHOD1(LookupRemoteConnectionInfo, void(RemoteConnectionInfo &));
|
||||||
MOCK_CONST_METHOD0(LookupAllRemoteConnections, std::vector<RemoteConnectionInfo>());
|
MOCK_CONST_METHOD0(LookupAllRemoteConnections, std::vector<RemoteConnectionInfo>());
|
||||||
MOCK_METHOD4(SubscribeObjectLocations,
|
MOCK_METHOD4(SubscribeObjectLocations,
|
||||||
|
@ -321,8 +321,8 @@ TEST_F(ReconstructionPolicyTest, TestReconstructionEvicted) {
|
||||||
TEST_F(ReconstructionPolicyTest, TestReconstructionObjectLost) {
|
TEST_F(ReconstructionPolicyTest, TestReconstructionObjectLost) {
|
||||||
TaskID task_id = ForNormalTask();
|
TaskID task_id = ForNormalTask();
|
||||||
ObjectID object_id = ObjectID::FromIndex(task_id, /*index=*/1);
|
ObjectID object_id = ObjectID::FromIndex(task_id, /*index=*/1);
|
||||||
NodeID client_id = NodeID::FromRandom();
|
NodeID node_id = NodeID::FromRandom();
|
||||||
mock_object_directory_->SetObjectLocations(object_id, {client_id});
|
mock_object_directory_->SetObjectLocations(object_id, {node_id});
|
||||||
|
|
||||||
// Listen for both objects.
|
// Listen for both objects.
|
||||||
reconstruction_policy_->ListenAndMaybeReconstruct(object_id, rpc::Address());
|
reconstruction_policy_->ListenAndMaybeReconstruct(object_id, rpc::Address());
|
||||||
|
@ -333,7 +333,7 @@ TEST_F(ReconstructionPolicyTest, TestReconstructionObjectLost) {
|
||||||
ASSERT_EQ(reconstructed_tasks_[task_id], 0);
|
ASSERT_EQ(reconstructed_tasks_[task_id], 0);
|
||||||
|
|
||||||
// Simulate evicting one of the objects.
|
// Simulate evicting one of the objects.
|
||||||
mock_object_directory_->HandleClientRemoved(client_id);
|
mock_object_directory_->HandleNodeRemoved(node_id);
|
||||||
// Run the test again.
|
// Run the test again.
|
||||||
Run(reconstruction_timeout_ms_ * 1.1);
|
Run(reconstruction_timeout_ms_ * 1.1);
|
||||||
// Check that reconstruction was triggered, since one of the objects was
|
// Check that reconstruction was triggered, since one of the objects was
|
||||||
|
|
|
@ -329,18 +329,18 @@ bool ClusterResourceScheduler::GetNodeResources(int64_t node_id,
|
||||||
|
|
||||||
int64_t ClusterResourceScheduler::NumNodes() { return nodes_.size(); }
|
int64_t ClusterResourceScheduler::NumNodes() { return nodes_.size(); }
|
||||||
|
|
||||||
void ClusterResourceScheduler::UpdateResourceCapacity(const std::string &client_id_string,
|
void ClusterResourceScheduler::UpdateResourceCapacity(const std::string &node_id_string,
|
||||||
const std::string &resource_name,
|
const std::string &resource_name,
|
||||||
double resource_total) {
|
double resource_total) {
|
||||||
int64_t client_id = string_to_int_map_.Get(client_id_string);
|
int64_t node_id = string_to_int_map_.Get(node_id_string);
|
||||||
|
|
||||||
auto it = nodes_.find(client_id);
|
auto it = nodes_.find(node_id);
|
||||||
if (it == nodes_.end()) {
|
if (it == nodes_.end()) {
|
||||||
NodeResources node_resources;
|
NodeResources node_resources;
|
||||||
node_resources.predefined_resources.resize(PredefinedResources_MAX);
|
node_resources.predefined_resources.resize(PredefinedResources_MAX);
|
||||||
client_id = string_to_int_map_.Insert(client_id_string);
|
node_id = string_to_int_map_.Insert(node_id_string);
|
||||||
RAY_CHECK(nodes_.emplace(client_id, node_resources).second);
|
RAY_CHECK(nodes_.emplace(node_id, node_resources).second);
|
||||||
it = nodes_.find(client_id);
|
it = nodes_.find(node_id);
|
||||||
RAY_CHECK(it != nodes_.end());
|
RAY_CHECK(it != nodes_.end());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -387,10 +387,10 @@ void ClusterResourceScheduler::UpdateResourceCapacity(const std::string &client_
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void ClusterResourceScheduler::DeleteResource(const std::string &client_id_string,
|
void ClusterResourceScheduler::DeleteResource(const std::string &node_id_string,
|
||||||
const std::string &resource_name) {
|
const std::string &resource_name) {
|
||||||
int64_t client_id = string_to_int_map_.Get(client_id_string);
|
int64_t node_id = string_to_int_map_.Get(node_id_string);
|
||||||
auto it = nodes_.find(client_id);
|
auto it = nodes_.find(node_id);
|
||||||
if (it == nodes_.end()) {
|
if (it == nodes_.end()) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
|
@ -30,16 +30,16 @@ SchedulingPolicy::SchedulingPolicy(const SchedulingQueue &scheduling_queue)
|
||||||
|
|
||||||
std::unordered_map<TaskID, NodeID> SchedulingPolicy::Schedule(
|
std::unordered_map<TaskID, NodeID> SchedulingPolicy::Schedule(
|
||||||
std::unordered_map<NodeID, SchedulingResources> &cluster_resources,
|
std::unordered_map<NodeID, SchedulingResources> &cluster_resources,
|
||||||
const NodeID &local_client_id) {
|
const NodeID &local_node_id) {
|
||||||
// The policy decision to be returned.
|
// The policy decision to be returned.
|
||||||
std::unordered_map<TaskID, NodeID> decision;
|
std::unordered_map<TaskID, NodeID> decision;
|
||||||
#ifndef NDEBUG
|
#ifndef NDEBUG
|
||||||
RAY_LOG(DEBUG) << "Cluster resource map: ";
|
RAY_LOG(DEBUG) << "Cluster resource map: ";
|
||||||
for (const auto &client_resource_pair : cluster_resources) {
|
for (const auto &node_resource_pair : cluster_resources) {
|
||||||
// pair = NodeID, SchedulingResources
|
// pair = NodeID, SchedulingResources
|
||||||
const NodeID &client_id = client_resource_pair.first;
|
const NodeID &node_id = node_resource_pair.first;
|
||||||
const SchedulingResources &resources = client_resource_pair.second;
|
const SchedulingResources &resources = node_resource_pair.second;
|
||||||
RAY_LOG(DEBUG) << "client_id: " << client_id << " "
|
RAY_LOG(DEBUG) << "node_id: " << node_id << " "
|
||||||
<< resources.GetAvailableResources().ToString();
|
<< resources.GetAvailableResources().ToString();
|
||||||
}
|
}
|
||||||
#endif
|
#endif
|
||||||
|
@ -54,7 +54,7 @@ std::unordered_map<TaskID, NodeID> SchedulingPolicy::Schedule(
|
||||||
const TaskID &task_id = spec.TaskId();
|
const TaskID &task_id = spec.TaskId();
|
||||||
|
|
||||||
// Try to place tasks locally first.
|
// Try to place tasks locally first.
|
||||||
const auto &local_resources = cluster_resources[local_client_id];
|
const auto &local_resources = cluster_resources[local_node_id];
|
||||||
ResourceSet available_local_resources =
|
ResourceSet available_local_resources =
|
||||||
ResourceSet(local_resources.GetAvailableResources());
|
ResourceSet(local_resources.GetAvailableResources());
|
||||||
// We have to subtract the current "load" because we set the current "load"
|
// We have to subtract the current "load" because we set the current "load"
|
||||||
|
@ -64,21 +64,21 @@ std::unordered_map<TaskID, NodeID> SchedulingPolicy::Schedule(
|
||||||
available_local_resources.SubtractResources(local_resources.GetLoadResources());
|
available_local_resources.SubtractResources(local_resources.GetLoadResources());
|
||||||
if (resource_demand.IsSubset(available_local_resources)) {
|
if (resource_demand.IsSubset(available_local_resources)) {
|
||||||
// This node is a feasible candidate.
|
// This node is a feasible candidate.
|
||||||
decision[task_id] = local_client_id;
|
decision[task_id] = local_node_id;
|
||||||
|
|
||||||
ResourceSet new_load(cluster_resources[local_client_id].GetLoadResources());
|
ResourceSet new_load(cluster_resources[local_node_id].GetLoadResources());
|
||||||
new_load.AddResources(resource_demand);
|
new_load.AddResources(resource_demand);
|
||||||
cluster_resources[local_client_id].SetLoadResources(std::move(new_load));
|
cluster_resources[local_node_id].SetLoadResources(std::move(new_load));
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
// Construct a set of viable node candidates and randomly pick between them.
|
// Construct a set of viable node candidates and randomly pick between them.
|
||||||
// Get all the client id keys and randomly pick.
|
// Get all the node id keys and randomly pick.
|
||||||
std::vector<NodeID> client_keys;
|
std::vector<NodeID> node_keys;
|
||||||
for (const auto &client_resource_pair : cluster_resources) {
|
for (const auto &node_resource_pair : cluster_resources) {
|
||||||
// pair = NodeID, SchedulingResources
|
// pair = NodeID, SchedulingResources
|
||||||
NodeID node_client_id = client_resource_pair.first;
|
NodeID node_id = node_resource_pair.first;
|
||||||
const auto &node_resources = client_resource_pair.second;
|
const auto &node_resources = node_resource_pair.second;
|
||||||
ResourceSet available_node_resources =
|
ResourceSet available_node_resources =
|
||||||
ResourceSet(node_resources.GetAvailableResources());
|
ResourceSet(node_resources.GetAvailableResources());
|
||||||
// We have to subtract the current "load" because we set the current "load"
|
// We have to subtract the current "load" because we set the current "load"
|
||||||
|
@ -86,55 +86,55 @@ std::unordered_map<TaskID, NodeID> SchedulingPolicy::Schedule(
|
||||||
// `SchedulingQueue::ready_queue_` in NodeManager::HandleWorkerAvailable's
|
// `SchedulingQueue::ready_queue_` in NodeManager::HandleWorkerAvailable's
|
||||||
// call to SchedulingQueue::GetResourceLoad.
|
// call to SchedulingQueue::GetResourceLoad.
|
||||||
available_node_resources.SubtractResources(node_resources.GetLoadResources());
|
available_node_resources.SubtractResources(node_resources.GetLoadResources());
|
||||||
RAY_LOG(DEBUG) << "client_id " << node_client_id
|
RAY_LOG(DEBUG) << "node_id " << node_id
|
||||||
<< " avail: " << node_resources.GetAvailableResources().ToString()
|
<< " avail: " << node_resources.GetAvailableResources().ToString()
|
||||||
<< " load: " << node_resources.GetLoadResources().ToString();
|
<< " load: " << node_resources.GetLoadResources().ToString();
|
||||||
|
|
||||||
if (resource_demand.IsSubset(available_node_resources)) {
|
if (resource_demand.IsSubset(available_node_resources)) {
|
||||||
// This node is a feasible candidate.
|
// This node is a feasible candidate.
|
||||||
client_keys.push_back(node_client_id);
|
node_keys.push_back(node_id);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!client_keys.empty()) {
|
if (!node_keys.empty()) {
|
||||||
// Choose index at random.
|
// Choose index at random.
|
||||||
// Initialize a uniform integer distribution over the key space.
|
// Initialize a uniform integer distribution over the key space.
|
||||||
// TODO(atumanov): change uniform random to discrete, weighted by resource capacity.
|
// TODO(atumanov): change uniform random to discrete, weighted by resource capacity.
|
||||||
std::uniform_int_distribution<int> distribution(0, client_keys.size() - 1);
|
std::uniform_int_distribution<int> distribution(0, node_keys.size() - 1);
|
||||||
int client_key_index = distribution(gen_);
|
int node_key_index = distribution(gen_);
|
||||||
const NodeID &dst_client_id = client_keys[client_key_index];
|
const NodeID &dst_node_id = node_keys[node_key_index];
|
||||||
decision[task_id] = dst_client_id;
|
decision[task_id] = dst_node_id;
|
||||||
// Update dst_client_id's load to keep track of remote task load until
|
// Update dst_node_id's load to keep track of remote task load until
|
||||||
// the next heartbeat.
|
// the next heartbeat.
|
||||||
ResourceSet new_load(cluster_resources[dst_client_id].GetLoadResources());
|
ResourceSet new_load(cluster_resources[dst_node_id].GetLoadResources());
|
||||||
new_load.AddResources(resource_demand);
|
new_load.AddResources(resource_demand);
|
||||||
cluster_resources[dst_client_id].SetLoadResources(std::move(new_load));
|
cluster_resources[dst_node_id].SetLoadResources(std::move(new_load));
|
||||||
} else {
|
} else {
|
||||||
// If the task doesn't fit, place randomly subject to hard constraints.
|
// If the task doesn't fit, place randomly subject to hard constraints.
|
||||||
for (const auto &client_resource_pair2 : cluster_resources) {
|
for (const auto &node_resource_pair2 : cluster_resources) {
|
||||||
// pair = NodeID, SchedulingResources
|
// pair = NodeID, SchedulingResources
|
||||||
NodeID node_client_id = client_resource_pair2.first;
|
NodeID node_id = node_resource_pair2.first;
|
||||||
const auto &node_resources = client_resource_pair2.second;
|
const auto &node_resources = node_resource_pair2.second;
|
||||||
if (resource_demand.IsSubset(node_resources.GetTotalResources())) {
|
if (resource_demand.IsSubset(node_resources.GetTotalResources())) {
|
||||||
// This node is a feasible candidate.
|
// This node is a feasible candidate.
|
||||||
client_keys.push_back(node_client_id);
|
node_keys.push_back(node_id);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
// client candidate list constructed, pick randomly.
|
// node candidate list constructed, pick randomly.
|
||||||
if (!client_keys.empty()) {
|
if (!node_keys.empty()) {
|
||||||
// Choose index at random.
|
// Choose index at random.
|
||||||
// Initialize a uniform integer distribution over the key space.
|
// Initialize a uniform integer distribution over the key space.
|
||||||
// TODO(atumanov): change uniform random to discrete, weighted by resource
|
// TODO(atumanov): change uniform random to discrete, weighted by resource
|
||||||
// capacity.
|
// capacity.
|
||||||
std::uniform_int_distribution<int> distribution(0, client_keys.size() - 1);
|
std::uniform_int_distribution<int> distribution(0, node_keys.size() - 1);
|
||||||
int client_key_index = distribution(gen_);
|
int node_key_index = distribution(gen_);
|
||||||
const NodeID &dst_client_id = client_keys[client_key_index];
|
const NodeID &dst_node_id = node_keys[node_key_index];
|
||||||
decision[task_id] = dst_client_id;
|
decision[task_id] = dst_node_id;
|
||||||
// Update dst_client_id's load to keep track of remote task load until
|
// Update dst_node_id's load to keep track of remote task load until
|
||||||
// the next heartbeat.
|
// the next heartbeat.
|
||||||
ResourceSet new_load(cluster_resources[dst_client_id].GetLoadResources());
|
ResourceSet new_load(cluster_resources[dst_node_id].GetLoadResources());
|
||||||
new_load.AddResources(resource_demand);
|
new_load.AddResources(resource_demand);
|
||||||
cluster_resources[dst_client_id].SetLoadResources(std::move(new_load));
|
cluster_resources[dst_node_id].SetLoadResources(std::move(new_load));
|
||||||
} else {
|
} else {
|
||||||
// There are no nodes that can feasibly execute this task. The task remains
|
// There are no nodes that can feasibly execute this task. The task remains
|
||||||
// placeable until cluster capacity becomes available.
|
// placeable until cluster capacity becomes available.
|
||||||
|
@ -143,7 +143,7 @@ std::unordered_map<TaskID, NodeID> SchedulingPolicy::Schedule(
|
||||||
<< spec.GetRequiredResources().ToString() << " for execution and "
|
<< spec.GetRequiredResources().ToString() << " for execution and "
|
||||||
<< spec.GetRequiredPlacementResources().ToString()
|
<< spec.GetRequiredPlacementResources().ToString()
|
||||||
<< " for placement, but no nodes have the necessary resources. "
|
<< " for placement, but no nodes have the necessary resources. "
|
||||||
<< "Check the client table to view node resources.";
|
<< "Check the node table to view node resources.";
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -153,27 +153,27 @@ std::unordered_map<TaskID, NodeID> SchedulingPolicy::Schedule(
|
||||||
|
|
||||||
bool SchedulingPolicy::ScheduleBundle(
|
bool SchedulingPolicy::ScheduleBundle(
|
||||||
std::unordered_map<NodeID, SchedulingResources> &cluster_resources,
|
std::unordered_map<NodeID, SchedulingResources> &cluster_resources,
|
||||||
const NodeID &local_client_id, const ray::BundleSpecification &bundle_spec) {
|
const NodeID &local_node_id, const ray::BundleSpecification &bundle_spec) {
|
||||||
#ifndef NDEBUG
|
#ifndef NDEBUG
|
||||||
RAY_LOG(DEBUG) << "Cluster resource map: ";
|
RAY_LOG(DEBUG) << "Cluster resource map: ";
|
||||||
for (const auto &client_resource_pair : cluster_resources) {
|
for (const auto &node_resource_pair : cluster_resources) {
|
||||||
const NodeID &client_id = client_resource_pair.first;
|
const NodeID &node_id = node_resource_pair.first;
|
||||||
const SchedulingResources &resources = client_resource_pair.second;
|
const SchedulingResources &resources = node_resource_pair.second;
|
||||||
RAY_LOG(DEBUG) << "client_id: " << client_id << " "
|
RAY_LOG(DEBUG) << "node_id: " << node_id << " "
|
||||||
<< resources.GetAvailableResources().ToString();
|
<< resources.GetAvailableResources().ToString();
|
||||||
}
|
}
|
||||||
#endif
|
#endif
|
||||||
const auto &client_resource_pair = cluster_resources.find(local_client_id);
|
const auto &node_resource_pair = cluster_resources.find(local_node_id);
|
||||||
if (client_resource_pair == cluster_resources.end()) {
|
if (node_resource_pair == cluster_resources.end()) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
const auto &resource_demand = bundle_spec.GetRequiredResources();
|
const auto &resource_demand = bundle_spec.GetRequiredResources();
|
||||||
NodeID node_client_id = client_resource_pair->first;
|
NodeID node_id = node_resource_pair->first;
|
||||||
const auto &node_resources = client_resource_pair->second;
|
const auto &node_resources = node_resource_pair->second;
|
||||||
ResourceSet available_node_resources =
|
ResourceSet available_node_resources =
|
||||||
ResourceSet(node_resources.GetAvailableResources());
|
ResourceSet(node_resources.GetAvailableResources());
|
||||||
available_node_resources.SubtractResources(node_resources.GetLoadResources());
|
available_node_resources.SubtractResources(node_resources.GetLoadResources());
|
||||||
RAY_LOG(DEBUG) << "Scheduling bundle, client id = " << node_client_id
|
RAY_LOG(DEBUG) << "Scheduling bundle, node id = " << node_id
|
||||||
<< ", available resources = "
|
<< ", available resources = "
|
||||||
<< node_resources.GetAvailableResources().ToString()
|
<< node_resources.GetAvailableResources().ToString()
|
||||||
<< ", resources load = " << node_resources.GetLoadResources().ToString()
|
<< ", resources load = " << node_resources.GetLoadResources().ToString()
|
||||||
|
|
|
@ -40,25 +40,25 @@ class SchedulingPolicy {
|
||||||
/// producing a mapping of tasks to raylets.
|
/// producing a mapping of tasks to raylets.
|
||||||
///
|
///
|
||||||
/// \param cluster_resources: a set of cluster resources containing resource and load
|
/// \param cluster_resources: a set of cluster resources containing resource and load
|
||||||
/// information for some subset of the cluster. For all client IDs in the returned
|
/// information for some subset of the cluster. For all node IDs in the returned
|
||||||
/// placement map, the corresponding SchedulingResources::resources_load_ is
|
/// placement map, the corresponding SchedulingResources::resources_load_ is
|
||||||
/// incremented by the aggregate resource demand of the tasks assigned to it.
|
/// incremented by the aggregate resource demand of the tasks assigned to it.
|
||||||
/// \param local_client_id The ID of the node manager that owns this
|
/// \param local_node_id The ID of the node manager that owns this
|
||||||
/// SchedulingPolicy object.
|
/// SchedulingPolicy object.
|
||||||
/// \return Scheduling decision, mapping tasks to raylets for placement.
|
/// \return Scheduling decision, mapping tasks to raylets for placement.
|
||||||
std::unordered_map<TaskID, NodeID> Schedule(
|
std::unordered_map<TaskID, NodeID> Schedule(
|
||||||
std::unordered_map<NodeID, SchedulingResources> &cluster_resources,
|
std::unordered_map<NodeID, SchedulingResources> &cluster_resources,
|
||||||
const NodeID &local_client_id);
|
const NodeID &local_node_id);
|
||||||
|
|
||||||
/// \param cluster_resources: a set of cluster resources containing resource and load
|
/// \param cluster_resources: a set of cluster resources containing resource and load
|
||||||
/// information for some subset of the cluster.
|
/// information for some subset of the cluster.
|
||||||
/// \param local_client_id The ID of the node manager that owns this
|
/// \param local_node_id The ID of the node manager that owns this
|
||||||
/// SchedulingPolicy object.
|
/// SchedulingPolicy object.
|
||||||
/// \param bundle_spec the description of a bundle which include the resource the bundle
|
/// \param bundle_spec the description of a bundle which include the resource the bundle
|
||||||
/// need. \return If this bundle can be scheduled in this node, return true; else return
|
/// need. \return If this bundle can be scheduled in this node, return true; else return
|
||||||
/// false.
|
/// false.
|
||||||
bool ScheduleBundle(std::unordered_map<NodeID, SchedulingResources> &cluster_resources,
|
bool ScheduleBundle(std::unordered_map<NodeID, SchedulingResources> &cluster_resources,
|
||||||
const NodeID &local_client_id,
|
const NodeID &local_node_id,
|
||||||
const ray::BundleSpecification &bundle_spec);
|
const ray::BundleSpecification &bundle_spec);
|
||||||
|
|
||||||
/// \brief Given a set of cluster resources, try to spillover infeasible tasks.
|
/// \brief Given a set of cluster resources, try to spillover infeasible tasks.
|
||||||
|
|
|
@ -309,10 +309,10 @@ Status raylet::RayletClient::NotifyActorResumedFromCheckpoint(
|
||||||
}
|
}
|
||||||
|
|
||||||
Status raylet::RayletClient::SetResource(const std::string &resource_name,
|
Status raylet::RayletClient::SetResource(const std::string &resource_name,
|
||||||
const double capacity, const NodeID &client_Id) {
|
const double capacity, const NodeID &node_id) {
|
||||||
flatbuffers::FlatBufferBuilder fbb;
|
flatbuffers::FlatBufferBuilder fbb;
|
||||||
auto message = protocol::CreateSetResourceRequest(fbb, fbb.CreateString(resource_name),
|
auto message = protocol::CreateSetResourceRequest(fbb, fbb.CreateString(resource_name),
|
||||||
capacity, to_flatbuf(fbb, client_Id));
|
capacity, to_flatbuf(fbb, node_id));
|
||||||
fbb.Finish(message);
|
fbb.Finish(message);
|
||||||
return conn_->WriteMessage(MessageType::SetResourceRequest, &fbb);
|
return conn_->WriteMessage(MessageType::SetResourceRequest, &fbb);
|
||||||
}
|
}
|
||||||
|
|
|
@ -334,10 +334,10 @@ class RayletClient : public PinObjectsInterface,
|
||||||
/// Sets a resource with the specified capacity and client id
|
/// Sets a resource with the specified capacity and client id
|
||||||
/// \param resource_name Name of the resource to be set
|
/// \param resource_name Name of the resource to be set
|
||||||
/// \param capacity Capacity of the resource
|
/// \param capacity Capacity of the resource
|
||||||
/// \param client_Id NodeID where the resource is to be set
|
/// \param node_id NodeID where the resource is to be set
|
||||||
/// \return ray::Status
|
/// \return ray::Status
|
||||||
ray::Status SetResource(const std::string &resource_name, const double capacity,
|
ray::Status SetResource(const std::string &resource_name, const double capacity,
|
||||||
const ray::NodeID &client_Id);
|
const ray::NodeID &node_id);
|
||||||
|
|
||||||
/// Ask the raylet to spill an object to external storage.
|
/// Ask the raylet to spill an object to external storage.
|
||||||
/// \param object_id The ID of the object to be spilled.
|
/// \param object_id The ID of the object to be spilled.
|
||||||
|
|
Loading…
Add table
Reference in a new issue