Spread even if nodes are not available (#23445)

Several changes to make spread scheduling work better under load:

* When nodes are not available, spread among feasible nodes.
* If grant_or_reject is true, don't spill back if the selected node is not available.
* Don't spill due to waiting for dependencies for spread tasks.
This commit is contained in:
Jiajun Yao 2022-04-20 07:35:15 -07:00 committed by GitHub
parent 261a8a7470
commit 6cfec51d1e
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
9 changed files with 110 additions and 47 deletions

View file

@ -319,6 +319,11 @@ bool TaskSpecification::IsActorTask() const {
return message_->type() == TaskType::ACTOR_TASK; return message_->type() == TaskType::ACTOR_TASK;
} }
bool TaskSpecification::IsSpreadSchedulingStrategy() const {
return message_->scheduling_strategy().scheduling_strategy_case() ==
rpc::SchedulingStrategy::SchedulingStrategyCase::kSpreadSchedulingStrategy;
}
// === Below are getter methods specific to actor creation tasks. // === Below are getter methods specific to actor creation tasks.
ActorID TaskSpecification::ActorCreationId() const { ActorID TaskSpecification::ActorCreationId() const {

View file

@ -360,6 +360,8 @@ class TaskSpecification : public MessageWrapper<rpc::TaskSpec> {
bool ExecuteOutOfOrder() const; bool ExecuteOutOfOrder() const;
bool IsSpreadSchedulingStrategy() const;
private: private:
void ComputeResources(); void ComputeResources();

View file

@ -315,19 +315,30 @@ void LocalTaskManager::SpillWaitingTasks() {
// pulled). If this is true, then we should force the task onto a remote // pulled). If this is true, then we should force the task onto a remote
// feasible node, even if we have enough resources available locally for // feasible node, even if we have enough resources available locally for
// placement. // placement.
bool force_spillback = task_dependency_manager_.TaskDependenciesBlocked(task_id); bool task_dependencies_blocked =
task_dependency_manager_.TaskDependenciesBlocked(task_id);
RAY_LOG(DEBUG) << "Attempting to spill back waiting task " << task_id RAY_LOG(DEBUG) << "Attempting to spill back waiting task " << task_id
<< " to remote node. Force spillback? " << force_spillback; << " to remote node. Dependencies blocked? "
<< task_dependencies_blocked;
bool is_infeasible; bool is_infeasible;
// TODO(swang): The policy currently does not account for the amount of // TODO(swang): The policy currently does not account for the amount of
// object store memory availability. Ideally, we should pick the node with // object store memory availability. Ideally, we should pick the node with
// the most memory availability. // the most memory availability.
auto scheduling_node_id = cluster_resource_scheduler_->GetBestSchedulableNode( scheduling::NodeID scheduling_node_id;
(*it)->task.GetTaskSpecification(), if (!task.GetTaskSpecification().IsSpreadSchedulingStrategy()) {
scheduling_node_id = cluster_resource_scheduler_->GetBestSchedulableNode(
task.GetTaskSpecification(),
/*prioritize_local_node*/ true, /*prioritize_local_node*/ true,
/*exclude_local_node*/ force_spillback, /*exclude_local_node*/ task_dependencies_blocked,
/*requires_object_store_memory*/ true, /*requires_object_store_memory*/ true,
&is_infeasible); &is_infeasible);
} else {
// If scheduling strategy is spread, we prefer honoring spread decision
// and waiting for task dependencies to be pulled
// locally than spilling back and causing uneven spread.
scheduling_node_id = scheduling::NodeID(self_node_id_.Binary());
}
if (!scheduling_node_id.IsNil() && if (!scheduling_node_id.IsNil() &&
scheduling_node_id.Binary() != self_node_id_.Binary()) { scheduling_node_id.Binary() != self_node_id_.Binary()) {
NodeID node_id = NodeID::FromBinary(scheduling_node_id.Binary()); NodeID node_id = NodeID::FromBinary(scheduling_node_id.Binary());

View file

@ -210,9 +210,11 @@ bool ClusterResourceScheduler::AllocateRemoteTaskResources(
} }
bool ClusterResourceScheduler::IsSchedulableOnNode( bool ClusterResourceScheduler::IsSchedulableOnNode(
scheduling::NodeID node_id, const absl::flat_hash_map<std::string, double> &shape) { scheduling::NodeID node_id,
const absl::flat_hash_map<std::string, double> &shape,
bool requires_object_store_memory) {
auto resource_request = auto resource_request =
ResourceMapToResourceRequest(shape, /*requires_object_store_memory=*/false); ResourceMapToResourceRequest(shape, requires_object_store_memory);
return IsSchedulable(resource_request, node_id); return IsSchedulable(resource_request, node_id);
} }
@ -226,21 +228,34 @@ scheduling::NodeID ClusterResourceScheduler::GetBestSchedulableNode(
// going through the full hybrid policy since we don't want spillback. // going through the full hybrid policy since we don't want spillback.
if (prioritize_local_node && !exclude_local_node && if (prioritize_local_node && !exclude_local_node &&
IsSchedulableOnNode(local_node_id_, IsSchedulableOnNode(local_node_id_,
task_spec.GetRequiredResources().GetResourceMap())) { task_spec.GetRequiredResources().GetResourceMap(),
requires_object_store_memory)) {
*is_infeasible = false; *is_infeasible = false;
return local_node_id_; return local_node_id_;
} }
// This argument is used to set violation, which is an unsupported feature now. // This argument is used to set violation, which is an unsupported feature now.
int64_t _unused; int64_t _unused;
return GetBestSchedulableNode( scheduling::NodeID best_node =
task_spec.GetRequiredPlacementResources().GetResourceMap(), GetBestSchedulableNode(task_spec.GetRequiredPlacementResources().GetResourceMap(),
task_spec.GetMessage().scheduling_strategy(), task_spec.GetMessage().scheduling_strategy(),
requires_object_store_memory, requires_object_store_memory,
task_spec.IsActorCreationTask(), task_spec.IsActorCreationTask(),
exclude_local_node, exclude_local_node,
&_unused, &_unused,
is_infeasible); is_infeasible);
// If there is no other available nodes, prefer waiting on the local node
// since the local node is chosen for a reason (e.g. spread).
if (prioritize_local_node && !best_node.IsNil() &&
!IsSchedulableOnNode(best_node,
task_spec.GetRequiredResources().GetResourceMap(),
requires_object_store_memory)) {
*is_infeasible = false;
return local_node_id_;
}
return best_node;
} }
SchedulingResult ClusterResourceScheduler::Schedule( SchedulingResult ClusterResourceScheduler::Schedule(

View file

@ -114,7 +114,8 @@ class ClusterResourceScheduler {
/// \param node_name Name of the node. /// \param node_name Name of the node.
/// \param shape The resource demand's shape. /// \param shape The resource demand's shape.
bool IsSchedulableOnNode(scheduling::NodeID node_id, bool IsSchedulableOnNode(scheduling::NodeID node_id,
const absl::flat_hash_map<std::string, double> &shape); const absl::flat_hash_map<std::string, double> &shape,
bool requires_object_store_memory);
LocalResourceManager &GetLocalResourceManager() { return *local_resource_manager_; } LocalResourceManager &GetLocalResourceManager() { return *local_resource_manager_; }
ClusterResourceManager &GetClusterResourceManager() { ClusterResourceManager &GetClusterResourceManager() {

View file

@ -1646,6 +1646,12 @@ TEST_F(ClusterTaskManagerTest, TestSpillWaitingTasks) {
auto missing_arg = task.GetTaskSpecification().GetDependencyIds()[0]; auto missing_arg = task.GetTaskSpecification().GetDependencyIds()[0];
missing_objects_.insert(missing_arg); missing_objects_.insert(missing_arg);
} }
if (i == 0) {
const_cast<TaskSpecification &>(task.GetTaskSpecification())
.GetMutableMessage()
.mutable_scheduling_strategy()
->mutable_spread_scheduling_strategy();
}
task_manager_.QueueAndScheduleTask(task, false, false, replies[i].get(), callback); task_manager_.QueueAndScheduleTask(task, false, false, replies[i].get(), callback);
pool_.TriggerCallbacks(); pool_.TriggerCallbacks();
} }
@ -1696,6 +1702,12 @@ TEST_F(ClusterTaskManagerTest, TestSpillWaitingTasks) {
// One task dispatched. // One task dispatched.
ASSERT_EQ(replies[4]->worker_address().port(), 1234); ASSERT_EQ(replies[4]->worker_address().port(), 1234);
// Spread task won't be spilled due to waiting for dependencies.
AddNode(remote_node_id, 8);
task_manager_.ScheduleAndDispatchTasks();
ASSERT_EQ(num_callbacks, 4);
ASSERT_EQ(replies[0]->retry_at_raylet_address().raylet_id(), "");
RayTask finished_task; RayTask finished_task;
local_task_manager_->TaskFinished(leased_workers_.begin()->second, &finished_task); local_task_manager_->TaskFinished(leased_workers_.begin()->second, &finished_task);
leased_workers_.clear(); leased_workers_.clear();

View file

@ -119,9 +119,9 @@ TEST_F(SchedulingPolicyTest, NodeAffinityPolicyTest) {
TEST_F(SchedulingPolicyTest, SpreadPolicyTest) { TEST_F(SchedulingPolicyTest, SpreadPolicyTest) {
ResourceRequest req = ResourceMapToResourceRequest({{"CPU", 1}}, false); ResourceRequest req = ResourceMapToResourceRequest({{"CPU", 1}}, false);
nodes.emplace(local_node, CreateNodeResources(20, 20, 0, 0, 0, 0)); nodes.emplace(local_node, CreateNodeResources(20, 20, 0, 0, 0, 1));
// Unavailable node // Unavailable node
nodes.emplace(remote_node, CreateNodeResources(0, 20, 0, 0, 0, 0)); nodes.emplace(remote_node, CreateNodeResources(0, 20, 0, 0, 0, 1));
// Infeasible node // Infeasible node
nodes.emplace(remote_node_2, CreateNodeResources(0, 0, 0, 0, 0, 0)); nodes.emplace(remote_node_2, CreateNodeResources(0, 0, 0, 0, 0, 0));
nodes.emplace(remote_node_3, CreateNodeResources(20, 20, 0, 0, 0, 0)); nodes.emplace(remote_node_3, CreateNodeResources(20, 20, 0, 0, 0, 0));
@ -137,8 +137,20 @@ TEST_F(SchedulingPolicyTest, SpreadPolicyTest) {
ASSERT_EQ(to_schedule, remote_node_3); ASSERT_EQ(to_schedule, remote_node_3);
to_schedule = scheduling_policy.Schedule( to_schedule = scheduling_policy.Schedule(
req, SchedulingOptions::Spread(/*force_spillback=*/true, false)); req, SchedulingOptions::Spread(/*avoid_local_node=*/true, false));
ASSERT_EQ(to_schedule, remote_node_3); ASSERT_EQ(to_schedule, remote_node_3);
// Spread across feasible nodes if there is no available nodes
req = ResourceMapToResourceRequest({{"GPU", 1}}, false);
to_schedule = scheduling_policy.Schedule(req, SchedulingOptions::Spread(false, false));
ASSERT_EQ(to_schedule, local_node);
to_schedule = scheduling_policy.Schedule(req, SchedulingOptions::Spread(false, false));
ASSERT_EQ(to_schedule, remote_node);
to_schedule = scheduling_policy.Schedule(
req, SchedulingOptions::Spread(false, /*require_node_available=*/true));
ASSERT_TRUE(to_schedule.IsNil());
} }
TEST_F(SchedulingPolicyTest, RandomPolicyTest) { TEST_F(SchedulingPolicyTest, RandomPolicyTest) {

View file

@ -35,6 +35,11 @@ scheduling::NodeID SpreadSchedulingPolicy::Schedule(
} }
std::sort(round.begin(), round.end()); std::sort(round.begin(), round.end());
// Spread among available nodes first.
// If there is no available nodes, we spread among feasible nodes.
for (bool available_nodes_only :
(options.require_node_available ? std::vector<bool>{true}
: std::vector<bool>{true, false})) {
size_t round_index = spread_scheduling_next_index_; size_t round_index = spread_scheduling_next_index_;
for (size_t i = 0; i < round.size(); ++i, ++round_index) { for (size_t i = 0; i < round.size(); ++i, ++round_index) {
const auto &node_id = round[round_index % round.size()]; const auto &node_id = round[round_index % round.size()];
@ -42,17 +47,22 @@ scheduling::NodeID SpreadSchedulingPolicy::Schedule(
if (node_id == local_node_id_ && options.avoid_local_node) { if (node_id == local_node_id_ && options.avoid_local_node) {
continue; continue;
} }
if (!is_node_available_(node_id) || if (!is_node_alive_(node_id) || !node.GetLocalView().IsFeasible(resource_request)) {
!node.GetLocalView().IsFeasible(resource_request) || continue;
!node.GetLocalView().IsAvailable(resource_request, true)) { }
if (available_nodes_only &&
!node.GetLocalView().IsAvailable(resource_request,
/*ignore_pull_manager_at_capacity=*/false)) {
continue; continue;
} }
spread_scheduling_next_index_ = ((round_index + 1) % round.size()); spread_scheduling_next_index_ = ((round_index + 1) % round.size());
return node_id; return node_id;
} }
options.scheduling_type = SchedulingType::HYBRID; }
return hybrid_policy_.Schedule(resource_request, options);
return scheduling::NodeID::Nil();
} }
} // namespace raylet_scheduling_policy } // namespace raylet_scheduling_policy

View file

@ -28,11 +28,8 @@ class SpreadSchedulingPolicy : public ISchedulingPolicy {
public: public:
SpreadSchedulingPolicy(scheduling::NodeID local_node_id, SpreadSchedulingPolicy(scheduling::NodeID local_node_id,
const absl::flat_hash_map<scheduling::NodeID, Node> &nodes, const absl::flat_hash_map<scheduling::NodeID, Node> &nodes,
std::function<bool(scheduling::NodeID)> is_node_available) std::function<bool(scheduling::NodeID)> is_node_alive)
: local_node_id_(local_node_id), : local_node_id_(local_node_id), nodes_(nodes), is_node_alive_(is_node_alive) {}
nodes_(nodes),
is_node_available_(is_node_available),
hybrid_policy_(local_node_id_, nodes_, is_node_available_) {}
scheduling::NodeID Schedule(const ResourceRequest &resource_request, scheduling::NodeID Schedule(const ResourceRequest &resource_request,
SchedulingOptions options) override; SchedulingOptions options) override;
@ -47,9 +44,7 @@ class SpreadSchedulingPolicy : public ISchedulingPolicy {
// but it should still be better than always scanning from 0 for spread scheduling. // but it should still be better than always scanning from 0 for spread scheduling.
size_t spread_scheduling_next_index_ = 0; size_t spread_scheduling_next_index_ = 0;
/// Function Checks if node is alive. /// Function Checks if node is alive.
std::function<bool(scheduling::NodeID)> is_node_available_; std::function<bool(scheduling::NodeID)> is_node_alive_;
/// Instance of hybrid policy;
HybridSchedulingPolicy hybrid_policy_;
}; };
} // namespace raylet_scheduling_policy } // namespace raylet_scheduling_policy
} // namespace ray } // namespace ray