Browse Source

prefix more internal functions with _starpu

Cédric Augonnet 15 years ago
parent
commit
95659a6ed2
42 changed files with 212 additions and 212 deletions
  1. 1 1
      mpi/starpu_mpi.c
  2. 8 8
      src/common/timing.c
  3. 1 1
      src/common/timing.h
  4. 3 3
      src/core/debug.c
  5. 3 3
      src/core/debug.h
  6. 2 2
      src/core/dependencies/data-concurrency.c
  7. 1 1
      src/core/dependencies/data-concurrency.h
  8. 15 15
      src/core/dependencies/tags.c
  9. 3 3
      src/core/dependencies/tags.h
  10. 6 6
      src/core/errorcheck.c
  11. 3 3
      src/core/errorcheck.h
  12. 6 6
      src/core/jobs.c
  13. 1 1
      src/core/mechanisms/deque_queues.c
  14. 2 2
      src/core/mechanisms/priority_queues.c
  15. 1 1
      src/core/perfmodel/perfmodel.h
  16. 9 9
      src/core/perfmodel/perfmodel_bus.c
  17. 1 1
      src/core/perfmodel/perfmodel_history.c
  18. 2 2
      src/core/policies/sched_policy.c
  19. 2 2
      src/core/progress_hook.c
  20. 5 5
      src/core/task.c
  21. 24 24
      src/core/topology.c
  22. 2 2
      src/core/topology.h
  23. 35 35
      src/core/workers.c
  24. 12 12
      src/core/workers.h
  25. 4 4
      src/datawizard/coherency.c
  26. 1 1
      src/datawizard/coherency.h
  27. 3 3
      src/datawizard/copy-driver.c
  28. 2 2
      src/datawizard/copy-driver.h
  29. 3 3
      src/datawizard/data_request.c
  30. 2 2
      src/datawizard/memalloc.c
  31. 3 3
      src/datawizard/memory_nodes.c
  32. 1 1
      src/datawizard/memory_nodes.h
  33. 2 2
      src/datawizard/progress.c
  34. 1 1
      src/datawizard/progress.h
  35. 2 2
      src/datawizard/user_interactions.c
  36. 1 1
      src/datawizard/write_back.c
  37. 10 10
      src/drivers/core/driver_core.c
  38. 1 1
      src/drivers/core/driver_core.h
  39. 12 12
      src/drivers/cuda/driver_cuda.c
  40. 2 2
      src/drivers/cuda/driver_cuda.h
  41. 12 12
      src/drivers/gordon/driver_gordon.c
  42. 2 2
      src/util/malloc.c

+ 1 - 1
mpi/starpu_mpi.c

@@ -472,7 +472,7 @@ static void handle_new_request(struct starpu_mpi_req_s *req)
 		starpu_mpi_req_list_push_front(detached_requests, req);
 		pthread_mutex_unlock(&mutex);
 
-		wake_all_blocked_workers();
+		starpu_wake_all_blocked_workers();
 
 		/* put the submitted request into the list of pending requests
 		 * so that it can be handled by the progression mechanisms */

+ 8 - 8
src/common/timing.c

@@ -20,13 +20,13 @@
 
 #define TICK_DIFF(t1, t2) ((long long)((t2).ts.tv_sec*1e9 + (t2).ts.tv_nsec) + \
 				- (long long)((t1).ts.tv_sec*1e9) + (long long)(t1).ts.tv_nsec)
-#define TIMING_DELAY(t1, t2) tick2usec(TICK_DIFF((t1), (t2)))
+#define TIMING_DELAY(t1, t2) _starpu_tick2usec(TICK_DIFF((t1), (t2)))
 
 void timing_init(void)
 {
 }
 
-inline double tick2usec(long long t)
+inline double _starpu_tick2usec(long long t)
 {
   return (double)(t)/1000;
 }
@@ -35,8 +35,8 @@ inline double timing_delay(tick_t *t1, tick_t *t2)
 {
 	double d1, d2;
 
-	d1 = tick2usec((t1->ts.tv_sec*1e9) + t1->ts.tv_nsec);
-	d2 = tick2usec((t2->ts.tv_sec*1e9) + t2->ts.tv_nsec);
+	d1 = _starpu_tick2usec((t1->ts.tv_sec*1e9) + t1->ts.tv_nsec);
+	d2 = _starpu_tick2usec((t2->ts.tv_sec*1e9) + t2->ts.tv_nsec);
 
 	return (d2 - d1);;
 }
@@ -47,7 +47,7 @@ inline double timing_now(void)
 	tick_t tick_now;
 	GET_TICK(tick_now);
 
-	return tick2usec(((tick_now).ts.tv_sec*1e9) + (tick_now).ts.tv_nsec);
+	return _starpu_tick2usec(((tick_now).ts.tv_sec*1e9) + (tick_now).ts.tv_nsec);
 }
 
 
@@ -56,7 +56,7 @@ inline double timing_now(void)
 
 #define TICK_RAW_DIFF(t1, t2) ((t2).tick - (t1).tick)
 #define TICK_DIFF(t1, t2) (TICK_RAW_DIFF(t1, t2) - residual)
-#define TIMING_DELAY(t1, t2) tick2usec(TICK_DIFF(t1, t2))
+#define TIMING_DELAY(t1, t2) _starpu_tick2usec(TICK_DIFF(t1, t2))
 
 static double scale = 0.0;
 static unsigned long long residual = 0;
@@ -95,7 +95,7 @@ void timing_init(void)
   inited = 1;
 }
 
-inline double tick2usec(long long t)
+inline double _starpu_tick2usec(long long t)
 {
   return (double)(t)*scale;
 }
@@ -110,7 +110,7 @@ inline double timing_now(void)
 	tick_t tick_now;
 	GET_TICK(tick_now);
 
-	return tick2usec(tick_now.tick);
+	return _starpu_tick2usec(tick_now.tick);
 }
 
 #endif // USE_SYNC_CLOCK

+ 1 - 1
src/common/timing.h

@@ -72,7 +72,7 @@ typedef union u_tick
 #endif // USE_SYNC_CLOCK
 
 void __attribute__ ((unused)) timing_init(void);
-inline double __attribute__ ((unused)) tick2usec(long long t);
+inline double __attribute__ ((unused)) _starpu_tick2usec(long long t);
 inline double __attribute__ ((unused)) timing_delay(tick_t *t1, tick_t *t2);
 
 inline double __attribute__ ((unused)) timing_now(void);

+ 3 - 3
src/core/debug.c

@@ -23,7 +23,7 @@ static pthread_mutex_t logfile_mutex = PTHREAD_MUTEX_INITIALIZER;
 static FILE *logfile;
 #endif
 
-void open_debug_logfile(void)
+void _starpu_open_debug_logfile(void)
 {
 #ifdef VERBOSE
 	/* what is  the name of the file ? default = "starpu.log" */
@@ -40,14 +40,14 @@ void open_debug_logfile(void)
 #endif
 }
 
-void close_debug_logfile(void)
+void _starpu_close_debug_logfile(void)
 {
 #ifdef VERBOSE
 	fclose(logfile);
 #endif
 }
 
