浏览代码

Add can_execute member to the codelet structure, so applications can fine-tune implementations usage. Make most schedulers enforce it.

Samuel Thibault 13 年之前
父节点
当前提交
2f4491e804

+ 30 - 18
doc/chapters/advanced-api.texi

@@ -167,7 +167,7 @@ Return the expected power consumption of the entire task bundle in J.
 @section Task Lists
 @section Task Lists
 
 
 @deftp {Data Type} {struct starpu_task_list}
 @deftp {Data Type} {struct starpu_task_list}
-todo
+Stores a double-chained list of tasks
 @end deftp
 @end deftp
 
 
 @deftypefun void starpu_task_list_init ({struct starpu_task_list *}@var{list})
 @deftypefun void starpu_task_list_init ({struct starpu_task_list *}@var{list})
@@ -182,11 +182,11 @@ Push a task at the front of a list
 Push a task at the back of a list
 Push a task at the back of a list
 @end deftypefun
 @end deftypefun
 
 
-@deftypefun struct starpu_task *starpu_task_list_front ({struct starpu_task_list *}@var{list})
+@deftypefun {struct starpu_task *}starpu_task_list_front ({struct starpu_task_list *}@var{list})
 Get the front of the list (without removing it)
 Get the front of the list (without removing it)
 @end deftypefun
 @end deftypefun
 
 
-@deftypefun struct starpu_task *starpu_task_list_back ({struct starpu_task_list *}@var{list})
+@deftypefun {struct starpu_task *}starpu_task_list_back ({struct starpu_task_list *}@var{list})
 Get the back of the list (without removing it)
 Get the back of the list (without removing it)
 @end deftypefun
 @end deftypefun
 
 
@@ -198,14 +198,26 @@ Test if a list is empty
 Remove an element from the list
 Remove an element from the list
 @end deftypefun
 @end deftypefun
 
 
-@deftypefun struct starpu_task *starpu_task_list_pop_front ({struct starpu_task_list *}@var{list})
+@deftypefun {struct starpu_task *}starpu_task_list_pop_front ({struct starpu_task_list *}@var{list})
 Remove the element at the front of the list
 Remove the element at the front of the list
 @end deftypefun
 @end deftypefun
 
 
-@deftypefun struct starpu_task *starpu_task_list_pop_back ({struct starpu_task_list *}@var{list})
+@deftypefun {struct starpu_task *}starpu_task_list_pop_back ({struct starpu_task_list *}@var{list})
 Remove the element at the back of the list
 Remove the element at the back of the list
 @end deftypefun
 @end deftypefun
 
 
+@deftypefun {struct starpu_task *}starpu_task_list_begin ({struct starpu_task_list *}@var{list})
+Get the first task of the list.
+@end deftypefun
+
+@deftypefun {struct starpu_task *}starpu_task_list_end ({struct starpu_task_list *}@var{list})
+Get the end of the list.
+@end deftypefun
+
+@deftypefun {struct starpu_task *}starpu_task_list_next ({struct starpu_task *}@var{task})
+Get the next task of the list. This is not erase-safe.
+@end deftypefun
+
 @node Defining a new scheduling policy
 @node Defining a new scheduling policy
 @section Defining a new scheduling policy
 @section Defining a new scheduling policy
 
 
@@ -234,14 +246,13 @@ Initialize the scheduling policy.
 Cleanup the scheduling policy.
 Cleanup the scheduling policy.
 @item @code{push_task}
 @item @code{push_task}
 Insert a task into the scheduler.
 Insert a task into the scheduler.
-@item @code{push_prio_task}
-Insert a priority task into the scheduler.
-@item @code{push_prio_notify}
-Notify the scheduler that a task was pushed on the worker. This method is
-called when a task that was explicitely assigned to a worker is scheduled. This
-method therefore permits to keep the state of of the scheduler coherent even
-when StarPU bypasses the scheduling strategy.
-@item @code{pop_task}
+@item @code{push_task_notify}
+Notify the scheduler that a task was pushed on a given worker. This method is
+called when a task that was explicitely assigned to a worker becomes ready and
+is about to be executed by the worker. This method therefore permits to keep
+the state of of the scheduler coherent even when StarPU bypasses the scheduling
+strategy.
+@item @code{pop_task} (optional)
 Get a task from the scheduler. The mutex associated to the worker is already
 Get a task from the scheduler. The mutex associated to the worker is already
 taken when this method is called. If this method is defined as @code{NULL}, the
 taken when this method is called. If this method is defined as @code{NULL}, the
 worker will only execute tasks from its local queue. In this case, the
 worker will only execute tasks from its local queue. In this case, the
@@ -250,13 +261,14 @@ assign tasks to the different workers.
 @item @code{pop_every_task}
 @item @code{pop_every_task}
 Remove all available tasks from the scheduler (tasks are chained by the means
 Remove all available tasks from the scheduler (tasks are chained by the means
 of the prev and next fields of the starpu_task structure). The mutex associated
 of the prev and next fields of the starpu_task structure). The mutex associated
-to the worker is already taken when this method is called.
+to the worker is already taken when this method is called. This is currently
+only used by the Gordon driver.
 @item @code{post_exec_hook} (optional)
 @item @code{post_exec_hook} (optional)
 This method is called every time a task has been executed.
 This method is called every time a task has been executed.
 @item @code{policy_name}
 @item @code{policy_name}
 Name of the policy (optional).
 Name of the policy (optional).
 @item @code{policy_description}
 @item @code{policy_description}
-Description of the policy (optionnal).
+Description of the policy (optional).
 @end table
 @end table
 @end deftp
 @end deftp
 
 
@@ -307,12 +319,12 @@ where the worker will pop tasks first. Setting @var{back} to 0 therefore ensures
 a FIFO ordering.
 a FIFO ordering.
 @end deftypefun
 @end deftypefun
 
 
-@deftypefun int starpu_worker_may_execute_task (unsigned @var{workerid}, {struct starpu_task *}@var{task}, unsigned {nimpl})
-Check if the worker specified by workerid can execute the codelet.
+@deftypefun int starpu_worker_may_run_task (unsigned @var{workerid}, {struct starpu_task *}@var{task}, unsigned {nimpl})
+Check if the worker specified by workerid can execute the codelet. Schedulers need to call it before assigning a task to a worker, otherwise the task may fail to execute.
 @end deftypefun
 @end deftypefun
 
 
 @deftypefun double starpu_timing_now (void)
 @deftypefun double starpu_timing_now (void)
