Corentin Salingue 12 vuotta sitten
vanhempi
commit
5a260508b9

+ 2 - 0
ChangeLog

@@ -30,6 +30,8 @@ New features:
 	  before the corresponding data, which allows the receiver to
 	  allocate data correctly, and to submit the matching receive of
 	  the envelope.
+  * New STARPU_COMMUTE flag which can be passed along STARPU_W or STARPU_RW to
+    let starpu commute write accesses.
 
 Small features:
   * Add cl_arg_free field to enable automatic free(cl_arg) on task

+ 2 - 1
doc/chapters/advanced-examples.texi

@@ -896,7 +896,8 @@ To benefit from parallel tasks, a parallel-task-aware StarPU scheduler has to
 be used. When exposed to codelets with a Fork or SPMD flag, the @code{pheft}
 (parallel-heft) and @code{peager} (parallel eager) schedulers will indeed also
 try to execute tasks with several CPUs. It will automatically try the various
-available combined worker sizes and thus be able to avoid choosing a large
+available combined worker sizes (making several measurements for each worker size)
+and thus be able to avoid choosing a large
 combined worker if the codelet does not actually scale so much.
 
 @subsection Combined workers

+ 8 - 2
doc/chapters/api.texi

@@ -568,6 +568,12 @@ ignored for now.
 @end table
 @end deftp
 
+In addition to that, @code{STARPU_COMMUTE} can be passed along @code{STARPU_W}
+or @code{STARPU_RW} to express that StarPU can let tasks commute, which is
+useful e.g. when bringing a contribution into some data, which can be done
+in any order (but still require sequential consistency against reads or
+non-commutative writes).
+
 @deftp {Data Type} {starpu_data_handle_t}
 StarPU uses @code{starpu_data_handle_t} as an opaque handle to manage a piece of
 data. Once a piece of data has been registered to StarPU, it is associated to a
@@ -4095,7 +4101,7 @@ The workerids managed by the collection
 The number of workerids
 @item @code{pthread_key_t cursor_key} (optional)
 The cursor needed to iterate the collection (depending on the data structure)
-@item @code{int type}
+@item @code{enum starpu_worker_collection_type type}
 The type of structure (currently STARPU_WORKER_LIST is the only one available)
 @item @code{unsigned (*has_next)(struct starpu_worker_collection *workers)}
 Checks if there is a next worker
@@ -4117,7 +4123,7 @@ Deinitialize the cursor if there is one
 @end table
 @end deftp
 
-@deftypefun struct starpu_worker_collection* starpu_sched_ctx_create_worker_collection (unsigned @var{sched_ctx_id}, int @var{type})
+@deftypefun struct starpu_worker_collection* starpu_sched_ctx_create_worker_collection (unsigned @var{sched_ctx_id}, enum starpu_worker_collection_type @var{type})
 Create a worker collection of the type indicated by the last parameter for the context specified through the first parameter.
 @end deftypefun
 

+ 3 - 1
doc/chapters/perf-optimization.texi

@@ -79,7 +79,9 @@ dependencies on that data.
 
 In the same vein, accumulation of results in the same data can become a
 bottleneck. The use of the @code{STARPU_REDUX} mode permits to optimize such
-accumulation (@pxref{Data reduction}).
+accumulation (@pxref{Data reduction}). To a lesser extent, the use of the
+@code{STARPU_COMMUTE} flag keeps the bottleneck, but at least permits the
+accumulation to happen in any order.
 
 Applications often need a data just for temporary results.  In such a case,
 registration can be made without an initial value, for instance this produces a vector data:

+ 3 - 1
include/starpu_data.h

@@ -35,7 +35,9 @@ enum starpu_data_access_mode
 	STARPU_W=(1<<1),
 	STARPU_RW=(STARPU_R|STARPU_W),
 	STARPU_SCRATCH=(1<<2),
-	STARPU_REDUX=(1<<3)
+	STARPU_REDUX=(1<<3),
+	STARPU_COMMUTE=(1<<4)
+	/* Note: other STARPU_* values in include/starpu_task_util.h */
 };
 
 struct starpu_data_descr

+ 0 - 15
include/starpu_data_interfaces.h

@@ -461,21 +461,6 @@ void starpu_multiformat_data_register(starpu_data_handle_t *handle, unsigned hom
 #define STARPU_MULTIFORMAT_GET_MIC_PTR(interface) (((struct starpu_multiformat_interface *)(interface))->mic_ptr)
 #define STARPU_MULTIFORMAT_GET_NX(interface)  (((struct starpu_multiformat_interface *)(interface))->nx)
 
-/* Generic type representing an interface, for now it's only used before
- * execution on message-passing devices but it can be useful in other cases.
- */
-union _starpu_interface
-{
-	struct starpu_matrix_interface matrix;
-	struct starpu_block_interface block;
-	struct starpu_vector_interface vector;
-	struct starpu_csr_interface csr;
-	struct starpu_coo_interface coo;
-	struct starpu_bcsr_interface bcsr;
-	struct starpu_variable_interface variable;
-	struct starpu_multiformat_interface multiformat;
-};
-
 enum starpu_data_interface_id starpu_data_get_interface_id(starpu_data_handle_t handle);
 
 int starpu_data_pack(starpu_data_handle_t handle, void **ptr, starpu_ssize_t *count);

+ 1 - 1
include/starpu_sched_ctx.h

@@ -114,7 +114,7 @@ void* starpu_sched_ctx_get_policy_data(unsigned sched_ctx_id);
  * WORKERS IN CONTEXT 
 */
 /* create a worker collection for a context, the type can be only STARPU_WORKER_LIST for now, which corresponds to a simple list */
-struct starpu_worker_collection* starpu_sched_ctx_create_worker_collection(unsigned sched_ctx_id, int type);
+struct starpu_worker_collection* starpu_sched_ctx_create_worker_collection(unsigned sched_ctx_id, enum starpu_worker_collection_type type);
 
 /* free the worker collection when removing the context */
 void starpu_sched_ctx_delete_worker_collection(unsigned sched_ctx_id);

+ 12 - 12
include/starpu_task_util.h

@@ -35,18 +35,18 @@ void starpu_create_sync_task(starpu_tag_t sync_tag, unsigned ndeps, starpu_tag_t
 				void (*callback)(void *), void *callback_arg);
 
 /* Constants used by the starpu_insert_task helper to determine the different types of argument */
-#define STARPU_VALUE		(1<<4)	/* Pointer to a constant value */
-#define STARPU_CALLBACK		(1<<5)	/* Callback function */
-#define STARPU_CALLBACK_WITH_ARG	(1<<6)	/* Callback function */
-#define STARPU_CALLBACK_ARG	(1<<7)	/* Argument of the callback function (of type void *) */
-#define STARPU_PRIORITY		(1<<8)	/* Priority associated to the task */
-#define STARPU_EXECUTE_ON_NODE	(1<<9)	/* Used by MPI to define which task is going to execute the codelet */
-#define STARPU_EXECUTE_ON_DATA	(1<<10)	/* Used by MPI to define which task is going to execute the codelet */
-#define STARPU_DATA_ARRAY       (1<<11) /* Array of data handles */
-#define STARPU_TAG              (1<<12) /* Tag */
-#define STARPU_HYPERVISOR_TAG	(1<<13)	/* Used to tag a task after whose execution we'll execute  a code */
-#define STARPU_FLOPS	        (1<<14)	/* Used to specify the number of flops needed to be executed by a task */
-#define STARPU_SCHED_CTX	(1<<15)	/* Used to specify the sched_ctx to which the task will be submitted */
+#define STARPU_VALUE		(1<<19)	/* Pointer to a constant value */
+#define STARPU_CALLBACK		(1<<20)	/* Callback function */
+#define STARPU_CALLBACK_WITH_ARG	(1<<21)	/* Callback function */
+#define STARPU_CALLBACK_ARG	(1<<22)	/* Argument of the callback function (of type void *) */
+#define STARPU_PRIORITY		(1<<23)	/* Priority associated to the task */
+#define STARPU_EXECUTE_ON_NODE	(1<<24)	/* Used by MPI to define which task is going to execute the codelet */
+#define STARPU_EXECUTE_ON_DATA	(1<<25)	/* Used by MPI to define which task is going to execute the codelet */
+#define STARPU_DATA_ARRAY       (1<<26) /* Array of data handles */
+#define STARPU_TAG              (1<<27) /* Tag */
+#define STARPU_HYPERVISOR_TAG	(1<<28)	/* Used to tag a task after whose execution we'll execute  a code */
+#define STARPU_FLOPS	        (1<<29)	/* Used to specify the number of flops needed to be executed by a task */
+#define STARPU_SCHED_CTX	(1<<30)	/* Used to specify the sched_ctx to which the task will be submitted */
 
 /* Wrapper to create a task. */
 int starpu_insert_task(struct starpu_codelet *cl, ...);

+ 8 - 6
include/starpu_worker.h

@@ -102,6 +102,12 @@ struct starpu_machine_topology
 	unsigned workers_scc_deviceid[STARPU_NMAXWORKERS];
 };
 
