Logo AND Algorithmique Numérique Distribuée

Public GIT Repository
Transform the thread pool into a parallel map data structure.
authorcristianrosa <cristianrosa@48e7efb5-ca39-0410-a469-dd3cf9ba447f>
Mon, 13 Dec 2010 08:58:18 +0000 (08:58 +0000)
committercristianrosa <cristianrosa@48e7efb5-ca39-0410-a469-dd3cf9ba447f>
Mon, 13 Dec 2010 08:58:18 +0000 (08:58 +0000)
It better suits our needs.

git-svn-id: svn+ssh://scm.gforge.inria.fr/svn/simgrid/simgrid/trunk@9142 48e7efb5-ca39-0410-a469-dd3cf9ba447f

buildtools/Cmake/DefinePackages.cmake
buildtools/Cmake/UnitTesting.cmake
include/xbt.h
include/xbt/parmap.h
src/xbt/parmap.c
src/xbt/parmap_private.h

index 197126e..e8726bf 100644 (file)
@@ -17,7 +17,7 @@ set(EXTRA_DIST
        src/xbt/backtrace_windows.c
        src/xbt/backtrace_dummy.c
        src/xbt/setset_private.h
-       src/xbt/threadpool_private.h
+       src/xbt/parmap_private.h
        src/xbt/mmalloc/attach.c
        src/xbt/mmalloc/detach.c        
        src/xbt/mmalloc/keys.c
@@ -164,7 +164,7 @@ set(XBT_SRC
        src/xbt/cunit.c
        src/xbt/graphxml_parse.c
        src/xbt/setset.c
-       src/xbt/threadpool.c
+       src/xbt/parmap.c
 )
 
 if(HAVE_MMAP)
index 327c252..51a4695 100644 (file)
@@ -13,7 +13,7 @@ set(TEST_CFILES
        src/xbt/xbt_sha.c
        src/xbt/config.c
        src/xbt/xbt_synchro.c
-       src/xbt/threadpool.c
+       src/xbt/parmap.c
 )
 set(TEST_UNITS
        ${CMAKE_CURRENT_BINARY_DIR}/src/cunit_unit.c
@@ -27,7 +27,7 @@ set(TEST_UNITS
        ${CMAKE_CURRENT_BINARY_DIR}/src/xbt_sha_unit.c
        ${CMAKE_CURRENT_BINARY_DIR}/src/config_unit.c
        ${CMAKE_CURRENT_BINARY_DIR}/src/xbt_synchro_unit.c
-       ${CMAKE_CURRENT_BINARY_DIR}/src/threadpool_unit.c
+       ${CMAKE_CURRENT_BINARY_DIR}/src/parmap_unit.c
        
        ${CMAKE_CURRENT_BINARY_DIR}/src/simgrid_units_main.c
 )
index c222c57..ec288fe 100644 (file)
@@ -26,7 +26,7 @@
 #include <xbt/swag.h>
 #include <xbt/heap.h>
 #include <xbt/queue.h>
-#include <xbt/threadpool.h>
+#include <xbt/parmap.h>
 
 #include <xbt/peer.h>
 #include <xbt/config.h>
index e941c9e..f661994 100644 (file)
@@ -6,41 +6,37 @@
 /* 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. */
 
-#ifndef _XBT_THREADPOOL_H
-#define _XBT_THREADPOOL_H
+#ifndef _XBT_PARMAP_H
+#define _XBT_PARMAP_H
 
 #include "xbt/misc.h"           /* SG_BEGIN_DECL */
 #include "xbt/function_types.h"
+#include "xbt/dynar.h"
 
 SG_BEGIN_DECL()
 
-/** @addtogroup XBT_threadpool
-  * @brief Pool of threads.
+/** @addtogroup XBT_parmap
+  * @brief Parallel map.
   *
-  * Jobs can be queued and the dispacher process can wait for the completion
-  * of all jobs.
-  * The call to "queue job" is non-blocking except the maximum amount of
-  * queued jobs is reached. In that case, it will block until a job is taken
-  * by a worker
+  * A function is applied to all the elements of a dynar in parallel
+  * using threads. The threads are persistent until the destruction
+  * of the parmap object.
   * @{
   */
   /** \brief Queue data type (opaque type) */
 
-typedef struct s_xbt_tpool *xbt_tpool_t;
+typedef struct s_xbt_parmap *xbt_parmap_t;
 
-XBT_PUBLIC(xbt_tpool_t) xbt_tpool_new(unsigned int num_workers,
-                                      unsigned int max_jobs);
+XBT_PUBLIC(xbt_parmap_t) xbt_parmap_new(unsigned int num_workers);
 
-XBT_PUBLIC(void) xbt_tpool_queue_job(xbt_tpool_t tpool, 
-                                     void_f_pvoid_t fun, 
-                                     void* fun_arg);
+XBT_PUBLIC(void) xbt_parmap_apply(xbt_parmap_t parmap,
+                                  void_f_pvoid_t fun,
+                                  xbt_dynar_t data);
 
-XBT_PUBLIC(void) xbt_tpool_wait_all(xbt_tpool_t tpool);
-
-XBT_PUBLIC(void) xbt_tpool_destroy(xbt_tpool_t tpool);
+XBT_PUBLIC(void) xbt_parmap_destroy(xbt_parmap_t parmap);
 
 /** @} */
 
 SG_END_DECL()
 
-#endif
\ No newline at end of file
+#endif
index f4f92cd..a11f84e 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"
+#include "parmap_private.h"
 
-XBT_LOG_NEW_DEFAULT_SUBCATEGORY(xbt_threadpool, xbt,
-                                "threadpool: pool of worker threads");
+XBT_LOG_NEW_DEFAULT_SUBCATEGORY(xbt_parmap, xbt,
+                                "parmap: parallel map");
 
-static void *_xbt_tpool_worker_main(void *tpool);
+static void *_xbt_parmap_worker_main(void *parmap);
 
-unsigned long tpoolcounter = 0;  /* Debug purposes */
-
-xbt_tpool_t xbt_tpool_new(unsigned int num_workers, unsigned int max_jobs)
+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);
+  DEBUG1("Create new parmap (%u workers)", num_workers);
 
   /* 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;
-  
+  xbt_parmap_t parmap = xbt_new0(s_xbt_parmap_t, 1);
+  parmap->mutex = xbt_os_mutex_init();
+  parmap->job_posted = xbt_os_cond_init();
+  parmap->all_done = xbt_os_cond_init();
+  parmap->flags = xbt_new0(e_xbt_parmap_flag_t, num_workers + 1);
+  parmap->num_workers = num_workers;
+  parmap->num_idle_workers = 0;
+  parmap->workers_max_id = 0;
+
+  /* Init our flag to wait (for workers' initialization) */
+  parmap->flags[num_workers] = PARMAP_WAIT;
+
   /* 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);
+    worker = xbt_os_thread_create(NULL, _xbt_parmap_worker_main, parmap, NULL);
     xbt_os_thread_detach(worker);
   }
   
-  return tpool;
+  /* wait for the workers to initialize */
+  xbt_os_mutex_acquire(parmap->mutex);
+  while(parmap->flags[num_workers] == PARMAP_WAIT)
+    xbt_os_cond_wait(parmap->all_done, parmap->mutex);
+  xbt_os_mutex_release(parmap->mutex);
+
+  return parmap;
 }
 
