gh-112175: Add eval_breaker to PyThreadState (#115194)

This change adds an `eval_breaker` field to `PyThreadState`. The primary
motivation is for performance in free-threaded builds: with thread-local eval
breakers, we can stop a specific thread (e.g., for an async exception) without
interrupting other threads.

The source of truth for the global instrumentation version is stored in the
`instrumentation_version` field in PyInterpreterState. Threads usually read the
version from their local `eval_breaker`, where it continues to be colocated
with the eval breaker bits.
This commit is contained in:
Brett Simmers 2024-02-20 06:57:48 -08:00 committed by GitHub
parent e71468ba4f
commit 0749244d13
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
19 changed files with 265 additions and 172 deletions

View file

@ -68,6 +68,11 @@ struct _ts {
PyThreadState *next; PyThreadState *next;
PyInterpreterState *interp; PyInterpreterState *interp;
/* The global instrumentation version in high bits, plus flags indicating
when to break out of the interpreter loop in lower bits. See details in
pycore_ceval.h. */
uintptr_t eval_breaker;
struct { struct {
/* Has been initialized to a safe state. /* Has been initialized to a safe state.

View file

@ -42,7 +42,7 @@ PyAPI_FUNC(int) _PyEval_MakePendingCalls(PyThreadState *);
extern void _Py_FinishPendingCalls(PyThreadState *tstate); extern void _Py_FinishPendingCalls(PyThreadState *tstate);
extern void _PyEval_InitState(PyInterpreterState *); extern void _PyEval_InitState(PyInterpreterState *);
extern void _PyEval_SignalReceived(PyInterpreterState *interp); extern void _PyEval_SignalReceived(void);
// bitwise flags: // bitwise flags:
#define _Py_PENDING_MAINTHREADONLY 1 #define _Py_PENDING_MAINTHREADONLY 1
@ -55,7 +55,6 @@ PyAPI_FUNC(int) _PyEval_AddPendingCall(
void *arg, void *arg,
int flags); int flags);
extern void _PyEval_SignalAsyncExc(PyInterpreterState *interp);
#ifdef HAVE_FORK #ifdef HAVE_FORK
extern PyStatus _PyEval_ReInitThreads(PyThreadState *tstate); extern PyStatus _PyEval_ReInitThreads(PyThreadState *tstate);
#endif #endif
@ -200,40 +199,43 @@ int _PyEval_UnpackIterable(PyThreadState *tstate, PyObject *v, int argcnt, int a
void _PyEval_FrameClearAndPop(PyThreadState *tstate, _PyInterpreterFrame *frame); void _PyEval_FrameClearAndPop(PyThreadState *tstate, _PyInterpreterFrame *frame);
#define _PY_GIL_DROP_REQUEST_BIT 0 /* Bits that can be set in PyThreadState.eval_breaker */
#define _PY_SIGNALS_PENDING_BIT 1 #define _PY_GIL_DROP_REQUEST_BIT (1U << 0)
#define _PY_CALLS_TO_DO_BIT 2 #define _PY_SIGNALS_PENDING_BIT (1U << 1)
#define _PY_ASYNC_EXCEPTION_BIT 3 #define _PY_CALLS_TO_DO_BIT (1U << 2)
#define _PY_GC_SCHEDULED_BIT 4 #define _PY_ASYNC_EXCEPTION_BIT (1U << 3)
#define _PY_EVAL_PLEASE_STOP_BIT 5 #define _PY_GC_SCHEDULED_BIT (1U << 4)
#define _PY_EVAL_EXPLICIT_MERGE_BIT 6 #define _PY_EVAL_PLEASE_STOP_BIT (1U << 5)
#define _PY_EVAL_EXPLICIT_MERGE_BIT (1U << 6)
/* Reserve a few bits for future use */ /* Reserve a few bits for future use */
#define _PY_EVAL_EVENTS_BITS 8 #define _PY_EVAL_EVENTS_BITS 8
#define _PY_EVAL_EVENTS_MASK ((1 << _PY_EVAL_EVENTS_BITS)-1) #define _PY_EVAL_EVENTS_MASK ((1 << _PY_EVAL_EVENTS_BITS)-1)
static inline void static inline void
_Py_set_eval_breaker_bit(PyInterpreterState *interp, uint32_t bit, uint32_t set) _Py_set_eval_breaker_bit(PyThreadState *tstate, uintptr_t bit)
{ {
assert(set == 0 || set == 1); _Py_atomic_or_uintptr(&tstate->eval_breaker, bit);
uintptr_t to_set = set << bit;
uintptr_t mask = ((uintptr_t)1) << bit;
uintptr_t old = _Py_atomic_load_uintptr(&interp->ceval.eval_breaker);
if ((old & mask) == to_set) {
return;
}
uintptr_t new;
do {
new = (old & ~mask) | to_set;
} while (!_Py_atomic_compare_exchange_uintptr(&interp->ceval.eval_breaker, &old, new));
} }
static inline bool static inline void
_Py_eval_breaker_bit_is_set(PyInterpreterState *interp, int32_t bit) _Py_unset_eval_breaker_bit(PyThreadState *tstate, uintptr_t bit)
{ {
return _Py_atomic_load_uintptr_relaxed(&interp->ceval.eval_breaker) & (((uintptr_t)1) << bit); _Py_atomic_and_uintptr(&tstate->eval_breaker, ~bit);
} }
static inline int
_Py_eval_breaker_bit_is_set(PyThreadState *tstate, uintptr_t bit)
{
uintptr_t b = _Py_atomic_load_uintptr_relaxed(&tstate->eval_breaker);
return (b & bit) != 0;
}
// Free-threaded builds use these functions to set or unset a bit on all
// threads in the given interpreter.
void _Py_set_eval_breaker_bit_all(PyInterpreterState *interp, uintptr_t bit);
void _Py_unset_eval_breaker_bit_all(PyInterpreterState *interp, uintptr_t bit);
#ifdef __cplusplus #ifdef __cplusplus
} }

View file

@ -78,13 +78,10 @@ struct _ceval_runtime_state {
struct _ceval_state { struct _ceval_state {
/* This single variable consolidates all requests to break out of /* This variable holds the global instrumentation version. When a thread is
* the fast path in the eval loop. running, this value is overlaid onto PyThreadState.eval_breaker so that
* It is by far the hottest field in this struct and changes in the instrumentation version will trigger the eval breaker. */
* should be placed at the beginning. */ uintptr_t instrumentation_version;
uintptr_t eval_breaker;
/* Avoid false sharing */
int64_t padding[7];
int recursion_limit; int recursion_limit;
struct _gil_runtime_state *gil; struct _gil_runtime_state *gil;
int own_gil; int own_gil;

View file

@ -286,7 +286,7 @@ extern PyObject *_PyGC_GetReferrers(PyInterpreterState *interp, PyObject *objs);
// Functions to clear types free lists // Functions to clear types free lists
extern void _PyGC_ClearAllFreeLists(PyInterpreterState *interp); extern void _PyGC_ClearAllFreeLists(PyInterpreterState *interp);
extern void _Py_ScheduleGC(PyInterpreterState *interp); extern void _Py_ScheduleGC(PyThreadState *tstate);
extern void _Py_RunGC(PyThreadState *tstate); extern void _Py_RunGC(PyThreadState *tstate);
#ifdef __cplusplus #ifdef __cplusplus

View file

@ -191,7 +191,10 @@ typedef struct pyruntimestate {
int64_t next_id; int64_t next_id;
} interpreters; } interpreters;
/* Platform-specific identifier and PyThreadState, respectively, for the
main thread in the main interpreter. */
unsigned long main_thread; unsigned long main_thread;
PyThreadState *main_tstate;
/* ---------- IMPORTANT --------------------------- /* ---------- IMPORTANT ---------------------------
The fields above this line are declared as early as The fields above this line are declared as early as

View file

@ -0,0 +1 @@
Every ``PyThreadState`` now has its own ``eval_breaker``, allowing specific threads to be interrupted.

View file

@ -276,11 +276,7 @@ trip_signal(int sig_num)
cleared in PyErr_CheckSignals() before .tripped. */ cleared in PyErr_CheckSignals() before .tripped. */
_Py_atomic_store_int(&is_tripped, 1); _Py_atomic_store_int(&is_tripped, 1);
/* Signals are always handled by the main interpreter */ _PyEval_SignalReceived();
PyInterpreterState *interp = _PyInterpreterState_Main();
/* Notify ceval.c */
_PyEval_SignalReceived(interp);
/* And then write to the wakeup fd *after* setting all the globals and /* And then write to the wakeup fd *after* setting all the globals and
doing the _PyEval_SignalReceived. We used to write to the wakeup fd doing the _PyEval_SignalReceived. We used to write to the wakeup fd
@ -303,6 +299,7 @@ trip_signal(int sig_num)
int fd = wakeup.fd; int fd = wakeup.fd;
if (fd != INVALID_FD) { if (fd != INVALID_FD) {
PyInterpreterState *interp = _PyInterpreterState_Main();
unsigned char byte = (unsigned char)sig_num; unsigned char byte = (unsigned char)sig_num;
#ifdef MS_WINDOWS #ifdef MS_WINDOWS
if (wakeup.use_send) { if (wakeup.use_send) {
@ -1770,8 +1767,8 @@ PyErr_CheckSignals(void)
Python code to ensure signals are handled. Checking for the GC here Python code to ensure signals are handled. Checking for the GC here
allows long running native code to clean cycles created using the C-API allows long running native code to clean cycles created using the C-API
even if it doesn't run the evaluation loop */ even if it doesn't run the evaluation loop */
if (_Py_eval_breaker_bit_is_set(tstate->interp, _PY_GC_SCHEDULED_BIT)) { if (_Py_eval_breaker_bit_is_set(tstate, _PY_GC_SCHEDULED_BIT)) {
_Py_set_eval_breaker_bit(tstate->interp, _PY_GC_SCHEDULED_BIT, 0); _Py_unset_eval_breaker_bit(tstate, _PY_GC_SCHEDULED_BIT);
_Py_RunGC(tstate); _Py_RunGC(tstate);
} }

View file

@ -94,7 +94,7 @@ _Py_brc_queue_object(PyObject *ob)
} }
// Notify owning thread // Notify owning thread
_Py_set_eval_breaker_bit(interp, _PY_EVAL_EXPLICIT_MERGE_BIT, 1); _Py_set_eval_breaker_bit(&tstate->base, _PY_EVAL_EXPLICIT_MERGE_BIT);
PyMutex_Unlock(&bucket->mutex); PyMutex_Unlock(&bucket->mutex);
} }