-Return the current date
+Return the current date in µs
 @end deftypefun
 @end deftypefun
 
 
 @deftypefun double starpu_task_expected_length ({struct starpu_task *}@var{task}, {enum starpu_perf_archtype} @var{arch}, unsigned @var{nimpl})
 @deftypefun double starpu_task_expected_length ({struct starpu_task *}@var{task}, {enum starpu_perf_archtype} @var{arch}, unsigned @var{nimpl})

+ 45 - 0
doc/chapters/advanced-examples.texi

@@ -11,6 +11,7 @@
 
 
 @menu
 @menu
 * Using multiple implementations of a codelet::
 * Using multiple implementations of a codelet::
+* Enabling implementation according to capabilities::
 * Task and Worker Profiling::   
 * Task and Worker Profiling::   
 * Partitioning Data::           Partitioning Data
 * Partitioning Data::           Partitioning Data
 * Performance model example::   
 * Performance model example::   
@@ -68,6 +69,50 @@ struct starpu_codelet cl = @{
 Scheduler which are multi-implementation aware (only @code{dmda}, @code{heft}
 Scheduler which are multi-implementation aware (only @code{dmda}, @code{heft}
 and @code{pheft} for now) will use the performance models of all the
 and @code{pheft} for now) will use the performance models of all the
 implementations it was given, and pick the one that seems to be the fastest.
 implementations it was given, and pick the one that seems to be the fastest.
+
+@node Enabling implementation according to capabilities
+@section Enabling implementation according to capabilities
+
+Some implementations may not run on some devices. For instance, some GPU
+devices do not support double floating point precision, and thus the kernel
+execution would just fail; or the GPU may not have enough shared memory for
+the implementation being used. The @code{can_execute} field of the @code{struct
+starpu_codelet} structure permits to express this. For instance:
+
+@cartouche
+@smallexample
+static int can_execute(unsigned workerid, struct starpu_task *task, unsigned nimpl)
+@{
+  const struct cudaDeviceProp *props;
+  if (starpu_worker_get_type(workerid) == STARPU_CPU_WORKER)
+    return 1;
+  /* Cuda device */
+  props = starpu_cuda_get_device_properties(workerid);
+  if (props->major >= 2 || props->minor >= 3)
+    /* At least compute capability 1.3, supports doubles */
+    return 1;
+  /* Old card, does not support doubles */
+  return 0;
+@}
+
+struct starpu_codelet cl = @{
+    .where = STARPU_CPU|STARPU_GPU,
+    .can_execute = can_execute,
+    .cpu_func = cpu_func,
+    .gpu_func = gpu_func
+    .nbuffers = 1
+@};
+@end smallexample
+@end cartouche
+
+This can be essential e.g. when running on a machine which mixes various models
+of GPUs, to take benefit from the new models without crashing on old models.
+
+Note: the @code{can_execute} function is called by the scheduler each time it
+tries to match a task with a worker, and should thus be very fast. The
+@code{starpu_cuda_get_device_properties} provides a quick access to CUDA
+properties of CUDA devices to achieve such efficiency.
+
 @node Task and Worker Profiling
 @node Task and Worker Profiling
 @section Task and Worker Profiling
 @section Task and Worker Profiling
 
 

+ 18 - 0
examples/reductions/dot_product.c

@@ -20,6 +20,7 @@
 #ifdef STARPU_USE_CUDA
 #ifdef STARPU_USE_CUDA
 #include <cuda.h>
 #include <cuda.h>
 #include <cublas.h>
 #include <cublas.h>
+#include <starpu_cuda.h>
 #endif
 #endif
 
 
 #define FPRINTF(ofile, fmt, args ...) do { if (!getenv("STARPU_SSILENT")) {fprintf(ofile, fmt, ##args); }} while(0)
 #define FPRINTF(ofile, fmt, args ...) do { if (!getenv("STARPU_SSILENT")) {fprintf(ofile, fmt, ##args); }} while(0)
@@ -37,6 +38,20 @@ static unsigned entries_per_block = 1024;
 static DOT_TYPE dot = 0.0f;
 static DOT_TYPE dot = 0.0f;
 static starpu_data_handle_t dot_handle;
 static starpu_data_handle_t dot_handle;
 
 
+static int can_execute(unsigned workerid, struct starpu_task *task, unsigned nimpl)
+{
+	const struct cudaDeviceProp *props;
+	if (starpu_worker_get_type(workerid) == STARPU_CPU_WORKER)
+		return 1;
+	/* Cuda device */
+	props = starpu_cuda_get_device_properties(workerid);
+	if (props->major >= 2 || props->minor >= 3)
+		/* At least compute capability 1.3, supports doubles */
+		return 0;
+	/* Old card, does not support doubles */
+	return 0;
+}
+
 /*
 /*
  *	Codelet to create a neutral element
  *	Codelet to create a neutral element
  */
  */
@@ -58,6 +73,7 @@ void init_cuda_func(void *descr[], void *cl_arg)
 
 
 static struct starpu_codelet init_codelet = {
 static struct starpu_codelet init_codelet = {
 	.where = STARPU_CPU|STARPU_CUDA,
 	.where = STARPU_CPU|STARPU_CUDA,
+	.can_execute = can_execute,
 	.cpu_func = init_cpu_func,
 	.cpu_func = init_cpu_func,
 #ifdef STARPU_USE_CUDA
 #ifdef STARPU_USE_CUDA
 	.cuda_func = init_cuda_func,
 	.cuda_func = init_cuda_func,
@@ -83,6 +99,7 @@ extern void redux_cuda_func(void *descr[], void *_args);
 
 
 static struct starpu_codelet redux_codelet = {
 static struct starpu_codelet redux_codelet = {
 	.where = STARPU_CPU|STARPU_CUDA,
 	.where = STARPU_CPU|STARPU_CUDA,
+	.can_execute = can_execute,
 	.cpu_func = redux_cpu_func,
 	.cpu_func = redux_cpu_func,
 #ifdef STARPU_USE_CUDA
 #ifdef STARPU_USE_CUDA
 	.cuda_func = redux_cuda_func,
 	.cuda_func = redux_cuda_func,
@@ -144,6 +161,7 @@ void dot_cuda_func(void *descr[], void *cl_arg)
 
 
 static struct starpu_codelet dot_codelet = {
 static struct starpu_codelet dot_codelet = {
 	.where = STARPU_CPU|STARPU_CUDA,
 	.where = STARPU_CPU|STARPU_CUDA,
+	.can_execute = can_execute,
 	.cpu_func = dot_cpu_func,
 	.cpu_func = dot_cpu_func,
 #ifdef STARPU_USE_CUDA
 #ifdef STARPU_USE_CUDA
 	.cuda_func = dot_cuda_func,
 	.cuda_func = dot_cuda_func,

+ 3 - 3
include/starpu_scheduler.h

@@ -119,7 +119,7 @@ void starpu_worker_set_sched_condition(int workerid, pthread_cond_t *sched_cond,
 #endif
 #endif
 
 
 /* Check if the worker specified by workerid can execute the codelet. */
 /* Check if the worker specified by workerid can execute the codelet. */
-int starpu_worker_may_execute_task(unsigned workerid, struct starpu_task *task, unsigned nimpl);
+int starpu_worker_can_execute_task(unsigned workerid, struct starpu_task *task, unsigned nimpl);
 
 
 /* The scheduling policy may put tasks directly into a worker's local queue so
 /* The scheduling policy may put tasks directly into a worker's local queue so
  * that it is not always necessary to create its own queue when the local queue
  * that it is not always necessary to create its own queue when the local queue
@@ -154,8 +154,8 @@ void starpu_sched_set_max_priority(int max_prio);
 int starpu_combined_worker_assign_workerid(int nworkers, int workerid_array[]);
 int starpu_combined_worker_assign_workerid(int nworkers, int workerid_array[]);
 /* Get the description of a combined worker */
 /* Get the description of a combined worker */
 int starpu_combined_worker_get_description(int workerid, int *worker_size, int **combined_workerid);
 int starpu_combined_worker_get_description(int workerid, int *worker_size, int **combined_workerid);
-/* Variant of starpu_worker_may_execute_task compatible with combined workers */
-int starpu_combined_worker_may_execute_task(unsigned workerid, struct starpu_task *task, unsigned nimpl);
+/* Variant of starpu_worker_can_execute_task compatible with combined workers */
+int starpu_combined_worker_can_execute_task(unsigned workerid, struct starpu_task *task, unsigned nimpl);
 
 
 /*
 /*
  *	Data prefetching
  *	Data prefetching

+ 2 - 0
include/starpu_task.h

@@ -74,9 +74,11 @@ typedef uint8_t starpu_gordon_func_t; /* Cell SPU */
  * A codelet describes the various function 
  * A codelet describes the various function 
  * that may be called from a worker
  * that may be called from a worker
  */
  */
+struct starpu_task;
 struct starpu_codelet {
 struct starpu_codelet {
 	/* where can it be performed ? */
 	/* where can it be performed ? */
 	uint32_t where;
 	uint32_t where;
+	int (*can_execute)(unsigned workerid, struct starpu_task *task, unsigned nimpl);
 	unsigned type;
 	unsigned type;
 	int max_parallelism;
 	int max_parallelism;
 
 

+ 1 - 1
src/core/task.c

@@ -258,7 +258,7 @@ int starpu_task_submit(struct starpu_task *task)
 		/* In case we require that a task should be explicitely
 		/* In case we require that a task should be explicitely
 		 * executed on a specific worker, we make sure that the worker
 		 * executed on a specific worker, we make sure that the worker
 		 * is able to execute this task.  */
 		 * is able to execute this task.  */
-		if (task->execute_on_a_specific_worker && !starpu_combined_worker_may_execute_task(task->workerid, task, 0)) {
+		if (task->execute_on_a_specific_worker && !starpu_combined_worker_can_execute_task(task->workerid, task, 0)) {
                         _STARPU_LOG_OUT_TAG("ENODEV");
                         _STARPU_LOG_OUT_TAG("ENODEV");
 			return -ENODEV;
 			return -ENODEV;
                 }
                 }

+ 11 - 12
src/core/workers.c

@@ -54,22 +54,22 @@ uint32_t _starpu_worker_exists(uint32_t task_mask)
 	return (task_mask & config.worker_mask);
 	return (task_mask & config.worker_mask);
 } 
 } 
 
 
-uint32_t _starpu_may_submit_cuda_task(void)
+uint32_t _starpu_can_submit_cuda_task(void)
 {
 {
 	return (STARPU_CUDA & config.worker_mask);
 	return (STARPU_CUDA & config.worker_mask);
 }
 }
 
 
-uint32_t _starpu_may_submit_cpu_task(void)
+uint32_t _starpu_can_submit_cpu_task(void)
 {
 {
 	return (STARPU_CPU & config.worker_mask);
 	return (STARPU_CPU & config.worker_mask);
 }
 }
 
 
-uint32_t _starpu_may_submit_opencl_task(void)
+uint32_t _starpu_can_submit_opencl_task(void)
 {
 {
 	return (STARPU_OPENCL & config.worker_mask);
 	return (STARPU_OPENCL & config.worker_mask);
 }
 }
 
 
-static int _starpu_may_use_nth_implementation(enum starpu_archtype arch, struct starpu_codelet *cl, unsigned nimpl)
+static int _starpu_can_use_nth_implementation(enum starpu_archtype arch, struct starpu_codelet *cl, unsigned nimpl)
 {
 {
 	switch(arch) {
 	switch(arch) {
 	case STARPU_CPU_WORKER:
 	case STARPU_CPU_WORKER:
@@ -94,18 +94,17 @@ static int _starpu_may_use_nth_implementation(enum starpu_archtype arch, struct
 }
 }
 
 
 
 
-int starpu_worker_may_execute_task(unsigned workerid, struct starpu_task *task, unsigned nimpl)
+int starpu_worker_can_execute_task(unsigned workerid, struct starpu_task *task, unsigned nimpl)
 {
 {
 	/* TODO: check that the task operand sizes will fit on that device */
 	/* TODO: check that the task operand sizes will fit on that device */
-	/* TODO: call application-provided function for various cases like
-	 * double support, shared memory size limit, etc. */
-	return ((task->cl->where & config.workers[workerid].worker_mask) &&
-		_starpu_may_use_nth_implementation(config.workers[workerid].arch, task->cl, nimpl));
+	return (task->cl->where & config.workers[workerid].worker_mask) &&
+		_starpu_can_use_nth_implementation(config.workers[workerid].arch, task->cl, nimpl) &&
+		(!task->cl->can_execute || task->cl->can_execute(workerid, task, nimpl));
 }
 }
 
 
 
 
 
 
-int starpu_combined_worker_may_execute_task(unsigned workerid, struct starpu_task *task, unsigned nimpl)
+int starpu_combined_worker_can_execute_task(unsigned workerid, struct starpu_task *task, unsigned nimpl)
 {
 {
 	/* TODO: check that the task operand sizes will fit on that device */
 	/* TODO: check that the task operand sizes will fit on that device */
 	/* TODO: call application-provided function for various cases like
 	/* TODO: call application-provided function for various cases like
@@ -118,7 +117,7 @@ int starpu_combined_worker_may_execute_task(unsigned workerid, struct starpu_tas
 	if (workerid < nworkers)
 	if (workerid < nworkers)
 	{
 	{
 		return !!((task->cl->where & config.workers[workerid].worker_mask) &&
 		return !!((task->cl->where & config.workers[workerid].worker_mask) &&
-				_starpu_may_use_nth_implementation(config.workers[workerid].arch, task->cl, nimpl));
+				_starpu_can_use_nth_implementation(config.workers[workerid].arch, task->cl, nimpl));
 	}
 	}
 	else {
 	else {
 		if ((cl->type == STARPU_SPMD) || (cl->type == STARPU_FORKJOIN))
 		if ((cl->type == STARPU_SPMD) || (cl->type == STARPU_FORKJOIN))
@@ -128,7 +127,7 @@ int starpu_combined_worker_may_execute_task(unsigned workerid, struct starpu_tas
 			/* Is the worker larger than requested ? */
 			/* Is the worker larger than requested ? */
 			int worker_size = (int)config.combined_workers[workerid - nworkers].worker_size;
 			int worker_size = (int)config.combined_workers[workerid - nworkers].worker_size;
 			return !!((worker_size <= task->cl->max_parallelism) &&
 			return !!((worker_size <= task->cl->max_parallelism) &&
-				_starpu_may_use_nth_implementation(config.workers[workerid].arch, task->cl, nimpl));
+				_starpu_can_use_nth_implementation(config.workers[workerid].arch, task->cl, nimpl));
 		}
 		}
 		else
 		else
 		{
 		{

+ 3 - 3
src/core/workers.h

@@ -162,13 +162,13 @@ unsigned _starpu_machine_is_running(void);
 uint32_t _starpu_worker_exists(uint32_t task_mask);
 uint32_t _starpu_worker_exists(uint32_t task_mask);
 
 
 /* Is there a worker that can execute CUDA code ? */
 /* Is there a worker that can execute CUDA code ? */
-uint32_t _starpu_may_submit_cuda_task(void);
+uint32_t _starpu_can_submit_cuda_task(void);
 
 
 /* Is there a worker that can execute CPU code ? */
 /* Is there a worker that can execute CPU code ? */
-uint32_t _starpu_may_submit_cpu_task(void);
+uint32_t _starpu_can_submit_cpu_task(void);
 
 
 /* Is there a worker that can execute OpenCL code ? */
 /* Is there a worker that can execute OpenCL code ? */
-uint32_t _starpu_may_submit_opencl_task(void);
+uint32_t _starpu_can_submit_opencl_task(void);
 
 
 /* Check whether there is anything that the worker should do instead of
 /* Check whether there is anything that the worker should do instead of
  * sleeping (waiting on something to happen). */
  * sleeping (waiting on something to happen). */

+ 44 - 45
src/sched_policies/deque_modeling_policy_data_aware.c

@@ -297,6 +297,7 @@ static int push_task_on_best_worker(struct starpu_task *task, int best_workerid,
 			&sched_mutex[best_workerid], &sched_cond[best_workerid], task);
 			&sched_mutex[best_workerid], &sched_cond[best_workerid], task);
 }
 }
 
 
+/* TODO: factorize with dmda!! */
 static int _dm_push_task(struct starpu_task *task, unsigned prio)
 static int _dm_push_task(struct starpu_task *task, unsigned prio)
 {
 {
 	/* find the queue */
 	/* find the queue */
@@ -316,10 +317,9 @@ static int _dm_push_task(struct starpu_task *task, unsigned prio)
 
 
 	unsigned best_impl = 0;
 	unsigned best_impl = 0;
 	unsigned nimpl;
 	unsigned nimpl;
-	for (worker = 0; worker < nworkers; worker++)
-	{
-		for (nimpl = 0; nimpl < STARPU_MAXIMPLEMENTATIONS; nimpl++)
-		{
+
+	for (worker = 0; worker < nworkers; worker++) {
+		for (nimpl = 0; nimpl < STARPU_MAXIMPLEMENTATIONS; nimpl++) {
 			double exp_end;
 			double exp_end;
 
 
 			fifo = queue_array[worker];
 			fifo = queue_array[worker];
@@ -328,7 +328,7 @@ static int _dm_push_task(struct starpu_task *task, unsigned prio)
 			fifo->exp_start = STARPU_MAX(fifo->exp_start, starpu_timing_now());
 			fifo->exp_start = STARPU_MAX(fifo->exp_start, starpu_timing_now());
 			fifo->exp_end = fifo->exp_start + fifo->exp_len;
 			fifo->exp_end = fifo->exp_start + fifo->exp_len;
 
 
-			if (!starpu_worker_may_execute_task(worker, task, nimpl))
+			if (!starpu_worker_can_execute_task(worker, task, nimpl))
 			{
 			{
 				/* no one on that queue may execute this task */
 				/* no one on that queue may execute this task */
 				continue;
 				continue;
@@ -347,6 +347,7 @@ static int _dm_push_task(struct starpu_task *task, unsigned prio)
 					) {
 					) {
 				ntasks_best_end = ntasks_end;
 				ntasks_best_end = ntasks_end;
 				ntasks_best = worker;
 				ntasks_best = worker;
+				best_impl = nimpl;
 			}
 			}
 
 
 			if (local_length == -1.0)
 			if (local_length == -1.0)
@@ -400,13 +401,13 @@ static int _dmda_push_task(struct starpu_task *task, unsigned prio)
 	   there is no performance prediction available yet */
 	   there is no performance prediction available yet */
 	int forced_best = -1;
 	int forced_best = -1;
 
 
-	double local_task_length[nworkers];
-	double local_data_penalty[nworkers];
-	double local_power[nworkers];
-	double exp_end[nworkers];
+	double local_task_length[nworkers][STARPU_MAXIMPLEMENTATIONS];
+	double local_data_penalty[nworkers][STARPU_MAXIMPLEMENTATIONS];
+	double local_power[nworkers][STARPU_MAXIMPLEMENTATIONS];
+	double exp_end[nworkers][STARPU_MAXIMPLEMENTATIONS];
 	double max_exp_end = 0.0;
 	double max_exp_end = 0.0;
 
 
-	double fitness[nworkers];
+	double fitness[nworkers][STARPU_MAXIMPLEMENTATIONS];
 
 
 	double best_exp_end = 10e240;
 	double best_exp_end = 10e240;
 	double model_best = 0.0;
 	double model_best = 0.0;
@@ -420,11 +421,10 @@ static int _dmda_push_task(struct starpu_task *task, unsigned prio)
 	int unknown = 0;
 	int unknown = 0;
 
 
 	unsigned best_impl = 0;
 	unsigned best_impl = 0;
-	unsigned nimpl=0;
-	for (worker = 0; worker < nworkers; worker++)
-	{
-		for(nimpl  = 0; nimpl < STARPU_MAXIMPLEMENTATIONS; nimpl++)
-	 	{
+	unsigned nimpl;
+
+	for (worker = 0; worker < nworkers; worker++) {
+		for (nimpl = 0; nimpl < STARPU_MAXIMPLEMENTATIONS; nimpl++) {
 			fifo = queue_array[worker];
 			fifo = queue_array[worker];
 
 
 			/* Sometimes workers didn't take the tasks as early as we expected */
 			/* Sometimes workers didn't take the tasks as early as we expected */
@@ -433,39 +433,39 @@ static int _dmda_push_task(struct starpu_task *task, unsigned prio)
 			if (fifo->exp_end > max_exp_end)
 			if (fifo->exp_end > max_exp_end)
 				max_exp_end = fifo->exp_end;
 				max_exp_end = fifo->exp_end;
 
 
-			if (!starpu_worker_may_execute_task(worker, task, nimpl))
+			if (!starpu_worker_can_execute_task(worker, task, nimpl))
 			{
 			{
 				/* no one on that queue may execute this task */
 				/* no one on that queue may execute this task */
 				continue;
 				continue;
 			}
 			}
 
 
 			enum starpu_perf_archtype perf_arch = starpu_worker_get_perf_archtype(worker);
 			enum starpu_perf_archtype perf_arch = starpu_worker_get_perf_archtype(worker);
-			local_task_length[worker] = starpu_task_expected_length(task, perf_arch, nimpl);
+			local_task_length[worker][nimpl] = starpu_task_expected_length(task, perf_arch, nimpl);
 
 
-			//_STARPU_DEBUG("Scheduler dmda: task length (%lf) worker (%u) kernel (%u) \n", local_task_length[worker],worker,nimpl);
+			//_STARPU_DEBUG("Scheduler dmda: task length (%lf) worker (%u) kernel (%u) \n", local_task_length[worker][nimpl],worker,nimpl);
 
 
 			unsigned memory_node = starpu_worker_get_memory_node(worker);
 			unsigned memory_node = starpu_worker_get_memory_node(worker);
-			local_data_penalty[worker] = starpu_task_expected_data_transfer_time(memory_node, task);
+			local_data_penalty[worker][nimpl] = starpu_task_expected_data_transfer_time(memory_node, task);
 
 
 			double ntasks_end = fifo->ntasks / starpu_worker_get_relative_speedup(perf_arch);
 			double ntasks_end = fifo->ntasks / starpu_worker_get_relative_speedup(perf_arch);
 
 
 			if (ntasks_best == -1
 			if (ntasks_best == -1
 					|| (!calibrating && ntasks_end < ntasks_best_end) /* Not calibrating, take better task */
 					|| (!calibrating && ntasks_end < ntasks_best_end) /* Not calibrating, take better task */
-					|| (!calibrating && local_task_length[worker] == -1.0) /* Not calibrating but this worker is being calibrated */
-					|| (calibrating && local_task_length[worker] == -1.0 && ntasks_end < ntasks_best_end) /* Calibrating, compete this worker with other non-calibrated */
+					|| (!calibrating && local_task_length[worker][nimpl] == -1.0) /* Not calibrating but this worker is being calibrated */
+					|| (calibrating && local_task_length[worker][nimpl] == -1.0 && ntasks_end < ntasks_best_end) /* Calibrating, compete this worker with other non-calibrated */
 					) {
 					) {
 				ntasks_best_end = ntasks_end;
 				ntasks_best_end = ntasks_end;
 				ntasks_best = worker;
 				ntasks_best = worker;
-
+				best_impl = nimpl;
 			}
 			}
 
 
-			if (local_task_length[worker] == -1.0)
+			if (local_task_length[worker][nimpl] == -1.0)
 				/* we are calibrating, we want to speed-up calibration time
 				/* we are calibrating, we want to speed-up calibration time
 				 * so we privilege non-calibrated tasks (but still
 				 * so we privilege non-calibrated tasks (but still
 				 * greedily distribute them to avoid dumb schedules) */
 				 * greedily distribute them to avoid dumb schedules) */
 				calibrating = 1;
 				calibrating = 1;
 
 
-			if (local_task_length[worker] <= 0.0)
+			if (local_task_length[worker][nimpl] <= 0.0)
 				/* there is no prediction available for that task
 				/* there is no prediction available for that task
 				 * with that arch yet, so switch to a greedy strategy */
 				 * with that arch yet, so switch to a greedy strategy */
 				unknown = 1;
 				unknown = 1;
@@ -473,22 +473,18 @@ static int _dmda_push_task(struct starpu_task *task, unsigned prio)
 			if (unknown)
 			if (unknown)
 					continue;
 					continue;
 
 
-			exp_end[worker] = fifo->exp_start + fifo->exp_len + local_task_length[worker];
+			exp_end[worker][nimpl] = fifo->exp_start + fifo->exp_len + local_task_length[worker][nimpl];
 
 
-			if (exp_end[worker] < best_exp_end)
+			if (exp_end[worker][nimpl] < best_exp_end)
 			{
 			{
 				/* a better solution was found */
 				/* a better solution was found */
-				best_exp_end = exp_end[worker];
+				best_exp_end = exp_end[worker][nimpl];
 				best_impl = nimpl;
 				best_impl = nimpl;
-
 			}
 			}
 
 
-
-
-			local_power[worker] = starpu_task_expected_power(task, perf_arch, nimpl);
-			if (local_power[worker] == -1.0)
-				local_power[worker] = 0.;
-
+			local_power[worker][nimpl] = starpu_task_expected_power(task, perf_arch, nimpl);
+			if (local_power[worker][nimpl] == -1.0)
+				local_power[worker][nimpl] = 0.;
 
 
 		 }
 		 }
 	}
 	}
@@ -501,30 +497,33 @@ static int _dmda_push_task(struct starpu_task *task, unsigned prio)
 	if (forced_best == -1)
 	if (forced_best == -1)
 	{
 	{
 		for (worker = 0; worker < nworkers; worker++)
 		for (worker = 0; worker < nworkers; worker++)
+		for (nimpl = 0; nimpl < STARPU_MAXIMPLEMENTATIONS; nimpl++)
 		{
 		{
-			if (!starpu_worker_may_execute_task(worker, task, 0))
+			if (!starpu_worker_can_execute_task(worker, task, nimpl))
 			{
 			{
 				/* no one on that queue may execute this task */
 				/* no one on that queue may execute this task */
 				continue;
 				continue;
 			}
 			}
 	
 	
-			fitness[worker] = alpha*(exp_end[worker] - best_exp_end) 
-					+ beta*(local_data_penalty[worker])
-					+ _gamma*(local_power[worker]);
+			fitness[worker][nimpl] = alpha*(exp_end[worker][nimpl] - best_exp_end) 
+				+ beta*(local_data_penalty[worker][nimpl])
+				+ _gamma*(local_power[worker][nimpl]);
 
 
-			if (exp_end[worker] > max_exp_end)
+			if (exp_end[worker][nimpl] > max_exp_end) {
 				/* This placement will make the computation
 				/* This placement will make the computation
 				 * longer, take into account the idle
 				 * longer, take into account the idle
 				 * consumption of other cpus */
 				 * consumption of other cpus */
-				fitness[worker] += _gamma * idle_power * (exp_end[worker] - max_exp_end) / 1000000.0;
+				fitness[worker][nimpl] += _gamma * idle_power * (exp_end[worker][nimpl] - max_exp_end) / 1000000.0;
+			}
 
 
-			if (best == -1 || fitness[worker] < best_fitness)
+			if (best == -1 || fitness[worker][nimpl] < best_fitness)
 			{
 			{
 				/* we found a better solution */
 				/* we found a better solution */
-				best_fitness = fitness[worker];
+				best_fitness = fitness[worker][nimpl];
 				best = worker;
 				best = worker;
+				best_impl = nimpl;
 
 
-	//			_STARPU_DEBUG("best fitness (worker %d) %e = alpha*(%e) + beta(%e) +gamma(%e)\n", worker, best_fitness, exp_end[worker] - best_exp_end, local_data_penalty[worker], local_power[worker]);
+				//			_STARPU_DEBUG("best fitness (worker %d) %e = alpha*(%e) + beta(%e) +gamma(%e)\n", worker, best_fitness, exp_end[worker][nimpl] - best_exp_end, local_data_penalty[worker][nimpl], local_power[worker][nimpl]);
 			}
 			}
 		}
 		}
 	}
 	}
@@ -542,8 +541,8 @@ static int _dmda_push_task(struct starpu_task *task, unsigned prio)
 	}
 	}
 	else 
 	else 
 	{
 	{
-		model_best = local_task_length[best];
-		//penality_best = local_data_penalty[best];
+		model_best = local_task_length[best][nimpl];
+		//penality_best = local_data_penalty[best][nimpl];
 	}
 	}
 
 
 
 

+ 18 - 14
src/sched_policies/deque_queues.c

@@ -1,6 +1,6 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  *
- * Copyright (C) 2010  Université de Bordeaux 1
+ * Copyright (C) 2010-2011  Université de Bordeaux 1
  * Copyright (C) 2010  Centre National de la Recherche Scientifique
  * Copyright (C) 2010  Centre National de la Recherche Scientifique
  * Copyright (C) 2011  Télécom-SudParis
  * Copyright (C) 2011  Télécom-SudParis
  *
  *
@@ -68,17 +68,22 @@ struct starpu_task *_starpu_deque_pop_task(struct _starpu_deque_jobq *deque_queu
 	}
 	}
 
 
 	/* TODO find a task that suits workerid */
 	/* TODO find a task that suits workerid */
-	if (deque_queue->njobs > 0) 
+	for (j  = starpu_job_list_begin(deque_queue->jobq);
+	     j != starpu_job_list_end(deque_queue->jobq);
+	     j  = starpu_job_list_next(j))
 	{
 	{
-		/* there is a task */
-		j = starpu_job_list_pop_front(deque_queue->jobq);
-	
+		unsigned nimpl;
 		STARPU_ASSERT(j);
 		STARPU_ASSERT(j);
-		deque_queue->njobs--;
-		
-		_STARPU_TRACE_JOB_POP(j, 0);
 
 
-		return j->task;
+		for (nimpl = 0; nimpl < STARPU_MAXIMPLEMENTATIONS; nimpl++)
+			if (starpu_worker_can_execute_task(workerid, j->task, nimpl))
+			{
+				j->nimpl = nimpl;
+				j = starpu_job_list_pop_front(deque_queue->jobq);
+				deque_queue->njobs--;
+				_STARPU_TRACE_JOB_POP(j, 0);
+				return j->task;
+			}
 	}
 	}
 
 
 	return NULL;
 	return NULL;
@@ -110,19 +115,18 @@ struct starpu_job_list_s *_starpu_deque_pop_every_task(struct _starpu_deque_jobq
 			i != starpu_job_list_end(old_list);
 			i != starpu_job_list_end(old_list);
 			i  = next_job)
 			i  = next_job)
 		{
 		{
+			unsigned nimpl;
 			next_job = starpu_job_list_next(i);
 			next_job = starpu_job_list_next(i);
 
 
-			/* In case there are multiples implementations of the
- 			 * codelet for a single device, We dont really care
-			 * about the implementation used, so let's try the 
-			 * first one. */
-			if (starpu_worker_may_execute_task(workerid, i->task, 0))
+			for (nimpl = 0; nimpl < STARPU_MAXIMPLEMENTATIONS; nimpl++)
+			if (starpu_worker_can_execute_task(workerid, i->task, nimpl))
 			{
 			{
 				/* this elements can be moved into the new list */
 				/* this elements can be moved into the new list */
 				new_list_size++;
 				new_list_size++;
 				
 				
 				starpu_job_list_erase(old_list, i);
 				starpu_job_list_erase(old_list, i);
 				starpu_job_list_push_back(new_list, i);
 				starpu_job_list_push_back(new_list, i);
+				i->nimpl = nimpl;
 			}
 			}
 		}
 		}
 
 

+ 1 - 1
src/sched_policies/eager_central_policy.c

@@ -46,7 +46,7 @@ static void initialize_eager_center_policy(struct starpu_machine_topology *topol
 static void deinitialize_eager_center_policy(__attribute__ ((unused)) struct starpu_machine_topology *topology, 
 static void deinitialize_eager_center_policy(__attribute__ ((unused)) struct starpu_machine_topology *topology, 
 		   __attribute__ ((unused)) struct starpu_sched_policy *_policy) 
 		   __attribute__ ((unused)) struct starpu_sched_policy *_policy) 
 {
 {
-	/* TODO check that there is no task left in the queue */
+	STARPU_ASSERT(!_starpu_fifo_pop_task(fifo, starpu_worker_get_id()));
 
 
 	/* deallocate the job queue */
 	/* deallocate the job queue */
 	_starpu_destroy_fifo(fifo);
 	_starpu_destroy_fifo(fifo);

+ 2 - 0
src/sched_policies/eager_central_priority_policy.c

@@ -122,6 +122,7 @@ static int _starpu_priority_push_task(struct starpu_task *task)
 
 
 static struct starpu_task *_starpu_priority_pop_task(void)
 static struct starpu_task *_starpu_priority_pop_task(void)
 {
 {
+	/* XXX FIXME: should call starpu_worker_can_execute_task!! */
 	struct starpu_task *task = NULL;
 	struct starpu_task *task = NULL;
 
 
 	/* block until some event happens */
 	/* block until some event happens */
@@ -150,6 +151,7 @@ static struct starpu_task *_starpu_priority_pop_task(void)
 			}
 			}
 		} while (!task && priolevel-- > 0);
 		} while (!task && priolevel-- > 0);
 	}
 	}
+	STARPU_ASSERT(starpu_worker_can_execute_task(starpu_worker_get_id(), task, 0) || !"prio does not support \"can_execute\"");
 
 
 	_STARPU_PTHREAD_MUTEX_UNLOCK(&global_sched_mutex);
 	_STARPU_PTHREAD_MUTEX_UNLOCK(&global_sched_mutex);
 
 

+ 22 - 16
src/sched_policies/fifo_queues.c

@@ -48,7 +48,6 @@ void _starpu_destroy_fifo(struct _starpu_fifo_taskq *fifo)
 }
 }
 
 
 /* TODO: revert front/back? */
 /* TODO: revert front/back? */
-
 int _starpu_fifo_push_task(struct _starpu_fifo_taskq *fifo_queue, pthread_mutex_t *sched_mutex, pthread_cond_t *sched_cond, struct starpu_task *task)
 int _starpu_fifo_push_task(struct _starpu_fifo_taskq *fifo_queue, pthread_mutex_t *sched_mutex, pthread_cond_t *sched_cond, struct starpu_task *task)
 {
 {
 	_STARPU_PTHREAD_MUTEX_LOCK(sched_mutex);
 	_STARPU_PTHREAD_MUTEX_LOCK(sched_mutex);
@@ -65,26 +64,29 @@ int _starpu_fifo_push_task(struct _starpu_fifo_taskq *fifo_queue, pthread_mutex_
 	return 0;
 	return 0;
 }
 }
 
 
-struct starpu_task *_starpu_fifo_pop_task(struct _starpu_fifo_taskq *fifo_queue, int workerid __attribute__ ((unused)))
+struct starpu_task *_starpu_fifo_pop_task(struct _starpu_fifo_taskq *fifo_queue, int workerid)
 {
 {
-	struct starpu_task *task = NULL;
-
-	if (fifo_queue->ntasks == 0)
-		return NULL;
+	struct starpu_task *task;
 
 
-	/* TODO: find a task that suits workerid */
-	if (fifo_queue->ntasks > 0) 
+	for (task  = starpu_task_list_begin(&fifo_queue->taskq);
+	     task != starpu_task_list_end(&fifo_queue->taskq);
+	     task  = starpu_task_list_next(task))
 	{
 	{
-		/* there is a task */
-		task = starpu_task_list_pop_back(&fifo_queue->taskq);
-	
+		unsigned nimpl;
 		STARPU_ASSERT(task);
 		STARPU_ASSERT(task);
-		fifo_queue->ntasks--;
-		
-		_STARPU_TRACE_JOB_POP(task, 0);
+
+		for (nimpl = 0; nimpl < STARPU_MAXIMPLEMENTATIONS; nimpl++)
+			if (starpu_worker_can_execute_task(workerid, task, nimpl))
+			{
+				_starpu_get_job_associated_to_task(task)->nimpl = nimpl;
+				starpu_task_list_erase(&fifo_queue->taskq, task);
+				fifo_queue->ntasks--;
+				_STARPU_TRACE_JOB_POP(task, 0);
+				return task;
+			}
 	}
 	}
 	
 	
-	return task;
+	return NULL;
 }
 }
 
 
 /* pop every task that can be executed on the calling driver */
 /* pop every task that can be executed on the calling driver */
@@ -110,9 +112,11 @@ struct starpu_task *_starpu_fifo_pop_every_task(struct _starpu_fifo_taskq *fifo_
 		task = starpu_task_list_front(old_list);
 		task = starpu_task_list_front(old_list);
 		while (task)
 		while (task)
 		{
 		{
+			unsigned nimpl;
 			next_task = task->next;
 			next_task = task->next;
 
 
-			if (starpu_worker_may_execute_task(workerid, task, 0))
+			for (nimpl = 0; nimpl < STARPU_MAXIMPLEMENTATIONS; nimpl++)
+			if (starpu_worker_can_execute_task(workerid, task, nimpl))
 			{
 			{
 				/* this elements can be moved into the new list */
 				/* this elements can be moved into the new list */
 				new_list_size++;
 				new_list_size++;
@@ -132,6 +136,8 @@ struct starpu_task *_starpu_fifo_pop_every_task(struct _starpu_fifo_taskq *fifo_
 					task->prev = NULL;
 					task->prev = NULL;
 					task->next = NULL;
 					task->next = NULL;
 				}
 				}
+				_starpu_get_job_associated_to_task(task)->nimpl = nimpl;
+				break;
 			}
 			}
 		
 		
 			task = next_task;
 			task = next_task;

+ 7 - 3
src/sched_policies/heft.c

@@ -208,6 +208,7 @@ static int push_task_on_best_worker(struct starpu_task *task, int best_workerid,
 	return starpu_push_local_task(best_workerid, task, prio);
 	return starpu_push_local_task(best_workerid, task, prio);
 }
 }
 
 
+/* TODO: factorize with dmda!! */
 static void compute_all_performance_predictions(struct starpu_task *task,
 static void compute_all_performance_predictions(struct starpu_task *task,
 					double local_task_length[STARPU_NMAXWORKERS][STARPU_MAXIMPLEMENTATIONS],
 					double local_task_length[STARPU_NMAXWORKERS][STARPU_MAXIMPLEMENTATIONS],
 					double exp_end[STARPU_NMAXWORKERS][STARPU_MAXIMPLEMENTATIONS],
 					double exp_end[STARPU_NMAXWORKERS][STARPU_MAXIMPLEMENTATIONS],
@@ -232,14 +233,14 @@ static void compute_all_performance_predictions(struct starpu_task *task,
 	unsigned nimpl;
 	unsigned nimpl;
 
 
 	for (worker = 0; worker < nworkers; worker++) {
 	for (worker = 0; worker < nworkers; worker++) {
-		for (nimpl = 0; nimpl <STARPU_MAXIMPLEMENTATIONS; nimpl++) {
+		for (nimpl = 0; nimpl < STARPU_MAXIMPLEMENTATIONS; nimpl++) {
 			/* Sometimes workers didn't take the tasks as early as we expected */
 			/* Sometimes workers didn't take the tasks as early as we expected */
 			exp_start[worker] = STARPU_MAX(exp_start[worker], starpu_timing_now());
 			exp_start[worker] = STARPU_MAX(exp_start[worker], starpu_timing_now());
 			exp_end[worker][nimpl] = exp_start[worker] + exp_len[worker];
 			exp_end[worker][nimpl] = exp_start[worker] + exp_len[worker];
 			if (exp_end[worker][nimpl] > max_exp_end)
 			if (exp_end[worker][nimpl] > max_exp_end)
 				max_exp_end = exp_end[worker][nimpl];
 				max_exp_end = exp_end[worker][nimpl];
 
 
-			if (!starpu_worker_may_execute_task(worker, task, nimpl))
+			if (!starpu_worker_can_execute_task(worker, task, nimpl))
 			{
 			{
 				/* no one on that queue may execute this task */
 				/* no one on that queue may execute this task */
 				continue;
 				continue;
@@ -298,8 +299,11 @@ static void compute_all_performance_predictions(struct starpu_task *task,
 			exp_end[worker][nimpl] = exp_start[worker] + exp_len[worker] + local_task_length[worker][nimpl];
 			exp_end[worker][nimpl] = exp_start[worker] + exp_len[worker] + local_task_length[worker][nimpl];
 
 
 			if (exp_end[worker][nimpl] < best_exp_end)
 			if (exp_end[worker][nimpl] < best_exp_end)
+			{
 				/* a better solution was found */
 				/* a better solution was found */
 				best_exp_end = exp_end[worker][nimpl];
 				best_exp_end = exp_end[worker][nimpl];
+				nimpl_best = nimpl;
+			}
 
 
 			if (local_power[worker][nimpl] == -1.0)
 			if (local_power[worker][nimpl] == -1.0)
 				local_power[worker][nimpl] = 0.;
 				local_power[worker][nimpl] = 0.;
@@ -365,7 +369,7 @@ static int _heft_push_task(struct starpu_task *task, unsigned prio)
 	for (worker = 0; worker < nworkers; worker++)
 	for (worker = 0; worker < nworkers; worker++)
 	{
 	{
 		for (nimpl = 0; nimpl < STARPU_MAXIMPLEMENTATIONS; nimpl++) {
 		for (nimpl = 0; nimpl < STARPU_MAXIMPLEMENTATIONS; nimpl++) {
-			if (!starpu_worker_may_execute_task(worker, task, nimpl))
+			if (!starpu_worker_can_execute_task(worker, task, nimpl))
 			{
 			{
 				/* no one on that queue may execute this task */
 				/* no one on that queue may execute this task */
 				continue;
 				continue;

+ 1 - 1
src/sched_policies/parallel_greedy.c

@@ -169,7 +169,7 @@ static struct starpu_task *pop_task_pgreedy_policy(void)
 			if (possible_combinations_size[workerid][i] > best_size)
 			if (possible_combinations_size[workerid][i] > best_size)
 			{
 			{
 				int combined_worker = possible_combinations[workerid][i];
 				int combined_worker = possible_combinations[workerid][i];
-				if (starpu_combined_worker_may_execute_task(combined_worker, task, 0))
+				if (starpu_combined_worker_can_execute_task(combined_worker, task, 0))
 				{
 				{
 					best_size = possible_combinations_size[workerid][i];
 					best_size = possible_combinations_size[workerid][i];
 					best_workerid = combined_worker;
 					best_workerid = combined_worker;

+ 1 - 1
src/sched_policies/parallel_heft.c

@@ -238,7 +238,7 @@ static int _parallel_heft_push_task(struct starpu_task *task, unsigned prio)
 	{
 	{
 		for (nimpl = 0; nimpl < STARPU_MAXIMPLEMENTATIONS; nimpl++)
 		for (nimpl = 0; nimpl < STARPU_MAXIMPLEMENTATIONS; nimpl++)
 		{
 		{
-			if (!starpu_combined_worker_may_execute_task(worker, task, nimpl))
+			if (!starpu_combined_worker_can_execute_task(worker, task, nimpl))
 			{
 			{
 				/* no one on that queue may execute this task */
 				/* no one on that queue may execute this task */
 				skip_worker[worker][nimpl] = 1;
 				skip_worker[worker][nimpl] = 1;

+ 4 - 4
src/util/malloc.c

@@ -74,7 +74,7 @@ int starpu_malloc(void **A, size_t dim)
 
 
 	STARPU_ASSERT(A);
 	STARPU_ASSERT(A);
 
 
-	if (_starpu_may_submit_cuda_task())
+	if (_starpu_can_submit_cuda_task())
 	{
 	{
 #ifdef STARPU_USE_CUDA
 #ifdef STARPU_USE_CUDA
 		int push_res;
 		int push_res;
@@ -98,7 +98,7 @@ int starpu_malloc(void **A, size_t dim)
 		STARPU_ASSERT(push_res != -ENODEV);
 		STARPU_ASSERT(push_res != -ENODEV);
 #endif
 #endif
 	}
 	}
-//	else if (_starpu_may_submit_opencl_task())
+//	else if (_starpu_can_submit_opencl_task())
 //	{
 //	{
 //#ifdef STARPU_USE_OPENCL
 //#ifdef STARPU_USE_OPENCL
 //		int push_res;
 //		int push_res;
@@ -171,7 +171,7 @@ int starpu_free(void *A)
 	if (STARPU_UNLIKELY(!_starpu_worker_may_perform_blocking_calls()))
 	if (STARPU_UNLIKELY(!_starpu_worker_may_perform_blocking_calls()))
 		return -EDEADLK;
 		return -EDEADLK;
 
 
-	if (_starpu_may_submit_cuda_task())
+	if (_starpu_can_submit_cuda_task())
 	{
 	{
 #ifdef STARPU_USE_CUDA
 #ifdef STARPU_USE_CUDA
 		int push_res;
 		int push_res;
@@ -190,7 +190,7 @@ int starpu_free(void *A)
 		STARPU_ASSERT(push_res != -ENODEV);
 		STARPU_ASSERT(push_res != -ENODEV);
 #endif
 #endif
 	}
 	}
-//	else if (_starpu_may_submit_opencl_task())
+//	else if (_starpu_can_submit_opencl_task())
 //	{
 //	{
 //#ifdef STARPU_USE_OPENCL
 //#ifdef STARPU_USE_OPENCL
 //		int push_res;
 //		int push_res;