Revise the locking API: deprecate the old locking callbacks and add trylock.

Previously, there was no good way to request different kinds of lock
(say, read/write vs writeonly or recursive vs nonrecursive), or for a
lock function to signal failure (which would be important for a
trylock mode).

This patch revises the lock API to be a bit more useful.  The older
lock calls are still supported for now.

We also add a debugging mode to catch common errors in using the
locking APIs.
This commit is contained in:
Nick Mathewson 2009-11-27 15:20:43 -05:00
parent e1ffbb82e3
commit 347952ffe0
12 changed files with 402 additions and 112 deletions

View File

@ -107,7 +107,7 @@ event-config.h: config.h
-e 's/#ifndef /#ifndef _EVENT_/' < config.h >> $@
echo "#endif" >> $@
CORE_SRC = event.c buffer.c \
CORE_SRC = event.c evthread.c buffer.c \
bufferevent.c bufferevent_sock.c bufferevent_filter.c \
bufferevent_pair.c listener.c \
evmap.c log.c evutil.c strlcpy.c $(SYS_SRC)

View File

@ -311,7 +311,7 @@ evbuffer_enable_locking(struct evbuffer *buf, void *lock)
return -1;
if (!lock) {
EVTHREAD_ALLOC_LOCK(lock);
EVTHREAD_ALLOC_LOCK(lock, EVTHREAD_LOCKTYPE_RECURSIVE);
if (!lock)
return -1;
buf->lock = lock;
@ -441,7 +441,7 @@ _evbuffer_decref_and_unlock(struct evbuffer *buffer)
EVBUFFER_UNLOCK(buffer, EVTHREAD_WRITE);
if (buffer->own_lock)
EVTHREAD_FREE_LOCK(buffer->lock);
EVTHREAD_FREE_LOCK(buffer->lock, EVTHREAD_LOCKTYPE_RECURSIVE);
mm_free(buffer);
}

View File

@ -508,7 +508,8 @@ _bufferevent_decref_and_unlock(struct bufferevent *bufev)
BEV_UNLOCK(bufev);
if (bufev_private->own_lock)
EVTHREAD_FREE_LOCK(bufev_private->lock);
EVTHREAD_FREE_LOCK(bufev_private->lock,
EVTHREAD_LOCKTYPE_RECURSIVE);
/* Free the actual allocated memory. */
mm_free(bufev - bufev->be_ops->mem_offset);
@ -549,7 +550,7 @@ bufferevent_enable_locking(struct bufferevent *bufev, void *lock)
BEV_UPCAST(bufev)->lock = lock;
BEV_UPCAST(bufev)->own_lock = 0;
} else if (!lock) {
EVTHREAD_ALLOC_LOCK(lock);
EVTHREAD_ALLOC_LOCK(lock, EVTHREAD_LOCKTYPE_RECURSIVE);
if (!lock)
return -1;
BEV_UPCAST(bufev)->lock = lock;

View File

@ -88,15 +88,13 @@ void event_deferred_cb_schedule(struct deferred_cb_queue *, struct deferred_cb *
#define LOCK_DEFERRED_QUEUE(q) \
do { \
if ((q)->lock) \
_evthread_locking_fn(EVTHREAD_LOCK|EVTHREAD_WRITE, \
(q)->lock); \
_evthread_lock_fns.lock(0, (q)->lock); \
} while (0)
#define UNLOCK_DEFERRED_QUEUE(q) \
do { \
if ((q)->lock) \
_evthread_locking_fn(EVTHREAD_UNLOCK|EVTHREAD_WRITE, \
(q)->lock); \
_evthread_lock_fns.unlock(0, (q)->lock); \
} while (0)
#endif

View File

@ -1688,7 +1688,7 @@ evdns_add_server_port_with_base(struct event_base *base, evutil_socket_t socket,
mm_free(port);
return NULL;
}
EVTHREAD_ALLOC_LOCK(port->lock);
EVTHREAD_ALLOC_LOCK(port->lock, EVTHREAD_LOCKTYPE_RECURSIVE);
return port;
}
@ -2082,7 +2082,7 @@ server_port_free(struct evdns_server_port *port)
port->socket = -1;
}
(void) event_del(&port->event);
EVTHREAD_FREE_LOCK(port->lock);
EVTHREAD_FREE_LOCK(port->lock, EVTHREAD_LOCKTYPE_RECURSIVE);
mm_free(port);
}
@ -3654,7 +3654,7 @@ evdns_base_new(struct event_base *event_base, int initialize_nameservers)
memset(base, 0, sizeof(struct evdns_base));
base->req_waiting_head = NULL;
EVTHREAD_ALLOC_LOCK(base->lock);
EVTHREAD_ALLOC_LOCK(base->lock, EVTHREAD_LOCKTYPE_RECURSIVE);
EVDNS_LOCK(base);
/* Set max requests inflight and allocate req_heads. */
@ -3773,7 +3773,7 @@ evdns_base_free_and_unlock(struct evdns_base *base, int fail_requests)
base->global_search_state = NULL;
}
EVDNS_UNLOCK(base);
EVTHREAD_FREE_LOCK(base->lock);
EVTHREAD_FREE_LOCK(base->lock, EVTHREAD_LOCKTYPE_RECURSIVE);
mm_free(base);
}