+/* types of structures the worker collection can implement */
+enum starpu_worker_collection_type
+{
+	STARPU_WORKER_LIST
+};
+
 /* generic structure used by the scheduling contexts to iterate the workers */
 struct starpu_worker_collection
 {
@@ -109,8 +115,8 @@ struct starpu_worker_collection
 	void *workerids;
 	/* the number of workers in the collection */
 	unsigned nworkers;
-	/* the type of structure (STARPU_WORKER_LIST,...) */
-	int type;
+	/* the type of structure */
+	enum starpu_worker_collection_type type;
 	/* checks if there is another element in collection */
 	unsigned (*has_next)(struct starpu_worker_collection *workers, struct starpu_sched_ctx_iterator *it);
 	/* return the next element in the collection */
@@ -127,10 +133,6 @@ struct starpu_worker_collection
 	void (*init_iterator)(struct starpu_worker_collection *workers, struct starpu_sched_ctx_iterator *it);
 };
 
-
-/* types of structures the worker collection can implement */
-#define STARPU_WORKER_LIST 0
-
 /* This function returns the number of workers (ie. processing units executing
  * StarPU tasks). The returned value should be at most STARPU_NMAXWORKERS. */
 unsigned starpu_worker_get_count(void);

+ 3 - 2
src/core/dependencies/data_concurrency.c

@@ -1,6 +1,6 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
- * Copyright (C) 2010-2012  Université de Bordeaux 1
+ * Copyright (C) 2010-2013  Université de Bordeaux 1
  * Copyright (C) 2010, 2011, 2012, 2013  Centre National de la Recherche Scientifique
  *
  * StarPU is free software; you can redistribute it and/or modify