-void xbt_tpool_destroy(xbt_tpool_t tpool)
+void xbt_parmap_destroy(xbt_parmap_t parmap)
 { 
-  DEBUG1("Destroy thread pool %p", tpool);
+  DEBUG1("Destroy parmap %p", parmap);
 
-  /* Lock the pool, then signal every worker an wait for each to finish */
-  xbt_os_mutex_acquire(tpool->mutex);
-  tpool->flags = TPOOL_DESTROY; 
+  unsigned int i;
+
+  /* Lock the parmap, then signal every worker an wait for each to finish */
+  xbt_os_mutex_acquire(parmap->mutex);
+  for(i=0; i < parmap->num_workers; i++){
+    parmap->flags[i] = PARMAP_DESTROY;
+  }
 
-  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);
+  xbt_os_cond_broadcast(parmap->job_posted);
+  while(parmap->num_workers){
+    DEBUG1("Still %u workers, waiting...", parmap->num_workers);
+    xbt_os_cond_wait(parmap->all_done, parmap->mutex);
   }
 
   /* 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_os_cond_destroy(parmap->job_posted);
+  xbt_os_cond_destroy(parmap->all_done);
+  xbt_free(parmap->flags);
+  xbt_os_mutex_release(parmap->mutex);
+  xbt_os_mutex_destroy(parmap->mutex);
+  xbt_free(parmap);
 }
 
-void xbt_tpool_queue_job(xbt_tpool_t tpool, void_f_pvoid_t fun, void* fun_arg)
+void xbt_parmap_apply(xbt_parmap_t parmap, void_f_pvoid_t fun, xbt_dynar_t data)
 {
-  s_xbt_tpool_job_t job;
-  job.fun = fun;
-  job.fun_arg = fun_arg;
+  unsigned int i;
+  unsigned int myflag_idx = parmap->num_workers;
 
-  /* 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); 
+  /* Assign resources to worker threads */
+  xbt_os_mutex_acquire(parmap->mutex);
+  parmap->fun = fun;
+  parmap->data = data;
+  parmap->num_idle_workers = 0;
 