43
event.c
View File

@ -352,9 +352,11 @@ event_base_new_with_config(struct event_config *cfg)
#ifndef _EVENT_DISABLE_THREAD_SUPPORT
if (!cfg || !(cfg->flags & EVENT_BASE_FLAG_NOLOCK)) {
int r;
EVTHREAD_ALLOC_LOCK(base->th_base_lock);
EVTHREAD_ALLOC_LOCK(base->th_base_lock,
EVTHREAD_LOCKTYPE_RECURSIVE);
base->defer_queue.lock = base->th_base_lock;
EVTHREAD_ALLOC_LOCK(base->current_event_lock);
EVTHREAD_ALLOC_LOCK(base->current_event_lock,
EVTHREAD_LOCKTYPE_RECURSIVE);
r = evthread_make_base_notifiable(base);
if (r<0) {
event_base_free(base);
@ -475,8 +477,9 @@ event_base_free(struct event_base *base)
evmap_io_clear(&base->io);
evmap_signal_clear(&base->sigmap);
EVTHREAD_FREE_LOCK(base->th_base_lock);
EVTHREAD_FREE_LOCK(base->current_event_lock);
EVTHREAD_FREE_LOCK(base->th_base_lock, EVTHREAD_LOCKTYPE_RECURSIVE);
EVTHREAD_FREE_LOCK(base->current_event_lock,
EVTHREAD_LOCKTYPE_RECURSIVE);
mm_free(base);
}
@ -2083,20 +2086,6 @@ event_set_mem_functions(void *(*malloc_fn)(size_t sz),
}
#endif
#ifndef _EVENT_DISABLE_THREAD_SUPPORT
/* support for threading */
void (*_evthread_locking_fn)(int mode, void *lock) = NULL;
unsigned long (*_evthread_id_fn)(void) = NULL;
void *(*_evthread_lock_alloc_fn)(void) = NULL;
void (*_evthread_lock_free_fn)(void *) = NULL;
void
evthread_set_locking_callback(void (*locking_fn)(int mode, void *lock))
{
_evthread_locking_fn = locking_fn;
}
#endif
#if defined(_EVENT_HAVE_EVENTFD) && defined(_EVENT_HAVE_SYS_EVENTFD_H)
static void
evthread_notify_drain_eventfd(int fd, short what, void *arg)
@ -2120,14 +2109,6 @@ evthread_notify_drain_default(evutil_socket_t fd, short what, void *arg)
#endif
}
#ifndef _EVENT_DISABLE_THREAD_SUPPORT
void
evthread_set_id_callback(unsigned long (*id_fn)(void))
{
_evthread_id_fn = id_fn;
}
#endif
int
evthread_make_base_notifiable(struct event_base *base)
{
@ -2191,16 +2172,6 @@ evthread_make_base_notifiable(struct event_base *base)
return event_add(&base->th_notify, NULL);
}
#ifndef _EVENT_DISABLE_THREAD_SUPPORT
void
evthread_set_lock_create_callbacks(void *(*alloc_fn)(void),
void (*free_fn)(void *))
{
_evthread_lock_alloc_fn = alloc_fn;
_evthread_lock_free_fn = free_fn;
}
#endif
void
event_base_dump_events(struct event_base *base, FILE *output)
{

View File

@ -38,10 +38,8 @@ struct event_base;
#ifndef _EVENT_DISABLE_THREAD_SUPPORT
/* Global function pointers to lock-related functions. NULL if locking isn't
enabled. */
extern void (*_evthread_locking_fn)(int mode, void *lock);
extern struct evthread_lock_callbacks _evthread_lock_fns;
extern unsigned long (*_evthread_id_fn)(void);
extern void *(*_evthread_lock_alloc_fn)(void);
extern void (*_evthread_lock_free_fn)(void *);
/** True iff the given event_base is set up to use locking */
#define EVBASE_USING_LOCKS(base) \
@ -59,29 +57,30 @@ extern void (*_evthread_lock_free_fn)(void *);
/** Allocate a new lock, and store it in lockvar, a void*. Sets lockvar to
NULL if locking is not enabled. */
#define EVTHREAD_ALLOC_LOCK(lockvar) \
((lockvar) = _evthread_lock_alloc_fn ? \
_evthread_lock_alloc_fn() : NULL)
#define EVTHREAD_ALLOC_LOCK(lockvar, locktype) \
((lockvar) = _evthread_lock_fns.alloc ? \
_evthread_lock_fns.alloc(locktype) : NULL)
/** Free a given lock, if it is present and locking is enabled. */
#define EVTHREAD_FREE_LOCK(lockvar) \
do { \
if (lockvar && _evthread_lock_free_fn) \
_evthread_lock_free_fn(lockvar); \
#define EVTHREAD_FREE_LOCK(lockvar, locktype) \
do { \
void *_lock_tmp_ = (lockvar); \
if (_lock_tmp_ && _evthread_lock_fns.free) \
_evthread_lock_fns.free(_lock_tmp_, (locktype)); \
} while (0)
/** Acquire a lock. */
#define EVLOCK_LOCK(lock,mode) \
#define EVLOCK_LOCK(lockvar,mode) \
do { \
if (lock) \
_evthread_locking_fn(EVTHREAD_LOCK|mode, lock); \
if (lockvar) \
_evthread_lock_fns.lock(mode, lockvar); \
} while (0)
/** Release a lock */
#define EVLOCK_UNLOCK(lock,mode) \
#define EVLOCK_UNLOCK(lockvar,mode) \
do { \
if (lock) \
_evthread_locking_fn(EVTHREAD_UNLOCK|mode, lock); \
if (lockvar) \
_evthread_lock_fns.unlock(mode, lockvar); \
} while (0)
/** Helper: put lockvar1 and lockvar2 into pointerwise ascending order. */
@ -119,24 +118,22 @@ extern void (*_evthread_lock_free_fn)(void *);
/** Lock an event_base, if it is set up for locking. Acquires the lock
in the base structure whose field is named 'lock'. */
#define EVBASE_ACQUIRE_LOCK(base, mode, lock) do { \
in the base structure whose field is named 'lck'. */
#define EVBASE_ACQUIRE_LOCK(base, mode, lockvar) do { \
if (EVBASE_USING_LOCKS(base)) \
_evthread_locking_fn(EVTHREAD_LOCK | mode, \
(base)->lock); \
_evthread_lock_fns.lock(mode, (base)->lockvar); \
} while (0)
/** Unlock an event_base, if it is set up for locking. */
#define EVBASE_RELEASE_LOCK(base, mode, lock) do { \
#define EVBASE_RELEASE_LOCK(base, mode, lockvar) do { \
if (EVBASE_USING_LOCKS(base)) \
_evthread_locking_fn(EVTHREAD_UNLOCK | mode, \
(base)->lock); \
_evthread_lock_fns.unlock(mode, (base)->lockvar); \
} while (0)
#else /* _EVENT_DISABLE_THREAD_SUPPORT */
#define EVTHREAD_GET_ID() 1
#define EVTHREAD_ALLOC_LOCK(lockvar) _EVUTIL_NIL_STMT
#define EVTHREAD_FREE_LOCK(lockvar) _EVUTIL_NIL_STMT
#define EVTHREAD_ALLOC_LOCK(lockvar, locktype) _EVUTIL_NIL_STMT
#define EVTHREAD_FREE_LOCK(lockvar, locktype) _EVUTIL_NIL_STMT
#define EVLOCK_LOCK(lockvar, mode) _EVUTIL_NIL_STMT
#define EVLOCK_UNLOCK(lockvar, mode) _EVUTIL_NIL_STMT

