mirror of
https://github.com/vale981/ray
synced 2025-03-06 02:21:39 -05:00
[core] Increase worker lease parallelism (#18647)
This commit is contained in:
parent
9ca34c7192
commit
92516981ea
27 changed files with 499 additions and 182 deletions
|
@ -168,7 +168,16 @@ def test_background_tasks_with_max_calls(shutdown_only):
|
|||
|
||||
@pytest.mark.skipif(sys.platform == "win32", reason="Failing on Windows.")
|
||||
def test_fair_queueing(shutdown_only):
|
||||
ray.init(num_cpus=1)
|
||||
ray.init(
|
||||
num_cpus=1,
|
||||
_system_config={
|
||||
# Having parallel leases is slow in this case
|
||||
# because tasks are scheduled FIFO,
|
||||
# the more parallism we have,
|
||||
# the more workers we need to start to execute f and g tasks
|
||||
# before we can execute the first h task.
|
||||
"max_pending_lease_requests_per_scheduling_category": 1
|
||||
})
|
||||
|
||||
@ray.remote
|
||||
def h():
|
||||
|
|
|
@ -287,8 +287,9 @@ def test_placement_group_load_report(ray_start_cluster):
|
|||
|
||||
def test_backlog_report(shutdown_only):
|
||||
cluster = ray.init(
|
||||
num_cpus=1, _system_config={
|
||||
"report_worker_backlog": True,
|
||||
num_cpus=1,
|
||||
_system_config={
|
||||
"max_pending_lease_requests_per_scheduling_category": 1
|
||||
})
|
||||
global_state_accessor = GlobalStateAccessor(
|
||||
cluster["redis_address"], ray.ray_constants.REDIS_DEFAULT_PASSWORD)
|
||||
|
@ -333,10 +334,7 @@ def test_backlog_report(shutdown_only):
|
|||
|
||||
|
||||
def test_heartbeat_ip(shutdown_only):
|
||||
cluster = ray.init(
|
||||
num_cpus=1, _system_config={
|
||||
"report_worker_backlog": True,
|
||||
})
|
||||
cluster = ray.init(num_cpus=1)
|
||||
global_state_accessor = GlobalStateAccessor(
|
||||
cluster["redis_address"], ray.ray_constants.REDIS_DEFAULT_PASSWORD)
|
||||
global_state_accessor.connect()
|
||||
|
|
|
@ -67,6 +67,11 @@ class MockNodeManager : public NodeManager {
|
|||
rpc::RequestWorkerLeaseReply *reply,
|
||||
rpc::SendReplyCallback send_reply_callback),
|
||||
(override));
|
||||
MOCK_METHOD(void, HandleReportWorkerBacklog,
|
||||
(const rpc::ReportWorkerBacklogRequest &request,
|
||||
rpc::ReportWorkerBacklogReply *reply,
|
||||
rpc::SendReplyCallback send_reply_callback),
|
||||
(override));
|
||||
MOCK_METHOD(void, HandleReturnWorker,
|
||||
(const rpc::ReturnWorkerRequest &request, rpc::ReturnWorkerReply *reply,
|
||||
rpc::SendReplyCallback send_reply_callback),
|
||||
|
|
|
@ -33,8 +33,6 @@ class MockClusterTaskManagerInterface : public ClusterTaskManagerInterface {
|
|||
(const, override));
|
||||
MOCK_METHOD(void, TaskFinished,
|
||||
(std::shared_ptr<WorkerInterface> worker, RayTask *task), (override));
|
||||
MOCK_METHOD(void, ReturnWorkerResources, (std::shared_ptr<WorkerInterface> worker),
|
||||
(override));
|
||||
MOCK_METHOD(bool, CancelTask, (const TaskID &task_id, bool runtime_env_setup_failed),
|
||||
(override));
|
||||
MOCK_METHOD(void, QueueAndScheduleTask,
|
||||
|
|
|
@ -116,6 +116,10 @@ class MockRayletClientInterface : public RayletClientInterface {
|
|||
MOCK_METHOD(ray::Status, WaitForDirectActorCallArgs,
|
||||
(const std::vector<rpc::ObjectReference> &references, int64_t tag),
|
||||
(override));
|
||||
MOCK_METHOD(void, ReportWorkerBacklog,
|
||||
(const WorkerID &worker_id,
|
||||
const std::vector<rpc::WorkerBacklogReport> &backlog_reports),
|
||||
(override));
|
||||
MOCK_METHOD(
|
||||
void, RequestWorkerLease,
|
||||
(const ray::TaskSpecification &resource_spec,
|
||||
|
|
|
@ -306,6 +306,9 @@ RAY_CONFIG(int64_t, task_rpc_inlined_bytes_limit, 10 * 1024 * 1024)
|
|||
/// pipelining task submission.
|
||||
RAY_CONFIG(uint32_t, max_tasks_in_flight_per_worker, 1)
|
||||
|
||||
/// Maximum number of pending lease requests per scheduling category
|
||||
RAY_CONFIG(uint64_t, max_pending_lease_requests_per_scheduling_category, 10)
|
||||
|
||||
/// Interval to restart dashboard agent after the process exit.
|
||||
RAY_CONFIG(uint32_t, agent_restart_interval_ms, 1000)
|
||||
|
||||
|
@ -323,10 +326,6 @@ RAY_CONFIG(uint32_t, agent_manager_retry_interval_ms, 1000);
|
|||
/// load reported by each raylet.
|
||||
RAY_CONFIG(int64_t, max_resource_shapes_per_load_report, 100)
|
||||
|
||||
/// If true, the worker's queue backlog size will be propagated to the heartbeat batch
|
||||
/// data.
|
||||
RAY_CONFIG(bool, report_worker_backlog, true)
|
||||
|
||||
/// The timeout for synchronous GCS requests in seconds.
|
||||
RAY_CONFIG(int64_t, gcs_server_request_timeout_seconds, 60)
|
||||
|
||||
|
|
|
@ -18,10 +18,9 @@
|
|||
|
||||
namespace ray {
|
||||
|
||||
RayTask::RayTask(const rpc::Task &message, int64_t backlog_size)
|
||||
RayTask::RayTask(const rpc::Task &message)
|
||||
: task_spec_(message.task_spec()),
|
||||
task_execution_spec_(message.task_execution_spec()),
|
||||
backlog_size_(backlog_size) {
|
||||
task_execution_spec_(message.task_execution_spec()) {
|
||||
ComputeDependencies();
|
||||
}
|
||||
|
||||
|
@ -50,10 +49,6 @@ void RayTask::CopyTaskExecutionSpec(const RayTask &task) {
|
|||
task_execution_spec_ = task.task_execution_spec_;
|
||||
}
|
||||
|
||||
void RayTask::SetBacklogSize(int64_t backlog_size) { backlog_size_ = backlog_size; }
|
||||
|
||||
int64_t RayTask::BacklogSize() const { return backlog_size_; }
|
||||
|
||||
std::string RayTask::DebugString() const {
|
||||
std::ostringstream stream;
|
||||
stream << "task_spec={" << task_spec_.DebugString() << "}, task_execution_spec={"
|
||||
|
|
|
@ -47,9 +47,7 @@ class RayTask {
|
|||
/// Construct a `RayTask` object from a protobuf message.
|
||||
///
|
||||
/// \param message The protobuf message.
|
||||
/// \param backlog_size The size of the task owner's backlog size for this
|
||||
/// task's shape.
|
||||
explicit RayTask(const rpc::Task &message, int64_t backlog_size = -1);
|
||||
explicit RayTask(const rpc::Task &message);
|
||||
|
||||
/// Construct a `RayTask` object from a `TaskSpecification` and a
|
||||
/// `TaskExecutionSpecification`.
|
||||
|
@ -103,10 +101,6 @@ class RayTask {
|
|||
/// Returns the cancellation task callback, or nullptr.
|
||||
const CancelTaskCallback &OnCancellation() const { return on_cancellation_; }
|
||||
|
||||
void SetBacklogSize(int64_t backlog_size);
|
||||
|
||||
int64_t BacklogSize() const;
|
||||
|
||||
std::string DebugString() const;
|
||||
|
||||
private:
|
||||
|
@ -133,8 +127,6 @@ class RayTask {
|
|||
/// For direct task calls, overrides the cancellation behaviour to send an
|
||||
/// RPC back to the submitting worker.
|
||||
mutable CancelTaskCallback on_cancellation_ = nullptr;
|
||||
/// The size of the core worker's backlog when this task was submitted.
|
||||
int64_t backlog_size_ = -1;
|
||||
};
|
||||
|
||||
} // namespace ray
|
||||
|
|
|
@ -657,7 +657,8 @@ CoreWorker::CoreWorker(const CoreWorkerOptions &options, const WorkerID &worker_
|
|||
std::move(lease_policy), memory_store_, task_manager_, local_raylet_id,
|
||||
RayConfig::instance().worker_lease_timeout_milliseconds(), actor_creator_,
|
||||
RayConfig::instance().max_tasks_in_flight_per_worker(),
|
||||
boost::asio::steady_timer(io_service_));
|
||||
boost::asio::steady_timer(io_service_),
|
||||
RayConfig::instance().max_pending_lease_requests_per_scheduling_category());
|
||||
auto report_locality_data_callback =
|
||||
[this](const ObjectID &object_id, const absl::flat_hash_set<NodeID> &locations,
|
||||
uint64_t object_size) {
|
||||
|
@ -985,6 +986,12 @@ void CoreWorker::InternalHeartbeat() {
|
|||
direct_actor_submitter_->CheckTimeoutTasks();
|
||||
}
|
||||
|
||||
// Periodically report the lastest backlog so that
|
||||
// local raylet will have the eventually consistent view of worker backlogs
|
||||
// even in cases where backlog reports from direct_task_transport
|
||||
// are lost or reordered.
|
||||
direct_task_submitter_->ReportWorkerBacklog();
|
||||
|
||||
// Check for unhandled exceptions to raise after a timeout on the driver.
|
||||
// Only do this for TTY, since shells like IPython sometimes save references
|
||||
// to the result and prevent normal result deletion from handling.
|
||||
|
|
|
@ -153,6 +153,19 @@ class MockRayletClient : public WorkerLeaseInterface {
|
|||
return Status::OK();
|
||||
}
|
||||
|
||||
void ReportWorkerBacklog(
|
||||
const WorkerID &worker_id,
|
||||
const std::vector<rpc::WorkerBacklogReport> &backlog_reports) override {
|
||||
reported_backlog_size = 0;
|
||||
reported_backlogs.clear();
|
||||
for (const auto &backlog_report : backlog_reports) {
|
||||
reported_backlog_size += backlog_report.backlog_size();
|
||||
const TaskSpecification resource_spec(backlog_report.resource_spec());
|
||||
const SchedulingClass scheduling_class = resource_spec.GetSchedulingClass();
|
||||
reported_backlogs[scheduling_class] = backlog_report.backlog_size();
|
||||
}
|
||||
}
|
||||
|
||||
void RequestWorkerLease(
|
||||
const TaskSpecification &resource_spec,
|
||||
const rpc::ClientCallback<rpc::RequestWorkerLeaseReply> &callback,
|
||||
|
@ -230,6 +243,8 @@ class MockRayletClient : public WorkerLeaseInterface {
|
|||
int num_workers_returned = 0;
|
||||
int num_workers_disconnected = 0;
|
||||
int num_leases_canceled = 0;
|
||||
int reported_backlog_size = 0;
|
||||
std::map<SchedulingClass, int64_t> reported_backlogs;
|
||||
std::list<rpc::ClientCallback<rpc::RequestWorkerLeaseReply>> callbacks = {};
|
||||
std::list<rpc::ClientCallback<rpc::CancelWorkerLeaseReply>> cancel_callbacks = {};
|
||||
};
|
||||
|
@ -625,9 +640,78 @@ TEST(DirectTaskTransportTest, TestConcurrentWorkerLeases) {
|
|||
auto task_finisher = std::make_shared<MockTaskFinisher>();
|
||||
auto actor_creator = std::make_shared<MockActorCreator>();
|
||||
auto lease_policy = std::make_shared<MockLeasePolicy>();
|
||||
CoreWorkerDirectTaskSubmitter submitter(address, raylet_client, client_pool, nullptr,
|
||||
lease_policy, store, task_finisher,
|
||||
NodeID::Nil(), kLongTimeout, actor_creator);
|
||||
CoreWorkerDirectTaskSubmitter submitter(
|
||||
address, raylet_client, client_pool, nullptr, lease_policy, store, task_finisher,
|
||||
NodeID::Nil(), kLongTimeout, actor_creator, 1, absl::nullopt, 2);
|
||||
|
||||
TaskSpecification task1 = BuildEmptyTaskSpec();
|
||||
TaskSpecification task2 = BuildEmptyTaskSpec();
|
||||
TaskSpecification task3 = BuildEmptyTaskSpec();
|
||||
|
||||
ASSERT_TRUE(submitter.SubmitTask(task1).ok());
|
||||
ASSERT_TRUE(submitter.SubmitTask(task2).ok());
|
||||
ASSERT_TRUE(submitter.SubmitTask(task3).ok());
|
||||
ASSERT_EQ(lease_policy->num_lease_policy_consults, 2);
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 2);
|
||||
ASSERT_EQ(raylet_client->num_workers_returned, 0);
|
||||
ASSERT_EQ(raylet_client->reported_backlog_size, 0);
|
||||
ASSERT_EQ(worker_client->callbacks.size(), 0);
|
||||
|
||||
// Trigger the periodic backlog report
|
||||
submitter.ReportWorkerBacklog();
|
||||
ASSERT_EQ(raylet_client->reported_backlog_size, 1);
|
||||
|
||||
// Task 1 is pushed; worker 3 is requested.
|
||||
ASSERT_TRUE(raylet_client->GrantWorkerLease("localhost", 1000, NodeID::Nil()));
|
||||
ASSERT_EQ(worker_client->callbacks.size(), 1);
|
||||
ASSERT_EQ(lease_policy->num_lease_policy_consults, 3);
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 3);
|
||||
ASSERT_EQ(raylet_client->reported_backlog_size, 0);
|
||||
|
||||
// Task 2 is pushed; no more workers requested.
|
||||
ASSERT_TRUE(raylet_client->GrantWorkerLease("localhost", 1001, NodeID::Nil()));
|
||||
ASSERT_EQ(worker_client->callbacks.size(), 2);
|
||||
ASSERT_EQ(lease_policy->num_lease_policy_consults, 3);
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 3);
|
||||
ASSERT_EQ(raylet_client->reported_backlog_size, 0);
|
||||
|
||||
// Task 3 is pushed; no more workers requested.
|
||||
ASSERT_TRUE(raylet_client->GrantWorkerLease("localhost", 1002, NodeID::Nil()));
|
||||
ASSERT_EQ(worker_client->callbacks.size(), 3);
|
||||
ASSERT_EQ(lease_policy->num_lease_policy_consults, 3);
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 3);
|
||||
ASSERT_EQ(raylet_client->reported_backlog_size, 0);
|
||||
|
||||
// All workers returned.
|
||||
while (!worker_client->callbacks.empty()) {
|
||||
ASSERT_TRUE(worker_client->ReplyPushTask());
|
||||
}
|
||||
ASSERT_EQ(raylet_client->num_workers_returned, 3);
|
||||
ASSERT_EQ(raylet_client->num_workers_disconnected, 0);
|
||||
ASSERT_EQ(task_finisher->num_tasks_complete, 3);
|
||||
ASSERT_EQ(task_finisher->num_tasks_failed, 0);
|
||||
ASSERT_EQ(raylet_client->num_leases_canceled, 0);
|
||||
ASSERT_EQ(raylet_client->reported_backlog_size, 0);
|
||||
ASSERT_FALSE(raylet_client->ReplyCancelWorkerLease());
|
||||
|
||||
// Check that there are no entries left in the scheduling_key_entries_ hashmap. These
|
||||
// would otherwise cause a memory leak.
|
||||
ASSERT_TRUE(submitter.CheckNoSchedulingKeyEntriesPublic());
|
||||
}
|
||||
|
||||
TEST(DirectTaskTransportTest, TestSubmitMultipleTasks) {
|
||||
rpc::Address address;
|
||||
auto raylet_client = std::make_shared<MockRayletClient>();
|
||||
auto worker_client = std::make_shared<MockWorkerClient>();
|
||||
auto store = std::make_shared<CoreWorkerMemoryStore>();
|
||||
auto client_pool = std::make_shared<rpc::CoreWorkerClientPool>(
|
||||
[&](const rpc::Address &addr) { return worker_client; });
|
||||
auto task_finisher = std::make_shared<MockTaskFinisher>();
|
||||
auto actor_creator = std::make_shared<MockActorCreator>();
|
||||
auto lease_policy = std::make_shared<MockLeasePolicy>();
|
||||
CoreWorkerDirectTaskSubmitter submitter(
|
||||
address, raylet_client, client_pool, nullptr, lease_policy, store, task_finisher,
|
||||
NodeID::Nil(), kLongTimeout, actor_creator, 1, absl::nullopt, 1);
|
||||
|
||||
TaskSpecification task1 = BuildEmptyTaskSpec();
|
||||
TaskSpecification task2 = BuildEmptyTaskSpec();
|
||||
|
@ -638,18 +722,21 @@ TEST(DirectTaskTransportTest, TestConcurrentWorkerLeases) {
|
|||
ASSERT_TRUE(submitter.SubmitTask(task3).ok());
|
||||
ASSERT_EQ(lease_policy->num_lease_policy_consults, 1);
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 1);
|
||||
ASSERT_EQ(raylet_client->reported_backlog_size, 0);
|
||||
|
||||
// Task 1 is pushed; worker 2 is requested.
|
||||
ASSERT_TRUE(raylet_client->GrantWorkerLease("localhost", 1000, NodeID::Nil()));
|
||||
ASSERT_EQ(worker_client->callbacks.size(), 1);
|
||||
ASSERT_EQ(lease_policy->num_lease_policy_consults, 2);
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 2);
|
||||
ASSERT_EQ(raylet_client->reported_backlog_size, 1);
|
||||
|
||||
// Task 2 is pushed; worker 3 is requested.
|
||||
ASSERT_TRUE(raylet_client->GrantWorkerLease("localhost", 1001, NodeID::Nil()));
|
||||
ASSERT_EQ(worker_client->callbacks.size(), 2);
|
||||
ASSERT_EQ(lease_policy->num_lease_policy_consults, 3);
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 3);
|
||||
ASSERT_EQ(raylet_client->reported_backlog_size, 0);
|
||||
|
||||
// Task 3 is pushed; no more workers requested.
|
||||
ASSERT_TRUE(raylet_client->GrantWorkerLease("localhost", 1002, NodeID::Nil()));
|
||||
|
@ -666,6 +753,7 @@ TEST(DirectTaskTransportTest, TestConcurrentWorkerLeases) {
|
|||
ASSERT_EQ(task_finisher->num_tasks_complete, 3);
|
||||
ASSERT_EQ(task_finisher->num_tasks_failed, 0);
|
||||
ASSERT_EQ(raylet_client->num_leases_canceled, 0);
|
||||
ASSERT_EQ(raylet_client->reported_backlog_size, 0);
|
||||
ASSERT_FALSE(raylet_client->ReplyCancelWorkerLease());
|
||||
|
||||
// Check that there are no entries left in the scheduling_key_entries_ hashmap. These
|
||||
|
@ -683,9 +771,9 @@ TEST(DirectTaskTransportTest, TestReuseWorkerLease) {
|
|||
auto task_finisher = std::make_shared<MockTaskFinisher>();
|
||||
auto actor_creator = std::make_shared<MockActorCreator>();
|
||||
auto lease_policy = std::make_shared<MockLeasePolicy>();
|
||||
CoreWorkerDirectTaskSubmitter submitter(address, raylet_client, client_pool, nullptr,
|
||||
lease_policy, store, task_finisher,
|
||||
NodeID::Nil(), kLongTimeout, actor_creator);
|
||||
CoreWorkerDirectTaskSubmitter submitter(
|
||||
address, raylet_client, client_pool, nullptr, lease_policy, store, task_finisher,
|
||||
NodeID::Nil(), kLongTimeout, actor_creator, 1, absl::nullopt, 1);
|
||||
|
||||
TaskSpecification task1 = BuildEmptyTaskSpec();
|
||||
TaskSpecification task2 = BuildEmptyTaskSpec();
|
||||
|
@ -746,9 +834,9 @@ TEST(DirectTaskTransportTest, TestRetryLeaseCancellation) {
|
|||
auto task_finisher = std::make_shared<MockTaskFinisher>();
|
||||
auto actor_creator = std::make_shared<MockActorCreator>();
|
||||
auto lease_policy = std::make_shared<MockLeasePolicy>();
|
||||
CoreWorkerDirectTaskSubmitter submitter(address, raylet_client, client_pool, nullptr,
|
||||
lease_policy, store, task_finisher,
|
||||
NodeID::Nil(), kLongTimeout, actor_creator);
|
||||
CoreWorkerDirectTaskSubmitter submitter(
|
||||
address, raylet_client, client_pool, nullptr, lease_policy, store, task_finisher,
|
||||
NodeID::Nil(), kLongTimeout, actor_creator, 1, absl::nullopt, 1);
|
||||
TaskSpecification task1 = BuildEmptyTaskSpec();
|
||||
TaskSpecification task2 = BuildEmptyTaskSpec();
|
||||
TaskSpecification task3 = BuildEmptyTaskSpec();
|
||||
|
@ -859,9 +947,9 @@ TEST(DirectTaskTransportTest, TestWorkerNotReusedOnError) {
|
|||
auto task_finisher = std::make_shared<MockTaskFinisher>();
|
||||
auto actor_creator = std::make_shared<MockActorCreator>();
|
||||
auto lease_policy = std::make_shared<MockLeasePolicy>();
|
||||
CoreWorkerDirectTaskSubmitter submitter(address, raylet_client, client_pool, nullptr,
|
||||
lease_policy, store, task_finisher,
|
||||
NodeID::Nil(), kLongTimeout, actor_creator);
|
||||
CoreWorkerDirectTaskSubmitter submitter(
|
||||
address, raylet_client, client_pool, nullptr, lease_policy, store, task_finisher,
|
||||
NodeID::Nil(), kLongTimeout, actor_creator, 1, absl::nullopt, 1);
|
||||
TaskSpecification task1 = BuildEmptyTaskSpec();
|
||||
TaskSpecification task2 = BuildEmptyTaskSpec();
|
||||
|
||||
|
@ -1075,9 +1163,9 @@ void TestSchedulingKey(const std::shared_ptr<CoreWorkerMemoryStore> store,
|
|||
auto task_finisher = std::make_shared<MockTaskFinisher>();
|
||||
auto actor_creator = std::make_shared<MockActorCreator>();
|
||||
auto lease_policy = std::make_shared<MockLeasePolicy>();
|
||||
CoreWorkerDirectTaskSubmitter submitter(address, raylet_client, client_pool, nullptr,
|
||||
lease_policy, store, task_finisher,
|
||||
NodeID::Nil(), kLongTimeout, actor_creator);
|
||||
CoreWorkerDirectTaskSubmitter submitter(
|
||||
address, raylet_client, client_pool, nullptr, lease_policy, store, task_finisher,
|
||||
NodeID::Nil(), kLongTimeout, actor_creator, 1, absl::nullopt, 1);
|
||||
|
||||
ASSERT_TRUE(submitter.SubmitTask(same1).ok());
|
||||
ASSERT_TRUE(submitter.SubmitTask(same2).ok());
|
||||
|
@ -1192,6 +1280,65 @@ TEST(DirectTaskTransportTest, TestSchedulingKeys) {
|
|||
TestSchedulingKey(store, same_deps_1, same_deps_2, different_deps);
|
||||
}
|
||||
|
||||
TEST(DirectTaskTransportTest, TestBacklogReport) {
|
||||
rpc::Address address;
|
||||
auto raylet_client = std::make_shared<MockRayletClient>();
|
||||
auto worker_client = std::make_shared<MockWorkerClient>();
|
||||
auto store = std::make_shared<CoreWorkerMemoryStore>();
|
||||
auto client_pool = std::make_shared<rpc::CoreWorkerClientPool>(
|
||||
[&](const rpc::Address &addr) { return worker_client; });
|
||||
auto task_finisher = std::make_shared<MockTaskFinisher>();
|
||||
auto actor_creator = std::make_shared<MockActorCreator>();
|
||||
auto lease_policy = std::make_shared<MockLeasePolicy>();
|
||||
CoreWorkerDirectTaskSubmitter submitter(
|
||||
address, raylet_client, client_pool, nullptr, lease_policy, store, task_finisher,
|
||||
NodeID::Nil(), kLongTimeout, actor_creator, 1, absl::nullopt, 1);
|
||||
|
||||
TaskSpecification task1 = BuildEmptyTaskSpec();
|
||||
|
||||
std::unordered_map<std::string, double> resources1({{"a", 1.0}});
|
||||
std::unordered_map<std::string, double> resources2({{"b", 2.0}});
|
||||
FunctionDescriptor descriptor1 =
|
||||
FunctionDescriptorBuilder::BuildPython("a", "", "", "");
|
||||
FunctionDescriptor descriptor2 =
|
||||
FunctionDescriptorBuilder::BuildPython("b", "", "", "");
|
||||
ObjectID plasma1 = ObjectID::FromRandom();
|
||||
ObjectID plasma2 = ObjectID::FromRandom();
|
||||
// Force plasma objects to be promoted.
|
||||
std::string meta = std::to_string(static_cast<int>(rpc::ErrorType::OBJECT_IN_PLASMA));
|
||||
auto metadata = const_cast<uint8_t *>(reinterpret_cast<const uint8_t *>(meta.data()));
|
||||
auto meta_buffer = std::make_shared<LocalMemoryBuffer>(metadata, meta.size());
|
||||
auto plasma_data = RayObject(nullptr, meta_buffer, std::vector<rpc::ObjectReference>());
|
||||
ASSERT_TRUE(store->Put(plasma_data, plasma1));
|
||||
ASSERT_TRUE(store->Put(plasma_data, plasma2));
|
||||
|
||||
// Same SchedulingClass, different SchedulingKey
|
||||
TaskSpecification task2 = BuildTaskSpec(resources1, descriptor1);
|
||||
task2.GetMutableMessage().add_args()->mutable_object_ref()->set_object_id(
|
||||
plasma1.Binary());
|
||||
TaskSpecification task3 = BuildTaskSpec(resources1, descriptor1);
|
||||
task3.GetMutableMessage().add_args()->mutable_object_ref()->set_object_id(
|
||||
plasma2.Binary());
|
||||
TestSchedulingKey(store, task2, task2, task3);
|
||||
|
||||
TaskSpecification task4 = BuildTaskSpec(resources2, descriptor2);
|
||||
|
||||
ASSERT_TRUE(submitter.SubmitTask(task1).ok());
|
||||
// One is requested and one is in the backlog for each SchedulingKey
|
||||
ASSERT_TRUE(submitter.SubmitTask(task2).ok());
|
||||
ASSERT_TRUE(submitter.SubmitTask(task2).ok());
|
||||
ASSERT_TRUE(submitter.SubmitTask(task3).ok());
|
||||
ASSERT_TRUE(submitter.SubmitTask(task3).ok());
|
||||
ASSERT_TRUE(submitter.SubmitTask(task4).ok());
|
||||
ASSERT_TRUE(submitter.SubmitTask(task4).ok());
|
||||
|
||||
submitter.ReportWorkerBacklog();
|
||||
ASSERT_EQ(raylet_client->reported_backlogs.size(), 3);
|
||||
ASSERT_EQ(raylet_client->reported_backlogs[task1.GetSchedulingClass()], 0);
|
||||
ASSERT_EQ(raylet_client->reported_backlogs[task2.GetSchedulingClass()], 2);
|
||||
ASSERT_EQ(raylet_client->reported_backlogs[task4.GetSchedulingClass()], 1);
|
||||
}
|
||||
|
||||
TEST(DirectTaskTransportTest, TestWorkerLeaseTimeout) {
|
||||
rpc::Address address;
|
||||
auto raylet_client = std::make_shared<MockRayletClient>();
|
||||
|
@ -1202,10 +1349,10 @@ TEST(DirectTaskTransportTest, TestWorkerLeaseTimeout) {
|
|||
auto task_finisher = std::make_shared<MockTaskFinisher>();
|
||||
auto actor_creator = std::make_shared<MockActorCreator>();
|
||||
auto lease_policy = std::make_shared<MockLeasePolicy>();
|
||||
CoreWorkerDirectTaskSubmitter submitter(address, raylet_client, client_pool, nullptr,
|
||||
lease_policy, store, task_finisher,
|
||||
NodeID::Nil(),
|
||||
/*lease_timeout_ms=*/5, actor_creator);
|
||||
CoreWorkerDirectTaskSubmitter submitter(
|
||||
address, raylet_client, client_pool, nullptr, lease_policy, store, task_finisher,
|
||||
NodeID::Nil(),
|
||||
/*lease_timeout_ms=*/5, actor_creator, 1, absl::nullopt, 1);
|
||||
TaskSpecification task1 = BuildEmptyTaskSpec();
|
||||
TaskSpecification task2 = BuildEmptyTaskSpec();
|
||||
TaskSpecification task3 = BuildEmptyTaskSpec();
|
||||
|
@ -1384,7 +1531,8 @@ TEST(DirectTaskTransportTest, TestPipeliningConcurrentWorkerLeases) {
|
|||
uint32_t max_tasks_in_flight_per_worker = 10;
|
||||
CoreWorkerDirectTaskSubmitter submitter(
|
||||
address, raylet_client, client_pool, nullptr, lease_policy, store, task_finisher,
|
||||
NodeID::Nil(), kLongTimeout, actor_creator, max_tasks_in_flight_per_worker);
|
||||
NodeID::Nil(), kLongTimeout, actor_creator, max_tasks_in_flight_per_worker,
|
||||
absl::nullopt, 1);
|
||||
|
||||
// Prepare 20 tasks and save them in a vector.
|
||||
std::vector<TaskSpecification> tasks;
|
||||
|
@ -1458,7 +1606,8 @@ TEST(DirectTaskTransportTest, TestPipeliningReuseWorkerLease) {
|
|||
uint32_t max_tasks_in_flight_per_worker = 10;
|
||||
CoreWorkerDirectTaskSubmitter submitter(
|
||||
address, raylet_client, client_pool, nullptr, lease_policy, store, task_finisher,
|
||||
NodeID::Nil(), kLongTimeout, actor_creator, max_tasks_in_flight_per_worker);
|
||||
NodeID::Nil(), kLongTimeout, actor_creator, max_tasks_in_flight_per_worker,
|
||||
absl::nullopt, 2);
|
||||
|
||||
// prepare 30 tasks and save them in a vector
|
||||
std::vector<TaskSpecification> tasks;
|
||||
|
@ -1467,16 +1616,16 @@ TEST(DirectTaskTransportTest, TestPipeliningReuseWorkerLease) {
|
|||
}
|
||||
ASSERT_EQ(tasks.size(), 30);
|
||||
|
||||
// Submit the 30 tasks and check that one worker is requested
|
||||
// Submit the 30 tasks and check that two workers are requested
|
||||
for (auto task : tasks) {
|
||||
ASSERT_TRUE(submitter.SubmitTask(task).ok());
|
||||
}
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 1);
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 2);
|
||||
|
||||
// Task 1-10 are pushed, and a new worker is requested.
|
||||
ASSERT_TRUE(raylet_client->GrantWorkerLease("localhost", 1000, NodeID::Nil()));
|
||||
ASSERT_EQ(worker_client->callbacks.size(), 10);
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 2);
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 3);
|
||||
// The lease is not cancelled, as there is more work to do
|
||||
ASSERT_EQ(raylet_client->num_leases_canceled, 0);
|
||||
|
||||
|
@ -1503,7 +1652,7 @@ TEST(DirectTaskTransportTest, TestPipeliningReuseWorkerLease) {
|
|||
ASSERT_EQ(raylet_client->num_workers_returned, 1);
|
||||
ASSERT_EQ(worker_client->callbacks.size(), 0);
|
||||
ASSERT_EQ(task_finisher->num_tasks_complete, 30);
|
||||
ASSERT_EQ(raylet_client->num_leases_canceled, 1);
|
||||
ASSERT_EQ(raylet_client->num_leases_canceled, 2);
|
||||
ASSERT_TRUE(raylet_client->ReplyCancelWorkerLease());
|
||||
|
||||
// The second lease request is returned immediately.
|
||||
|
@ -1513,8 +1662,19 @@ TEST(DirectTaskTransportTest, TestPipeliningReuseWorkerLease) {
|
|||
ASSERT_EQ(raylet_client->num_workers_disconnected, 0);
|
||||
ASSERT_EQ(task_finisher->num_tasks_complete, 30);
|
||||
ASSERT_EQ(task_finisher->num_tasks_failed, 0);
|
||||
ASSERT_EQ(raylet_client->num_leases_canceled, 1);
|
||||
ASSERT_FALSE(raylet_client->ReplyCancelWorkerLease());
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 3);
|
||||
ASSERT_EQ(raylet_client->num_leases_canceled, 3);
|
||||
ASSERT_TRUE(raylet_client->ReplyCancelWorkerLease());
|
||||
|
||||
// The third lease request is returned immediately.
|
||||
ASSERT_TRUE(raylet_client->GrantWorkerLease("localhost", 1001, NodeID::Nil()));
|
||||
ASSERT_EQ(worker_client->callbacks.size(), 0);
|
||||
ASSERT_EQ(raylet_client->num_workers_returned, 3);
|
||||
ASSERT_EQ(raylet_client->num_workers_disconnected, 0);
|
||||
ASSERT_EQ(task_finisher->num_tasks_complete, 30);
|
||||
ASSERT_EQ(task_finisher->num_tasks_failed, 0);
|
||||
ASSERT_EQ(raylet_client->num_leases_canceled, 3);
|
||||
ASSERT_TRUE(raylet_client->ReplyCancelWorkerLease());
|
||||
|
||||
// Check that there are no entries left in the scheduling_key_entries_ hashmap. These
|
||||
// would otherwise cause a memory leak.
|
||||
|
@ -1538,7 +1698,8 @@ TEST(DirectTaskTransportTest, TestPipeliningNumberOfWorkersRequested) {
|
|||
uint32_t max_tasks_in_flight_per_worker = 10;
|
||||
CoreWorkerDirectTaskSubmitter submitter(
|
||||
address, raylet_client, client_pool, nullptr, lease_policy, store, task_finisher,
|
||||
NodeID::Nil(), kLongTimeout, actor_creator, max_tasks_in_flight_per_worker);
|
||||
NodeID::Nil(), kLongTimeout, actor_creator, max_tasks_in_flight_per_worker,
|
||||
absl::nullopt, 1);
|
||||
|
||||
// prepare 30 tasks and save them in a vector
|
||||
std::vector<TaskSpecification> tasks;
|
||||
|
@ -1723,7 +1884,8 @@ TEST(DirectTaskTransportTest, TestStealingTasks) {
|
|||
uint32_t max_tasks_in_flight_per_worker = 10;
|
||||
CoreWorkerDirectTaskSubmitter submitter(
|
||||
address, raylet_client, client_pool, nullptr, lease_policy, store, task_finisher,
|
||||
NodeID::Nil(), kLongTimeout, actor_creator, max_tasks_in_flight_per_worker);
|
||||
NodeID::Nil(), kLongTimeout, actor_creator, max_tasks_in_flight_per_worker,
|
||||
absl::nullopt, 1);
|
||||
|
||||
// prepare 20 tasks and save them in a vector
|
||||
std::vector<TaskSpecification> tasks;
|
||||
|
@ -1903,7 +2065,8 @@ TEST(DirectTaskTransportTest, TestNoStealingByExpiredWorker) {
|
|||
uint32_t max_tasks_in_flight_per_worker = 10;
|
||||
CoreWorkerDirectTaskSubmitter submitter(
|
||||
address, raylet_client, client_pool, nullptr, lease_policy, store, task_finisher,
|
||||
NodeID::Nil(), 1000, actor_creator, max_tasks_in_flight_per_worker);
|
||||
NodeID::Nil(), 1000, actor_creator, max_tasks_in_flight_per_worker, absl::nullopt,
|
||||
1);
|
||||
|
||||
// prepare 30 tasks and save them in a vector
|
||||
std::vector<TaskSpecification> tasks;
|
||||
|
@ -2041,23 +2204,24 @@ TEST(DirectTaskTransportTest, TestNoWorkerRequestedIfStealingUnavailable) {
|
|||
uint32_t max_tasks_in_flight_per_worker = 10;
|
||||
CoreWorkerDirectTaskSubmitter submitter(
|
||||
address, raylet_client, client_pool, nullptr, lease_policy, store, task_finisher,
|
||||
NodeID::Nil(), kLongTimeout, actor_creator, max_tasks_in_flight_per_worker);
|
||||
NodeID::Nil(), kLongTimeout, actor_creator, max_tasks_in_flight_per_worker,
|
||||
absl::nullopt, 2);
|
||||
|
||||
// prepare 2 tasks and save them in a vector
|
||||
// prepare 10 tasks and save them in a vector
|
||||
std::vector<TaskSpecification> tasks;
|
||||
for (int i = 0; i < 10; i++) {
|
||||
tasks.push_back(BuildEmptyTaskSpec());
|
||||
}
|
||||
ASSERT_EQ(tasks.size(), 10);
|
||||
|
||||
// submit both tasks
|
||||
// submit all tasks
|
||||
for (int i = 1; i <= 10; i++) {
|
||||
auto task = tasks.front();
|
||||
ASSERT_TRUE(submitter.SubmitTask(task).ok());
|
||||
tasks.erase(tasks.begin());
|
||||
}
|
||||
ASSERT_EQ(tasks.size(), 0);
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 1);
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 2);
|
||||
ASSERT_EQ(task_finisher->num_tasks_complete, 0);
|
||||
ASSERT_EQ(task_finisher->num_tasks_failed, 0);
|
||||
ASSERT_EQ(raylet_client->num_leases_canceled, 0);
|
||||
|
@ -2068,7 +2232,7 @@ TEST(DirectTaskTransportTest, TestNoWorkerRequestedIfStealingUnavailable) {
|
|||
std::string worker1_id = "worker1_ID_abcdefghijklmnopq";
|
||||
ASSERT_TRUE(raylet_client->GrantWorkerLease("localhost", 1001, NodeID::Nil(), false,
|
||||
worker1_id));
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 2);
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 3);
|
||||
ASSERT_EQ(raylet_client->num_workers_disconnected, 0);
|
||||
ASSERT_EQ(raylet_client->num_workers_returned, 0);
|
||||
ASSERT_EQ(task_finisher->num_tasks_complete, 0);
|
||||
|
@ -2082,7 +2246,7 @@ TEST(DirectTaskTransportTest, TestNoWorkerRequestedIfStealingUnavailable) {
|
|||
ASSERT_TRUE(worker_client->ReplyPushTask());
|
||||
}
|
||||
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 2);
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 3);
|
||||
ASSERT_EQ(raylet_client->num_workers_disconnected, 0);
|
||||
ASSERT_EQ(raylet_client->num_workers_returned, 0);
|
||||
ASSERT_EQ(task_finisher->num_tasks_complete, 9);
|
||||
|
@ -2098,23 +2262,23 @@ TEST(DirectTaskTransportTest, TestNoWorkerRequestedIfStealingUnavailable) {
|
|||
worker2_id));
|
||||
|
||||
// Check that no more workers are requested now that there are no more stealable tasks.
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 2);
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 3);
|
||||
ASSERT_EQ(raylet_client->num_workers_disconnected, 0);
|
||||
ASSERT_EQ(raylet_client->num_workers_returned, 1);
|
||||
ASSERT_EQ(task_finisher->num_tasks_complete, 9);
|
||||
ASSERT_EQ(task_finisher->num_tasks_failed, 0);
|
||||
ASSERT_EQ(raylet_client->num_leases_canceled, 0);
|
||||
ASSERT_EQ(raylet_client->num_leases_canceled, 1);
|
||||
ASSERT_EQ(worker_client->callbacks.size(), 1);
|
||||
ASSERT_EQ(worker_client->steal_callbacks.size(), 0);
|
||||
|
||||
// Last task runs and first worker is returned
|
||||
ASSERT_TRUE(worker_client->ReplyPushTask());
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 2);
|
||||
ASSERT_EQ(raylet_client->num_workers_requested, 3);
|
||||
ASSERT_EQ(raylet_client->num_workers_returned, 2);
|
||||
ASSERT_EQ(raylet_client->num_workers_disconnected, 0);
|
||||
ASSERT_EQ(task_finisher->num_tasks_complete, 10);
|
||||
ASSERT_EQ(task_finisher->num_tasks_failed, 0);
|
||||
ASSERT_EQ(raylet_client->num_leases_canceled, 0);
|
||||
ASSERT_EQ(raylet_client->num_leases_canceled, 2);
|
||||
ASSERT_EQ(worker_client->callbacks.size(), 0);
|
||||
ASSERT_EQ(worker_client->steal_callbacks.size(), 0);
|
||||
}
|
||||
|
|
|
@ -374,15 +374,14 @@ void CoreWorkerDirectTaskSubmitter::CancelWorkerLeaseIfNeeded(
|
|||
RAY_LOG(DEBUG)
|
||||
<< "Task queue is empty, and there are no stealable tasks; canceling lease request";
|
||||
|
||||
auto &pending_lease_request = scheduling_key_entry.pending_lease_request;
|
||||
if (pending_lease_request.first) {
|
||||
for (auto &pending_lease_request : scheduling_key_entry.pending_lease_requests) {
|
||||
// There is an in-flight lease request. Cancel it.
|
||||
auto &lease_client = pending_lease_request.first;
|
||||
auto &lease_id = pending_lease_request.second;
|
||||
RAY_LOG(DEBUG) << "Canceling lease request " << lease_id;
|
||||
auto lease_client = GetOrConnectLeaseClient(&pending_lease_request.second);
|
||||
auto &task_id = pending_lease_request.first;
|
||||
RAY_LOG(DEBUG) << "Canceling lease request " << task_id;
|
||||
lease_client->CancelWorkerLease(
|
||||
lease_id, [this, scheduling_key](const Status &status,
|
||||
const rpc::CancelWorkerLeaseReply &reply) {
|
||||
task_id, [this, scheduling_key](const Status &status,
|
||||
const rpc::CancelWorkerLeaseReply &reply) {
|
||||
absl::MutexLock lock(&mu_);
|
||||
if (status.ok() && !reply.success()) {
|
||||
// The cancellation request can fail if the raylet does not have
|
||||
|
@ -423,15 +422,58 @@ CoreWorkerDirectTaskSubmitter::GetOrConnectLeaseClient(
|
|||
return lease_client;
|
||||
}
|
||||
|
||||
void CoreWorkerDirectTaskSubmitter::ReportWorkerBacklog() {
|
||||
absl::MutexLock lock(&mu_);
|
||||
ReportWorkerBacklogInternal();
|
||||
}
|
||||
|
||||
void CoreWorkerDirectTaskSubmitter::ReportWorkerBacklogInternal() {
|
||||
absl::flat_hash_map<SchedulingClass, std::pair<TaskSpecification, int64_t>> backlogs;
|
||||
for (auto &scheduling_key_and_entry : scheduling_key_entries_) {
|
||||
const SchedulingClass scheduling_class = std::get<0>(scheduling_key_and_entry.first);
|
||||
if (backlogs.find(scheduling_class) == backlogs.end()) {
|
||||
backlogs[scheduling_class].first = scheduling_key_and_entry.second.resource_spec;
|
||||
backlogs[scheduling_class].second = 0;
|
||||
}
|
||||
// We report backlog size per scheduling class not per scheduling key
|
||||
// so we need to aggregate backlog sizes of different scheduling keys
|
||||
// with the same scheduling class
|
||||
backlogs[scheduling_class].second += scheduling_key_and_entry.second.BacklogSize();
|
||||
scheduling_key_and_entry.second.last_reported_backlog_size =
|
||||
scheduling_key_and_entry.second.BacklogSize();
|
||||
}
|
||||
|
||||
std::vector<rpc::WorkerBacklogReport> backlog_reports;
|
||||
for (const auto &backlog : backlogs) {
|
||||
rpc::WorkerBacklogReport backlog_report;
|
||||
backlog_report.mutable_resource_spec()->CopyFrom(backlog.second.first.GetMessage());
|
||||
backlog_report.set_backlog_size(backlog.second.second);
|
||||
backlog_reports.emplace_back(backlog_report);
|
||||
}
|
||||
local_lease_client_->ReportWorkerBacklog(WorkerID::FromBinary(rpc_address_.worker_id()),
|
||||
backlog_reports);
|
||||
}
|
||||
|
||||
void CoreWorkerDirectTaskSubmitter::ReportWorkerBacklogIfNeeded(
|
||||
const SchedulingKey &scheduling_key) {
|
||||
const auto &scheduling_key_entry = scheduling_key_entries_[scheduling_key];
|
||||
|
||||
if (scheduling_key_entry.last_reported_backlog_size !=
|
||||
scheduling_key_entry.BacklogSize()) {
|
||||
ReportWorkerBacklogInternal();
|
||||
}
|
||||
}
|
||||
|
||||
void CoreWorkerDirectTaskSubmitter::RequestNewWorkerIfNeeded(
|
||||
const SchedulingKey &scheduling_key, const rpc::Address *raylet_address) {
|
||||
auto &scheduling_key_entry = scheduling_key_entries_[scheduling_key];
|
||||
auto &pending_lease_request = scheduling_key_entry.pending_lease_request;
|
||||
|
||||
if (pending_lease_request.first) {
|
||||
// There's already an outstanding lease request for this type of task.
|
||||
if (scheduling_key_entry.pending_lease_requests.size() ==
|
||||
max_pending_lease_requests_per_scheduling_category_) {
|
||||
return;
|
||||
}
|
||||
RAY_CHECK(scheduling_key_entry.pending_lease_requests.size() <
|
||||
max_pending_lease_requests_per_scheduling_category_);
|
||||
|
||||
// Check whether we really need a new worker or whether we have
|
||||
// enough room in an existing worker's pipeline to send the new tasks. If the pipelines
|
||||
|
@ -444,7 +486,7 @@ void CoreWorkerDirectTaskSubmitter::RequestNewWorkerIfNeeded(
|
|||
return;
|
||||
}
|
||||
|
||||
auto &task_queue = scheduling_key_entry.task_queue;
|
||||
const auto &task_queue = scheduling_key_entry.task_queue;
|
||||
// Check if the task queue is empty. If that is the case, it only makes sense to
|
||||
// consider requesting a new worker if work stealing is enabled, and there is at least a
|
||||
// worker with stealable tasks. If work stealing is not enabled, or there is no tasks
|
||||
|
@ -461,15 +503,18 @@ void CoreWorkerDirectTaskSubmitter::RequestNewWorkerIfNeeded(
|
|||
}
|
||||
return;
|
||||
}
|
||||
} else if (scheduling_key_entry.task_queue.size() <=
|
||||
scheduling_key_entry.pending_lease_requests.size()) {
|
||||
// All tasks have corresponding pending leases, no need to request more
|
||||
return;
|
||||
}
|
||||
|
||||
num_leases_requested_++;
|
||||
// Create a TaskSpecification with an overwritten TaskID to make sure we don't reuse the
|
||||
// same TaskID to request a worker
|
||||
num_leases_requested_++;
|
||||
auto resource_spec_msg = scheduling_key_entry.resource_spec.GetMutableMessage();
|
||||
resource_spec_msg.set_task_id(TaskID::ForFakeTask().Binary());
|
||||
TaskSpecification resource_spec = TaskSpecification(resource_spec_msg);
|
||||
|
||||
const TaskSpecification resource_spec = TaskSpecification(resource_spec_msg);
|
||||
rpc::Address best_node_address;
|
||||
if (raylet_address == nullptr) {
|
||||
// If no raylet address is given, find the best worker for our next lease request.
|
||||
|
@ -478,22 +523,17 @@ void CoreWorkerDirectTaskSubmitter::RequestNewWorkerIfNeeded(
|
|||
}
|
||||
|
||||
auto lease_client = GetOrConnectLeaseClient(raylet_address);
|
||||
TaskID task_id = resource_spec.TaskId();
|
||||
// Subtract 1 so we don't double count the task we are requesting for.
|
||||
int64_t queue_size = task_queue.size() - 1;
|
||||
const TaskID task_id = resource_spec.TaskId();
|
||||
|
||||
lease_client->RequestWorkerLease(
|
||||
resource_spec,
|
||||
[this, scheduling_key](const Status &status,
|
||||
const rpc::RequestWorkerLeaseReply &reply) {
|
||||
[this, scheduling_key, task_id, raylet_address = *raylet_address](
|
||||
const Status &status, const rpc::RequestWorkerLeaseReply &reply) {
|
||||
absl::MutexLock lock(&mu_);
|
||||
|
||||
auto &scheduling_key_entry = scheduling_key_entries_[scheduling_key];
|
||||
auto &pending_lease_request = scheduling_key_entry.pending_lease_request;
|
||||
RAY_CHECK(pending_lease_request.first);
|
||||
auto lease_client = std::move(pending_lease_request.first);
|
||||
const auto task_id = pending_lease_request.second;
|
||||
pending_lease_request = std::make_pair(nullptr, TaskID::Nil());
|
||||
auto lease_client = GetOrConnectLeaseClient(&raylet_address);
|
||||
scheduling_key_entry.pending_lease_requests.erase(task_id);
|
||||
|
||||
if (status.ok()) {
|
||||
if (reply.runtime_env_setup_failed()) {
|
||||
|
@ -551,8 +591,9 @@ void CoreWorkerDirectTaskSubmitter::RequestNewWorkerIfNeeded(
|
|||
RAY_LOG(FATAL) << status.ToString();
|
||||
}
|
||||
},
|
||||
queue_size);
|
||||
pending_lease_request = std::make_pair(lease_client, task_id);
|
||||
task_queue.size());
|
||||
scheduling_key_entry.pending_lease_requests.emplace(task_id, *raylet_address);
|
||||
ReportWorkerBacklogIfNeeded(scheduling_key);
|
||||
}
|
||||
|
||||
void CoreWorkerDirectTaskSubmitter::PushNormalTask(
|
||||
|
|
|
@ -66,7 +66,9 @@ class CoreWorkerDirectTaskSubmitter {
|
|||
int64_t lease_timeout_ms, std::shared_ptr<ActorCreatorInterface> actor_creator,
|
||||
uint32_t max_tasks_in_flight_per_worker =
|
||||
::RayConfig::instance().max_tasks_in_flight_per_worker(),
|
||||
absl::optional<boost::asio::steady_timer> cancel_timer = absl::nullopt)
|
||||
absl::optional<boost::asio::steady_timer> cancel_timer = absl::nullopt,
|
||||
uint64_t max_pending_lease_requests_per_scheduling_category =
|
||||
::RayConfig::instance().max_pending_lease_requests_per_scheduling_category())
|
||||
: rpc_address_(rpc_address),
|
||||
local_lease_client_(lease_client),
|
||||
lease_client_factory_(lease_client_factory),
|
||||
|
@ -78,6 +80,8 @@ class CoreWorkerDirectTaskSubmitter {
|
|||
actor_creator_(actor_creator),
|
||||
client_cache_(core_worker_client_pool),
|
||||
max_tasks_in_flight_per_worker_(max_tasks_in_flight_per_worker),
|
||||
max_pending_lease_requests_per_scheduling_category_(
|
||||
max_pending_lease_requests_per_scheduling_category),
|
||||
cancel_retry_timer_(std::move(cancel_timer)) {}
|
||||
|
||||
/// Schedule a task for direct submission to a worker.
|
||||
|
@ -107,6 +111,11 @@ class CoreWorkerDirectTaskSubmitter {
|
|||
return num_leases_requested_;
|
||||
}
|
||||
|
||||
/// Report worker backlog information to the local raylet.
|
||||
/// Since each worker only reports to its local rayet
|
||||
/// we avoid double counting backlogs in autoscaler.
|
||||
void ReportWorkerBacklog();
|
||||
|
||||
private:
|
||||
/// Schedule more work onto an idle worker or return it back to the raylet if
|
||||
/// no more tasks are queued for submission. If an error was encountered
|
||||
|
@ -127,6 +136,14 @@ class CoreWorkerDirectTaskSubmitter {
|
|||
std::shared_ptr<WorkerLeaseInterface> GetOrConnectLeaseClient(
|
||||
const rpc::Address *raylet_address) EXCLUSIVE_LOCKS_REQUIRED(mu_);
|
||||
|
||||
/// Report worker backlog information to the local raylet
|
||||
void ReportWorkerBacklogInternal() EXCLUSIVE_LOCKS_REQUIRED(mu_);
|
||||
|
||||
/// Report backlog if the backlog size is changed for this scheduling key
|
||||
/// since last report
|
||||
void ReportWorkerBacklogIfNeeded(const SchedulingKey &scheduling_key)
|
||||
EXCLUSIVE_LOCKS_REQUIRED(mu_);
|
||||
|
||||
/// Request a new worker from the raylet if no such requests are currently in
|
||||
/// flight and there are tasks queued. If a raylet address is provided, then
|
||||
/// the worker should be requested from the raylet at that address. Else, the
|
||||
|
@ -237,6 +254,9 @@ class CoreWorkerDirectTaskSubmitter {
|
|||
// worker using a single lease.
|
||||
const uint32_t max_tasks_in_flight_per_worker_;
|
||||
|
||||
// Max number of pending lease requests per SchedulingKey.
|
||||
const uint64_t max_pending_lease_requests_per_scheduling_category_;
|
||||
|
||||
/// A LeaseEntry struct is used to condense the metadata about a single executor:
|
||||
/// (1) The lease client through which the worker should be returned
|
||||
/// (2) The expiration time of a worker's lease.
|
||||
|
@ -296,8 +316,7 @@ class CoreWorkerDirectTaskSubmitter {
|
|||
|
||||
struct SchedulingKeyEntry {
|
||||
// Keep track of pending worker lease requests to the raylet.
|
||||
std::pair<std::shared_ptr<WorkerLeaseInterface>, TaskID> pending_lease_request =
|
||||
std::make_pair(nullptr, TaskID::Nil());
|
||||
absl::flat_hash_map<TaskID, rpc::Address> pending_lease_requests;
|
||||
TaskSpecification resource_spec = TaskSpecification();
|
||||
// Tasks that are queued for execution. We keep an individual queue per
|
||||
// scheduling class to ensure fairness.
|
||||
|
@ -308,11 +327,12 @@ class CoreWorkerDirectTaskSubmitter {
|
|||
absl::flat_hash_set<rpc::WorkerAddress>();
|
||||
// Keep track of how many tasks with this SchedulingKey are in flight, in total
|
||||
uint32_t total_tasks_in_flight = 0;
|
||||
int64_t last_reported_backlog_size = 0;
|
||||
|
||||
// Check whether it's safe to delete this SchedulingKeyEntry from the
|
||||
// scheduling_key_entries_ hashmap.
|
||||
inline bool CanDelete() const {
|
||||
if (!pending_lease_request.first && task_queue.empty() &&
|
||||
if (pending_lease_requests.empty() && task_queue.empty() &&
|
||||
active_workers.size() == 0 && total_tasks_in_flight == 0) {
|
||||
return true;
|
||||
}
|
||||
|
@ -339,6 +359,18 @@ class CoreWorkerDirectTaskSubmitter {
|
|||
// If any worker has more than one task in flight, then that task can be stolen.
|
||||
return total_tasks_in_flight > active_workers.size();
|
||||
}
|
||||
|
||||
// Get the current backlog size for this scheduling key
|
||||
[[nodiscard]] inline int64_t BacklogSize() const {
|
||||
if (task_queue.size() < pending_lease_requests.size()) {
|
||||
// During work stealing we may have more pending lease requests than the number of
|
||||
// queued tasks
|
||||
return 0;
|
||||
}
|
||||
|
||||
// Subtract tasks with pending lease requests so we don't double count them.
|
||||
return task_queue.size() - pending_lease_requests.size();
|
||||
}
|
||||
};
|
||||
|
||||
// For each Scheduling Key, scheduling_key_entries_ contains a SchedulingKeyEntry struct
|
||||
|
|
|
@ -38,7 +38,6 @@ GcsActorScheduler::GcsActorScheduler(
|
|||
gcs_pub_sub_(std::move(gcs_pub_sub)),
|
||||
schedule_failure_handler_(std::move(schedule_failure_handler)),
|
||||
schedule_success_handler_(std::move(schedule_success_handler)),
|
||||
report_worker_backlog_(RayConfig::instance().report_worker_backlog()),
|
||||
raylet_client_pool_(raylet_client_pool),
|
||||
core_worker_clients_(client_factory) {
|
||||
RAY_CHECK(schedule_failure_handler_ != nullptr && schedule_success_handler_ != nullptr);
|
||||
|
@ -230,14 +229,13 @@ void GcsActorScheduler::LeaseWorkerFromNode(std::shared_ptr<GcsActor> actor,
|
|||
auto lease_client = GetOrConnectLeaseClient(remote_address);
|
||||
// Actor leases should be sent to the raylet immediately, so we should never build up a
|
||||
// backlog in GCS.
|
||||
int backlog_size = report_worker_backlog_ ? 0 : -1;
|
||||
lease_client->RequestWorkerLease(
|
||||
actor->GetActorTableData().task_spec(),
|
||||
[this, actor, node](const Status &status,
|
||||
const rpc::RequestWorkerLeaseReply &reply) {
|
||||
HandleWorkerLeaseReply(actor, node, status, reply);
|
||||
},
|
||||
backlog_size);
|
||||
0);
|
||||
}
|
||||
|
||||
void GcsActorScheduler::RetryLeasingWorkerFromNode(
|
||||
|
|
|
@ -312,8 +312,6 @@ class GcsActorScheduler : public GcsActorSchedulerInterface {
|
|||
/// The handler to handle the successful scheduling.
|
||||
std::function<void(std::shared_ptr<GcsActor>, const rpc::PushTaskReply &reply)>
|
||||
schedule_success_handler_;
|
||||
/// Whether or not to report the backlog of actors waiting to be scheduled.
|
||||
bool report_worker_backlog_;
|
||||
/// The nodes which are releasing unused workers.
|
||||
absl::flat_hash_set<NodeID> nodes_of_releasing_unused_workers_;
|
||||
/// The cached raylet clients used to communicate with raylet.
|
||||
|
|
|
@ -13,6 +13,7 @@
|
|||
// limitations under the License.
|
||||
|
||||
#include "ray/gcs/gcs_server/gcs_resource_manager.h"
|
||||
|
||||
#include "ray/common/ray_config.h"
|
||||
#include "ray/stats/stats.h"
|
||||
|
||||
|
@ -233,10 +234,8 @@ void GcsResourceManager::HandleGetAllResourceUsage(
|
|||
aggregate_demand.set_num_infeasible_requests_queued(
|
||||
aggregate_demand.num_infeasible_requests_queued() +
|
||||
demand.num_infeasible_requests_queued());
|
||||
if (RayConfig::instance().report_worker_backlog()) {
|
||||
aggregate_demand.set_backlog_size(aggregate_demand.backlog_size() +
|
||||
demand.backlog_size());
|
||||
}
|
||||
aggregate_demand.set_backlog_size(aggregate_demand.backlog_size() +
|
||||
demand.backlog_size());
|
||||
}
|
||||
|
||||
batch->add_batch()->CopyFrom(usage.second);
|
||||
|
|
|
@ -70,6 +70,10 @@ struct GcsServerMocker {
|
|||
return Status::OK();
|
||||
}
|
||||
|
||||
void ReportWorkerBacklog(
|
||||
const WorkerID &worker_id,
|
||||
const std::vector<rpc::WorkerBacklogReport> &backlog_reports) override {}
|
||||
|
||||
/// WorkerLeaseInterface
|
||||
void RequestWorkerLease(
|
||||
const ray::TaskSpecification &resource_spec,
|
||||
|
|
|
@ -20,6 +20,24 @@ package ray.rpc;
|
|||
import "src/ray/protobuf/common.proto";
|
||||
import "src/ray/protobuf/gcs.proto";
|
||||
|
||||
message WorkerBacklogReport {
|
||||
// TaskSpec indicating the scheduling class.
|
||||
// Cannot send scheduling class directly
|
||||
// since it's local to each process.
|
||||
TaskSpec resource_spec = 1;
|
||||
// Size of the backlog for the above scheduling class.
|
||||
int64 backlog_size = 2;
|
||||
}
|
||||
|
||||
message ReportWorkerBacklogRequest {
|
||||
// Unique id of the worker that's reporting the backlog
|
||||
bytes worker_id = 1;
|
||||
// Backlog report per scheduling class
|
||||
repeated WorkerBacklogReport backlog_reports = 2;
|
||||
}
|
||||
|
||||
message ReportWorkerBacklogReply {}
|
||||
|
||||
// Request a worker from the raylet with the specified resources.
|
||||
message RequestWorkerLeaseRequest {
|
||||
// TaskSpec containing the requested resources.
|
||||
|
@ -255,6 +273,8 @@ service NodeManagerService {
|
|||
returns (RequestResourceReportReply);
|
||||
// Request a worker from the raylet.
|
||||
rpc RequestWorkerLease(RequestWorkerLeaseRequest) returns (RequestWorkerLeaseReply);
|
||||
// Report task backlog information from a worker to the raylet
|
||||
rpc ReportWorkerBacklog(ReportWorkerBacklogRequest) returns (ReportWorkerBacklogReply);
|
||||
// Release a worker back to its raylet.
|
||||
rpc ReturnWorker(ReturnWorkerRequest) returns (ReturnWorkerReply);
|
||||
// This method is only used by GCS, and the purpose is to release leased workers
|
||||
|
|
|
@ -1257,6 +1257,8 @@ void NodeManager::DisconnectClient(
|
|||
// Return the resources that were being used by this worker.
|
||||
cluster_task_manager_->ReleaseWorkerResources(worker);
|
||||
|
||||
cluster_task_manager_->ClearWorkerBacklog(worker->WorkerId());
|
||||
|
||||
// Since some resources may have been released, we can try to dispatch more tasks.
|
||||
cluster_task_manager_->ScheduleAndDispatchTasks();
|
||||
} else if (is_driver) {
|
||||
|
@ -1502,19 +1504,28 @@ void NodeManager::HandleRequestResourceReport(
|
|||
send_reply_callback(Status::OK(), nullptr, nullptr);
|
||||
}
|
||||
|
||||
void NodeManager::HandleReportWorkerBacklog(
|
||||
const rpc::ReportWorkerBacklogRequest &request, rpc::ReportWorkerBacklogReply *reply,
|
||||
rpc::SendReplyCallback send_reply_callback) {
|
||||
const WorkerID worker_id = WorkerID::FromBinary(request.worker_id());
|
||||
cluster_task_manager_->ClearWorkerBacklog(worker_id);
|
||||
std::unordered_set<SchedulingClass> seen;
|
||||
for (const auto &backlog_report : request.backlog_reports()) {
|
||||
const TaskSpecification resource_spec(backlog_report.resource_spec());
|
||||
const SchedulingClass scheduling_class = resource_spec.GetSchedulingClass();
|
||||
RAY_CHECK(seen.find(scheduling_class) == seen.end());
|
||||
cluster_task_manager_->SetWorkerBacklog(scheduling_class, worker_id,
|
||||
backlog_report.backlog_size());
|
||||
}
|
||||
send_reply_callback(Status::OK(), nullptr, nullptr);
|
||||
}
|
||||
|
||||
void NodeManager::HandleRequestWorkerLease(const rpc::RequestWorkerLeaseRequest &request,
|
||||
rpc::RequestWorkerLeaseReply *reply,
|
||||
rpc::SendReplyCallback send_reply_callback) {
|
||||
rpc::Task task_message;
|
||||
task_message.mutable_task_spec()->CopyFrom(request.resource_spec());
|
||||
auto backlog_size = -1;
|
||||
if (RayConfig::instance().report_worker_backlog()) {
|
||||
// We add 1 to the backlog size because we need a worker to fulfill the
|
||||
// current request, as well as workers to serve the requests in the
|
||||
// backlog.
|
||||
backlog_size = request.backlog_size() + 1;
|
||||
}
|
||||
RayTask task(task_message, backlog_size);
|
||||
RayTask task(task_message);
|
||||
bool is_actor_creation_task = task.GetTaskSpecification().IsActorCreationTask();
|
||||
ActorID actor_id = ActorID::Nil();
|
||||
metrics_num_task_scheduled_ += 1;
|
||||
|
@ -1664,7 +1675,7 @@ void NodeManager::HandleReturnWorker(const rpc::ReturnWorkerRequest &request,
|
|||
if (worker->IsBlocked()) {
|
||||
HandleDirectCallTaskUnblocked(worker);
|
||||
}
|
||||
cluster_task_manager_->ReturnWorkerResources(worker);
|
||||
cluster_task_manager_->ReleaseWorkerResources(worker);
|
||||
HandleWorkerAvailable(worker);
|
||||
}
|
||||
} else {
|
||||
|
|
|
@ -487,6 +487,11 @@ class NodeManager : public rpc::NodeManagerServiceHandler {
|
|||
rpc::RequestWorkerLeaseReply *reply,
|
||||
rpc::SendReplyCallback send_reply_callback) override;
|
||||
|
||||
/// Handle a `ReportWorkerBacklog` request.
|
||||
void HandleReportWorkerBacklog(const rpc::ReportWorkerBacklogRequest &request,
|
||||
rpc::ReportWorkerBacklogReply *reply,
|
||||
rpc::SendReplyCallback send_reply_callback) override;
|
||||
|
||||
/// Handle a `ReturnWorker` request.
|
||||
void HandleReturnWorker(const rpc::ReturnWorkerRequest &request,
|
||||
rpc::ReturnWorkerReply *reply,
|
||||
|
|
|
@ -48,7 +48,6 @@ ClusterTaskManager::ClusterTaskManager(
|
|||
announce_infeasible_task_(announce_infeasible_task),
|
||||
max_resource_shapes_per_load_report_(
|
||||
RayConfig::instance().max_resource_shapes_per_load_report()),
|
||||
report_worker_backlog_(RayConfig::instance().report_worker_backlog()),
|
||||
worker_pool_(worker_pool),
|
||||
leased_workers_(leased_workers),
|
||||
get_task_arguments_(get_task_arguments),
|
||||
|
@ -426,7 +425,6 @@ void ClusterTaskManager::QueueAndScheduleTask(
|
|||
} else {
|
||||
tasks_to_schedule_[scheduling_class].push_back(work);
|
||||
}
|
||||
AddToBacklogTracker(task);
|
||||
ScheduleAndDispatchTasks();
|
||||
}
|
||||
|
||||
|
@ -563,12 +561,6 @@ void ClusterTaskManager::ReleaseTaskArgs(const TaskID &task_id) {
|
|||
}
|
||||
}
|
||||
|
||||
void ClusterTaskManager::ReturnWorkerResources(std::shared_ptr<WorkerInterface> worker) {
|
||||
// TODO(Shanly): This method will be removed and can be replaced by
|
||||
// `ReleaseWorkerResources` directly once we remove the legacy scheduler.
|
||||
ReleaseWorkerResources(worker);
|
||||
}
|
||||
|
||||
void ReplyCancelled(std::shared_ptr<Work> &work, bool runtime_env_setup_failed) {
|
||||
auto reply = work->reply;
|
||||
auto callback = work->callback;
|
||||
|
@ -587,7 +579,6 @@ bool ClusterTaskManager::CancelTask(const TaskID &task_id,
|
|||
for (auto work_it = work_queue.begin(); work_it != work_queue.end(); work_it++) {
|
||||
const auto &task = (*work_it)->task;
|
||||
if (task.GetTaskSpecification().TaskId() == task_id) {
|
||||
RemoveFromBacklogTracker(task);
|
||||
RAY_LOG(DEBUG) << "Canceling task " << task_id << " from schedule queue.";
|
||||
ReplyCancelled(*work_it, runtime_env_setup_failed);
|
||||
work_queue.erase(work_it);
|
||||
|
@ -604,7 +595,6 @@ bool ClusterTaskManager::CancelTask(const TaskID &task_id,
|
|||
for (auto work_it = work_queue.begin(); work_it != work_queue.end(); work_it++) {
|
||||
const auto &task = (*work_it)->task;
|
||||
if (task.GetTaskSpecification().TaskId() == task_id) {
|
||||
RemoveFromBacklogTracker(task);
|
||||
RAY_LOG(DEBUG) << "Canceling task " << task_id << " from dispatch queue.";
|
||||
ReplyCancelled(*work_it, runtime_env_setup_failed);
|
||||
if ((*work_it)->status == WorkStatus::WAITING_FOR_WORKER) {
|
||||
|
@ -634,7 +624,6 @@ bool ClusterTaskManager::CancelTask(const TaskID &task_id,
|
|||
for (auto work_it = work_queue.begin(); work_it != work_queue.end(); work_it++) {
|
||||
const auto &task = (*work_it)->task;
|
||||
if (task.GetTaskSpecification().TaskId() == task_id) {
|
||||
RemoveFromBacklogTracker(task);
|
||||
RAY_LOG(DEBUG) << "Canceling task " << task_id << " from infeasible queue.";
|
||||
ReplyCancelled(*work_it, runtime_env_setup_failed);
|
||||
work_queue.erase(work_it);
|
||||
|
@ -649,7 +638,6 @@ bool ClusterTaskManager::CancelTask(const TaskID &task_id,
|
|||
auto iter = waiting_tasks_index_.find(task_id);
|
||||
if (iter != waiting_tasks_index_.end()) {
|
||||
const auto &task = (*iter->second)->task;
|
||||
RemoveFromBacklogTracker(task);
|
||||
ReplyCancelled(*iter->second, runtime_env_setup_failed);
|
||||
if (!task.GetTaskSpecification().GetDependencies().empty()) {
|
||||
task_dependency_manager_.RemoveTaskDependencies(
|
||||
|
@ -744,11 +732,7 @@ void ClusterTaskManager::FillResourceUsage(
|
|||
|
||||
by_shape_entry->set_num_ready_requests_queued(ready_count);
|
||||
by_shape_entry->set_num_infeasible_requests_queued(infeasible_count);
|
||||
|
||||
auto backlog_it = backlog_tracker_.find(one_cpu_scheduling_cls);
|
||||
if (backlog_it != backlog_tracker_.end()) {
|
||||
by_shape_entry->set_backlog_size(backlog_it->second);
|
||||
}
|
||||
by_shape_entry->set_backlog_size(TotalBacklogSize(one_cpu_scheduling_cls));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -786,10 +770,7 @@ void ClusterTaskManager::FillResourceUsage(
|
|||
// ClusterResourceScheduler::GetBestSchedulableNode for more details.
|
||||
int num_ready = by_shape_entry->num_ready_requests_queued();
|
||||
by_shape_entry->set_num_ready_requests_queued(num_ready + count);
|
||||
auto backlog_it = backlog_tracker_.find(scheduling_class);
|
||||
if (backlog_it != backlog_tracker_.end()) {
|
||||
by_shape_entry->set_backlog_size(backlog_it->second);
|
||||
}
|
||||
by_shape_entry->set_backlog_size(TotalBacklogSize(scheduling_class));
|
||||
}
|
||||
|
||||
for (const auto &pair : tasks_to_dispatch_) {
|
||||
|
@ -822,10 +803,7 @@ void ClusterTaskManager::FillResourceUsage(
|
|||
}
|
||||
int num_ready = by_shape_entry->num_ready_requests_queued();
|
||||
by_shape_entry->set_num_ready_requests_queued(num_ready + count);
|
||||
auto backlog_it = backlog_tracker_.find(scheduling_class);
|
||||
if (backlog_it != backlog_tracker_.end()) {
|
||||
by_shape_entry->set_backlog_size(backlog_it->second);
|
||||
}
|
||||
by_shape_entry->set_backlog_size(TotalBacklogSize(scheduling_class));
|
||||
}
|
||||
|
||||
for (const auto &pair : infeasible_tasks_) {
|
||||
|
@ -861,10 +839,7 @@ void ClusterTaskManager::FillResourceUsage(
|
|||
// ClusterResourceScheduler::GetBestSchedulableNode for more details.
|
||||
int num_infeasible = by_shape_entry->num_infeasible_requests_queued();
|
||||
by_shape_entry->set_num_infeasible_requests_queued(num_infeasible + count);
|
||||
auto backlog_it = backlog_tracker_.find(scheduling_class);
|
||||
if (backlog_it != backlog_tracker_.end()) {
|
||||
by_shape_entry->set_backlog_size(backlog_it->second);
|
||||
}
|
||||
by_shape_entry->set_backlog_size(TotalBacklogSize(scheduling_class));
|
||||
}
|
||||
|
||||
if (RayConfig::instance().enable_light_weight_resource_report()) {
|
||||
|
@ -1018,7 +993,6 @@ void ClusterTaskManager::Dispatch(
|
|||
|
||||
RAY_CHECK(leased_workers.find(worker->WorkerId()) == leased_workers.end());
|
||||
leased_workers[worker->WorkerId()] = worker;
|
||||
RemoveFromBacklogTracker(task);
|
||||
|
||||
// Update our internal view of the cluster state.
|
||||
std::shared_ptr<TaskResourceInstances> allocated_resources;
|
||||
|
@ -1074,7 +1048,6 @@ void ClusterTaskManager::Spillback(const NodeID &spillback_to,
|
|||
metric_tasks_spilled_++;
|
||||
const auto &task = work->task;
|
||||
const auto &task_spec = task.GetTaskSpecification();
|
||||
RemoveFromBacklogTracker(task);
|
||||
RAY_LOG(DEBUG) << "Spilling task " << task_spec.TaskId() << " to node " << spillback_to;
|
||||
|
||||
if (!cluster_resource_scheduler_->AllocateRemoteTaskResources(
|
||||
|
@ -1101,23 +1074,44 @@ void ClusterTaskManager::Spillback(const NodeID &spillback_to,
|
|||
send_reply_callback();
|
||||
}
|
||||
|
||||
void ClusterTaskManager::AddToBacklogTracker(const RayTask &task) {
|
||||
if (report_worker_backlog_) {
|
||||
auto cls = task.GetTaskSpecification().GetSchedulingClass();
|
||||
backlog_tracker_[cls] += task.BacklogSize();
|
||||
void ClusterTaskManager::ClearWorkerBacklog(const WorkerID &worker_id) {
|
||||
for (auto it = backlog_tracker_.begin(); it != backlog_tracker_.end();) {
|
||||
it->second.erase(worker_id);
|
||||
if (it->second.empty()) {
|
||||
it = backlog_tracker_.erase(it);
|
||||
} else {
|
||||
++it;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void ClusterTaskManager::RemoveFromBacklogTracker(const RayTask &task) {
|
||||
if (report_worker_backlog_) {
|
||||
SchedulingClass cls = task.GetTaskSpecification().GetSchedulingClass();
|
||||
backlog_tracker_[cls] -= task.BacklogSize();
|
||||
if (backlog_tracker_[cls] == 0) {
|
||||
backlog_tracker_.erase(backlog_tracker_.find(cls));
|
||||
void ClusterTaskManager::SetWorkerBacklog(SchedulingClass scheduling_class,
|
||||
const WorkerID &worker_id,
|
||||
int64_t backlog_size) {
|
||||
if (backlog_size == 0) {
|
||||
backlog_tracker_[scheduling_class].erase(worker_id);
|
||||
if (backlog_tracker_[scheduling_class].empty()) {
|
||||
backlog_tracker_.erase(scheduling_class);
|
||||
}
|
||||
} else {
|
||||
backlog_tracker_[scheduling_class][worker_id] = backlog_size;
|
||||
}
|
||||
}
|
||||
|
||||
int64_t ClusterTaskManager::TotalBacklogSize(SchedulingClass scheduling_class) {
|
||||
auto backlog_it = backlog_tracker_.find(scheduling_class);
|
||||
if (backlog_it == backlog_tracker_.end()) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
int64_t sum = 0;
|
||||
for (const auto &worker_id_and_backlog_size : backlog_it->second) {
|
||||
sum += worker_id_and_backlog_size.second;
|
||||
}
|
||||
|
||||
return sum;
|
||||
}
|
||||
|
||||
void ClusterTaskManager::ReleaseWorkerResources(std::shared_ptr<WorkerInterface> worker) {
|
||||
RAY_CHECK(worker != nullptr);
|
||||
auto allocated_instances = worker->GetAllocatedInstances();
|
||||
|
|
|
@ -102,6 +102,11 @@ class ClusterTaskManager : public ClusterTaskManagerInterface {
|
|||
get_task_arguments,
|
||||
size_t max_pinned_task_arguments_bytes);
|
||||
|
||||
void SetWorkerBacklog(SchedulingClass scheduling_class, const WorkerID &worker_id,
|
||||
int64_t backlog_size) override;
|
||||
|
||||
void ClearWorkerBacklog(const WorkerID &worker_id) override;
|
||||
|
||||
/// (Step 1) Queue tasks and schedule.
|
||||
/// Queue task and schedule. This hanppens when processing the worker lease request.
|
||||
///
|
||||
|
@ -125,13 +130,6 @@ class ClusterTaskManager : public ClusterTaskManagerInterface {
|
|||
/// \param task: Output parameter.
|
||||
void TaskFinished(std::shared_ptr<WorkerInterface> worker, RayTask *task) override;
|
||||
|
||||
/// Return worker resources.
|
||||
/// This method will be removed and can be replaced by `ReleaseWorkerResources` directly
|
||||
/// once we remove the legacy scheduler.
|
||||
///
|
||||
/// \param worker: The worker which was running the task.
|
||||
void ReturnWorkerResources(std::shared_ptr<WorkerInterface> worker) override;
|
||||
|
||||
/// Attempt to cancel an already queued task.
|
||||
///
|
||||
/// \param task_id: The id of the task to remove.
|
||||
|
@ -261,7 +259,6 @@ class ClusterTaskManager : public ClusterTaskManagerInterface {
|
|||
std::function<void(const RayTask &)> announce_infeasible_task_;
|
||||
|
||||
const int max_resource_shapes_per_load_report_;
|
||||
const bool report_worker_backlog_;
|
||||
|
||||
/// TODO(swang): Add index from TaskID -> Work to avoid having to iterate
|
||||
/// through queues to cancel tasks, etc.
|
||||
|
@ -307,8 +304,9 @@ class ClusterTaskManager : public ClusterTaskManagerInterface {
|
|||
std::unordered_map<SchedulingClass, std::deque<std::shared_ptr<Work>>>
|
||||
infeasible_tasks_;
|
||||
|
||||
/// Track the cumulative backlog of all workers requesting a lease to this raylet.
|
||||
std::unordered_map<SchedulingClass, int> backlog_tracker_;
|
||||
/// Track the backlog of all workers belonging to this raylet.
|
||||
std::unordered_map<SchedulingClass, std::unordered_map<WorkerID, int64_t>>
|
||||
backlog_tracker_;
|
||||
|
||||
/// TODO(Shanly): Remove `worker_pool_` and `leased_workers_` and make them as
|
||||
/// parameters of methods if necessary once we remove the legacy scheduler.
|
||||
|
@ -360,8 +358,8 @@ class ClusterTaskManager : public ClusterTaskManagerInterface {
|
|||
|
||||
void Spillback(const NodeID &spillback_to, const std::shared_ptr<Work> &work);
|
||||
|
||||
void AddToBacklogTracker(const RayTask &task);
|
||||
void RemoveFromBacklogTracker(const RayTask &task);
|
||||
/// Sum up the backlog size across all workers for a given scheduling class.
|
||||
int64_t TotalBacklogSize(SchedulingClass scheduling_class);
|
||||
|
||||
// Helper function to pin a task's args immediately before dispatch. This
|
||||
// returns false if there are missing args (due to eviction) or if there is
|
||||
|
|
|
@ -78,13 +78,6 @@ class ClusterTaskManagerInterface {
|
|||
/// \param task: Output parameter.
|
||||
virtual void TaskFinished(std::shared_ptr<WorkerInterface> worker, RayTask *task) = 0;
|
||||
|
||||
/// Return worker resources.
|
||||
/// This method will be removed and can be replaced by `ReleaseWorkerResources` directly
|
||||
/// once we remove the legacy scheduler
|
||||
///
|
||||
/// \param worker: The worker which was running the task.
|
||||
virtual void ReturnWorkerResources(std::shared_ptr<WorkerInterface> worker) = 0;
|
||||
|
||||
/// Attempt to cancel an already queued task.
|
||||
///
|
||||
/// \param task_id: The id of the task to remove.
|
||||
|
@ -96,6 +89,20 @@ class ClusterTaskManagerInterface {
|
|||
virtual bool CancelTask(const TaskID &task_id,
|
||||
bool runtime_env_setup_failed = false) = 0;
|
||||
|
||||
/// Set the worker backlog size for a particular scheduling class.
|
||||
///
|
||||
/// \param scheduling_class: The scheduling class this backlog is for.
|
||||
/// \param worker_id: The ID of the worker that owns the backlog information.
|
||||
/// \param backlog_size: The size of the backlog.
|
||||
virtual void SetWorkerBacklog(SchedulingClass scheduling_class,
|
||||
const WorkerID &worker_id, int64_t backlog_size) = 0;
|
||||
|
||||
/// Remove all backlog information about the given worker.
|
||||
///
|
||||
/// \param worker_id: The ID of the worker owning the backlog information
|
||||
/// that we want to remove.
|
||||
virtual void ClearWorkerBacklog(const WorkerID &worker_id) = 0;
|
||||
|
||||
/// Queue task and schedule. This hanppens when processing the worker lease request.
|
||||
///
|
||||
/// \param task: The incoming task to be queued and scheduled.
|
||||
|
|
|
@ -871,7 +871,7 @@ TEST_F(ClusterTaskManagerTest, HeartbeatTest) {
|
|||
TEST_F(ClusterTaskManagerTest, BacklogReportTest) {
|
||||
/*
|
||||
Test basic scheduler functionality:
|
||||
1. Queue and attempt to schedule/dispatch atest with no workers available
|
||||
1. Queue and attempt to schedule/dispatch a test with no workers available
|
||||
2. A worker becomes available, dispatch again.
|
||||
*/
|
||||
rpc::RequestWorkerLeaseReply reply;
|
||||
|
@ -884,18 +884,21 @@ TEST_F(ClusterTaskManagerTest, BacklogReportTest) {
|
|||
|
||||
std::vector<TaskID> to_cancel;
|
||||
|
||||
// Don't add these fist 2 tasks to `to_cancel`.
|
||||
const WorkerID worker_id_submitting_first_task = WorkerID::FromRandom();
|
||||
// Don't add the fist task to `to_cancel`.
|
||||
for (int i = 0; i < 1; i++) {
|
||||
RayTask task = CreateTask({{ray::kCPU_ResourceLabel, 8}});
|
||||
task.SetBacklogSize(10 - i);
|
||||
task_manager_.QueueAndScheduleTask(task, &reply, callback);
|
||||
task_manager_.SetWorkerBacklog(task.GetTaskSpecification().GetSchedulingClass(),
|
||||
worker_id_submitting_first_task, 10 - i);
|
||||
pool_.TriggerCallbacks();
|
||||
}
|
||||
|
||||
for (int i = 1; i < 10; i++) {
|
||||
RayTask task = CreateTask({{ray::kCPU_ResourceLabel, 8}});
|
||||
task.SetBacklogSize(10 - i);
|
||||
task_manager_.QueueAndScheduleTask(task, &reply, callback);
|
||||
task_manager_.SetWorkerBacklog(task.GetTaskSpecification().GetSchedulingClass(),
|
||||
WorkerID::FromRandom(), 10 - i);
|
||||
pool_.TriggerCallbacks();
|
||||
to_cancel.push_back(task.GetTaskSpecification().TaskId());
|
||||
}
|
||||
|
@ -921,6 +924,7 @@ TEST_F(ClusterTaskManagerTest, BacklogReportTest) {
|
|||
std::make_shared<MockWorker>(WorkerID::FromRandom(), 1234);
|
||||
pool_.PushWorker(worker);
|
||||
task_manager_.ScheduleAndDispatchTasks();
|
||||
task_manager_.ClearWorkerBacklog(worker_id_submitting_first_task);
|
||||
pool_.TriggerCallbacks();
|
||||
|
||||
{
|
||||
|
|
|
@ -321,6 +321,20 @@ void raylet::RayletClient::RequestObjectSpillage(
|
|||
grpc_client_->RequestObjectSpillage(request, callback);
|
||||
}
|
||||
|
||||
void raylet::RayletClient::ReportWorkerBacklog(
|
||||
const WorkerID &worker_id,
|
||||
const std::vector<rpc::WorkerBacklogReport> &backlog_reports) {
|
||||
rpc::ReportWorkerBacklogRequest request;
|
||||
request.set_worker_id(worker_id.Binary());
|
||||
request.mutable_backlog_reports()->Add(backlog_reports.begin(), backlog_reports.end());
|
||||
grpc_client_->ReportWorkerBacklog(
|
||||
request, [](const Status &status, const rpc::ReportWorkerBacklogReply &reply) {
|
||||
if (!status.ok()) {
|
||||
RAY_LOG(INFO) << "Error reporting task backlog information: " << status;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
Status raylet::RayletClient::ReturnWorker(int worker_port, const WorkerID &worker_id,
|
||||
bool disconnect_worker) {
|
||||
rpc::ReturnWorkerRequest request;
|
||||
|
|
|
@ -93,6 +93,14 @@ class WorkerLeaseInterface {
|
|||
const TaskID &task_id,
|
||||
const rpc::ClientCallback<rpc::CancelWorkerLeaseReply> &callback) = 0;
|
||||
|
||||
/// Report the backlog size of a given worker and a given scheduling class to the
|
||||
/// raylet.
|
||||
/// \param worker_id The ID of the worker that reports the backlog size.
|
||||
/// \param backlog_reports The backlog report for each scheduling class
|
||||
virtual void ReportWorkerBacklog(
|
||||
const WorkerID &worker_id,
|
||||
const std::vector<rpc::WorkerBacklogReport> &backlog_reports) = 0;
|
||||
|
||||
virtual ~WorkerLeaseInterface(){};
|
||||
};
|
||||
|
||||
|
@ -377,6 +385,11 @@ class RayletClient : public RayletClientInterface {
|
|||
ray::Status ReturnWorker(int worker_port, const WorkerID &worker_id,
|
||||
bool disconnect_worker) override;
|
||||
|
||||
/// Implements WorkerLeaseInterface.
|
||||
void ReportWorkerBacklog(
|
||||
const WorkerID &worker_id,
|
||||
const std::vector<rpc::WorkerBacklogReport> &backlog_reports) override;
|
||||
|
||||
/// Implements WorkerLeaseInterface.
|
||||
void ReleaseUnusedWorkers(
|
||||
const std::vector<WorkerID> &workers_in_use,
|
||||
|
|
|
@ -79,6 +79,9 @@ class NodeManagerWorkerClient
|
|||
/// Request a worker lease.
|
||||
VOID_RPC_CLIENT_METHOD(NodeManagerService, RequestWorkerLease, grpc_client_, )
|
||||
|
||||
/// Report task backlog information
|
||||
VOID_RPC_CLIENT_METHOD(NodeManagerService, ReportWorkerBacklog, grpc_client_, )
|
||||
|
||||
/// Return a worker lease.
|
||||
VOID_RPC_CLIENT_METHOD(NodeManagerService, ReturnWorker, grpc_client_, )
|
||||
|
||||
|
|
|
@ -28,6 +28,7 @@ namespace rpc {
|
|||
RPC_SERVICE_HANDLER(NodeManagerService, UpdateResourceUsage, -1) \
|
||||
RPC_SERVICE_HANDLER(NodeManagerService, RequestResourceReport, -1) \
|
||||
RPC_SERVICE_HANDLER(NodeManagerService, RequestWorkerLease, -1) \
|
||||
RPC_SERVICE_HANDLER(NodeManagerService, ReportWorkerBacklog, -1) \
|
||||
RPC_SERVICE_HANDLER(NodeManagerService, ReturnWorker, -1) \
|
||||
RPC_SERVICE_HANDLER(NodeManagerService, ReleaseUnusedWorkers, -1) \
|
||||
RPC_SERVICE_HANDLER(NodeManagerService, CancelWorkerLease, -1) \
|
||||
|
@ -70,6 +71,10 @@ class NodeManagerServiceHandler {
|
|||
RequestWorkerLeaseReply *reply,
|
||||
SendReplyCallback send_reply_callback) = 0;
|
||||
|
||||
virtual void HandleReportWorkerBacklog(const ReportWorkerBacklogRequest &request,
|
||||
ReportWorkerBacklogReply *reply,
|
||||
SendReplyCallback send_reply_callback) = 0;
|
||||
|
||||
virtual void HandleReturnWorker(const ReturnWorkerRequest &request,
|
||||
ReturnWorkerReply *reply,
|
||||
SendReplyCallback send_reply_callback) = 0;
|
||||
|
|
Loading…
Add table
Reference in a new issue