Skip to content

Conversation

@dayshah
Copy link
Contributor

@dayshah dayshah commented Oct 7, 2025

Why are these changes needed?

Currently if the task_execution_handler in _raylet.pyx throws an exception before we can get into the actual task handling code that catches all exceptions, cython will end up just continuing execution to return Status::OK. This means that we could finish ExecuteTask with Status::OK and a return objects vector that has empty objects leading to the dreaded objects_valid check failure.

One of the primary reasons this can happen is due to a RecursionError that can be thrown pretty easily (and in an unexpected way to users) due to the way we do asyncio through boost fibers which makes the Python interpreter think our recursion depth is very high. We try to handle this with this code

ray/python/ray/_raylet.pyx

Lines 681 to 701 in 534b0e4

cdef increase_recursion_limit():
"""Double the recusion limit if current depth is close to the limit"""
cdef:
CPyThreadState * s = <CPyThreadState *> PyThreadState_Get()
int current_limit = Py_GetRecursionLimit()
int new_limit = current_limit * 2
cdef extern from *:
"""
#if PY_VERSION_HEX >= 0x30C0000
#define CURRENT_DEPTH(x) ((x)->py_recursion_limit - (x)->py_recursion_remaining)
#elif PY_VERSION_HEX >= 0x30B00A4
#define CURRENT_DEPTH(x) ((x)->recursion_limit - (x)->recursion_remaining)
#else
#define CURRENT_DEPTH(x) ((x)->recursion_depth)
#endif
"""
int CURRENT_DEPTH(CPyThreadState *x)
int current_depth = CURRENT_DEPTH(s)
if current_limit - current_depth < 500:
Py_SetRecursionLimit(new_limit)

but it doesn't actually work in the Python 3.12 (PY_VERSION_HEX >= 0x30C0000) case. This is because CPython changed to having separate variables for c recursion and py recursion in 3.12 here python/cpython#96510.

CPython just sets the c_recursion_remaining of the thread state to C_RECURSION_LIMIT at thread state initialization and then only uses c_recursion_remaining, so we can just to increase c_recursion_remaining by 1k once it drops below 1k to keep it over 1k.
https://github.com/python/cpython/blob/bfb9e2f4a4e690099ec2ec53c08b90f4d64fde36/Python/pystate.c#L1353

Repro

Note that there is a test that does something very similar to this and triggers the same condition: test_asyncio.py::test_asyncio_actor_high_concurrency, but it seems like it might not running on Python 3.12 on CI now, trying to get to the bottom of that.

import ray
import asyncio

@ray.remote
class AsyncConcurrencyBatcher:
    def __init__(self, batch_size):
        self.batch = []
        self.event = asyncio.Event()
        self.batch_size = batch_size

    async def add(self, x):
        self.batch.append(x)
        if len(self.batch) >= self.batch_size:
            self.event.set()
        else:
            await self.event.wait()
        return sorted(self.batch)


batch_size = 746
actor = AsyncConcurrencyBatcher.remote(
    batch_size
)
ray.get([actor.add.remote(i) for i in range(batch_size)])

Related issue number

#57173

@dayshah dayshah added the go add ONLY when ready to merge, run all tests label Oct 7, 2025
@dayshah dayshah marked this pull request as ready for review October 7, 2025 05:33
@dayshah dayshah requested a review from a team as a code owner October 7, 2025 05:33
cursor[bot]

This comment was marked as outdated.

Signed-off-by: dayshah <[email protected]>
cursor[bot]

This comment was marked as outdated.

cdef increase_recursion_limit():
"""Double the recusion limit if current depth is close to the limit"""
"""
Ray does some weird things with asio fibers and asyncio to run asyncio actors.
Copy link
Collaborator

Choose a reason for hiding this comment

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

Killing boost fiber is on the @edoakes's wishlist

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ya ik I talked to him about it, he said he's gonna try prioritizing next quarter 👀

ctypedef struct CPyThreadState "PyThreadState":
int recursion_limit
int recursion_remaining
int c_recursion_remaining
Copy link
Collaborator

Choose a reason for hiding this comment

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

This field exists in all Python versions?

Copy link
Contributor Author

@dayshah dayshah Oct 7, 2025

Choose a reason for hiding this comment

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

no it doesn't but having it here doesn't cause any issues even if used with old python versions

recursion_limit and recursion_remaining also don't exist for all python versions

Copy link
Collaborator

Choose a reason for hiding this comment

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

oh, interesting

"""
#if PY_VERSION_HEX >= 0x30C0000
#define CURRENT_DEPTH(x) ((x)->py_recursion_limit - (x)->py_recursion_remaining)
std::pair<bool, int> DoOrGetRecursionMadness(PyThreadState *x) {
Copy link
Collaborator

Choose a reason for hiding this comment

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

What does Do mean here? I thought this function is supposed to be read-only.

Copy link
Contributor Author

@dayshah dayshah Oct 7, 2025

Choose a reason for hiding this comment

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

for 3.12 it has to increase c_recursion_remaining now

Copy link
Collaborator

Choose a reason for hiding this comment

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

Py_SetRecursionLimit is just a C function so it can be called inside cdef extern from *: as well so we can do all the adjustment inside the C function we define? (Just trying to see how to make the code cleaner)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Oh ya nice suggestion, didn't think of that.

Cleaned up code to just have everything happen in the cdef extern

Signed-off-by: dayshah <[email protected]>
Signed-off-by: dayshah <[email protected]>
cursor[bot]

This comment was marked as outdated.

"""
#if PY_VERSION_HEX >= 0x30C0000
#define CURRENT_DEPTH(x) ((x)->py_recursion_limit - (x)->py_recursion_remaining)
std::pair<bool, int> DoOrGetRecursionMadness(PyThreadState *x) {
Copy link
Collaborator

Choose a reason for hiding this comment

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

Py_SetRecursionLimit is just a C function so it can be called inside cdef extern from *: as well so we can do all the adjustment inside the C function we define? (Just trying to see how to make the code cleaner)

@ray-gardener ray-gardener bot added the core Issues that should be addressed in Ray Core label Oct 7, 2025
cursor[bot]

This comment was marked as outdated.

@dayshah dayshah requested a review from jjyao October 7, 2025 07:35
Copy link
Collaborator

@edoakes edoakes left a comment

Choose a reason for hiding this comment

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

Good work piecing this together. I think we have enough datapoints to indicate that we really need to get away from boost fiber...

Signed-off-by: dayshah <[email protected]>
@dayshah dayshah enabled auto-merge (squash) October 7, 2025 16:39
@dayshah dayshah merged commit de76a0b into ray-project:master Oct 7, 2025
6 of 7 checks passed
@dayshah dayshah deleted the recursion-objvalid branch October 7, 2025 18:44
aslonnie added a commit that referenced this pull request Oct 8, 2025
cherrypick #57247 #57253 #57138

Signed-off-by: Lonnie Liu <[email protected]>
aslonnie added a commit that referenced this pull request Oct 8, 2025
liulehui pushed a commit to liulehui/ray that referenced this pull request Oct 9, 2025
joshkodi pushed a commit to joshkodi/ray that referenced this pull request Oct 13, 2025
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
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.

3 participants