233
evthread.c Normal file
View File

@ -0,0 +1,233 @@
/*
* Copyright (c) 2008-2009 Niels Provos, Nick Mathewson
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* 1. Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* 3. The name of the author may not be used to endorse or promote products
* derived from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
* IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
* OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
* IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
* INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
* NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
* DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
* THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
* THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
*/
#include "event-config.h"
#ifndef _EVENT_DISABLE_THREAD_SUPPORT
#include <event2/thread.h>
#include <stdlib.h>
#include <string.h>
#include "log-internal.h"
#include "mm-internal.h"
#include "util-internal.h"
/* globals */
static int lock_debugging_enabled = 0;
struct evthread_lock_callbacks _evthread_lock_fns = {
0, 0, NULL, NULL, NULL, NULL
};
/* Used for debugging */
static struct evthread_lock_callbacks _original_lock_fns = {
0, 0, NULL, NULL, NULL, NULL
};
unsigned long (*_evthread_id_fn)(void) = NULL;
void
evthread_set_id_callback(unsigned long (*id_fn)(void))
{
_evthread_id_fn = id_fn;
}
int
evthread_set_lock_callbacks(const struct evthread_lock_callbacks *cbs)
{
struct evthread_lock_callbacks *target =
lock_debugging_enabled ? &_original_lock_fns : &_evthread_lock_fns;
if (!cbs) {
memset(target, 0, sizeof(_evthread_lock_fns));
return 0;
}
if (cbs->alloc && cbs->free && cbs->lock && cbs->unlock) {
memcpy(target, cbs, sizeof(_evthread_lock_fns));
return 0;
} else {
return -1;
}
}
#ifndef DISABLE_OBSOLETE_LOCK_API
/* Obsolete: for compatibility only. Remove these before 2.0.x-stable! */
static void (*_obsolete_locking_fn)(int, void *) = NULL;
static void *(*_obsolete_lock_alloc_fn)(void) = NULL;
static void (*_obsolete_lock_free_fn)(void *) = NULL;
static void
api_warn(void)
{
static int warned = 0;
if (!warned) {
warned = 1;
event_warnx("evthread_set_locking_callback and "
"evthread_set_lock_create_callbacks are obsolete; use "
"evthread_set_lock_callbacks instead.");
}
}
static void *
compat_lock_alloc(unsigned locktype)
{
if (_obsolete_lock_alloc_fn)
return _obsolete_lock_alloc_fn();
return NULL;
}
static void
compat_lock_free(void *lock, unsigned locktype)
{
if (_obsolete_lock_free_fn)
_obsolete_lock_free_fn(lock);
}
static int
compat_lock_lock(unsigned mode, void *lock)
{
_obsolete_locking_fn(EVTHREAD_LOCK|EVTHREAD_WRITE, lock);
return 0;
}
static int
compat_lock_unlock(unsigned mode, void *lock)
{
_obsolete_locking_fn(EVTHREAD_UNLOCK|EVTHREAD_WRITE, lock);
return 0;
}
void
evthread_set_locking_callback(void (*locking_fn)(int mode, void *lock))
{
api_warn();
if (locking_fn) {
_evthread_lock_fns.lock = compat_lock_lock;
_evthread_lock_fns.unlock = compat_lock_unlock;
} else {
_evthread_lock_fns.lock = NULL;
_evthread_lock_fns.unlock = NULL;
}
_obsolete_locking_fn = locking_fn;
}
void
evthread_set_lock_create_callbacks(void *(*alloc_fn)(void),
void (*free_fn)(void *))
{
api_warn();
_obsolete_lock_alloc_fn = alloc_fn;
_obsolete_lock_free_fn = free_fn;
_evthread_lock_fns.alloc = alloc_fn ? compat_lock_alloc : NULL;
_evthread_lock_fns.free = free_fn ? compat_lock_free : NULL;
}
#endif
struct debug_lock {
unsigned locktype;
int count;
void *lock;
};
static void *
debug_lock_alloc(unsigned locktype)
{
struct debug_lock *result = mm_malloc(sizeof(struct debug_lock));
if (!result)
return NULL;
if (_original_lock_fns.alloc) {
if (!(result->lock = _original_lock_fns.alloc(
locktype|EVTHREAD_LOCKTYPE_RECURSIVE))) {
mm_free(result);
return NULL;
}
}
result->locktype = locktype;
result->count = 0;
return result;
}
static void
debug_lock_free(void *lock_, unsigned locktype)
{
struct debug_lock *lock = lock_;
EVUTIL_ASSERT(lock->count == 0);
if (_original_lock_fns.free) {
_original_lock_fns.free(lock->lock,
lock->locktype|EVTHREAD_LOCKTYPE_RECURSIVE);
}
lock->lock = NULL;
lock->count = -100;
mm_free(lock);
}
static int
debug_lock_lock(unsigned mode, void *lock_)
{
struct debug_lock *lock = lock_;
int res = 0;
if (_original_lock_fns.lock)
res = _original_lock_fns.lock(mode, lock->lock);
if (!res) {
++lock->count;
if (!(lock->locktype & EVTHREAD_LOCKTYPE_RECURSIVE))
EVUTIL_ASSERT(lock->count == 1);
}
return res;
}
static int
debug_lock_unlock(unsigned mode, void *lock_)
{
struct debug_lock *lock = lock_;
int res = 0;
--lock->count;
EVUTIL_ASSERT(lock->count >= 0);
if (_original_lock_fns.unlock)
res = _original_lock_fns.unlock(mode, lock->lock);
return res;
}
void
evthread_enable_lock_debuging(void)
{
struct evthread_lock_callbacks cbs = {
EVTHREAD_LOCK_API_VERSION,
EVTHREAD_LOCKTYPE_RECURSIVE,
debug_lock_alloc,
debug_lock_free,
debug_lock_lock,
debug_lock_unlock
};
memcpy(&_original_lock_fns, &_evthread_lock_fns,
sizeof(struct evthread_lock_callbacks));
memcpy(&_evthread_lock_fns, &cbs,
sizeof(struct evthread_lock_callbacks));
lock_debugging_enabled = 1;
}
#endif

