Selaa lähdekoodia

Prefixing of src/core/workers.h

find . -type f -not -name "*svn*"|xargs sed -i s/"\bworker_s\b"/starpu_worker_s/g
find . -type f -not -name "*svn*"|xargs sed -i s/"\bBENCHMARK_COMM\b"/STARPU_BENCHMARK_COMM/g
find . -type f -not -name "*svn*"|xargs sed -i s/"\bworker_set_s\b"/starpu_worker_set_s/g
find . -type f -not -name "*svn*"|xargs sed -i s/"\bmachine_config_s\b"/starpu_machine_config_s/g
find . -type f -not -name "*svn*"|xargs sed -i s/"\bdisplay_general_stats\b"/_starpu_display_general_stats/g
find . -type f -not -name "*svn*"|xargs sed -i s/"\bmay_submit_cuda_task\b"/_starpu_may_submit_cuda_task/g
find . -type f -not -name "*svn*"|xargs sed -i s/"\bmay_submit_cpu_task\b"/_starpu_may_submit_cpu_task/g
Nathalie Furmento 15 vuotta sitten
vanhempi
commit
666bebaa6d

+ 2 - 2
src/core/errorcheck.c

@@ -19,7 +19,7 @@
 
 void _starpu_set_local_worker_status(starpu_worker_status st)
 {
-	struct worker_s *worker = _starpu_get_local_worker_key();
+	struct starpu_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
@@ -30,7 +30,7 @@ void _starpu_set_local_worker_status(starpu_worker_status st)
 
 starpu_worker_status _starpu_get_local_worker_status(void)
 {
-	struct worker_s *worker = _starpu_get_local_worker_key();
+	struct starpu_worker_s *worker = _starpu_get_local_worker_key();
 	if (STARPU_UNLIKELY(!worker))
 		return STATUS_INVALID;
 

+ 2 - 2
src/core/jobs.c

@@ -264,7 +264,7 @@ unsigned _starpu_enforce_deps_starting_from_data(starpu_job_t j)
 
 
 
-struct starpu_job_s *_starpu_pop_local_task(struct worker_s *worker)
+struct starpu_job_s *_starpu_pop_local_task(struct starpu_worker_s *worker)
 {
 	struct starpu_job_s *j = NULL;
 
@@ -278,7 +278,7 @@ struct starpu_job_s *_starpu_pop_local_task(struct worker_s *worker)
 	return j;
 }
 
-int _starpu_push_local_task(struct worker_s *worker, struct starpu_job_s *j)
+int _starpu_push_local_task(struct starpu_worker_s *worker, struct starpu_job_s *j)
 {
 	/* TODO check that the worker is able to execute the task ! */
 

+ 3 - 3
src/core/jobs.h

@@ -39,7 +39,7 @@
 #include <cuda.h>
 #endif
 
-struct worker_s;
+struct starpu_worker_s;
 
 /* codelet function */
 typedef void (*cl_func)(void **, void *);
@@ -83,7 +83,7 @@ unsigned _starpu_enforce_deps_starting_from_data(starpu_job_t j);
 void _starpu_handle_job_termination(starpu_job_t j);
 size_t _starpu_job_get_data_size(starpu_job_t j);
 
-starpu_job_t _starpu_pop_local_task(struct worker_s *worker);
-int _starpu_push_local_task(struct worker_s *worker, starpu_job_t j);
+starpu_job_t _starpu_pop_local_task(struct starpu_worker_s *worker);
+int _starpu_push_local_task(struct starpu_worker_s *worker, starpu_job_t j);
 
 #endif // __JOBS_H__

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

@@ -25,7 +25,7 @@
 
 void setup_queues(void (*init_queue_design)(void),
 		  struct jobq_s *(*func_init_queue)(void), 
-		  struct machine_config_s *config) 
+		  struct starpu_machine_config_s *config) 
 {
 	unsigned worker;
 
@@ -33,7 +33,7 @@ void setup_queues(void (*init_queue_design)(void),
 
 	for (worker = 0; worker < config->nworkers; worker++)
 	{
-		struct  worker_s *workerarg = &config->workers[worker];
+		struct  starpu_worker_s *workerarg = &config->workers[worker];
 		
 		workerarg->jobq = func_init_queue();
 	}

+ 2 - 2
src/core/mechanisms/queues.h

@@ -61,11 +61,11 @@ struct jobq_s {
 	pthread_mutex_t activity_mutex;
 };
 
-struct machine_config_s;
+struct starpu_machine_config_s;
 
 void setup_queues(void (*init_queue_design)(void),
                   struct jobq_s *(*func_init_queue)(void),
-                  struct machine_config_s *config);
+                  struct starpu_machine_config_s *config);
 
 struct jobq_s *get_local_queue(void);
 void set_local_queue(struct jobq_s *jobq);

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

@@ -57,7 +57,7 @@ static struct cudadev_timing cudadev_timing_per_cpu[STARPU_MAXNODES][MAXCPUS];
 
 static void measure_bandwith_between_host_and_dev_on_cpu(int dev, int cpu)
 {
-	struct machine_config_s *config = _starpu_get_machine_config();
+	struct starpu_machine_config_s *config = _starpu_get_machine_config();
 	_starpu_bind_thread_on_cpu(config, cpu);
 
 	/* Initiliaze CUDA context on the device */
@@ -212,7 +212,7 @@ static void benchmark_all_cuda_devices(void)
 		STARPU_ABORT();
 	}
 
-	struct machine_config_s *config = _starpu_get_machine_config();
+	struct starpu_machine_config_s *config = _starpu_get_machine_config();
 	unsigned ncpus = _starpu_topology_get_nhwcpu(config);
 
         cudaGetDeviceCount(&ncuda);
@@ -271,7 +271,7 @@ static void load_bus_affinity_file_content(void)
 	STARPU_ASSERT(f);
 
 #ifdef STARPU_USE_CUDA
-	struct machine_config_s *config = _starpu_get_machine_config();
+	struct starpu_machine_config_s *config = _starpu_get_machine_config();
 	unsigned ncpus = _starpu_topology_get_nhwcpu(config);
 
         cudaGetDeviceCount(&ncuda);
@@ -321,7 +321,7 @@ static void write_bus_affinity_file_content(void)
 	}
 
 #ifdef STARPU_USE_CUDA
-	struct machine_config_s *config = _starpu_get_machine_config();
+	struct starpu_machine_config_s *config = _starpu_get_machine_config();
 	unsigned ncpus = _starpu_topology_get_nhwcpu(config);
 	unsigned cpu;
 

+ 1 - 1
src/core/policies/deque-modeling-policy-data-aware.c

@@ -207,7 +207,7 @@ static struct jobq_s *init_dmda_fifo(void)
 	return q;
 }
 
-static void initialize_dmda_policy(struct machine_config_s *config, 
+static void initialize_dmda_policy(struct starpu_machine_config_s *config, 
 	 __attribute__ ((unused)) struct sched_policy_s *_policy) 
 {
 	nworkers = 0;

+ 1 - 1
src/core/policies/deque-modeling-policy.c

@@ -169,7 +169,7 @@ static struct jobq_s *init_dm_fifo(void)
 	return q;
 }
 
-static void initialize_dm_policy(struct machine_config_s *config, 
+static void initialize_dm_policy(struct starpu_machine_config_s *config, 
 	 __attribute__ ((unused)) struct sched_policy_s *_policy) 
 {
 	nworkers = 0;

+ 1 - 1
src/core/policies/eager-central-policy.c

@@ -44,7 +44,7 @@ static struct jobq_s *func_init_central_queue(void)
 	return jobq;
 }
 
-static void initialize_eager_center_policy(struct machine_config_s *config, 
+static void initialize_eager_center_policy(struct starpu_machine_config_s *config, 
 		   __attribute__ ((unused)) struct sched_policy_s *_policy) 
 {
 	setup_queues(init_central_queue_design, func_init_central_queue, config);

+ 1 - 1
src/core/policies/eager-central-priority-policy.c

@@ -37,7 +37,7 @@ static struct jobq_s *func_init_priority_queue(void)
 	return jobq;
 }
 
-static void initialize_eager_center_priority_policy(struct machine_config_s *config, 
+static void initialize_eager_center_priority_policy(struct starpu_machine_config_s *config, 
 			__attribute__ ((unused))	struct sched_policy_s *_policy) 
 {
 	setup_queues(init_priority_queue_design, func_init_priority_queue, config);

+ 1 - 1
src/core/policies/no-prio-policy.c

@@ -43,7 +43,7 @@ static struct jobq_s *func_init_central_queue(void)
 	return jobq;
 }
 
-void initialize_no_prio_policy(struct machine_config_s *config, 
+void initialize_no_prio_policy(struct starpu_machine_config_s *config, 
 	   __attribute__ ((unused)) struct sched_policy_s *_policy) 
 {
 	setup_queues(init_no_prio_design, func_init_central_queue, config);

+ 1 - 1
src/core/policies/random-policy.c

@@ -95,7 +95,7 @@ static struct jobq_s *init_random_fifo(void)
 	return q;
 }
 
-static void initialize_random_policy(struct machine_config_s *config, 
+static void initialize_random_policy(struct starpu_machine_config_s *config, 
 	 __attribute__ ((unused)) struct sched_policy_s *_policy) 
 {
 	nworkers = 0;

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

@@ -125,7 +125,7 @@ static void display_sched_help_message(void)
 	 }
 }
 
-static struct sched_policy_s *select_sched_policy(struct machine_config_s *config)
+static struct sched_policy_s *select_sched_policy(struct starpu_machine_config_s *config)
 {
 	struct sched_policy_s *selected_policy = NULL;
 	struct starpu_conf *user_conf = config->user_conf;
@@ -155,7 +155,7 @@ static struct sched_policy_s *select_sched_policy(struct machine_config_s *confi
 	return &sched_eager_policy;
 }
 
-void init_sched_policy(struct machine_config_s *config)
+void init_sched_policy(struct starpu_machine_config_s *config)
 {
 	/* Perhaps we have to display some help */
 	display_sched_help_message();
@@ -172,7 +172,7 @@ void init_sched_policy(struct machine_config_s *config)
 	policy.init_sched(config, &policy);
 }
 
-void deinit_sched_policy(struct machine_config_s *config)
+void deinit_sched_policy(struct starpu_machine_config_s *config)
 {
 	if (policy.deinit_sched)
 		policy.deinit_sched(config, &policy);
@@ -200,7 +200,7 @@ int push_task(starpu_job_t j)
 	{
 		struct starpu_task *task = j->task;
 		unsigned workerid = task->workerid;
-		struct worker_s *worker = _starpu_get_worker_struct(workerid);
+		struct starpu_worker_s *worker = _starpu_get_worker_struct(workerid);
 		
 		if (use_prefetch)
 		{

+ 5 - 5
src/core/policies/sched_policy.h

@@ -25,14 +25,14 @@
 
 #include <core/workers.h>
 
-struct machine_config_s;
+struct starpu_machine_config_s;
 
 struct sched_policy_s {
 	/* create all the queues */
-	void (*init_sched)(struct machine_config_s *, struct sched_policy_s *);
+	void (*init_sched)(struct starpu_machine_config_s *, struct sched_policy_s *);
 
 	/* cleanup method at termination */
-	void (*deinit_sched)(struct machine_config_s *, struct sched_policy_s *);
+	void (*deinit_sched)(struct starpu_machine_config_s *, struct sched_policy_s *);
 
 	/* anyone can request which queue it is associated to */
 	struct jobq_s *(*get_local_queue)(struct sched_policy_s *);
@@ -52,8 +52,8 @@ struct sched_policy_s {
 
 struct sched_policy_s *get_sched_policy(void);
 
-void init_sched_policy(struct machine_config_s *config);
-void deinit_sched_policy(struct machine_config_s *config);
+void init_sched_policy(struct starpu_machine_config_s *config);
+void deinit_sched_policy(struct starpu_machine_config_s *config);
 //void set_local_queue(struct jobq_s *jobq);
 
 int push_task(starpu_job_t task);

+ 2 - 2
src/core/policies/work-stealing-policy.c

@@ -17,7 +17,7 @@
 #include <core/policies/work-stealing-policy.h>
 
 /* save the general machine configuration */
-//static struct machine_config_s *machineconfig;
+//static struct starpu_machine_config_s *machineconfig;
 
 static unsigned nworkers;
 static unsigned rr_worker;
@@ -174,7 +174,7 @@ static struct jobq_s *init_ws_deque(void)
 	return q;
 }
 
-static void initialize_ws_policy(struct machine_config_s *config, 
+static void initialize_ws_policy(struct starpu_machine_config_s *config, 
 				__attribute__ ((unused)) struct sched_policy_s *_policy) 
 {
 	nworkers = 0;

+ 14 - 14
src/core/topology.c

@@ -32,10 +32,10 @@
 		
 static unsigned topology_is_initialized = 0;
 
-static void _starpu_initialize_workers_bindid(struct machine_config_s *config);
+static void _starpu_initialize_workers_bindid(struct starpu_machine_config_s *config);
 
 #ifdef STARPU_USE_CUDA
-static void _starpu_initialize_workers_gpuid(struct machine_config_s *config);
+static void _starpu_initialize_workers_gpuid(struct starpu_machine_config_s *config);
 static unsigned may_bind_automatically = 0;
 #endif
 
@@ -44,7 +44,7 @@ static unsigned may_bind_automatically = 0;
  */
 
 #ifdef STARPU_USE_CUDA
-static void _starpu_initialize_workers_gpuid(struct machine_config_s *config)
+static void _starpu_initialize_workers_gpuid(struct starpu_machine_config_s *config)
 {
 	char *strval;
 	unsigned i;
@@ -114,14 +114,14 @@ static void _starpu_initialize_workers_gpuid(struct machine_config_s *config)
 }
 #endif
 
-static inline int _starpu_get_next_gpuid(struct machine_config_s *config)
+static inline int _starpu_get_next_gpuid(struct starpu_machine_config_s *config)
 {
 	unsigned i = ((config->current_gpuid++) % config->ncudagpus);
 
 	return (int)config->workers_gpuid[i];
 }
 
-static void _starpu_init_topology(struct machine_config_s *config)
+static void _starpu_init_topology(struct starpu_machine_config_s *config)
 {
 	if (!topology_is_initialized)
 	{
@@ -154,14 +154,14 @@ static void _starpu_init_topology(struct machine_config_s *config)
 	}
 }
 
-unsigned _starpu_topology_get_nhwcpu(struct machine_config_s *config)
+unsigned _starpu_topology_get_nhwcpu(struct starpu_machine_config_s *config)
 {
 	_starpu_init_topology(config);
 	
 	return config->nhwcpus;
 }
 
-static int _starpu_init_machine_config(struct machine_config_s *config,
+static int _starpu_init_machine_config(struct starpu_machine_config_s *config,
 				struct starpu_conf *user_conf)
 {
 	int explicitval __attribute__((unused));
@@ -304,7 +304,7 @@ static int _starpu_init_machine_config(struct machine_config_s *config,
 /*
  * Bind workers on the different processors
  */
-static void _starpu_initialize_workers_bindid(struct machine_config_s *config)
+static void _starpu_initialize_workers_bindid(struct starpu_machine_config_s *config)
 {
 	char *strval;
 	unsigned i;
@@ -374,7 +374,7 @@ static void _starpu_initialize_workers_bindid(struct machine_config_s *config)
  * worker. In case a list of preferred cpus was specified, we look for a an
  * available cpu among the list if possible, otherwise a round-robin policy is
  * used. */
-static inline int _starpu_get_next_bindid(struct machine_config_s *config,
+static inline int _starpu_get_next_bindid(struct starpu_machine_config_s *config,
 				int *preferred_binding, int npreferred)
 {
 	unsigned found = 0;
@@ -413,7 +413,7 @@ static inline int _starpu_get_next_bindid(struct machine_config_s *config,
 	return (int)config->workers_bindid[i];
 }
 
-void _starpu_bind_thread_on_cpu(struct machine_config_s *config __attribute__((unused)), unsigned cpuid)
+void _starpu_bind_thread_on_cpu(struct starpu_machine_config_s *config __attribute__((unused)), unsigned cpuid)
 {
 #ifdef STARPU_HAVE_HWLOC
 	int ret;
@@ -456,7 +456,7 @@ void _starpu_bind_thread_on_cpu(struct machine_config_s *config __attribute__((u
 #endif
 }
 
-static void _starpu_init_workers_binding(struct machine_config_s *config)
+static void _starpu_init_workers_binding(struct starpu_machine_config_s *config)
 {
 	/* launch one thread per CPU */
 	unsigned ram_memory_node;
@@ -473,7 +473,7 @@ static void _starpu_init_workers_binding(struct machine_config_s *config)
 	{
 		unsigned memory_node = -1;
 		unsigned is_a_set_of_accelerators = 0;
-		struct worker_s *workerarg = &config->workers[worker];
+		struct starpu_worker_s *workerarg = &config->workers[worker];
 
 		/* Perhaps the worker has some "favourite" bindings  */
 		int *preferred_binding = NULL;
@@ -523,7 +523,7 @@ static void _starpu_init_workers_binding(struct machine_config_s *config)
 }
 
 
-int starpu_build_topology(struct machine_config_s *config)
+int starpu_build_topology(struct starpu_machine_config_s *config)
 {
 	int ret;
 
@@ -541,7 +541,7 @@ int starpu_build_topology(struct machine_config_s *config)
 	return 0;
 }
 
-void starpu_destroy_topology(struct machine_config_s *config __attribute__ ((unused)))
+void starpu_destroy_topology(struct starpu_machine_config_s *config __attribute__ ((unused)))
 {
 	/* cleanup StarPU internal data structures */
 	_starpu_deinit_memory_nodes();

+ 6 - 6
src/core/topology.h

@@ -30,7 +30,7 @@
 #include <starpu.h>
 
 /* TODO actually move this struct into this header */
-struct machine_config_s;
+struct starpu_machine_config_s;
 
 /* This structure is "inspired" by the hwloc project
  * (see http://www.open-mpi.org/projects/hwloc/) */
@@ -59,16 +59,16 @@ struct starpu_topo_obj_t {
 
 	/* flags */
 	unsigned is_a_worker;
-	struct worker_s *worker; /* (ignored if !is_a_worker) */
+	struct starpu_worker_s *worker; /* (ignored if !is_a_worker) */
 };
 
-int starpu_build_topology(struct machine_config_s *config);
+int starpu_build_topology(struct starpu_machine_config_s *config);
 
-void starpu_destroy_topology(struct machine_config_s *config);
+void starpu_destroy_topology(struct starpu_machine_config_s *config);
 
 /* returns the number of physical cpus */
-unsigned _starpu_topology_get_nhwcpu(struct machine_config_s *config);
+unsigned _starpu_topology_get_nhwcpu(struct starpu_machine_config_s *config);
 
-void _starpu_bind_thread_on_cpu(struct machine_config_s *config, unsigned cpuid);
+void _starpu_bind_thread_on_cpu(struct starpu_machine_config_s *config, unsigned cpuid);
 
 #endif // __TOPOLOGY_H__

+ 17 - 17
src/core/workers.c

@@ -26,9 +26,9 @@
 
 static pthread_key_t worker_key;
 
-static struct machine_config_s config;
+static struct starpu_machine_config_s config;
 
-struct machine_config_s *_starpu_get_machine_config(void)
+struct starpu_machine_config_s *_starpu_get_machine_config(void)
 {
 	return &config;
 }
@@ -41,12 +41,12 @@ inline uint32_t _starpu_worker_exists(uint32_t task_mask)
 	return (task_mask & config.worker_mask);
 } 
 
-inline uint32_t may_submit_cuda_task(void)
+inline uint32_t _starpu_may_submit_cuda_task(void)
 {
 	return (STARPU_CUDA & config.worker_mask);
 }
 
-inline uint32_t may_submit_cpu_task(void)
+inline uint32_t _starpu_may_submit_cpu_task(void)
 {
 	return (STARPU_CPU & config.worker_mask);
 }
@@ -62,10 +62,10 @@ inline uint32_t _starpu_worker_may_execute_task(unsigned workerid, uint32_t wher
 
 #ifdef STARPU_USE_GORDON
 static unsigned gordon_inited = 0;	
-static struct worker_set_s gordon_worker_set;
+static struct starpu_worker_set_s gordon_worker_set;
 #endif
 
-static void _starpu_init_worker_queue(struct worker_s *workerarg)
+static void _starpu_init_worker_queue(struct starpu_worker_s *workerarg)
 {
 	struct jobq_s *jobq = workerarg->jobq;
 
@@ -92,7 +92,7 @@ static void _starpu_init_worker_queue(struct worker_s *workerarg)
 	_starpu_memory_node_attach_queue(jobq, workerarg->memory_node);
 }
 
-static void _starpu_init_workers(struct machine_config_s *config)
+static void _starpu_init_workers(struct starpu_machine_config_s *config)
 {
 	config->running = 1;
 
@@ -102,7 +102,7 @@ static void _starpu_init_workers(struct machine_config_s *config)
 	unsigned worker;
 	for (worker = 0; worker < config->nworkers; worker++)
 	{
-		struct worker_s *workerarg = &config->workers[worker];
+		struct starpu_worker_s *workerarg = &config->workers[worker];
 
 		workerarg->config = config;
 
@@ -179,7 +179,7 @@ static void _starpu_init_workers(struct machine_config_s *config)
 
 	for (worker = 0; worker < config->nworkers; worker++)
 	{
-		struct worker_s *workerarg = &config->workers[worker];
+		struct starpu_worker_s *workerarg = &config->workers[worker];
 
 		switch (workerarg->arch) {
 			case STARPU_CPU_WORKER:
@@ -202,12 +202,12 @@ static void _starpu_init_workers(struct machine_config_s *config)
 
 }
 
-void _starpu_set_local_worker_key(struct worker_s *worker)
+void _starpu_set_local_worker_key(struct starpu_worker_s *worker)
 {
 	pthread_setspecific(worker_key, worker);
 }
 
-struct worker_s *_starpu_get_local_worker_key(void)
+struct starpu_worker_s *_starpu_get_local_worker_key(void)
 {
 	return pthread_getspecific(worker_key);
 }
@@ -255,7 +255,7 @@ int starpu_init(struct starpu_conf *user_conf)
  * Handle runtime termination 
  */
 
-static void _starpu_terminate_workers(struct machine_config_s *config)
+static void _starpu_terminate_workers(struct starpu_machine_config_s *config)
 {
 	int status;
 	unsigned workerid;
@@ -268,8 +268,8 @@ static void _starpu_terminate_workers(struct machine_config_s *config)
 		fprintf(stderr, "wait for worker %d\n", workerid);
 #endif
 
-		struct worker_set_s *set = config->workers[workerid].set;
-		struct worker_s *worker = &config->workers[workerid];
+		struct starpu_worker_set_s *set = config->workers[workerid].set;
+		struct starpu_worker_s *worker = &config->workers[workerid];
 
 		/* in case StarPU termination code is called from a callback,
  		 * we have to check if pthread_self() is the worker itself */
@@ -405,7 +405,7 @@ static void _starpu_operate_on_all_queues(queue_op op)
 	pthread_rwlock_unlock(&descr->attached_queues_rwlock);
 }
 
-static void _starpu_kill_all_workers(struct machine_config_s *config)
+static void _starpu_kill_all_workers(struct starpu_machine_config_s *config)
 {
 	/* lock all workers and the scheduler (in the proper order) to make
 	   sure everyone will notice the termination */
@@ -483,7 +483,7 @@ unsigned starpu_get_spu_worker_count(void)
  * that is not controlled by StarPU, starpu_get_worker_id returns -1. */
 int starpu_get_worker_id(void)
 {
-	struct worker_s * worker;
+	struct starpu_worker_s * worker;
 
 	worker = _starpu_get_local_worker_key();
 	if (worker)
@@ -497,7 +497,7 @@ int starpu_get_worker_id(void)
 	}
 }
 
-struct worker_s *_starpu_get_worker_struct(unsigned id)
+struct starpu_worker_s *_starpu_get_worker_struct(unsigned id)
 {
 	return &config.workers[id];
 }

+ 16 - 16
src/core/workers.h

@@ -55,13 +55,13 @@
 #define STARPU_GORDON_ALPHA	6.0f /* XXX this is a random value ... */
 
 #ifdef STARPU_DATA_STATS
-#define BENCHMARK_COMM	1
+#define STARPU_BENCHMARK_COMM	1
 #else
-#define BENCHMARK_COMM	0
+#define STARPU_BENCHMARK_COMM	0
 #endif
 
-struct worker_s {
-	struct machine_config_s *config;
+struct starpu_worker_s {
+	struct starpu_machine_config_s *config;
         pthread_mutex_t mutex;
 	enum starpu_archtype arch; /* what is the type of worker ? */
 	uint32_t worker_mask; /* what is the type of worker ? */
@@ -75,7 +75,7 @@ struct worker_s {
 	struct jobq_s *jobq; /* in which queue will that worker get/put tasks ? */
 	struct starpu_job_list_s *local_jobs; /* this queue contains tasks that have been explicitely submitted to that queue */
 	pthread_mutex_t local_jobs_mutex; /* protect the local_jobs list */
-	struct worker_set_s *set; /* in case this worker belongs to a set */
+	struct starpu_worker_set_s *set; /* in case this worker belongs to a set */
 	struct starpu_job_list_s *terminated_jobs; /* list of pending jobs which were executed */
 	unsigned worker_is_running;
 	unsigned worker_is_initialized;
@@ -85,18 +85,18 @@ struct worker_s {
 
 /* in case a single CPU worker may control multiple 
  * accelerators (eg. Gordon for n SPUs) */
-struct worker_set_s {
+struct starpu_worker_set_s {
         pthread_mutex_t mutex;
 	pthread_t worker_thread; /* the thread which runs the worker */
 	unsigned nworkers;
 	unsigned joined; /* only one thread may call pthread_join*/
 	void *retval;
-	struct worker_s *workers;
+	struct starpu_worker_s *workers;
         pthread_cond_t ready_cond; /* indicate when the set is ready */
 	unsigned set_is_initialized;
 };
 
-struct machine_config_s {
+struct starpu_machine_config_s {
 	unsigned nworkers;
 
 #ifdef STARPU_HAVE_HWLOC
@@ -118,7 +118,7 @@ struct machine_config_s {
 	int current_gpuid;
 	unsigned workers_gpuid[STARPU_NMAXWORKERS];
 	
-	struct worker_s workers[STARPU_NMAXWORKERS];
+	struct starpu_worker_s workers[STARPU_NMAXWORKERS];
 	uint32_t worker_mask;
 
 	struct starpu_topo_obj_t *topology;
@@ -131,13 +131,13 @@ struct machine_config_s {
 	unsigned running;
 };
 
-void display_general_stats(void);
+void _starpu_display_general_stats(void);
 
 unsigned _starpu_machine_is_running(void);
 
 inline uint32_t _starpu_worker_exists(uint32_t task_mask);
-inline uint32_t may_submit_cuda_task(void);
-inline uint32_t may_submit_cpu_task(void);
+inline uint32_t _starpu_may_submit_cuda_task(void);
+inline uint32_t _starpu_may_submit_cpu_task(void);
 inline uint32_t _starpu_worker_may_execute_task(unsigned workerid, uint32_t where);
 unsigned _starpu_worker_can_block(unsigned memnode);
 
@@ -145,12 +145,12 @@ 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 _starpu_set_local_worker_key(struct worker_s *worker);
-struct worker_s *_starpu_get_local_worker_key(void);
+void _starpu_set_local_worker_key(struct starpu_worker_s *worker);
+struct starpu_worker_s *_starpu_get_local_worker_key(void);
 
-struct worker_s *_starpu_get_worker_struct(unsigned id);
+struct starpu_worker_s *_starpu_get_worker_struct(unsigned id);
 
-struct machine_config_s *_starpu_get_machine_config(void);
+struct starpu_machine_config_s *_starpu_get_machine_config(void);
 
 /* TODO move */
 unsigned _starpu_execute_registered_progression_hooks(void);

+ 1 - 1
src/datawizard/memory_nodes.c

@@ -146,7 +146,7 @@ void _starpu_memory_node_attach_queue(struct jobq_s *q, unsigned nodeid)
 
 unsigned starpu_get_worker_memory_node(unsigned workerid)
 {
-	struct worker_s *worker = _starpu_get_worker_struct(workerid);
+	struct starpu_worker_s *worker = _starpu_get_worker_struct(workerid);
 
 	return worker->memory_node;
 }

+ 7 - 7
src/drivers/cpu/driver_cpu.c

@@ -20,7 +20,7 @@
 #include "driver_cpu.h"
 #include <core/policies/sched_policy.h>
 
-static int execute_job_on_cpu(starpu_job_t j, struct worker_s *cpu_args)
+static int execute_job_on_cpu(starpu_job_t j, struct starpu_worker_s *cpu_args)
 {
 	int ret;
 	tick_t codelet_start, codelet_end;
@@ -36,12 +36,12 @@ static int execute_job_on_cpu(starpu_job_t j, struct worker_s *cpu_args)
 	if (cl->model && cl->model->benchmarking)
 		calibrate_model = 1;
 
-	if (calibrate_model || BENCHMARK_COMM)
+	if (calibrate_model || STARPU_BENCHMARK_COMM)
 		GET_TICK(codelet_start_comm);
 
 	ret = _starpu_fetch_task_input(task, 0);
 
-	if (calibrate_model || BENCHMARK_COMM)
+	if (calibrate_model || STARPU_BENCHMARK_COMM)
 		GET_TICK(codelet_end_comm);
 
 	if (ret != 0) {
@@ -52,7 +52,7 @@ static int execute_job_on_cpu(starpu_job_t j, struct worker_s *cpu_args)
 
 	TRACE_START_CODELET_BODY(j);
 
-	if (calibrate_model || BENCHMARK_COMM)
+	if (calibrate_model || STARPU_BENCHMARK_COMM)
 		GET_TICK(codelet_start);
 
 	cpu_args->status = STATUS_EXECUTING;
@@ -61,7 +61,7 @@ static int execute_job_on_cpu(starpu_job_t j, struct worker_s *cpu_args)
 
 	cl->per_worker_stats[cpu_args->workerid]++;
 	
-	if (calibrate_model || BENCHMARK_COMM)
+	if (calibrate_model || STARPU_BENCHMARK_COMM)
 		GET_TICK(codelet_end);
 
 	TRACE_END_CODELET_BODY(j);
@@ -70,7 +70,7 @@ static int execute_job_on_cpu(starpu_job_t j, struct worker_s *cpu_args)
 	_starpu_push_task_output(task, 0);
 
 //#ifdef STARPU_MODEL_DEBUG
-	if (calibrate_model || BENCHMARK_COMM)
+	if (calibrate_model || STARPU_BENCHMARK_COMM)
 	{
 		double measured = timing_delay(&codelet_start, &codelet_end);
 		double measured_comm = timing_delay(&codelet_start_comm, &codelet_end_comm);
@@ -96,7 +96,7 @@ static int execute_job_on_cpu(starpu_job_t j, struct worker_s *cpu_args)
 
 void *_starpu_cpu_worker(void *arg)
 {
-	struct worker_s *cpu_arg = arg;
+	struct starpu_worker_s *cpu_arg = arg;
 
 #ifdef STARPU_USE_FXT
 	fxt_register_thread(cpu_arg->bindid);

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

@@ -77,7 +77,7 @@ void _starpu_init_cuda(void)
 	assert(ncudagpus <= STARPU_MAXCUDADEVS);
 }
 
-static int execute_job_on_cuda(starpu_job_t j, struct worker_s *args)
+static int execute_job_on_cuda(starpu_job_t j, struct starpu_worker_s *args)
 {
 	int ret;
 //	uint32_t mask = (1<<0);
@@ -100,7 +100,7 @@ static int execute_job_on_cuda(starpu_job_t j, struct worker_s *args)
 		calibrate_model = 1;
 
 	/* we do not take communication into account when modeling the performance */
-	if (BENCHMARK_COMM)
+	if (STARPU_BENCHMARK_COMM)
 	{
 		cures = cudaThreadSynchronize();
 		if (STARPU_UNLIKELY(cures))
@@ -116,7 +116,7 @@ static int execute_job_on_cuda(starpu_job_t j, struct worker_s *args)
 		return -EAGAIN;
 	}
 
-	if (calibrate_model || BENCHMARK_COMM)
+	if (calibrate_model || STARPU_BENCHMARK_COMM)
 	{
 		cures = cudaThreadSynchronize();
 		if (STARPU_UNLIKELY(cures))
@@ -140,7 +140,7 @@ static int execute_job_on_cuda(starpu_job_t j, struct worker_s *args)
 
 	TRACE_END_CODELET_BODY(j);	
 
-	if (calibrate_model || BENCHMARK_COMM)
+	if (calibrate_model || STARPU_BENCHMARK_COMM)
 	{
 		double measured = timing_delay(&codelet_start, &codelet_end);
 		double measured_comm = timing_delay(&codelet_start_comm, &codelet_end_comm);
@@ -165,7 +165,7 @@ static int execute_job_on_cuda(starpu_job_t j, struct worker_s *args)
 
 void *_starpu_cuda_worker(void *arg)
 {
-	struct worker_s* args = arg;
+	struct starpu_worker_s* args = arg;
 
 	int devid = args->id;
 	unsigned memory_node = args->memory_node;

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

@@ -33,7 +33,7 @@ pthread_mutex_t progress_mutex;
 
 struct gordon_task_wrapper_s {
 	/* who has executed that ? */
-	struct worker_s *worker;
+	struct starpu_worker_s *worker;
 
 	struct starpu_job_list_s *list;	/* StarPU */
 	struct gordon_ppu_job_s *gordon_job; /* gordon*/
@@ -49,7 +49,7 @@ void *gordon_worker_progress(void *arg)
 	fprintf(stderr, "gordon_worker_progress\n");
 
 	/* fix the thread on the correct cpu */
-	struct worker_set_s *gordon_set_arg = arg;
+	struct starpu_worker_set_s *gordon_set_arg = arg;
 	unsigned prog_thread_bind_id = 
 		(gordon_set_arg->workers[0].bindid + 1)%(gordon_set_arg->config->nhwcores);
 	_starpu_bind_thread_on_cpu(gordon_set_arg->config, prog_thread_bind_id);
@@ -182,7 +182,7 @@ static void gordon_callback_list_func(void *arg)
 
 	/* we don't know who will execute that codelet : so we actually defer the
  	 * execution of the StarPU codelet and the job termination later */
-	struct worker_s *worker = task_wrapper->worker;
+	struct starpu_worker_s *worker = task_wrapper->worker;
 	STARPU_ASSERT(worker);
 
 	wrapper_list = task_wrapper->list;
@@ -230,7 +230,7 @@ static void gordon_callback_func(void *arg)
 
 	/* we don't know who will execute that codelet : so we actually defer the
  	 * execution of the StarPU codelet and the job termination later */
-	struct worker_s *worker = task_wrapper->worker;
+	struct starpu_worker_s *worker = task_wrapper->worker;
 	STARPU_ASSERT(worker);
 
 	task_wrapper->terminated = 1;
@@ -243,7 +243,7 @@ static void gordon_callback_func(void *arg)
 	free(task_wrapper);
 }
 
-int inject_task(starpu_job_t j, struct worker_s *worker)
+int inject_task(starpu_job_t j, struct starpu_worker_s *worker)
 {
 	struct starpu_task *task = j->task;
 	int ret = _starpu_fetch_task_input(task, 0);
@@ -263,7 +263,7 @@ int inject_task(starpu_job_t j, struct worker_s *worker)
 	return 0;
 }
 
-int inject_task_list(struct starpu_job_list_s *list, struct worker_s *worker)
+int inject_task_list(struct starpu_job_list_s *list, struct starpu_worker_s *worker)
 {
 	/* first put back all tasks that can not be performed by Gordon */
 	unsigned nvalids = 0;
@@ -324,7 +324,7 @@ int inject_task_list(struct starpu_job_list_s *list, struct worker_s *worker)
 	return 0;
 }
 
-void *gordon_worker_inject(struct worker_set_s *arg)
+void *gordon_worker_inject(struct starpu_worker_set_s *arg)
 {
 
 	while(_starpu_machine_is_running()) {
@@ -410,7 +410,7 @@ void *gordon_worker_inject(struct worker_set_s *arg)
 
 void *_starpu_gordon_worker(void *arg)
 {
-	struct worker_set_s *gordon_set_arg = arg;
+	struct starpu_worker_set_s *gordon_set_arg = arg;
 
 	_starpu_bind_thread_on_cpu(gordon_set_arg->config, gordon_set_arg->workers[0].bindid);
 
@@ -426,7 +426,7 @@ void *_starpu_gordon_worker(void *arg)
 	unsigned spu;
 	for (spu = 0; spu < gordon_set_arg->nworkers; spu++)
 	{
-		struct worker_s *worker = &gordon_set_arg->workers[spu];
+		struct starpu_worker_s *worker = &gordon_set_arg->workers[spu];
 		snprintf(worker->name, 32, "SPU %d", worker->id);
 	}
 

+ 2 - 2
src/util/malloc.c

@@ -55,7 +55,7 @@ int starpu_malloc_pinned_if_possible(void **A, size_t dim)
 
 	STARPU_ASSERT(A);
 
-	if (may_submit_cuda_task())
+	if (_starpu_may_submit_cuda_task())
 	{
 #ifdef STARPU_USE_CUDA
 		int push_res;
@@ -107,7 +107,7 @@ int starpu_free_pinned_if_possible(void *A)
 	if (STARPU_UNLIKELY(!_starpu_worker_may_perform_blocking_calls()))
 		return -EDEADLK;
 
-	if (may_submit_cuda_task())
+	if (_starpu_may_submit_cuda_task())
 	{
 #ifdef STARPU_USE_CUDA
 		int push_res;