Skip to content

[core] Correctly fail worker lease request if a task becomes infeasible after scheduling#52295

Merged
jjyao merged 21 commits intoray-project:masterfrom
dayshah:fail-task-req
Jun 8, 2025
Merged

[core] Correctly fail worker lease request if a task becomes infeasible after scheduling#52295
jjyao merged 21 commits intoray-project:masterfrom
dayshah:fail-task-req

Conversation

@dayshah
Copy link
Copy Markdown
Contributor

@dayshah dayshah commented Apr 14, 2025

Why are these changes needed?

Here it's possible for the Work object to be erased from the dispatch queue without having the callback that replies to the WorkerLeaseRequest called (this happens when the task becomes infeasible after its added to the local task manager dispatch queue). The reply callback exists inside the Work object. The task submitter will then be left hanging forever while the reply callback is nowhere to be found.

Ideally this should only happen in the local task manager if resources total value change after the task is scheduled on a node and the only resource whose total can be dynamically changed is placement group bundles when the placement group is removed. However when placement group is removed, all tasks inside the local task manager should be cancelled immediately so this case should ideally not be hit.

Even though we think this case should never happen, we're deciding to log an error and reply instead of check fail since there are signs that it can possibly happen in a way that we are not aware of as of today.

To do this we also need to refactor a bit to have a CancelTaskToDispatch that doesn't erase from tasks_to_dispatch_.

Related issue number

Checks

  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
    • I've added any new APIs to the API Reference. For example, if I added a
      method in Tune, I've added it in doc/source/tune/api/ under the
      corresponding .rst file.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

Signed-off-by: dayshah <dhyey2019@gmail.com>
@dayshah dayshah added the go add ONLY when ready to merge, run all tests label Apr 14, 2025
@dayshah dayshah marked this pull request as ready for review April 21, 2025 23:28
Signed-off-by: dayshah <dhyey2019@gmail.com>
@dayshah dayshah requested review from edoakes and jjyao April 21, 2025 23:42
@dayshah dayshah changed the title [core] Correctly fail worker lease request [core] Correctly fail worker lease request when a task becomes infeasible after scheduling Apr 21, 2025
@dayshah dayshah changed the title [core] Correctly fail worker lease request when a task becomes infeasible after scheduling [core] Correctly fail worker lease request if a task becomes infeasible after scheduling Apr 21, 2025
Comment on lines +410 to +416
// This should ideally only happen when resources on the node are changed,
// and the only dynamic resource is placement group bundles.
for (const auto &work : dispatch_queue) {
CancelTask(work->task.GetTaskSpecification().TaskId(),
rpc::RequestWorkerLeaseReply::SCHEDULING_CANCELLED_UNSCHEDULABLE,
"Scheduling failed due to the task becoming infeasible.");
}
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Actually after #51125 (comment), this should be a check failure.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Ya makes sense, so these tasks should just be cancelled from node manager when resources are lost, and we should never hit this case.

I'm still a bit nervous about turning this into a check failure though because of signs of possibly hitting this case even without using placement groups. I can update the comment and do an RAY_LOG(ERROR) that this case should never be hit and reported as an issue on github maybe?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

added a log

Signed-off-by: Dhyey Shah <dhyey2019@gmail.com>
Signed-off-by: Dhyey Shah <dhyey2019@gmail.com>
Signed-off-by: Dhyey Shah <dhyey2019@gmail.com>
dayshah added 2 commits May 27, 2025 11:59
Signed-off-by: dayshah <dhyey2019@gmail.com>
Signed-off-by: dayshah <dhyey2019@gmail.com>
@dayshah dayshah requested a review from jjyao May 27, 2025 20:04
@dayshah
Copy link
Copy Markdown
Contributor Author

dayshah commented May 27, 2025

@jjyao to review

if (is_infeasible) {
// TODO(scv119): fail the request.
// Call CancelTask
TaskSpecification front_task = dispatch_queue.front()->task.GetTaskSpecification();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Should we add a unit test even though we think it should never happen?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Added a unit test, good thing you asked for a unit test, discovered a bug 💀

It would segfault before because the erase_if call erases the queue from the map if the queue size gets to 0, and the reference to the queue would be invalidated.

dayshah added 3 commits May 31, 2025 14:52
Signed-off-by: dayshah <dhyey2019@gmail.com>
Signed-off-by: dayshah <dhyey2019@gmail.com>
Signed-off-by: dayshah <dhyey2019@gmail.com>
@dayshah dayshah requested a review from jjyao May 31, 2025 23:53
@edoakes edoakes requested a review from a team June 2, 2025 15:49
Signed-off-by: dayshah <dhyey2019@gmail.com>
Signed-off-by: dayshah <dhyey2019@gmail.com>
@dayshah dayshah requested a review from jjyao June 3, 2025 06:52
Signed-off-by: dayshah <dhyey2019@gmail.com>
@dayshah dayshah requested a review from jjyao June 3, 2025 16:37
Signed-off-by: dayshah <dhyey2019@gmail.com>
Signed-off-by: dayshah <dhyey2019@gmail.com>
@dayshah dayshah requested a review from jjyao June 4, 2025 04:50
Signed-off-by: dayshah <dhyey2019@gmail.com>
@dayshah dayshah requested a review from jjyao June 4, 2025 06:00
int num_callbacks_called = 0;
auto callback = [&num_callbacks_called]() { ++num_callbacks_called; };
rpc::RequestWorkerLeaseReply reply1;
local_task_manager_->WaitForTaskArgsRequests(std::make_shared<internal::Work>(
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

We should use QueueAndScheduleTask to add the task. To avoid the check failure, we should update the total resource after the tasks are queued to make the task infeasible.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

done

the test above actually tests behavior that's impossible with the public api

dayshah added 2 commits June 6, 2025 16:52
Signed-off-by: dayshah <dhyey2019@gmail.com>
Signed-off-by: dayshah <dhyey2019@gmail.com>
@dayshah dayshah requested a review from jjyao June 7, 2025 23:15
@dayshah
Copy link
Copy Markdown
Contributor Author

dayshah commented Jun 7, 2025

@jjyao ready for re-re-review

Signed-off-by: dayshah <dhyey2019@gmail.com>
Signed-off-by: dayshah <dhyey2019@gmail.com>
@jjyao jjyao enabled auto-merge (squash) June 8, 2025 04:38
@jjyao jjyao merged commit 9d6c721 into ray-project:master Jun 8, 2025
6 checks passed
@dayshah dayshah deleted the fail-task-req branch June 8, 2025 05:33
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

go add ONLY when ready to merge, run all tests

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants