From b35b80c389150e3197a9470e0ecccb4a7c78bd48 Mon Sep 17 00:00:00 2001 From: Neil Schemenauer Date: Fri, 22 Jan 2010 09:20:56 -0600 Subject: [PATCH 1/3] Merge in the new GIL. --- Include/ceval.h | 8 +- Include/pystate.h | 2 + Include/sysmodule.h | 1 - Lib/test/test_sys.py | 15 +++ Objects/longobject.c | 5 +- Python/ceval.c | 153 +++++++++++++++-------- Python/ceval_gil.h | 335 ++++++++++++++++++++++++++++++++++++++++++++++++++ Python/pystate.c | 1 + Python/sysmodule.c | 64 +++++++++- 9 files changed, 518 insertions(+), 66 deletions(-) create mode 100644 Python/ceval_gil.h diff --git a/Include/ceval.h b/Include/ceval.h index f441197..6b5b14c 100644 --- a/Include/ceval.h +++ b/Include/ceval.h @@ -69,10 +69,6 @@ PyAPI_FUNC(PyObject *) PyEval_GetCallStats(PyObject *); PyAPI_FUNC(PyObject *) PyEval_EvalFrame(struct _frame *); PyAPI_FUNC(PyObject *) PyEval_EvalFrameEx(struct _frame *f, int exc); -/* this used to be handled on a per-thread basis - now just two globals */ -PyAPI_DATA(volatile int) _Py_Ticker; -PyAPI_DATA(int) _Py_CheckInterval; - /* Interface for threads. A module that plans to do a blocking system call (or something else @@ -131,6 +127,9 @@ PyAPI_FUNC(void) PyEval_AcquireThread(PyThreadState *tstate); PyAPI_FUNC(void) PyEval_ReleaseThread(PyThreadState *tstate); PyAPI_FUNC(void) PyEval_ReInitThreads(void); +PyAPI_FUNC(void) _PyEval_SetSwitchInterval(unsigned long microseconds); +PyAPI_FUNC(unsigned long) _PyEval_GetSwitchInterval(void); + #define Py_BEGIN_ALLOW_THREADS { \ PyThreadState *_save; \ _save = PyEval_SaveThread(); @@ -149,6 +148,7 @@ PyAPI_FUNC(void) PyEval_ReInitThreads(void); #endif /* !WITH_THREAD */ PyAPI_FUNC(int) _PyEval_SliceIndex(PyObject *, Py_ssize_t *); +PyAPI_FUNC(void) _PyEval_SignalAsyncExc(void); #ifdef __cplusplus diff --git a/Include/pystate.h b/Include/pystate.h index 4919d99..8cac8ca 100644 --- a/Include/pystate.h +++ b/Include/pystate.h @@ -82,6 +82,8 @@ typedef struct _ts { PyObject *dict; /* Stores per-thread state */ + /* XXX doesn't mean anything anymore (the comment below is obsolete) + => deprecate or remove? */ /* tick_counter is incremented whenever the check_interval ticker * reaches zero. The purpose is to give a useful measure of the number * of interpreted bytecode instructions in a given thread. This diff --git a/Include/sysmodule.h b/Include/sysmodule.h index 536a180..542c3d3 100644 --- a/Include/sysmodule.h +++ b/Include/sysmodule.h @@ -19,7 +19,6 @@ PyAPI_FUNC(void) PySys_WriteStderr(const char *format, ...) Py_GCC_ATTRIBUTE((format(printf, 1, 2))); PyAPI_DATA(PyObject *) _PySys_TraceFunc, *_PySys_ProfileFunc; -PyAPI_DATA(int) _PySys_CheckInterval; PyAPI_FUNC(void) PySys_ResetWarnOptions(void); PyAPI_FUNC(void) PySys_AddWarnOption(char *); diff --git a/Lib/test/test_sys.py b/Lib/test/test_sys.py index 6768775..7333569 100644 --- a/Lib/test/test_sys.py +++ b/Lib/test/test_sys.py @@ -193,6 +193,21 @@ class SysModuleTest(unittest.TestCase): sys.setcheckinterval(n) self.assertEquals(sys.getcheckinterval(), n) + def test_switchinterval(self): + self.assertRaises(TypeError, sys.setswitchinterval) + self.assertRaises(TypeError, sys.setswitchinterval, "a") + self.assertRaises(ValueError, sys.setswitchinterval, -1.0) + self.assertRaises(ValueError, sys.setswitchinterval, 0.0) + orig = sys.getswitchinterval() + # sanity check + self.assertTrue(orig < 0.5, orig) + try: + for n in 0.00001, 0.05, 3.0, orig: + sys.setswitchinterval(n) + self.assertAlmostEquals(sys.getswitchinterval(), n) + finally: + sys.setswitchinterval(orig) + def test_recursionlimit(self): self.assertRaises(TypeError, sys.getrecursionlimit, 42) oldlimit = sys.getrecursionlimit() diff --git a/Objects/longobject.c b/Objects/longobject.c index 51442d3..02c68b6 100644 --- a/Objects/longobject.c +++ b/Objects/longobject.c @@ -34,10 +34,7 @@ #define MIN(x, y) ((x) > (y) ? (y) : (x)) #define SIGCHECK(PyTryBlock) \ - if (--_Py_Ticker < 0) { \ - _Py_Ticker = _Py_CheckInterval; \ - if (PyErr_CheckSignals()) PyTryBlock \ - } + if (PyErr_CheckSignals()) PyTryBlock \ /* Normalize (remove leading zeros from) a long int object. Doesn't attempt to free the storage--in most cases, due to the nature diff --git a/Python/ceval.c b/Python/ceval.c index d501a4e..2e09d34 100644 --- a/Python/ceval.c +++ b/Python/ceval.c @@ -225,6 +225,28 @@ PyEval_GetCallStats(PyObject *self) #endif +#define COMPUTE_EVAL_BREAKER() \ + (eval_breaker = gil_drop_request | pendingcalls_to_do | pending_async_exc) + +#define SET_GIL_DROP_REQUEST() \ + do { gil_drop_request = 1; eval_breaker = 1; } while (0) + +#define RESET_GIL_DROP_REQUEST() \ + do { gil_drop_request = 0; COMPUTE_EVAL_BREAKER(); } while (0) + +#define SIGNAL_PENDING_CALLS() \ + do { pendingcalls_to_do = 1; eval_breaker = 1; } while (0) + +#define UNSIGNAL_PENDING_CALLS() \ + do { pendingcalls_to_do = 0; COMPUTE_EVAL_BREAKER(); } while (0) + +#define SIGNAL_ASYNC_EXC() \ + do { pending_async_exc = 1; eval_breaker = 1; } while (0) + +#define UNSIGNAL_ASYNC_EXC() \ + do { pending_async_exc = 0; COMPUTE_EVAL_BREAKER(); } while (0) + + #ifdef WITH_THREAD #ifdef HAVE_ERRNO_H @@ -232,36 +254,55 @@ PyEval_GetCallStats(PyObject *self) #endif #include "pythread.h" -static PyThread_type_lock interpreter_lock = 0; /* This is the GIL */ static PyThread_type_lock pending_lock = 0; /* for pending calls */ static long main_thread = 0; +/* This single variable consolidates all requests to break out of the fast path + in the eval loop. */ +static volatile int eval_breaker = 0; +/* Request for droppping the GIL */ +static volatile int gil_drop_request = 0; +/* Request for running pending calls */ +static volatile int pendingcalls_to_do = 0; +/* Request for looking at the `async_exc` field of the current thread state */ +static volatile int pending_async_exc = 0; + +#include "ceval_gil.h" int PyEval_ThreadsInitialized(void) { - return interpreter_lock != 0; + return gil_created(); } void PyEval_InitThreads(void) { - if (interpreter_lock) + if (gil_created()) return; - interpreter_lock = PyThread_allocate_lock(); - PyThread_acquire_lock(interpreter_lock, 1); + create_gil(); + take_gil(PyThreadState_GET()); main_thread = PyThread_get_thread_ident(); + if (!pending_lock) + pending_lock = PyThread_allocate_lock(); } void PyEval_AcquireLock(void) { - PyThread_acquire_lock(interpreter_lock, 1); + PyThreadState *tstate = PyThreadState_GET(); + if (tstate == NULL) + Py_FatalError("PyEval_AcquireLock: current thread state is NULL"); + take_gil(tstate); } void PyEval_ReleaseLock(void) { - PyThread_release_lock(interpreter_lock); + /* This function must succeed when the current thread state is NULL. + We therefore avoid PyThreadState_GET() which dumps a fatal error + in debug mode. + */ + drop_gil(_PyThreadState_Current); } void @@ -270,8 +311,8 @@ PyEval_AcquireThread(PyThreadState *tstate) if (tstate == NULL) Py_FatalError("PyEval_AcquireThread: NULL new thread state"); /* Check someone has called PyEval_InitThreads() to create the lock */ - assert(interpreter_lock); - PyThread_acquire_lock(interpreter_lock, 1); + assert(gil_created()); + take_gil(tstate); if (PyThreadState_Swap(tstate) != NULL) Py_FatalError( "PyEval_AcquireThread: non-NULL old thread state"); @@ -284,7 +325,7 @@ PyEval_ReleaseThread(PyThreadState *tstate) Py_FatalError("PyEval_ReleaseThread: NULL thread state"); if (PyThreadState_Swap(NULL) != tstate) Py_FatalError("PyEval_ReleaseThread: wrong thread state"); - PyThread_release_lock(interpreter_lock); + drop_gil(tstate); } /* This function is called from PyOS_AfterFork to ensure that newly @@ -296,17 +337,17 @@ void PyEval_ReInitThreads(void) { PyObject *threading, *result; - PyThreadState *tstate; + PyThreadState *tstate = PyThreadState_GET(); - if (!interpreter_lock) + if (!gil_created()) return; /*XXX Can't use PyThread_free_lock here because it does too much error-checking. Doing this cleanly would require adding a new function to each thread_*.h. Instead, just create a new lock and waste a little bit of memory */ - interpreter_lock = PyThread_allocate_lock(); + recreate_gil(); pending_lock = PyThread_allocate_lock(); - PyThread_acquire_lock(interpreter_lock, 1); + take_gil(tstate); main_thread = PyThread_get_thread_ident(); /* Update the threading module with the new state. @@ -326,7 +367,21 @@ PyEval_ReInitThreads(void) Py_DECREF(result); Py_DECREF(threading); } -#endif + +#else +static int eval_breaker = 0; +static int gil_drop_request = 0; +static int pending_async_exc = 0; +#endif /* WITH_THREAD */ + +/* This function is used to signal that async exceptions are waiting to be + raised, therefore it is also useful in non-threaded builds. */ + +void +_PyEval_SignalAsyncExc(void) +{ + SIGNAL_ASYNC_EXC(); +} /* Functions save_thread and restore_thread are always defined so dynamically loaded modules needn't be compiled separately for use @@ -339,8 +394,8 @@ PyEval_SaveThread(void) if (tstate == NULL) Py_FatalError("PyEval_SaveThread: NULL tstate"); #ifdef WITH_THREAD - if (interpreter_lock) - PyThread_release_lock(interpreter_lock); + if (gil_created()) + drop_gil(tstate); #endif return tstate; } @@ -351,9 +406,9 @@ PyEval_RestoreThread(PyThreadState *tstate) if (tstate == NULL) Py_FatalError("PyEval_RestoreThread: NULL tstate"); #ifdef WITH_THREAD - if (interpreter_lock) { + if (gil_created()) { int err = errno; - PyThread_acquire_lock(interpreter_lock, 1); + take_gil(tstate); errno = err; } #endif @@ -399,7 +454,6 @@ static struct { } pendingcalls[NPENDINGCALLS]; static int pendingfirst = 0; static int pendinglast = 0; -static volatile int pendingcalls_to_do = 1; /* trigger initialization of lock */ static char pendingbusy = 0; int @@ -438,8 +492,7 @@ Py_AddPendingCall(int (*func)(void *), void *arg) pendinglast = j; } /* signal main loop */ - _Py_Ticker = 0; - pendingcalls_to_do = 1; + SIGNAL_PENDING_CALLS(); if (lock != NULL) PyThread_release_lock(lock); return result; @@ -481,7 +534,10 @@ Py_MakePendingCalls(void) arg = pendingcalls[j].arg; pendingfirst = (j + 1) % NPENDINGCALLS; } - pendingcalls_to_do = pendingfirst != pendinglast; + if (pendingfirst != pendinglast) + SIGNAL_PENDING_CALLS(); + else + UNSIGNAL_PENDING_CALLS(); PyThread_release_lock(pending_lock); /* having released the lock, perform the callback */ if (func == NULL) @@ -547,8 +603,7 @@ Py_AddPendingCall(int (*func)(void *), void *arg) pendingcalls[i].arg = arg; pendinglast = j; - _Py_Ticker = 0; - pendingcalls_to_do = 1; /* Signal main loop */ + SIGNAL_PENDING_CALLS(); busy = 0; /* XXX End critical section */ return 0; @@ -561,7 +616,7 @@ Py_MakePendingCalls(void) if (busy) return 0; busy = 1; - pendingcalls_to_do = 0; + UNSIGNAL_PENDING_CALLS(); for (;;) { int i; int (*func)(void *); @@ -574,7 +629,7 @@ Py_MakePendingCalls(void) pendingfirst = (i + 1) % NPENDINGCALLS; if (func(arg) < 0) { busy = 0; - pendingcalls_to_do = 1; /* We're not done yet */ + SIGNAL_PENDING_CALLS(); /* We're not done yet */ return -1; } } @@ -655,10 +710,7 @@ static int unpack_iterable(PyObject *, int, PyObject **); fast_next_opcode*/ static int _Py_TracingPossible = 0; -/* for manipulating the thread switch and periodic "stuff" - used to be - per thread, now just a pair o' globals */ -int _Py_CheckInterval = 100; -volatile int _Py_Ticker = 0; /* so that we hit a "tick" first thing */ + PyObject * PyEval_EvalCode(PyCodeObject *co, PyObject *globals, PyObject *locals) @@ -983,13 +1035,12 @@ PyEval_EvalFrameEx(PyFrameObject *f, int throwflag) async I/O handler); see Py_AddPendingCall() and Py_MakePendingCalls() above. */ - if (--_Py_Ticker < 0) { + if (eval_breaker) { if (*next_instr == SETUP_FINALLY) { /* Make the last opcode before a try: finally: block uninterruptable. */ goto fast_next_opcode; } - _Py_Ticker = _Py_CheckInterval; tstate->tick_counter++; #ifdef WITH_TSC ticked = 1; @@ -999,39 +1050,31 @@ PyEval_EvalFrameEx(PyFrameObject *f, int throwflag) why = WHY_EXCEPTION; goto on_error; } - if (pendingcalls_to_do) - /* MakePendingCalls() didn't succeed. - Force early re-execution of this - "periodic" code, possibly after - a thread switch */ - _Py_Ticker = 0; } + if (gil_drop_request) { #ifdef WITH_THREAD - if (interpreter_lock) { /* Give another thread a chance */ - if (PyThreadState_Swap(NULL) != tstate) Py_FatalError("ceval: tstate mix-up"); - PyThread_release_lock(interpreter_lock); + drop_gil(tstate); /* Other threads may run now */ - PyThread_acquire_lock(interpreter_lock, 1); + take_gil(tstate); if (PyThreadState_Swap(tstate) != NULL) Py_FatalError("ceval: orphan tstate"); - - /* Check for thread interrupts */ - - if (tstate->async_exc != NULL) { - x = tstate->async_exc; - tstate->async_exc = NULL; - PyErr_SetNone(x); - Py_DECREF(x); - why = WHY_EXCEPTION; - goto on_error; - } - } #endif + } + /* Check for asynchronous exceptions. */ + if (tstate->async_exc != NULL) { + x = tstate->async_exc; + tstate->async_exc = NULL; + UNSIGNAL_ASYNC_EXC(); + PyErr_SetNone(x); + Py_DECREF(x); + why = WHY_EXCEPTION; + goto on_error; + } } fast_next_opcode: diff --git a/Python/ceval_gil.h b/Python/ceval_gil.h new file mode 100644 index 0000000..2687f95 --- /dev/null +++ b/Python/ceval_gil.h @@ -0,0 +1,335 @@ +/* + * Implementation of the Global Interpreter Lock (GIL). + */ + +#include +#include + + +/* First some general settings */ + +/* microseconds (the Python API uses seconds, though) */ +#define DEFAULT_INTERVAL 5000 +static unsigned long gil_interval = DEFAULT_INTERVAL; +#define INTERVAL (gil_interval >= 1 ? gil_interval : 1) + +/* Enable if you want to force the switching of threads at least every `gil_interval` */ +#undef FORCE_SWITCHING +#define FORCE_SWITCHING + + +/* + Notes about the implementation: + + - The GIL is just a boolean variable (gil_locked) whose access is protected + by a mutex (gil_mutex), and whose changes are signalled by a condition + variable (gil_cond). gil_mutex is taken for short periods of time, + and therefore mostly uncontended. + + - In the GIL-holding thread, the main loop (PyEval_EvalFrameEx) must be + able to release the GIL on demand by another thread. A volatile boolean + variable (gil_drop_request) is used for that purpose, which is checked + at every turn of the eval loop. That variable is set after a wait of + `interval` microseconds on `gil_cond` has timed out. + + [Actually, another volatile boolean variable (eval_breaker) is used + which ORs several conditions into one. Volatile booleans are + sufficient as inter-thread signalling means since Python is run + on cache-coherent architectures only.] + + - A thread wanting to take the GIL will first let pass a given amount of + time (`interval` microseconds) before setting gil_drop_request. This + encourages a defined switching period, but doesn't enforce it since + opcodes can take an arbitrary time to execute. + + The `interval` value is available for the user to read and modify + using the Python API `sys.{get,set}switchinterval()`. + + - When a thread releases the GIL and gil_drop_request is set, that thread + ensures that another GIL-awaiting thread gets scheduled. + It does so by waiting on a condition variable (switch_cond) until + the value of gil_last_holder is changed to something else than its + own thread state pointer, indicating that another thread was able to + take the GIL. + + This is meant to prohibit the latency-adverse behaviour on multi-core + machines where one thread would speculatively release the GIL, but still + run and end up being the first to re-acquire it, making the "timeslices" + much longer than expected. + (Note: this mechanism is enabled with FORCE_SWITCHING above) +*/ + +#ifndef _POSIX_THREADS +/* This means pthreads are not implemented in libc headers, hence the macro + not present in unistd.h. But they still can be implemented as an external + library (e.g. gnu pth in pthread emulation) */ +# ifdef HAVE_PTHREAD_H +# include /* _POSIX_THREADS */ +# endif +#endif + + +#ifdef _POSIX_THREADS + +/* + * POSIX support + */ + +#include + +#define ADD_MICROSECONDS(tv, interval) \ +do { \ + tv.tv_usec += (long) interval; \ + tv.tv_sec += tv.tv_usec / 1000000; \ + tv.tv_usec %= 1000000; \ +} while (0) + +/* We assume all modern POSIX systems have gettimeofday() */ +#ifdef GETTIMEOFDAY_NO_TZ +#define GETTIMEOFDAY(ptv) gettimeofday(ptv) +#else +#define GETTIMEOFDAY(ptv) gettimeofday(ptv, (struct timezone *)NULL) +#endif + +#define MUTEX_T pthread_mutex_t +#define MUTEX_INIT(mut) \ + if (pthread_mutex_init(&mut, NULL)) { \ + Py_FatalError("pthread_mutex_init(" #mut ") failed"); }; +#define MUTEX_LOCK(mut) \ + if (pthread_mutex_lock(&mut)) { \ + Py_FatalError("pthread_mutex_lock(" #mut ") failed"); }; +#define MUTEX_UNLOCK(mut) \ + if (pthread_mutex_unlock(&mut)) { \ + Py_FatalError("pthread_mutex_unlock(" #mut ") failed"); }; + +#define COND_T pthread_cond_t +#define COND_INIT(cond) \ + if (pthread_cond_init(&cond, NULL)) { \ + Py_FatalError("pthread_cond_init(" #cond ") failed"); }; +#define COND_PREPARE(cond) +#define COND_SIGNAL(cond) \ + if (pthread_cond_signal(&cond)) { \ + Py_FatalError("pthread_cond_signal(" #cond ") failed"); }; +#define COND_WAIT(cond, mut) \ + if (pthread_cond_wait(&cond, &mut)) { \ + Py_FatalError("pthread_cond_wait(" #cond ") failed"); }; +#define COND_TIMED_WAIT(cond, mut, microseconds, timeout_result) \ + { \ + int r; \ + struct timespec ts; \ + struct timeval deadline; \ + \ + GETTIMEOFDAY(&deadline); \ + ADD_MICROSECONDS(deadline, microseconds); \ + ts.tv_sec = deadline.tv_sec; \ + ts.tv_nsec = deadline.tv_usec * 1000; \ + \ + r = pthread_cond_timedwait(&cond, &mut, &ts); \ + if (r == ETIMEDOUT) \ + timeout_result = 1; \ + else if (r) \ + Py_FatalError("pthread_cond_timedwait(" #cond ") failed"); \ + else \ + timeout_result = 0; \ + } \ + +#elif defined(NT_THREADS) + +/* + * Windows (2000 and later, as well as (hopefully) CE) support + */ + +#include + +#define MUTEX_T HANDLE +#define MUTEX_INIT(mut) \ + if (!(mut = CreateMutex(NULL, FALSE, NULL))) { \ + Py_FatalError("CreateMutex(" #mut ") failed"); }; +#define MUTEX_LOCK(mut) \ + if (WaitForSingleObject(mut, INFINITE) != WAIT_OBJECT_0) { \ + Py_FatalError("WaitForSingleObject(" #mut ") failed"); }; +#define MUTEX_UNLOCK(mut) \ + if (!ReleaseMutex(mut)) { \ + Py_FatalError("ReleaseMutex(" #mut ") failed"); }; + +/* We emulate condition variables with events. It is sufficient here. + (WaitForMultipleObjects() allows the event to be caught and the mutex + to be taken atomically) */ +#define COND_T HANDLE +#define COND_INIT(cond) \ + /* auto-reset, non-signalled */ \ + if (!(cond = CreateEvent(NULL, FALSE, FALSE, NULL))) { \ + Py_FatalError("CreateMutex(" #cond ") failed"); }; +#define COND_PREPARE(cond) \ + if (!ResetEvent(cond)) { \ + Py_FatalError("ResetEvent(" #cond ") failed"); }; +#define COND_SIGNAL(cond) \ + if (!SetEvent(cond)) { \ + Py_FatalError("SetEvent(" #cond ") failed"); }; +#define COND_WAIT(cond, mut) \ + { \ + DWORD r; \ + HANDLE objects[2] = { cond, mut }; \ + MUTEX_UNLOCK(mut); \ + r = WaitForMultipleObjects(2, objects, TRUE, INFINITE); \ + if (r != WAIT_OBJECT_0) \ + Py_FatalError("WaitForSingleObject(" #cond ") failed"); \ + } +#define COND_TIMED_WAIT(cond, mut, microseconds, timeout_result) \ + { \ + DWORD r; \ + HANDLE objects[2] = { cond, mut }; \ + MUTEX_UNLOCK(mut); \ + r = WaitForMultipleObjects(2, objects, TRUE, microseconds / 1000); \ + if (r == WAIT_TIMEOUT) { \ + MUTEX_LOCK(mut); \ + timeout_result = 1; \ + } \ + else if (r != WAIT_OBJECT_0) \ + Py_FatalError("WaitForSingleObject(" #cond ") failed"); \ + else \ + timeout_result = 0; \ + } + +#else + +#error You need either a POSIX-compatible or a Windows system! + +#endif /* _POSIX_THREADS, NT_THREADS */ + + +/* Whether the GIL is already taken (-1 if uninitialized). This is volatile + because it can be read without any lock taken in ceval.c. */ +static volatile int gil_locked = -1; +/* Number of GIL switches since the beginning. */ +static unsigned long gil_switch_number = 0; +/* Last thread holding / having held the GIL. This helps us know whether + anyone else was scheduled after we dropped the GIL. */ +static PyThreadState *gil_last_holder = NULL; + +/* This condition variable allows one or several threads to wait until + the GIL is released. In addition, the mutex also protects the above + variables. */ +static COND_T gil_cond; +static MUTEX_T gil_mutex; + +#ifdef FORCE_SWITCHING +/* This condition variable helps the GIL-releasing thread wait for + a GIL-awaiting thread to be scheduled and take the GIL. */ +static COND_T switch_cond; +static MUTEX_T switch_mutex; +#endif + + +static int gil_created(void) +{ + return gil_locked >= 0; +} + +static void create_gil(void) +{ + MUTEX_INIT(gil_mutex); +#ifdef FORCE_SWITCHING + MUTEX_INIT(switch_mutex); +#endif + COND_INIT(gil_cond); +#ifdef FORCE_SWITCHING + COND_INIT(switch_cond); +#endif + gil_locked = 0; + gil_last_holder = NULL; +} + +static void recreate_gil(void) +{ + create_gil(); +} + +static void drop_gil(PyThreadState *tstate) +{ + /* NOTE: tstate is allowed to be NULL. */ + if (!gil_locked) + Py_FatalError("drop_gil: GIL is not locked"); + if (tstate != NULL && tstate != gil_last_holder) + Py_FatalError("drop_gil: wrong thread state"); + + MUTEX_LOCK(gil_mutex); + gil_locked = 0; + COND_SIGNAL(gil_cond); +#ifdef FORCE_SWITCHING + COND_PREPARE(switch_cond); +#endif + MUTEX_UNLOCK(gil_mutex); + +#ifdef FORCE_SWITCHING + if (gil_drop_request) { + MUTEX_LOCK(switch_mutex); + /* Not switched yet => wait */ + if (gil_last_holder == tstate) + COND_WAIT(switch_cond, switch_mutex); + MUTEX_UNLOCK(switch_mutex); + } +#endif +} + +static void take_gil(PyThreadState *tstate) +{ + int err; + if (tstate == NULL) + Py_FatalError("take_gil: NULL tstate"); + + err = errno; + MUTEX_LOCK(gil_mutex); + + if (!gil_locked) + goto _ready; + + COND_PREPARE(gil_cond); + while (gil_locked) { + int timed_out = 0; + unsigned long saved_switchnum; + + saved_switchnum = gil_switch_number; + COND_TIMED_WAIT(gil_cond, gil_mutex, INTERVAL, timed_out); + /* If we timed out and no switch occurred in the meantime, it is time + to ask the GIL-holding thread to drop it. */ + if (timed_out && gil_locked && gil_switch_number == saved_switchnum) { + SET_GIL_DROP_REQUEST(); + } + } +_ready: +#ifdef FORCE_SWITCHING + /* This mutex must be taken before modifying gil_last_holder (see drop_gil()). */ + MUTEX_LOCK(switch_mutex); +#endif + /* We now hold the GIL */ + gil_locked = 1; + + if (tstate != gil_last_holder) { + gil_last_holder = tstate; + ++gil_switch_number; + } +#ifdef FORCE_SWITCHING + COND_SIGNAL(switch_cond); + MUTEX_UNLOCK(switch_mutex); +#endif + if (gil_drop_request) { + RESET_GIL_DROP_REQUEST(); + } + if (tstate->async_exc != NULL) { + _PyEval_SignalAsyncExc(); + } + + MUTEX_UNLOCK(gil_mutex); + errno = err; +} + +void _PyEval_SetSwitchInterval(unsigned long microseconds) +{ + gil_interval = microseconds; +} + +unsigned long _PyEval_GetSwitchInterval() +{ + return gil_interval; +} diff --git a/Python/pystate.c b/Python/pystate.c index da417c1..6e597d2 100644 --- a/Python/pystate.c +++ b/Python/pystate.c @@ -395,6 +395,7 @@ PyThreadState_SetAsyncExc(long id, PyObject *exc) { p->async_exc = exc; HEAD_UNLOCK(); Py_XDECREF(old_exc); + _PyEval_SignalAsyncExc(); return 1; } } diff --git a/Python/sysmodule.c b/Python/sysmodule.c index c9bc4a3..184de52 100644 --- a/Python/sysmodule.c +++ b/Python/sysmodule.c @@ -461,10 +461,18 @@ Return the profiling function set with sys.setprofile.\n\ See the profiler chapter in the library manual." ); +/* TODO: deprecate */ +static int _check_interval = 100; + static PyObject * sys_setcheckinterval(PyObject *self, PyObject *args) { - if (!PyArg_ParseTuple(args, "i:setcheckinterval", &_Py_CheckInterval)) + if (PyErr_WarnEx(PyExc_DeprecationWarning, + "sys.getcheckinterval() and sys.setcheckinterval() " + "are deprecated. Use sys.setswitchinterval() " + "instead.", 1) < 0) + return NULL; + if (!PyArg_ParseTuple(args, "i:setcheckinterval", &_check_interval)) return NULL; Py_INCREF(Py_None); return Py_None; @@ -480,13 +488,59 @@ n instructions. This also affects how often thread switches occur." static PyObject * sys_getcheckinterval(PyObject *self, PyObject *args) { - return PyInt_FromLong(_Py_CheckInterval); + if (PyErr_WarnEx(PyExc_DeprecationWarning, + "sys.getcheckinterval() and sys.setcheckinterval() " + "are deprecated. Use sys.getswitchinterval() " + "instead.", 1) < 0) + return NULL; + return PyLong_FromLong(_check_interval); } PyDoc_STRVAR(getcheckinterval_doc, "getcheckinterval() -> current check interval; see setcheckinterval()." ); +#ifdef WITH_THREAD +static PyObject * +sys_setswitchinterval(PyObject *self, PyObject *args) +{ + double d; + if (!PyArg_ParseTuple(args, "d:setswitchinterval", &d)) + return NULL; + if (d <= 0.0) { + PyErr_SetString(PyExc_ValueError, + "switch interval must be strictly positive"); + return NULL; + } + _PyEval_SetSwitchInterval((unsigned long) (1e6 * d)); + Py_INCREF(Py_None); + return Py_None; +} + +PyDoc_STRVAR(setswitchinterval_doc, +"setswitchinterval(n)\n\ +\n\ +Set the ideal thread switching delay inside the Python interpreter\n\ +The actual frequency of switching threads can be lower if the\n\ +interpreter executes long sequences of uninterruptible code\n\ +(this is implementation-specific and workload-dependent).\n\ +\n\ +The parameter must represent the desired switching delay in seconds\n\ +A typical value is 0.005 (5 milliseconds)." +); + +static PyObject * +sys_getswitchinterval(PyObject *self, PyObject *args) +{ + return PyFloat_FromDouble(1e-6 * _PyEval_GetSwitchInterval()); +} + +PyDoc_STRVAR(getswitchinterval_doc, +"getswitchinterval() -> current thread switch interval; see setswitchinterval()." +); + +#endif /* WITH_THREAD */ + #ifdef WITH_TSC static PyObject * sys_settscdump(PyObject *self, PyObject *args) @@ -917,6 +971,12 @@ static PyMethodDef sys_methods[] = { setcheckinterval_doc}, {"getcheckinterval", sys_getcheckinterval, METH_NOARGS, getcheckinterval_doc}, +#ifdef WITH_THREAD + {"setswitchinterval", sys_setswitchinterval, METH_VARARGS, + setswitchinterval_doc}, + {"getswitchinterval", sys_getswitchinterval, METH_NOARGS, + getswitchinterval_doc}, +#endif #ifdef HAVE_DLOPEN {"setdlopenflags", sys_setdlopenflags, METH_VARARGS, setdlopenflags_doc}, -- 1.6.5 From 821f3cb81cd995a91c44f74b9d862645a79c5da9 Mon Sep 17 00:00:00 2001 From: Neil Schemenauer Date: Fri, 22 Jan 2010 09:10:09 -0600 Subject: [PATCH 2/3] Remove obsolete comment. --- Python/sysmodule.c | 1 - 1 files changed, 0 insertions(+), 1 deletions(-) diff --git a/Python/sysmodule.c b/Python/sysmodule.c index 184de52..2089169 100644 --- a/Python/sysmodule.c +++ b/Python/sysmodule.c @@ -461,7 +461,6 @@ Return the profiling function set with sys.setprofile.\n\ See the profiler chapter in the library manual." ); -/* TODO: deprecate */ static int _check_interval = 100; static PyObject * -- 1.6.5 From 189446bbd4fcf46eb9e5f927a6158e91c5251d18 Mon Sep 17 00:00:00 2001 From: Neil Schemenauer Date: Fri, 22 Jan 2010 09:16:54 -0600 Subject: [PATCH 3/3] Add a couple of words about the new GIL implementation --- Doc/library/sys.rst | 28 ++++++++++++++++++++++++++++ Doc/whatsnew/2.7.rst | 23 +++++++++++++++++++++++ 2 files changed, 51 insertions(+), 0 deletions(-) diff --git a/Doc/library/sys.rst b/Doc/library/sys.rst index 5b07f36..4c23568 100644 --- a/Doc/library/sys.rst +++ b/Doc/library/sys.rst @@ -355,6 +355,9 @@ always available. .. versionadded:: 2.3 + .. deprecated:: 2.7 + Use :func:`getswitchinterval` instead. + .. function:: getdefaultencoding() @@ -426,6 +429,14 @@ always available. .. versionadded:: 2.6 +.. function:: getswitchinterval() + + Return the interpreter's "thread switch interval"; see + :func:`setswitchinterval`. + + .. versionadded:: 2.7 + + .. function:: _getframe([depth]) Return a frame object from the call stack. If optional integer *depth* is @@ -726,6 +737,11 @@ always available. performance for programs using threads. Setting it to a value ``<=`` 0 checks every virtual instruction, maximizing responsiveness as well as overhead. + .. deprecated:: 2.7 + This function doesn't have an effect anymore, as the internal logic + for thread switching and asynchronous tasks has been rewritten. + Use :func:`setswitchinterval` instead. + .. function:: setdefaultencoding(name) @@ -785,6 +801,18 @@ always available. limit can lead to a crash. +.. function:: setswitchinterval(interval) + + Set the interpreter's thread switch interval (in seconds). This floating-point + value determines the ideal duration of the "timeslices" allocated to + concurrently running Python threads. Please note that the actual value + can be higher, especially if long-running internal functions or methods + are used. Also, which thread becomes scheduled at the end of the interval + is the operating system's decision. The interpreter doesn't have its + own scheduler. + + .. versionadded:: 2.7 + .. function:: settrace(tracefunc) .. index:: diff --git a/Doc/whatsnew/2.7.rst b/Doc/whatsnew/2.7.rst index 124f247..69b1c6e 100644 --- a/Doc/whatsnew/2.7.rst +++ b/Doc/whatsnew/2.7.rst @@ -474,6 +474,29 @@ Several performance enhancements have been added: .. ====================================================================== +Multi-threading +=============== + +* The mechanism for serializing execution of concurrently running Python + threads (generally known as the GIL or Global Interpreter Lock) has been + rewritten. Among the objectives were more predictable switching intervals + and reduced overhead due to lock contention and the number of ensuing + system calls. The notion of a "check interval" to allow thread switches + has been abandoned and replaced by an absolute duration expressed in + seconds. This parameter is tunable through :func:`sys.setswitchinterval()`. + It currently defaults to 5 milliseconds. + + Additional details about the implementation can be read from a `python-dev + mailing-list message + `_ + (however, "priority requests" as exposed in this message have not been + kept for inclusion). + + (Contributed by Antoine Pitrou) + + +.. ====================================================================== + New and Improved Modules ======================== -- 1.6.5