Rename fields/variables from client id to node id (#12457)

This commit is contained in:
Tao Wang 2020-11-30 14:33:36 +08:00 committed by GitHub
parent 3964defbe1
commit b85c6abc3e
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
43 changed files with 670 additions and 682 deletions

View file

@ -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)

View file

@ -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.

View file

@ -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:

View file

@ -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)

View file

@ -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);
} }

View file

@ -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.

View file

@ -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,

View file

@ -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);

View file

@ -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());

View file

@ -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);
} }
}; };

View file

@ -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);

View file

@ -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

View file

@ -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);
} }

View file

@ -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;

View file

@ -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.

View file

@ -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_; }

View file

@ -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_;

View file

@ -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>;

View file

@ -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:

View file

@ -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> &notifications) { const std::vector<GcsNodeInfo> &notifications) {
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 &notification : notifications) { for (auto &notification : 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,

View file

@ -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_;

View file

@ -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);
} }

View file

@ -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);

View file

@ -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;

View file

@ -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;

View file

@ -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_;

View file

@ -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();
} }

View file

@ -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;

View file

@ -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--;
} }
} }

View file

@ -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 {

View file

@ -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

View file

@ -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;
} }

View file

@ -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 {

View file

@ -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()) {

View file

@ -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

View file

@ -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);

View file

@ -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.

View file

@ -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

View file

@ -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;
} }

View file

@ -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()

View file

@ -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.

View file

@ -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);
} }

View file

@ -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.