Logo AND Algorithmique Numérique Distribuée

Public GIT Repository
Remove xbt_parmap_get_worker_id to simplify the parmap interface
[simgrid.git] / src / xbt / parmap.c
index f4f92cd..7bb119b 100644 (file)
 
 /* 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 "threadpool_private.h"
-
-XBT_LOG_NEW_DEFAULT_SUBCATEGORY(xbt_threadpool, xbt,
-                                "threadpool: pool of worker threads");
-
-static void *_xbt_tpool_worker_main(void *tpool);
-
-unsigned long tpoolcounter = 0;  /* Debug purposes */
-
-xbt_tpool_t xbt_tpool_new(unsigned int num_workers, unsigned int max_jobs)
+#include "gras_config.h"
+#include <unistd.h>
+
+#ifndef _XBT_WIN32
+#include <sys/syscall.h>
+#endif
+
+#ifdef HAVE_FUTEX_H
+#include <linux/futex.h>
+#endif
+
+#include "xbt/parmap.h"
+#include "xbt/log.h"
+#include "xbt/function_types.h"
+#include "xbt/dynar.h"
+#include "xbt/xbt_os_thread.h"
+#include "xbt/sysdep.h"
+
+XBT_LOG_NEW_DEFAULT_SUBCATEGORY(xbt_parmap, xbt, "parmap: parallel map");
+XBT_LOG_NEW_SUBCATEGORY(xbt_parmap_unit, xbt_parmap, "parmap unit testing");
+
+typedef enum {
+  PARMAP_WORK = 0,
+  PARMAP_DESTROY
+} e_xbt_parmap_flag_t;
+
+static void xbt_parmap_start(xbt_parmap_t parmap);
+static void xbt_parmap_signal(xbt_parmap_t parmap);
+static void xbt_parmap_wait(xbt_parmap_t parmap);
+static void xbt_parmap_end(xbt_parmap_t parmap);
+static void *xbt_parmap_worker_main(void *parmap);
+
+#ifdef HAVE_FUTEX_H
+static void futex_wait(int *uaddr, int val);
+static void futex_wake(int *uaddr, int val);
+#endif
+
+/**
+ * \brief Parallel map structure
+ */
+typedef struct s_xbt_parmap {
+  e_xbt_parmap_flag_t status;      /* is the parmap active or being destroyed? */
+
+  int work;                        /* index of the current round (1 is the first) */
+  int done;                        /* number of rounds already done */
+  unsigned int thread_counter;     /* number of threads currently working */
+  unsigned int num_workers;        /* total number of worker threads */
+  unsigned int workers_max_id;     /* id of the next worker thread to create */
+  void_f_pvoid_t fun;              /* function to run in parallel on each element of data */
+  xbt_dynar_t data;                /* parameters to pass to fun in parallel */
+  unsigned int index;              /* index of the next element of data to pick */
+} s_xbt_parmap_t;
+
+/**
+ * \brief Creates a parallel map object
+ * \param num_workers number of worker threads to create
+ * \return the parmap created
+ */
+xbt_parmap_t xbt_parmap_new(unsigned int num_workers)
 {
   unsigned int i;
   xbt_os_thread_t worker = NULL;
 
-  DEBUG2("Create new thread pool (%u, %u)", num_workers, max_jobs);
-
-  /* Initialize thread pool data structure */
-  xbt_tpool_t tpool = xbt_new0(s_xbt_tpool_t, 1);
-  tpool->mutex = xbt_os_mutex_init();
-  tpool->job_posted = xbt_os_cond_init();
-  tpool->job_taken = xbt_os_cond_init();
-  tpool->idle_worker = xbt_os_cond_init();
-  tpool->jobs_queue = xbt_dynar_new(sizeof(s_xbt_tpool_job_t), NULL);
-  tpool->num_workers = num_workers;
-  tpool->num_idle_workers = 0;
-  tpool->max_jobs = max_jobs;
-  
-  /* Create the pool of worker threads */
-  for(i=0; i < num_workers; i++){
-    worker = xbt_os_thread_create(NULL, _xbt_tpool_worker_main, tpool, NULL);
-    xbt_os_thread_detach(worker);
-  }
-  
-  return tpool;
-}
+  XBT_DEBUG("Create new parmap (%u workers)", num_workers);
 