@@ -55,6 +55,7 @@ static struct _starpu_data_requester *may_unlock_data_req_list_head(starpu_data_
 	if (handle->refcnt == 0)
 		return _starpu_data_requester_list_pop_front(req_list);
 
+	/* Already writing to it, do not let another write access through */
 	if (handle->current_mode == STARPU_W)
 		return NULL;
 
@@ -193,7 +194,7 @@ static unsigned attempt_to_submit_data_request_from_job(struct _starpu_job *j, u
 	/* Note that we do not access j->task->handles, but j->ordered_buffers
 	 * which is a sorted copy of it. */
 	starpu_data_handle_t handle = _STARPU_JOB_GET_ORDERED_BUFFER_HANDLE(j, buffer_index);
-	enum starpu_data_access_mode mode = _STARPU_JOB_GET_ORDERED_BUFFER_MODE(j, buffer_index);
+	enum starpu_data_access_mode mode = _STARPU_JOB_GET_ORDERED_BUFFER_MODE(j, buffer_index) & ~STARPU_COMMUTE;
 
 	return _starpu_attempt_to_submit_data_request(1, handle, mode, NULL, NULL, j, buffer_index);
 }

+ 100 - 130
src/core/dependencies/implicit_data_deps.c

@@ -46,23 +46,22 @@ static void _starpu_add_dependency(starpu_data_handle_t handle STARPU_ATTRIBUTE_
 	_starpu_add_ghost_dependency(handle, _starpu_get_job_associated_to_task(previous)->job_id, next);
 }
 
-/* Read after Write (RAW) or Read after Read (RAR) */
-static void _starpu_add_reader_after_writer(starpu_data_handle_t handle, struct starpu_task *pre_sync_task, struct starpu_task *post_sync_task)
+/* Add pre_sync_task as new accessor among the existing ones, making it depend on the last synchronization task if any.  */
+static void _starpu_add_accessor(starpu_data_handle_t handle, struct starpu_task *pre_sync_task, struct starpu_task *post_sync_task)
 {
 	/* Add this task to the list of readers */
 	struct _starpu_task_wrapper_list *link = (struct _starpu_task_wrapper_list *) malloc(sizeof(struct _starpu_task_wrapper_list));
 	link->task = post_sync_task;
-	link->next = handle->last_submitted_readers;
-	handle->last_submitted_readers = link;
+	link->next = handle->last_submitted_accessors;
+	handle->last_submitted_accessors = link;
 
-	/* This task depends on the previous writer if any */
-	if (handle->last_submitted_writer && handle->last_submitted_writer != post_sync_task)
+	/* This task depends on the previous synchronization task if any */
+	if (handle->last_sync_task && handle->last_sync_task != post_sync_task)
 	{
-		_STARPU_DEP_DEBUG("RAW %p\n", handle);
-		struct starpu_task *task_array[1] = {handle->last_submitted_writer};
+		struct starpu_task *task_array[1] = {handle->last_sync_task};
 		_starpu_task_declare_deps_array(pre_sync_task, 1, task_array, 0);
-		_starpu_add_dependency(handle, handle->last_submitted_writer, pre_sync_task);
-		_STARPU_DEP_DEBUG("dep %p -> %p\n", handle->last_submitted_writer, pre_sync_task);
+		_starpu_add_dependency(handle, handle->last_sync_task, pre_sync_task);
+		_STARPU_DEP_DEBUG("dep %p -> %p\n", handle->last_sync_task, pre_sync_task);
 	}
         else
         {
@@ -82,12 +81,12 @@ static void _starpu_add_reader_after_writer(starpu_data_handle_t handle, struct
 #ifdef HAVE_AYUDAME_H
 		|| AYU_event
 #endif
-		) && handle->last_submitted_ghost_writer_id_is_valid)
+		) && handle->last_submitted_ghost_sync_id_is_valid)
 	{
-		_STARPU_TRACE_GHOST_TASK_DEPS(handle->last_submitted_ghost_writer_id,
+		_STARPU_TRACE_GHOST_TASK_DEPS(handle->last_submitted_ghost_sync_id,
 			_starpu_get_job_associated_to_task(pre_sync_task)->job_id);
-		_starpu_add_ghost_dependency(handle, handle->last_submitted_ghost_writer_id, pre_sync_task);
-		_STARPU_DEP_DEBUG("dep ID%lu -> %p\n", handle->last_submitted_ghost_writer_id, pre_sync_task);
+		_starpu_add_ghost_dependency(handle, handle->last_submitted_ghost_sync_id, pre_sync_task);
+		_STARPU_DEP_DEBUG("dep ID%lu -> %p\n", handle->last_submitted_ghost_sync_id, pre_sync_task);
 	}
 
 	if (!pre_sync_task->cl) {
@@ -99,27 +98,27 @@ static void _starpu_add_reader_after_writer(starpu_data_handle_t handle, struct
 	}
 }
 
-/* Write after Read (WAR) */
-static void _starpu_add_writer_after_readers(starpu_data_handle_t handle, struct starpu_task *pre_sync_task, struct starpu_task *post_sync_task)
+/* This adds a new synchronization task which depends on all the previous accessors */
+static void _starpu_add_sync_task(starpu_data_handle_t handle, struct starpu_task *pre_sync_task, struct starpu_task *post_sync_task)
 {
-	/* Count the readers */
-	unsigned nreaders = 0;
+	/* Count the existing accessors */
+	unsigned naccessors = 0;
 	struct _starpu_task_wrapper_list *l;
-	l = handle->last_submitted_readers;
+	l = handle->last_submitted_accessors;
 	while (l)
 	{
 		if (l->task != post_sync_task)
-			nreaders++;
+			naccessors++;
 		l = l->next;
 	}
-	_STARPU_DEP_DEBUG("%d readers\n", nreaders);
+	_STARPU_DEP_DEBUG("%d accessors\n", naccessors);
 
-	if (nreaders > 0)
+	if (naccessors > 0)
 	{
 		/* Put all tasks in the list into task_array */
-		struct starpu_task *task_array[nreaders];
+		struct starpu_task *task_array[naccessors];
 		unsigned i = 0;
-		l = handle->last_submitted_readers;
+		l = handle->last_submitted_accessors;
 		while (l)
 		{
 			STARPU_ASSERT(l->task);
@@ -134,80 +133,31 @@ static void _starpu_add_writer_after_readers(starpu_data_handle_t handle, struct
 			l = l->next;
 			free(prev);
 		}
-		_starpu_task_declare_deps_array(pre_sync_task, nreaders, task_array, 0);
+		_starpu_task_declare_deps_array(pre_sync_task, naccessors, task_array, 0);
 	}
 #ifndef STARPU_USE_FXT
 	if (_starpu_bound_recording)
 #endif
 	{
-		/* Declare all dependencies with ghost readers */
-		struct _starpu_jobid_list *ghost_readers_id = handle->last_submitted_ghost_readers_id;
-		while (ghost_readers_id)
+		/* Declare all dependencies with ghost accessors */
+		struct _starpu_jobid_list *ghost_accessors_id = handle->last_submitted_ghost_accessors_id;
+		while (ghost_accessors_id)
 		{
-			unsigned long id = ghost_readers_id->id;
+			unsigned long id = ghost_accessors_id->id;
 			_STARPU_TRACE_GHOST_TASK_DEPS(id,
 				_starpu_get_job_associated_to_task(pre_sync_task)->job_id);
 			_starpu_add_ghost_dependency(handle, id, pre_sync_task);
 			_STARPU_DEP_DEBUG("dep ID%lu -> %p\n", id, pre_sync_task);
 
-			struct _starpu_jobid_list *prev = ghost_readers_id;
-			ghost_readers_id = ghost_readers_id->next;
+			struct _starpu_jobid_list *prev = ghost_accessors_id;
+			ghost_accessors_id = ghost_accessors_id->next;
 			free(prev);
 		}
-		handle->last_submitted_ghost_readers_id = NULL;
+		handle->last_submitted_ghost_accessors_id = NULL;
 	}
 
-	handle->last_submitted_readers = NULL;
-	handle->last_submitted_writer = post_sync_task;
-
-	if (!post_sync_task->cl) {
-		/* Add a reference to be released in _starpu_handle_job_termination */
-		_starpu_spin_lock(&handle->header_lock);
-		handle->busy_count++;
-		_starpu_spin_unlock(&handle->header_lock);
-		_starpu_get_job_associated_to_task(post_sync_task)->implicit_dep_handle = handle;
-	}
-}
-
-/* Write after Write (WAW) */
-static void _starpu_add_writer_after_writer(starpu_data_handle_t handle, struct starpu_task *pre_sync_task, struct starpu_task *post_sync_task)
-{
-	/* (Read) Write */
-	/* This task depends on the previous writer */
-	if (handle->last_submitted_writer && handle->last_submitted_writer != post_sync_task)
-	{
-		struct starpu_task *task_array[1] = {handle->last_submitted_writer};
-		_starpu_task_declare_deps_array(pre_sync_task, 1, task_array, 0);
-		_starpu_add_dependency(handle, handle->last_submitted_writer, pre_sync_task);
-		_STARPU_DEP_DEBUG("dep %p -> %p\n", handle->last_submitted_writer, pre_sync_task);
-	}
-        else
-        {
-		_STARPU_DEP_DEBUG("No dep\n");
-        }
-
-	/* If there is a ghost writer instead, we
-	 * should declare a ghost dependency here, and
-	 * invalidate the ghost value. */
-#ifndef STARPU_USE_FXT
-	if (_starpu_bound_recording)
-#endif
-	{
-		if (handle->last_submitted_ghost_writer_id_is_valid)
-		{
-			_STARPU_TRACE_GHOST_TASK_DEPS(handle->last_submitted_ghost_writer_id, 
-				_starpu_get_job_associated_to_task(pre_sync_task)->job_id);
-			_starpu_add_ghost_dependency(handle, handle->last_submitted_ghost_writer_id, pre_sync_task);
-			_STARPU_DEP_DEBUG("dep ID%lu -> %p\n", handle->last_submitted_ghost_writer_id, pre_sync_task);
-			handle->last_submitted_ghost_writer_id_is_valid = 0;
-		}
-                else
-                {
-			_STARPU_DEP_DEBUG("No dep ID\n");
-                }
-	}
-
-	handle->last_submitted_writer = post_sync_task;
+	handle->last_submitted_accessors = NULL;
+	handle->last_sync_task = post_sync_task;
 
 	if (!post_sync_task->cl) {
 		/* Add a reference to be released in _starpu_handle_job_termination */
@@ -245,7 +195,6 @@ struct starpu_task *_starpu_detect_implicit_data_deps_with_handle(struct starpu_
 		if (pre_sync_job->reduction_task || post_sync_job->reduction_task)
 			return NULL;
 
-		_STARPU_DEP_DEBUG("Tasks %p %p\n", pre_sync_task, post_sync_task);
 		/* In case we are generating the DAG, we add an implicit
 		 * dependency between the pre and the post sync tasks in case
 		 * they are not the same. */
@@ -261,54 +210,75 @@ struct starpu_task *_starpu_detect_implicit_data_deps_with_handle(struct starpu_
 
 		enum starpu_data_access_mode previous_mode = handle->last_submitted_mode;
 
-		if (mode & STARPU_W)
+		_STARPU_DEP_DEBUG("Handle %p Tasks %p %p %x->%x\n", handle, pre_sync_task, post_sync_task, previous_mode, mode);
+
+		/*
+		 * Tasks can access the data concurrently only if they have the
+		 * same access mode, which can only be either:
+		 * - write with STARPU_COMMUTE
+		 * - read
+		 * - redux
+		 *
+		 * In other cases, the tasks have to depend on each other.
+		 */
+
+		if ((mode & STARPU_W && mode & STARPU_COMMUTE && previous_mode & STARPU_W && previous_mode && STARPU_COMMUTE)
+		  || (mode == STARPU_R && previous_mode == STARPU_R)
+		  || (mode == STARPU_REDUX && previous_mode == STARPU_REDUX))
 		{
-			_STARPU_DEP_DEBUG("W %p\n", handle);
-			if (previous_mode & STARPU_W)
-			{
-				_STARPU_DEP_DEBUG("WAW %p\n", handle);
-				_starpu_add_writer_after_writer(handle, pre_sync_task, post_sync_task);
-			}
-			else
-			{
-				/* The task submitted previously were in read-only
-				 * mode: this task must depend on all those read-only
-				 * tasks and we get rid of the list of readers */
-				_STARPU_DEP_DEBUG("WAR %p\n", handle);
-				_starpu_add_writer_after_readers(handle, pre_sync_task, post_sync_task);
-			}
+			_STARPU_DEP_DEBUG("concurrently\n");
+			/* Can access concurrently with current tasks */
+			_starpu_add_accessor(handle, pre_sync_task, post_sync_task);
 		}
 		else
 		{
-			_STARPU_DEP_DEBUG("R %p %d -> %d\n", handle, previous_mode, mode);
-			/* Add a reader, after a writer or a reader. */
-			STARPU_ASSERT(pre_sync_task);
-			STARPU_ASSERT(post_sync_task);
+			/* Can not access concurrently, have to wait for existing accessors */
+			struct _starpu_task_wrapper_list *l = handle->last_submitted_accessors;
+			_STARPU_DEP_DEBUG("dependency\n");
 
-			STARPU_ASSERT(mode & (STARPU_R|STARPU_REDUX));
-
-			if (!(previous_mode & STARPU_W) && (mode != previous_mode))
+			if (l && l->next)
 			{
-				/* Read after Redux or Redux after Read: we
-				 * insert a dummy synchronization task so that
-				 * we don't need to have a gigantic number of
-				 * dependencies between all readers and all
-				 * redux tasks. */
-
-				/* Create an empty task */
-				struct starpu_task *new_sync_task;
-				new_sync_task = starpu_task_create();
-				STARPU_ASSERT(new_sync_task);
-				new_sync_task->cl = NULL;
+				/* Several previous accessors */
+
+				if (mode == STARPU_W)
+				{
+					/* Optimization: this task can not
+					 * combine with others anyway, use it
+					 * as synchronization task by making it
+					 * wait for the previous ones. */
+					_starpu_add_sync_task(handle, pre_sync_task, post_sync_task);
+				} else {
+					_STARPU_DEP_DEBUG("several predecessors, adding sync task\n");
+					/* insert an empty synchronization task
+					 * which waits for the whole set,
+					 * instead of creating a quadratic
+					 * number of dependencies. */
+					struct starpu_task *sync_task = starpu_task_create();
+					STARPU_ASSERT(sync_task);
+					sync_task->cl = NULL;
 #ifdef STARPU_USE_FXT
-				_starpu_get_job_associated_to_task(new_sync_task)->model_name = "sync_task_redux";
+					_starpu_get_job_associated_to_task(sync_task)->model_name = "sync_task_redux";
 #endif
+					/* Make this task wait for the previous ones */
+					_starpu_add_sync_task(handle, sync_task, sync_task);
+					/* And the requested task wait for this one */
+					_starpu_add_accessor(handle, pre_sync_task, post_sync_task);
 
-				_starpu_add_writer_after_readers(handle, new_sync_task, new_sync_task);
-
-				task = new_sync_task;
+					task = sync_task;
+				}
+			}
+			else
+			{
+				if (l)
+				{
+					/* One previous accessor, make it the sync
+					 * task, and start depending on it. */
+					handle->last_sync_task = l->task;
+					handle->last_submitted_accessors = NULL;
+					free(l);
+				}
+				_starpu_add_accessor(handle, pre_sync_task, post_sync_task);
 			}
-			_starpu_add_reader_after_writer(handle, pre_sync_task, post_sync_task);
 		}
 		handle->last_submitted_mode = mode;
 	}
@@ -374,18 +344,18 @@ void _starpu_release_data_enforce_sequential_consistency(struct starpu_task *tas
 
 		/* If this is the last writer, there is no point in adding
 		 * extra deps to that tasks that does not exists anymore */
-		if (task == handle->last_submitted_writer)
+		if (task == handle->last_sync_task)
 		{
-			handle->last_submitted_writer = NULL;
+			handle->last_sync_task = NULL;
 
 #ifndef STARPU_USE_FXT
 			if (_starpu_bound_recording)
 #endif
 			{
 				/* Save the previous writer as the ghost last writer */
-				handle->last_submitted_ghost_writer_id_is_valid = 1;
+				handle->last_submitted_ghost_sync_id_is_valid = 1;
 				struct _starpu_job *ghost_job = _starpu_get_job_associated_to_task(task);
-				handle->last_submitted_ghost_writer_id = ghost_job->job_id;
+				handle->last_submitted_ghost_sync_id = ghost_job->job_id;
 			}
 		}
 
@@ -397,7 +367,7 @@ void _starpu_release_data_enforce_sequential_consistency(struct starpu_task *tas
 		/* Same if this is one of the readers: we go through the list
 		 * of readers and remove the task if it is found. */
 		struct _starpu_task_wrapper_list *l;
-		l = handle->last_submitted_readers;
+		l = handle->last_submitted_accessors;
 		struct _starpu_task_wrapper_list *prev = NULL;
 #ifdef STARPU_DEVEL
 #warning TODO: use double-linked list to make finding ourself fast
@@ -419,9 +389,9 @@ void _starpu_release_data_enforce_sequential_consistency(struct starpu_task *tas
 					struct _starpu_job *ghost_reader_job = _starpu_get_job_associated_to_task(task);
 					struct _starpu_jobid_list *link = (struct _starpu_jobid_list *) malloc(sizeof(struct _starpu_jobid_list));
 					STARPU_ASSERT(link);
-					link->next = handle->last_submitted_ghost_readers_id;
+					link->next = handle->last_submitted_ghost_accessors_id;
 					link->id = ghost_reader_job->job_id;
-					handle->last_submitted_ghost_readers_id = link;
+					handle->last_submitted_ghost_accessors_id = link;
 				}
 
 				if (prev)
@@ -431,7 +401,7 @@ void _starpu_release_data_enforce_sequential_consistency(struct starpu_task *tas
 				else
 				{
 					/* This is the first element of the list */
-					handle->last_submitted_readers = next;
+					handle->last_submitted_accessors = next;
 				}
 
 				/* XXX can we really find the same task again

+ 2 - 1
src/core/sched_ctx.c

@@ -842,7 +842,7 @@ void* starpu_sched_ctx_get_policy_data(unsigned sched_ctx_id)
 	return sched_ctx->policy_data;
 }
 
-struct starpu_worker_collection* starpu_sched_ctx_create_worker_collection(unsigned sched_ctx_id, int worker_collection_type)
+struct starpu_worker_collection* starpu_sched_ctx_create_worker_collection(unsigned sched_ctx_id, enum starpu_worker_collection_type  worker_collection_type)
 {
 	struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
 	sched_ctx->workers = (struct starpu_worker_collection*)malloc(sizeof(struct starpu_worker_collection));
@@ -881,6 +881,7 @@ static unsigned _get_workers_list(struct _starpu_sched_ctx *sched_ctx, int **wor
 	}
 	return nworkers;
 }
+
 void starpu_sched_ctx_delete_worker_collection(unsigned sched_ctx_id)
 {
 	struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);

+ 3 - 0
src/datawizard/coherency.c

@@ -371,6 +371,9 @@ struct _starpu_data_request *_starpu_create_request_to_fetch_data(starpu_data_ha
 								  unsigned async,
 								  void (*callback_func)(void *), void *callback_arg)
 {
+	/* We don't care about commuting for data requests, that was handled before. */
+	mode &= ~STARPU_COMMUTE;
+
 	/* This function is called with handle's header lock taken */
 	_starpu_spin_checklocked(&handle->header_lock);
 

+ 10 - 7
src/datawizard/coherency.h

@@ -1,6 +1,6 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
- * Copyright (C) 2009-2012  Université de Bordeaux 1
+ * Copyright (C) 2009-2013  Université de Bordeaux 1
  * Copyright (C) 2010, 2011, 2012, 2013  Centre National de la Recherche Scientifique
  *
  * StarPU is free software; you can redistribute it and/or modify
@@ -106,6 +106,9 @@ struct _starpu_data_state
 	 * the req_list anymore), i.e. the number of holders of the
 	 * current_mode rwlock */
 	unsigned refcnt;
+	/* Current access mode. Is always either STARPU_R, STARPU_W,
+	 * STARPU_SCRATCH or STARPU_REDUX, but never a combination such as
+	 * STARPU_RW. */
 	enum starpu_data_access_mode current_mode;
 	/* protect meta data */
 	struct _starpu_spinlock header_lock;
@@ -138,7 +141,7 @@ struct _starpu_data_state
 	/* Footprint which identifies data layout */
 	uint32_t footprint;
 
-	/* where is the data home ? -1 if none yet */
+	/* where is the data home, i.e. which node it was registered from ? -1 if none yet */
 	int home_node;
 
 	/* what is the default write-through mask for that data ? */
@@ -163,8 +166,8 @@ struct _starpu_data_state
 	 * read-only mode should depend on that task implicitely if the
 	 * sequential_consistency flag is enabled. */
 	enum starpu_data_access_mode last_submitted_mode;
-	struct starpu_task *last_submitted_writer;
-	struct _starpu_task_wrapper_list *last_submitted_readers;
+	struct starpu_task *last_sync_task;
+	struct _starpu_task_wrapper_list *last_submitted_accessors;
 
 	/* If FxT is enabled, we keep track of "ghost dependencies": that is to
 	 * say the dependencies that are not needed anymore, but that should
@@ -172,9 +175,9 @@ struct _starpu_data_state
 	 * f(Aw) g(Aw), and that g is submitted after the termination of f, we
 	 * want to have f->g appear in the DAG even if StarPU does not need to
 	 * enforce this dependency anymore.*/
-	unsigned last_submitted_ghost_writer_id_is_valid;
-	unsigned long last_submitted_ghost_writer_id;
-	struct _starpu_jobid_list *last_submitted_ghost_readers_id;
+	unsigned last_submitted_ghost_sync_id_is_valid;
+	unsigned long last_submitted_ghost_sync_id;
+	struct _starpu_jobid_list *last_submitted_ghost_accessors_id;
 
 	struct _starpu_task_wrapper_list *post_sync_tasks;
 	unsigned post_sync_tasks_cnt;

+ 4 - 1
src/datawizard/data_request.h

@@ -43,7 +43,10 @@ LIST_TYPE(_starpu_data_request,
 	struct _starpu_data_replicate *src_replicate;
 	struct _starpu_data_replicate *dst_replicate;
 
-	/* Which memory node will actually perform the transfer */
+	/* Which memory node will actually perform the transfer.
+	 * This is important in the CUDA/OpenCL case, where only the worker for
+	 * the node can make the CUDA/OpenCL calls.
+	 */
 	unsigned handling_node;
 
 	/*

+ 5 - 5
src/datawizard/filters.c

@@ -184,8 +184,8 @@ void starpu_data_partition(starpu_data_handle_t initial_handle, struct starpu_da
 
 		STARPU_PTHREAD_MUTEX_INIT(&child->sequential_consistency_mutex, NULL);
 		child->last_submitted_mode = STARPU_R;
-		child->last_submitted_writer = NULL;
-		child->last_submitted_readers = NULL;
+		child->last_sync_task = NULL;
+		child->last_submitted_accessors = NULL;
 		child->post_sync_tasks = NULL;
 		child->post_sync_tasks_cnt = 0;
 
@@ -195,9 +195,9 @@ void starpu_data_partition(starpu_data_handle_t initial_handle, struct starpu_da
 		child->init_cl = initial_handle->init_cl;
 
 #ifdef STARPU_USE_FXT
-		child->last_submitted_ghost_writer_id_is_valid = 0;
-		child->last_submitted_ghost_writer_id = 0;
-		child->last_submitted_ghost_readers_id = NULL;
+		child->last_submitted_ghost_sync_id_is_valid = 0;
+		child->last_submitted_ghost_sync_id = 0;
+		child->last_submitted_ghost_accessors_id = NULL;
 #endif
 
 		for (node = 0; node < STARPU_MAXNODES; node++)

+ 9 - 196
src/datawizard/interfaces/block_interface.c

@@ -29,7 +29,6 @@
 #include <drivers/scc/driver_scc_source.h>
 #include <drivers/mic/driver_mic_source.h>
 
-static int copy_ram_to_ram(void *src_interface, unsigned src_node STARPU_ATTRIBUTE_UNUSED, void *dst_interface, unsigned dst_node STARPU_ATTRIBUTE_UNUSED);
 #ifdef STARPU_USE_CUDA
 static int copy_ram_to_cuda(void *src_interface, unsigned src_node STARPU_ATTRIBUTE_UNUSED, void *dst_interface, unsigned dst_node STARPU_ATTRIBUTE_UNUSED);
 static int copy_cuda_to_ram(void *src_interface, unsigned src_node STARPU_ATTRIBUTE_UNUSED, void *dst_interface, unsigned dst_node STARPU_ATTRIBUTE_UNUSED);
@@ -45,21 +44,10 @@ static int copy_ram_to_opencl_async(void *src_interface, unsigned src_node STARP
 static int copy_opencl_to_ram_async(void *src_interface, unsigned src_node STARPU_ATTRIBUTE_UNUSED, void *dst_interface, unsigned dst_node STARPU_ATTRIBUTE_UNUSED, cl_event *event);
 static int copy_opencl_to_opencl_async(void *src_interface, unsigned src_node STARPU_ATTRIBUTE_UNUSED, void *dst_interface, unsigned dst_node STARPU_ATTRIBUTE_UNUSED, cl_event *event);
 #endif
-#ifdef STARPU_USE_SCC
-static int copy_scc_src_to_sink(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node);
-static int copy_scc_sink_to_src(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node);
-static int copy_scc_sink_to_sink(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node);
-#endif
-#ifdef STARPU_USE_MIC
-static int copy_ram_to_mic(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node);
-static int copy_mic_to_ram(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node);
-static int copy_ram_to_mic_async(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node);
-static int copy_mic_to_ram_async(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node);
-#endif
+static int copy_any_to_any(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node, void *async_data);
 
 static const struct starpu_data_copy_methods block_copy_data_methods_s =
 {
-	.ram_to_ram = copy_ram_to_ram,
 #ifdef STARPU_USE_CUDA
 	.ram_to_cuda = copy_ram_to_cuda,
 	.cuda_to_ram = copy_cuda_to_ram,
@@ -75,17 +63,7 @@ static const struct starpu_data_copy_methods block_copy_data_methods_s =
 	.opencl_to_ram_async = copy_opencl_to_ram_async,
 	.opencl_to_opencl_async = copy_opencl_to_opencl_async,
 #endif
-#ifdef STARPU_USE_SCC
-	.scc_src_to_sink = copy_scc_src_to_sink,
-	.scc_sink_to_src = copy_scc_sink_to_src,
-	.scc_sink_to_sink = copy_scc_sink_to_sink,
-#endif
-#ifdef STARPU_USE_MIC
-	.ram_to_mic = copy_ram_to_mic,
-	.mic_to_ram = copy_mic_to_ram,
-	.ram_to_mic_async = copy_ram_to_mic_async,
-	.mic_to_ram_async = copy_mic_to_ram_async,
-#endif
+	.any_to_any = copy_any_to_any,
 };
 
 
@@ -615,175 +593,11 @@ static int copy_opencl_to_opencl(void *src_interface, unsigned src_node STARPU_A
 
 #endif
 
-#ifdef STARPU_USE_SCC
-static int copy_scc_src_to_sink(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node)
-{
-	uint32_t nx = STARPU_BLOCK_GET_NX(dst_interface);
-	uint32_t ny = STARPU_BLOCK_GET_NY(dst_interface);
-	uint32_t nz = STARPU_BLOCK_GET_NZ(dst_interface);
-
-	size_t elemsize = STARPU_BLOCK_GET_ELEMSIZE(dst_interface);
-
-	uint32_t src_ldy = STARPU_BLOCK_GET_LDY(src_interface);
-	uint32_t src_ldz = STARPU_BLOCK_GET_LDZ(src_interface);
-	uint32_t dst_ldy = STARPU_BLOCK_GET_LDY(dst_interface);
-	uint32_t dst_ldz = STARPU_BLOCK_GET_LDZ(dst_interface);
-
-	void *src_ptr = (void *)STARPU_BLOCK_GET_PTR(src_interface);
-	void *dst_ptr = (void *)STARPU_BLOCK_GET_PTR(dst_interface);
-
-	unsigned y, z;
-	for (z = 0; z < nz; ++z)
-	{
-		for (y = 0; y < ny; ++y)
-		{
-			uint32_t src_offset = (y*src_ldy + z*src_ldz) * elemsize;
-			uint32_t dst_offset = (y*dst_ldy + z*dst_ldz) * elemsize;
-
-			_starpu_scc_copy_src_to_sink(src_ptr + src_offset, src_node,
-							dst_ptr + dst_offset, dst_node, nx*elemsize);
-		}
-	}
-
-	_STARPU_TRACE_DATA_COPY(src_node, dst_node, nx*ny*nz*elemsize);
-
-	return 0;
-}
-
-static int copy_scc_sink_to_src(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node)
-{
-	uint32_t nx = STARPU_BLOCK_GET_NX(dst_interface);
-	uint32_t ny = STARPU_BLOCK_GET_NY(dst_interface);
-	uint32_t nz = STARPU_BLOCK_GET_NZ(dst_interface);
-
-	size_t elemsize = STARPU_BLOCK_GET_ELEMSIZE(dst_interface);
-
-	uint32_t src_ldy = STARPU_BLOCK_GET_LDY(src_interface);
-	uint32_t src_ldz = STARPU_BLOCK_GET_LDZ(src_interface);
-	uint32_t dst_ldy = STARPU_BLOCK_GET_LDY(dst_interface);
-	uint32_t dst_ldz = STARPU_BLOCK_GET_LDZ(dst_interface);
-
-	void *src_ptr = (void *)STARPU_BLOCK_GET_PTR(src_interface);
-	void *dst_ptr = (void *)STARPU_BLOCK_GET_PTR(dst_interface);
-
-	unsigned y, z;
-	for (z = 0; z < nz; ++z)
-	{
-		for (y = 0; y < ny; ++y)
-		{
-			uint32_t src_offset = (y*src_ldy + z*src_ldz) * elemsize;
-			uint32_t dst_offset = (y*dst_ldy + z*dst_ldz) * elemsize;
-
-			_starpu_scc_copy_sink_to_src(src_ptr + src_offset, src_node,
-							dst_ptr + dst_offset, dst_node, nx*elemsize);
-		}
-	}
-
-	_STARPU_TRACE_DATA_COPY(src_node, dst_node, nx*ny*nz*elemsize);
-
-	return 0;
-}
-
-static int copy_scc_sink_to_sink(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node)
-{
-	uint32_t nx = STARPU_BLOCK_GET_NX(dst_interface);
-	uint32_t ny = STARPU_BLOCK_GET_NY(dst_interface);
-	uint32_t nz = STARPU_BLOCK_GET_NZ(dst_interface);
-
-	size_t elemsize = STARPU_BLOCK_GET_ELEMSIZE(dst_interface);
-
-	uint32_t src_ldy = STARPU_BLOCK_GET_LDY(src_interface);
-	uint32_t src_ldz = STARPU_BLOCK_GET_LDZ(src_interface);
-	uint32_t dst_ldy = STARPU_BLOCK_GET_LDY(dst_interface);
-	uint32_t dst_ldz = STARPU_BLOCK_GET_LDZ(dst_interface);
-
-	void *src_ptr = (void *)STARPU_BLOCK_GET_PTR(src_interface);
-	void *dst_ptr = (void *)STARPU_BLOCK_GET_PTR(dst_interface);
-
-	unsigned y, z;
-	for (z = 0; z < nz; ++z)
-	{
-		for (y = 0; y < ny; ++y)
-		{
-			uint32_t src_offset = (y*src_ldy + z*src_ldz) * elemsize;
-			uint32_t dst_offset = (y*dst_ldy + z*dst_ldz) * elemsize;
-
-			_starpu_scc_copy_sink_to_sink(src_ptr + src_offset, src_node,
-					dst_ptr + dst_offset, dst_node, nx*elemsize);
-		}
-	}
-
-	_STARPU_TRACE_DATA_COPY(src_node, dst_node, nx*ny*nz*elemsize);
-
-	return 0;
-}
-#endif /* STARPU_USE_SCC */
-
-#ifdef STARPU_USE_MIC
-static int copy_mic_common(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node,
-						   int (*copy_func)(void *, unsigned, void *, unsigned, size_t))
-{
-	struct starpu_block_interface *src_block = src_interface;
-	struct starpu_block_interface *dst_block = dst_interface;
-	
-	uint32_t nx = dst_block->nx;
-	uint32_t ny = dst_block->ny;
-	uint32_t nz = dst_block->nz;
-	size_t elemsize = dst_block->elemsize;
-
-	uint32_t ldy_src = src_block->ldy;
-	uint32_t ldz_src = src_block->ldz;
-	uint32_t ldy_dst = dst_block->ldy;
-	uint32_t ldz_dst = dst_block->ldz;
-
-	uintptr_t ptr_src = src_block->ptr;
-	uintptr_t ptr_dst = dst_block->ptr;
-
-	unsigned y, z;
-	for (z = 0; z < nz; z++)
-	{
-		for (y = 0; y < ny; y++)
-		{
-			uint32_t src_offset = (y*ldy_src + z*ldz_src)*elemsize;
-			uint32_t dst_offset = (y*ldy_dst + z*ldz_dst)*elemsize;
-
-			copy_func((void *)(ptr_src + src_offset), src_node, (void *)(ptr_dst + dst_offset), dst_node, nx*elemsize);
-		}
-	}
-
-	_STARPU_TRACE_DATA_COPY(src_node, dst_node, nx*ny*nz*elemsize);
-
-	return 0;
-
-}
-static int copy_ram_to_mic(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node)
-{
-	return copy_mic_common(src_interface, src_node, dst_interface, dst_node, _starpu_mic_copy_ram_to_mic);
-}
-
-static int copy_mic_to_ram(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node)
-{
-	return copy_mic_common(src_interface, src_node, dst_interface, dst_node, _starpu_mic_copy_mic_to_ram);
-}
-
-static int copy_ram_to_mic_async(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node)
-{
-	copy_mic_common(src_interface, src_node, dst_interface, dst_node, _starpu_mic_copy_ram_to_mic_async);
-	return -EAGAIN;
-}
-
-static int copy_mic_to_ram_async(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node)
-{
-	copy_mic_common(src_interface, src_node, dst_interface, dst_node, _starpu_mic_copy_mic_to_ram_async);
-	return -EAGAIN;
-}
-#endif
-
-/* as not all platform easily have a BLAS lib installed ... */
-static int copy_ram_to_ram(void *src_interface, unsigned src_node STARPU_ATTRIBUTE_UNUSED, void *dst_interface, unsigned dst_node STARPU_ATTRIBUTE_UNUSED)
+static int copy_any_to_any(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node, void *async_data)
 {
 	struct starpu_block_interface *src_block = (struct starpu_block_interface *) src_interface;
 	struct starpu_block_interface *dst_block = (struct starpu_block_interface *) dst_interface;
+	int ret;
 
 	uint32_t nx = dst_block->nx;
 	uint32_t ny = dst_block->ny;
@@ -795,9 +609,6 @@ static int copy_ram_to_ram(void *src_interface, unsigned src_node STARPU_ATTRIBU
 	uint32_t ldy_dst = dst_block->ldy;
 	uint32_t ldz_dst = dst_block->ldz;
 
-	uintptr_t ptr_src = src_block->ptr;
-	uintptr_t ptr_dst = dst_block->ptr;
-
 	unsigned y, z;
 	for (z = 0; z < nz; z++)
 	{
@@ -806,12 +617,14 @@ static int copy_ram_to_ram(void *src_interface, unsigned src_node STARPU_ATTRIBU
 			uint32_t src_offset = (y*ldy_src + z*ldz_src)*elemsize;
 			uint32_t dst_offset = (y*ldy_dst + z*ldz_dst)*elemsize;
 
-			memcpy((void *)(ptr_dst + dst_offset),
-				(void *)(ptr_src + src_offset), nx*elemsize);
+			if (starpu_interface_copy(src_block->dev_handle, src_block->offset + src_offset, src_node,
+			                          dst_block->dev_handle, dst_block->offset + dst_offset, dst_node,
+			                          nx*elemsize, async_data))
+				ret = -EAGAIN;
 		}
 	}
 
 	_STARPU_TRACE_DATA_COPY(src_node, dst_node, nx*ny*nz*elemsize);
 
-	return 0;
+	return ret;
 }

+ 5 - 5
src/datawizard/interfaces/data_interface.c

@@ -197,8 +197,8 @@ static void _starpu_register_new_data(starpu_data_handle_t handle,
 
 	STARPU_PTHREAD_MUTEX_INIT(&handle->sequential_consistency_mutex, NULL);
 	handle->last_submitted_mode = STARPU_R;
-	handle->last_submitted_writer = NULL;
-	handle->last_submitted_readers = NULL;
+	handle->last_sync_task = NULL;
+	handle->last_submitted_accessors = NULL;
 	handle->post_sync_tasks = NULL;
 	handle->post_sync_tasks_cnt = 0;
 
@@ -210,9 +210,9 @@ static void _starpu_register_new_data(starpu_data_handle_t handle,
 	handle->reduction_req_list = _starpu_data_requester_list_new();
 
 #ifdef STARPU_USE_FXT
-	handle->last_submitted_ghost_writer_id_is_valid = 0;
-	handle->last_submitted_ghost_writer_id = 0;
-	handle->last_submitted_ghost_readers_id = NULL;
+	handle->last_submitted_ghost_sync_id_is_valid = 0;
+	handle->last_submitted_ghost_sync_id = 0;
+	handle->last_submitted_ghost_accessors_id = NULL;
 #endif
 
 	handle->wt_mask = wt_mask;

+ 16 - 1
src/datawizard/interfaces/data_interface.h

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2009-2012  Université de Bordeaux 1
- * Copyright (C) 2010, 2012  Centre National de la Recherche Scientifique
+ * Copyright (C) 2010, 2012, 2013  Centre National de la Recherche Scientifique
  *
  * StarPU is free software; you can redistribute it and/or modify
  * it under the terms of the GNU Lesser General Public License as published by
@@ -21,6 +21,21 @@
 #include <starpu.h>
 #include <common/config.h>
 
+/* Generic type representing an interface, for now it's only used before
+ * execution on message-passing devices but it can be useful in other cases.
+ */
+union _starpu_interface
+{
+	struct starpu_matrix_interface matrix;
+	struct starpu_block_interface block;
+	struct starpu_vector_interface vector;
+	struct starpu_csr_interface csr;
+	struct starpu_coo_interface coo;
+	struct starpu_bcsr_interface bcsr;
+	struct starpu_variable_interface variable;
+	struct starpu_multiformat_interface multiformat;
+};
+
 /* Some data interfaces or filters use this interface internally */
 extern struct starpu_data_interface_ops starpu_interface_matrix_ops;
 extern struct starpu_data_interface_ops starpu_interface_block_ops;

+ 9 - 174
src/datawizard/interfaces/matrix_interface.c

@@ -30,7 +30,6 @@
 /* If you can promise that there is no stride in your matrices, you can define this */
 // #define NO_STRIDE
 
-static int copy_ram_to_ram(void *src_interface, unsigned src_node STARPU_ATTRIBUTE_UNUSED, void *dst_interface, unsigned dst_node STARPU_ATTRIBUTE_UNUSED);
 #ifdef STARPU_USE_CUDA
 static int copy_ram_to_cuda(void *src_interface, unsigned src_node STARPU_ATTRIBUTE_UNUSED, void *dst_interface, unsigned dst_node STARPU_ATTRIBUTE_UNUSED);
 static int copy_cuda_to_ram(void *src_interface, unsigned src_node STARPU_ATTRIBUTE_UNUSED, void *dst_interface, unsigned dst_node STARPU_ATTRIBUTE_UNUSED);
@@ -49,21 +48,10 @@ static int copy_ram_to_opencl_async(void *src_interface, unsigned src_node STARP
 static int copy_opencl_to_ram_async(void *src_interface, unsigned src_node STARPU_ATTRIBUTE_UNUSED, void *dst_interface, unsigned dst_node STARPU_ATTRIBUTE_UNUSED, cl_event *event);
 static int copy_opencl_to_opencl_async(void *src_interface, unsigned src_node STARPU_ATTRIBUTE_UNUSED, void *dst_interface, unsigned dst_node STARPU_ATTRIBUTE_UNUSED, cl_event *event);
 #endif
-#ifdef STARPU_USE_SCC
-static int copy_scc_src_to_sink(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node);
-static int copy_scc_sink_to_src(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node);
-static int copy_scc_sink_to_sink(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node);
-#endif
-#ifdef STARPU_USE_MIC
-static int copy_ram_to_mic(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node);
-static int copy_mic_to_ram(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node);
-static int copy_ram_to_mic_async(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node);
-static int copy_mic_to_ram_async(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node);
-#endif
+static int copy_any_to_any(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node, void *async_data);
 
 static const struct starpu_data_copy_methods matrix_copy_data_methods_s =
 {
-	.ram_to_ram = copy_ram_to_ram,
 #ifdef STARPU_USE_CUDA
 	.ram_to_cuda = copy_ram_to_cuda,
 	.cuda_to_ram = copy_cuda_to_ram,
@@ -89,17 +77,7 @@ static const struct starpu_data_copy_methods matrix_copy_data_methods_s =
 	.opencl_to_ram_async = copy_opencl_to_ram_async,
 	.opencl_to_opencl_async = copy_opencl_to_opencl_async,
 #endif
-#ifdef STARPU_USE_SCC
-	.scc_src_to_sink = copy_scc_src_to_sink,
-	.scc_sink_to_src = copy_scc_sink_to_src,
-	.scc_sink_to_sink = copy_scc_sink_to_sink,
-#endif
-#ifdef STARPU_USE_MIC
-	.ram_to_mic = copy_ram_to_mic,
-	.mic_to_ram = copy_mic_to_ram,
-	.ram_to_mic_async = copy_ram_to_mic_async,
-	.mic_to_ram_async = copy_mic_to_ram_async,
-#endif
+	.any_to_any = copy_any_to_any,
 };
 
 static void register_matrix_handle(starpu_data_handle_t handle, unsigned home_node, void *data_interface);
@@ -589,152 +567,11 @@ static int copy_opencl_to_opencl(void *src_interface, unsigned src_node STARPU_A
 
 #endif
 
-#ifdef STARPU_USE_SCC
-static int copy_scc_src_to_sink(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node)
-{
-	uint32_t nx = STARPU_MATRIX_GET_NX(dst_interface);
-	uint32_t ny = STARPU_MATRIX_GET_NY(dst_interface);
-
-	size_t elemsize = STARPU_MATRIX_GET_ELEMSIZE(dst_interface);
-
-	uint32_t src_ld = STARPU_MATRIX_GET_LD(src_interface);
-	uint32_t dst_ld = STARPU_MATRIX_GET_LD(dst_interface);
-
-	void *src_ptr = (void *)STARPU_MATRIX_GET_PTR(src_interface);
-	void *dst_ptr = (void *)STARPU_MATRIX_GET_PTR(dst_interface);
-
-	unsigned y;
-	for (y = 0; y < ny; ++y)
-	{
-		uint32_t src_offset = y*src_ld*elemsize;
-		uint32_t dst_offset = y*dst_ld*elemsize;
-
-		_starpu_scc_copy_src_to_sink(src_ptr + src_offset, src_node,
-						dst_ptr + dst_offset, dst_node, nx*elemsize);
-	}
-
-	_STARPU_TRACE_DATA_COPY(src_node, dst_node, nx*ny*elemsize);
-
-	return 0;
-}
-
-static int copy_scc_sink_to_src(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node)
-{
-	uint32_t nx = STARPU_MATRIX_GET_NX(dst_interface);
-	uint32_t ny = STARPU_MATRIX_GET_NY(dst_interface);
-
-	size_t elemsize = STARPU_MATRIX_GET_ELEMSIZE(dst_interface);
-
-	uint32_t src_ld = STARPU_MATRIX_GET_LD(src_interface);
-	uint32_t dst_ld = STARPU_MATRIX_GET_LD(dst_interface);
-
-	void *src_ptr = (void *)STARPU_MATRIX_GET_PTR(src_interface);
-	void *dst_ptr = (void *)STARPU_MATRIX_GET_PTR(dst_interface);
-
-	unsigned y;
-	for (y = 0; y < ny; ++y)
-	{
-		uint32_t src_offset = y*src_ld*elemsize;
-		uint32_t dst_offset = y*dst_ld*elemsize;
-
-		_starpu_scc_copy_sink_to_src(src_ptr + src_offset, src_node,
-						dst_ptr + dst_offset, dst_node, nx*elemsize);
-	}
-
-	_STARPU_TRACE_DATA_COPY(src_node, dst_node, nx*ny*elemsize);
-
-	return 0;
-}
-
-static int copy_scc_sink_to_sink(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node)
-{
-	uint32_t nx = STARPU_MATRIX_GET_NX(dst_interface);
-	uint32_t ny = STARPU_MATRIX_GET_NY(dst_interface);
-
-	size_t elemsize = STARPU_MATRIX_GET_ELEMSIZE(dst_interface);
-
-	uint32_t src_ld = STARPU_MATRIX_GET_LD(src_interface);
-	uint32_t dst_ld = STARPU_MATRIX_GET_LD(dst_interface);
-
-	void *src_ptr = (void *)STARPU_MATRIX_GET_PTR(src_interface);
-	void *dst_ptr = (void *)STARPU_MATRIX_GET_PTR(dst_interface);
-
-	unsigned y;
-	for (y = 0; y < ny; ++y)
-	{
-		uint32_t src_offset = y*src_ld*elemsize;
-		uint32_t dst_offset = y*dst_ld*elemsize;
-
-		_starpu_scc_copy_sink_to_sink(src_ptr + src_offset, src_node,
-						dst_ptr + dst_offset, dst_node, nx*elemsize);
-	}
-
-	_STARPU_TRACE_DATA_COPY(src_node, dst_node, nx*ny*elemsize);
-
-	return 0;
-}
-#endif /* STARPU_USE_SCC */
-
-#ifdef STARPU_USE_MIC
-static int copy_mic_common(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node,
-						   int (*copy_func)(void *, unsigned, void *, unsigned, size_t))
-{
-	struct starpu_matrix_interface *src_matrix = src_interface;
-	struct starpu_matrix_interface *dst_matrix = dst_interface;
-
-	unsigned y;
-	uint32_t nx = dst_matrix->nx;
-	uint32_t ny = dst_matrix->ny;
-	size_t elemsize = dst_matrix->elemsize;
-
-	uint32_t ld_src = src_matrix->ld;
-	uint32_t ld_dst = dst_matrix->ld;
-
-	uintptr_t ptr_src = src_matrix->ptr;
-	uintptr_t ptr_dst = dst_matrix->ptr;
-
-
-	for (y = 0; y < ny; y++)
-	{
-		uint32_t src_offset = y*ld_src*elemsize;
-		uint32_t dst_offset = y*ld_dst*elemsize;
-
-		copy_func((void *)(ptr_src + src_offset), src_node, (void *)(ptr_dst + dst_offset), dst_node, nx*elemsize);
-	}
-
-	_STARPU_TRACE_DATA_COPY(src_node, dst_node, (size_t)nx*ny*elemsize);
-
-	return 0;
-}
-
-static int copy_ram_to_mic(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node)
-{
-	return copy_mic_common(src_interface, src_node, dst_interface, dst_node, _starpu_mic_copy_ram_to_mic);
-}
-
-static int copy_mic_to_ram(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node)
-{
-	return copy_mic_common(src_interface, src_node, dst_interface, dst_node, _starpu_mic_copy_mic_to_ram);
-}
-
-static int copy_ram_to_mic_async(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node)
-{
-	copy_mic_common(src_interface, src_node, dst_interface, dst_node, _starpu_mic_copy_ram_to_mic_async);
-	return -EAGAIN;
-}
-
-static int copy_mic_to_ram_async(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node)
-{
-	copy_mic_common(src_interface, src_node, dst_interface, dst_node, _starpu_mic_copy_mic_to_ram_async);
-	return -EAGAIN;
-}
-#endif
-
-/* as not all platform easily have a  lib installed ... */
-static int copy_ram_to_ram(void *src_interface, unsigned src_node STARPU_ATTRIBUTE_UNUSED, void *dst_interface, unsigned dst_node STARPU_ATTRIBUTE_UNUSED)
+static int copy_any_to_any(void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node, void *async_data)
 {
 	struct starpu_matrix_interface *src_matrix = (struct starpu_matrix_interface *) src_interface;
 	struct starpu_matrix_interface *dst_matrix = (struct starpu_matrix_interface *) dst_interface;
+	int ret;
 
 	unsigned y;
 	uint32_t nx = dst_matrix->nx;
@@ -744,20 +581,18 @@ static int copy_ram_to_ram(void *src_interface, unsigned src_node STARPU_ATTRIBU
 	uint32_t ld_src = src_matrix->ld;
 	uint32_t ld_dst = dst_matrix->ld;
 
-	uintptr_t ptr_src = src_matrix->ptr;
-	uintptr_t ptr_dst = dst_matrix->ptr;
-
-
 	for (y = 0; y < ny; y++)
 	{
 		uint32_t src_offset = y*ld_src*elemsize;
 		uint32_t dst_offset = y*ld_dst*elemsize;
 
-		memcpy((void *)(ptr_dst + dst_offset),
-			(void *)(ptr_src + src_offset), nx*elemsize);
+		if (starpu_interface_copy(src_matrix->dev_handle, src_matrix->offset + src_offset, src_node,
+		                          dst_matrix->dev_handle, dst_matrix->offset + dst_offset, dst_node,
+		                          nx*elemsize, async_data))
+			ret = -EAGAIN;
 	}
 
 	_STARPU_TRACE_DATA_COPY(src_node, dst_node, (size_t)nx*ny*elemsize);
 
-	return 0;
+	return ret;
 }

+ 6 - 6
src/datawizard/reduction.c

@@ -333,13 +333,13 @@ void _starpu_data_end_reduction_mode(starpu_data_handle_t handle)
 			redux_task->cl = handle->redux_cl;
 			STARPU_ASSERT(redux_task->cl);
 
-			if (!redux_task->cl->modes[0])
-				redux_task->cl->modes[0] = STARPU_RW;
-			if (!redux_task->cl->modes[1])
-				redux_task->cl->modes[1] = STARPU_R;
+			if (!(STARPU_CODELET_GET_MODE(redux_task->cl, 0))
+				STARPU_CODELET_SET_MODE(redux_task->cl, STARPU_RW, 0);
+			if (!(STARPU_CODELET_GET_MODE(redux_task->cl, 1))
+				STARPU_CODELET_SET_MODE(redux_task->cl, STARPU_R, 1);
 
-			STARPU_ASSERT_MSG(redux_task->cl->modes[0] == STARPU_RW, "First parameter of reduction codelet has to be RW");
-			STARPU_ASSERT_MSG(redux_task->cl->modes[1] == STARPU_R, "Second parameter of reduction codelet has to be R");
+			STARPU_ASSERT_MSG(STARPU_CODELET_GET_MODE(redux_task->cl, 0) == STARPU_RW, "First parameter of reduction codelet has to be RW");
+			STARPU_ASSERT_MSG(STARPU_CODELET_GET_MODE(redux_task->cl, 1) == STARPU_R, "Second parameter of reduction codelet has to be R");
 
 			STARPU_TASK_SET_HANDLE(redux_task, handle, 0);
 			STARPU_TASK_SET_HANDLE(redux_task, replicate_array[replicate], 1);

+ 1 - 0
src/drivers/mp_common/mp_common.c

@@ -17,6 +17,7 @@
 #include <stdlib.h>
 #include <pthread.h>
 
+#include <datawizard/interfaces/data_interface.h>
 #include <drivers/mp_common/mp_common.h>
 #include <drivers/mp_common/sink_common.h>
 #include <drivers/mic/driver_mic_common.h>

+ 1 - 0
src/drivers/mp_common/source_common.c

@@ -20,6 +20,7 @@
 
 #include <starpu.h>
 #include <datawizard/coherency.h>
+#include <datawizard/interfaces/data_interface.h>
 #include <drivers/mp_common/mp_common.h>
 
 int

+ 1 - 0
src/drivers/scc/driver_scc_sink.c

@@ -17,6 +17,7 @@
 
 #include <RCCE.h>
 
+#include <datawizard/interfaces/data_interface.h>
 #include <drivers/mp_common/sink_common.h>
 #include <drivers/scc/driver_scc_common.h>
 #include <drivers/scc/driver_scc_sink.h>

+ 3 - 3
src/util/starpu_insert_task_utils.c

@@ -54,7 +54,7 @@ size_t _starpu_insert_task_get_arg_size(va_list varg_list)
 
 	while ((arg_type = va_arg(varg_list, int)) != 0)
 	{
-		if (arg_type==STARPU_R || arg_type==STARPU_W || arg_type==STARPU_RW || arg_type == STARPU_SCRATCH || arg_type == STARPU_REDUX)
+		if (arg_type & STARPU_R || arg_type & STARPU_W || arg_type & STARPU_SCRATCH || arg_type & STARPU_REDUX)
 		{
 			(void)va_arg(varg_list, starpu_data_handle_t);
 		}
@@ -135,7 +135,7 @@ int _starpu_codelet_pack_args(void **arg_buffer, size_t arg_buffer_size, va_list
 
 	while((arg_type = va_arg(varg_list, int)) != 0)
 	{
-		if (arg_type==STARPU_R || arg_type==STARPU_W || arg_type==STARPU_RW || arg_type == STARPU_SCRATCH || arg_type == STARPU_REDUX)
+		if (arg_type & STARPU_R || arg_type & STARPU_W || arg_type & STARPU_SCRATCH || arg_type & STARPU_REDUX)
 		{
 			(void)va_arg(varg_list, starpu_data_handle_t);
 		}
@@ -230,7 +230,7 @@ int _starpu_insert_task_create_and_submit(void *arg_buffer, size_t arg_buffer_si
 
 	while((arg_type = va_arg(varg_list, int)) != 0)
 	{
-		if (arg_type==STARPU_R || arg_type==STARPU_W || arg_type==STARPU_RW || arg_type == STARPU_SCRATCH || arg_type == STARPU_REDUX)
+		if (arg_type & STARPU_R || arg_type & STARPU_W || arg_type & STARPU_SCRATCH || arg_type & STARPU_REDUX)
 		{
 			/* We have an access mode : we expect to find a handle */
 			starpu_data_handle_t handle = va_arg(varg_list, starpu_data_handle_t);

+ 1 - 0
tests/Makefile.am

@@ -146,6 +146,7 @@ noinst_PROGRAMS =				\
 	datawizard/acquire_cb_insert		\
 	datawizard/acquire_release		\
 	datawizard/acquire_release2		\
+	datawizard/commute			\
 	datawizard/copy				\
 	datawizard/data_implicit_deps		\
 	datawizard/data_lookup			\

+ 178 - 0
tests/datawizard/commute.c

@@ -0,0 +1,178 @@
+/* StarPU --- Runtime system for heterogeneous multicore architectures.
+ *
+ * Copyright (C) 2013 Université de Bordeaux 1
+ *
+ * StarPU is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU Lesser General Public License as published by
+ * the Free Software Foundation; either version 2.1 of the License, or (at
+ * your option) any later version.
+ *
+ * StarPU is distributed in the hope that it will be useful, but
+ * WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.
+ *
+ * See the GNU Lesser General Public License in COPYING.LGPL for more details.
+ */
+
+#include <config.h>
+#include <starpu.h>
+#include "../helper.h"
+
+void begin(void *descr[], void *_args __attribute__((unused)))
+{
+	int *x = (int *)STARPU_VARIABLE_GET_PTR(descr[0]);
+
+	*x = 0;
+}
+
+static struct starpu_codelet codelet_begin =
+{
+	.cpu_funcs = {begin, NULL},
+	.cpu_funcs_name = {"begin", NULL},
+	.nbuffers = 1,
+};
+
+
+
+void commute1(void *descr[], void *_args __attribute__((unused)))
+{
+	int *x = (int *)STARPU_VARIABLE_GET_PTR(descr[0]);
+
+	*x = 1;
+}
+
+static struct starpu_codelet codelet_commute1 =
+{
+	.cpu_funcs = {commute1, NULL},
+	.cpu_funcs_name = {"commute1", NULL},
+	.nbuffers = 1,
+	.modes = {STARPU_RW | STARPU_COMMUTE}
+};
+
+
+
+void commute2(void *descr[], void *_args __attribute__((unused)))
+{
+	int *x = (int *)STARPU_VARIABLE_GET_PTR(descr[0]);
+
+	*x = 2;
+}
+
+static struct starpu_codelet codelet_commute2 =
+{
+	.cpu_funcs = {commute2, NULL},
+	.cpu_funcs_name = {"commute2", NULL},
+	.nbuffers = 1,
+	.modes = {STARPU_W | STARPU_COMMUTE}
+};
+
+void commute3(void *descr[] __attribute__((unused)), void *_args __attribute__((unused)))
+{
+}
+
+static struct starpu_codelet codelet_commute3 =
+{
+	.cpu_funcs = {commute3, NULL},
+	.cpu_funcs_name = {"commute3", NULL},
+	.nbuffers = 1,
+	.modes = {STARPU_RW | STARPU_COMMUTE}
+};
+
+
+
+static struct starpu_codelet codelet_end;
+void end(void *descr[], void *_args __attribute__((unused)))
+{
+	int *x = (int *)STARPU_VARIABLE_GET_PTR(descr[0]);
+
+	if (codelet_end.modes[0] & STARPU_W)
+		(*x)++;
+}
+
+static struct starpu_codelet codelet_end =
+{
+	.cpu_funcs = {end, NULL},
+	.cpu_funcs_name = {"end", NULL},
+	.nbuffers = 1,
+};
+
+static int x;
+static starpu_data_handle_t x_handle, f_handle;
+
+static void test(enum starpu_data_access_mode begin_mode, enum starpu_data_access_mode end_mode, int order)
+{
+	struct starpu_task *begin_t, *commute1_t, *commute2_t, *end_t;
+
+	codelet_begin.modes[0] = begin_mode;
+	codelet_end.modes[0] = end_mode;
+
+	begin_t = starpu_task_create();
+	begin_t->cl = &codelet_begin;
+	begin_t->handles[0] = x_handle;
+	begin_t->use_tag = 1;
+	begin_t->tag_id = 0;
+
+	commute1_t = starpu_task_create();
+	commute1_t->cl = &codelet_commute1;
+	commute1_t->handles[0] = x_handle;
+
+	commute2_t = starpu_task_create();
+	commute2_t->cl = &codelet_commute2;
+	commute2_t->handles[0] = x_handle;
+
+	if (order)
+		starpu_task_declare_deps_array(commute2_t, 1, &commute1_t);
+	else
+		starpu_task_declare_deps_array(commute1_t, 1, &commute2_t);
+
+	end_t = starpu_task_create();
+	end_t->cl = &codelet_end;
+	end_t->handles[0] = x_handle;
+	end_t->detach = 0;
+
+	if (starpu_task_submit(begin_t) == -ENODEV)
+		exit(STARPU_TEST_SKIPPED);
+	if (starpu_task_submit(commute1_t) == -ENODEV)
+		exit(STARPU_TEST_SKIPPED);
+	if (starpu_task_submit(commute2_t) == -ENODEV)
+		exit(STARPU_TEST_SKIPPED);
+	starpu_insert_task(&codelet_commute3, STARPU_RW|STARPU_COMMUTE, x_handle, 0);
+	if (starpu_task_submit(end_t) == -ENODEV)
+		exit(STARPU_TEST_SKIPPED);
+
+	starpu_task_wait(end_t);
+	starpu_data_acquire(x_handle, STARPU_R);
+	if (x != 1 + order + !!(end_mode & STARPU_W))
+		exit(EXIT_FAILURE);
+	starpu_data_release(x_handle);
+}
+
+int main(int argc, char **argv)
+{
+        int i, ret;
+
+	ret = starpu_initialize(NULL, &argc, &argv);
+	if (ret == -ENODEV) return STARPU_TEST_SKIPPED;
+	STARPU_CHECK_RETURN_VALUE(ret, "starpu_init");
+
+	/* Declare x */
+	starpu_variable_data_register(&x_handle, 0, (uintptr_t)&x, sizeof(x));
+
+	for (i = 0; i <= 1; i++)
+	{
+		test(STARPU_R, STARPU_R, i);
+		test(STARPU_W, STARPU_R, i);
+		test(STARPU_W, STARPU_RW, i);
+		test(STARPU_R, STARPU_RW, i);
+	}
+
+	starpu_shutdown();
+	STARPU_RETURN(0);
+
+enodev:
+	fprintf(stderr, "WARNING: No one can execute this task\n");
+	/* yes, we do not perform the computation but we did detect that no one
+ 	 * could perform the kernel, so this is not an error from StarPU */
+	starpu_shutdown();
+	return STARPU_TEST_SKIPPED;
+}

+ 2 - 2
tests/datawizard/sync_and_notify_data.c

@@ -1,6 +1,6 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
- * Copyright (C) 2010, 2012  Université de Bordeaux 1
+ * Copyright (C) 2010, 2012-2013  Université de Bordeaux 1
  * Copyright (C) 2010, 2011, 2012, 2013  Centre National de la Recherche Scientifique
  * Copyright (C) 2012 inria
  *
@@ -113,6 +113,7 @@ int main(int argc, char **argv)
 #ifdef STARPU_USE_OPENCL
 				.opencl_funcs = {opencl_codelet_incA, NULL},
 #endif
+				.cpu_funcs_name = {"cpu_codelet_incA", NULL},
 				.nbuffers = 1,
 				.modes = {STARPU_RW}
 			};
@@ -149,7 +150,6 @@ int main(int argc, char **argv)
 #ifdef STARPU_USE_OPENCL
 				.opencl_funcs = {opencl_codelet_incC, NULL},
 #endif
-				.cpu_funcs_name = {"cpu_codelet_incA", NULL},
 				.cpu_funcs_name = {"cpu_codelet_incC", NULL},
 				.nbuffers = 1,
 				.modes = {STARPU_RW}

+ 2 - 2
tests/datawizard/sync_and_notify_data_implicit.c

@@ -1,6 +1,6 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
- * Copyright (C) 2010  Université de Bordeaux 1
+ * Copyright (C) 2010, 2013  Université de Bordeaux 1
  * Copyright (C) 2010, 2011, 2012, 2013  Centre National de la Recherche Scientifique
  *
  * StarPU is free software; you can redistribute it and/or modify
@@ -86,6 +86,7 @@ static struct starpu_codelet cl_inc_a =
 #ifdef STARPU_USE_OPENCL
 	.opencl_funcs = {opencl_codelet_incA, NULL},
 #endif
+	.cpu_funcs_name = {"cpu_codelet_incA", NULL},
 	.nbuffers = 1,
 	.modes = {STARPU_RW}
 };
@@ -100,7 +101,6 @@ struct starpu_codelet cl_inc_c =
 #ifdef STARPU_USE_OPENCL
 	.opencl_funcs = {opencl_codelet_incC, NULL},
 #endif
-	.cpu_funcs_name = {"cpu_codelet_incA", NULL},
 	.cpu_funcs_name = {"cpu_codelet_incC", NULL},
 	.nbuffers = 1,
 	.modes = {STARPU_RW}