View file

@ -8,7 +8,6 @@
#include "Python.h" #include "Python.h"
#include "pycore_abstract.h" // _PyIndex_Check() #include "pycore_abstract.h" // _PyIndex_Check()
#include "pycore_ceval.h" // _PyEval_SignalAsyncExc()
#include "pycore_code.h" #include "pycore_code.h"
#include "pycore_emscripten_signal.h" // _Py_CHECK_EMSCRIPTEN_SIGNALS #include "pycore_emscripten_signal.h" // _Py_CHECK_EMSCRIPTEN_SIGNALS
#include "pycore_function.h" #include "pycore_function.h"
@ -146,7 +145,7 @@ dummy_func(
TIER_ONE_ONLY TIER_ONE_ONLY
assert(frame == tstate->current_frame); assert(frame == tstate->current_frame);
uintptr_t global_version = uintptr_t global_version =
_Py_atomic_load_uintptr_relaxed(&tstate->interp->ceval.eval_breaker) & _Py_atomic_load_uintptr_relaxed(&tstate->eval_breaker) &
~_PY_EVAL_EVENTS_MASK; ~_PY_EVAL_EVENTS_MASK;
uintptr_t code_version = _PyFrame_GetCode(frame)->_co_instrumentation_version; uintptr_t code_version = _PyFrame_GetCode(frame)->_co_instrumentation_version;
assert((code_version & 255) == 0); assert((code_version & 255) == 0);
@ -168,14 +167,14 @@ dummy_func(
DEOPT_IF(_Py_emscripten_signal_clock == 0); DEOPT_IF(_Py_emscripten_signal_clock == 0);
_Py_emscripten_signal_clock -= Py_EMSCRIPTEN_SIGNAL_HANDLING; _Py_emscripten_signal_clock -= Py_EMSCRIPTEN_SIGNAL_HANDLING;
#endif #endif
uintptr_t eval_breaker = _Py_atomic_load_uintptr_relaxed(&tstate->interp->ceval.eval_breaker); uintptr_t eval_breaker = _Py_atomic_load_uintptr_relaxed(&tstate->eval_breaker);
uintptr_t version = _PyFrame_GetCode(frame)->_co_instrumentation_version; uintptr_t version = _PyFrame_GetCode(frame)->_co_instrumentation_version;
assert((version & _PY_EVAL_EVENTS_MASK) == 0); assert((version & _PY_EVAL_EVENTS_MASK) == 0);
DEOPT_IF(eval_breaker != version); DEOPT_IF(eval_breaker != version);
} }
inst(INSTRUMENTED_RESUME, (--)) { inst(INSTRUMENTED_RESUME, (--)) {
uintptr_t global_version = _Py_atomic_load_uintptr_relaxed(&tstate->interp->ceval.eval_breaker) & ~_PY_EVAL_EVENTS_MASK; uintptr_t global_version = _Py_atomic_load_uintptr_relaxed(&tstate->eval_breaker) & ~_PY_EVAL_EVENTS_MASK;
uintptr_t code_version = _PyFrame_GetCode(frame)->_co_instrumentation_version; uintptr_t code_version = _PyFrame_GetCode(frame)->_co_instrumentation_version;
if (code_version != global_version) { if (code_version != global_version) {
if (_Py_Instrument(_PyFrame_GetCode(frame), tstate->interp)) { if (_Py_Instrument(_PyFrame_GetCode(frame), tstate->interp)) {

View file

@ -5,7 +5,7 @@
#include "Python.h" #include "Python.h"
#include "pycore_abstract.h" // _PyIndex_Check() #include "pycore_abstract.h" // _PyIndex_Check()
#include "pycore_call.h" // _PyObject_CallNoArgs() #include "pycore_call.h" // _PyObject_CallNoArgs()
#include "pycore_ceval.h" // _PyEval_SignalAsyncExc() #include "pycore_ceval.h"
#include "pycore_code.h" #include "pycore_code.h"
#include "pycore_emscripten_signal.h" // _Py_CHECK_EMSCRIPTEN_SIGNALS #include "pycore_emscripten_signal.h" // _Py_CHECK_EMSCRIPTEN_SIGNALS
#include "pycore_function.h" #include "pycore_function.h"

View file

@ -56,60 +56,52 @@
#define _Py_atomic_load_relaxed_int32(ATOMIC_VAL) _Py_atomic_load_relaxed(ATOMIC_VAL) #define _Py_atomic_load_relaxed_int32(ATOMIC_VAL) _Py_atomic_load_relaxed(ATOMIC_VAL)
#endif #endif
/* bpo-40010: eval_breaker should be recomputed if there // Atomically copy the bits indicated by mask between two values.
is a pending signal: signal received by another thread which cannot
handle signals.
Similarly, we set CALLS_TO_DO and ASYNC_EXCEPTION to match the thread.
*/
static inline void static inline void
update_eval_breaker_from_thread(PyInterpreterState *interp, PyThreadState *tstate) copy_eval_breaker_bits(uintptr_t *from, uintptr_t *to, uintptr_t mask)
{ {
if (tstate == NULL) { uintptr_t from_bits = _Py_atomic_load_uintptr_relaxed(from) & mask;
uintptr_t old_value = _Py_atomic_load_uintptr_relaxed(to);
uintptr_t to_bits = old_value & mask;
if (from_bits == to_bits) {
return; return;
} }
if (_Py_IsMainThread()) { uintptr_t new_value;
int32_t calls_to_do = _Py_atomic_load_int32_relaxed( do {
new_value = (old_value & ~mask) | from_bits;
} while (!_Py_atomic_compare_exchange_uintptr(to, &old_value, new_value));
}
// When attaching a thread, set the global instrumentation version and
// _PY_CALLS_TO_DO_BIT from the current state of the interpreter.
static inline void
update_eval_breaker_for_thread(PyInterpreterState *interp, PyThreadState *tstate)
{
#ifdef Py_GIL_DISABLED
// Free-threaded builds eagerly update the eval_breaker on *all* threads as
// needed, so this function doesn't apply.
return;
#endif
int32_t calls_to_do = _Py_atomic_load_int32_relaxed(
&interp->ceval.pending.calls_to_do);
if (calls_to_do) {
_Py_set_eval_breaker_bit(tstate, _PY_CALLS_TO_DO_BIT);
}
else if (_Py_IsMainThread()) {
calls_to_do = _Py_atomic_load_int32_relaxed(
&_PyRuntime.ceval.pending_mainthread.calls_to_do); &_PyRuntime.ceval.pending_mainthread.calls_to_do);
if (calls_to_do) { if (calls_to_do) {
_Py_set_eval_breaker_bit(interp, _PY_CALLS_TO_DO_BIT, 1); _Py_set_eval_breaker_bit(tstate, _PY_CALLS_TO_DO_BIT);
}
if (_Py_ThreadCanHandleSignals(interp)) {
if (_Py_atomic_load_int(&_PyRuntime.signals.is_tripped)) {
_Py_set_eval_breaker_bit(interp, _PY_SIGNALS_PENDING_BIT, 1);
}
} }
} }
if (tstate->async_exc != NULL) {
_Py_set_eval_breaker_bit(interp, _PY_ASYNC_EXCEPTION_BIT, 1);
}
}
static inline void // _PY_CALLS_TO_DO_BIT was derived from other state above, so the only bits
SET_GIL_DROP_REQUEST(PyInterpreterState *interp) // we copy from our interpreter's state are the instrumentation version.
{ copy_eval_breaker_bits(&interp->ceval.instrumentation_version,
_Py_set_eval_breaker_bit(interp, _PY_GIL_DROP_REQUEST_BIT, 1); &tstate->eval_breaker,
} ~_PY_EVAL_EVENTS_MASK);
static inline void
RESET_GIL_DROP_REQUEST(PyInterpreterState *interp)
{
_Py_set_eval_breaker_bit(interp, _PY_GIL_DROP_REQUEST_BIT, 0);
}
static inline void
SIGNAL_PENDING_CALLS(PyInterpreterState *interp)
{
_Py_set_eval_breaker_bit(interp, _PY_CALLS_TO_DO_BIT, 1);
}
static inline void
UNSIGNAL_PENDING_CALLS(PyInterpreterState *interp)
{
_Py_set_eval_breaker_bit(interp, _PY_CALLS_TO_DO_BIT, 0);
} }
/* /*
@ -254,13 +246,14 @@ drop_gil(PyInterpreterState *interp, PyThreadState *tstate)
the GIL, and that's the only time we might delete the the GIL, and that's the only time we might delete the
interpreter, so checking tstate first prevents the crash. interpreter, so checking tstate first prevents the crash.
See https://github.com/python/cpython/issues/104341. */ See https://github.com/python/cpython/issues/104341. */
if (tstate != NULL && _Py_eval_breaker_bit_is_set(interp, _PY_GIL_DROP_REQUEST_BIT)) { if (tstate != NULL &&
_Py_eval_breaker_bit_is_set(tstate, _PY_GIL_DROP_REQUEST_BIT)) {
MUTEX_LOCK(gil->switch_mutex); MUTEX_LOCK(gil->switch_mutex);
/* Not switched yet => wait */ /* Not switched yet => wait */
if (((PyThreadState*)_Py_atomic_load_ptr_relaxed(&gil->last_holder)) == tstate) if (((PyThreadState*)_Py_atomic_load_ptr_relaxed(&gil->last_holder)) == tstate)
{ {
assert(_PyThreadState_CheckConsistency(tstate)); assert(_PyThreadState_CheckConsistency(tstate));
RESET_GIL_DROP_REQUEST(tstate->interp); _Py_unset_eval_breaker_bit(tstate, _PY_GIL_DROP_REQUEST_BIT);
/* NOTE: if COND_WAIT does not atomically start waiting when /* NOTE: if COND_WAIT does not atomically start waiting when
releasing the mutex, another thread can run through, take releasing the mutex, another thread can run through, take
the GIL and drop it again, and reset the condition the GIL and drop it again, and reset the condition
@ -321,6 +314,8 @@ take_gil(PyThreadState *tstate)
_Py_atomic_load_int_relaxed(&gil->locked) && _Py_atomic_load_int_relaxed(&gil->locked) &&
gil->switch_number == saved_switchnum) gil->switch_number == saved_switchnum)
{ {
PyThreadState *holder_tstate =
(PyThreadState*)_Py_atomic_load_ptr_relaxed(&gil->last_holder);
if (_PyThreadState_MustExit(tstate)) { if (_PyThreadState_MustExit(tstate)) {
MUTEX_UNLOCK(gil->mutex); MUTEX_UNLOCK(gil->mutex);
// gh-96387: If the loop requested a drop request in a previous // gh-96387: If the loop requested a drop request in a previous
@ -330,13 +325,13 @@ take_gil(PyThreadState *tstate)
// may have to request again a drop request (iterate one more // may have to request again a drop request (iterate one more
// time). // time).
if (drop_requested) { if (drop_requested) {
RESET_GIL_DROP_REQUEST(interp); _Py_unset_eval_breaker_bit(holder_tstate, _PY_GIL_DROP_REQUEST_BIT);
} }
PyThread_exit_thread(); PyThread_exit_thread();
} }
assert(_PyThreadState_CheckConsistency(tstate)); assert(_PyThreadState_CheckConsistency(tstate));
SET_GIL_DROP_REQUEST(interp); _Py_set_eval_breaker_bit(holder_tstate, _PY_GIL_DROP_REQUEST_BIT);
drop_requested = 1; drop_requested = 1;
} }
} }
@ -369,13 +364,15 @@ take_gil(PyThreadState *tstate)
in take_gil() while the main thread called in take_gil() while the main thread called
wait_for_thread_shutdown() from Py_Finalize(). */ wait_for_thread_shutdown() from Py_Finalize(). */
MUTEX_UNLOCK(gil->mutex); MUTEX_UNLOCK(gil->mutex);
drop_gil(interp, tstate); /* Passing NULL to drop_gil() indicates that this thread is about to
terminate and will never hold the GIL again. */
drop_gil(interp, NULL);
PyThread_exit_thread(); PyThread_exit_thread();
} }
assert(_PyThreadState_CheckConsistency(tstate)); assert(_PyThreadState_CheckConsistency(tstate));
RESET_GIL_DROP_REQUEST(interp); _Py_unset_eval_breaker_bit(tstate, _PY_GIL_DROP_REQUEST_BIT);
update_eval_breaker_from_thread(interp, tstate); update_eval_breaker_for_thread(interp, tstate);
MUTEX_UNLOCK(gil->mutex); MUTEX_UNLOCK(gil->mutex);
@ -590,15 +587,6 @@ _PyEval_ReInitThreads(PyThreadState *tstate)
} }
#endif #endif
/* This function is used to signal that async exceptions are waiting to be
raised. */
void
_PyEval_SignalAsyncExc(PyInterpreterState *interp)
{
_Py_set_eval_breaker_bit(interp, _PY_ASYNC_EXCEPTION_BIT, 1);
}
PyThreadState * PyThreadState *
PyEval_SaveThread(void) PyEval_SaveThread(void)
{ {
@ -646,11 +634,9 @@ PyEval_RestoreThread(PyThreadState *tstate)
*/ */
void void
_PyEval_SignalReceived(PyInterpreterState *interp) _PyEval_SignalReceived(void)
{ {
if (_Py_ThreadCanHandleSignals(interp)) { _Py_set_eval_breaker_bit(_PyRuntime.main_tstate, _PY_SIGNALS_PENDING_BIT);
_Py_set_eval_breaker_bit(interp, _PY_SIGNALS_PENDING_BIT, 1);
}
} }
/* Push one item onto the queue while holding the lock. */ /* Push one item onto the queue while holding the lock. */
@ -702,6 +688,26 @@ _pop_pending_call(struct _pending_calls *pending,
} }
} }
#ifndef Py_GIL_DISABLED
static void
signal_active_thread(PyInterpreterState *interp, uintptr_t bit)
{
struct _gil_runtime_state *gil = interp->ceval.gil;
// If a thread from the targeted interpreter is holding the GIL, signal
// that thread. Otherwise, the next thread to run from the targeted
// interpreter will have its bit set as part of taking the GIL.
MUTEX_LOCK(gil->mutex);
if (_Py_atomic_load_int_relaxed(&gil->locked)) {
PyThreadState *holder = (PyThreadState*)_Py_atomic_load_ptr_relaxed(&gil->last_holder);
if (holder->interp == interp) {
_Py_set_eval_breaker_bit(holder, bit);
}
}
MUTEX_UNLOCK(gil->mutex);
}
#endif
/* This implementation is thread-safe. It allows /* This implementation is thread-safe. It allows
scheduling to be made from any thread, and even from an executing scheduling to be made from any thread, and even from an executing
callback. callback.
@ -711,10 +717,9 @@ int
_PyEval_AddPendingCall(PyInterpreterState *interp, _PyEval_AddPendingCall(PyInterpreterState *interp,
_Py_pending_call_func func, void *arg, int flags) _Py_pending_call_func func, void *arg, int flags)
{ {
assert(!(flags & _Py_PENDING_MAINTHREADONLY)
|| _Py_IsMainInterpreter(interp));
struct _pending_calls *pending = &interp->ceval.pending; struct _pending_calls *pending = &interp->ceval.pending;
if (flags & _Py_PENDING_MAINTHREADONLY) { int main_only = (flags & _Py_PENDING_MAINTHREADONLY) != 0;
if (main_only) {
/* The main thread only exists in the main interpreter. */ /* The main thread only exists in the main interpreter. */
assert(_Py_IsMainInterpreter(interp)); assert(_Py_IsMainInterpreter(interp));
pending = &_PyRuntime.ceval.pending_mainthread; pending = &_PyRuntime.ceval.pending_mainthread;
@ -724,8 +729,17 @@ _PyEval_AddPendingCall(PyInterpreterState *interp,
int result = _push_pending_call(pending, func, arg, flags); int result = _push_pending_call(pending, func, arg, flags);
PyMutex_Unlock(&pending->mutex); PyMutex_Unlock(&pending->mutex);
/* signal main loop */ if (main_only) {
SIGNAL_PENDING_CALLS(interp); _Py_set_eval_breaker_bit(_PyRuntime.main_tstate, _PY_CALLS_TO_DO_BIT);
}
else {
#ifdef Py_GIL_DISABLED
_Py_set_eval_breaker_bit_all(interp, _PY_CALLS_TO_DO_BIT);
#else
signal_active_thread(interp, _PY_CALLS_TO_DO_BIT);
#endif
}
return result; return result;
} }
@ -742,13 +756,13 @@ static int
handle_signals(PyThreadState *tstate) handle_signals(PyThreadState *tstate)
{ {
assert(_PyThreadState_CheckConsistency(tstate)); assert(_PyThreadState_CheckConsistency(tstate));
_Py_set_eval_breaker_bit(tstate->interp, _PY_SIGNALS_PENDING_BIT, 0); _Py_unset_eval_breaker_bit(tstate, _PY_SIGNALS_PENDING_BIT);
if (!_Py_ThreadCanHandleSignals(tstate->interp)) { if (!_Py_ThreadCanHandleSignals(tstate->interp)) {
return 0; return 0;
} }
if (_PyErr_CheckSignalsTstate(tstate) < 0) { if (_PyErr_CheckSignalsTstate(tstate) < 0) {
/* On failure, re-schedule a call to handle_signals(). */ /* On failure, re-schedule a call to handle_signals(). */
_Py_set_eval_breaker_bit(tstate->interp, _PY_SIGNALS_PENDING_BIT, 1); _Py_set_eval_breaker_bit(tstate, _PY_SIGNALS_PENDING_BIT);
return -1; return -1;
} }
return 0; return 0;
@ -783,9 +797,30 @@ _make_pending_calls(struct _pending_calls *pending)
return 0; return 0;
} }
static int static void
make_pending_calls(PyInterpreterState *interp) signal_pending_calls(PyThreadState *tstate, PyInterpreterState *interp)
{ {
#ifdef Py_GIL_DISABLED
_Py_set_eval_breaker_bit_all(interp, _PY_CALLS_TO_DO_BIT);
#else
_Py_set_eval_breaker_bit(tstate, _PY_CALLS_TO_DO_BIT);
#endif
}
static void
unsignal_pending_calls(PyThreadState *tstate, PyInterpreterState *interp)
{
#ifdef Py_GIL_DISABLED
_Py_unset_eval_breaker_bit_all(interp, _PY_CALLS_TO_DO_BIT);
#else
_Py_unset_eval_breaker_bit(tstate, _PY_CALLS_TO_DO_BIT);
#endif
}
static int
make_pending_calls(PyThreadState *tstate)
{
PyInterpreterState *interp = tstate->interp;
struct _pending_calls *pending = &interp->ceval.pending; struct _pending_calls *pending = &interp->ceval.pending;
struct _pending_calls *pending_main = &_PyRuntime.ceval.pending_mainthread; struct _pending_calls *pending_main = &_PyRuntime.ceval.pending_mainthread;
@ -811,12 +846,12 @@ make_pending_calls(PyInterpreterState *interp)
/* unsignal before starting to call callbacks, so that any callback /* unsignal before starting to call callbacks, so that any callback
added in-between re-signals */ added in-between re-signals */
UNSIGNAL_PENDING_CALLS(interp); unsignal_pending_calls(tstate, interp);
if (_make_pending_calls(pending) != 0) { if (_make_pending_calls(pending) != 0) {
pending->busy = 0; pending->busy = 0;
/* There might not be more calls to make, but we play it safe. */ /* There might not be more calls to make, but we play it safe. */
SIGNAL_PENDING_CALLS(interp); signal_pending_calls(tstate, interp);
return -1; return -1;
} }
@ -824,7 +859,7 @@ make_pending_calls(PyInterpreterState *interp)
if (_make_pending_calls(pending_main) != 0) { if (_make_pending_calls(pending_main) != 0) {
pending->busy = 0; pending->busy = 0;
/* There might not be more calls to make, but we play it safe. */ /* There might not be more calls to make, but we play it safe. */
SIGNAL_PENDING_CALLS(interp); signal_pending_calls(tstate, interp);
return -1; return -1;
} }
} }
@ -833,13 +868,37 @@ make_pending_calls(PyInterpreterState *interp)
return 0; return 0;
} }
void
_Py_set_eval_breaker_bit_all(PyInterpreterState *interp, uintptr_t bit)
{
_PyRuntimeState *runtime = &_PyRuntime;
HEAD_LOCK(runtime);
for (PyThreadState *tstate = interp->threads.head; tstate != NULL; tstate = tstate->next) {
_Py_set_eval_breaker_bit(tstate, bit);
}
HEAD_UNLOCK(runtime);
}
void
_Py_unset_eval_breaker_bit_all(PyInterpreterState *interp, uintptr_t bit)
{
_PyRuntimeState *runtime = &_PyRuntime;
HEAD_LOCK(runtime);
for (PyThreadState *tstate = interp->threads.head; tstate != NULL; tstate = tstate->next) {
_Py_unset_eval_breaker_bit(tstate, bit);
}
HEAD_UNLOCK(runtime);
}
void void
_Py_FinishPendingCalls(PyThreadState *tstate) _Py_FinishPendingCalls(PyThreadState *tstate)
{ {
assert(PyGILState_Check()); assert(PyGILState_Check());
assert(_PyThreadState_CheckConsistency(tstate)); assert(_PyThreadState_CheckConsistency(tstate));
if (make_pending_calls(tstate->interp) < 0) { if (make_pending_calls(tstate) < 0) {
PyObject *exc = _PyErr_GetRaisedException(tstate); PyObject *exc = _PyErr_GetRaisedException(tstate);
PyErr_BadInternalCall(); PyErr_BadInternalCall();
_PyErr_ChainExceptions1(exc); _PyErr_ChainExceptions1(exc);
@ -862,7 +921,7 @@ _PyEval_MakePendingCalls(PyThreadState *tstate)
} }
} }
res = make_pending_calls(tstate->interp); res = make_pending_calls(tstate);
if (res != 0) { if (res != 0) {
return res; return res;
} }
@ -955,11 +1014,11 @@ _PyEval_InitState(PyInterpreterState *interp)
int int
_Py_HandlePending(PyThreadState *tstate) _Py_HandlePending(PyThreadState *tstate)
{ {
PyInterpreterState *interp = tstate->interp; uintptr_t breaker = _Py_atomic_load_uintptr_relaxed(&tstate->eval_breaker);
/* Stop-the-world */ /* Stop-the-world */
if (_Py_eval_breaker_bit_is_set(interp, _PY_EVAL_PLEASE_STOP_BIT)) { if ((breaker & _PY_EVAL_PLEASE_STOP_BIT) != 0) {
_Py_set_eval_breaker_bit(interp, _PY_EVAL_PLEASE_STOP_BIT, 0); _Py_unset_eval_breaker_bit(tstate, _PY_EVAL_PLEASE_STOP_BIT);
_PyThreadState_Suspend(tstate); _PyThreadState_Suspend(tstate);
/* The attach blocks until the stop-the-world event is complete. */ /* The attach blocks until the stop-the-world event is complete. */
@ -967,35 +1026,35 @@ _Py_HandlePending(PyThreadState *tstate)
} }
/* Pending signals */ /* Pending signals */
if (_Py_eval_breaker_bit_is_set(interp, _PY_SIGNALS_PENDING_BIT)) { if ((breaker & _PY_SIGNALS_PENDING_BIT) != 0) {
if (handle_signals(tstate) != 0) { if (handle_signals(tstate) != 0) {
return -1; return -1;
} }
} }
/* Pending calls */ /* Pending calls */
if (_Py_eval_breaker_bit_is_set(interp, _PY_CALLS_TO_DO_BIT)) { if ((breaker & _PY_CALLS_TO_DO_BIT) != 0) {
if (make_pending_calls(interp) != 0) { if (make_pending_calls(tstate) != 0) {
return -1; return -1;
} }
} }
#ifdef Py_GIL_DISABLED #ifdef Py_GIL_DISABLED
/* Objects with refcounts to merge */ /* Objects with refcounts to merge */
if (_Py_eval_breaker_bit_is_set(interp, _PY_EVAL_EXPLICIT_MERGE_BIT)) { if ((breaker & _PY_EVAL_EXPLICIT_MERGE_BIT) != 0) {
_Py_set_eval_breaker_bit(interp, _PY_EVAL_EXPLICIT_MERGE_BIT, 0); _Py_unset_eval_breaker_bit(tstate, _PY_EVAL_EXPLICIT_MERGE_BIT);
_Py_brc_merge_refcounts(tstate); _Py_brc_merge_refcounts(tstate);
} }
#endif #endif
/* GC scheduled to run */ /* GC scheduled to run */
if (_Py_eval_breaker_bit_is_set(interp, _PY_GC_SCHEDULED_BIT)) { if ((breaker & _PY_GC_SCHEDULED_BIT) != 0) {
_Py_set_eval_breaker_bit(interp, _PY_GC_SCHEDULED_BIT, 0); _Py_unset_eval_breaker_bit(tstate, _PY_GC_SCHEDULED_BIT);
_Py_RunGC(tstate); _Py_RunGC(tstate);
} }
/* GIL drop request */ /* GIL drop request */
if (_Py_eval_breaker_bit_is_set(interp, _PY_GIL_DROP_REQUEST_BIT)) { if ((breaker & _PY_GIL_DROP_REQUEST_BIT) != 0) {
/* Give another thread a chance */ /* Give another thread a chance */
_PyThreadState_Detach(tstate); _PyThreadState_Detach(tstate);
@ -1005,11 +1064,10 @@ _Py_HandlePending(PyThreadState *tstate)
} }
/* Check for asynchronous exception. */ /* Check for asynchronous exception. */
if (_Py_eval_breaker_bit_is_set(interp, _PY_ASYNC_EXCEPTION_BIT)) { if ((breaker & _PY_ASYNC_EXCEPTION_BIT) != 0) {
_Py_set_eval_breaker_bit(interp, _PY_ASYNC_EXCEPTION_BIT, 0); _Py_unset_eval_breaker_bit(tstate, _PY_ASYNC_EXCEPTION_BIT);
if (tstate->async_exc != NULL) { PyObject *exc = _Py_atomic_exchange_ptr(&tstate->async_exc, NULL);
PyObject *exc = tstate->async_exc; if (exc != NULL) {
tstate->async_exc = NULL;
_PyErr_SetNone(tstate, exc); _PyErr_SetNone(tstate, exc);
Py_DECREF(exc); Py_DECREF(exc);
return -1; return -1;
@ -1017,4 +1075,3 @@ _Py_HandlePending(PyThreadState *tstate)
} }
return 0; return 0;
} }

View file

@ -124,7 +124,7 @@
#define CHECK_EVAL_BREAKER() \ #define CHECK_EVAL_BREAKER() \
_Py_CHECK_EMSCRIPTEN_SIGNALS_PERIODICALLY(); \ _Py_CHECK_EMSCRIPTEN_SIGNALS_PERIODICALLY(); \
QSBR_QUIESCENT_STATE(tstate); \ QSBR_QUIESCENT_STATE(tstate); \
if (_Py_atomic_load_uintptr_relaxed(&tstate->interp->ceval.eval_breaker) & _PY_EVAL_EVENTS_MASK) { \ if (_Py_atomic_load_uintptr_relaxed(&tstate->eval_breaker) & _PY_EVAL_EVENTS_MASK) { \
if (_Py_HandlePending(tstate) != 0) { \ if (_Py_HandlePending(tstate) != 0) { \
GOTO_ERROR(error); \ GOTO_ERROR(error); \
} \ } \

View file

@ -17,7 +17,7 @@
if (_Py_emscripten_signal_clock == 0) goto deoptimize; if (_Py_emscripten_signal_clock == 0) goto deoptimize;
_Py_emscripten_signal_clock -= Py_EMSCRIPTEN_SIGNAL_HANDLING; _Py_emscripten_signal_clock -= Py_EMSCRIPTEN_SIGNAL_HANDLING;
#endif #endif
uintptr_t eval_breaker = _Py_atomic_load_uintptr_relaxed(&tstate->interp->ceval.eval_breaker); uintptr_t eval_breaker = _Py_atomic_load_uintptr_relaxed(&tstate->eval_breaker);
uintptr_t version = _PyFrame_GetCode(frame)->_co_instrumentation_version; uintptr_t version = _PyFrame_GetCode(frame)->_co_instrumentation_version;
assert((version & _PY_EVAL_EVENTS_MASK) == 0); assert((version & _PY_EVAL_EVENTS_MASK) == 0);
if (eval_breaker != version) goto deoptimize; if (eval_breaker != version) goto deoptimize;

View file

@ -1771,9 +1771,12 @@ PyObject_IS_GC(PyObject *obj)
} }
void void
_Py_ScheduleGC(PyInterpreterState *interp) _Py_ScheduleGC(PyThreadState *tstate)
{ {
_Py_set_eval_breaker_bit(interp, _PY_GC_SCHEDULED_BIT, 1); if (!_Py_eval_breaker_bit_is_set(tstate, _PY_GC_SCHEDULED_BIT))
{
_Py_set_eval_breaker_bit(tstate, _PY_GC_SCHEDULED_BIT);
}
} }
void void
@ -1794,7 +1797,7 @@ _PyObject_GC_Link(PyObject *op)
!_Py_atomic_load_int_relaxed(&gcstate->collecting) && !_Py_atomic_load_int_relaxed(&gcstate->collecting) &&
!_PyErr_Occurred(tstate)) !_PyErr_Occurred(tstate))
{ {
_Py_ScheduleGC(tstate->interp); _Py_ScheduleGC(tstate);
} }
} }

