Skip to content

Conversation

@jjyao
Copy link
Collaborator

@jjyao jjyao commented Oct 17, 2025

Description

Symptom

(raylet, ip=100.64.64.117) {"asctime":"2025-10-15 05:48:10,420","levelname":"E","message":"A task got scheduled to a node even though it was infeasible. Please report an issue on GitHub.\nTask: Type=NORMAL_TASK, Language=PYTHON, Resources: {CPU: 0.99, memory: 168, }, function_descriptor={type=PythonFunctionDescriptor, module_name=ray.data._internal.execution.operators.map_operator, class_name=, function_name=_map_task, function_hash=cefe38212d404ebb9356658cbe7bec90}, task_id=665063b9b316bad4955fe79478e285f461a0bd4a02000000, task_name=Write, job_id=02000000, num_args=10, num_returns=1, max_retries=-1, depth=1, attempt_number=0, runtime_env_hash=-457748260, eager_install=1, setup_timeout_seconds=600","component":"raylet","filename":"local_task_manager.cc","lineno":404}

and task fails with TaskUnschedulableError

Root Cause

In master, we assumed that once a lease request is added to the local lease manager, the lease cannot become infeasible. It turned out to be not true since a node can be drained and once a node is in the draining state, it's excluded from the scheduling decision and a lease can now become infeasible (if the draining node was the only feasible node in the cluster).

Fix

When a node is being drained, we should fail fast on all leases that are in the local lease manager regardless whether they are waiting for workers, pulling args or what and add them back to the cluster lease manager so scheduler can run the scheduling logic again to find a new node.

@jjyao jjyao added the go add ONLY when ready to merge, run all tests label Oct 17, 2025
jjyao added 3 commits October 17, 2025 08:38
Signed-off-by: Jiajun Yao <[email protected]>
Signed-off-by: Jiajun Yao <[email protected]>
Signed-off-by: Jiajun Yao <[email protected]>
Comment on lines +35 to +46
namespace {
void ReplyCancelled(const std::shared_ptr<internal::Work> &work,
rpc::RequestWorkerLeaseReply::SchedulingFailureType failure_type,
const std::string &scheduling_failure_message) {
auto reply = work->reply_;
reply->set_canceled(true);
reply->set_failure_type(failure_type);
reply->set_scheduling_failure_message(scheduling_failure_message);
work->send_reply_callback_(Status::OK(), nullptr, nullptr);
}
} // namespace

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pure move.

Comment on lines 2082 to 2096
if (reply->is_accepted()) {
// Fail fast on the leases in the local lease maanger
// and add them back to the cluster lease manager so a new node
// can be selected by the scheduler.
auto cancelled_works = local_lease_manager_.CancelLeasesWithoutReply(
[&](const std::shared_ptr<internal::Work> &work) { return true; });
for (const auto &work : cancelled_works) {
cluster_lease_manager_.QueueAndScheduleLease(work->lease_,
work->grant_or_reject_,
work->is_selected_based_on_locality_,
work->reply_,
work->send_reply_callback_);
}
}

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actual fix

@jjyao jjyao marked this pull request as ready for review October 17, 2025 22:29
@jjyao jjyao requested a review from a team as a code owner October 17, 2025 22:29
@ray-gardener ray-gardener bot added the core Issues that should be addressed in Ray Core label Oct 18, 2025
Copy link
Contributor

@dayshah dayshah left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

logic lgtm, nits

Python integration test looks a little prone to flakiness, probably not necessary to test this from python

Signed-off-by: Jiajun Yao <[email protected]>
@jjyao jjyao enabled auto-merge (squash) October 19, 2025 04:19
@jjyao jjyao merged commit 993139e into ray-project:master Oct 19, 2025
7 checks passed
@jjyao jjyao deleted the jjyao/draining branch October 19, 2025 15:45
justinyeh1995 pushed a commit to justinyeh1995/ray that referenced this pull request Oct 20, 2025
xinyuangui2 pushed a commit to xinyuangui2/ray that referenced this pull request Oct 22, 2025
elliot-barn pushed a commit that referenced this pull request Oct 23, 2025
landscapepainter pushed a commit to landscapepainter/ray that referenced this pull request Nov 17, 2025
Aydin-ab pushed a commit to Aydin-ab/ray-aydin that referenced this pull request Nov 19, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

core Issues that should be addressed in Ray Core go add ONLY when ready to merge, run all tests

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants