X-Git-Url: http://info.iut-bm.univ-fcomte.fr/pub/gitweb/simgrid.git/blobdiff_plain/0446fc9e3f379b9aff5e0bb44cf06d06b9e663cc..586776316687c1b369137bdbcd2107761ed5ba9a:/src/xbt/xbt_os_thread.c diff --git a/src/xbt/xbt_os_thread.c b/src/xbt/xbt_os_thread.c index 4a793e4eec..04a497835f 100644 --- a/src/xbt/xbt_os_thread.c +++ b/src/xbt/xbt_os_thread.c @@ -1,564 +1,491 @@ -/* $Id$ */ - /* xbt_os_thread -- portability layer over the pthread API */ /* Used in RL to get win/lin portability, and in SG when CONTEXT_THREAD */ -/* in SG, when using CONTEXT_UCONTEXT, xbt_os_thread_stub is used instead */ +/* in SG, when using HAVE_UCONTEXT_CONTEXTS, xbt_os_thread_stub is used instead */ -/* Copyright 2006,2007 Malek Cherier, Martin Quinson - * All right reserved. */ +/* Copyright (c) 2007-2017. The SimGrid Team. + * All rights reserved. */ /* This program is free software; you can redistribute it and/or modify it * under the terms of the license (GNU LGPL) which comes with this package. */ +#include "src/internal_config.h" +#if HAVE_PTHREAD_SETAFFINITY +#define _GNU_SOURCE +#include +#endif + +#include + +#if defined(__FreeBSD__) +#include "pthread_np.h" +#define cpu_set_t cpuset_t +#endif + +#include +#include +#include + +#if defined(_WIN32) +#include +#elif defined(__MACH__) && defined(__APPLE__) +#include +#include +#include +#else +#include +#endif + #include "xbt/sysdep.h" #include "xbt/ex.h" -#include "xbt/ex_interface.h" /* We play crude games with exceptions */ -#include "portable.h" -#include "xbt/xbt_os_time.h" /* Portable time facilities */ -#include "xbt/xbt_os_thread.h" /* This module */ -#include "xbt_modinter.h" /* Initialization/finalization of this module */ +#include "src/internal_config.h" +#include "xbt/xbt_os_time.h" /* Portable time facilities */ +#include "xbt/xbt_os_thread.h" /* This module */ +#include "src/xbt_modinter.h" /* Initialization/finalization of this module */ -XBT_LOG_NEW_DEFAULT_SUBCATEGORY(xbt_sync_os,xbt,"Synchronization mechanism (OS-level)"); - -/* ********************************* PTHREAD IMPLEMENTATION ************************************ */ -#ifdef HAVE_PTHREAD_H -#include +XBT_LOG_NEW_DEFAULT_SUBCATEGORY(xbt_sync_os, xbt, "Synchronization mechanism (OS-level)"); +/* use named semaphore when sem_init() does not work */ +#if !HAVE_SEM_INIT +static int next_sem_ID = 0; +static xbt_os_mutex_t next_sem_ID_lock; +#endif typedef struct xbt_os_thread_ { - pthread_t t; - char *name; - void *param; - pvoid_f_pvoid_t start_routine; - ex_ctx_t *exception; -} s_xbt_os_thread_t ; + pthread_t t; + int detached; + char *name; + void *param; + pvoid_f_pvoid_t start_routine; + void *extra_data; +} s_xbt_os_thread_t; static xbt_os_thread_t main_thread = NULL; /* thread-specific data containing the xbt_os_thread_t structure */ static pthread_key_t xbt_self_thread_key; static int thread_mod_inited = 0; -/* frees the xbt_os_thread_t corresponding to the current thread */ -static void xbt_os_thread_free_thread_data(void*d){ - free(d); -} +/* defaults attribute for pthreads */ +//FIXME: find where to put this +static pthread_attr_t thread_attr; -/* callback: context fetching */ -static ex_ctx_t *_os_thread_ex_ctx(void) { - return xbt_os_thread_self()->exception; +/* frees the xbt_os_thread_t corresponding to the current thread */ +static void xbt_os_thread_free_thread_data(xbt_os_thread_t thread) +{ + if (thread == main_thread) /* just killed main thread */ + main_thread = NULL; + free(thread->name); + free(thread); } /* callback: termination */ -static void _os_thread_ex_terminate(xbt_ex_t * e) { +static void _os_thread_ex_terminate(xbt_ex_t * e) +{ xbt_ex_display(e); - - abort(); + xbt_abort(); /* FIXME: there should be a configuration variable to choose to kill everyone or only this one */ } -void xbt_os_thread_mod_init(void) { - int errcode; - - if (thread_mod_inited) - return; - - if ((errcode=pthread_key_create(&xbt_self_thread_key, NULL))) - THROW0(system_error,errcode,"pthread_key_create failed for xbt_self_thread_key"); +void xbt_os_thread_mod_preinit(void) +{ + if (thread_mod_inited) + return; + + int errcode = pthread_key_create(&xbt_self_thread_key, NULL); + xbt_assert(errcode == 0, "pthread_key_create failed for xbt_self_thread_key"); + + main_thread = xbt_new(s_xbt_os_thread_t, 1); + main_thread->name = NULL; + main_thread->detached = 0; + main_thread->name = xbt_strdup("main"); + main_thread->param = NULL; + main_thread->start_routine = NULL; + main_thread->extra_data = NULL; - main_thread=xbt_new(s_xbt_os_thread_t,1); - main_thread->name = (char*)"main"; - main_thread->start_routine = NULL; - main_thread->param = NULL; - main_thread->exception = xbt_new(ex_ctx_t, 1); - XBT_CTX_INITIALIZE(main_thread->exception); + if ((errcode = pthread_setspecific(xbt_self_thread_key, main_thread))) + THROWF(system_error, errcode, + "Impossible to set the SimGrid identity descriptor to the main thread (pthread_setspecific failed)"); - __xbt_ex_ctx = _os_thread_ex_ctx; - __xbt_ex_terminate = _os_thread_ex_terminate; + pthread_attr_init(&thread_attr); - thread_mod_inited = 1; + thread_mod_inited = 1; + +#if !HAVE_SEM_INIT + next_sem_ID_lock = xbt_os_mutex_init(); +#endif } -void xbt_os_thread_mod_exit(void) { - /* FIXME: don't try to free our key on shutdown. - Valgrind detects no leak if we don't, and whine if we try to */ -// int errcode; - -// if ((errcode=pthread_key_delete(xbt_self_thread_key))) -// THROW0(system_error,errcode,"pthread_key_delete failed for xbt_self_thread_key"); + +void xbt_os_thread_mod_postexit(void) +{ + /* FIXME: don't try to free our key on shutdown. + Valgrind detects no leak if we don't, and whine if we try to */ + // int errcode; + + // if ((errcode=pthread_key_delete(xbt_self_thread_key))) + // THROWF(system_error,errcode,"pthread_key_delete failed for xbt_self_thread_key"); + free(main_thread->name); + free(main_thread); + main_thread = NULL; + thread_mod_inited = 0; +#if !HAVE_SEM_INIT + xbt_os_mutex_destroy(next_sem_ID_lock); +#endif } -static void * wrapper_start_routine(void *s) { - xbt_os_thread_t t = s; - int errcode; +/** Calls pthread_atfork() if present, and raise an exception otherwise. + * + * The only known user of this wrapper is mmalloc_preinit(), but it is absolutely mandatory there: + * when used with tesh, mmalloc *must* be mutex protected and resistant to forks. + * This functionality is the only way to get it working (by ensuring that the mutex is consistently released on forks) + */ - if ((errcode=pthread_setspecific(xbt_self_thread_key,t))) - THROW0(system_error,errcode, - "pthread_setspecific failed for xbt_self_thread_key"); - return (*t->start_routine)(t->param); +/* this function is critical to tesh+mmalloc, don't mess with it */ +int xbt_os_thread_atfork(void (*prepare)(void), void (*parent)(void), void (*child)(void)) +{ + return pthread_atfork(prepare, parent, child); } -xbt_os_thread_t xbt_os_thread_create(const char*name, - pvoid_f_pvoid_t start_routine, - void* param) { - int errcode; - xbt_os_thread_t res_thread=xbt_new(s_xbt_os_thread_t,1); - res_thread->name = xbt_strdup(name); - res_thread->start_routine = start_routine; - res_thread->param = param; - res_thread->exception = xbt_new(ex_ctx_t, 1); - XBT_CTX_INITIALIZE(res_thread->exception); - - if ((errcode = pthread_create(&(res_thread->t), NULL, - wrapper_start_routine, res_thread))) - THROW1(system_error,errcode, - "pthread_create failed: %s",strerror(errcode)); +static void *wrapper_start_routine(void *s) +{ + xbt_os_thread_t t = s; + + int errcode = pthread_setspecific(xbt_self_thread_key, t); + xbt_assert(errcode == 0, "pthread_setspecific failed for xbt_self_thread_key"); - return res_thread; + void *res = t->start_routine(t->param); + if (t->detached) + xbt_os_thread_free_thread_data(t); + return res; } -const char* xbt_os_thread_name(xbt_os_thread_t t) { - return t->name; +xbt_os_thread_t xbt_os_thread_create(const char *name, pvoid_f_pvoid_t start_routine, void *param, void *extra_data) +{ + xbt_os_thread_t res_thread = xbt_new(s_xbt_os_thread_t, 1); + res_thread->detached = 0; + res_thread->name = xbt_strdup(name); + res_thread->start_routine = start_routine; + res_thread->param = param; + res_thread->extra_data = extra_data; + + int errcode = pthread_create(&(res_thread->t), &thread_attr, wrapper_start_routine, res_thread); + xbt_assert(errcode == 0, "pthread_create failed: %s", strerror(errcode)); + + return res_thread; +} + +/** Bind the thread to the given core, if possible. + * + * If pthread_setaffinity_np is not usable on that (non-gnu) platform, this function does nothing. + */ +int xbt_os_thread_bind(xbt_os_thread_t thread, int cpu){ + int errcode = 0; +#if HAVE_PTHREAD_SETAFFINITY + pthread_t pthread = thread->t; + cpu_set_t cpuset; + CPU_ZERO(&cpuset); + CPU_SET(cpu, &cpuset); + errcode = pthread_setaffinity_np(pthread, sizeof(cpu_set_t), &cpuset); +#endif + return errcode; } -const char* xbt_os_thread_self_name(void) { - xbt_os_thread_t self = xbt_os_thread_self(); - return self?self->name:"main"; +void xbt_os_thread_setstacksize(int stack_size) +{ + size_t alignment[] = { + xbt_pagesize, +#ifdef PTHREAD_STACK_MIN + PTHREAD_STACK_MIN, +#endif + 0 + }; + + xbt_assert(stack_size >= 0, "stack size %d is negative, maybe it exceeds MAX_INT?", stack_size); + + size_t sz = stack_size; + int res = pthread_attr_setstacksize(&thread_attr, sz); + + for (int i = 0; res == EINVAL && alignment[i] > 0; i++) { + /* Invalid size, try again with next multiple of alignment[i]. */ + size_t rem = sz % alignment[i]; + if (rem != 0 || sz == 0) { + size_t sz2 = sz - rem + alignment[i]; + XBT_DEBUG("pthread_attr_setstacksize failed for %zu, try again with %zu", sz, sz2); + sz = sz2; + res = pthread_attr_setstacksize(&thread_attr, sz); + } + } + + if (res == EINVAL) + XBT_WARN("invalid stack size (maybe too big): %zu", sz); + else if (res != 0) + XBT_WARN("unknown error %d in pthread stacksize setting: %zu", res, sz); +} + +void xbt_os_thread_setguardsize(int guard_size) +{ +#ifdef WIN32 + THROW_UNIMPLEMENTED; //pthread_attr_setguardsize is not implemented in pthread.h on windows +#else + size_t sz = guard_size; + int res = pthread_attr_setguardsize(&thread_attr, sz); + if (res) + XBT_WARN("pthread_attr_setguardsize failed (%d) for size: %zu", res, sz); +#endif } -void -xbt_os_thread_join(xbt_os_thread_t thread,void ** thread_return) { - - int errcode; - - if ((errcode = pthread_join(thread->t,thread_return))) - THROW1(system_error,errcode, "pthread_join failed: %s", - strerror(errcode)); - if (thread->exception) - free(thread->exception); - if (thread == main_thread) /* just killed main thread */ - main_thread = NULL; +const char *xbt_os_thread_self_name(void) +{ + xbt_os_thread_t me = xbt_os_thread_self(); + return me ? (const char *)me->name : "main"; +} - free(thread); -} +void xbt_os_thread_join(xbt_os_thread_t thread, void **thread_return) +{ + int errcode = pthread_join(thread->t, thread_return); -void xbt_os_thread_exit(int *retval) { - pthread_exit(retval); + xbt_assert(errcode==0, "pthread_join failed: %s", strerror(errcode)); + xbt_os_thread_free_thread_data(thread); } -xbt_os_thread_t xbt_os_thread_self(void) { - xbt_os_thread_t res; +void xbt_os_thread_exit(int *retval) +{ + pthread_exit(retval); +} +xbt_os_thread_t xbt_os_thread_self(void ) +{ if (!thread_mod_inited) return NULL; - - res = pthread_getspecific(xbt_self_thread_key); - if (!res) - res = main_thread; - return res; + return pthread_getspecific(xbt_self_thread_key); } -#include -void xbt_os_thread_yield(void) { - sched_yield(); +void xbt_os_thread_key_create(xbt_os_thread_key_t* key) +{ + int errcode = pthread_key_create(key, NULL); + xbt_assert(errcode==0 , "pthread_key_create failed"); } -void xbt_os_thread_cancel(xbt_os_thread_t t) { - pthread_cancel(t->t); -} -/****** mutex related functions ******/ -typedef struct xbt_os_mutex_ { - /* KEEP IT IN SYNC WITH xbt_thread.c */ - pthread_mutex_t m; -} s_xbt_os_mutex_t; -xbt_os_mutex_t xbt_os_mutex_init(void) { - xbt_os_mutex_t res = xbt_new(s_xbt_os_mutex_t,1); - int errcode; - - if ((errcode = pthread_mutex_init(&(res->m),NULL))) - THROW1(system_error,errcode,"pthread_mutex_init() failed: %s", - strerror(errcode)); - - return res; +void xbt_os_thread_set_specific(xbt_os_thread_key_t key, void* value) +{ + int errcode = pthread_setspecific(key, value); + xbt_assert(errcode==0, "pthread_setspecific failed"); } -void xbt_os_mutex_lock(xbt_os_mutex_t mutex) { - int errcode; - - if ((errcode=pthread_mutex_lock(&(mutex->m)))) - THROW2(system_error,errcode,"pthread_mutex_lock(%p) failed: %s", - mutex, strerror(errcode)); +void* xbt_os_thread_get_specific(xbt_os_thread_key_t key) +{ + return pthread_getspecific(key); } -void xbt_os_mutex_unlock(xbt_os_mutex_t mutex) { - int errcode; - - if ((errcode=pthread_mutex_unlock(&(mutex->m)))) - THROW2(system_error,errcode,"pthread_mutex_unlock(%p) failed: %s", - mutex, strerror(errcode)); +void xbt_os_thread_detach(xbt_os_thread_t thread) +{ + thread->detached = 1; + pthread_detach(thread->t); } -void xbt_os_mutex_destroy(xbt_os_mutex_t mutex) { - int errcode; - - if (!mutex) return; - - if ((errcode=pthread_mutex_destroy(&(mutex->m)))) - THROW2(system_error,errcode,"pthread_mutex_destroy(%p) failed: %s", - mutex, strerror(errcode)); - free(mutex); +#include +void xbt_os_thread_yield(void) +{ + sched_yield(); } -/***** condition related functions *****/ -typedef struct xbt_os_cond_ { - /* KEEP IT IN SYNC WITH xbt_thread.c */ - pthread_cond_t c; -} s_xbt_os_cond_t; - -xbt_os_cond_t xbt_os_cond_init(void) { - xbt_os_cond_t res = xbt_new(s_xbt_os_cond_t,1); - int errcode; - if ((errcode=pthread_cond_init(&(res->c),NULL))) - THROW1(system_error,errcode,"pthread_cond_init() failed: %s", - strerror(errcode)); - - return res; +void xbt_os_thread_cancel(xbt_os_thread_t t) +{ + pthread_cancel(t->t); } -void xbt_os_cond_wait(xbt_os_cond_t cond, xbt_os_mutex_t mutex) { - int errcode; - if ((errcode=pthread_cond_wait(&(cond->c),&(mutex->m)))) - THROW3(system_error,errcode,"pthread_cond_wait(%p,%p) failed: %s", - cond,mutex, strerror(errcode)); -} +/****** mutex related functions ******/ +typedef struct xbt_os_mutex_ { + pthread_mutex_t m; +} s_xbt_os_mutex_t; #include #include -void xbt_os_cond_timedwait(xbt_os_cond_t cond, xbt_os_mutex_t mutex, double delay) { - int errcode; - struct timespec ts_end; - double end = delay + xbt_os_time(); - - if (delay < 0) { - xbt_os_cond_wait(cond,mutex); - } else { - ts_end.tv_sec = (time_t) floor(end); - ts_end.tv_nsec = (long) ( ( end - ts_end.tv_sec) * 1000000000); - DEBUG3("pthread_cond_timedwait(%p,%p,%p)",&(cond->c),&(mutex->m), &ts_end); - switch ( (errcode=pthread_cond_timedwait(&(cond->c),&(mutex->m), &ts_end)) ) { - case 0: - return; - case ETIMEDOUT: - THROW3(timeout_error,errcode,"condition %p (mutex %p) wasn't signaled before timeout (%f)", - cond,mutex, delay); - default: - THROW4(system_error,errcode,"pthread_cond_timedwait(%p,%p,%f) failed: %s", - cond,mutex, delay, strerror(errcode)); - } - } -} - -void xbt_os_cond_signal(xbt_os_cond_t cond) { - int errcode; - if ((errcode=pthread_cond_signal(&(cond->c)))) - THROW2(system_error,errcode,"pthread_cond_signal(%p) failed: %s", - cond, strerror(errcode)); -} - -void xbt_os_cond_broadcast(xbt_os_cond_t cond){ - int errcode; - if ((errcode=pthread_cond_broadcast(&(cond->c)))) - THROW2(system_error,errcode,"pthread_cond_broadcast(%p) failed: %s", - cond, strerror(errcode)); -} -void xbt_os_cond_destroy(xbt_os_cond_t cond){ - int errcode; - - if (!cond) return; - - if ((errcode=pthread_cond_destroy(&(cond->c)))) - THROW2(system_error,errcode,"pthread_cond_destroy(%p) failed: %s", - cond, strerror(errcode)); - free(cond); -} - -void *xbt_os_thread_getparam(void) { - xbt_os_thread_t t = xbt_os_thread_self(); - return t?t->param:NULL; -} - -/* ********************************* WINDOWS IMPLEMENTATION ************************************ */ - -#elif defined(WIN32) -typedef struct xbt_os_thread_ { - char *name; - HANDLE handle; /* the win thread handle */ - unsigned long id; /* the win thread id */ - pvoid_f_pvoid_t start_routine; - void* param; -} s_xbt_os_thread_t ; +xbt_os_mutex_t xbt_os_mutex_init(void) +{ + pthread_mutexattr_t Attr; + pthread_mutexattr_init(&Attr); + pthread_mutexattr_settype(&Attr, PTHREAD_MUTEX_RECURSIVE); -/* key to the TLS containing the xbt_os_thread_t structure */ -static unsigned long xbt_self_thread_key; + xbt_os_mutex_t res = xbt_new(s_xbt_os_mutex_t, 1); + int errcode = pthread_mutex_init(&(res->m), &Attr); + xbt_assert(errcode==0, "pthread_mutex_init() failed: %s", strerror(errcode)); -void xbt_os_thread_mod_init(void) { - xbt_self_thread_key = TlsAlloc(); -} -void xbt_os_thread_mod_exit(void) { - - if (!TlsFree(xbt_self_thread_key)) - THROW0(system_error,(int)GetLastError(),"TlsFree() failed to cleanup the thread submodule"); + return res; } -static DWORD WINAPI wrapper_start_routine(void *s) { - xbt_os_thread_t t = (xbt_os_thread_t)s; - - if(!TlsSetValue(xbt_self_thread_key,t)) - THROW0(system_error,(int)GetLastError(),"TlsSetValue of data describing the created thread failed"); - - return (DWORD)t->start_routine(t->param); +void xbt_os_mutex_acquire(xbt_os_mutex_t mutex) +{ + int errcode = pthread_mutex_lock(&(mutex->m)); + xbt_assert(errcode==0, "pthread_mutex_lock(%p) failed: %s", mutex, strerror(errcode)); } - -xbt_os_thread_t xbt_os_thread_create(const char *name,pvoid_f_pvoid_t start_routine, - void* param) { - - xbt_os_thread_t t = xbt_new(s_xbt_os_thread_t,1); - - t->name = xbt_strdup(name); - t->start_routine = start_routine ; - t->param = param; - - t->handle = CreateThread(NULL,0, - (LPTHREAD_START_ROUTINE)wrapper_start_routine, - t,0,&(t->id)); - - if(!t->handle) { - xbt_free(t); - THROW0(system_error,(int)GetLastError(),"CreateThread failed"); - } - - return t; +void xbt_os_mutex_release(xbt_os_mutex_t mutex) +{ + int errcode = pthread_mutex_unlock(&(mutex->m)); + xbt_assert(errcode==0, "pthread_mutex_unlock(%p) failed: %s", mutex, strerror(errcode)); } -const char* xbt_os_thread_name(xbt_os_thread_t t) { - return t->name; -} +void xbt_os_mutex_destroy(xbt_os_mutex_t mutex) +{ + if (!mutex) + return; -const char* xbt_os_thread_self_name(void) { - xbt_os_thread_t t = xbt_os_thread_self(); - return t?t->name:"main"; + int errcode = pthread_mutex_destroy(&(mutex->m)); + xbt_assert(errcode == 0, "pthread_mutex_destroy(%p) failed: %s", mutex, strerror(errcode)); + free(mutex); } -void -xbt_os_thread_join(xbt_os_thread_t thread,void ** thread_return) { +/***** condition related functions *****/ +typedef struct xbt_os_cond_ { + pthread_cond_t c; +} s_xbt_os_cond_t; - if(WAIT_OBJECT_0 != WaitForSingleObject(thread->handle,INFINITE)) - THROW0(system_error,(int)GetLastError(), "WaitForSingleObject failed"); - - if(thread_return){ - - if(!GetExitCodeThread(thread->handle,(DWORD*)(*thread_return))) - THROW0(system_error,(int)GetLastError(), "GetExitCodeThread failed"); - } - - CloseHandle(thread->handle); - free(thread->name); - free(thread); +xbt_os_cond_t xbt_os_cond_init(void) +{ + xbt_os_cond_t res = xbt_new(s_xbt_os_cond_t, 1); + int errcode = pthread_cond_init(&(res->c), NULL); + xbt_assert(errcode==0, "pthread_cond_init() failed: %s", strerror(errcode)); + return res; } -void xbt_os_thread_exit(int *retval) { - if(retval) - ExitThread(*retval); - else - ExitThread(0); +void xbt_os_cond_wait(xbt_os_cond_t cond, xbt_os_mutex_t mutex) +{ + int errcode = pthread_cond_wait(&(cond->c), &(mutex->m)); + xbt_assert(errcode==0, "pthread_cond_wait(%p,%p) failed: %s", cond, mutex, strerror(errcode)); } -xbt_os_thread_t xbt_os_thread_self(void) { - return TlsGetValue(xbt_self_thread_key); +void xbt_os_cond_signal(xbt_os_cond_t cond) +{ + int errcode = pthread_cond_signal(&(cond->c)); + xbt_assert(errcode==0, "pthread_cond_signal(%p) failed: %s", cond, strerror(errcode)); } -void *xbt_os_thread_getparam(void) { - xbt_os_thread_t t = xbt_os_thread_self(); - return t->param; +void xbt_os_cond_broadcast(xbt_os_cond_t cond) +{ + int errcode = pthread_cond_broadcast(&(cond->c)); + xbt_assert(errcode==0, "pthread_cond_broadcast(%p) failed: %s", cond, strerror(errcode)); } +void xbt_os_cond_destroy(xbt_os_cond_t cond) +{ + if (!cond) + return; -void xbt_os_thread_yield(void) { - Sleep(0); -} -void xbt_os_thread_cancel(xbt_os_thread_t t) { - THROW_UNIMPLEMENTED; + int errcode = pthread_cond_destroy(&(cond->c)); + xbt_assert(errcode==0, "pthread_cond_destroy(%p) failed: %s", cond, strerror(errcode)); + free(cond); } -/****** mutex related functions ******/ -typedef struct xbt_os_mutex_ { - /* KEEP IT IN SYNC WITH xbt_thread.c */ - CRITICAL_SECTION lock; -} s_xbt_os_mutex_t; +typedef struct xbt_os_sem_ { +#if !HAVE_SEM_INIT + char *name; +#endif + sem_t s; + sem_t *ps; +} s_xbt_os_sem_t; -xbt_os_mutex_t xbt_os_mutex_init(void) { - xbt_os_mutex_t res = xbt_new(s_xbt_os_mutex_t,1); +#ifndef SEM_FAILED +#define SEM_FAILED (-1) +#endif - /* initialize the critical section object */ - InitializeCriticalSection(&(res->lock)); - - return res; -} +xbt_os_sem_t xbt_os_sem_init(unsigned int value) +{ + xbt_os_sem_t res = xbt_new(s_xbt_os_sem_t, 1); + + /* On some systems (MAC OS X), only the stub of sem_init is to be found. + * Any attempt to use it leads to ENOSYS (function not implemented). + * If such a prehistoric system is detected, do the job with sem_open instead + */ +#if HAVE_SEM_INIT + if (sem_init(&(res->s), 0, value) != 0) + THROWF(system_error, errno, "sem_init() failed: %s", strerror(errno)); + res->ps = &(res->s); + +#else /* damn, no sem_init(). Reimplement it */ + + xbt_os_mutex_acquire(next_sem_ID_lock); + res->name = bprintf("/sg-%d", ++next_sem_ID); + xbt_os_mutex_release(next_sem_ID_lock); + + sem_unlink(res->name); + res->ps = sem_open(res->name, O_CREAT, 0644, value); + if ((res->ps == (sem_t *) SEM_FAILED) && (errno == ENAMETOOLONG)) { + /* Old darwins only allow 13 chars. Did you create *that* amount of semaphores? */ + res->name[13] = '\0'; + sem_unlink(res->name); + res->ps = sem_open(res->name, O_CREAT, 0644, value); + } + if (res->ps == (sem_t *) SEM_FAILED) + THROWF(system_error, errno, "sem_open() failed: %s", strerror(errno)); + + /* Remove the name from the semaphore namespace: we never join on it */ + if (sem_unlink(res->name) < 0) + THROWF(system_error, errno, "sem_unlink() failed: %s", + strerror(errno)); -void xbt_os_mutex_lock(xbt_os_mutex_t mutex) { +#endif - EnterCriticalSection(& mutex->lock); + return res; } -void xbt_os_mutex_unlock(xbt_os_mutex_t mutex) { - - LeaveCriticalSection (& mutex->lock); - +void xbt_os_sem_acquire(xbt_os_sem_t sem) +{ + if (sem_wait(sem->ps) < 0) + THROWF(system_error, errno, "sem_wait() failed: %s", strerror(errno)); } -void xbt_os_mutex_destroy(xbt_os_mutex_t mutex) { - - if (!mutex) return; - - DeleteCriticalSection(& mutex->lock); - free(mutex); +void xbt_os_sem_release(xbt_os_sem_t sem) +{ + if (sem_post(sem->ps) < 0) + THROWF(system_error, errno, "sem_post() failed: %s", strerror(errno)); } -/***** condition related functions *****/ - enum { /* KEEP IT IN SYNC WITH xbt_thread.c */ - SIGNAL = 0, - BROADCAST = 1, - MAX_EVENTS = 2 - }; - -typedef struct xbt_os_cond_ { - /* KEEP IT IN SYNC WITH xbt_thread.c */ - HANDLE events[MAX_EVENTS]; - - unsigned int waiters_count; /* the number of waiters */ - CRITICAL_SECTION waiters_count_lock; /* protect access to waiters_count */ -} s_xbt_os_cond_t; +void xbt_os_sem_destroy(xbt_os_sem_t sem) +{ +#if HAVE_SEM_INIT + if (sem_destroy(sem->ps) < 0) + THROWF(system_error, errno, "sem_destroy() failed: %s", strerror(errno)); +#else + if (sem_close(sem->ps) < 0) + THROWF(system_error, errno, "sem_close() failed: %s", strerror(errno)); + xbt_free(sem->name); +#endif + xbt_free(sem); +} + +/** @brief Returns the amount of cores on the current host */ +int xbt_os_get_numcores(void) { +#ifdef WIN32 + SYSTEM_INFO sysinfo; + GetSystemInfo(&sysinfo); + return sysinfo.dwNumberOfProcessors; +#elif defined(__APPLE__) && defined(__MACH__) + int nm[2]; + size_t len = 4; + uint32_t count; + + nm[0] = CTL_HW; nm[1] = HW_AVAILCPU; + sysctl(nm, 2, &count, &len, NULL, 0); + + if(count < 1) { + nm[1] = HW_NCPU; + sysctl(nm, 2, &count, &len, NULL, 0); + if(count < 1) { count = 1; } + } + return count; +#else + return sysconf(_SC_NPROCESSORS_ONLN); +#endif +} -xbt_os_cond_t xbt_os_cond_init(void) { - - xbt_os_cond_t res = xbt_new0(s_xbt_os_cond_t,1); - - memset(& res->waiters_count_lock,0,sizeof(CRITICAL_SECTION)); - - /* initialize the critical section object */ - InitializeCriticalSection(& res->waiters_count_lock); - - res->waiters_count = 0; - - /* Create an auto-reset event */ - res->events[SIGNAL] = CreateEvent (NULL, FALSE, FALSE, NULL); - - if(!res->events[SIGNAL]){ - DeleteCriticalSection(& res->waiters_count_lock); - free(res); - THROW0(system_error,0,"CreateEvent failed for the signals"); - } - - /* Create a manual-reset event. */ - res->events[BROADCAST] = CreateEvent (NULL, TRUE, FALSE,NULL); - - if(!res->events[BROADCAST]){ - - DeleteCriticalSection(& res->waiters_count_lock); - CloseHandle(res->events[SIGNAL]); - free(res); - THROW0(system_error,0,"CreateEvent failed for the broadcasts"); - } - - return res; -} - -void xbt_os_cond_wait(xbt_os_cond_t cond, xbt_os_mutex_t mutex) { - - unsigned long wait_result; - int is_last_waiter; - - /* lock the threads counter and increment it */ - EnterCriticalSection (& cond->waiters_count_lock); - cond->waiters_count++; - LeaveCriticalSection (& cond->waiters_count_lock); - - /* unlock the mutex associate with the condition */ - LeaveCriticalSection (& mutex->lock); - - /* wait for a signal (broadcast or no) */ - wait_result = WaitForMultipleObjects (2, cond->events, FALSE, INFINITE); - - if(wait_result == WAIT_FAILED) - THROW0(system_error,0,"WaitForMultipleObjects failed, so we cannot wait on the condition"); - - /* we have a signal lock the condition */ - EnterCriticalSection (& cond->waiters_count_lock); - cond->waiters_count--; - - /* it's the last waiter or it's a broadcast ? */ - is_last_waiter = ((wait_result == WAIT_OBJECT_0 + BROADCAST - 1) && (cond->waiters_count == 0)); - - LeaveCriticalSection (& cond->waiters_count_lock); - - /* yes it's the last waiter or it's a broadcast - * only reset the manual event (the automatic event is reset in the WaitForMultipleObjects() function - * by the system. - */ - if (is_last_waiter) - if(!ResetEvent (cond->events[BROADCAST])) - THROW0(system_error,0,"ResetEvent failed"); - - /* relock the mutex associated with the condition in accordance with the posix thread specification */ - EnterCriticalSection (& mutex->lock); -} -void xbt_os_cond_timedwait(xbt_os_cond_t cond, xbt_os_mutex_t mutex, double delay) { - THROW_UNIMPLEMENTED; -} - -void xbt_os_cond_signal(xbt_os_cond_t cond) { - int have_waiters; - - EnterCriticalSection (& cond->waiters_count_lock); - have_waiters = cond->waiters_count > 0; - LeaveCriticalSection (& cond->waiters_count_lock); - - if (have_waiters) - if(!SetEvent(cond->events[SIGNAL])) - THROW0(system_error,0,"SetEvent failed"); - - xbt_os_thread_yield(); -} - -void xbt_os_cond_broadcast(xbt_os_cond_t cond){ - int have_waiters; - - EnterCriticalSection (& cond->waiters_count_lock); - have_waiters = cond->waiters_count > 0; - LeaveCriticalSection (& cond->waiters_count_lock); - - if (have_waiters) - SetEvent(cond->events[BROADCAST]); -} - -void xbt_os_cond_destroy(xbt_os_cond_t cond){ - int error = 0; - - if (!cond) return; - - if(!CloseHandle(cond->events[SIGNAL])) - error = 1; - - if(!CloseHandle(cond->events[BROADCAST])) - error = 1; - - DeleteCriticalSection(& cond->waiters_count_lock); - - xbt_free(cond); - - if (error) - THROW0(system_error,0,"Error while destroying the condition"); +void xbt_os_thread_set_extra_data(void *data) +{ + xbt_os_thread_self()->extra_data = data; } -#endif +void *xbt_os_thread_get_extra_data(void) +{ + xbt_os_thread_t thread = xbt_os_thread_self(); + return thread ? thread->extra_data : NULL; +}