View file

@ -981,7 +981,7 @@ record_allocation(PyThreadState *tstate)
if (gc_should_collect(gcstate) && if (gc_should_collect(gcstate) &&
!_Py_atomic_load_int_relaxed(&gcstate->collecting)) !_Py_atomic_load_int_relaxed(&gcstate->collecting))
{ {
_Py_ScheduleGC(tstate->interp); _Py_ScheduleGC(tstate);
} }
} }
} }
@ -1564,9 +1564,12 @@ PyObject_IS_GC(PyObject *obj)
} }
void void
_Py_ScheduleGC(PyInterpreterState *interp) _Py_ScheduleGC(PyThreadState *tstate)
{ {
_Py_set_eval_breaker_bit(interp, _PY_GC_SCHEDULED_BIT, 1); if (!_Py_eval_breaker_bit_is_set(tstate, _PY_GC_SCHEDULED_BIT))
{
_Py_set_eval_breaker_bit(tstate, _PY_GC_SCHEDULED_BIT);
}
} }
void void

View file

@ -3139,7 +3139,7 @@
_Py_CODEUNIT *this_instr = frame->instr_ptr = next_instr; _Py_CODEUNIT *this_instr = frame->instr_ptr = next_instr;
next_instr += 1; next_instr += 1;
INSTRUCTION_STATS(INSTRUMENTED_RESUME); INSTRUCTION_STATS(INSTRUMENTED_RESUME);
uintptr_t global_version = _Py_atomic_load_uintptr_relaxed(&tstate->interp->ceval.eval_breaker) & ~_PY_EVAL_EVENTS_MASK; uintptr_t global_version = _Py_atomic_load_uintptr_relaxed(&tstate->eval_breaker) & ~_PY_EVAL_EVENTS_MASK;
uintptr_t code_version = _PyFrame_GetCode(frame)->_co_instrumentation_version; uintptr_t code_version = _PyFrame_GetCode(frame)->_co_instrumentation_version;
if (code_version != global_version) { if (code_version != global_version) {
if (_Py_Instrument(_PyFrame_GetCode(frame), tstate->interp)) { if (_Py_Instrument(_PyFrame_GetCode(frame), tstate->interp)) {
@ -4809,7 +4809,7 @@
TIER_ONE_ONLY TIER_ONE_ONLY
assert(frame == tstate->current_frame); assert(frame == tstate->current_frame);
uintptr_t global_version = uintptr_t global_version =
_Py_atomic_load_uintptr_relaxed(&tstate->interp->ceval.eval_breaker) & _Py_atomic_load_uintptr_relaxed(&tstate->eval_breaker) &
~_PY_EVAL_EVENTS_MASK; ~_PY_EVAL_EVENTS_MASK;
uintptr_t code_version = _PyFrame_GetCode(frame)->_co_instrumentation_version; uintptr_t code_version = _PyFrame_GetCode(frame)->_co_instrumentation_version;
assert((code_version & 255) == 0); assert((code_version & 255) == 0);
@ -4836,7 +4836,7 @@
DEOPT_IF(_Py_emscripten_signal_clock == 0, RESUME); DEOPT_IF(_Py_emscripten_signal_clock == 0, RESUME);
_Py_emscripten_signal_clock -= Py_EMSCRIPTEN_SIGNAL_HANDLING; _Py_emscripten_signal_clock -= Py_EMSCRIPTEN_SIGNAL_HANDLING;
#endif #endif
uintptr_t eval_breaker = _Py_atomic_load_uintptr_relaxed(&tstate->interp->ceval.eval_breaker); uintptr_t eval_breaker = _Py_atomic_load_uintptr_relaxed(&tstate->eval_breaker);
uintptr_t version = _PyFrame_GetCode(frame)->_co_instrumentation_version; uintptr_t version = _PyFrame_GetCode(frame)->_co_instrumentation_version;
assert((version & _PY_EVAL_EVENTS_MASK) == 0); assert((version & _PY_EVAL_EVENTS_MASK) == 0);
DEOPT_IF(eval_breaker != version, RESUME); DEOPT_IF(eval_breaker != version, RESUME);

View file

@ -891,18 +891,43 @@ static inline int most_significant_bit(uint8_t bits) {
static uint32_t static uint32_t
global_version(PyInterpreterState *interp) global_version(PyInterpreterState *interp)
{ {
return interp->ceval.eval_breaker & ~_PY_EVAL_EVENTS_MASK; return (uint32_t)_Py_atomic_load_uintptr_relaxed(
&interp->ceval.instrumentation_version);
}
/* Atomically set the given version in the given location, without touching
anything in _PY_EVAL_EVENTS_MASK. */
static void
set_version_raw(uintptr_t *ptr, uint32_t version)
{
uintptr_t old = _Py_atomic_load_uintptr_relaxed(ptr);
uintptr_t new;
do {
new = (old & _PY_EVAL_EVENTS_MASK) | version;
} while (!_Py_atomic_compare_exchange_uintptr(ptr, &old, new));
} }
static void static void
set_global_version(PyInterpreterState *interp, uint32_t version) set_global_version(PyThreadState *tstate, uint32_t version)
{ {
assert((version & _PY_EVAL_EVENTS_MASK) == 0); assert((version & _PY_EVAL_EVENTS_MASK) == 0);
uintptr_t old = _Py_atomic_load_uintptr(&interp->ceval.eval_breaker); PyInterpreterState *interp = tstate->interp;
intptr_t new; set_version_raw(&interp->ceval.instrumentation_version, version);
do {
new = (old & _PY_EVAL_EVENTS_MASK) | version; #ifdef Py_GIL_DISABLED
} while (!_Py_atomic_compare_exchange_uintptr(&interp->ceval.eval_breaker, &old, new)); // Set the version on all threads in free-threaded builds.
_PyRuntimeState *runtime = &_PyRuntime;
HEAD_LOCK(runtime);
for (tstate = interp->threads.head; tstate;
tstate = PyThreadState_Next(tstate)) {
set_version_raw(&tstate->eval_breaker, version);
};
HEAD_UNLOCK(runtime);
#else
// Normal builds take the current version from instrumentation_version when
// attaching a thread, so we only have to set the current thread's version.
set_version_raw(&tstate->eval_breaker, version);
#endif
} }
static bool static bool
@ -1566,7 +1591,7 @@ _Py_Instrument(PyCodeObject *code, PyInterpreterState *interp)
{ {
if (is_version_up_to_date(code, interp)) { if (is_version_up_to_date(code, interp)) {
assert( assert(
(interp->ceval.eval_breaker & ~_PY_EVAL_EVENTS_MASK) == 0 || interp->ceval.instrumentation_version == 0 ||
instrumentation_cross_checks(interp, code) instrumentation_cross_checks(interp, code)
); );
return 0; return 0;
@ -1778,7 +1803,8 @@ int
_PyMonitoring_SetEvents(int tool_id, _PyMonitoringEventSet events) _PyMonitoring_SetEvents(int tool_id, _PyMonitoringEventSet events)
{ {
assert(0 <= tool_id && tool_id < PY_MONITORING_TOOL_IDS); assert(0 <= tool_id && tool_id < PY_MONITORING_TOOL_IDS);
PyInterpreterState *interp = _PyInterpreterState_GET(); PyThreadState *tstate = _PyThreadState_GET();
PyInterpreterState *interp = tstate->interp;
assert(events < (1 << _PY_MONITORING_UNGROUPED_EVENTS)); assert(events < (1 << _PY_MONITORING_UNGROUPED_EVENTS));
if (check_tool(interp, tool_id)) { if (check_tool(interp, tool_id)) {
return -1; return -1;
@ -1793,7 +1819,7 @@ _PyMonitoring_SetEvents(int tool_id, _PyMonitoringEventSet events)
PyErr_Format(PyExc_OverflowError, "events set too many times"); PyErr_Format(PyExc_OverflowError, "events set too many times");
return -1; return -1;
} }
set_global_version(interp, new_version); set_global_version(tstate, new_version);
_Py_Executors_InvalidateAll(interp); _Py_Executors_InvalidateAll(interp);
return instrument_all_executing_code_objects(interp); return instrument_all_executing_code_objects(interp);
} }
@ -2122,7 +2148,8 @@ monitoring_restart_events_impl(PyObject *module)
* last restart version > instrumented version for all code objects * last restart version > instrumented version for all code objects
* last restart version < current version * last restart version < current version
*/ */
PyInterpreterState *interp = _PyInterpreterState_GET(); PyThreadState *tstate = _PyThreadState_GET();
PyInterpreterState *interp = tstate->interp;
uint32_t restart_version = global_version(interp) + MONITORING_VERSION_INCREMENT; uint32_t restart_version = global_version(interp) + MONITORING_VERSION_INCREMENT;
uint32_t new_version = restart_version + MONITORING_VERSION_INCREMENT; uint32_t new_version = restart_version + MONITORING_VERSION_INCREMENT;
if (new_version <= MONITORING_VERSION_INCREMENT) { if (new_version <= MONITORING_VERSION_INCREMENT) {
@ -2130,7 +2157,7 @@ monitoring_restart_events_impl(PyObject *module)
return NULL; return NULL;
} }
interp->last_restart_version = restart_version; interp->last_restart_version = restart_version;
set_global_version(interp, new_version); set_global_version(tstate, new_version);
if (instrument_all_executing_code_objects(interp)) { if (instrument_all_executing_code_objects(interp)) {
return NULL; return NULL;
} }

View file

@ -663,6 +663,7 @@ pycore_create_interpreter(_PyRuntimeState *runtime,
if (tstate == NULL) { if (tstate == NULL) {
return _PyStatus_ERR("can't make first thread"); return _PyStatus_ERR("can't make first thread");
} }
runtime->main_tstate = tstate;
_PyThreadState_Bind(tstate); _PyThreadState_Bind(tstate);
init_interp_create_gil(tstate, config.gil); init_interp_create_gil(tstate, config.gil);

View file

@ -794,9 +794,7 @@ interpreter_clear(PyInterpreterState *interp, PyThreadState *tstate)
Py_CLEAR(interp->audit_hooks); Py_CLEAR(interp->audit_hooks);
// At this time, all the threads should be cleared so we don't need interp->ceval.instrumentation_version = 0;
// atomic operations for eval_breaker
interp->ceval.eval_breaker = 0;
for (int i = 0; i < _PY_MONITORING_UNGROUPED_EVENTS; i++) { for (int i = 0; i < _PY_MONITORING_UNGROUPED_EVENTS; i++) {
interp->monitors.tools[i] = 0; interp->monitors.tools[i] = 0;
@ -1318,6 +1316,8 @@ init_threadstate(_PyThreadStateImpl *_tstate,
assert(interp != NULL); assert(interp != NULL);
tstate->interp = interp; tstate->interp = interp;
tstate->eval_breaker =
_Py_atomic_load_uintptr_relaxed(&interp->ceval.instrumentation_version);
// next/prev are set in add_threadstate(). // next/prev are set in add_threadstate().
assert(tstate->next == NULL); assert(tstate->next == NULL);
@ -2021,8 +2021,7 @@ park_detached_threads(struct _stoptheworld_state *stw)
} }
} }
else if (state == _Py_THREAD_ATTACHED && t != stw->requester) { else if (state == _Py_THREAD_ATTACHED && t != stw->requester) {
// TODO: set this per-thread, rather than per-interpreter. _Py_set_eval_breaker_bit(t, _PY_EVAL_PLEASE_STOP_BIT);
_Py_set_eval_breaker_bit(t->interp, _PY_EVAL_PLEASE_STOP_BIT, 1);
} }
} }
stw->thread_countdown -= num_parked; stw->thread_countdown -= num_parked;
@ -2186,19 +2185,18 @@ PyThreadState_SetAsyncExc(unsigned long id, PyObject *exc)
* deadlock, we need to release head_mutex before * deadlock, we need to release head_mutex before
* the decref. * the decref.
*/ */
PyObject *old_exc = tstate->async_exc; Py_XINCREF(exc);
tstate->async_exc = Py_XNewRef(exc); PyObject *old_exc = _Py_atomic_exchange_ptr(&tstate->async_exc, exc);
HEAD_UNLOCK(runtime); HEAD_UNLOCK(runtime);
Py_XDECREF(old_exc); Py_XDECREF(old_exc);
_PyEval_SignalAsyncExc(tstate->interp); _Py_set_eval_breaker_bit(tstate, _PY_ASYNC_EXCEPTION_BIT);
return 1; return 1;
} }
HEAD_UNLOCK(runtime); HEAD_UNLOCK(runtime);
return 0; return 0;
} }
//--------------------------------- //---------------------------------
// API for the current thread state // API for the current thread state
//--------------------------------- //---------------------------------