-  DEBUG3("Queue job %p (%p) to thread pool %p", fun, fun_arg, tpool);
+  /* Set worker flags to work */
+  for(i=0; i < parmap->num_workers; i++){
+    parmap->flags[i] = PARMAP_WORK;
+  }
 
-  /* 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;
-}
+  /* Set our flag to wait (for the job to be completed)*/
+  parmap->flags[myflag_idx] = PARMAP_WAIT;
 
-void xbt_tpool_wait_all(xbt_tpool_t tpool)
-{
-  DEBUG1("Wait all workers in thread pool %p", tpool);
-  xbt_os_mutex_acquire(tpool->mutex);
+  /* Notify workers that there is a job */
+  xbt_os_cond_broadcast(parmap->job_posted);
+  DEBUG0("Job dispatched, lets wait...");
 
-  while(tpool->num_idle_workers < tpool->num_workers
-        || xbt_dynar_length(tpool->jobs_queue) > 0)
-    xbt_os_cond_wait(tpool->idle_worker, tpool->mutex);
+  /* wait for the workers to finish */
+  while(parmap->flags[myflag_idx] == PARMAP_WAIT)
+    xbt_os_cond_wait(parmap->all_done, parmap->mutex);
 
-  xbt_os_mutex_release(tpool->mutex);
-  DEBUG1("Wait all workers done in thread pool %p", tpool);
+  DEBUG0("Job done");
+  parmap->fun = NULL;
+  parmap->data = NULL;
+
+  xbt_os_mutex_release(parmap->mutex);    
   return;
 }
 
-static void *_xbt_tpool_worker_main(void *arg)
+static void *_xbt_parmap_worker_main(void *arg)
 {
-  s_xbt_tpool_job_t job;
-  xbt_tpool_t tpool = (xbt_tpool_t)arg;
+  unsigned int data_start, data_end, data_size, worker_id;
+  xbt_parmap_t parmap = (xbt_parmap_t)arg;
 
-  unsigned long i = tpoolcounter++; /* Debug purposes */
-  DEBUG1("New worker thread created (%lu)", i);
-  
+  /* Fetch a worker id */
+  xbt_os_mutex_acquire(parmap->mutex);
+  worker_id = parmap->workers_max_id++;
+  xbt_os_mutex_release(parmap->mutex);
 
+  DEBUG1("New worker thread created (%u)", worker_id);
+  
   /* Worker's main loop */
   while(1){
-    xbt_os_mutex_acquire(tpool->mutex);
-
-    tpool->num_idle_workers++;
-    xbt_os_cond_signal(tpool->idle_worker);
-
-    /* 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);
+    xbt_os_mutex_acquire(parmap->mutex);
+    parmap->flags[worker_id] = PARMAP_WAIT;
+    parmap->num_idle_workers++;
+
+    /* If everybody is done set the parmap work flag and signal it */
+    if(parmap->num_idle_workers == parmap->num_workers){
+      DEBUG1("Worker %u: All done, signal the parmap", worker_id);
+      parmap->flags[parmap->num_workers] = PARMAP_WORK;
+      xbt_os_cond_signal(parmap->all_done);
     }
 
-    DEBUG1("Worker %lu got a job", i);
+    /* If the wait flag is set then ... wait. */
+    while(parmap->flags[worker_id] == PARMAP_WAIT)
+      xbt_os_cond_wait(parmap->job_posted, parmap->mutex);
 
-    /* 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);
+    DEBUG1("Worker %u got a job", worker_id);
+
+    /* If we are shutting down, the last worker is going to signal the
+     * parmap so it can finish destroying the data structure */
+    if(parmap->flags[worker_id] == PARMAP_DESTROY){
+      DEBUG1("Shutting down worker %u", worker_id);
+      parmap->num_workers--;
+      if(parmap->num_workers == 0)
+        xbt_os_cond_signal(parmap->all_done);
+      xbt_os_mutex_release(parmap->mutex);
       return NULL;
     }
+    xbt_os_mutex_release(parmap->mutex);
 
-    /* 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);
+    /* Compute how much data does every worker gets */
+    data_size = (xbt_dynar_length(parmap->data) / parmap->num_workers)
+                + ((xbt_dynar_length(parmap->data) % parmap->num_workers) ? 1 : 0);
+
+    /* Each worker data segment starts in a position associated with its id*/
+    data_start = data_size * worker_id;
+
+    /* The end of the worker data segment must be bounded by the end of the data vector */
+    data_end = MIN(data_start + data_size, xbt_dynar_length(parmap->data));
+
+    DEBUG4("Worker %u: data_start=%u data_end=%u (data_size=%u)", worker_id, data_start, data_end, data_size);
+
+    /* While the worker don't pass the end of it data segment apply the function */
+    while(data_start < data_end){
+      parmap->fun(*(void **)xbt_dynar_get_ptr(parmap->data, data_start));
+      data_start++;
+    }
   }
 }
 
