mirror of
https://github.com/vale981/ray
synced 2025-03-06 10:31:39 -05:00
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:
parent
261a8a7470
commit
6cfec51d1e
9 changed files with 110 additions and 47 deletions
|
@ -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 {
|
||||||
|
|
|
@ -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();
|
||||||
|
|
||||||
|
|
|
@ -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()) {
|
||||||
/*prioritize_local_node*/ true,
|
scheduling_node_id = cluster_resource_scheduler_->GetBestSchedulableNode(
|
||||||
/*exclude_local_node*/ force_spillback,
|
task.GetTaskSpecification(),
|
||||||
/*requires_object_store_memory*/ true,
|
/*prioritize_local_node*/ true,
|
||||||
&is_infeasible);
|
/*exclude_local_node*/ task_dependencies_blocked,
|
||||||
|
/*requires_object_store_memory*/ true,
|
||||||
|
&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());
|
||||||
|
|
|
@ -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(
|
||||||
|
|
|
@ -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() {
|
||||||
|
|
|
@ -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();
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
|
@ -35,24 +35,34 @@ scheduling::NodeID SpreadSchedulingPolicy::Schedule(
|
||||||
}
|
}
|
||||||
std::sort(round.begin(), round.end());
|
std::sort(round.begin(), round.end());
|
||||||
|
|
||||||
size_t round_index = spread_scheduling_next_index_;
|
// Spread among available nodes first.
|
||||||
for (size_t i = 0; i < round.size(); ++i, ++round_index) {
|
// If there is no available nodes, we spread among feasible nodes.
|
||||||
const auto &node_id = round[round_index % round.size()];
|
for (bool available_nodes_only :
|
||||||
const auto &node = map_find_or_die(nodes_, node_id);
|
(options.require_node_available ? std::vector<bool>{true}
|
||||||
if (node_id == local_node_id_ && options.avoid_local_node) {
|
: std::vector<bool>{true, false})) {
|
||||||
continue;
|
size_t round_index = spread_scheduling_next_index_;
|
||||||
}
|
for (size_t i = 0; i < round.size(); ++i, ++round_index) {
|
||||||
if (!is_node_available_(node_id) ||
|
const auto &node_id = round[round_index % round.size()];
|
||||||
!node.GetLocalView().IsFeasible(resource_request) ||
|
const auto &node = map_find_or_die(nodes_, node_id);
|
||||||
!node.GetLocalView().IsAvailable(resource_request, true)) {
|
if (node_id == local_node_id_ && options.avoid_local_node) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
if (!is_node_alive_(node_id) || !node.GetLocalView().IsFeasible(resource_request)) {
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
spread_scheduling_next_index_ = ((round_index + 1) % round.size());
|
if (available_nodes_only &&
|
||||||
return node_id;
|
!node.GetLocalView().IsAvailable(resource_request,
|
||||||
|
/*ignore_pull_manager_at_capacity=*/false)) {
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
spread_scheduling_next_index_ = ((round_index + 1) % round.size());
|
||||||
|
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
|
||||||
|
|
|
@ -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
|
||||||
|
|
Loading…
Add table
Reference in a new issue