* Private libroot locking primitives:

- Reimplemented mutex to use the _kern_mutex*() syscalls.
  - The initializer functions cannot fail anymore -- changed their return type
    to void.
  - Changed the initializer function semantics to not copy the name by default
    anymore (as in the kernel). Also added *_etc() versions of them that take an
    additional flags.
  - Added static initializer macros.
  - Made the mutex (and thus recursive_lock) lock functions non-interruptable.
  - Got rid of the "lazy" version. They are no longer needed, since the
    initialization of the standard types can be done statically and cannot fail.
* Adjusted libroot, runtime loader, and other code using the private libroot
  locking primitives to the new semantics.
* pthreads mutexes and condition variables:
  - Reimplemented using the _kern_mutex*() syscalls.
  - Consistently use POSIX error codes.
  - Fixed some not quite POSIX compliant behavior.


git-svn-id: file:///srv/svn/repos/haiku/haiku/trunk@36323 a95241bf-73f2-0310-859d-f6bbb57e9c96
This commit is contained in:
Ingo Weinhold 2010-04-15 21:52:49 +00:00
parent 251e72542e
commit f71274580b
19 changed files with 452 additions and 681 deletions

View File

@ -29,18 +29,18 @@ typedef struct _pthread_spinlock *pthread_spinlock_t;
struct _pthread_mutex {
uint32_t flags;
int32_t count;
int32_t sem;
int32_t lock;
int32_t unused;
int32_t owner;
int32_t owner_count;
};
struct _pthread_cond {
uint32_t flags;
int32_t sem;
int32_t unused;
pthread_mutex_t *mutex;
int32_t waiter_count;
int32_t event_counter;
int32_t lock;
};
struct _pthread_once {

View File

@ -13,7 +13,6 @@
extern "C" {
#endif
extern status_t __init_fork(void);
extern status_t __register_atfork(void(*prepare)(void), void(*parent)(void),
void(*child)(void));

View File

@ -30,7 +30,6 @@ status_t __flatten_process_args(const char* const* args, int32 argCount,
const char* const* env, int32 envCount, char*** _flatArgs,
size_t* _flatSize);
void _call_atexit_hooks_for_range(addr_t start, addr_t size);
void __init_exit_stack_lock(void);
void __init_env(const struct user_space_program_args *args);
void __init_heap(void);

View File

@ -12,31 +12,22 @@ extern "C" {
#endif
typedef struct mutex {
int32 benaphore;
sem_id semaphore;
const char* name;
int32 lock;
uint32 flags;
} mutex;
status_t mutex_init(mutex *lock, const char *name);
#define MUTEX_FLAG_CLONE_NAME 0x1
#define MUTEX_INITIALIZER(name) { name, 0, 0 }
void mutex_init(mutex *lock, const char *name);
void mutex_init_etc(mutex *lock, const char *name, uint32 flags);
void mutex_destroy(mutex *lock);
status_t mutex_lock(mutex *lock);
void mutex_unlock(mutex *lock);
typedef struct lazy_mutex {
int32 benaphore;
sem_id semaphore;
const char* name;
} lazy_mutex;
status_t lazy_mutex_init(lazy_mutex *lock, const char *name);
// name will not be cloned and must rename valid
void lazy_mutex_destroy(lazy_mutex *lock);
status_t lazy_mutex_lock(lazy_mutex *lock);
void lazy_mutex_unlock(lazy_mutex *lock);
typedef struct rw_lock {
const char * name;
mutex lock;
struct rw_lock_waiter * waiters;
struct rw_lock_waiter * last_waiter;
@ -46,7 +37,12 @@ typedef struct rw_lock {
int32 owner_count;
} rw_lock;
status_t rw_lock_init(rw_lock *lock, const char *name);
#define RW_LOCK_FLAG_CLONE_NAME MUTEX_FLAG_CLONE_NAME
#define RW_LOCK_INITIALIZER(name) { MUTEX_INITIALIZER(name), NULL, \
NULL, -1, 0, 0, 0 }
void rw_lock_init(rw_lock *lock, const char *name);
void rw_lock_init_etc(rw_lock *lock, const char *name, uint32 flags);
void rw_lock_destroy(rw_lock *lock);
status_t rw_lock_read_lock(rw_lock *lock);
status_t rw_lock_read_unlock(rw_lock *lock);
@ -57,31 +53,21 @@ status_t rw_lock_write_unlock(rw_lock *lock);
typedef struct recursive_lock {
mutex lock;
thread_id holder;
int recursion;
int32 recursion;
} recursive_lock;
status_t recursive_lock_init(recursive_lock *lock, const char *name);
#define RECURSIVE_LOCK_FLAG_CLONE_NAME MUTEX_FLAG_CLONE_NAME
#define RECURSIVE_LOCK_INITIALIZER(name) { MUTEX_INITIALIZER(name), -1, 0 }
void recursive_lock_init(recursive_lock *lock, const char *name);
void recursive_lock_init_etc(recursive_lock *lock, const char *name,
uint32 flags);
void recursive_lock_destroy(recursive_lock *lock);
status_t recursive_lock_lock(recursive_lock *lock);
void recursive_lock_unlock(recursive_lock *lock);
int32 recursive_lock_get_recursion(recursive_lock *lock);
typedef struct lazy_recursive_lock {
lazy_mutex lock;
thread_id holder;
int recursion;
} lazy_recursive_lock;
status_t lazy_recursive_lock_init(lazy_recursive_lock *lock,
const char *name);
// name will not be cloned and must rename valid
void lazy_recursive_lock_destroy(lazy_recursive_lock *lock);
status_t lazy_recursive_lock_lock(lazy_recursive_lock *lock);
void lazy_recursive_lock_unlock(lazy_recursive_lock *lock);
int32 lazy_recursive_lock_get_recursion(lazy_recursive_lock *lock);
#define INIT_ONCE_UNINITIALIZED -1
#define INIT_ONCE_INITIALIZED -4

View File

@ -75,7 +75,7 @@ typedef DoublyLinkedList<rtm_pool> PoolList;
const static uint32 kAlignment = 256;
// all memory chunks will be a multiple of this
static mutex sPoolsLock = {-1, -1};
static mutex sPoolsLock = MUTEX_INITIALIZER("rtm pools");
static PoolList sPools;
@ -270,13 +270,6 @@ pool_for(void* buffer)
}
static void
pool_init(void)
{
mutex_init(&sPoolsLock, "rtm pools");
}
// #pragma mark - public API
@ -287,14 +280,10 @@ rtm_create_pool(rtm_pool** _pool, size_t totalSize, const char* name)
if (pool == NULL)
return B_NO_MEMORY;
if (name == NULL)
name = "realtime pool";
status_t status = mutex_init(&pool->lock, name);
if (status != B_OK) {
free(pool);
return status;
}
if (name != NULL)
mutex_init_etc(&pool->lock, name, MUTEX_FLAG_CLONE_NAME);
else
mutex_init(&pool->lock, "realtime pool");
// Allocate enough space for at least one allocation over \a totalSize
pool->max_size = (totalSize + sizeof(FreeChunk) - 1 + B_PAGE_SIZE)
@ -321,9 +310,6 @@ rtm_create_pool(rtm_pool** _pool, size_t totalSize, const char* name)
*_pool = pool;
static pthread_once_t sOnce = PTHREAD_ONCE_INIT;
pthread_once(&sOnce, &pool_init);
MutexLocker _(&sPoolsLock);
sPools.Add(pool);
return B_OK;

View File

@ -52,9 +52,7 @@ initialize_before(image_id imageID)
__gRuntimeLoader->call_atexit_hooks_for_range
= _call_atexit_hooks_for_range;
__init_exit_stack_lock();
__init_time();
__init_fork();
__init_heap();
__init_env(__gRuntimeLoader->program_args);
__init_pwd_backend();

View File

@ -1,6 +1,6 @@
/*
* Copyright 2009, Michael Lotz, mmlr@mlotz.ch.
* Copyright 2008-2009, Ingo Weinhold, ingo_weinhold@gmx.de.
* Copyright 2008-2010, Ingo Weinhold, ingo_weinhold@gmx.de.
* Copyright 2002-2009, Axel Dörfler, axeld@pinc-software.de. All rights reserved.
* Distributed under the terms of the MIT License.
*
@ -11,154 +11,72 @@
#include <locks.h>
#include <OS.h>
#include <stdlib.h>
#include <string.h>
#include <syscalls.h>
#include <user_mutex_defs.h>
// #pragma mark - mutex
status_t
void
mutex_init(mutex *lock, const char *name)
{
if (lock == NULL || name == NULL)
return B_BAD_VALUE;
lock->name = name;
lock->lock = 0;
lock->flags = 0;
}
lock->benaphore = 0;
lock->semaphore = create_sem(0, name);
if (lock->semaphore < 0)
return lock->semaphore;
return B_OK;
void
mutex_init_etc(mutex *lock, const char *name, uint32 flags)
{
lock->name = (flags & MUTEX_FLAG_CLONE_NAME) != 0 ? strdup(name) : name;
lock->lock = 0;
lock->flags = flags;
}
void
mutex_destroy(mutex *lock)
{
delete_sem(lock->semaphore);
if ((lock->flags & MUTEX_FLAG_CLONE_NAME) != 0)
free(const_cast<char*>(lock->name));
}
status_t
mutex_lock(mutex *lock)
{
if (atomic_add(&lock->benaphore, 1) == 0)
// set the locked flag
int32 oldValue = atomic_or(&lock->lock, B_USER_MUTEX_LOCKED);
if ((oldValue & (B_USER_MUTEX_LOCKED | B_USER_MUTEX_WAITING)) == 0
|| (oldValue & B_USER_MUTEX_DISABLED) != 0) {
// No one has the lock or is waiting for it, or the mutex has been
// disabled.
return B_OK;
}
status_t result;
// we have to call the kernel
status_t error;
do {
result = acquire_sem(lock->semaphore);
} while (result == B_INTERRUPTED);
error = _kern_mutex_lock(&lock->lock, lock->name, 0, 0);
} while (error == B_INTERRUPTED);
return result;
return error;
}
void
mutex_unlock(mutex *lock)
{
if (atomic_add(&lock->benaphore, -1) != 1)
release_sem(lock->semaphore);
}
// #pragma mark - lazy mutex
enum {
STATE_UNINITIALIZED = -1,
STATE_INITIALIZING = -2,
STATE_SPIN_LOCKED = -3,
STATE_SPIN_UNLOCKED = -4
};
static inline bool
lazy_mutex_ensure_init(lazy_mutex *lock)
{
int32 value = atomic_test_and_set((vint32*)&lock->semaphore,
STATE_INITIALIZING, STATE_UNINITIALIZED);
if (value >= 0)
return true;
if (value == STATE_UNINITIALIZED) {
// we're the first -- perform the initialization
sem_id semaphore = create_sem(0, lock->name);
if (semaphore < 0)
semaphore = STATE_SPIN_UNLOCKED;
atomic_set((vint32*)&lock->semaphore, semaphore);
return semaphore >= 0;
}
if (value == STATE_INITIALIZING) {
// someone else is initializing -- spin until that is done
while (atomic_get((vint32*)&lock->semaphore) == STATE_INITIALIZING) {
}
}
return lock->semaphore >= 0;
}
status_t
lazy_mutex_init(lazy_mutex *lock, const char *name)
{
if (lock == NULL || name == NULL)
return B_BAD_VALUE;
lock->benaphore = 0;
lock->semaphore = STATE_UNINITIALIZED;
lock->name = name;
return B_OK;
}
void
lazy_mutex_destroy(lazy_mutex *lock)
{
if (lock->semaphore >= 0)
delete_sem(lock->semaphore);
}
status_t
lazy_mutex_lock(lazy_mutex *lock)
{
if (atomic_add(&lock->benaphore, 1) == 0)
return B_OK;
if (lazy_mutex_ensure_init(lock)) {
// acquire the semaphore
status_t result;
do {
result = acquire_sem(lock->semaphore);
} while (result == B_INTERRUPTED);
return result;
} else {
// the semaphore creation failed -- so we use it like a spinlock instead
while (atomic_test_and_set((vint32*)&lock->semaphore,
STATE_SPIN_LOCKED, STATE_SPIN_UNLOCKED)
!= STATE_SPIN_UNLOCKED) {
}
return B_OK;
}
}
void
lazy_mutex_unlock(lazy_mutex *lock)
{
if (atomic_add(&lock->benaphore, -1) == 1)
return;
if (lazy_mutex_ensure_init(lock)) {
// release the semaphore
release_sem(lock->semaphore);
} else {
// the semaphore creation failed -- so we use it like a spinlock instead
atomic_set((vint32*)&lock->semaphore, STATE_SPIN_UNLOCKED);
// clear the locked flag
int32 oldValue = atomic_and(&lock->lock, ~(int32)B_USER_MUTEX_LOCKED);
if ((oldValue & B_USER_MUTEX_WAITING) != 0
&& (oldValue & B_USER_MUTEX_DISABLED) == 0) {
_kern_mutex_unlock(&lock->lock, 0);
}
}

View File

@ -1,6 +1,6 @@
/*
* Copyright 2009, Michael Lotz, mmlr@mlotz.ch.
* Copyright 2008-2009, Ingo Weinhold, ingo_weinhold@gmx.de.
* Copyright 2008-2010, Ingo Weinhold, ingo_weinhold@gmx.de.
* Copyright 2002-2009, Axel Dörfler, axeld@pinc-software.de. All rights reserved.
* Distributed under the terms of the MIT License.
*
@ -27,12 +27,19 @@ recursive_lock_get_recursion(recursive_lock *lock)
}
status_t
void
recursive_lock_init(recursive_lock *lock, const char *name)
{
recursive_lock_init_etc(lock, name, 0);
}
void
recursive_lock_init_etc(recursive_lock *lock, const char *name, uint32 flags)
{
lock->holder = -1;
lock->recursion = 0;
return mutex_init(&lock->lock, name != NULL ? name : "recursive lock");
mutex_init_etc(&lock->lock, name, flags);
}
@ -74,65 +81,3 @@ recursive_lock_unlock(recursive_lock *lock)
mutex_unlock(&lock->lock);
}
}
// #pragma mark - lazy recursive lock
int32
lazy_recursive_lock_get_recursion(lazy_recursive_lock *lock)
{
if (lock->holder == find_thread(NULL))
return lock->recursion;
return -1;
}
status_t
lazy_recursive_lock_init(lazy_recursive_lock *lock, const char *name)
{
lock->holder = -1;
lock->recursion = 0;
return lazy_mutex_init(&lock->lock, name != NULL ? name : "recursive lock");
}
void
lazy_recursive_lock_destroy(lazy_recursive_lock *lock)
{
if (lock == NULL)
return;
lazy_mutex_destroy(&lock->lock);
}
status_t
lazy_recursive_lock_lock(lazy_recursive_lock *lock)
{
thread_id thread = find_thread(NULL);
if (thread != lock->holder) {
lazy_mutex_lock(&lock->lock);
lock->holder = thread;
}
lock->recursion++;
return B_OK;
}
void
lazy_recursive_lock_unlock(lazy_recursive_lock *lock)
{
if (find_thread(NULL) != lock->holder) {
debugger("lazy_recursive_lock unlocked by non-holder thread!\n");
return;
}
if (--lock->recursion == 0) {
lock->holder = -1;
lazy_mutex_unlock(&lock->lock);
}
}

View File

@ -79,16 +79,22 @@ rw_lock_unblock(rw_lock *lock)
}
status_t
void
rw_lock_init(rw_lock *lock, const char *name)
{
lock->name = name;
rw_lock_init_etc(lock, name, 0);
}
void
rw_lock_init_etc(rw_lock *lock, const char *name, uint32 flags)
{
lock->waiters = NULL;
lock->holder = -1;
lock->reader_count = 0;
lock->writer_count = 0;
lock->owner_count = 0;
return mutex_init(&lock->lock, name);
mutex_init_etc(&lock->lock, name, flags);
}

View File

@ -11,10 +11,10 @@ MergeObject posix_pthread.o :
pthread_attr.c
pthread_cancel.cpp
pthread_cleanup.cpp
pthread_cond.c
pthread_cond.cpp
pthread_condattr.c
pthread_key.cpp
pthread_mutex.c
pthread_mutex.cpp
pthread_mutexattr.c
pthread_once.cpp
pthread_rwlock.cpp

View File

@ -1,183 +0,0 @@
/*
* Copyright 2007, Ryan Leavengood, leavengood@gmail.com.
* All rights reserved. Distributed under the terms of the MIT License.
*/
#include <pthread.h>
#include "pthread_private.h"
#include <stdlib.h>
#include <stdio.h>
#include <string.h>
#define COND_FLAG_SHARED 0x01
static const pthread_condattr pthread_condattr_default = {
false
};
int
pthread_cond_init(pthread_cond_t *cond, const pthread_condattr_t *_attr)
{
const pthread_condattr *attr = NULL;
if (cond == NULL)
return B_BAD_VALUE;
if (_attr != NULL)
attr = *_attr;
else
attr = &pthread_condattr_default;
cond->flags = 0;
if (attr->process_shared)
cond->flags |= COND_FLAG_SHARED;
cond->sem = create_sem(0, "pthread_cond");
if (cond->sem < B_OK) {
return B_WOULD_BLOCK;
// stupid error code (EAGAIN) but demanded by POSIX
}
cond->mutex = NULL;
cond->waiter_count = 0;
cond->event_counter = 0;
return B_OK;
}
int
pthread_cond_destroy(pthread_cond_t *cond)
{
if (cond == NULL)
return B_BAD_VALUE;
delete_sem(cond->sem);
return B_OK;
}
static status_t
cond_wait(pthread_cond_t *cond, pthread_mutex_t *mutex, bigtime_t timeout)
{
status_t status;
int32 event;
if (cond == NULL || mutex == NULL)
return B_BAD_VALUE;
if (mutex->owner != find_thread(NULL))
// POSIX suggests EPERM (= B_NOT_ALLOWED) to be returned
// if this thread does not own the mutex
return B_NOT_ALLOWED;
// lazy init
if (cond->sem == -42) {
// Note, this is thread-safe, since another thread would be required to
// hold the same mutex.
sem_id sem = create_sem(0, "pthread_cond");
if (sem < 0)
return EAGAIN;
if (atomic_test_and_set((vint32*)&cond->sem, sem, -42) != -42)
delete_sem(sem);
}
if (cond->mutex && cond->mutex != mutex)
// POSIX suggests EINVAL (= B_BAD_VALUE) to be returned if
// the same condition variable is used with multiple mutexes
return B_BAD_VALUE;
cond->mutex = mutex;
cond->waiter_count++;
event = atomic_get((vint32*)&cond->event_counter);
pthread_mutex_unlock(mutex);
do {
status = acquire_sem_etc(cond->sem, 1,
timeout == B_INFINITE_TIMEOUT ? 0 : B_ABSOLUTE_REAL_TIME_TIMEOUT,
timeout);
} while (status == B_OK
&& atomic_get((vint32*)&cond->event_counter) == event);
pthread_mutex_lock(mutex);
cond->waiter_count--;
// If there are no more waiters, we can change mutexes
if (cond->waiter_count == 0)
cond->mutex = NULL;
return status;
}
static status_t
cond_signal(pthread_cond_t *cond, bool broadcast)
{
int32 waiterCount;
if (cond == NULL)
return B_BAD_VALUE;
waiterCount = cond->waiter_count;
if (waiterCount > 0) {
atomic_add((vint32*)&cond->event_counter, 1);
return release_sem_etc(cond->sem, broadcast ? waiterCount : 1, 0);
}
return 0;
}
int
pthread_cond_wait(pthread_cond_t *cond, pthread_mutex_t *_mutex)
{
return cond_wait(cond, _mutex, B_INFINITE_TIMEOUT);
}
int
pthread_cond_timedwait(pthread_cond_t *cond, pthread_mutex_t *mutex,
const struct timespec *tv)
{
bool invalidTime = false;
status_t status;
bigtime_t timeout = 0;
if (tv && tv->tv_nsec < 1000*1000*1000 && tv->tv_nsec >= 0)
timeout = tv->tv_sec * 1000000LL + tv->tv_nsec / 1000LL;
else
invalidTime = true;
status = cond_wait(cond, mutex, timeout);
if (status != B_OK && invalidTime) {
// POSIX requires EINVAL (= B_BAD_VALUE) to be returned
// if the timespec structure was invalid
return B_BAD_VALUE;
}
return status;
}
int
pthread_cond_broadcast(pthread_cond_t *cond)
{
return cond_signal(cond, true);
}
int
pthread_cond_signal(pthread_cond_t *cond)
{
return cond_signal(cond, false);
}

View File

@ -0,0 +1,141 @@
/*
* Copyright 2010, Ingo Weinhold, ingo_weinhold@gmx.de.
* Copyright 2007, Ryan Leavengood, leavengood@gmail.com.
* All rights reserved. Distributed under the terms of the MIT License.
*/
#include <pthread.h>
#include "pthread_private.h"
#include <stdlib.h>
#include <stdio.h>
#include <string.h>
#include <syscalls.h>
#include <user_mutex_defs.h>
#define COND_FLAG_SHARED 0x01
static const pthread_condattr pthread_condattr_default = {
false
};
int
pthread_cond_init(pthread_cond_t* cond, const pthread_condattr_t* _attr)
{
const pthread_condattr* attr = _attr != NULL
? *_attr : &pthread_condattr_default;
cond->flags = 0;
if (attr->process_shared)
cond->flags |= COND_FLAG_SHARED;
cond->mutex = NULL;
cond->waiter_count = 0;
cond->lock = 0;
return 0;
}
int
pthread_cond_destroy(pthread_cond_t* cond)
{
return 0;
}
static status_t
cond_wait(pthread_cond_t* cond, pthread_mutex_t* mutex, bigtime_t timeout)
{
if (mutex->owner != find_thread(NULL)) {
// calling thread isn't mutex owner
return EPERM;
}
if (cond->mutex != NULL && cond->mutex != mutex) {
// condition variable already used with different mutex
return EINVAL;
}
cond->mutex = mutex;
cond->waiter_count++;
// make sure the user mutex we use for blocking is locked
atomic_or((int32*)&cond->lock, B_USER_MUTEX_LOCKED);
// atomically unlock the mutex and start waiting on the user mutex
mutex->owner = -1;
mutex->owner_count = 0;
status_t status = _kern_mutex_switch_lock((int32*)&mutex->lock,
(int32*)&cond->lock, "pthread condition",
timeout == B_INFINITE_TIMEOUT ? 0 : B_ABSOLUTE_REAL_TIME_TIMEOUT,
timeout);
if (status == B_INTERRUPTED) {
// EINTR is not an allowed return value. We either have to restart
// waiting -- which we can't atomically -- or return a spurious 0.
status = 0;
}
pthread_mutex_lock(mutex);
cond->waiter_count--;
// If there are no more waiters, we can change mutexes.
if (cond->waiter_count == 0)
cond->mutex = NULL;
return status;
}
static inline void
cond_signal(pthread_cond_t* cond, bool broadcast)
{
if (cond->waiter_count == 0)
return;
// release the condition lock
_kern_mutex_unlock((int32*)&cond->lock,
broadcast ? B_USER_MUTEX_UNBLOCK_ALL : 0);
}
int
pthread_cond_wait(pthread_cond_t* cond, pthread_mutex_t* _mutex)
{
return cond_wait(cond, _mutex, B_INFINITE_TIMEOUT);
}
int
pthread_cond_timedwait(pthread_cond_t* cond, pthread_mutex_t* mutex,
const struct timespec* tv)
{
if (tv == NULL || tv->tv_nsec < 0 || tv->tv_nsec >= 1000 * 1000 * 1000)
return EINVAL;
return cond_wait(cond, mutex,
tv->tv_sec * 1000000LL + tv->tv_nsec / 1000LL);
}
int
pthread_cond_broadcast(pthread_cond_t* cond)
{
cond_signal(cond, true);
return 0;
}
int
pthread_cond_signal(pthread_cond_t* cond)
{
cond_signal(cond, false);
return 0;
}

View File

@ -1,199 +0,0 @@
/*
* Copyright 2003-2005, Axel Dörfler, axeld@pinc-software.de. All rights reserved.
* Distributed under the terms of the MIT License.
*/
#include <pthread.h>
#include "pthread_private.h"
#include <stdlib.h>
#include <stdio.h>
#include <string.h>
#define MUTEX_FLAG_SHARED 0x80000000
#define MUTEX_TYPE_BITS 0x0000000f
#define MUTEX_TYPE(mutex) ((mutex)->flags & MUTEX_TYPE_BITS)
static const pthread_mutexattr pthread_mutexattr_default = {
PTHREAD_MUTEX_DEFAULT,
false
};
int
pthread_mutex_init(pthread_mutex_t *mutex, const pthread_mutexattr_t *_attr)
{
const pthread_mutexattr *attr = NULL;
if (mutex == NULL)
return B_BAD_VALUE;
if (_attr != NULL)
attr = *_attr;
else
attr = &pthread_mutexattr_default;
mutex->sem = create_sem(0, "pthread_mutex");
if (mutex->sem < B_OK) {
return B_WOULD_BLOCK;
// stupid error code (EAGAIN) but demanded by POSIX
}
mutex->count = 0;
mutex->owner = -1;
mutex->owner_count = 0;
mutex->flags = attr->type | (attr->process_shared ? MUTEX_FLAG_SHARED : 0);
return B_OK;
}
int
pthread_mutex_destroy(pthread_mutex_t *mutex)
{
if (mutex == NULL)
return B_BAD_VALUE;
delete_sem(mutex->sem);
return B_OK;
}
static status_t
mutex_lock(pthread_mutex_t *mutex, bigtime_t timeout)
{
thread_id thisThread = find_thread(NULL);
status_t status = B_OK;
if (mutex == NULL)
return B_BAD_VALUE;
// If statically initialized, we need to create the semaphore, now.
if (mutex->sem == -42) {
sem_id sem = create_sem(0, "pthread_mutex");
if (sem < 0)
return EAGAIN;
if (atomic_test_and_set((vint32*)&mutex->sem, sem, -42) != -42)
delete_sem(sem);
}
if (MUTEX_TYPE(mutex) == PTHREAD_MUTEX_ERRORCHECK
&& mutex->owner == thisThread) {
// we detect this kind of deadlock and return an error
return EDEADLK;
}
if (MUTEX_TYPE(mutex) == PTHREAD_MUTEX_RECURSIVE
&& mutex->owner == thisThread) {
// if we already hold the mutex, we don't need to grab it again
mutex->owner_count++;
return B_OK;
}
if (atomic_add((vint32*)&mutex->count, 1) > 0) {
// this mutex is already locked by someone else, so we need
// to wait
status = acquire_sem_etc(mutex->sem, 1,
timeout == B_INFINITE_TIMEOUT ? 0 : B_ABSOLUTE_REAL_TIME_TIMEOUT,
timeout);
}
if (status == B_OK) {
// we have locked the mutex for the first time
mutex->owner = thisThread;
mutex->owner_count = 1;
}
return status;
}
int
pthread_mutex_lock(pthread_mutex_t *mutex)
{
return mutex_lock(mutex, B_INFINITE_TIMEOUT);
}
int
pthread_mutex_trylock(pthread_mutex_t *mutex)
{
status_t status = mutex_lock(mutex, 0);
return status == B_WOULD_BLOCK ? EBUSY : status;
}
int
pthread_mutex_timedlock(pthread_mutex_t *mutex, const struct timespec *tv)
{
bool invalidTime = false;
status_t status;
bigtime_t timeout = 0;
if (tv && tv->tv_nsec < 1000*1000*1000 && tv->tv_nsec >= 0)
timeout = tv->tv_sec * 1000000LL + tv->tv_nsec / 1000LL;
else
invalidTime = true;
status = mutex_lock(mutex, timeout);
if (status != B_OK && invalidTime) {
// POSIX requires EINVAL (= B_BAD_VALUE) to be returned
// if the timespec structure was invalid
return B_BAD_VALUE;
}
return status;
}
int
pthread_mutex_unlock(pthread_mutex_t *mutex)
{
if (mutex == NULL)
return B_BAD_VALUE;
if (mutex->owner != find_thread(NULL))
return EPERM;
if (MUTEX_TYPE(mutex) == PTHREAD_MUTEX_RECURSIVE
&& mutex->owner_count-- > 1) {
return B_OK;
}
mutex->owner = -1;
if (atomic_add((vint32*)&mutex->count, -1) > 1)
return release_sem(mutex->sem);
return B_OK;
}
int
pthread_mutex_getprioceiling(pthread_mutex_t *mutex, int *_prioCeiling)
{
if (mutex == NULL || _prioCeiling == NULL)
return B_BAD_VALUE;
*_prioCeiling = 0;
// not implemented
return B_OK;
}
int
pthread_mutex_setprioceiling(pthread_mutex_t *mutex, int prioCeiling,
int *_oldCeiling)
{
if (mutex == NULL)
return B_BAD_VALUE;
// not implemented
return B_NOT_ALLOWED;
}

View File

@ -0,0 +1,186 @@
/*
* Copyright 2010, Ingo Weinhold, ingo_weinhold@gmx.de.
* Copyright 2003-2005, Axel Dörfler, axeld@pinc-software.de. All rights reserved.
* Distributed under the terms of the MIT License.
*/
#include <pthread.h>
#include "pthread_private.h"
#include <stdio.h>
#include <stdlib.h>
#include <string.h>
#include <syscalls.h>
#include <user_mutex_defs.h>
#define MUTEX_FLAG_SHARED 0x80000000
#define MUTEX_TYPE_BITS 0x0000000f
#define MUTEX_TYPE(mutex) ((mutex)->flags & MUTEX_TYPE_BITS)
static const pthread_mutexattr pthread_mutexattr_default = {
PTHREAD_MUTEX_DEFAULT,
false
};
int
pthread_mutex_init(pthread_mutex_t* mutex, const pthread_mutexattr_t* _attr)
{
const pthread_mutexattr* attr = _attr != NULL
? *_attr : &pthread_mutexattr_default;
mutex->lock = 0;
mutex->owner = -1;
mutex->owner_count = 0;
mutex->flags = attr->type | (attr->process_shared ? MUTEX_FLAG_SHARED : 0);
return 0;
}
int
pthread_mutex_destroy(pthread_mutex_t* mutex)
{
return 0;
}
static status_t
mutex_lock(pthread_mutex_t* mutex, bigtime_t timeout)
{
thread_id thisThread = find_thread(NULL);
if (mutex->owner == thisThread) {
// recursive locking handling
if (MUTEX_TYPE(mutex) == PTHREAD_MUTEX_RECURSIVE) {
if (mutex->owner_count == INT32_MAX)
return EAGAIN;
mutex->owner_count++;
return 0;
}
// deadlock check (not for PTHREAD_MUTEX_NORMAL as per the specs)
if (MUTEX_TYPE(mutex) == PTHREAD_MUTEX_ERRORCHECK
|| MUTEX_TYPE(mutex) == PTHREAD_MUTEX_DEFAULT) {
// we detect this kind of deadlock and return an error
return timeout < 0 ? EBUSY : EDEADLK;
}
}
// set the locked flag
int32 oldValue = atomic_or((int32*)&mutex->lock, B_USER_MUTEX_LOCKED);
if ((oldValue & (B_USER_MUTEX_LOCKED | B_USER_MUTEX_WAITING)) != 0) {
// someone else has the lock or is at least waiting for it
if (timeout < 0)
return EBUSY;
// we have to call the kernel
status_t error;
do {
error = _kern_mutex_lock((int32*)&mutex->lock, NULL,
timeout == B_INFINITE_TIMEOUT
? 0 : B_ABSOLUTE_REAL_TIME_TIMEOUT,
timeout);
} while (error == B_INTERRUPTED);
if (error != B_OK)
return error;
}
// we have locked the mutex for the first time
mutex->owner = thisThread;
mutex->owner_count = 1;
return 0;
}
int
pthread_mutex_lock(pthread_mutex_t* mutex)
{
return mutex_lock(mutex, B_INFINITE_TIMEOUT);
}
int
pthread_mutex_trylock(pthread_mutex_t* mutex)
{
return mutex_lock(mutex, -1);
}
int
pthread_mutex_timedlock(pthread_mutex_t* mutex, const struct timespec* tv)
{
// translate the timeout
bool invalidTime = false;
bigtime_t timeout = 0;
if (tv && tv->tv_nsec < 1000 * 1000 * 1000 && tv->tv_nsec >= 0)
timeout = tv->tv_sec * 1000000LL + tv->tv_nsec / 1000LL;
else
invalidTime = true;
status_t status = mutex_lock(mutex, timeout);
if (status != B_OK && invalidTime) {
// The timespec was not valid and the mutex could not be locked
// immediately.
return EINVAL;
}
return status;
}
int
pthread_mutex_unlock(pthread_mutex_t* mutex)
{
if (mutex->owner != find_thread(NULL))
return EPERM;
if (MUTEX_TYPE(mutex) == PTHREAD_MUTEX_RECURSIVE
&& --mutex->owner_count > 0) {
// still locked
return 0;
}
mutex->owner = -1;
// clear the locked flag
int32 oldValue = atomic_and((int32*)&mutex->lock,
~(int32)B_USER_MUTEX_LOCKED);
if ((oldValue & B_USER_MUTEX_WAITING) != 0)
_kern_mutex_unlock((int32*)&mutex->lock, 0);
return 0;
}
int
pthread_mutex_getprioceiling(pthread_mutex_t* mutex, int* _prioCeiling)
{
if (mutex == NULL || _prioCeiling == NULL)
return EINVAL;
*_prioCeiling = 0;
// not implemented
return 0;
}
int
pthread_mutex_setprioceiling(pthread_mutex_t* mutex, int prioCeiling,
int* _oldCeiling)
{
if (mutex == NULL)
return EINVAL;
// not implemented
return EPERM;
}

View File

@ -18,7 +18,9 @@
#include <user_runtime.h>
static lazy_mutex sEnvLock;
static const char* const kEnvLockName = "env lock";
static mutex sEnvLock = MUTEX_INITIALIZER(kEnvLockName);
static char **sManagedEnviron;
char **environ = NULL;
@ -27,14 +29,14 @@ char **environ = NULL;
static inline void
lock_variables(void)
{
lazy_mutex_lock(&sEnvLock);
mutex_lock(&sEnvLock);
}
static inline void
unlock_variables(void)
{
lazy_mutex_unlock(&sEnvLock);
mutex_unlock(&sEnvLock);
}
@ -186,7 +188,7 @@ update_variable(const char *name, int32 length, const char *value,
static void
environ_fork_hook(void)
{
lazy_mutex_init(&sEnvLock, "env lock");
mutex_init(&sEnvLock, kEnvLockName);
}
@ -199,7 +201,6 @@ __init_env(const struct user_space_program_args *args)
// Following POSIX, there is no need to make any of the environment
// functions thread-safe - but we do it anyway as much as possible to
// protect our implementation
lazy_mutex_init(&sEnvLock, "env lock");
environ = args->env;
sManagedEnviron = NULL;

View File

@ -25,26 +25,27 @@ extern void _IO_cleanup(void);
extern void _thread_do_exit_work(void);
struct exit_stack_info {
void (*exit_stack[ATEXIT_MAX])(void);
int32 stack_size;
lazy_recursive_lock lock;
void (*exit_stack[ATEXIT_MAX])(void);
int32 stack_size;
recursive_lock lock;
};
static struct exit_stack_info sExitStackInfo = { {}, 0, {} };
static struct exit_stack_info sExitStackInfo
= { {}, 0, RECURSIVE_LOCK_INITIALIZER("exit stack lock") };
static void inline
_exit_stack_lock()
{
lazy_recursive_lock_lock(&sExitStackInfo.lock);
recursive_lock_lock(&sExitStackInfo.lock);
}
static void inline
_exit_stack_unlock()
{
lazy_recursive_lock_unlock(&sExitStackInfo.lock);
recursive_lock_unlock(&sExitStackInfo.lock);
}
@ -83,16 +84,6 @@ _call_atexit_hooks_for_range(addr_t start, addr_t size)
}
void
__init_exit_stack_lock(void)
{
status_t error = lazy_recursive_lock_init(&sExitStackInfo.lock,
"exit stack lock");
if (error != B_OK)
debugger("failed to create exit stack lock");
}
// #pragma mark - public API

View File

@ -21,9 +21,11 @@ typedef struct fork_hook {
void (*function)(void);
} fork_hook;
#define FORK_LOCK_NAME "fork lock"
static fork_hook *sPrepareHooks, *sParentHooks, *sChildHooks;
static fork_hook *sLastParentHook, *sLastChildHook;
static lazy_mutex sForkLock;
static mutex sForkLock = MUTEX_INITIALIZER(FORK_LOCK_NAME);
extern thread_id __main_thread_id;
@ -93,14 +95,6 @@ call_fork_hooks(fork_hook *hook)
}
status_t
__init_fork(void)
{
lazy_mutex_init(&sForkLock, "fork lock");
return B_OK;
}
/** Private support function that registers the hooks that will be executed
* before and after the team is fork()ed.
* It is called from pthread_atfork() and atfork().
@ -109,7 +103,7 @@ __init_fork(void)
status_t
__register_atfork(void (*prepare)(void), void (*parent)(void), void (*child)(void))
{
status_t status = lazy_mutex_lock(&sForkLock);
status_t status = mutex_lock(&sForkLock);
if (status != B_OK)
return status;
@ -122,7 +116,7 @@ __register_atfork(void (*prepare)(void), void (*parent)(void), void (*child)(voi
if (status == B_OK && child)
status = add_fork_hook(&sChildHooks, &sLastChildHook, child);
lazy_mutex_unlock(&sForkLock);
mutex_unlock(&sForkLock);
return status;
}
@ -133,7 +127,7 @@ fork(void)
thread_id thread;
status_t status;
status = lazy_mutex_lock(&sForkLock);
status = mutex_lock(&sForkLock);
if (status != B_OK)
return status;
@ -143,7 +137,7 @@ fork(void)
thread = _kern_fork();
if (thread < 0) {
// something went wrong
lazy_mutex_unlock(&sForkLock);
mutex_unlock(&sForkLock);
errno = thread;
return -1;
}
@ -152,7 +146,10 @@ fork(void)
// we are the child
// ToDo: initialize child
__main_thread_id = find_thread(NULL);
__init_fork();
mutex_init(&sForkLock, FORK_LOCK_NAME);
// TODO: The lock is already initialized and we in the fork()ing
// process we should make sure that it is in a consistent state when
// calling the kernel.
__gRuntimeLoader->reinit_after_fork();
__reinit_pwd_backend_after_fork();
@ -160,7 +157,7 @@ fork(void)
} else {
// we are the parent
call_fork_hooks(sParentHooks);
lazy_mutex_unlock(&sForkLock);
mutex_unlock(&sForkLock);
}
return thread;

View File

@ -1,8 +1,9 @@
/*
* Copyright 2008, Ingo Weinhold, ingo_weinhold@gmx.de. All rights reserved.
* Copyright 2008-2010, Ingo Weinhold, ingo_weinhold@gmx.de.
* Distributed under the terms of the MIT License.
*/
#include <user_group.h>
#include <ctype.h>
@ -24,25 +25,27 @@
using BPrivate::Tokenizer;
static const char* const kUserGroupLockName = "user group";
const char* BPrivate::kPasswdFile = "/etc/passwd";
const char* BPrivate::kGroupFile = "/etc/group";
const char* BPrivate::kShadowPwdFile = "/etc/shadow";
static lazy_mutex sUserGroupLock;
static mutex sUserGroupLock = MUTEX_INITIALIZER(kUserGroupLockName);
static port_id sRegistrarPort = -1;
status_t
BPrivate::user_group_lock()
{
return lazy_mutex_lock(&sUserGroupLock);
return mutex_lock(&sUserGroupLock);
}
status_t
BPrivate::user_group_unlock()
{
lazy_mutex_unlock(&sUserGroupLock);
mutex_unlock(&sUserGroupLock);
return B_OK;
}
@ -385,12 +388,11 @@ BPrivate::parse_shadow_pwd_line(char* line, char*& name, char*& password,
void
__init_pwd_backend(void)
{
lazy_mutex_init(&sUserGroupLock, "user group");
}
void
__reinit_pwd_backend_after_fork(void)
{
lazy_mutex_init(&sUserGroupLock, "user group");
mutex_init(&sUserGroupLock, kUserGroupLockName);
}

View File

@ -38,6 +38,8 @@
// a handle returned by load_library() (dlopen())
#define RLD_GLOBAL_SCOPE ((void*)-2l)
static const char* const kLockName = "runtime loader";
typedef void (*init_term_function)(image_id);
@ -47,7 +49,7 @@ image_t* gProgramImage;
static image_t** sPreloadedImages = NULL;
static uint32 sPreloadedImageCount = 0;
static recursive_lock sLock;
static recursive_lock sLock = RECURSIVE_LOCK_INITIALIZER(kLockName);
static inline void
@ -928,8 +930,6 @@ terminate_program(void)
void
rldelf_init(void)
{
recursive_lock_init(&sLock, "runtime loader");
init_add_ons();
// create the debug area
@ -962,9 +962,7 @@ rldelf_init(void)
status_t
elf_reinit_after_fork(void)
{
status_t error = recursive_lock_init(&sLock, "runtime loader");
if (error != B_OK)
return error;
recursive_lock_init(&sLock, kLockName);
// We also need to update the IDs of our images. We are the child and
// and have cloned images with different IDs. Since in most cases (fork()