@@ -148,36 +175,39 @@ static void *_xbt_tpool_worker_main(void *arg)
 #include "xbt.h"
 #include "xbt/ex.h"
 
-XBT_TEST_SUITE("tpool", "Thread pool");
+XBT_TEST_SUITE("parmap", "Parallel Map");
+
+XBT_LOG_NEW_DEFAULT_SUBCATEGORY(xbt_parmap_unit, xbt,
+                                "unit parmap");
+
 
-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);
+  INFO1("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_add0("Create the parmap");
 
-  unsigned long i,j;
-  /* Create thread pool */
-  tpool = xbt_tpool_new(5, 10);
+  unsigned long 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(5);
+
+  for(j=0; j < 200; j++){
+    xbt_dynar_push_as(data, void *, (void *)j);
   }
 
-  /* Destroy thread pool */
-  xbt_tpool_destroy(tpool);
+  xbt_parmap_apply(parmap, fun, data);
+
+  /* Destroy the parmap */
+  xbt_parmap_destroy(parmap);
 }
 
 #endif /* SIMGRID_TEST */
index e51a27f..283b025 100644 (file)
@@ -7,32 +7,28 @@
 #ifndef _XBT_THREADPOOL_PRIVATE_H
 #define _XBT_THREADPOOL_PRIVATE_H
 
-#include "xbt/threadpool.h"
+#include "xbt/parmap.h"
 #include "xbt/xbt_os_thread.h"
 #include "xbt/sysdep.h"
 #include "xbt/dynar.h"
 #include "xbt/log.h"
 
 typedef enum{
-  TPOOL_WAIT = 0,
-  TPOOL_DESTROY
-} e_xbt_tpool_flag_t;
+  PARMAP_WAIT = 0,
+  PARMAP_WORK,
+  PARMAP_DESTROY,
+} e_xbt_parmap_flag_t;
 
-typedef struct s_xbt_tpool {
+typedef struct s_xbt_parmap {
   xbt_os_mutex_t mutex;           /* pool's mutex */
   xbt_os_cond_t job_posted;       /* job is posted */
-  xbt_os_cond_t job_taken;        /* job is taken */
-  xbt_os_cond_t idle_worker;      /* job is done */
-  xbt_dynar_t jobs_queue;
-  e_xbt_tpool_flag_t flags;
+  xbt_os_cond_t all_done;         /* job is done */
+  e_xbt_parmap_flag_t *flags;     /* Per thread flag + lastone for the parmap */
   unsigned int num_workers;
   unsigned int num_idle_workers;
-  unsigned int max_jobs;
-} s_xbt_tpool_t;
-
-typedef struct s_xbt_tpool_job {
+  unsigned int workers_max_id;
   void_f_pvoid_t fun;
-  void *fun_arg;
-} s_xbt_tpool_job_t, *xbt_tpool_job_t;
+  xbt_dynar_t data;
+} s_xbt_parmap_t;
 
 #endif