Logo AND Algorithmique Numérique Distribuée

Public GIT Repository
implement xbt_os_thread_cancel() for windows and make a little changes (test pointer...
[simgrid.git] / src / xbt / xbt_os_thread.c
index 45aa0fe..46196d6 100644 (file)
@@ -22,15 +22,21 @@ XBT_LOG_NEW_DEFAULT_SUBCATEGORY(xbt_sync_os,xbt,"Synchronization mechanism (OS-l
 
 /* ********************************* PTHREAD IMPLEMENTATION ************************************ */
 #ifdef HAVE_PTHREAD_H
+
 #include <pthread.h>
 #include <semaphore.h>
 
-/* use named sempahore instead */
-#ifndef HAVE_SEM_WAIT
-#  define MAX_SEM_NAME ((size_t)29)
-#  define MIN_SEM_NAME ((size_t)11)
-   static int __next_sem_ID = 0;
-   static pthread_mutex_t __next_sem_ID_lock;
+#ifdef HAVE_MUTEX_TIMEDLOCK
+/* redefine the function header since we fail to get this from system headers on amd (at least) */
+int pthread_mutex_timedlock(pthread_mutex_t *mutex,
+                           const struct timespec *abs_timeout);
+#endif
+  
+
+/* use named sempahore when sem_init() does not work */
+#ifndef HAVE_SEM_INIT
+  static int next_sem_ID = 0;
+  static xbt_os_mutex_t next_sem_ID_lock;
 #endif
 
 typedef struct xbt_os_thread_ {
@@ -83,17 +89,12 @@ void xbt_os_thread_mod_init(void) {
    __xbt_ex_ctx = _os_thread_ex_ctx;
    __xbt_ex_terminate = _os_thread_ex_terminate;
 
+   thread_mod_inited = 1;
+   
    #ifndef HAVE_SEM_WAIT
-
-   /* initialize the mutex use to protect the incrementation of the variable __next_sem_ID
-    * used to build the name of the named sempahore
-    */
-   if ((errcode = pthread_mutex_init(&__next_sem_ID_lock,NULL)))
-     THROW1(system_error,errcode,"pthread_mutex_init() failed: %s",strerror(errcode));
-
+   next_sem_ID_lock = xbt_os_mutex_init();
    #endif
 
-   thread_mod_inited = 1;
 }
 void xbt_os_thread_mod_exit(void) {
    /* FIXME: don't try to free our key on shutdown.
@@ -102,6 +103,13 @@ void xbt_os_thread_mod_exit(void) {
 
 //   if ((errcode=pthread_key_delete(xbt_self_thread_key)))
 //     THROW0(system_error,errcode,"pthread_key_delete failed for xbt_self_thread_key");
+   free(main_thread->exception);
+   free(main_thread);
+   main_thread = NULL;
+   thread_mod_inited=0;
+#ifndef HAVE_SEM_WAIT
+   xbt_os_mutex_destroy(next_sem_ID_lock);
+#endif
 }
 
 static void * wrapper_start_routine(void *s) {
@@ -114,6 +122,7 @@ static void * wrapper_start_routine(void *s) {
 
   return (*(t->start_routine))(t->param);
 }
+
 xbt_os_thread_t xbt_os_thread_create(const char*name,
                                     pvoid_f_pvoid_t start_routine,
                                     void* param)  {
@@ -153,6 +162,9 @@ xbt_os_thread_join(xbt_os_thread_t thread,void ** thread_return) {
    if (thread->exception)
      free(thread->exception);
 
+   if (thread->name)
+     free(thread->name);
+   
    if (thread == main_thread) /* just killed main thread */
      main_thread = NULL;
 
@@ -211,74 +223,62 @@ void xbt_os_mutex_acquire(xbt_os_mutex_t mutex) {
            mutex, strerror(errcode));
 }
 
-void xbt_os_mutex_tryacquire(xbt_os_mutex_t mutex) {
-   int errcode;
-
-   if ((errcode=pthread_mutex_trylock(&(mutex->m))))
-     THROW2(system_error,errcode,"pthread_mutex_trylock(%p) failed: %s",
-           mutex, strerror(errcode));
-}
-
-
-
-#ifndef HAVE_PTHREAD_MUTEX_TIMEDLOCK
-/* if the function is not availabled or if is MAC OS X use pthread_mutex_trylock() to define
- * it.
- */
-#include <time.h> /* declaration of the timespec structure and of the nanosleep() function */
-int pthread_mutex_timedlock(pthread_mutex_t * mutex, const struct timespec * abs_timeout)
-{
-       int rv;
-       long ellapsed_time = 0;
-       struct timespec ts;
-
-       do
-       {
-               /* the mutex could not be acquired because it was already locked by an other thread */
-               rv = pthread_mutex_trylock(mutex);
-
-               ts.tv_sec = 0;
-               ts.tv_nsec = 2.5e7 /* (25 ms (2 context switch + 5 ms)) */;
-
-               do
-               {
-                       nanosleep(&ts, &ts);
-               }while(EINTR == errno);
-
-               ellapsed_time += ts.tv_nsec;
-
-       }
-       while (/* locked */ (EBUSY == rv) && /* !timeout */ (ellapsed_time < ((long)(abs_timeout->tv_sec * 1e9) + abs_timeout->tv_nsec)));
-
-       return (EBUSY == rv) ? ETIMEDOUT : rv;
-}
-
-#endif
 
 void xbt_os_mutex_timedacquire(xbt_os_mutex_t mutex, double delay) {
    int errcode;
-   struct timespec ts_end;
-   double end = delay + xbt_os_time();
-
+       
    if (delay < 0) {
       xbt_os_mutex_acquire(mutex);
+      
+   } else if (delay == 0) {
+      errcode=pthread_mutex_trylock(&(mutex->m));
+       
+      switch (errcode) {
+       case 0:
+        return;         
+       case ETIMEDOUT:
+        THROW1(timeout_error,0,"mutex %p not ready",mutex);    
+       default:
+        THROW2(system_error,errcode,"xbt_mutex_timedacquire(%p) failed: %s",mutex, strerror(errcode));
+      }
+
+               
    } else {
+      
+#ifdef HAVE_MUTEX_TIMEDLOCK
+      struct timespec ts_end;
+      double end = delay + xbt_os_time();
+      
       ts_end.tv_sec = (time_t) floor(end);
       ts_end.tv_nsec = (long)  ( ( end - ts_end.tv_sec) * 1000000000);
       DEBUG2("pthread_mutex_timedlock(%p,%p)",&(mutex->m), &ts_end);
 
-   switch ((errcode=pthread_mutex_timedlock(&(mutex->m),&ts_end)))
-     {
-       case 0:
-               return;
-
-               case ETIMEDOUT:
-               THROW2(timeout_error,errcode,"mutex %p wasn't signaled before timeout (%f)",mutex,delay);
-
-               default:
-               THROW3(system_error,errcode,"pthread_mutex_timedlock(%p,%f) failed: %s",mutex,delay, strerror(errcode));
-               }
-    }
+      errcode=pthread_mutex_timedlock(&(mutex->m),&ts_end);
+      
+#else /* Well, let's reimplement it since those lazy libc dudes didn't */
+      double start = xbt_os_time();
+      do {
+        errcode = pthread_mutex_trylock(&(mutex->m));
+        if (errcode == EBUSY)
+          xbt_os_thread_yield();
+      } while (errcode == EBUSY && xbt_os_time()-start <delay);
+      
+      if (errcode == EBUSY)
+       errcode = ETIMEDOUT;
+      
+#endif /* HAVE_MUTEX_TIMEDLOCK */
+      
+      switch (errcode) {
+       case 0:
+        return;
+        
+       case ETIMEDOUT:
+        THROW2(timeout_error,delay,"mutex %p wasn't signaled before timeout (%f)",mutex,delay);
+        
+       default:
+        THROW3(system_error,errcode,"pthread_mutex_timedlock(%p,%f) failed: %s",mutex,delay, strerror(errcode));
+      }
+   }
 }
 
 void xbt_os_mutex_release(xbt_os_mutex_t mutex) {
@@ -378,134 +378,115 @@ void *xbt_os_thread_getparam(void) {
 }
 
 typedef struct xbt_os_sem_ {
-   #ifndef HAVE_SEM_WAIT
+   #ifndef HAVE_SEM_INIT
    char* name;
-   sem_t* s;
-   #else
-   sem_t s;
    #endif
+   sem_t s;
+   sem_t *ps;
 }s_xbt_os_sem_t ;
 
-xbt_os_sem_t
-xbt_os_sem_init(unsigned int value)
-{
-       xbt_os_sem_t res = xbt_new(s_xbt_os_sem_t,1);
-       int errcode;
-
-       /* On MAC OS X, it seems that the sem_init is failing with ENOSYS,
-        * which means the sem_init function is not implemented use sem_open()
-        * instead
-        */
-       #ifndef HAVE_SEM_INIT
-       res->name = (char*) calloc(MAX_SEM_NAME + 1,sizeof(char));
-
-       if((errcode = pthread_mutex_lock(&__next_sem_ID_lock)))
-               THROW1(system_error,errcode,"pthread_mutex_lock() failed: %s", strerror(errcode));
-
-       __next_sem_ID++;
-
-       if((errcode = pthread_mutex_unlock(&__next_sem_ID_lock)))
-               THROW1(system_error,errcode,"pthread_mutex_unlock() failed: %s", strerror(errcode));
+#ifndef SEM_FAILED
+#define SEM_FAILED (-1)
+#endif
 
-       sprintf(res->name,"/%d",__next_sem_ID);
+xbt_os_sem_t
+xbt_os_sem_init(unsigned int value) {
+   xbt_os_sem_t res = xbt_new(s_xbt_os_sem_t,1);
 
-       if((res->s = sem_open(res->name, O_CREAT | O_EXCL, 0644, value)) == (sem_t*)SEM_FAILED)
-               THROW1(system_error,errno,"sem_open() failed: %s",strerror(errno));
+   /* 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
+    */
+#ifdef HAVE_SEM_INIT
+   if(sem_init(&(res->s),0,value) != 0)
+     THROW1(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("/%d.%d",(*xbt_getpid)(),++next_sem_ID);
+   xbt_os_mutex_release(next_sem_ID_lock);
+
+   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';
+      res->ps = sem_open(res->name, O_CREAT, 0644, 1);
+   }
+   if ((res->ps == (sem_t *)SEM_FAILED))
+     THROW1(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)
+     THROW1(system_error,errno,"sem_unlink() failed: %s", strerror(errno));
 
-       #else
-       /* sem_init() is implemented, use it */
-       if(sem_init(&(res->s),0,value) < 0)
-               THROW1(system_error,errno,"sem_init() failed: %s",
-           strerror(errno));
-       #endif
+#endif
 
    return res;
 }
 
 void
-xbt_os_sem_acquire(xbt_os_sem_t sem)
-{
-       if(!sem)
-               THROW0(arg_error,EINVAL,"Cannot acquire of the NULL semaphore");
-       #ifndef HAVE_SEM_WAIT
-       if(sem_wait((sem->s)) < 0)
-               THROW1(system_error,errno,"sem_wait() failed: %s",
-           strerror(errno));
-       #else
-       if(sem_wait(&(sem->s)) < 0)
-               THROW1(system_error,errno,"sem_wait() failed: %s",
-           strerror(errno));
-       #endif
+xbt_os_sem_acquire(xbt_os_sem_t sem) {
+   if(!sem)
+     THROW0(arg_error,EINVAL,"Cannot acquire of the NULL semaphore");
+   if(sem_wait(sem->ps) < 0)
+     THROW1(system_error,errno,"sem_wait() failed: %s", strerror(errno));
 }
 
-#ifndef HAVE_SEM_TIMEDWAIT
-/* if the function is not availabled or if is MAC OS X use sem_trywait() to define
- * it.
- */
-#include <time.h> /* declaration of the timespec structure and of the nanosleep() function */
-int sem_timedwait(sem_t* sem, const struct timespec * abs_timeout)
-{
-       int rv;
-       long ellapsed_time = 0;
-       struct timespec ts;
-
-       do
-       {
-               rv = sem_trywait(sem);
-               ts.tv_sec = 0;
-               ts.tv_nsec = 2.5e7 /* (25 ms (2 * context switch + 5 ms)) */;
-
-               do
-               {
-                       nanosleep(&ts, &ts);
-               }while(EINTR == errno);
-
-               ellapsed_time += ts.tv_nsec;
+void xbt_os_sem_timedacquire(xbt_os_sem_t sem, double delay) {
+   int errcode;
 
-       }
-       while(/* locked */ (EAGAIN == rv) && /* !timeout */ ellapsed_time < ((long)(abs_timeout->tv_sec * 1e9) + abs_timeout->tv_nsec));
+   if(!sem)
+     THROW0(arg_error,EINVAL,"Cannot acquire of the NULL semaphore");
 
-       return (EAGAIN == rv) ? ETIMEDOUT : rv;
-}
+   if (delay < 0) {
+      xbt_os_sem_acquire(sem);
+   } else if (delay==0) {
+      errcode = sem_trywait(sem->ps);
 
+      switch (errcode) {
+       case 0:
+        return;         
+       case ETIMEDOUT:
+        THROW1(timeout_error,0,"semaphore %p not ready",sem);
+       default:
+        THROW2(system_error,errcode,"xbt_os_sem_timedacquire(%p) failed: %s",sem, strerror(errcode));
+      }
+      
+   } else {
+#ifdef HAVE_SEM_WAIT
+      struct timespec ts_end;
+      double end = delay + xbt_os_time();
+      
+      ts_end.tv_sec = (time_t) floor(end);
+      ts_end.tv_nsec = (long)  ( ( end - ts_end.tv_sec) * 1000000000);
+      DEBUG2("sem_timedwait(%p,%p)",sem->ps,&ts_end);
+      errcode = sem_timedwait(sem->s,&ts_end);
+      
+#else /* Okay, reimplement this function then */
+      double start = xbt_os_time();
+      do {
+        errcode = sem_trywait(sem->ps);
+        if (errcode == EBUSY)
+          xbt_os_thread_yield();
+      } while (errcode == EBUSY && xbt_os_time()-start <delay);
+      
+      if (errcode == EBUSY)
+       errcode = ETIMEDOUT;
 #endif
+      
+      switch (errcode) {
+       case 0:
+        return;
 
-void xbt_os_sem_timedacquire(xbt_os_sem_t sem,double timeout)
-{
-       int errcode;
-       struct timespec ts_end;
-       double end = timeout + xbt_os_time();
-
-       if(!sem)
-               THROW0(arg_error,EINVAL,"Cannot acquire of the NULL semaphore");
-
-       if (timeout < 0)
-       {
-               xbt_os_sem_acquire(sem);
-       }
-       else
-       {
-               ts_end.tv_sec = (time_t) floor(end);
-               ts_end.tv_nsec = (long)  ( ( end - ts_end.tv_sec) * 1000000000);
-               DEBUG2("sem_timedwait(%p,%p)",&(sem->s),&ts_end);
-
-               #ifndef HAVE_SEM_WAIT
-               switch ((errcode = sem_timedwait(sem->s,&ts_end)))
-               #else
-               switch ((errcode = sem_timedwait(&(sem->s),&ts_end)))
-               #endif
-               {
-                       case 0:
-                       return;
-
-                       case ETIMEDOUT:
-                       THROW2(timeout_error,errcode,"semaphore %p wasn't signaled before timeout (%f)",sem,timeout);
-
-                       default:
-                       THROW3(system_error,errcode,"sem_timedwait(%p,%f) failed: %s",sem,timeout, strerror(errcode));
-               }
-       }
+       case ETIMEDOUT:
+        THROW2(timeout_error,delay,"semaphore %p wasn't signaled before timeout (%f)",sem,delay);
 
+       default:
+        THROW3(system_error,errcode,"sem_timedwait(%p,%f) failed: %s",sem,delay, strerror(errcode));
+      }
+   }
 }
 
 void
@@ -514,61 +495,40 @@ xbt_os_sem_release(xbt_os_sem_t sem)
        if(!sem)
                THROW0(arg_error,EINVAL,"Cannot release of the NULL semaphore");
 
-       #ifndef HAVE_SEM_WAIT
-       if(sem_post((sem->s)) < 0)
-               THROW1(system_error,errno,"sem_post() failed: %s",
-           strerror(errno));
-       #else
-       if(sem_post(&(sem->s)) < 0)
+       if(sem_post(sem->ps) < 0)
                THROW1(system_error,errno,"sem_post() failed: %s",
            strerror(errno));
-       #endif
 }
 
 void
 xbt_os_sem_destroy(xbt_os_sem_t sem)
 {
-       if(!sem)
-               THROW0(arg_error,EINVAL,"Cannot destroy the NULL sempahore");
-
-       #ifndef HAVE_SEM_WAIT
-       /* MAC OS X does not implement the sem_init() function so,
-        * we use the named semaphore (sem_open)
-        */
-       if(sem_close((sem->s)) < 0)
-               THROW1(system_error,errno,"sem_close() failed: %s",
-           strerror(errno));
+   if(!sem)
+     THROW0(arg_error,EINVAL,"Cannot destroy the NULL sempahore");
 
-       if(sem_unlink(sem->name) < 0)
-               THROW1(system_error,errno,"sem_unlink() failed: %s",
+#ifdef HAVE_SEM_INIT
+   if(sem_destroy(sem->ps)) < 0)
+     THROW1(system_error,errno,"sem_destroy() failed: %s",
            strerror(errno));
-
-       xbt_free(sem->name);
-
-       #else
-       if(sem_destroy(&(sem->s)) < 0)
-               THROW1(system_error,errno,"sem_destroy() failed: %s",
+#else
+   if(sem_close(sem->ps) < 0)
+     THROW1(system_error,errno,"sem_close() failed: %s",
            strerror(errno));
-       #endif
+   xbt_free(sem->name);
 
-       xbt_free(sem);
+#endif
+   xbt_free(sem);
 }
 
 void
 xbt_os_sem_get_value(xbt_os_sem_t sem, int* svalue)
 {
        if(!sem)
-               THROW0(arg_error,EINVAL,"Cannot get the value of the NULL semaphore",);
+               THROW0(arg_error,EINVAL,"Cannot get the value of the NULL semaphore");
 
-       #ifndef HAVE_SEM_WAIT
-       if(sem_getvalue((sem->s),svalue) < 0)
-               THROW1(system_error,errno,"sem_getvalue() failed: %s",
-           strerror(errno));
-       #else
        if(sem_getvalue(&(sem->s),svalue) < 0)
                THROW1(system_error,errno,"sem_getvalue() failed: %s",
            strerror(errno));
-       #endif
 }
 
 /* ********************************* WINDOWS IMPLEMENTATION ************************************ */
@@ -607,14 +567,15 @@ void xbt_os_thread_mod_exit(void) {
 
 static DWORD WINAPI  wrapper_start_routine(void *s) {
   xbt_os_thread_t t = (xbt_os_thread_t)s;
-  void* rv;
+  DWORD* rv;
 
     if(!TlsSetValue(xbt_self_thread_key,t))
      THROW0(system_error,(int)GetLastError(),"TlsSetValue of data describing the created thread failed");
 
-   rv = (*(t->start_routine))(t->param);
+   rv = (DWORD*)((t->start_routine)(t->param));
 
-   return *((DWORD*)rv);
+   return rv ? *rv : 0;
+   
 }
 
 
@@ -661,7 +622,10 @@ xbt_os_thread_join(xbt_os_thread_t thread,void ** thread_return) {
        }
 
        CloseHandle(thread->handle);
-       free(thread->name);
+       
+       if(thread->name)
+               free(thread->name);
+       
        free(thread);
 }
 
@@ -686,7 +650,8 @@ void xbt_os_thread_yield(void) {
     Sleep(0);
 }
 void xbt_os_thread_cancel(xbt_os_thread_t t) {
-   THROW_UNIMPLEMENTED;
+  if(!TerminateThread(t->handle,0))
+               THROW0(system_error,(int)GetLastError(), "TerminateThread failed");
 }
 
 /****** mutex related functions ******/
@@ -705,22 +670,16 @@ xbt_os_mutex_t xbt_os_mutex_init(void) {
 }
 
 void xbt_os_mutex_acquire(xbt_os_mutex_t mutex) {
-
    EnterCriticalSection(& mutex->lock);
 }
 
-void xbt_os_mutex_tryacquire(xbt_os_mutex_t mutex)
-{
-       TryEnterCriticalSection(&mutex->lock);
-}
-
 void xbt_os_mutex_timedacquire(xbt_os_mutex_t mutex, double delay) {
        THROW_UNIMPLEMENTED;
 }
 
 void xbt_os_mutex_release(xbt_os_mutex_t mutex) {
 
-   LeaveCriticalSection (& mutex->lock);
+   LeaveCriticalSection (&mutex->lock);
 
 }
 
@@ -822,7 +781,7 @@ void xbt_os_cond_wait(xbt_os_cond_t cond, xbt_os_mutex_t mutex) {
 }
 void xbt_os_cond_timedwait(xbt_os_cond_t cond, xbt_os_mutex_t mutex, double delay) {
 
-        unsigned long wait_result = WAIT_TIMEOUT;
+   unsigned long wait_result = WAIT_TIMEOUT;
    int is_last_waiter;
    unsigned long end = (unsigned long)(delay * 1000);
 
@@ -830,7 +789,7 @@ void xbt_os_cond_timedwait(xbt_os_cond_t cond, xbt_os_mutex_t mutex, double dela
    if (delay < 0) {
       xbt_os_cond_wait(cond,mutex);
    } else {
-         DEBUG3("xbt_cond_timedwait(%p,%p,%ul)",&(cond->events),&(mutex->lock),end);
+         DEBUG3("xbt_cond_timedwait(%p,%p,%lu)",&(cond->events),&(mutex->lock),end);
 
    /* lock the threads counter and increment it */
    EnterCriticalSection (& cond->waiters_count_lock);
@@ -923,6 +882,10 @@ typedef struct xbt_os_sem_ {
    CRITICAL_SECTION value_lock;  /* protect access to value of the semaphore  */
 }s_xbt_os_sem_t ;
 
+#ifndef INT_MAX
+# define INT_MAX 32767 /* let's be safe by underestimating this value: this is for 16bits only */
+#endif
+
 xbt_os_sem_t
 xbt_os_sem_init(unsigned int value)
 {
@@ -974,7 +937,7 @@ void xbt_os_sem_timedacquire(xbt_os_sem_t sem, double timeout)
        {
                xbt_os_sem_acquire(sem);
        }
-       else
+       else /* timeout can be zero <-> try acquire ) */
        {
 
                seconds = (long) floor(end);
@@ -994,7 +957,6 @@ void xbt_os_sem_timedacquire(xbt_os_sem_t sem, double timeout)
                        return;
 
                        default:
-
                        THROW3(system_error,GetLastError(),"WaitForSingleObject(%p,%f) failed: %s",sem,timeout, strerror(GetLastError()));
                }
        }