View File

@ -38,12 +38,15 @@ struct event_base;
static pthread_mutexattr_t attr_recursive;
static void *
evthread_posix_lock_create(void)
evthread_posix_lock_alloc(unsigned locktype)
{
pthread_mutexattr_t *attr = NULL;
pthread_mutex_t *lock = mm_malloc(sizeof(pthread_mutex_t));
if (!lock)
return NULL;
if (pthread_mutex_init(lock, &attr_recursive)) {
if (locktype & EVTHREAD_LOCKTYPE_RECURSIVE)
attr = &attr_recursive;
if (pthread_mutex_init(lock, attr)) {
mm_free(lock);
return NULL;
}
@ -51,21 +54,28 @@ evthread_posix_lock_create(void)
}
static void
evthread_posix_lock_free(void *_lock)
evthread_posix_lock_free(void *_lock, unsigned locktype)
{
pthread_mutex_t *lock = _lock;
pthread_mutex_destroy(lock);
mm_free(lock);
}
static void
evthread_posix_lock(int mode, void *_lock)
static int
evthread_posix_lock(unsigned mode, void *_lock)
{
pthread_mutex_t *lock = _lock;
if (0 != (mode & EVTHREAD_LOCK))
pthread_mutex_lock(lock);
if (mode & EVTHREAD_TRY)
return pthread_mutex_trylock(lock);
else
pthread_mutex_unlock(lock);
return pthread_mutex_lock(lock);
}
static int
evthread_posix_unlock(unsigned mode, void *_lock)
{
pthread_mutex_t *lock = _lock;
return pthread_mutex_unlock(lock);
}
static unsigned long
@ -82,16 +92,21 @@ evthread_posix_get_id(void)
int
evthread_use_pthreads(void)
{
struct evthread_lock_callbacks cbs = {
EVTHREAD_LOCK_API_VERSION,
EVTHREAD_LOCKTYPE_RECURSIVE,
evthread_posix_lock_alloc,
evthread_posix_lock_free,
evthread_posix_lock,
evthread_posix_unlock
};
/* Set ourselves up to get recursive locks. */
if (pthread_mutexattr_init(&attr_recursive))
return -1;
if (pthread_mutexattr_settype(&attr_recursive, PTHREAD_MUTEX_RECURSIVE))
return -1;
evthread_set_lock_create_callbacks(
evthread_posix_lock_create,
evthread_posix_lock_free);
evthread_set_locking_callback(evthread_posix_lock);
evthread_set_lock_callbacks(&cbs);
evthread_set_id_callback(evthread_posix_get_id);
return 0;
}

View File

@ -39,7 +39,7 @@ struct event_base;
#include "mm-internal.h"
static void *
evthread_win32_lock_create(void)
evthread_win32_lock_create(unsigned locktype)
{
CRITICAL_SECTION *lock = mm_malloc(sizeof(CRITICAL_SECTION));
if (!lock)
@ -49,20 +49,29 @@ evthread_win32_lock_create(void)
}
static void
evthread_win32_lock_free(void *_lock)
evthread_win32_lock_free(void *_lock, unsigned locktype)
{
CRITICAL_SECTION *lock = _lock;
DeleteCriticalSection(lock);
}
static void
evthread_win32_lock(int mode, void *_lock)
evthread_win32_lock(unsigned mode, void *_lock)
{
CRITICAL_SECTION *lock = _lock;
if (0 != (mode & EVTHREAD_LOCK))
if ((mode & EVTHREAD_TRY)) {
return TryEnterCriticalSection(lock) != 0;
} else {
EnterCriticalSection(lock);
else
LeaveCriticalSection(lock);
return 0;
}
}
static void
evthread_win32_unlock(unsigned mode, void *_lock)
{
CRITICAL_SECTION *lock = _lock;
LeaveCriticalSection(lock)
}
static unsigned long
@ -74,10 +83,16 @@ evthread_win32_get_id(void)
int
evthread_use_windows_threads(void)
{
evthread_set_lock_create_callbacks(
evthread_win32_lock_create,
evthread_win32_lock_free);
evthread_set_locking_callback(evthread_win32_lock);
struct evthread_lock_callbacks cbs = {
EVTHREAD_LOCK_API_VERSION,
EVTHREAD_LOCKTYPE_RECURSIVE,
evthread_win32_lock_alloc,
evthread_win32_lock_free,
evthread_win32_lock,
evthread_win32_unlock
};
evthread_set_lock_callbacks(&cbs);
evthread_set_id_callback(evthread_win32_get_id);
return 0;
}

View File

@ -59,39 +59,89 @@ extern "C" {
#include <event-config.h>
/* combine (lock|unlock) with (read|write) */
/** A flag passed to an obsolete locking callback to indicate that this call
* is for acquiring a lock. */
#define EVTHREAD_LOCK 0x01
/** A flag passed to an obsolete locking callback to indicate that this call
* is for releasing a lock. */
#define EVTHREAD_UNLOCK 0x02
/** A flag passed to a locking callback when the lock was allocated as a
* read-write lock, and we want to acquire or release the lock for writing. */
#define EVTHREAD_WRITE 0x04
/** A flag passed to a locking callback when the lock was allocated as a
* read-write lock, and we want to acquire or release the lock for reading. */
#define EVTHREAD_READ 0x08
/** A flag passed to a locking callback when we don't want to block waiting
* for the lock; if we can't get the lock immediately, we will instead
* return nonzero from the locking callback. */
#define EVTHREAD_TRY 0x10
#ifndef _EVENT_DISABLE_THREAD_SUPPORT
/**
Sets the functions Libevent should use for allocating and freeing
locks. This needs to be called in addition to
evthread_set_locking_callback() before using Libevent in a
multi-threaded application.
Locks must be recursive. That is, it must be safe for a thread to
acquire a lock that it already holds.
@param alloc_fn function to be called when allocating a new lock
@param free_fn function to be called to a free a lock
@deprecated Use evthread_set_lock_callbacks instead. This API will
go away before Libevent 2.0.x-stable.
*/
void evthread_set_lock_create_callbacks(
void *(*alloc_fn)(void), void (*free_fn)(void *));
/**
Sets the function Libevent should use for locking.
@param locking_fn the function that Libevent should invoke to acquire
or release a lock. mode has either EVTHREAD_LOCK or EVTHREAD_UNLOCK
set, and in addition, either EVTHREAD_WRITE or EVTHREAD_READ.
@deprecated Use evthread_set_lock_callbacks instead. This API will
go away before Libevent 2.0.x-stable.
*/
void evthread_set_locking_callback(
void (*locking_fn)(int mode, void *lock));
#define EVTHREAD_LOCK_API_VERSION 1
/** A recursive lock is one that can be acquired multiple times at once by the
* same thread. No other process can allocate the lock until the thread that
* has been holding it has unlocked it as many times as it locked it. */
#define EVTHREAD_LOCKTYPE_RECURSIVE 1
/* A read-write lock is one that allows multiple simultaneous readers, but
* where any one writer excludes all other writers and readers. */
#define EVTHREAD_LOCKTYPE_READWRITE 2
/** This structure describes the interface a threading library uses for
* locking. It's used to tell evthread_set_lock_callbacks how to use
* locking on this platform.
*/
struct evthread_lock_callbacks {
/** The current version of the locking API. Set this to
* EVTHREAD_LOCK_API_VERSION */
int lock_api_version;
/** Which kinds of locks does this version of the locking API
* support? A bitfield of EVTHREAD_LOCKTYPE_RECURSIVE and
* EVTHREAD_LOCKTYPE_READWRITE.
*
* (Note that RECURSIVE locks are currently mandatory, and
* READWRITE locks are not currently used.)
**/
unsigned supported_locktypes;
/** Function to allocate and initialize new lock of type 'locktype'.
* Returns NULL on failure. */
void *(*alloc)(unsigned locktype);
/** Funtion to release all storage held in 'lock', which was created
* with type 'locktype'. */
void (*free)(void *lock, unsigned locktype);
/** Acquire an already-allocated lock at 'lock' with mode 'mode'.
* Returns 0 on success, and nonzero on failure. */
int (*lock)(unsigned mode, void *lock);
/** Release a lock at 'lock' using mode 'mode'. Returns 0 on success,
* and nonzero on failure. */
int (*unlock)(unsigned mode, void *lock);
};
/** Sets a group of functions that Libevent should use for locking.
* For full information on the required callback API, see the
* documentation for the individual members of evthread_lock_callbacks.
*
* Note that if you're using Windows or the Pthreads threading library, you
* probably shouldn't call this function; instead, use
* evthread_use_windos_threads() or evthread_use_posix_threads() if you can.
*/
int evthread_set_lock_callbacks(const struct evthread_lock_callbacks *);
/**
Sets the function for determining the thread id.
@ -119,10 +169,15 @@ int evthread_use_windows_threads(void);
@return 0 on success, -1 on failure. */
int evthread_use_pthreads(void);
#define EVTHREAD_USE_PTHREADS_IMPLEMENTED 1
/* XXXXX */
void evthread_enable_lock_debuging(void);
#endif
#endif /* _EVENT_DISABLE_THREAD_SUPPORT */
struct event_base;
/** Make sure it's safe to tell an event base to wake up from another thread.
or a signal handler.

View File

@ -350,6 +350,11 @@ main(int argc, const char **argv)
tinytest_skip(testgroups, "http/connection_retry");
#endif
#ifndef _EVENT_DISABLE_THREAD_SUPPORT
if (!getenv("EVENT_NO_DEBUG_LOCKS"))
evthread_enable_lock_debuging();
#endif
if (tinytest_main(argc,argv,testgroups))
return 1;