-void xbt_tpool_destroy(xbt_tpool_t tpool)
-{ 
-  DEBUG1("Destroy thread pool %p", tpool);
+  /* Initialize the thread pool data structure */
+  xbt_parmap_t parmap = xbt_new0(s_xbt_parmap_t, 1);
 
-  /* Lock the pool, then signal every worker an wait for each to finish */
-  xbt_os_mutex_acquire(tpool->mutex);
-  tpool->flags = TPOOL_DESTROY; 
+  parmap->num_workers = num_workers;
+  parmap->status = PARMAP_WORK;
 
-  while(tpool->num_workers){
-    DEBUG1("Still %u workers, waiting...", tpool->num_workers);
-    xbt_os_cond_signal(tpool->job_posted);
-    xbt_os_cond_wait(tpool->job_taken, tpool->mutex);
+  /* Create the pool of worker threads */
+  for (i = 0; i < num_workers; i++) {
+    worker = xbt_os_thread_create(NULL, xbt_parmap_worker_main, parmap, NULL);
+    xbt_os_thread_detach(worker);
   }
-
-  /* Destroy pool's data structures */
-  xbt_os_cond_destroy(tpool->job_posted);
-  xbt_os_cond_destroy(tpool->job_taken);
-  xbt_os_cond_destroy(tpool->idle_worker);
-  xbt_os_mutex_release(tpool->mutex);
-  xbt_os_mutex_destroy(tpool->mutex);  
-  xbt_free(tpool);
+  xbt_parmap_start(parmap);
+  return parmap;
 }
 
-void xbt_tpool_queue_job(xbt_tpool_t tpool, void_f_pvoid_t fun, void* fun_arg)
+/**
+ * \brief Destroys a parmap
+ * \param parmap the parmap to destroy
+ */
+void xbt_parmap_destroy(xbt_parmap_t parmap)
 {
-  s_xbt_tpool_job_t job;
-  job.fun = fun;
-  job.fun_arg = fun_arg;
-
-  /* Wait until we can lock on the pool with some space on it for the job */
-  xbt_os_mutex_acquire(tpool->mutex);
-  while(xbt_dynar_length(tpool->jobs_queue) == tpool->max_jobs)
-    xbt_os_cond_wait(tpool->job_taken, tpool->mutex); 
-
-  DEBUG3("Queue job %p (%p) to thread pool %p", fun, fun_arg, tpool);
-
-  /* Push the job in the queue, signal the workers and unlock the pool */
-  xbt_dynar_push_as(tpool->jobs_queue, s_xbt_tpool_job_t, job);
-  xbt_os_cond_signal(tpool->job_posted);
-  xbt_os_mutex_release(tpool->mutex);    
-  return;
+  parmap->status = PARMAP_DESTROY;
+  xbt_parmap_signal(parmap);
+  xbt_free(parmap);
 }
 
-void xbt_tpool_wait_all(xbt_tpool_t tpool)
+/**
+ * \brief Applies a list of tasks in parallel.
+ * \param parmap a parallel map object
+ * \param fun the function to call in parallel
+ * \param data each element of this dynar will be passed as an argument to fun
+ */
+void xbt_parmap_apply(xbt_parmap_t parmap, void_f_pvoid_t fun, xbt_dynar_t data)
 {
-  DEBUG1("Wait all workers in thread pool %p", tpool);
-  xbt_os_mutex_acquire(tpool->mutex);
-
-  while(tpool->num_idle_workers < tpool->num_workers
-        || xbt_dynar_length(tpool->jobs_queue) > 0)
-    xbt_os_cond_wait(tpool->idle_worker, tpool->mutex);
+  /* Assign resources to worker threads */
+  parmap->fun = fun;
+  parmap->data = data;
+  parmap->index = 0;
+  xbt_parmap_signal(parmap);
+  XBT_DEBUG("Job done");
+}
 
-  xbt_os_mutex_release(tpool->mutex);
-  DEBUG1("Wait all workers done in thread pool %p", tpool);
-  return;
+/**
+ * \brief Returns a next task to process.
+ *
+ * Worker threads call this function to get more work.
+ *
+ * \return the next task to process, or NULL if there is no more work
+ */
+void* xbt_parmap_next(xbt_parmap_t parmap)
+{
+  unsigned int index = __sync_fetch_and_add(&parmap->index, 1);
+  if (index < xbt_dynar_length(parmap->data)) {
+    return xbt_dynar_get_as(parmap->data, index, void*);
+  }
+  return NULL;
 }
 
-static void *_xbt_tpool_worker_main(void *arg)
+/**
+ * \brief Main function of a worker thread.
+ * \param arg the parmap
+ */
+static void *xbt_parmap_worker_main(void *arg)
 {
-  s_xbt_tpool_job_t job;
-  xbt_tpool_t tpool = (xbt_tpool_t)arg;
+  unsigned int worker_id;
+  xbt_parmap_t parmap = (xbt_parmap_t) arg;
+
+  /* Fetch a worker id */
+  worker_id = __sync_fetch_and_add(&parmap->workers_max_id, 1);
+  xbt_os_thread_set_extra_data((void*) (unsigned long) worker_id);
 
-  unsigned long i = tpoolcounter++; /* Debug purposes */
-  DEBUG1("New worker thread created (%lu)", i);
-  
+  XBT_DEBUG("New worker thread created (%u)", worker_id);
 
   /* Worker's main loop */
-  while(1){
-    xbt_os_mutex_acquire(tpool->mutex);
+  while (1) {
+    xbt_parmap_wait(parmap);
+    if (parmap->status == PARMAP_WORK) {
 
-    tpool->num_idle_workers++;
-    xbt_os_cond_signal(tpool->idle_worker);
+      XBT_DEBUG("Worker %u got a job", worker_id);
 
-    /* If there are no jobs in the queue wait for one */
-    while(xbt_dynar_length(tpool->jobs_queue) == 0
-          && tpool->flags != TPOOL_DESTROY){
-      DEBUG1("Worker %lu waiting for a job", i);
-      xbt_os_cond_wait(tpool->job_posted, tpool->mutex);
-    }
+      void* work = xbt_parmap_next(parmap);
+      if (work != NULL) {
+        parmap->fun(work);
+      }
 
-    DEBUG1("Worker %lu got a job", i);
+      XBT_DEBUG("Worker %u has finished", worker_id);
 
-    /* If we are shutting down, signal the destroyer so it can kill the other */
-    /* workers, unlock the pool and return  */
-    if(tpool->flags == TPOOL_DESTROY){
-      DEBUG1("Shutting down worker %lu", i);
-      tpool->num_idle_workers--;
-      tpool->num_workers--;
-      xbt_os_cond_signal(tpool->job_taken);
-      xbt_os_mutex_release(tpool->mutex);
+    /* We are destroying the parmap */
+    } else {
+      xbt_parmap_end(parmap);
+      XBT_DEBUG("Shutting down worker %u", worker_id);
       return NULL;
     }
+  }
+}
+
+#ifdef HAVE_FUTEX_H
+static void futex_wait(int *uaddr, int val)
+{
+  XBT_VERB("Waiting on futex %p", uaddr);
+  syscall(SYS_futex, uaddr, FUTEX_WAIT_PRIVATE, val, NULL, NULL, 0);
+}
+
+static void futex_wake(int *uaddr, int val)
+{
+  XBT_VERB("Waking futex %p", uaddr);
+  syscall(SYS_futex, uaddr, FUTEX_WAKE_PRIVATE, val, NULL, NULL, 0);
+}
+#endif
+
+/**
+ * \brief Starts the parmap: waits for all workers to be ready and returns.
+ *
+ * This function is called by the controller thread.
+ *
+ * \param parmap a parmap
+ */
+static void xbt_parmap_start(xbt_parmap_t parmap)
+{
+#ifdef HAVE_FUTEX_H
+  int myflag = parmap->done;
+  if (parmap->thread_counter < parmap->num_workers) {
+    /* wait for all workers to be ready */
+    futex_wait(&parmap->done, myflag);
+  }
+#endif
+}
+
+/**
+ * \brief Wakes all workers and waits for them to finish the tasks.
+ *
+ * This function is called by the controller thread.
+ *
+ * \param parmap a parmap
+ */
+static void xbt_parmap_signal(xbt_parmap_t parmap)
+{
+#ifdef HAVE_FUTEX_H
+  int myflag = parmap->done;
+  parmap->thread_counter = 0;
+  parmap->work++;
+
+  /* wake all workers */
+  futex_wake(&parmap->work, parmap->num_workers);
+
+  /* wait for all of them to finish */
+  futex_wait(&parmap->done, myflag);
+#endif
+}
 
-    /* Get a job, signal the pool to inform jobs submitters and unlock it */
-    job = xbt_dynar_pop_as(tpool->jobs_queue, s_xbt_tpool_job_t);
-    xbt_os_cond_signal(tpool->job_taken);
-    tpool->num_idle_workers--;
-    xbt_os_mutex_release(tpool->mutex);
-  
-    /* Run the job and loop again ... */
-    job.fun(job.fun_arg);
+/**
+ * \brief Waits for some work to process.
+ *
+ * This function is called by each worker when it has no more work to do.
+ *
+ * \param parmap a parmap
+ */
+static void xbt_parmap_wait(xbt_parmap_t parmap)
+{
+#ifdef HAVE_FUTEX_H
+  int myflag;
+  unsigned int mycount;
+
+  myflag = parmap->work;
+  mycount = __sync_add_and_fetch(&parmap->thread_counter, 1);
+  if (mycount == parmap->num_workers) {
+    /* all workers have finished, wake the controller */
+    parmap->done++;
+    futex_wake(&parmap->done, 1);
   }
+
+  /* wait for more work */
+  futex_wait(&parmap->work, myflag);
+#endif
+}
+
+/**
+ * \brief Ends the parmap: wakes the controller thread when all workers terminate.
+ *
+ * This function is called by all worker threads when they end.
+ *
+ * \param parmap a parmap
+ */
+static void xbt_parmap_end(xbt_parmap_t parmap)
+{
+#ifdef HAVE_FUTEX_H
+  unsigned int mycount;
+
+  mycount = __sync_add_and_fetch(&parmap->thread_counter, 1);
+  if (mycount == parmap->num_workers) {
+    /* all workers have finished, wake the controller */
+    parmap->done++;
+    futex_wake(&parmap->done, 1);
+  }
+#endif
 }
 
 #ifdef SIMGRID_TEST
 #include "xbt.h"
 #include "xbt/ex.h"
 
-XBT_TEST_SUITE("tpool", "Thread pool");
+XBT_TEST_SUITE("parmap", "Parallel Map");
+XBT_LOG_EXTERNAL_DEFAULT_CATEGORY(xbt_parmap_unit);
 
-xbt_tpool_t tpool;
+xbt_parmap_t parmap;
 
-void job(void *arg);
+void fun(void *arg);
 
-void job (void *arg)
+void fun(void *arg)
 {
-  xbt_test_log1("I'm job %lu", (unsigned long)arg);
+  //XBT_INFO("I'm job %lu", (unsigned long)arg);
 }
 
-XBT_TEST_UNIT("basic", test_tpool_basic, "Basic usage")
+XBT_TEST_UNIT("basic", test_parmap_basic, "Basic usage")
 {
-  xbt_test_add0("Create thread pool");
+  xbt_test_add("Create the parmap");
 
-  unsigned long i,j;
-  /* Create thread pool */
-  tpool = xbt_tpool_new(5, 10);
+  unsigned long i, j;
+  xbt_dynar_t data = xbt_dynar_new(sizeof(void *), NULL);
 
-  for(j=0; j < 10; j++){
-    /* Queue some work */
-    for(i=0; i < 20; i++){
-      xbt_tpool_queue_job(tpool, job, (void*)i);
-    }
-    /* Wait for everyone */
-    xbt_tpool_wait_all(tpool);
+  /* Create the parallel map */
+  parmap = xbt_parmap_new(10);
+
+  for (j = 0; j < 100; j++) {
+    xbt_dynar_push_as(data, void *, (void *)j);
+  }
+
+  for (i = 0; i < 5; i++) {
+    xbt_parmap_apply(parmap, fun, data);
   }
 
-  /* Destroy thread pool */
-  xbt_tpool_destroy(tpool);
+  /* Destroy the parmap */
+  xbt_parmap_destroy(parmap);
+  xbt_dynar_free(&data);
 }
 
 #endif /* SIMGRID_TEST */