mirror of
https://github.com/python/cpython.git
synced 2025-07-19 09:15:34 +00:00
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:
parent
e71468ba4f
commit
0749244d13
19 changed files with 265 additions and 172 deletions
|
@ -94,7 +94,7 @@ _Py_brc_queue_object(PyObject *ob)
|
|||
}
|
||||
|
||||
// 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);
|
||||
}
|
||||
|
|
|
@ -8,7 +8,6 @@
|
|||
|
||||
#include "Python.h"
|
||||
#include "pycore_abstract.h" // _PyIndex_Check()
|
||||
#include "pycore_ceval.h" // _PyEval_SignalAsyncExc()
|
||||
#include "pycore_code.h"
|
||||
#include "pycore_emscripten_signal.h" // _Py_CHECK_EMSCRIPTEN_SIGNALS
|
||||
#include "pycore_function.h"
|
||||
|
@ -146,7 +145,7 @@ dummy_func(
|
|||
TIER_ONE_ONLY
|
||||
assert(frame == tstate->current_frame);
|
||||
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;
|
||||
uintptr_t code_version = _PyFrame_GetCode(frame)->_co_instrumentation_version;
|
||||
assert((code_version & 255) == 0);
|
||||
|
@ -168,14 +167,14 @@ dummy_func(
|
|||
DEOPT_IF(_Py_emscripten_signal_clock == 0);
|
||||
_Py_emscripten_signal_clock -= Py_EMSCRIPTEN_SIGNAL_HANDLING;
|
||||
#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;
|
||||
assert((version & _PY_EVAL_EVENTS_MASK) == 0);
|
||||
DEOPT_IF(eval_breaker != version);
|
||||
}
|
||||
|
||||
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;
|
||||
if (code_version != global_version) {
|
||||
if (_Py_Instrument(_PyFrame_GetCode(frame), tstate->interp)) {
|
||||
|
|
|
@ -5,7 +5,7 @@
|
|||
#include "Python.h"
|
||||
#include "pycore_abstract.h" // _PyIndex_Check()
|
||||
#include "pycore_call.h" // _PyObject_CallNoArgs()
|
||||
#include "pycore_ceval.h" // _PyEval_SignalAsyncExc()
|
||||
#include "pycore_ceval.h"
|
||||
#include "pycore_code.h"
|
||||
#include "pycore_emscripten_signal.h" // _Py_CHECK_EMSCRIPTEN_SIGNALS
|
||||
#include "pycore_function.h"
|
||||
|
|
|
@ -56,60 +56,52 @@
|
|||
#define _Py_atomic_load_relaxed_int32(ATOMIC_VAL) _Py_atomic_load_relaxed(ATOMIC_VAL)
|
||||
#endif
|
||||
|
||||
/* bpo-40010: eval_breaker should be recomputed if there
|
||||
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.
|
||||
*/
|
||||
// Atomically copy the bits indicated by mask between two values.
|
||||
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;
|
||||
}
|
||||
|
||||
if (_Py_IsMainThread()) {
|
||||
int32_t calls_to_do = _Py_atomic_load_int32_relaxed(
|
||||
uintptr_t new_value;
|
||||
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);
|
||||
if (calls_to_do) {
|
||||
_Py_set_eval_breaker_bit(interp, _PY_CALLS_TO_DO_BIT, 1);
|
||||
}
|
||||
if (_Py_ThreadCanHandleSignals(interp)) {
|
||||
if (_Py_atomic_load_int(&_PyRuntime.signals.is_tripped)) {
|
||||
_Py_set_eval_breaker_bit(interp, _PY_SIGNALS_PENDING_BIT, 1);
|
||||
}
|
||||
_Py_set_eval_breaker_bit(tstate, _PY_CALLS_TO_DO_BIT);
|
||||
}
|
||||
}
|
||||
if (tstate->async_exc != NULL) {
|
||||
_Py_set_eval_breaker_bit(interp, _PY_ASYNC_EXCEPTION_BIT, 1);
|
||||
}
|
||||
}
|
||||
|
||||
static inline void
|
||||
SET_GIL_DROP_REQUEST(PyInterpreterState *interp)
|
||||
{
|
||||
_Py_set_eval_breaker_bit(interp, _PY_GIL_DROP_REQUEST_BIT, 1);
|
||||
}
|
||||
|
||||
|
||||
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);
|
||||
// _PY_CALLS_TO_DO_BIT was derived from other state above, so the only bits
|
||||
// we copy from our interpreter's state are the instrumentation version.
|
||||
copy_eval_breaker_bits(&interp->ceval.instrumentation_version,
|
||||
&tstate->eval_breaker,
|
||||
~_PY_EVAL_EVENTS_MASK);
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -254,13 +246,14 @@ drop_gil(PyInterpreterState *interp, PyThreadState *tstate)
|
|||
the GIL, and that's the only time we might delete the
|
||||
interpreter, so checking tstate first prevents the crash.
|
||||
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);
|
||||
/* Not switched yet => wait */
|
||||
if (((PyThreadState*)_Py_atomic_load_ptr_relaxed(&gil->last_holder)) == 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
|
||||
releasing the mutex, another thread can run through, take
|
||||
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) &&
|
||||
gil->switch_number == saved_switchnum)
|
||||
{
|
||||
PyThreadState *holder_tstate =
|
||||
(PyThreadState*)_Py_atomic_load_ptr_relaxed(&gil->last_holder);
|
||||
if (_PyThreadState_MustExit(tstate)) {
|
||||
MUTEX_UNLOCK(gil->mutex);
|
||||
// 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
|
||||
// time).
|
||||
if (drop_requested) {
|
||||
RESET_GIL_DROP_REQUEST(interp);
|
||||
_Py_unset_eval_breaker_bit(holder_tstate, _PY_GIL_DROP_REQUEST_BIT);
|
||||
}
|
||||
PyThread_exit_thread();
|
||||
}
|
||||
assert(_PyThreadState_CheckConsistency(tstate));
|
||||
|
||||
SET_GIL_DROP_REQUEST(interp);
|
||||
_Py_set_eval_breaker_bit(holder_tstate, _PY_GIL_DROP_REQUEST_BIT);
|
||||
drop_requested = 1;
|
||||
}
|
||||
}
|
||||
|
@ -369,13 +364,15 @@ take_gil(PyThreadState *tstate)
|
|||
in take_gil() while the main thread called
|
||||
wait_for_thread_shutdown() from Py_Finalize(). */
|
||||
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();
|
||||
}
|
||||
assert(_PyThreadState_CheckConsistency(tstate));
|
||||
|
||||
RESET_GIL_DROP_REQUEST(interp);
|
||||
update_eval_breaker_from_thread(interp, tstate);
|
||||
_Py_unset_eval_breaker_bit(tstate, _PY_GIL_DROP_REQUEST_BIT);
|
||||
update_eval_breaker_for_thread(interp, tstate);
|
||||
|
||||
MUTEX_UNLOCK(gil->mutex);
|
||||
|
||||
|
@ -590,15 +587,6 @@ _PyEval_ReInitThreads(PyThreadState *tstate)
|
|||
}
|
||||
#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 *
|
||||
PyEval_SaveThread(void)
|
||||
{
|
||||
|
@ -646,11 +634,9 @@ PyEval_RestoreThread(PyThreadState *tstate)
|
|||
*/
|
||||
|
||||
void
|
||||
_PyEval_SignalReceived(PyInterpreterState *interp)
|
||||
_PyEval_SignalReceived(void)
|
||||
{
|
||||
if (_Py_ThreadCanHandleSignals(interp)) {
|
||||
_Py_set_eval_breaker_bit(interp, _PY_SIGNALS_PENDING_BIT, 1);
|
||||
}
|
||||
_Py_set_eval_breaker_bit(_PyRuntime.main_tstate, _PY_SIGNALS_PENDING_BIT);
|
||||
}
|
||||
|
||||
/* 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
|
||||
scheduling to be made from any thread, and even from an executing
|
||||
callback.
|
||||
|
@ -711,10 +717,9 @@ int
|
|||
_PyEval_AddPendingCall(PyInterpreterState *interp,
|
||||
_Py_pending_call_func func, void *arg, int flags)
|
||||
{
|
||||
assert(!(flags & _Py_PENDING_MAINTHREADONLY)
|
||||
|| _Py_IsMainInterpreter(interp));
|
||||
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. */
|
||||
assert(_Py_IsMainInterpreter(interp));
|
||||
pending = &_PyRuntime.ceval.pending_mainthread;
|
||||
|
@ -724,8 +729,17 @@ _PyEval_AddPendingCall(PyInterpreterState *interp,
|
|||
int result = _push_pending_call(pending, func, arg, flags);
|
||||
PyMutex_Unlock(&pending->mutex);
|
||||
|
||||
/* signal main loop */
|
||||
SIGNAL_PENDING_CALLS(interp);
|
||||
if (main_only) {
|
||||
_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;
|
||||
}
|
||||
|
||||
|
@ -742,13 +756,13 @@ static int
|
|||
handle_signals(PyThreadState *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)) {
|
||||
return 0;
|
||||
}
|
||||
if (_PyErr_CheckSignalsTstate(tstate) < 0) {
|
||||
/* 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 0;
|
||||
|
@ -783,9 +797,30 @@ _make_pending_calls(struct _pending_calls *pending)
|
|||
return 0;
|
||||
}
|
||||
|
||||
static int
|
||||
make_pending_calls(PyInterpreterState *interp)
|
||||
static void
|
||||
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_main = &_PyRuntime.ceval.pending_mainthread;
|
||||
|
||||
|
@ -811,12 +846,12 @@ make_pending_calls(PyInterpreterState *interp)
|
|||
|
||||
/* unsignal before starting to call callbacks, so that any callback
|
||||
added in-between re-signals */
|
||||
UNSIGNAL_PENDING_CALLS(interp);
|
||||
unsignal_pending_calls(tstate, interp);
|
||||
|
||||
if (_make_pending_calls(pending) != 0) {
|
||||
pending->busy = 0;
|
||||
/* There might not be more calls to make, but we play it safe. */
|
||||
SIGNAL_PENDING_CALLS(interp);
|
||||
signal_pending_calls(tstate, interp);
|
||||
return -1;
|
||||
}
|
||||
|
||||
|
@ -824,7 +859,7 @@ make_pending_calls(PyInterpreterState *interp)
|
|||
if (_make_pending_calls(pending_main) != 0) {
|
||||
pending->busy = 0;
|
||||
/* There might not be more calls to make, but we play it safe. */
|
||||
SIGNAL_PENDING_CALLS(interp);
|
||||
signal_pending_calls(tstate, interp);
|
||||
return -1;
|
||||
}
|
||||
}
|
||||
|
@ -833,13 +868,37 @@ make_pending_calls(PyInterpreterState *interp)
|
|||
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
|
||||
_Py_FinishPendingCalls(PyThreadState *tstate)
|
||||
{
|
||||
assert(PyGILState_Check());
|
||||
assert(_PyThreadState_CheckConsistency(tstate));
|
||||
|
||||
if (make_pending_calls(tstate->interp) < 0) {
|
||||
if (make_pending_calls(tstate) < 0) {
|
||||
PyObject *exc = _PyErr_GetRaisedException(tstate);
|
||||
PyErr_BadInternalCall();
|
||||
_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) {
|
||||
return res;
|
||||
}
|
||||
|
@ -955,11 +1014,11 @@ _PyEval_InitState(PyInterpreterState *interp)
|
|||
int
|
||||
_Py_HandlePending(PyThreadState *tstate)
|
||||
{
|
||||
PyInterpreterState *interp = tstate->interp;
|
||||
uintptr_t breaker = _Py_atomic_load_uintptr_relaxed(&tstate->eval_breaker);
|
||||
|
||||
/* Stop-the-world */
|
||||
if (_Py_eval_breaker_bit_is_set(interp, _PY_EVAL_PLEASE_STOP_BIT)) {
|
||||
_Py_set_eval_breaker_bit(interp, _PY_EVAL_PLEASE_STOP_BIT, 0);
|
||||
if ((breaker & _PY_EVAL_PLEASE_STOP_BIT) != 0) {
|
||||
_Py_unset_eval_breaker_bit(tstate, _PY_EVAL_PLEASE_STOP_BIT);
|
||||
_PyThreadState_Suspend(tstate);
|
||||
|
||||
/* The attach blocks until the stop-the-world event is complete. */
|
||||
|
@ -967,35 +1026,35 @@ _Py_HandlePending(PyThreadState *tstate)
|
|||
}
|
||||
|
||||
/* 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) {
|
||||
return -1;
|
||||
}
|
||||
}
|
||||
|
||||
/* Pending calls */
|
||||
if (_Py_eval_breaker_bit_is_set(interp, _PY_CALLS_TO_DO_BIT)) {
|
||||
if (make_pending_calls(interp) != 0) {
|
||||
if ((breaker & _PY_CALLS_TO_DO_BIT) != 0) {
|
||||
if (make_pending_calls(tstate) != 0) {
|
||||
return -1;
|
||||
}
|
||||
}
|
||||
|
||||
#ifdef Py_GIL_DISABLED
|
||||
/* Objects with refcounts to merge */
|
||||
if (_Py_eval_breaker_bit_is_set(interp, _PY_EVAL_EXPLICIT_MERGE_BIT)) {
|
||||
_Py_set_eval_breaker_bit(interp, _PY_EVAL_EXPLICIT_MERGE_BIT, 0);
|
||||
if ((breaker & _PY_EVAL_EXPLICIT_MERGE_BIT) != 0) {
|
||||
_Py_unset_eval_breaker_bit(tstate, _PY_EVAL_EXPLICIT_MERGE_BIT);
|
||||
_Py_brc_merge_refcounts(tstate);
|
||||
}
|
||||
#endif
|
||||
|
||||
/* GC scheduled to run */
|
||||
if (_Py_eval_breaker_bit_is_set(interp, _PY_GC_SCHEDULED_BIT)) {
|
||||
_Py_set_eval_breaker_bit(interp, _PY_GC_SCHEDULED_BIT, 0);
|
||||
if ((breaker & _PY_GC_SCHEDULED_BIT) != 0) {
|
||||
_Py_unset_eval_breaker_bit(tstate, _PY_GC_SCHEDULED_BIT);
|
||||
_Py_RunGC(tstate);
|
||||
}
|
||||
|
||||
/* 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 */
|
||||
_PyThreadState_Detach(tstate);
|
||||
|
||||
|
@ -1005,11 +1064,10 @@ _Py_HandlePending(PyThreadState *tstate)
|
|||
}
|
||||
|
||||
/* Check for asynchronous exception. */
|
||||
if (_Py_eval_breaker_bit_is_set(interp, _PY_ASYNC_EXCEPTION_BIT)) {
|
||||
_Py_set_eval_breaker_bit(interp, _PY_ASYNC_EXCEPTION_BIT, 0);
|
||||
if (tstate->async_exc != NULL) {
|
||||
PyObject *exc = tstate->async_exc;
|
||||
tstate->async_exc = NULL;
|
||||
if ((breaker & _PY_ASYNC_EXCEPTION_BIT) != 0) {
|
||||
_Py_unset_eval_breaker_bit(tstate, _PY_ASYNC_EXCEPTION_BIT);
|
||||
PyObject *exc = _Py_atomic_exchange_ptr(&tstate->async_exc, NULL);
|
||||
if (exc != NULL) {
|
||||
_PyErr_SetNone(tstate, exc);
|
||||
Py_DECREF(exc);
|
||||
return -1;
|
||||
|
@ -1017,4 +1075,3 @@ _Py_HandlePending(PyThreadState *tstate)
|
|||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
|
|
@ -124,7 +124,7 @@
|
|||
#define CHECK_EVAL_BREAKER() \
|
||||
_Py_CHECK_EMSCRIPTEN_SIGNALS_PERIODICALLY(); \
|
||||
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) { \
|
||||
GOTO_ERROR(error); \
|
||||
} \
|
||||
|
|
2
Python/executor_cases.c.h
generated
2
Python/executor_cases.c.h
generated
|
@ -17,7 +17,7 @@
|
|||
if (_Py_emscripten_signal_clock == 0) goto deoptimize;
|
||||
_Py_emscripten_signal_clock -= Py_EMSCRIPTEN_SIGNAL_HANDLING;
|
||||
#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;
|
||||
assert((version & _PY_EVAL_EVENTS_MASK) == 0);
|
||||
if (eval_breaker != version) goto deoptimize;
|
||||
|
|
|
@ -1771,9 +1771,12 @@ PyObject_IS_GC(PyObject *obj)
|
|||
}
|
||||
|
||||
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
|
||||
|
@ -1794,7 +1797,7 @@ _PyObject_GC_Link(PyObject *op)
|
|||
!_Py_atomic_load_int_relaxed(&gcstate->collecting) &&
|
||||
!_PyErr_Occurred(tstate))
|
||||
{
|
||||
_Py_ScheduleGC(tstate->interp);
|
||||
_Py_ScheduleGC(tstate);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -981,7 +981,7 @@ record_allocation(PyThreadState *tstate)
|
|||
if (gc_should_collect(gcstate) &&
|
||||
!_Py_atomic_load_int_relaxed(&gcstate->collecting))
|
||||
{
|
||||
_Py_ScheduleGC(tstate->interp);
|
||||
_Py_ScheduleGC(tstate);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1564,9 +1564,12 @@ PyObject_IS_GC(PyObject *obj)
|
|||
}
|
||||
|
||||
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
|
||||
|
|
6
Python/generated_cases.c.h
generated
6
Python/generated_cases.c.h
generated
|
@ -3139,7 +3139,7 @@
|
|||
_Py_CODEUNIT *this_instr = frame->instr_ptr = next_instr;
|
||||
next_instr += 1;
|
||||
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;
|
||||
if (code_version != global_version) {
|
||||
if (_Py_Instrument(_PyFrame_GetCode(frame), tstate->interp)) {
|
||||
|
@ -4809,7 +4809,7 @@
|
|||
TIER_ONE_ONLY
|
||||
assert(frame == tstate->current_frame);
|
||||
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;
|
||||
uintptr_t code_version = _PyFrame_GetCode(frame)->_co_instrumentation_version;
|
||||
assert((code_version & 255) == 0);
|
||||
|
@ -4836,7 +4836,7 @@
|
|||
DEOPT_IF(_Py_emscripten_signal_clock == 0, RESUME);
|
||||
_Py_emscripten_signal_clock -= Py_EMSCRIPTEN_SIGNAL_HANDLING;
|
||||
#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;
|
||||
assert((version & _PY_EVAL_EVENTS_MASK) == 0);
|
||||
DEOPT_IF(eval_breaker != version, RESUME);
|
||||
|
|
|
@ -891,18 +891,43 @@ static inline int most_significant_bit(uint8_t bits) {
|
|||
static uint32_t
|
||||
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
|
||||
set_global_version(PyInterpreterState *interp, uint32_t version)
|
||||
set_global_version(PyThreadState *tstate, uint32_t version)
|
||||
{
|
||||
assert((version & _PY_EVAL_EVENTS_MASK) == 0);
|
||||
uintptr_t old = _Py_atomic_load_uintptr(&interp->ceval.eval_breaker);
|
||||
intptr_t new;
|
||||
do {
|
||||
new = (old & _PY_EVAL_EVENTS_MASK) | version;
|
||||
} while (!_Py_atomic_compare_exchange_uintptr(&interp->ceval.eval_breaker, &old, new));
|
||||
PyInterpreterState *interp = tstate->interp;
|
||||
set_version_raw(&interp->ceval.instrumentation_version, version);
|
||||
|
||||
#ifdef Py_GIL_DISABLED
|
||||
// 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
|
||||
|
@ -1566,7 +1591,7 @@ _Py_Instrument(PyCodeObject *code, PyInterpreterState *interp)
|
|||
{
|
||||
if (is_version_up_to_date(code, interp)) {
|
||||
assert(
|
||||
(interp->ceval.eval_breaker & ~_PY_EVAL_EVENTS_MASK) == 0 ||
|
||||
interp->ceval.instrumentation_version == 0 ||
|
||||
instrumentation_cross_checks(interp, code)
|
||||
);
|
||||
return 0;
|
||||
|
@ -1778,7 +1803,8 @@ int
|
|||
_PyMonitoring_SetEvents(int tool_id, _PyMonitoringEventSet events)
|
||||
{
|
||||
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));
|
||||
if (check_tool(interp, tool_id)) {
|
||||
return -1;
|
||||
|
@ -1793,7 +1819,7 @@ _PyMonitoring_SetEvents(int tool_id, _PyMonitoringEventSet events)
|
|||
PyErr_Format(PyExc_OverflowError, "events set too many times");
|
||||
return -1;
|
||||
}
|
||||
set_global_version(interp, new_version);
|
||||
set_global_version(tstate, new_version);
|
||||
_Py_Executors_InvalidateAll(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 < 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 new_version = restart_version + MONITORING_VERSION_INCREMENT;
|
||||
if (new_version <= MONITORING_VERSION_INCREMENT) {
|
||||
|
@ -2130,7 +2157,7 @@ monitoring_restart_events_impl(PyObject *module)
|
|||
return NULL;
|
||||
}
|
||||
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)) {
|
||||
return NULL;
|
||||
}
|
||||
|
|
|
@ -663,6 +663,7 @@ pycore_create_interpreter(_PyRuntimeState *runtime,
|
|||
if (tstate == NULL) {
|
||||
return _PyStatus_ERR("can't make first thread");
|
||||
}
|
||||
runtime->main_tstate = tstate;
|
||||
_PyThreadState_Bind(tstate);
|
||||
|
||||
init_interp_create_gil(tstate, config.gil);
|
||||
|
|
|
@ -794,9 +794,7 @@ interpreter_clear(PyInterpreterState *interp, PyThreadState *tstate)
|
|||
|
||||
Py_CLEAR(interp->audit_hooks);
|
||||
|
||||
// At this time, all the threads should be cleared so we don't need
|
||||
// atomic operations for eval_breaker
|
||||
interp->ceval.eval_breaker = 0;
|
||||
interp->ceval.instrumentation_version = 0;
|
||||
|
||||
for (int i = 0; i < _PY_MONITORING_UNGROUPED_EVENTS; i++) {
|
||||
interp->monitors.tools[i] = 0;
|
||||
|
@ -1318,6 +1316,8 @@ init_threadstate(_PyThreadStateImpl *_tstate,
|
|||
|
||||
assert(interp != NULL);
|
||||
tstate->interp = interp;
|
||||
tstate->eval_breaker =
|
||||
_Py_atomic_load_uintptr_relaxed(&interp->ceval.instrumentation_version);
|
||||
|
||||
// next/prev are set in add_threadstate().
|
||||
assert(tstate->next == NULL);
|
||||
|
@ -2021,8 +2021,7 @@ park_detached_threads(struct _stoptheworld_state *stw)
|
|||
}
|
||||
}
|
||||
else if (state == _Py_THREAD_ATTACHED && t != stw->requester) {
|
||||
// TODO: set this per-thread, rather than per-interpreter.
|
||||
_Py_set_eval_breaker_bit(t->interp, _PY_EVAL_PLEASE_STOP_BIT, 1);
|
||||
_Py_set_eval_breaker_bit(t, _PY_EVAL_PLEASE_STOP_BIT);
|
||||
}
|
||||
}
|
||||
stw->thread_countdown -= num_parked;
|
||||
|
@ -2186,19 +2185,18 @@ PyThreadState_SetAsyncExc(unsigned long id, PyObject *exc)
|
|||
* deadlock, we need to release head_mutex before
|
||||
* the decref.
|
||||
*/
|
||||
PyObject *old_exc = tstate->async_exc;
|
||||
tstate->async_exc = Py_XNewRef(exc);
|
||||
Py_XINCREF(exc);
|
||||
PyObject *old_exc = _Py_atomic_exchange_ptr(&tstate->async_exc, exc);
|
||||
HEAD_UNLOCK(runtime);
|
||||
|
||||
Py_XDECREF(old_exc);
|
||||
_PyEval_SignalAsyncExc(tstate->interp);
|
||||
_Py_set_eval_breaker_bit(tstate, _PY_ASYNC_EXCEPTION_BIT);
|
||||
return 1;
|
||||
}
|
||||
HEAD_UNLOCK(runtime);
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
||||
//---------------------------------
|
||||
// API for the current thread state
|
||||
//---------------------------------
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue