GH-109369: Merge all eval-breaker flags and monitoring version into one word. (GH-109846)

This commit is contained in:
Mark Shannon 2023-10-04 16:09:48 +01:00 committed by GitHub
parent 7c149a76b2
commit bf4bc36069
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
13 changed files with 188 additions and 234 deletions

View file

@ -167,7 +167,7 @@ typedef struct {
PyObject *co_weakreflist; /* to support weakrefs to code objects */ \
_PyExecutorArray *co_executors; /* executors from optimizer */ \
_PyCoCached *_co_cached; /* cached co_* attributes */ \
uint64_t _co_instrumentation_version; /* current instrumentation version */ \
uintptr_t _co_instrumentation_version; /* current instrumentation version */ \
_PyCoMonitoringData *_co_monitoring; /* Monitoring data */ \
int _co_firsttraceable; /* index of first traceable instruction */ \
/* Scratch space for extra data relating to the code object. \

View file

@ -193,6 +193,39 @@ int _PyEval_UnpackIterable(PyThreadState *tstate, PyObject *v, int argcnt, int a
void _PyEval_FrameClearAndPop(PyThreadState *tstate, _PyInterpreterFrame *frame);
#define _PY_GIL_DROP_REQUEST_BIT 0
#define _PY_SIGNALS_PENDING_BIT 1
#define _PY_CALLS_TO_DO_BIT 2
#define _PY_ASYNC_EXCEPTION_BIT 3
#define _PY_GC_SCHEDULED_BIT 4
/* Reserve a few bits for future use */
#define _PY_EVAL_EVENTS_BITS 8
#define _PY_EVAL_EVENTS_MASK ((1 << _PY_EVAL_EVENTS_BITS)-1)
static inline void
_Py_set_eval_breaker_bit(PyInterpreterState *interp, uint32_t bit, uint32_t set)
{
assert(set == 0 || set == 1);
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
_Py_eval_breaker_bit_is_set(PyInterpreterState *interp, int32_t bit)
{
return _Py_atomic_load_uintptr_relaxed(&interp->ceval.eval_breaker) & (((uintptr_t)1) << bit);
}
#ifdef __cplusplus
}
#endif

View file

@ -17,11 +17,7 @@ struct _pending_calls {
int busy;
PyThread_type_lock lock;
/* Request for running pending calls. */
_Py_atomic_int calls_to_do;
/* Request for looking at the `async_exc` field of the current
thread state.
Guarded by the GIL. */
int async_exc;
int32_t calls_to_do;
#define NPENDINGCALLS 32
struct _pending_call {
_Py_pending_call_func func;
@ -62,11 +58,6 @@ struct _ceval_runtime_state {
int _not_used;
#endif
} perf;
/* Request for checking signals. It is shared by all interpreters (see
bpo-40513). Any thread of any interpreter can receive a signal, but only
the main thread of the main interpreter can handle signals: see
_Py_ThreadCanHandleSignals(). */
_Py_atomic_int signals_pending;
/* Pending calls to be made only on the main thread. */
struct _pending_calls pending_mainthread;
};
@ -87,14 +78,12 @@ struct _ceval_state {
* the fast path in the eval loop.
* It is by far the hottest field in this struct and
* should be placed at the beginning. */
_Py_atomic_int eval_breaker;
/* Request for dropping the GIL */
_Py_atomic_int gil_drop_request;
uintptr_t eval_breaker;
/* Avoid false sharing */
int64_t padding[7];
int recursion_limit;
struct _gil_runtime_state *gil;
int own_gil;
/* The GC is ready to be executed */
_Py_atomic_int gc_scheduled;
struct _pending_calls pending;
};

View file

@ -67,8 +67,7 @@ struct _is {
int _initialized;
int finalizing;
uint64_t monitoring_version;
uint64_t last_restart_version;
uintptr_t last_restart_version;
struct pythreads {
uint64_t next_unique_id;
/* The linked list of threads, newest first. */

View file

@ -0,0 +1,2 @@
The internal eval_breaker and supporting flags, plus the monitoring version
have been merged into a single atomic integer to speed up checks.

View file

@ -24,6 +24,7 @@
*/
#include "Python.h"
#include "pycore_ceval.h" // _Py_set_eval_breaker_bit()
#include "pycore_context.h"
#include "pycore_dict.h" // _PyDict_MaybeUntrack()
#include "pycore_initconfig.h"
@ -2274,11 +2275,7 @@ _Py_ScheduleGC(PyInterpreterState *interp)
if (gcstate->collecting == 1) {
return;
}
struct _ceval_state *ceval = &interp->ceval;
if (!_Py_atomic_load_relaxed(&ceval->gc_scheduled)) {
_Py_atomic_store_relaxed(&ceval->gc_scheduled, 1);
_Py_atomic_store_relaxed(&ceval->eval_breaker, 1);
}
_Py_set_eval_breaker_bit(interp, _PY_GC_SCHEDULED_BIT, 1);
}
void

View file

@ -1767,9 +1767,8 @@ PyErr_CheckSignals(void)
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
even if it doesn't run the evaluation loop */
struct _ceval_state *interp_ceval_state = &tstate->interp->ceval;
if (_Py_atomic_load_relaxed(&interp_ceval_state->gc_scheduled)) {
_Py_atomic_store_relaxed(&interp_ceval_state->gc_scheduled, 0);
if (_Py_eval_breaker_bit_is_set(tstate->interp, _PY_GC_SCHEDULED_BIT)) {
_Py_set_eval_breaker_bit(tstate->interp, _PY_GC_SCHEDULED_BIT, 0);
_Py_RunGC(tstate);
}

View file

@ -136,7 +136,12 @@ dummy_func(
inst(RESUME, (--)) {
TIER_ONE_ONLY
assert(frame == tstate->current_frame);
if (_PyFrame_GetCode(frame)->_co_instrumentation_version != tstate->interp->monitoring_version) {
uintptr_t global_version =
_Py_atomic_load_uintptr_relaxed(&tstate->interp->ceval.eval_breaker) &
~_PY_EVAL_EVENTS_MASK;
uintptr_t code_version = _PyFrame_GetCode(frame)->_co_instrumentation_version;
assert((code_version & 255) == 0);
if (code_version != global_version) {
int err = _Py_Instrument(_PyFrame_GetCode(frame), tstate->interp);
ERROR_IF(err, error);
next_instr--;
@ -154,17 +159,16 @@ dummy_func(
DEOPT_IF(_Py_emscripten_signal_clock == 0);
_Py_emscripten_signal_clock -= Py_EMSCRIPTEN_SIGNAL_HANDLING;
#endif
/* Possibly combine these two checks */
DEOPT_IF(_PyFrame_GetCode(frame)->_co_instrumentation_version != tstate->interp->monitoring_version);
DEOPT_IF(_Py_atomic_load_relaxed_int32(&tstate->interp->ceval.eval_breaker));
uintptr_t eval_breaker = _Py_atomic_load_uintptr_relaxed(&tstate->interp->ceval.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, (--)) {
/* Possible performance enhancement:
* We need to check the eval breaker anyway, can we
* combine the instrument verison check and the eval breaker test?
*/
if (_PyFrame_GetCode(frame)->_co_instrumentation_version != tstate->interp->monitoring_version) {
uintptr_t global_version = _Py_atomic_load_uintptr_relaxed(&tstate->interp->ceval.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)) {
goto error;
}

View file

@ -57,113 +57,62 @@
#define _Py_atomic_load_relaxed_int32(ATOMIC_VAL) _Py_atomic_load_relaxed(ATOMIC_VAL)
#endif
/* This can set eval_breaker to 0 even though gil_drop_request became
1. We believe this is all right because the eval loop will release
the GIL eventually anyway. */
/* 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.
*/
static inline void
COMPUTE_EVAL_BREAKER(PyInterpreterState *interp,
struct _ceval_runtime_state *ceval,
struct _ceval_state *ceval2)
update_eval_breaker_from_thread(PyInterpreterState *interp, PyThreadState *tstate)
{
_Py_atomic_store_relaxed(&ceval2->eval_breaker,
_Py_atomic_load_relaxed_int32(&ceval2->gil_drop_request)
| (_Py_atomic_load_relaxed_int32(&ceval->signals_pending)
&& _Py_ThreadCanHandleSignals(interp))
| (_Py_atomic_load_relaxed_int32(&ceval2->pending.calls_to_do))
| (_Py_IsMainThread() && _Py_IsMainInterpreter(interp)
&&_Py_atomic_load_relaxed_int32(&ceval->pending_mainthread.calls_to_do))
| ceval2->pending.async_exc
| _Py_atomic_load_relaxed_int32(&ceval2->gc_scheduled));
}
if (tstate == NULL) {
return;
}
if (_Py_IsMainThread()) {
int32_t 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(&_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
SET_GIL_DROP_REQUEST(PyInterpreterState *interp)
{
struct _ceval_state *ceval2 = &interp->ceval;
_Py_atomic_store_relaxed(&ceval2->gil_drop_request, 1);
_Py_atomic_store_relaxed(&ceval2->eval_breaker, 1);
_Py_set_eval_breaker_bit(interp, _PY_GIL_DROP_REQUEST_BIT, 1);
}
static inline void
RESET_GIL_DROP_REQUEST(PyInterpreterState *interp)
{
struct _ceval_runtime_state *ceval = &interp->runtime->ceval;
struct _ceval_state *ceval2 = &interp->ceval;
_Py_atomic_store_relaxed(&ceval2->gil_drop_request, 0);
COMPUTE_EVAL_BREAKER(interp, ceval, ceval2);
_Py_set_eval_breaker_bit(interp, _PY_GIL_DROP_REQUEST_BIT, 0);
}
static inline void
SIGNAL_PENDING_CALLS(struct _pending_calls *pending, PyInterpreterState *interp)
SIGNAL_PENDING_CALLS(PyInterpreterState *interp)
{
struct _ceval_runtime_state *ceval = &interp->runtime->ceval;
struct _ceval_state *ceval2 = &interp->ceval;
_Py_atomic_store_relaxed(&pending->calls_to_do, 1);
COMPUTE_EVAL_BREAKER(interp, ceval, ceval2);
_Py_set_eval_breaker_bit(interp, _PY_CALLS_TO_DO_BIT, 1);
}
static inline void
UNSIGNAL_PENDING_CALLS(PyInterpreterState *interp)
{
struct _ceval_runtime_state *ceval = &interp->runtime->ceval;
struct _ceval_state *ceval2 = &interp->ceval;
if (_Py_IsMainThread() && _Py_IsMainInterpreter(interp)) {
_Py_atomic_store_relaxed(&ceval->pending_mainthread.calls_to_do, 0);
}
_Py_atomic_store_relaxed(&ceval2->pending.calls_to_do, 0);
COMPUTE_EVAL_BREAKER(interp, ceval, ceval2);
_Py_set_eval_breaker_bit(interp, _PY_CALLS_TO_DO_BIT, 0);
}
static inline void
SIGNAL_PENDING_SIGNALS(PyInterpreterState *interp, int force)
{
struct _ceval_runtime_state *ceval = &interp->runtime->ceval;
struct _ceval_state *ceval2 = &interp->ceval;
_Py_atomic_store_relaxed(&ceval->signals_pending, 1);
if (force) {
_Py_atomic_store_relaxed(&ceval2->eval_breaker, 1);
}
else {
/* eval_breaker is not set to 1 if thread_can_handle_signals() is false */
COMPUTE_EVAL_BREAKER(interp, ceval, ceval2);
}
}
static inline void
UNSIGNAL_PENDING_SIGNALS(PyInterpreterState *interp)
{
struct _ceval_runtime_state *ceval = &interp->runtime->ceval;
struct _ceval_state *ceval2 = &interp->ceval;
_Py_atomic_store_relaxed(&ceval->signals_pending, 0);
COMPUTE_EVAL_BREAKER(interp, ceval, ceval2);
}
static inline void
SIGNAL_ASYNC_EXC(PyInterpreterState *interp)
{
struct _ceval_state *ceval2 = &interp->ceval;
ceval2->pending.async_exc = 1;
_Py_atomic_store_relaxed(&ceval2->eval_breaker, 1);
}
static inline void
UNSIGNAL_ASYNC_EXC(PyInterpreterState *interp)
{
struct _ceval_runtime_state *ceval = &interp->runtime->ceval;
struct _ceval_state *ceval2 = &interp->ceval;
ceval2->pending.async_exc = 0;
COMPUTE_EVAL_BREAKER(interp, ceval, ceval2);
}
/*
* Implementation of the Global Interpreter Lock (GIL).
*/
@ -271,8 +220,9 @@ static void recreate_gil(struct _gil_runtime_state *gil)
#endif
static void
drop_gil(struct _ceval_state *ceval, PyThreadState *tstate)
drop_gil(PyInterpreterState *interp, PyThreadState *tstate)
{
struct _ceval_state *ceval = &interp->ceval;
/* If tstate is NULL, the caller is indicating that we're releasing
the GIL for the last time in this thread. This is particularly
relevant when the current thread state is finalizing or its
@ -310,7 +260,7 @@ drop_gil(struct _ceval_state *ceval, 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_atomic_load_relaxed(&ceval->gil_drop_request)) {
if (tstate != NULL && _Py_eval_breaker_bit_is_set(interp, _PY_GIL_DROP_REQUEST_BIT)) {
MUTEX_LOCK(gil->switch_mutex);
/* Not switched yet => wait */
if (((PyThreadState*)_Py_atomic_load_relaxed(&gil->last_holder)) == tstate)
@ -356,8 +306,7 @@ take_gil(PyThreadState *tstate)
assert(_PyThreadState_CheckConsistency(tstate));
PyInterpreterState *interp = tstate->interp;
struct _ceval_state *ceval = &interp->ceval;
struct _gil_runtime_state *gil = ceval->gil;
struct _gil_runtime_state *gil = interp->ceval.gil;
/* Check that _PyEval_InitThreads() was called to create the lock */
assert(gil_created(gil));
@ -431,27 +380,13 @@ 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(ceval, tstate);
drop_gil(interp, tstate);
PyThread_exit_thread();
}
assert(_PyThreadState_CheckConsistency(tstate));
if (_Py_atomic_load_relaxed(&ceval->gil_drop_request)) {
RESET_GIL_DROP_REQUEST(interp);
}
else {
/* bpo-40010: eval_breaker should be recomputed to be set to 1 if there
is a pending signal: signal received by another thread which cannot
handle signals.
Note: RESET_GIL_DROP_REQUEST() calls COMPUTE_EVAL_BREAKER(). */
COMPUTE_EVAL_BREAKER(interp, &_PyRuntime.ceval, ceval);
}
/* Don't access tstate if the thread must exit */
if (tstate->async_exc != NULL) {
_PyEval_SignalAsyncExc(tstate->interp);
}
RESET_GIL_DROP_REQUEST(interp);
update_eval_breaker_from_thread(interp, tstate);
MUTEX_UNLOCK(gil->mutex);
@ -611,8 +546,7 @@ PyEval_ReleaseLock(void)
/* This function must succeed when the current thread state is NULL.
We therefore avoid PyThreadState_Get() which dumps a fatal error
in debug mode. */
struct _ceval_state *ceval = &tstate->interp->ceval;
drop_gil(ceval, tstate);
drop_gil(tstate->interp, tstate);
}
void
@ -628,8 +562,7 @@ _PyEval_ReleaseLock(PyInterpreterState *interp, PyThreadState *tstate)
/* If tstate is NULL then we do not expect the current thread
to acquire the GIL ever again. */
assert(tstate == NULL || tstate->interp == interp);
struct _ceval_state *ceval = &interp->ceval;
drop_gil(ceval, tstate);
drop_gil(interp, tstate);
}
void
@ -653,8 +586,7 @@ PyEval_ReleaseThread(PyThreadState *tstate)
if (new_tstate != tstate) {
Py_FatalError("wrong thread state");
}
struct _ceval_state *ceval = &tstate->interp->ceval;
drop_gil(ceval, tstate);
drop_gil(tstate->interp, tstate);
}
#ifdef HAVE_FORK
@ -691,7 +623,7 @@ _PyEval_ReInitThreads(PyThreadState *tstate)
void
_PyEval_SignalAsyncExc(PyInterpreterState *interp)
{
SIGNAL_ASYNC_EXC(interp);
_Py_set_eval_breaker_bit(interp, _PY_ASYNC_EXCEPTION_BIT, 1);
}
PyThreadState *
@ -700,9 +632,8 @@ PyEval_SaveThread(void)
PyThreadState *tstate = _PyThreadState_SwapNoGIL(NULL);
_Py_EnsureTstateNotNULL(tstate);
struct _ceval_state *ceval = &tstate->interp->ceval;
assert(gil_created(ceval->gil));
drop_gil(ceval, tstate);
assert(gil_created(tstate->interp->ceval.gil));
drop_gil(tstate->interp, tstate);
return tstate;
}
@ -742,22 +673,9 @@ PyEval_RestoreThread(PyThreadState *tstate)
void
_PyEval_SignalReceived(PyInterpreterState *interp)
{
#ifdef MS_WINDOWS
// bpo-42296: On Windows, _PyEval_SignalReceived() is called from a signal
// handler which can run in a thread different than the Python thread, in
// which case _Py_ThreadCanHandleSignals() is wrong. Ignore
// _Py_ThreadCanHandleSignals() and always set eval_breaker to 1.
//
// The next eval_frame_handle_pending() call will call
// _Py_ThreadCanHandleSignals() to recompute eval_breaker.
int force = 1;
#else
int force = 0;
#endif
/* bpo-30703: Function called when the C signal handler of Python gets a
signal. We cannot queue a callback using _PyEval_AddPendingCall() since
that function is not async-signal-safe. */
SIGNAL_PENDING_SIGNALS(interp, force);
if (_Py_ThreadCanHandleSignals(interp)) {
_Py_set_eval_breaker_bit(interp, _PY_SIGNALS_PENDING_BIT, 1);
}
}
/* Push one item onto the queue while holding the lock. */
@ -773,6 +691,8 @@ _push_pending_call(struct _pending_calls *pending,
pending->calls[i].func = func;
pending->calls[i].arg = arg;
pending->last = j;
assert(pending->calls_to_do < NPENDINGCALLS);
pending->calls_to_do++;
return 0;
}
@ -800,6 +720,8 @@ _pop_pending_call(struct _pending_calls *pending,
if (i >= 0) {
pending->calls[i] = (struct _pending_call){0};
pending->first = (i + 1) % NPENDINGCALLS;
assert(pending->calls_to_do > 0);
pending->calls_to_do--;
}
}
@ -829,7 +751,7 @@ _PyEval_AddPendingCall(PyInterpreterState *interp,
PyThread_release_lock(pending->lock);
/* signal main loop */
SIGNAL_PENDING_CALLS(pending, interp);
SIGNAL_PENDING_CALLS(interp);
return result;
}
@ -846,33 +768,18 @@ static int
handle_signals(PyThreadState *tstate)
{
assert(_PyThreadState_CheckConsistency(tstate));
_Py_set_eval_breaker_bit(tstate->interp, _PY_SIGNALS_PENDING_BIT, 0);
if (!_Py_ThreadCanHandleSignals(tstate->interp)) {
return 0;
}
UNSIGNAL_PENDING_SIGNALS(tstate->interp);
if (_PyErr_CheckSignalsTstate(tstate) < 0) {
/* On failure, re-schedule a call to handle_signals(). */
SIGNAL_PENDING_SIGNALS(tstate->interp, 0);
_Py_set_eval_breaker_bit(tstate->interp, _PY_SIGNALS_PENDING_BIT, 1);
return -1;
}
return 0;
}
static inline int
maybe_has_pending_calls(PyInterpreterState *interp)
{
struct _pending_calls *pending = &interp->ceval.pending;
if (_Py_atomic_load_relaxed_int32(&pending->calls_to_do)) {
return 1;
}
if (!_Py_IsMainThread() || !_Py_IsMainInterpreter(interp)) {
return 0;
}
pending = &_PyRuntime.ceval.pending_mainthread;
return _Py_atomic_load_relaxed_int32(&pending->calls_to_do);
}
static int
_make_pending_calls(struct _pending_calls *pending)
{
@ -930,7 +837,7 @@ make_pending_calls(PyInterpreterState *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(pending, interp);
SIGNAL_PENDING_CALLS(interp);
return -1;
}
@ -938,7 +845,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(pending_main, interp);
SIGNAL_PENDING_CALLS(interp);
return -1;
}
}
@ -1083,38 +990,35 @@ _PyEval_FiniState(struct _ceval_state *ceval)
int
_Py_HandlePending(PyThreadState *tstate)
{
_PyRuntimeState * const runtime = &_PyRuntime;
struct _ceval_runtime_state *ceval = &runtime->ceval;
struct _ceval_state *interp_ceval_state = &tstate->interp->ceval;
PyInterpreterState *interp = tstate->interp;
/* Pending signals */
if (_Py_atomic_load_relaxed_int32(&ceval->signals_pending)) {
if (_Py_eval_breaker_bit_is_set(interp, _PY_SIGNALS_PENDING_BIT)) {
if (handle_signals(tstate) != 0) {
return -1;
}
}
/* Pending calls */
if (maybe_has_pending_calls(tstate->interp)) {
if (make_pending_calls(tstate->interp) != 0) {
if (_Py_eval_breaker_bit_is_set(interp, _PY_CALLS_TO_DO_BIT)) {
if (make_pending_calls(interp) != 0) {
return -1;
}
}
/* GC scheduled to run */
if (_Py_atomic_load_relaxed_int32(&interp_ceval_state->gc_scheduled)) {
_Py_atomic_store_relaxed(&interp_ceval_state->gc_scheduled, 0);
COMPUTE_EVAL_BREAKER(tstate->interp, ceval, interp_ceval_state);
if (_Py_eval_breaker_bit_is_set(interp, _PY_GC_SCHEDULED_BIT)) {
_Py_set_eval_breaker_bit(interp, _PY_GC_SCHEDULED_BIT, 0);
_Py_RunGC(tstate);
}
/* GIL drop request */
if (_Py_atomic_load_relaxed_int32(&interp_ceval_state->gil_drop_request)) {
if (_Py_eval_breaker_bit_is_set(interp, _PY_GIL_DROP_REQUEST_BIT)) {
/* Give another thread a chance */
if (_PyThreadState_SwapNoGIL(NULL) != tstate) {
Py_FatalError("tstate mix-up");
}
drop_gil(interp_ceval_state, tstate);
drop_gil(interp, tstate);
/* Other threads may run now */
@ -1126,27 +1030,16 @@ _Py_HandlePending(PyThreadState *tstate)
}
/* Check for asynchronous exception. */
if (tstate->async_exc != NULL) {
PyObject *exc = tstate->async_exc;
tstate->async_exc = NULL;
UNSIGNAL_ASYNC_EXC(tstate->interp);
_PyErr_SetNone(tstate, exc);
Py_DECREF(exc);
return -1;
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;
_PyErr_SetNone(tstate, exc);
Py_DECREF(exc);
return -1;
}
}
// It is possible that some of the conditions that trigger the eval breaker
// are called in a different thread than the Python thread. An example of
// this is bpo-42296: On Windows, _PyEval_SignalReceived() can be called in
// a different thread than the Python thread, in which case
// _Py_ThreadCanHandleSignals() is wrong. Recompute eval_breaker in the
// current Python thread with the correct _Py_ThreadCanHandleSignals()
// value. It prevents to interrupt the eval loop at every instruction if
// the current Python thread cannot handle signals (if
// _Py_ThreadCanHandleSignals() is false).
COMPUTE_EVAL_BREAKER(tstate->interp, ceval, interp_ceval_state);
return 0;
}

View file

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

View file

@ -12,9 +12,10 @@
DEOPT_IF(_Py_emscripten_signal_clock == 0, RESUME);
_Py_emscripten_signal_clock -= Py_EMSCRIPTEN_SIGNAL_HANDLING;
#endif
/* Possibly combine these two checks */
DEOPT_IF(_PyFrame_GetCode(frame)->_co_instrumentation_version != tstate->interp->monitoring_version, RESUME);
DEOPT_IF(_Py_atomic_load_relaxed_int32(&tstate->interp->ceval.eval_breaker), RESUME);
uintptr_t eval_breaker = _Py_atomic_load_uintptr_relaxed(&tstate->interp->ceval.eval_breaker);
uintptr_t version = _PyFrame_GetCode(frame)->_co_instrumentation_version;
assert((version & _PY_EVAL_EVENTS_MASK) == 0);
DEOPT_IF(eval_breaker != version, RESUME);
break;
}

View file

@ -12,7 +12,12 @@
static_assert(0 == 0, "incorrect cache size");
TIER_ONE_ONLY
assert(frame == tstate->current_frame);
if (_PyFrame_GetCode(frame)->_co_instrumentation_version != tstate->interp->monitoring_version) {
uintptr_t global_version =
_Py_atomic_load_uintptr_relaxed(&tstate->interp->ceval.eval_breaker) &
~_PY_EVAL_EVENTS_MASK;
uintptr_t code_version = _PyFrame_GetCode(frame)->_co_instrumentation_version;
assert((code_version & 255) == 0);
if (code_version != global_version) {
int err = _Py_Instrument(_PyFrame_GetCode(frame), tstate->interp);
if (err) goto error;
next_instr--;
@ -31,18 +36,17 @@
DEOPT_IF(_Py_emscripten_signal_clock == 0, RESUME);
_Py_emscripten_signal_clock -= Py_EMSCRIPTEN_SIGNAL_HANDLING;
#endif
/* Possibly combine these two checks */
DEOPT_IF(_PyFrame_GetCode(frame)->_co_instrumentation_version != tstate->interp->monitoring_version, RESUME);
DEOPT_IF(_Py_atomic_load_relaxed_int32(&tstate->interp->ceval.eval_breaker), RESUME);
uintptr_t eval_breaker = _Py_atomic_load_uintptr_relaxed(&tstate->interp->ceval.eval_breaker);
uintptr_t version = _PyFrame_GetCode(frame)->_co_instrumentation_version;
assert((version & _PY_EVAL_EVENTS_MASK) == 0);
DEOPT_IF(eval_breaker != version, RESUME);
DISPATCH();
}
TARGET(INSTRUMENTED_RESUME) {
/* Possible performance enhancement:
* We need to check the eval breaker anyway, can we
* combine the instrument verison check and the eval breaker test?
*/
if (_PyFrame_GetCode(frame)->_co_instrumentation_version != tstate->interp->monitoring_version) {
uintptr_t global_version = _Py_atomic_load_uintptr_relaxed(&tstate->interp->ceval.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)) {
goto error;
}

View file

@ -4,6 +4,7 @@
#include "pycore_bitutils.h" // _Py_popcount32
#include "pycore_call.h"
#include "pycore_ceval.h" // _PY_EVAL_EVENTS_BITS
#include "pycore_code.h" // _PyCode_Clear_Executors()
#include "pycore_frame.h"
#include "pycore_interp.h"
@ -895,10 +896,27 @@ static inline int most_significant_bit(uint8_t bits) {
return MOST_SIGNIFICANT_BITS[bits];
}
static uint32_t
global_version(PyInterpreterState *interp)
{
return interp->ceval.eval_breaker & ~_PY_EVAL_EVENTS_MASK;
}
static void
set_global_version(PyInterpreterState *interp, 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));
}
static bool
is_version_up_to_date(PyCodeObject *code, PyInterpreterState *interp)
{
return interp->monitoring_version == code->_co_instrumentation_version;
return global_version(interp) == code->_co_instrumentation_version;
}
#ifndef NDEBUG
@ -1556,7 +1574,7 @@ _Py_Instrument(PyCodeObject *code, PyInterpreterState *interp)
{
if (is_version_up_to_date(code, interp)) {
assert(
interp->monitoring_version == 0 ||
(interp->ceval.eval_breaker & ~_PY_EVAL_EVENTS_MASK) == 0 ||
instrumentation_cross_checks(interp, code)
);
return 0;
@ -1594,7 +1612,7 @@ _Py_Instrument(PyCodeObject *code, PyInterpreterState *interp)
assert(monitors_are_empty(monitors_and(new_events, removed_events)));
}
code->_co_monitoring->active_monitors = active_events;
code->_co_instrumentation_version = interp->monitoring_version;
code->_co_instrumentation_version = global_version(interp);
if (monitors_are_empty(new_events) && monitors_are_empty(removed_events)) {
#ifdef INSTRUMENT_DEBUG
sanity_check_instrumentation(code);
@ -1761,6 +1779,10 @@ check_tool(PyInterpreterState *interp, int tool_id)
return 0;
}
/* We share the eval-breaker with flags, so the monitoring
* version goes in the top 24 bits */
#define MONITORING_VERSION_INCREMENT (1 << _PY_EVAL_EVENTS_BITS)
int
_PyMonitoring_SetEvents(int tool_id, _PyMonitoringEventSet events)
{
@ -1775,7 +1797,12 @@ _PyMonitoring_SetEvents(int tool_id, _PyMonitoringEventSet events)
return 0;
}
set_events(&interp->monitors, tool_id, events);
interp->monitoring_version++;
uint32_t new_version = global_version(interp) + MONITORING_VERSION_INCREMENT;
if (new_version == 0) {
PyErr_Format(PyExc_OverflowError, "events set too many times");
return -1;
}
set_global_version(interp, new_version);
return instrument_all_executing_code_objects(interp);
}
@ -1803,7 +1830,7 @@ _PyMonitoring_SetLocalEvents(PyCodeObject *code, int tool_id, _PyMonitoringEvent
set_local_events(local, tool_id, events);
if (is_version_up_to_date(code, interp)) {
/* Force instrumentation update */
code->_co_instrumentation_version = UINT64_MAX;
code->_co_instrumentation_version -= MONITORING_VERSION_INCREMENT;
}
if (_Py_Instrument(code, interp)) {
return -1;
@ -2086,8 +2113,14 @@ monitoring_restart_events_impl(PyObject *module)
* last restart version < current version
*/
PyInterpreterState *interp = _PyInterpreterState_GET();
interp->last_restart_version = interp->monitoring_version + 1;
interp->monitoring_version = interp->last_restart_version + 1;
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) {
PyErr_Format(PyExc_OverflowError, "events set too many times");
return NULL;
}
interp->last_restart_version = restart_version;
set_global_version(interp, new_version);
if (instrument_all_executing_code_objects(interp)) {
return NULL;
}