-void print_to_logfile(const char *format __attribute__((unused)), ...)
+void _starpu_print_to_logfile(const char *format __attribute__((unused)), ...)
 {
 #ifdef VERBOSE
 	va_list args;

+ 3 - 3
src/core/debug.h

@@ -24,9 +24,9 @@
 #include <common/config.h>
 #include <core/workers.h>
 
-void open_debug_logfile(void);
-void close_debug_logfile(void);
+void _starpu_open_debug_logfile(void);
+void _starpu_close_debug_logfile(void);
 
-void print_to_logfile(const char *format, ...);
+void _starpu_print_to_logfile(const char *format, ...);
 
 #endif // __DEBUG_H__

+ 2 - 2
src/core/dependencies/data-concurrency.c

@@ -118,7 +118,7 @@ static unsigned attempt_to_submit_data_request_from_job(job_t j, unsigned buffer
 	starpu_access_mode mode = j->task->buffers[buffer_index].mode;
 
 	while (starpu_spin_trylock(&handle->header_lock))
-		datawizard_progress(get_local_memory_node(), 0);
+		_starpu_datawizard_progress(get_local_memory_node(), 0);
 
 	if (handle->refcnt == 0)
 	{
@@ -182,7 +182,7 @@ static unsigned _submit_job_enforce_data_deps(job_t j, unsigned start_buffer_ind
    with concurrent data-access at the same time in the scheduling engine (eg.
    there can be 2 tasks reading a piece of data, but there cannot be one
    reading and another writing) */
-unsigned submit_job_enforce_data_deps(job_t j)
+unsigned _starpu_submit_job_enforce_data_deps(job_t j)
 {
 	if ((j->task->cl == NULL) || (j->task->cl->nbuffers == 0))
 		return 0;

+ 1 - 1
src/core/dependencies/data-concurrency.h

@@ -19,7 +19,7 @@
 
 #include <core/jobs.h>
 
-unsigned submit_job_enforce_data_deps(job_t j);
+unsigned _starpu_submit_job_enforce_data_deps(job_t j);
 
 void notify_data_dependencies(starpu_data_handle handle);
 

+ 15 - 15
src/core/dependencies/tags.c

@@ -53,7 +53,7 @@ static cg_t *create_cg(unsigned ntags, struct tag_s *tag, unsigned is_apps_cg)
 	return cg;
 }
 
-static struct tag_s *tag_init(starpu_tag_t id)
+static struct tag_s *_starpu_tag_init(starpu_tag_t id)
 {
 	struct tag_s *tag;
 	tag = malloc(sizeof(struct tag_s));
@@ -129,7 +129,7 @@ static struct tag_s *gettag_struct(starpu_tag_t id)
 
 	if (tag == NULL) {
 		/* the tag does not exist yet : create an entry */
-		tag = tag_init(id);
+		tag = _starpu_tag_init(id);
 
 		void *old;
 		old = htbl_insert_tag(&tag_htbl, id, tag);
@@ -143,7 +143,7 @@ static struct tag_s *gettag_struct(starpu_tag_t id)
 }
 
 /* lock should be taken */
-static void tag_set_ready(struct tag_s *tag)
+static void _starpu_tag_set_ready(struct tag_s *tag)
 {
 	/* mark this tag as ready to run */
 	tag->state = READY;
@@ -157,7 +157,7 @@ static void tag_set_ready(struct tag_s *tag)
 	starpu_spin_unlock(&tag->lock);
 
 	/* enforce data dependencies */
-	if (submit_job_enforce_data_deps(j))
+	if (_starpu_submit_job_enforce_data_deps(j))
 	{
 		starpu_spin_lock(&tag->lock);
 		return;
@@ -192,14 +192,14 @@ static void notify_cg(cg_t *cg)
 			if ((tag->state == BLOCKED) 
 				&& (tag->ndeps == tag->ndeps_completed)) {
 				tag->ndeps_completed = 0;
-				tag_set_ready(tag);
+				_starpu_tag_set_ready(tag);
 			}
 		}
 	}
 }
 
 /* the lock must be taken ! */
-static void tag_add_succ(struct tag_s *tag, cg_t *cg)
+static void _starpu_tag_add_succ(struct tag_s *tag, cg_t *cg)
 {
 	STARPU_ASSERT(tag);
 
@@ -227,7 +227,7 @@ static void tag_add_succ(struct tag_s *tag, cg_t *cg)
 	}
 }
 
-static void notify_tag_dependencies(struct tag_s *tag)
+static void _starpu_notify_tag_dependencies(struct tag_s *tag)
 {
 	unsigned nsuccs;
 	unsigned succ;
@@ -265,24 +265,24 @@ static void notify_tag_dependencies(struct tag_s *tag)
 	starpu_spin_unlock(&tag->lock);
 }
 
-void notify_dependencies(struct job_s *j)
+void _starpu_notify_dependencies(struct job_s *j)
 {
 	STARPU_ASSERT(j);
 	STARPU_ASSERT(j->task);
 	
 	/* in case there are dependencies, wake up the proper tasks */
 	if (j->task->use_tag)
-		notify_tag_dependencies(j->tag);
+		_starpu_notify_tag_dependencies(j->tag);
 }
 
 void starpu_tag_notify_from_apps(starpu_tag_t id)
 {
 	struct tag_s *tag = gettag_struct(id);
 
-	notify_tag_dependencies(tag);
+	_starpu_notify_tag_dependencies(tag);
 }
 
-void tag_declare(starpu_tag_t id, struct job_s *job)
+void _starpu_tag_declare(starpu_tag_t id, struct job_s *job)
 {
 	TRACE_CODELET_TAG(id, job);
 	job->task->use_tag = 1;
@@ -321,7 +321,7 @@ void starpu_tag_declare_deps_array(starpu_tag_t id, unsigned ndeps, starpu_tag_t
 		TRACE_CODELET_TAG_DEPS(id, dep_id);
 		struct tag_s *tag_dep = gettag_struct(dep_id);
 		starpu_spin_lock(&tag_dep->lock);
-		tag_add_succ(tag_dep, cg);
+		_starpu_tag_add_succ(tag_dep, cg);
 		starpu_spin_unlock(&tag_dep->lock);
 	}
 
@@ -353,7 +353,7 @@ void starpu_tag_declare_deps(starpu_tag_t id, unsigned ndeps, ...)
 		TRACE_CODELET_TAG_DEPS(id, dep_id);
 		struct tag_s *tag_dep = gettag_struct(dep_id);
 		starpu_spin_lock(&tag_dep->lock);
-		tag_add_succ(tag_dep, cg);
+		_starpu_tag_add_succ(tag_dep, cg);
 		starpu_spin_unlock(&tag_dep->lock);
 	}
 	va_end(pa);
@@ -370,7 +370,7 @@ int starpu_tag_wait_array(unsigned ntags, starpu_tag_t *id)
 	struct tag_s *tag_array[ntags];
 
 	/* It is forbidden to block within callbacks or codelets */
-	if (STARPU_UNLIKELY(!worker_may_perform_blocking_calls()))
+	if (STARPU_UNLIKELY(!_starpu_worker_may_perform_blocking_calls()))
 		return -EDEADLK;
 
 	/* only wait the tags that are not done yet */
@@ -403,7 +403,7 @@ int starpu_tag_wait_array(unsigned ntags, starpu_tag_t *id)
 
 	for (i = 0; i < current; i++)
 	{
-		tag_add_succ(tag_array[i], cg);
+		_starpu_tag_add_succ(tag_array[i], cg);
 		starpu_spin_unlock(&tag_array[i]->lock);
 	}
 

+ 3 - 3
src/core/dependencies/tags.h

@@ -37,7 +37,7 @@
 typedef enum {
 	/* this tag is not declared by any task */
 	INVALID_STATE,
-	/* tag_declare was called to associate the tag to a task */
+	/* _starpu_tag_declare was called to associate the tag to a task */
 	ASSOCIATED,
 	/* some task dependencies are not fulfilled yet */
 	BLOCKED,
@@ -90,8 +90,8 @@ typedef struct _cg_t {
 
 void starpu_tag_declare_deps(starpu_tag_t id, unsigned ndeps, ...);
 
-void notify_dependencies(struct job_s *j);
-void tag_declare(starpu_tag_t id, struct job_s *job);
+void _starpu_notify_dependencies(struct job_s *j);
+void _starpu_tag_declare(starpu_tag_t id, struct job_s *job);
 
 unsigned submit_job_enforce_task_deps(struct job_s *j);
 

+ 6 - 6
src/core/errorcheck.c

@@ -17,9 +17,9 @@
 #include <core/errorcheck.h>
 #include <core/workers.h>
 
-void set_local_worker_status(worker_status st)
+void _starpu_set_local_worker_status(worker_status st)
 {
-	struct worker_s *worker = get_local_worker_key();
+	struct worker_s *worker = _starpu_get_local_worker_key();
 
 	/* It is possible that we call this function from the application (and
 	 * thereforce outside a worker), for instance if we are executing the
@@ -28,9 +28,9 @@ void set_local_worker_status(worker_status st)
 		worker->status = st;
 }
 
-worker_status get_local_worker_status(void)
+worker_status _starpu_get_local_worker_status(void)
 {
-	struct worker_s *worker = get_local_worker_key();
+	struct worker_s *worker = _starpu_get_local_worker_key();
 	if (STARPU_UNLIKELY(!worker))
 		return STATUS_INVALID;
 
@@ -39,9 +39,9 @@ worker_status get_local_worker_status(void)
 
 /* It is forbidden to call blocking operations with Callback and during the
  * execution of a task. */
-unsigned worker_may_perform_blocking_calls(void)
+unsigned _starpu_worker_may_perform_blocking_calls(void)
 {
-	worker_status st = get_local_worker_status();
+	worker_status st = _starpu_get_local_worker_status();
 
 	return ( !(st == STATUS_CALLBACK) && !(st == STATUS_EXECUTING));
 }

+ 3 - 3
src/core/errorcheck.h

@@ -33,9 +33,9 @@ typedef enum {
 	STATUS_CALLBACK
 } worker_status;
 
-void set_local_worker_status(worker_status st);
-worker_status get_local_worker_status(void);
+void _starpu_set_local_worker_status(worker_status st);
+worker_status _starpu_get_local_worker_status(void);
 
-unsigned worker_may_perform_blocking_calls(void);
+unsigned _starpu_worker_may_perform_blocking_calls(void);
 
 #endif // __ERRORCHECK_H__

+ 6 - 6
src/core/jobs.c

@@ -55,7 +55,7 @@ job_t __attribute__((malloc)) _starpu_job_create(struct starpu_task *task)
 	pthread_cond_init(&job->sync_cond, NULL);
 
 	if (task->use_tag)
-		tag_declare(task->tag_id, job);
+		_starpu_tag_declare(task->tag_id, job);
 
 	return job;
 }
@@ -81,7 +81,7 @@ void _starpu_handle_job_termination(job_t j)
 		fprintf(stderr, "OOPS ... job %p was already terminated !!\n", j);
 
 	/* in case there are dependencies, wake up the proper tasks */
-	notify_dependencies(j);
+	_starpu_notify_dependencies(j);
 
 	/* the callback is executed after the dependencies so that we may remove the tag 
  	 * of the task itself */
@@ -89,13 +89,13 @@ void _starpu_handle_job_termination(job_t j)
 	{
 		/* so that we can check whether we are doing blocking calls
 		 * within the callback */
-		set_local_worker_status(STATUS_CALLBACK);
+		_starpu_set_local_worker_status(STATUS_CALLBACK);
 
 		TRACE_START_CALLBACK(j);
 		task->callback_func(task->callback_arg);
 		TRACE_END_CALLBACK(j);
 
-		set_local_worker_status(STATUS_UNKNOWN);
+		_starpu_set_local_worker_status(STATUS_UNKNOWN);
 	}
 
 	if (!task->detach)
@@ -162,7 +162,7 @@ unsigned _starpu_enforce_deps_and_schedule(job_t j)
 		return 0;
 
 	/* enforce data dependencies */
-	if (submit_job_enforce_data_deps(j))
+	if (_starpu_submit_job_enforce_data_deps(j))
 		return 0;
 
 	ret = push_task(j);
@@ -195,7 +195,7 @@ int _starpu_push_local_task(struct worker_s *worker, struct job_s *j)
 	pthread_mutex_unlock(&worker->local_jobs_mutex);
 
 	/* XXX that's a bit excessive ... */
-	wake_all_blocked_workers_on_node(worker->memory_node);
+	_starpu_wake_all_blocked_workers_on_node(worker->memory_node);
 
 	return 0;
 }

+ 1 - 1
src/core/mechanisms/deque_queues.c

@@ -121,7 +121,7 @@ job_t deque_pop_task(struct jobq_s *q)
 	STARPU_ASSERT(q);
 	struct deque_jobq_s *deque_queue = q->queue;
 
-	if ((deque_queue->njobs == 0) && machine_is_running())
+	if ((deque_queue->njobs == 0) && _starpu_machine_is_running())
 	{
 		return NULL;
 	}

+ 2 - 2
src/core/mechanisms/priority_queues.c

@@ -98,10 +98,10 @@ job_t priority_pop_task(struct jobq_s *q)
 	/* block until some event happens */
 	pthread_mutex_lock(&q->activity_mutex);
 
-	if ((queue->total_njobs == 0) && machine_is_running())
+	if ((queue->total_njobs == 0) && _starpu_machine_is_running())
 	{
 #ifdef NON_BLOCKING_DRIVERS
-		datawizard_progress(q->memory_node, 1);
+		_starpu_datawizard_progress(q->memory_node, 1);
 #else
 		pthread_cond_wait(&q->activity_cond, &q->activity_mutex);
 #endif

+ 1 - 1
src/core/perfmodel/perfmodel.h

@@ -92,7 +92,7 @@ void dump_registered_models(void);
 double job_expected_length(uint32_t who, struct job_s *j, enum starpu_perf_archtype arch);
 double regression_based_job_expected_length(struct starpu_perfmodel_t *model,
 		uint32_t who, struct job_s *j);
-void update_perfmodel_history(struct job_s *j, enum starpu_perf_archtype arch,
+void _starpu_update_perfmodel_history(struct job_s *j, enum starpu_perf_archtype arch,
 				unsigned cpuid, double measured);
 
 double data_expected_penalty(struct jobq_s *q, struct starpu_task *task);

+ 9 - 9
src/core/perfmodel/perfmodel_bus.c

@@ -52,14 +52,14 @@ static struct cudadev_timing cudadev_timing_per_cpu[MAXNODES][MAXCPUS];
 
 static void measure_bandwith_between_host_and_dev_on_cpu(int dev, int cpu)
 {
-	struct machine_config_s *config = get_machine_config();
-	bind_thread_on_cpu(config, cpu);
+	struct machine_config_s *config = _starpu_get_machine_config();
+	_starpu_bind_thread_on_cpu(config, cpu);
 
 	/* Initiliaze CUDA context on the device */
 	cudaSetDevice(dev);
 
 	/* hack to avoid third party libs to rebind threads */
-	bind_thread_on_cpu(config, cpu);
+	_starpu_bind_thread_on_cpu(config, cpu);
 
 	/* hack to force the initialization */
 	cudaFree(0);
@@ -190,8 +190,8 @@ static void benchmark_all_cuda_devices(void)
 	}
 
 #ifdef USE_CUDA
-	struct machine_config_s *config = get_machine_config();
-	unsigned ncores = topology_get_nhwcore(config);
+	struct machine_config_s *config = _starpu_get_machine_config();
+	unsigned ncores = _starpu_topology_get_nhwcore(config);
 
         cudaGetDeviceCount(&ncuda);
 	int i;
@@ -248,8 +248,8 @@ static void load_bus_affinity_file_content(void)
 	STARPU_ASSERT(f);
 
 #ifdef USE_CUDA
-	struct machine_config_s *config = get_machine_config();
-	unsigned ncores = topology_get_nhwcore(config);
+	struct machine_config_s *config = _starpu_get_machine_config();
+	unsigned ncores = _starpu_topology_get_nhwcore(config);
 
         cudaGetDeviceCount(&ncuda);
 
@@ -297,8 +297,8 @@ static void write_bus_affinity_file_content(void)
 	}
 
 #ifdef USE_CUDA
-	struct machine_config_s *config = get_machine_config();
-	unsigned ncores = topology_get_nhwcore(config);
+	struct machine_config_s *config = _starpu_get_machine_config();
+	unsigned ncores = _starpu_topology_get_nhwcore(config);
 
 	int gpu;
 	for (gpu = 0; gpu < ncuda; gpu++)

+ 1 - 1
src/core/perfmodel/perfmodel_history.c

@@ -542,7 +542,7 @@ double history_based_job_expected_length(struct starpu_perfmodel_t *model, enum
 	return exp;
 }
 
-void update_perfmodel_history(job_t j, enum starpu_perf_archtype arch, unsigned cpuid __attribute__((unused)), double measured)
+void _starpu_update_perfmodel_history(job_t j, enum starpu_perf_archtype arch, unsigned cpuid __attribute__((unused)), double measured)
 {
 	struct starpu_perfmodel_t *model = j->task->cl->model;
 

+ 2 - 2
src/core/policies/sched_policy.c

@@ -192,7 +192,7 @@ int push_task(job_t j)
 
 	if (STARPU_UNLIKELY(j->task->execute_on_a_specific_worker))
 	{
-		struct worker_s *worker = get_worker_struct(j->task->workerid);
+		struct worker_s *worker = _starpu_get_worker_struct(j->task->workerid);
 		return _starpu_push_local_task(worker, j);
 	}
 	else {
@@ -243,7 +243,7 @@ void wait_on_sched_event(void)
 
 	handle_all_pending_node_data_requests(get_local_memory_node());
 
-	if (machine_is_running())
+	if (_starpu_machine_is_running())
 	{
 #ifndef NON_BLOCKING_DRIVERS
 		pthread_cond_wait(&q->activity_cond, &q->activity_mutex);

+ 2 - 2
src/core/progress_hook.c

@@ -50,7 +50,7 @@ int starpu_register_progression_hook(unsigned (*func)(void *arg), void *arg)
 
 	pthread_mutex_unlock(&progression_hook_mutex);
 
-	wake_all_blocked_workers();
+	starpu_wake_all_blocked_workers();
 
 	/* We could not find an empty slot */
 	return -1;
@@ -63,7 +63,7 @@ void starpu_deregister_progression_hook(int hook_id)
 	pthread_mutex_unlock(&progression_hook_mutex);
 }
 
-unsigned execute_registered_progression_hooks(void)
+unsigned _starpu_execute_registered_progression_hooks(void)
 {
 	/* By default, it is possible to block, but if some progression hooks
 	 * requires that it's not blocking, we disable blocking. */

+ 5 - 5
src/core/task.c

@@ -91,7 +91,7 @@ int starpu_wait_task(struct starpu_task *task)
 	if (task->detach || task->synchronous)
 		return -EINVAL;
 
-	if (STARPU_UNLIKELY(!worker_may_perform_blocking_calls()))
+	if (STARPU_UNLIKELY(!_starpu_worker_may_perform_blocking_calls()))
 		return -EDEADLK;
 
 	job_t j = (struct job_s *)task->starpu_private;
@@ -116,7 +116,7 @@ int starpu_submit_task(struct starpu_task *task)
 	{
 		/* Perhaps it is not possible to submit a synchronous
 		 * (blocking) task */
-		if (STARPU_UNLIKELY(!worker_may_perform_blocking_calls()))
+		if (STARPU_UNLIKELY(!_starpu_worker_may_perform_blocking_calls()))
 			return -EDEADLK;
 
 		task->detach = 0;
@@ -127,14 +127,14 @@ int starpu_submit_task(struct starpu_task *task)
 	if (task->cl)
 	{
 		uint32_t where = task->cl->where;
-		if (!worker_exists(where))
+		if (!_starpu_worker_exists(where))
 			return -ENODEV;
 
 		/* In case we require that a task should be explicitely
 		 * executed on a specific worker, we make sure that the worker
 		 * is able to execute this task.  */
 		if (task->execute_on_a_specific_worker 
-			&& !worker_may_execute_task(task->workerid, where))
+			&& !_starpu_worker_may_execute_task(task->workerid, where))
 			return -ENODEV;
 	}
 
@@ -194,7 +194,7 @@ int starpu_wait_all_tasks(void)
 {
 	int res;
 
-	if (STARPU_UNLIKELY(!worker_may_perform_blocking_calls()))
+	if (STARPU_UNLIKELY(!_starpu_worker_may_perform_blocking_calls()))
 		return -EDEADLK;
 
 

+ 24 - 24
src/core/topology.c

@@ -28,10 +28,10 @@
 		
 static unsigned topology_is_initialized = 0;
 
-static void initialize_workers_bindid(struct machine_config_s *config);
+static void _starpu_initialize_workers_bindid(struct machine_config_s *config);
 
 #ifdef USE_CUDA
-static void initialize_workers_gpuid(struct machine_config_s *config);
+static void _starpu_initialize_workers_gpuid(struct machine_config_s *config);
 static unsigned may_bind_automatically = 0;
 #endif
 
@@ -40,7 +40,7 @@ static unsigned may_bind_automatically = 0;
  */
 
 #ifdef USE_CUDA
-static void initialize_workers_gpuid(struct machine_config_s *config)
+static void _starpu_initialize_workers_gpuid(struct machine_config_s *config)
 {
 	char *strval;
 	unsigned i;
@@ -110,14 +110,14 @@ static void initialize_workers_gpuid(struct machine_config_s *config)
 }
 #endif
 
-static inline int get_next_gpuid(struct machine_config_s *config)
+static inline int _starpu_get_next_gpuid(struct machine_config_s *config)
 {
 	unsigned i = ((config->current_gpuid++) % config->ncudagpus);
 
 	return (int)config->workers_gpuid[i];
 }
 
-static void init_topology(struct machine_config_s *config)
+static void _starpu_init_topology(struct machine_config_s *config)
 {
 	if (!topology_is_initialized)
 	{
@@ -143,14 +143,14 @@ static void init_topology(struct machine_config_s *config)
 	}
 }
 
-unsigned topology_get_nhwcore(struct machine_config_s *config)
+unsigned _starpu_topology_get_nhwcore(struct machine_config_s *config)
 {
-	init_topology(config);
+	_starpu_init_topology(config);
 	
 	return config->nhwcores;
 }
 
-static int init_machine_config(struct machine_config_s *config,
+static int _starpu_init_machine_config(struct machine_config_s *config,
 				struct starpu_conf *user_conf)
 {
 	int explicitval __attribute__((unused));
@@ -158,9 +158,9 @@ static int init_machine_config(struct machine_config_s *config,
 
 	config->nworkers = 0;
 
-	init_topology(config);
+	_starpu_init_topology(config);
 
-	initialize_workers_bindid(config);
+	_starpu_initialize_workers_bindid(config);
 
 #ifdef USE_CUDA
 	if (user_conf && (user_conf->ncuda == 0))
@@ -170,7 +170,7 @@ static int init_machine_config(struct machine_config_s *config,
 	}
 	else {
 		/* we need to initialize CUDA early to count the number of devices */
-		init_cuda();
+		_starpu_init_cuda();
 
 		if (user_conf && (user_conf->ncuda != -1))
 		{
@@ -194,13 +194,13 @@ static int init_machine_config(struct machine_config_s *config,
 	if (config->ncudagpus > 0)
 		use_accelerator = 1;
 
-	initialize_workers_gpuid(config);
+	_starpu_initialize_workers_gpuid(config);
 
 	unsigned cudagpu;
 	for (cudagpu = 0; cudagpu < config->ncudagpus; cudagpu++)
 	{
 		config->workers[config->nworkers + cudagpu].arch = STARPU_CUDA_WORKER;
-		int devid = get_next_gpuid(config);
+		int devid = _starpu_get_next_gpuid(config);
 		enum starpu_perf_archtype arch = STARPU_CUDA_DEFAULT + devid;
 		config->workers[config->nworkers + cudagpu].id = devid;
 		config->workers[config->nworkers + cudagpu].perf_arch = arch; 
@@ -293,7 +293,7 @@ static int init_machine_config(struct machine_config_s *config,
 /*
  * Bind workers on the different processors
  */
-static void initialize_workers_bindid(struct machine_config_s *config)
+static void _starpu_initialize_workers_bindid(struct machine_config_s *config)
 {
 	char *strval;
 	unsigned i;
@@ -363,7 +363,7 @@ static void initialize_workers_bindid(struct machine_config_s *config)
  * worker. In case a list of preferred cores was specified, we look for a an
  * available core among the list if possible, otherwise a round-robin policy is
  * used. */
-static inline int get_next_bindid(struct machine_config_s *config,
+static inline int _starpu_get_next_bindid(struct machine_config_s *config,
 				int *preferred_binding, int npreferred)
 {
 	unsigned found = 0;
@@ -402,12 +402,12 @@ static inline int get_next_bindid(struct machine_config_s *config,
 	return (int)config->workers_bindid[i];
 }
 
-void bind_thread_on_cpu(struct machine_config_s *config __attribute__((unused)), unsigned coreid)
+void _starpu_bind_thread_on_cpu(struct machine_config_s *config __attribute__((unused)), unsigned coreid)
 {
 	int ret;
 
 #ifdef HAVE_HWLOC
-	init_topology(config);
+	_starpu_init_topology(config);
 
 	hwloc_obj_t obj = hwloc_get_obj_by_depth(config->hwtopology, config->core_depth, coreid);
 	hwloc_cpuset_t set = obj->cpuset;
@@ -439,7 +439,7 @@ void bind_thread_on_cpu(struct machine_config_s *config __attribute__((unused)),
 #endif
 }
 
-static void init_workers_binding(struct machine_config_s *config)
+static void _starpu_init_workers_binding(struct machine_config_s *config)
 {
 	/* launch one thread per CPU */
 	unsigned ram_memory_node;
@@ -449,7 +449,7 @@ static void init_workers_binding(struct machine_config_s *config)
 
 	/* note that even if the CPU core are not used, we always have a RAM node */
 	/* TODO : support NUMA  ;) */
-	ram_memory_node = register_memory_node(RAM);
+	ram_memory_node = _starpu_register_memory_node(RAM);
 
 	unsigned worker;
 	for (worker = 0; worker < config->nworkers; worker++)
@@ -484,7 +484,7 @@ static void init_workers_binding(struct machine_config_s *config)
 					npreferred = config->nhwcores;
 				}
 				is_a_set_of_accelerators = 0;
-				memory_node = register_memory_node(CUDA_RAM);
+				memory_node = _starpu_register_memory_node(CUDA_RAM);
 				break;
 #endif
 			default:
@@ -493,12 +493,12 @@ static void init_workers_binding(struct machine_config_s *config)
 
 		if (is_a_set_of_accelerators) {
 			if (accelerator_bindid == -1)
-				accelerator_bindid = get_next_bindid(config, preferred_binding, npreferred);
+				accelerator_bindid = _starpu_get_next_bindid(config, preferred_binding, npreferred);
 
 			workerarg->bindid = accelerator_bindid;
 		}
 		else {
-			workerarg->bindid = get_next_bindid(config, preferred_binding, npreferred);
+			workerarg->bindid = _starpu_get_next_bindid(config, preferred_binding, npreferred);
 		}
 
 		workerarg->memory_node = memory_node;
@@ -512,14 +512,14 @@ int starpu_build_topology(struct machine_config_s *config)
 
 	struct starpu_conf *user_conf = config->user_conf;
 
-	ret = init_machine_config(config, user_conf);
+	ret = _starpu_init_machine_config(config, user_conf);
 	if (ret)
 		return ret;
 
 	/* for the data management library */
 	init_memory_nodes();
 
-	init_workers_binding(config);
+	_starpu_init_workers_binding(config);
 
 	return 0;
 }

+ 2 - 2
src/core/topology.h

@@ -67,8 +67,8 @@ int starpu_build_topology(struct machine_config_s *config);
 void starpu_destroy_topology(struct machine_config_s *config);
 
 /* returns the number of physical cores */
-unsigned topology_get_nhwcore(struct machine_config_s *config);
+unsigned _starpu_topology_get_nhwcore(struct machine_config_s *config);
 
-void bind_thread_on_cpu(struct machine_config_s *config, unsigned coreid);
+void _starpu_bind_thread_on_cpu(struct machine_config_s *config, unsigned coreid);
 
 #endif // __TOPOLOGY_H__

+ 35 - 35
src/core/workers.c

@@ -24,7 +24,7 @@ static pthread_key_t worker_key;
 
 static struct machine_config_s config;
 
-struct machine_config_s *get_machine_config(void)
+struct machine_config_s *_starpu_get_machine_config(void)
 {
 	return &config;
 }
@@ -32,7 +32,7 @@ struct machine_config_s *get_machine_config(void)
 /* in case a task is submitted, we may check whether there exists a worker
    that may execute the task or not */
 
-inline uint32_t worker_exists(uint32_t task_mask)
+inline uint32_t _starpu_worker_exists(uint32_t task_mask)
 {
 	return (task_mask & config.worker_mask);
 } 
@@ -47,7 +47,7 @@ inline uint32_t may_submit_core_task(void)
 	return (CORE & config.worker_mask);
 }
 
-inline uint32_t worker_may_execute_task(unsigned workerid, uint32_t where)
+inline uint32_t _starpu_worker_may_execute_task(unsigned workerid, uint32_t where)
 {
 	return (where & config.workers[workerid].worker_mask);
 }
@@ -61,7 +61,7 @@ static unsigned gordon_inited = 0;
 static struct worker_set_s gordon_worker_set;
 #endif
 
-static void init_worker_queue(struct worker_s *workerarg)
+static void _starpu_init_worker_queue(struct worker_s *workerarg)
 {
 	struct jobq_s *jobq = workerarg->jobq;
 
@@ -88,7 +88,7 @@ static void init_worker_queue(struct worker_s *workerarg)
 	memory_node_attach_queue(jobq, workerarg->memory_node);
 }
 
-static void init_workers(struct machine_config_s *config)
+static void _starpu_init_workers(struct machine_config_s *config)
 {
 	config->running = 1;
 
@@ -119,7 +119,7 @@ static void init_workers(struct machine_config_s *config)
 	
 		workerarg->status = STATUS_INITIALIZING;
 
-		init_worker_queue(workerarg);
+		_starpu_init_worker_queue(workerarg);
 
 		switch (workerarg->arch) {
 #ifdef USE_CPUS
@@ -127,7 +127,7 @@ static void init_workers(struct machine_config_s *config)
 				workerarg->set = NULL;
 				workerarg->worker_is_initialized = 0;
 				pthread_create(&workerarg->worker_thread, 
-						NULL, core_worker, workerarg);
+						NULL, _starpu_core_worker, workerarg);
 				break;
 #endif
 #ifdef USE_CUDA
@@ -135,7 +135,7 @@ static void init_workers(struct machine_config_s *config)
 				workerarg->set = NULL;
 				workerarg->worker_is_initialized = 0;
 				pthread_create(&workerarg->worker_thread, 
-						NULL, cuda_worker, workerarg);
+						NULL, _starpu_cuda_worker, workerarg);
 
 				break;
 #endif
@@ -198,12 +198,12 @@ static void init_workers(struct machine_config_s *config)
 
 }
 
-void set_local_worker_key(struct worker_s *worker)
+void _starpu_set_local_worker_key(struct worker_s *worker)
 {
 	pthread_setspecific(worker_key, worker);
 }
 
-struct worker_s *get_local_worker_key(void)
+struct worker_s *_starpu_get_local_worker_key(void)
 {
 	return pthread_getspecific(worker_key);
 }
@@ -218,7 +218,7 @@ int starpu_init(struct starpu_conf *user_conf)
 	start_fxt_profiling();
 #endif
 	
-	open_debug_logfile();
+	_starpu_open_debug_logfile();
 
 	timing_init();
 
@@ -237,7 +237,7 @@ int starpu_init(struct starpu_conf *user_conf)
 	/* initialize the queue containing the jobs */
 	init_sched_policy(&config);
 
-	init_workers(&config);
+	_starpu_init_workers(&config);
 
 	return 0;
 }
@@ -246,14 +246,14 @@ int starpu_init(struct starpu_conf *user_conf)
  * Handle runtime termination 
  */
 
-static void terminate_workers(struct machine_config_s *config)
+static void _starpu_terminate_workers(struct machine_config_s *config)
 {
 	int status;
 	unsigned workerid;
 
 	for (workerid = 0; workerid < config->nworkers; workerid++)
 	{
-		wake_all_blocked_workers();
+		starpu_wake_all_blocked_workers();
 		
 #ifdef VERBOSE
 		fprintf(stderr, "wait for worker %d\n", workerid);
@@ -294,22 +294,22 @@ static void terminate_workers(struct machine_config_s *config)
 	}
 }
 
-unsigned machine_is_running(void)
+unsigned _starpu_machine_is_running(void)
 {
 	return config.running;
 }
 
-unsigned worker_can_block(unsigned memnode)
+unsigned _starpu_worker_can_block(unsigned memnode)
 {
 	unsigned can_block = 1;
 
 	if (!check_that_no_data_request_exists(memnode))
 		can_block = 0;
 
-	if (!machine_is_running())
+	if (!_starpu_machine_is_running())
 		can_block = 0;
 
-	if (!execute_registered_progression_hooks())
+	if (!_starpu_execute_registered_progression_hooks())
 		can_block = 0;
 
 	return can_block;
@@ -321,7 +321,7 @@ typedef enum {
 	UNLOCK
 } queue_op;
 
-static void operate_on_all_queues_attached_to_node(unsigned nodeid, queue_op op)
+static void _starpu_operate_on_all_queues_attached_to_node(unsigned nodeid, queue_op op)
 {
 	unsigned q_id;
 	struct jobq_s *q;
@@ -351,22 +351,22 @@ static void operate_on_all_queues_attached_to_node(unsigned nodeid, queue_op op)
 	pthread_rwlock_unlock(&descr->attached_queues_rwlock);
 }
 
-inline void lock_all_queues_attached_to_node(unsigned node)
+inline void _starpu_lock_all_queues_attached_to_node(unsigned node)
 {
-	operate_on_all_queues_attached_to_node(node, LOCK);
+	_starpu_operate_on_all_queues_attached_to_node(node, LOCK);
 }
 
-inline void unlock_all_queues_attached_to_node(unsigned node)
+inline void _starpu_unlock_all_queues_attached_to_node(unsigned node)
 {
-	operate_on_all_queues_attached_to_node(node, UNLOCK);
+	_starpu_operate_on_all_queues_attached_to_node(node, UNLOCK);
 }
 
-inline void broadcast_all_queues_attached_to_node(unsigned node)
+inline void _starpu_broadcast_all_queues_attached_to_node(unsigned node)
 {
-	operate_on_all_queues_attached_to_node(node, BROADCAST);
+	_starpu_operate_on_all_queues_attached_to_node(node, BROADCAST);
 }
 
-static void operate_on_all_queues(queue_op op)
+static void _starpu_operate_on_all_queues(queue_op op)
 {
 	unsigned q_id;
 	struct jobq_s *q;
@@ -396,7 +396,7 @@ static void operate_on_all_queues(queue_op op)
 	pthread_rwlock_unlock(&descr->attached_queues_rwlock);
 }
 
-static void kill_all_workers(struct machine_config_s *config)
+static void _starpu_kill_all_workers(struct machine_config_s *config)
 {
 	/* lock all workers and the scheduler (in the proper order) to make
 	   sure everyone will notice the termination */
@@ -405,17 +405,17 @@ static void kill_all_workers(struct machine_config_s *config)
 
 	struct sched_policy_s *sched = get_sched_policy();
 
-	operate_on_all_queues(LOCK);
+	_starpu_operate_on_all_queues(LOCK);
 	pthread_mutex_lock(&sched->sched_activity_mutex);
 	
 	/* set the flag which will tell workers to stop */
 	config->running = 0;
 
-	operate_on_all_queues(BROADCAST);
+	_starpu_operate_on_all_queues(BROADCAST);
 	pthread_cond_broadcast(&sched->sched_activity_cond);
 
 	pthread_mutex_unlock(&sched->sched_activity_mutex);
-	operate_on_all_queues(UNLOCK);
+	_starpu_operate_on_all_queues(UNLOCK);
 }
 
 void starpu_shutdown(void)
@@ -424,7 +424,7 @@ void starpu_shutdown(void)
 	display_alloc_cache_stats();
 
 	/* tell all workers to shutdown */
-	kill_all_workers(&config);
+	_starpu_kill_all_workers(&config);
 
 #ifdef DATA_STATS
 	display_comm_ammounts();
@@ -434,13 +434,13 @@ void starpu_shutdown(void)
 		dump_registered_models();
 
 	/* wait for their termination */
-	terminate_workers(&config);
+	_starpu_terminate_workers(&config);
 
 	deinit_sched_policy(&config);
 
 	starpu_destroy_topology(&config);
 
-	close_debug_logfile();
+	_starpu_close_debug_logfile();
 }
 
 unsigned starpu_get_worker_count(void)
@@ -472,7 +472,7 @@ int starpu_get_worker_id(void)
 {
 	struct worker_s * worker;
 
-	worker = get_local_worker_key();
+	worker = _starpu_get_local_worker_key();
 	if (worker)
 	{
 		return worker->workerid;
@@ -484,7 +484,7 @@ int starpu_get_worker_id(void)
 	}
 }
 
-struct worker_s *get_worker_struct(unsigned id)
+struct worker_s *_starpu_get_worker_struct(unsigned id)
 {
 	return &config.workers[id];
 }

+ 12 - 12
src/core/workers.h

@@ -133,26 +133,26 @@ struct machine_config_s {
 
 void display_general_stats(void);
 
-unsigned machine_is_running(void);
+unsigned _starpu_machine_is_running(void);
 
-inline uint32_t worker_exists(uint32_t task_mask);
+inline uint32_t _starpu_worker_exists(uint32_t task_mask);
 inline uint32_t may_submit_cuda_task(void);
 inline uint32_t may_submit_core_task(void);
-inline uint32_t worker_may_execute_task(unsigned workerid, uint32_t where);
-unsigned worker_can_block(unsigned memnode);
+inline uint32_t _starpu_worker_may_execute_task(unsigned workerid, uint32_t where);
+unsigned _starpu_worker_can_block(unsigned memnode);
 
-inline void lock_all_queues_attached_to_node(unsigned node);
-inline void unlock_all_queues_attached_to_node(unsigned node);
-inline void broadcast_all_queues_attached_to_node(unsigned node);
+inline void _starpu_lock_all_queues_attached_to_node(unsigned node);
+inline void _starpu_unlock_all_queues_attached_to_node(unsigned node);
+inline void _starpu_broadcast_all_queues_attached_to_node(unsigned node);
 
-void set_local_worker_key(struct worker_s *worker);
-struct worker_s *get_local_worker_key(void);
+void _starpu_set_local_worker_key(struct worker_s *worker);
+struct worker_s *_starpu_get_local_worker_key(void);
 
-struct worker_s *get_worker_struct(unsigned id);
+struct worker_s *_starpu_get_worker_struct(unsigned id);
 
-struct machine_config_s *get_machine_config(void);
+struct machine_config_s *_starpu_get_machine_config(void);
 
 /* TODO move */
-unsigned execute_registered_progression_hooks(void);
+unsigned _starpu_execute_registered_progression_hooks(void);
 
 #endif // __WORKERS_H__

+ 4 - 4
src/datawizard/coherency.c

@@ -144,7 +144,7 @@ int fetch_data_on_node(starpu_data_handle handle, uint32_t requesting_node,
 	uint32_t local_node = get_local_memory_node();
 
 	while (starpu_spin_trylock(&handle->header_lock))
-		datawizard_progress(local_node, 1);
+		_starpu_datawizard_progress(local_node, 1);
 
 	if (!is_prefetch)
 		handle->per_node[requesting_node].refcnt++;
@@ -308,7 +308,7 @@ void release_data_on_node(starpu_data_handle handle, uint32_t default_wb_mask, u
 
 	uint32_t local_node = get_local_memory_node();
 	while (starpu_spin_trylock(&handle->header_lock))
-		datawizard_progress(local_node, 1);
+		_starpu_datawizard_progress(local_node, 1);
 
 	handle->per_node[memory_node].refcnt--;
 
@@ -344,11 +344,11 @@ int prefetch_task_input_on_node(struct starpu_task *task, uint32_t node)
 
 
 
-int fetch_task_input(struct starpu_task *task, uint32_t mask)
+int _starpu_fetch_task_input(struct starpu_task *task, uint32_t mask)
 {
 	TRACE_START_FETCH_INPUT(NULL);
 
-//	fprintf(stderr, "fetch_task_input\n");
+//	fprintf(stderr, "_starpu_fetch_task_input\n");
 
 	uint32_t local_memory_node = get_local_memory_node();
 

+ 1 - 1
src/datawizard/coherency.h

@@ -121,7 +121,7 @@ uint32_t get_data_refcnt(struct starpu_data_state_t *state, uint32_t node);
 void push_task_output(struct starpu_task *task, uint32_t mask);
 
 __attribute__((warn_unused_result))
-int fetch_task_input(struct starpu_task *task, uint32_t mask);
+int _starpu_fetch_task_input(struct starpu_task *task, uint32_t mask);
 
 unsigned is_data_present_or_requested(struct starpu_data_state_t *state, uint32_t node);
 

+ 3 - 3
src/datawizard/copy-driver.c

@@ -22,7 +22,7 @@
 #include "copy-driver.h"
 #include "memalloc.h"
 
-void wake_all_blocked_workers_on_node(unsigned nodeid)
+void _starpu_wake_all_blocked_workers_on_node(unsigned nodeid)
 {
 	/* wake up all queues on that node */
 	unsigned q_id;
@@ -46,7 +46,7 @@ void wake_all_blocked_workers_on_node(unsigned nodeid)
 	pthread_rwlock_unlock(&descr->attached_queues_rwlock);
 }
 
-void wake_all_blocked_workers(void)
+void starpu_wake_all_blocked_workers(void)
 {
 	/* workers may be blocked on the policy's global condition */
 	struct sched_policy_s *sched = get_sched_policy();
@@ -62,7 +62,7 @@ void wake_all_blocked_workers(void)
 	unsigned nnodes = get_memory_nodes_count();
 	for (node = 0; node < nnodes; node++)
 	{
-		wake_all_blocked_workers_on_node(node);
+		_starpu_wake_all_blocked_workers_on_node(node);
 	}
 }
 

+ 2 - 2
src/datawizard/copy-driver.h

@@ -66,8 +66,8 @@ struct copy_data_methods_s {
 #endif
 };
 
-void wake_all_blocked_workers(void);
-void wake_all_blocked_workers_on_node(unsigned nodeid);
+void starpu_wake_all_blocked_workers(void);
+void _starpu_wake_all_blocked_workers_on_node(unsigned nodeid);
 
 __attribute__((warn_unused_result))
 int driver_copy_data_1_to_1(starpu_data_handle handle, uint32_t node, 

+ 3 - 3
src/datawizard/data_request.c

@@ -151,9 +151,9 @@ int wait_data_request_completion(data_request_t r, unsigned may_alloc)
 
 		starpu_spin_unlock(&r->lock);
 
-		wake_all_blocked_workers_on_node(r->handling_node);
+		_starpu_wake_all_blocked_workers_on_node(r->handling_node);
 
-		datawizard_progress(local_node, may_alloc);
+		_starpu_datawizard_progress(local_node, may_alloc);
 
 	} while (1);
 
@@ -198,7 +198,7 @@ void post_data_request(data_request_t r, uint32_t handling_node)
 	res = pthread_mutex_unlock(&data_requests_list_mutex[handling_node]);
 	STARPU_ASSERT(!res);
 
-	wake_all_blocked_workers_on_node(handling_node);
+	_starpu_wake_all_blocked_workers_on_node(handling_node);
 }
 
 static void handle_data_request_completion(data_request_t r)

+ 2 - 2
src/datawizard/memalloc.c

@@ -50,7 +50,7 @@ static void lock_all_subtree(starpu_data_handle handle)
 	{
 		/* this is a leaf */
 		while (starpu_spin_trylock(&handle->header_lock))
-			datawizard_progress(get_local_memory_node(), 0);
+			_starpu_datawizard_progress(get_local_memory_node(), 0);
 	}
 	else {
 		/* lock all sub-subtrees children */
@@ -494,7 +494,7 @@ static size_t liberate_memory_on_node(mem_chunk_t mc, uint32_t node)
 	starpu_data_handle handle = mc->data;
 
 //	while (starpu_spin_trylock(&handle->header_lock))
-//		datawizard_progress(get_local_memory_node());
+//		_starpu_datawizard_progress(get_local_memory_node());
 
 #warning can we block here ?
 //	starpu_spin_lock(&handle->header_lock);

+ 3 - 3
src/datawizard/memory_nodes.c

@@ -28,7 +28,7 @@ static pthread_key_t memory_node_key;
 void init_memory_nodes(void)
 {
 	/* there is no node yet, subsequent nodes will be 
-	 * added using register_memory_node */
+	 * added using _starpu_register_memory_node */
 	descr.nnodes = 0;
 
 	pthread_key_create(&memory_node_key, NULL);
@@ -85,7 +85,7 @@ unsigned get_memory_nodes_count(void)
 	return descr.nnodes;
 }
 
-unsigned register_memory_node(node_kind kind)
+unsigned _starpu_register_memory_node(node_kind kind)
 {
 	unsigned nnodes;
 	/* ATOMIC_ADD returns the new value ... */
@@ -146,7 +146,7 @@ void memory_node_attach_queue(struct jobq_s *q, unsigned nodeid)
 
 unsigned starpu_get_worker_memory_node(unsigned workerid)
 {
-	struct worker_s *worker = get_worker_struct(workerid);
+	struct worker_s *worker = _starpu_get_worker_struct(workerid);
 
 	return worker->memory_node;
 }

+ 1 - 1
src/datawizard/memory_nodes.h

@@ -50,7 +50,7 @@ void init_memory_nodes(void);
 void deinit_memory_nodes(void);
 void set_local_memory_node_key(unsigned *node);
 unsigned get_local_memory_node(void);
-unsigned register_memory_node(node_kind kind);
+unsigned _starpu_register_memory_node(node_kind kind);
 void memory_node_attach_queue(struct jobq_s *q, unsigned nodeid);
 
 node_kind get_node_kind(uint32_t node);

+ 2 - 2
src/datawizard/progress.c

@@ -19,11 +19,11 @@
 #include <datawizard/progress.h>
 #include <datawizard/data_request.h>
 
-void datawizard_progress(uint32_t memory_node, unsigned may_alloc)
+void _starpu_datawizard_progress(uint32_t memory_node, unsigned may_alloc)
 {
 	/* in case some other driver requested data */
 	handle_pending_node_data_requests(memory_node);
 	handle_node_data_requests(memory_node, may_alloc);
 
-	execute_registered_progression_hooks();
+	_starpu_execute_registered_progression_hooks();
 }

+ 1 - 1
src/datawizard/progress.h

@@ -19,6 +19,6 @@
 
 #include <stdint.h>
 
-void datawizard_progress(uint32_t memory_node, unsigned may_alloc);
+void _starpu_datawizard_progress(uint32_t memory_node, unsigned may_alloc);
 
 #endif

+ 2 - 2
src/datawizard/user_interactions.c

@@ -89,7 +89,7 @@ int starpu_sync_data_with_mem(starpu_data_handle handle, starpu_access_mode mode
 	STARPU_ASSERT(handle);
 
 	/* it is forbidden to call this function from a callback or a codelet */
-	if (STARPU_UNLIKELY(!worker_may_perform_blocking_calls()))
+	if (STARPU_UNLIKELY(!_starpu_worker_may_perform_blocking_calls()))
 		return -EDEADLK;
 
 	struct state_and_node statenode =
@@ -190,7 +190,7 @@ int starpu_prefetch_data_on_node(starpu_data_handle handle, unsigned node, unsig
 	STARPU_ASSERT(handle);
 
 	/* it is forbidden to call this function from a callback or a codelet */
-	if (STARPU_UNLIKELY(!worker_may_perform_blocking_calls()))
+	if (STARPU_UNLIKELY(!_starpu_worker_may_perform_blocking_calls()))
 		return -EDEADLK;
 
 	struct state_and_node statenode =

+ 1 - 1
src/datawizard/write_back.c

@@ -26,7 +26,7 @@ void write_through_data(starpu_data_handle handle, uint32_t requesting_node,
 	}
 
 	while (starpu_spin_trylock(&handle->header_lock))
-		datawizard_progress(requesting_node, 1);
+		_starpu_datawizard_progress(requesting_node, 1);
 
 	/* first commit all changes onto the nodes specified by the mask */
 	uint32_t node;

+ 10 - 10
src/drivers/core/driver_core.c

@@ -39,7 +39,7 @@ static int execute_job_on_core(job_t j, struct worker_s *core_args)
 	if (calibrate_model || BENCHMARK_COMM)
 		GET_TICK(codelet_start_comm);
 
-	ret = fetch_task_input(task, 0);
+	ret = _starpu_fetch_task_input(task, 0);
 
 	if (calibrate_model || BENCHMARK_COMM)
 		GET_TICK(codelet_end_comm);
@@ -85,7 +85,7 @@ static int execute_job_on_core(job_t j, struct worker_s *core_args)
 		core_args->jobq->total_computation_time_error += error;
 
 		if (calibrate_model)
-			update_perfmodel_history(j, core_args->arch, core_args->id, measured);
+			_starpu_update_perfmodel_history(j, core_args->arch, core_args->id, measured);
 	}
 //#endif
 
@@ -94,7 +94,7 @@ static int execute_job_on_core(job_t j, struct worker_s *core_args)
 	return STARPU_SUCCESS;
 }
 
-void *core_worker(void *arg)
+void *_starpu_core_worker(void *arg)
 {
 	struct worker_s *core_arg = arg;
 
@@ -103,7 +103,7 @@ void *core_worker(void *arg)
 #endif
 	TRACE_WORKER_INIT_START(FUT_CORE_KEY, core_arg->memory_node);
 
-	bind_thread_on_cpu(core_arg->config, core_arg->bindid);
+	_starpu_bind_thread_on_cpu(core_arg->config, core_arg->bindid);
 
 #ifdef VERBOSE
         fprintf(stderr, "core worker %d is ready on logical core %d\n", core_arg->id, core_arg->bindid);
@@ -113,7 +113,7 @@ void *core_worker(void *arg)
 
 	set_local_queue(core_arg->jobq);
 
-	set_local_worker_key(core_arg);
+	_starpu_set_local_worker_key(core_arg);
 
 	snprintf(core_arg->name, 32, "CORE %d", core_arg->id);
 
@@ -143,13 +143,13 @@ void *core_worker(void *arg)
 	struct jobq_s *queue = policy->get_local_queue(policy);
 	unsigned memnode = core_arg->memory_node;
 
-	while (machine_is_running())
+	while (_starpu_machine_is_running())
 	{
 		TRACE_START_PROGRESS(memnode);
-		datawizard_progress(memnode, 1);
+		_starpu_datawizard_progress(memnode, 1);
 		TRACE_END_PROGRESS(memnode);
 
-		execute_registered_progression_hooks();
+		_starpu_execute_registered_progression_hooks();
 
 		jobq_lock(queue);
 
@@ -161,7 +161,7 @@ void *core_worker(void *arg)
 			j = pop_task();
 
                 if (j == NULL) {
-			if (worker_can_block(memnode))
+			if (_starpu_worker_can_block(memnode))
 				pthread_cond_wait(&queue->activity_cond, &queue->activity_mutex);
 			jobq_unlock(queue);
  			continue;
@@ -206,7 +206,7 @@ void *core_worker(void *arg)
 		ratio = core_arg->jobq->total_computation_time_error/core_arg->jobq->total_computation_time;
 	}
 
-	print_to_logfile("MODEL ERROR: CORE %d ERROR %lf EXEC %lf RATIO %lf NTASKS %d\n", core_arg->id, core_arg->jobq->total_computation_time_error, core_arg->jobq->total_computation_time, ratio, core_arg->jobq->total_job_performed);
+	_starpu_print_to_logfile("MODEL ERROR: CORE %d ERROR %lf EXEC %lf RATIO %lf NTASKS %d\n", core_arg->id, core_arg->jobq->total_computation_time_error, core_arg->jobq->total_computation_time, ratio, core_arg->jobq->total_job_performed);
 #endif
 
 	TRACE_WORKER_DEINIT_END(FUT_CORE_KEY);

+ 1 - 1
src/drivers/core/driver_core.h

@@ -32,7 +32,7 @@
 
 #include <starpu.h>
 
-void *core_worker(void *);
+void *_starpu_core_worker(void *);
 
 #ifndef NMAXCORES
 #define NMAXCORES       4

+ 12 - 12
src/drivers/cuda/driver_cuda.c

@@ -56,7 +56,7 @@ unsigned get_cuda_device_count(void)
 	return (unsigned)cnt;
 }
 
-void init_cuda(void)
+void _starpu_init_cuda(void)
 {
 	ncudagpus = get_cuda_device_count();
 	assert(ncudagpus <= MAXCUDADEVS);
@@ -93,7 +93,7 @@ static int execute_job_on_cuda(job_t j, struct worker_s *args)
 		GET_TICK(codelet_start_comm);
 	}
 
-	ret = fetch_task_input(task, mask);
+	ret = _starpu_fetch_task_input(task, mask);
 	if (ret != 0) {
 		/* there was not enough memory, so the input of
 		 * the codelet cannot be fetched ... put the 
@@ -138,7 +138,7 @@ static int execute_job_on_cuda(job_t j, struct worker_s *args)
 		args->jobq->total_computation_time_error += error;
 
 		if (calibrate_model)
-			update_perfmodel_history(j, args->perf_arch, (unsigned)args->id, measured);
+			_starpu_update_perfmodel_history(j, args->perf_arch, (unsigned)args->id, measured);
 	}
 
 	args->jobq->total_job_performed++;
@@ -148,7 +148,7 @@ static int execute_job_on_cuda(job_t j, struct worker_s *args)
 	return STARPU_SUCCESS;
 }
 
-void *cuda_worker(void *arg)
+void *_starpu_cuda_worker(void *arg)
 {
 	struct worker_s* args = arg;
 
@@ -160,13 +160,13 @@ void *cuda_worker(void *arg)
 #endif
 	TRACE_WORKER_INIT_START(FUT_CUDA_KEY, memory_node);
 
-	bind_thread_on_cpu(args->config, args->bindid);
+	_starpu_bind_thread_on_cpu(args->config, args->bindid);
 
 	set_local_memory_node_key(&(args->memory_node));
 
 	set_local_queue(args->jobq);
 
-	set_local_worker_key(args);
+	_starpu_set_local_worker_key(args);
 
 	/* this is only useful (and meaningful) is there is a single
 	   memory node "related" to that queue */
@@ -180,7 +180,7 @@ void *cuda_worker(void *arg)
 	init_context(devid);
 
 	/* one more time to avoid hacks from third party lib :) */
-	bind_thread_on_cpu(args->config, args->bindid);
+	_starpu_bind_thread_on_cpu(args->config, args->bindid);
 
 	args->status = STATUS_UNKNOWN;
 
@@ -210,13 +210,13 @@ void *cuda_worker(void *arg)
 	struct jobq_s *queue = policy->get_local_queue(policy);
 	unsigned memnode = args->memory_node;
 	
-	while (machine_is_running())
+	while (_starpu_machine_is_running())
 	{
 		TRACE_START_PROGRESS(memnode);
-		datawizard_progress(memnode, 1);
+		_starpu_datawizard_progress(memnode, 1);
 		TRACE_END_PROGRESS(memnode);
 
-		execute_registered_progression_hooks();
+		_starpu_execute_registered_progression_hooks();
 	
 		jobq_lock(queue);
 
@@ -228,7 +228,7 @@ void *cuda_worker(void *arg)
 			j = pop_task();
 
 		if (j == NULL) {
-			if (worker_can_block(memnode))
+			if (_starpu_worker_can_block(memnode))
 				pthread_cond_wait(&queue->activity_cond, &queue->activity_mutex);
 			jobq_unlock(queue);
 			continue;
@@ -280,7 +280,7 @@ void *cuda_worker(void *arg)
 	}
 
 
-	print_to_logfile("MODEL ERROR: CUDA %d ERROR %lf EXEC %lf RATIO %lf NTASKS %d\n", args->id, args->jobq->total_computation_time_error, args->jobq->total_computation_time, ratio, args->jobq->total_job_performed);
+	_starpu_print_to_logfile("MODEL ERROR: CUDA %d ERROR %lf EXEC %lf RATIO %lf NTASKS %d\n", args->id, args->jobq->total_computation_time_error, args->jobq->total_computation_time, ratio, args->jobq->total_job_performed);
 #endif
 
 	TRACE_WORKER_DEINIT_END(FUT_CUDA_KEY);

+ 2 - 2
src/drivers/cuda/driver_cuda.h

@@ -44,8 +44,8 @@
 unsigned get_cuda_device_count(void);
 
 #ifdef USE_CUDA
-void init_cuda(void);
-void *cuda_worker(void *);
+void _starpu_init_cuda(void);
+void *_starpu_cuda_worker(void *);
 #endif
 
 #endif //  __DRIVER_CUDA_H__

+ 12 - 12
src/drivers/gordon/driver_gordon.c

@@ -52,7 +52,7 @@ void *gordon_worker_progress(void *arg)
 	struct worker_set_s *gordon_set_arg = arg;
 	unsigned prog_thread_bind_id = 
 		(gordon_set_arg->workers[0].bindid + 1)%(gordon_set_arg->config->nhwcores);
-	bind_thread_on_cpu(gordon_set_arg->config, prog_thread_bind_id);
+	_starpu_bind_thread_on_cpu(gordon_set_arg->config, prog_thread_bind_id);
 
 	pthread_mutex_lock(&progress_mutex);
 	progress_thread_is_inited = 1;
@@ -68,7 +68,7 @@ void *gordon_worker_progress(void *arg)
 		if (ret)
 		{
 			/* possibly wake the thread that injects work */
-			wake_all_blocked_workers();
+			starpu_wake_all_blocked_workers();
 		}
 	}
 
@@ -172,7 +172,7 @@ static void handle_terminated_job(job_t j)
 {
 	push_task_output(j->task, 0);
 	_starpu_handle_job_termination(j);
-	wake_all_blocked_workers();
+	starpu_wake_all_blocked_workers();
 }
 
 static void gordon_callback_list_func(void *arg)
@@ -205,12 +205,12 @@ static void gordon_callback_list_func(void *arg)
 			double measured = (double)gordon_task->measured;
 			unsigned cpuid = 0; /* XXX */
 
-			update_perfmodel_history(j, STARPU_GORDON_DEFAULT, cpuid, measured);
+			_starpu_update_perfmodel_history(j, STARPU_GORDON_DEFAULT, cpuid, measured);
 		}
 
 		push_task_output(j->task, 0);
 		_starpu_handle_job_termination(j);
-		//wake_all_blocked_workers();
+		//starpu_wake_all_blocked_workers();
 
 		task_cnt++;
 	}
@@ -218,7 +218,7 @@ static void gordon_callback_list_func(void *arg)
 	/* the job list was allocated by the gordon driver itself */
 	job_list_delete(wrapper_list);
 
-	wake_all_blocked_workers();
+	starpu_wake_all_blocked_workers();
 	free(task_wrapper->gordon_job);
 	free(task_wrapper);
 }
@@ -239,14 +239,14 @@ static void gordon_callback_func(void *arg)
 
 	handle_terminated_job(task_wrapper->j);
 
-	wake_all_blocked_workers();
+	starpu_wake_all_blocked_workers();
 	free(task_wrapper);
 }
 
 int inject_task(job_t j, struct worker_s *worker)
 {
 	struct starpu_task *task = j->task;
-	int ret = fetch_task_input(task, 0);
+	int ret = _starpu_fetch_task_input(task, 0);
 
 	if (ret != 0) {
 		/* there was not enough memory so the codelet cannot be executed right now ... */
@@ -304,7 +304,7 @@ int inject_task_list(struct job_list_s *list, struct worker_s *worker)
 		int ret;
 
 		struct starpu_task *task = j->task;
-		ret = fetch_task_input(task, 0);
+		ret = _starpu_fetch_task_input(task, 0);
 		STARPU_ASSERT(!ret);
 
 		gordon_jobs[index].index = task->cl->gordon_func;
@@ -327,7 +327,7 @@ int inject_task_list(struct job_list_s *list, struct worker_s *worker)
 void *gordon_worker_inject(struct worker_set_s *arg)
 {
 
-	while(machine_is_running()) {
+	while(_starpu_machine_is_running()) {
 		if (gordon_busy_enough()) {
 			/* gordon already has enough work, wait a little TODO */
 			wait_on_sched_event();
@@ -412,7 +412,7 @@ void *gordon_worker(void *arg)
 {
 	struct worker_set_s *gordon_set_arg = arg;
 
-	bind_thread_on_cpu(gordon_set_arg->config, gordon_set_arg->workers[0].bindid);
+	_starpu_bind_thread_on_cpu(gordon_set_arg->config, gordon_set_arg->workers[0].bindid);
 
 	/* TODO set_local_memory_node per SPU */
 	gordon_init(gordon_set_arg->nworkers);	
@@ -420,7 +420,7 @@ void *gordon_worker(void *arg)
 	/* NB: On SPUs, the worker_key is set to NULL since there is no point
 	 * in associating the PPU thread with a specific SPU (worker) while
 	 * it's handling multiple processing units. */
-	set_local_worker_key(NULL);
+	_starpu_set_local_worker_key(NULL);
 
 	/* TODO set workers' name field */
 	unsigned spu;

+ 2 - 2
src/util/malloc.c

@@ -50,7 +50,7 @@ static starpu_codelet malloc_pinned_cl = {
 
 int starpu_malloc_pinned_if_possible(void **A, size_t dim)
 {
-	if (STARPU_UNLIKELY(!worker_may_perform_blocking_calls()))
+	if (STARPU_UNLIKELY(!_starpu_worker_may_perform_blocking_calls()))
 		return -EDEADLK;
 
 	STARPU_ASSERT(A);
@@ -104,7 +104,7 @@ static starpu_codelet free_pinned_cl = {
 
 int starpu_free_pinned_if_possible(void *A)
 {
-	if (STARPU_UNLIKELY(!worker_may_perform_blocking_calls()))
+	if (STARPU_UNLIKELY(!_starpu_worker_may_perform_blocking_calls()))
 		return -EDEADLK;
 
 	if (may_submit_cuda_task())