public inbox for gcc-patches@gcc.gnu.org
 help / color / mirror / Atom feed
* [gomp4.1] depend nowait support for target {update,{enter,exit} data}
@ 2015-09-08  9:26 Jakub Jelinek
  2015-10-02 19:28 ` Ilya Verbin
  0 siblings, 1 reply; 23+ messages in thread
From: Jakub Jelinek @ 2015-09-08  9:26 UTC (permalink / raw)
  To: Ilya Verbin, Aldy Hernandez; +Cc: gcc-patches

Hi!

This patch does two things:
1) removes fatal error from #pragma omp target update if object is not
mapped (at all, partial mapping is still a fatal error); the 4.1 draft spec
says that nothing is copied if the object is not mapped (first hunk)

2) implements nowait support for #pragma omp target {update,{enter,exit} data}
- if depend clause is not present, nowait is ignored, similarly if there is
no team (not inside of a parallel), or if the encountering task is final,
or if no children of the current task had depend clauses yet.  Otherwise,
a task is created, and when the dependencies are resolved and the task
scheduler will schedule it, it will perform the required update/enter/exit
action(s).  If there are depend clauses, the "target task" is not really
executed "immediately" as the spec says, but the spec is broken and I
believe is going to change (the question is when and to what wording).

nowait support for #pragma omp target is not implemented yet, supposedly we
need to mark those somehow (some flag) already in the struct gomp_task
structure, essentially it will need either 2 or 3 callbacks
(the current one, executed when the dependencies are resolved (it actually
waits until some thread schedules it after that point, I think it is
undesirable to run it with the tasking lock held), which would perform
the gomp_map_vars and initiate the running of the region, and then some
query routine which would poll the plugin whether the task is done or not,
and either perform the finalization (unmap_vars) if it is done (and in any
case return bool whether it should be polled again or not), and if the
finalization is not done there, also another callback for the finalization.
Also, there is the issue that if we are waiting for task that needs to be
polled, and we don't have any further tasks to run, we shouldn't really
attempt to sleep on some semaphore (e.g. in taskwait, end of
taskgroup, etc.) or barrier, but rather either need to keep polling it, or
call the query hook with some argument that it should sleep in there until
the work is done by the offloading device.
Also, there needs to be a way for the target nowait first callback to say
that it is using host fallback and thus acts as a normal task, therefore
once the task fn finishes, the task is done.

2015-09-08  Jakub Jelinek  <jakub@redhat.com>

	* target.c (gomp_update): Remove fatal error if object is not mapped.

	* target.c (GOMP_target_update_41): Handle nowait update with
	dependencies.  Don't call gomp_update if parallel or taskgroup has
	been cancelled.
	(GOMP_target_enter_exit_data): Likewise.
	(gomp_target_task_fn): New function.
	* task.c (gomp_task_handle_depend): New function, copied from...
	(GOMP_task): ... here.  Use gomp_task_handle_depend.
	(gomp_create_target_task): New function.
	* libgomp.h (struct gomp_target_task): New type.
	(gomp_create_target_task, gomp_target_task_fn): New prototypes.
	* testsuite/libgomp.c/target-27.c: New test.

--- libgomp/target.c.jj	2015-09-03 16:51:06.000000000 +0200
+++ libgomp/target.c	2015-09-08 09:55:24.591484158 +0200
@@ -899,13 +899,6 @@ gomp_update (struct gomp_device_descr *d
 						- n->host_start),
 				      cur_node.host_end - cur_node.host_start);
 	  }
-	else
-	  {
-	    gomp_mutex_unlock (&devicep->lock);
-	    gomp_fatal ("Trying to update [%p..%p) object that is not mapped",
-			(void *) cur_node.host_start,
-			(void *) cur_node.host_end);
-	  }
       }
   gomp_mutex_unlock (&devicep->lock);
 }
@@ -1460,18 +1453,50 @@ GOMP_target_update_41 (int device, size_
   /* If there are depend clauses, but nowait is not present,
      block the parent task until the dependencies are resolved
      and then just continue with the rest of the function as if it
-     is a merged task.  */
+     is a merged task.  Until we are able to schedule task during
+     variable mapping or unmapping, ignore nowait if depend clauses
+     are not present.  */
   if (depend != NULL)
     {
       struct gomp_thread *thr = gomp_thread ();
       if (thr->task && thr->task->depend_hash)
-	gomp_task_maybe_wait_for_dependencies (depend);
+	{
+	  if ((flags & GOMP_TARGET_FLAG_NOWAIT)
+	      && thr->ts.team
+	      && !thr->task->final_task)
+	    {
+	      gomp_create_target_task (devicep, (void (*) (void *)) NULL,
+				       mapnum, hostaddrs, sizes, kinds,
+				       flags | GOMP_TARGET_FLAG_UPDATE,
+				       depend);
+	      return;
+	    }
+
+	  struct gomp_team *team = thr->ts.team;
+	  /* If parallel or taskgroup has been cancelled, don't start new
+	     tasks.  */
+	  if (team
+	      && (gomp_team_barrier_cancelled (&team->barrier)
+		  || (thr->task->taskgroup
+		      && thr->task->taskgroup->cancelled)))
+	    return;
+
+	  gomp_task_maybe_wait_for_dependencies (depend);
+	}
     }
 
   if (devicep == NULL
       || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
     return;
 
+  struct gomp_thread *thr = gomp_thread ();
+  struct gomp_team *team = thr->ts.team;
+  /* If parallel or taskgroup has been cancelled, don't start new tasks.  */
+  if (team
+      && (gomp_team_barrier_cancelled (&team->barrier)
+	  || (thr->task->taskgroup && thr->task->taskgroup->cancelled)))
+    return;
+
   gomp_update (devicep, mapnum, hostaddrs, sizes, kinds, true);
 }
 
@@ -1548,18 +1573,49 @@ GOMP_target_enter_exit_data (int device,
   /* If there are depend clauses, but nowait is not present,
      block the parent task until the dependencies are resolved
      and then just continue with the rest of the function as if it
-     is a merged task.  */
+     is a merged task.  Until we are able to schedule task during
+     variable mapping or unmapping, ignore nowait if depend clauses
+     are not present.  */
   if (depend != NULL)
     {
       struct gomp_thread *thr = gomp_thread ();
       if (thr->task && thr->task->depend_hash)
-	gomp_task_maybe_wait_for_dependencies (depend);
+	{
+	  if ((flags & GOMP_TARGET_FLAG_NOWAIT)
+	      && thr->ts.team
+	      && !thr->task->final_task)
+	    {
+	      gomp_create_target_task (devicep, (void (*) (void *)) NULL,
+				       mapnum, hostaddrs, sizes, kinds,
+				       flags, depend);
+	      return;
+	    }
+
+	  struct gomp_team *team = thr->ts.team;
+	  /* If parallel or taskgroup has been cancelled, don't start new
+	     tasks.  */
+	  if (team
+	      && (gomp_team_barrier_cancelled (&team->barrier)
+		  || (thr->task->taskgroup
+		      && thr->task->taskgroup->cancelled)))
+	    return;
+
+	  gomp_task_maybe_wait_for_dependencies (depend);
+	}
     }
 
   if (devicep == NULL
       || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
     return;
 
+  struct gomp_thread *thr = gomp_thread ();
+  struct gomp_team *team = thr->ts.team;
+  /* If parallel or taskgroup has been cancelled, don't start new tasks.  */
+  if (team
+      && (gomp_team_barrier_cancelled (&team->barrier)
+	  || (thr->task->taskgroup && thr->task->taskgroup->cancelled)))
+    return;
+
   size_t i;
   if ((flags & GOMP_TARGET_FLAG_EXIT_DATA) == 0)
     for (i = 0; i < mapnum; i++)
@@ -1577,6 +1633,40 @@ GOMP_target_enter_exit_data (int device,
 }
 
 void
+gomp_target_task_fn (void *data)
+{
+  struct gomp_target_task *ttask = (struct gomp_target_task *) data;
+  if (ttask->fn != NULL)
+    {
+      /* GOMP_target_41 */
+    }
+  else if (ttask->devicep == NULL
+	   || !(ttask->devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+    return;
+
+  size_t i;
+  if (ttask->flags & GOMP_TARGET_FLAG_UPDATE)
+    gomp_update (ttask->devicep, ttask->mapnum, ttask->hostaddrs, ttask->sizes,
+		 ttask->kinds, true);
+  else if ((ttask->flags & GOMP_TARGET_FLAG_EXIT_DATA) == 0)
+    for (i = 0; i < ttask->mapnum; i++)
+      if ((ttask->kinds[i] & 0xff) == GOMP_MAP_STRUCT)
+	{
+	  gomp_map_vars (ttask->devicep, ttask->sizes[i] + 1,
+			 &ttask->hostaddrs[i], NULL, &ttask->sizes[i],
+			 &ttask->kinds[i], true, GOMP_MAP_VARS_ENTER_DATA);
+	  i += ttask->sizes[i];
+	}
+      else
+	gomp_map_vars (ttask->devicep, 1, &ttask->hostaddrs[i], NULL,
+		       &ttask->sizes[i], &ttask->kinds[i],
+		       true, GOMP_MAP_VARS_ENTER_DATA);
+  else
+    gomp_exit_data (ttask->devicep, ttask->mapnum, ttask->hostaddrs,
+		    ttask->sizes, ttask->kinds);
+}
+
+void
 GOMP_teams (unsigned int num_teams, unsigned int thread_limit)
 {
   if (thread_limit)
--- libgomp/task.c.jj	2015-09-02 15:22:14.000000000 +0200
+++ libgomp/task.c	2015-09-08 10:20:54.163978966 +0200
@@ -108,6 +108,123 @@ gomp_clear_parent (struct gomp_task *chi
     while (task != children);
 }
 
+/* Helper function for GOMP_task and gomp_create_target_task.  Depend clause
+   handling for undeferred task creation.  */
+
+static void
+gomp_task_handle_depend (struct gomp_task *task, struct gomp_task *parent,
+			 void **depend)
+{
+  size_t ndepend = (uintptr_t) depend[0];
+  size_t nout = (uintptr_t) depend[1];
+  size_t i;
+  hash_entry_type ent;
+
+  task->depend_count = ndepend;
+  task->num_dependees = 0;
+  if (parent->depend_hash == NULL)
+    parent->depend_hash = htab_create (2 * ndepend > 12 ? 2 * ndepend : 12);
+  for (i = 0; i < ndepend; i++)
+    {
+      task->depend[i].addr = depend[2 + i];
+      task->depend[i].next = NULL;
+      task->depend[i].prev = NULL;
+      task->depend[i].task = task;
+      task->depend[i].is_in = i >= nout;
+      task->depend[i].redundant = false;
+      task->depend[i].redundant_out = false;
+
+      hash_entry_type *slot = htab_find_slot (&parent->depend_hash,
+					      &task->depend[i], INSERT);
+      hash_entry_type out = NULL, last = NULL;
+      if (*slot)
+	{
+	  /* If multiple depends on the same task are the same, all but the
+	     first one are redundant.  As inout/out come first, if any of them
+	     is inout/out, it will win, which is the right semantics.  */
+	  if ((*slot)->task == task)
+	    {
+	      task->depend[i].redundant = true;
+	      continue;
+	    }
+	  for (ent = *slot; ent; ent = ent->next)
+	    {
+	      if (ent->redundant_out)
+		break;
+
+	      last = ent;
+
+	      /* depend(in:...) doesn't depend on earlier depend(in:...).  */
+	      if (i >= nout && ent->is_in)
+		continue;
+
+	      if (!ent->is_in)
+		out = ent;
+
+	      struct gomp_task *tsk = ent->task;
+	      if (tsk->dependers == NULL)
+		{
+		  tsk->dependers
+		    = gomp_malloc (sizeof (struct gomp_dependers_vec)
+				   + 6 * sizeof (struct gomp_task *));
+		  tsk->dependers->n_elem = 1;
+		  tsk->dependers->allocated = 6;
+		  tsk->dependers->elem[0] = task;
+		  task->num_dependees++;
+		  continue;
+		}
+	      /* We already have some other dependency on tsk from earlier
+		 depend clause.  */
+	      else if (tsk->dependers->n_elem
+		       && (tsk->dependers->elem[tsk->dependers->n_elem - 1]
+			   == task))
+		continue;
+	      else if (tsk->dependers->n_elem == tsk->dependers->allocated)
+		{
+		  tsk->dependers->allocated
+		    = tsk->dependers->allocated * 2 + 2;
+		  tsk->dependers
+		    = gomp_realloc (tsk->dependers,
+				    sizeof (struct gomp_dependers_vec)
+				    + (tsk->dependers->allocated
+				       * sizeof (struct gomp_task *)));
+		}
+	      tsk->dependers->elem[tsk->dependers->n_elem++] = task;
+	      task->num_dependees++;
+	    }
+	  task->depend[i].next = *slot;
+	  (*slot)->prev = &task->depend[i];
+	}
+      *slot = &task->depend[i];
+
+      /* There is no need to store more than one depend({,in}out:) task per
+	 address in the hash table chain for the purpose of creation of
+	 deferred tasks, because each out depends on all earlier outs, thus it
+	 is enough to record just the last depend({,in}out:).  For depend(in:),
+	 we need to keep all of the previous ones not terminated yet, because
+	 a later depend({,in}out:) might need to depend on all of them.  So, if
+	 the new task's clause is depend({,in}out:), we know there is at most
+	 one other depend({,in}out:) clause in the list (out).  For
+	 non-deferred tasks we want to see all outs, so they are moved to the
+	 end of the chain, after first redundant_out entry all following
+	 entries should be redundant_out.  */
+      if (!task->depend[i].is_in && out)
+	{
+	  if (out != last)
+	    {
+	      out->next->prev = out->prev;
+	      out->prev->next = out->next;
+	      out->next = last->next;
+	      out->prev = last;
+	      last->next = out;
+	      if (out->next)
+		out->next->prev = out;
+	    }
+	  out->redundant_out = true;
+	}
+    }
+}
+
 /* Called when encountering an explicit task directive.  If IF_CLAUSE is
    false, then we must not delay in executing the task.  If UNTIED is true,
    then the task may be executed by any member of the team.
@@ -248,123 +365,7 @@ GOMP_task (void (*fn) (void *), void *da
 	taskgroup->num_children++;
       if (depend_size)
 	{
-	  size_t ndepend = (uintptr_t) depend[0];
-	  size_t nout = (uintptr_t) depend[1];
-	  size_t i;
-	  hash_entry_type ent;
-
-	  task->depend_count = ndepend;
-	  task->num_dependees = 0;
-	  if (parent->depend_hash == NULL)
-	    parent->depend_hash
-	      = htab_create (2 * ndepend > 12 ? 2 * ndepend : 12);
-	  for (i = 0; i < ndepend; i++)
-	    {
-	      task->depend[i].addr = depend[2 + i];
-	      task->depend[i].next = NULL;
-	      task->depend[i].prev = NULL;
-	      task->depend[i].task = task;
-	      task->depend[i].is_in = i >= nout;
-	      task->depend[i].redundant = false;
-	      task->depend[i].redundant_out = false;
-
-	      hash_entry_type *slot
-		= htab_find_slot (&parent->depend_hash, &task->depend[i],
-				  INSERT);
-	      hash_entry_type out = NULL, last = NULL;
-	      if (*slot)
-		{
-		  /* If multiple depends on the same task are the
-		     same, all but the first one are redundant.
-		     As inout/out come first, if any of them is
-		     inout/out, it will win, which is the right
-		     semantics.  */
-		  if ((*slot)->task == task)
-		    {
-		      task->depend[i].redundant = true;
-		      continue;
-		    }
-		  for (ent = *slot; ent; ent = ent->next)
-		    {
-		      if (ent->redundant_out)
-			break;
-
-		      last = ent;
-
-		      /* depend(in:...) doesn't depend on earlier
-			 depend(in:...).  */
-		      if (i >= nout && ent->is_in)
-			continue;
-
-		      if (!ent->is_in)
-			out = ent;
-
-		      struct gomp_task *tsk = ent->task;
-		      if (tsk->dependers == NULL)
-			{
-			  tsk->dependers
-			    = gomp_malloc (sizeof (struct gomp_dependers_vec)
-					   + 6 * sizeof (struct gomp_task *));
-			  tsk->dependers->n_elem = 1;
-			  tsk->dependers->allocated = 6;
-			  tsk->dependers->elem[0] = task;
-			  task->num_dependees++;
-			  continue;
-			}
-		      /* We already have some other dependency on tsk
-			 from earlier depend clause.  */
-		      else if (tsk->dependers->n_elem
-			       && (tsk->dependers->elem[tsk->dependers->n_elem
-							- 1]
-				   == task))
-			continue;
-		      else if (tsk->dependers->n_elem
-			       == tsk->dependers->allocated)
-			{
-			  tsk->dependers->allocated
-			    = tsk->dependers->allocated * 2 + 2;
-			  tsk->dependers
-			    = gomp_realloc (tsk->dependers,
-					    sizeof (struct gomp_dependers_vec)
-					    + (tsk->dependers->allocated
-					       * sizeof (struct gomp_task *)));
-			}
-		      tsk->dependers->elem[tsk->dependers->n_elem++] = task;
-		      task->num_dependees++;
-		    }
-		  task->depend[i].next = *slot;
-		  (*slot)->prev = &task->depend[i];
-		}
-	      *slot = &task->depend[i];
-
-	      /* There is no need to store more than one depend({,in}out:)
-		 task per address in the hash table chain for the purpose
-		 of creation of deferred tasks, because each out
-		 depends on all earlier outs, thus it is enough to record
-		 just the last depend({,in}out:).  For depend(in:), we need
-		 to keep all of the previous ones not terminated yet, because
-		 a later depend({,in}out:) might need to depend on all of
-		 them.  So, if the new task's clause is depend({,in}out:),
-		 we know there is at most one other depend({,in}out:) clause
-		 in the list (out).  For non-deferred tasks we want to see
-		 all outs, so they are moved to the end of the chain,
-		 after first redundant_out entry all following entries
-		 should be redundant_out.  */
-	      if (!task->depend[i].is_in && out)
-		{
-		  if (out != last)
-		    {
-		      out->next->prev = out->prev;
-		      out->prev->next = out->next;
-		      out->next = last->next;
-		      out->prev = last;
-		      last->next = out;
-		      if (out->next)
-			out->next->prev = out;
-		    }
-		  out->redundant_out = true;
-		}
-	    }
+	  gomp_task_handle_depend (task, parent, depend);
 	  if (task->num_dependees)
 	    {
 	      gomp_mutex_unlock (&team->task_lock);
@@ -444,6 +445,128 @@ ialias (GOMP_taskgroup_end)
 #undef UTYPE
 #undef GOMP_taskloop
 
+/* Called for nowait target tasks.  */
+
+void
+gomp_create_target_task (struct gomp_device_descr *devicep,
+			 void (*fn) (void *), size_t mapnum, void **hostaddrs,
+			 size_t *sizes, unsigned short *kinds,
+			 unsigned int flags, void **depend)
+{
+  struct gomp_thread *thr = gomp_thread ();
+  struct gomp_team *team = thr->ts.team;
+
+  /* If parallel or taskgroup has been cancelled, don't start new tasks.  */
+  if (team
+      && (gomp_team_barrier_cancelled (&team->barrier)
+	  || (thr->task->taskgroup && thr->task->taskgroup->cancelled)))
+    return;
+
+  struct gomp_target_task *ttask;
+  struct gomp_task *task;
+  struct gomp_task *parent = thr->task;
+  struct gomp_taskgroup *taskgroup = parent->taskgroup;
+  bool do_wake;
+  size_t depend_size = 0;
+
+  if (depend != NULL)
+    depend_size = ((uintptr_t) depend[0]
+		   * sizeof (struct gomp_task_depend_entry));
+  task = gomp_malloc (sizeof (*task) + depend_size
+		      + sizeof (*ttask)
+		      + mapnum * (sizeof (void *) + sizeof (size_t)
+				  + sizeof (unsigned short)));
+  gomp_init_task (task, parent, gomp_icv (false));
+  task->kind = GOMP_TASK_WAITING;
+  task->in_tied_task = parent->in_tied_task;
+  task->taskgroup = taskgroup;
+  ttask = (struct gomp_target_task *) &task->depend[(uintptr_t) depend[0]];
+  ttask->devicep = devicep;
+  ttask->fn = fn;
+  ttask->mapnum = mapnum;
+  memcpy (ttask->hostaddrs, hostaddrs, mapnum * sizeof (void *));
+  ttask->sizes = (size_t *) &ttask->hostaddrs[mapnum];
+  memcpy (ttask->sizes, sizes, mapnum * sizeof (size_t));
+  ttask->kinds = (unsigned short *) &ttask->sizes[mapnum];
+  memcpy (ttask->kinds, kinds, mapnum * sizeof (unsigned short));
+  ttask->flags = flags;
+  task->fn = gomp_target_task_fn;
+  task->fn_data = ttask;
+  task->final_task = 0;
+  gomp_mutex_lock (&team->task_lock);
+  /* If parallel or taskgroup has been cancelled, don't start new tasks.  */
+  if (__builtin_expect (gomp_team_barrier_cancelled (&team->barrier)
+			|| (taskgroup && taskgroup->cancelled), 0))
+    {
+      gomp_mutex_unlock (&team->task_lock);
+      gomp_finish_task (task);
+      free (task);
+      return;
+    }
+  if (taskgroup)
+    taskgroup->num_children++;
+  if (depend_size)
+    {
+      gomp_task_handle_depend (task, parent, depend);
+      if (task->num_dependees)
+	{
+	  gomp_mutex_unlock (&team->task_lock);
+	  return;
+	}
+    }
+  if (parent->children)
+    {
+      task->next_child = parent->children;
+      task->prev_child = parent->children->prev_child;
+      task->next_child->prev_child = task;
+      task->prev_child->next_child = task;
+    }
+  else
+    {
+      task->next_child = task;
+      task->prev_child = task;
+    }
+  parent->children = task;
+  if (taskgroup)
+    {
+      /* If applicable, place task into its taskgroup.  */
+      if (taskgroup->children)
+	{
+	  task->next_taskgroup = taskgroup->children;
+	  task->prev_taskgroup = taskgroup->children->prev_taskgroup;
+	  task->next_taskgroup->prev_taskgroup = task;
+	  task->prev_taskgroup->next_taskgroup = task;
+	}
+      else
+	{
+	  task->next_taskgroup = task;
+	  task->prev_taskgroup = task;
+	}
+      taskgroup->children = task;
+    }
+  if (team->task_queue)
+    {
+      task->next_queue = team->task_queue;
+      task->prev_queue = team->task_queue->prev_queue;
+      task->next_queue->prev_queue = task;
+      task->prev_queue->next_queue = task;
+    }
+  else
+    {
+      task->next_queue = task;
+      task->prev_queue = task;
+      team->task_queue = task;
+    }
+  ++team->task_count;
+  ++team->task_queued_count;
+  gomp_team_barrier_set_task_pending (&team->barrier);
+  do_wake = team->task_running_count + !parent->in_tied_task
+	    < team->nthreads;
+  gomp_mutex_unlock (&team->task_lock);
+  if (do_wake)
+    gomp_team_barrier_wake (&team->barrier, 1);
+}
+
 static inline bool
 gomp_task_run_pre (struct gomp_task *child_task, struct gomp_task *parent,
 		   struct gomp_taskgroup *taskgroup, struct gomp_team *team)
--- libgomp/libgomp.h.jj	2015-09-03 16:33:42.000000000 +0200
+++ libgomp/libgomp.h	2015-09-08 10:18:44.164897688 +0200
@@ -374,6 +374,17 @@ struct gomp_taskgroup
   size_t num_children;
 };
 
+struct gomp_target_task
+{
+  struct gomp_device_descr *devicep;
+  void (*fn) (void *);
+  size_t mapnum;
+  size_t *sizes;
+  unsigned short *kinds;
+  unsigned int flags;
+  void *hostaddrs[];
+};
+
 /* This structure describes a "team" of threads.  These are the threads
    that are spawned by a PARALLEL constructs, as well as the work sharing
    constructs that the team encounters.  */
@@ -653,6 +664,10 @@ extern void gomp_init_task (struct gomp_
 extern void gomp_end_task (void);
 extern void gomp_barrier_handle_tasks (gomp_barrier_state_t);
 extern void gomp_task_maybe_wait_for_dependencies (void **);
+extern void gomp_create_target_task (struct gomp_device_descr *,
+				     void (*) (void *), size_t, void **,
+				     size_t *, unsigned short *, unsigned int,
+				     void **);
 
 static void inline
 gomp_finish_task (struct gomp_task *task)
@@ -673,6 +688,7 @@ extern void gomp_free_thread (void *);
 
 extern void gomp_init_targets_once (void);
 extern int gomp_get_num_devices (void);
+extern void gomp_target_task_fn (void *);
 
 typedef struct splay_tree_node_s *splay_tree_node;
 typedef struct splay_tree_s *splay_tree;
--- libgomp/testsuite/libgomp.c/target-27.c.jj	2015-09-07 17:56:17.503966807 +0200
+++ libgomp/testsuite/libgomp.c/target-27.c	2015-09-07 18:46:23.000000000 +0200
@@ -0,0 +1,67 @@
+#include <stdlib.h>
+#include <unistd.h>
+
+int
+main ()
+{
+  int x = 0, y = 0, z = 0, err;
+  int shared_mem = 0;
+  #pragma omp target map(to: shared_mem)
+  shared_mem = 1;
+  #pragma omp parallel
+  #pragma omp single
+  {
+    #pragma omp task depend(in: x)
+    {
+      usleep (5000);
+      x = 1;
+    }
+    #pragma omp task depend(in: x)
+    {
+      usleep (6000);
+      y = 2;
+    }
+    #pragma omp task depend(out: z)
+    {
+      usleep (7000);
+      z = 3;
+    }
+    #pragma omp target enter data map(to: x, y, z) depend(inout: x, z) nowait
+    #pragma omp task depend(inout: x, z)
+    {
+      x++; y++; z++;
+    }
+    #pragma omp target update to(x, y) depend(inout: x) nowait
+    #pragma omp target enter data map(always, to: z) depend(inout: z) nowait
+    #pragma omp target map (alloc: x, y, z) map (from: err) depend(inout: x, z)
+    {
+      err = x != 2 || y != 3 || z != 4;
+      x = 5; y = 6; z = 7;
+    }
+    #pragma omp task depend(in: x)
+    {
+      usleep (5000);
+      if (!shared_mem)
+	x = 1;
+    }
+    #pragma omp task depend(in: x)
+    {
+      usleep (6000);
+      if (!shared_mem)
+	y = 2;
+    }
+    #pragma omp task depend(out: z)
+    {
+      usleep (3000);
+      if (!shared_mem)
+	z = 3;
+    }
+    #pragma omp target exit data map(release: z) depend(inout: z) nowait
+    #pragma omp target exit data map(from: x, y) depend(inout: x) nowait
+    #pragma omp target exit data map(from: z) depend(inout: z) nowait
+    #pragma omp taskwait
+    if (err || x != 5 || y != 6 || z != 7)
+      abort ();
+  }
+  return 0;
+}

	Jakub

^ permalink raw reply	[flat|nested] 23+ messages in thread

* Re: [gomp4.1] depend nowait support for target {update,{enter,exit} data}
  2015-09-08  9:26 [gomp4.1] depend nowait support for target {update,{enter,exit} data} Jakub Jelinek
@ 2015-10-02 19:28 ` Ilya Verbin
  2015-10-15 14:02   ` Jakub Jelinek
  0 siblings, 1 reply; 23+ messages in thread
From: Ilya Verbin @ 2015-10-02 19:28 UTC (permalink / raw)
  To: Jakub Jelinek; +Cc: Aldy Hernandez, gcc-patches, Kirill Yukhin

Hi!

On Tue, Sep 08, 2015 at 11:20:14 +0200, Jakub Jelinek wrote:
> nowait support for #pragma omp target is not implemented yet, supposedly we
> need to mark those somehow (some flag) already in the struct gomp_task
> structure, essentially it will need either 2 or 3 callbacks
> (the current one, executed when the dependencies are resolved (it actually
> waits until some thread schedules it after that point, I think it is
> undesirable to run it with the tasking lock held), which would perform
> the gomp_map_vars and initiate the running of the region, and then some
> query routine which would poll the plugin whether the task is done or not,
> and either perform the finalization (unmap_vars) if it is done (and in any
> case return bool whether it should be polled again or not), and if the
> finalization is not done there, also another callback for the finalization.
> Also, there is the issue that if we are waiting for task that needs to be
> polled, and we don't have any further tasks to run, we shouldn't really
> attempt to sleep on some semaphore (e.g. in taskwait, end of
> taskgroup, etc.) or barrier, but rather either need to keep polling it, or
> call the query hook with some argument that it should sleep in there until
> the work is done by the offloading device.
> Also, there needs to be a way for the target nowait first callback to say
> that it is using host fallback and thus acts as a normal task, therefore
> once the task fn finishes, the task is done.

Here is my WIP patch.  target.c part is obviously incorrect, but it demonstrates
a possible libgomp <-> plugin interface for running a target task function
asynchronously and checking whether it is completed or not.
(Refactored liboffloadmic/runtime/emulator from trunk is required to run
target-tmp.c testcase.)


diff --git a/libgomp/libgomp.h b/libgomp/libgomp.h
index d798321..8e2b5aa 100644
--- a/libgomp/libgomp.h
+++ b/libgomp/libgomp.h
@@ -872,6 +872,8 @@ struct gomp_device_descr
   void *(*host2dev_func) (int, void *, const void *, size_t);
   void *(*dev2dev_func) (int, void *, const void *, size_t);
   void (*run_func) (int, void *, void *);
+  void (*async_run_func) (int, void *, void *, const void *);
+  bool (*async_is_completed_func) (int, const void *);
 
   /* Splay tree containing information about mapped memory regions.  */
   struct splay_tree_s mem_map;
diff --git a/libgomp/target.c b/libgomp/target.c
index 77bd442..31f034c 100644
--- a/libgomp/target.c
+++ b/libgomp/target.c
@@ -45,6 +45,10 @@
 #include "plugin-suffix.h"
 #endif
 
+/* FIXME: TMP */
+#include <stdio.h>
+#include <unistd.h>
+
 static void gomp_target_init (void);
 
 /* The whole initialization code for offloading plugins is only run one.  */
@@ -1227,6 +1231,44 @@ gomp_target_fallback (void (*fn) (void *), void **hostaddrs)
   *thr = old_thr;
 }
 
+/* Host fallback with firstprivate map-type handling.  */
+
+static void
+gomp_target_fallback_firstprivate (void (*fn) (void *), size_t mapnum,
+				   void **hostaddrs, size_t *sizes,
+				   unsigned short *kinds)
+{
+  size_t i, tgt_align = 0, tgt_size = 0;
+  char *tgt = NULL;
+  for (i = 0; i < mapnum; i++)
+    if ((kinds[i] & 0xff) == GOMP_MAP_FIRSTPRIVATE)
+      {
+	size_t align = (size_t) 1 << (kinds[i] >> 8);
+	if (tgt_align < align)
+	  tgt_align = align;
+	tgt_size = (tgt_size + align - 1) & ~(align - 1);
+	tgt_size += sizes[i];
+      }
+  if (tgt_align)
+    {
+      tgt = gomp_alloca (tgt_size + tgt_align - 1);
+      uintptr_t al = (uintptr_t) tgt & (tgt_align - 1);
+      if (al)
+	tgt += tgt_align - al;
+      tgt_size = 0;
+      for (i = 0; i < mapnum; i++)
+	if ((kinds[i] & 0xff) == GOMP_MAP_FIRSTPRIVATE)
+	  {
+	    size_t align = (size_t) 1 << (kinds[i] >> 8);
+	    tgt_size = (tgt_size + align - 1) & ~(align - 1);
+	    memcpy (tgt + tgt_size, hostaddrs[i], sizes[i]);
+	    hostaddrs[i] = tgt + tgt_size;
+	    tgt_size = tgt_size + sizes[i];
+	  }
+    }
+  gomp_target_fallback (fn, hostaddrs);
+}
+
 /* Helper function of GOMP_target{,_41} routines.  */
 
 static void *
@@ -1311,40 +1353,19 @@ GOMP_target_41 (int device, void (*fn) (void *), size_t mapnum,
   if (devicep == NULL
       || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
     {
-      size_t i, tgt_align = 0, tgt_size = 0;
-      char *tgt = NULL;
-      for (i = 0; i < mapnum; i++)
-	if ((kinds[i] & 0xff) == GOMP_MAP_FIRSTPRIVATE)
-	  {
-	    size_t align = (size_t) 1 << (kinds[i] >> 8);
-	    if (tgt_align < align)
-	      tgt_align = align;
-	    tgt_size = (tgt_size + align - 1) & ~(align - 1);
-	    tgt_size += sizes[i];
-	  }
-      if (tgt_align)
-	{
-	  tgt = gomp_alloca (tgt_size + tgt_align - 1);
-	  uintptr_t al = (uintptr_t) tgt & (tgt_align - 1);
-	  if (al)
-	    tgt += tgt_align - al;
-	  tgt_size = 0;
-	  for (i = 0; i < mapnum; i++)
-	    if ((kinds[i] & 0xff) == GOMP_MAP_FIRSTPRIVATE)
-	      {
-		size_t align = (size_t) 1 << (kinds[i] >> 8);
-		tgt_size = (tgt_size + align - 1) & ~(align - 1);
-		memcpy (tgt + tgt_size, hostaddrs[i], sizes[i]);
-		hostaddrs[i] = tgt + tgt_size;
-		tgt_size = tgt_size + sizes[i];
-	      }
-	}
-      gomp_target_fallback (fn, hostaddrs);
+      gomp_target_fallback_firstprivate (fn, mapnum, hostaddrs, sizes, kinds);
       return;
     }
 
   void *fn_addr = gomp_get_target_fn_addr (devicep, fn);
 
+  if (flags & GOMP_TARGET_FLAG_NOWAIT)
+    {
+      gomp_create_target_task (devicep, fn_addr, mapnum, hostaddrs, sizes,
+			       kinds, flags, depend);
+      return;
+    }
+
   struct target_mem_desc *tgt_vars
     = gomp_map_vars (devicep, mapnum, hostaddrs, NULL, sizes, kinds, true,
 		     GOMP_MAP_VARS_TARGET);
@@ -1636,34 +1657,58 @@ void
 gomp_target_task_fn (void *data)
 {
   struct gomp_target_task *ttask = (struct gomp_target_task *) data;
+  struct gomp_device_descr *devicep = ttask->devicep;
+
   if (ttask->fn != NULL)
     {
-      /* GOMP_target_41 */
+      if (devicep == NULL
+	  || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+	{
+	  /* FIXME: Save host fn addr into gomp_target_task?  */
+	  gomp_target_fallback_firstprivate (NULL, ttask->mapnum,
+					     ttask->hostaddrs, ttask->sizes,
+					     ttask->kinds);
+	  return;
+	}
+
+      struct target_mem_desc *tgt_vars
+	= gomp_map_vars (devicep, ttask->mapnum, ttask->hostaddrs, NULL,
+			 ttask->sizes, ttask->kinds, true,
+			 GOMP_MAP_VARS_TARGET);
+      devicep->async_run_func (devicep->target_id, ttask->fn,
+			       (void *) tgt_vars->tgt_start, data);
+
+      /* FIXME: TMP example of checking for completion.
+	 Alternatively the plugin can set some completion flag in ttask.  */
+      while (!devicep->async_is_completed_func (devicep->target_id, data))
+	{
+	  fprintf (stderr, "-");
+	  usleep (100000);
+	}
     }
-  else if (ttask->devicep == NULL
-	   || !(ttask->devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+  else if (devicep == NULL
+	   || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
     return;
 
   size_t i;
   if (ttask->flags & GOMP_TARGET_FLAG_UPDATE)
-    gomp_update (ttask->devicep, ttask->mapnum, ttask->hostaddrs, ttask->sizes,
+    gomp_update (devicep, ttask->mapnum, ttask->hostaddrs, ttask->sizes,
 		 ttask->kinds, true);
   else if ((ttask->flags & GOMP_TARGET_FLAG_EXIT_DATA) == 0)
     for (i = 0; i < ttask->mapnum; i++)
       if ((ttask->kinds[i] & 0xff) == GOMP_MAP_STRUCT)
 	{
-	  gomp_map_vars (ttask->devicep, ttask->sizes[i] + 1,
-			 &ttask->hostaddrs[i], NULL, &ttask->sizes[i],
-			 &ttask->kinds[i], true, GOMP_MAP_VARS_ENTER_DATA);
+	  gomp_map_vars (devicep, ttask->sizes[i] + 1, &ttask->hostaddrs[i],
+			 NULL, &ttask->sizes[i], &ttask->kinds[i], true,
+			 GOMP_MAP_VARS_ENTER_DATA);
 	  i += ttask->sizes[i];
 	}
       else
-	gomp_map_vars (ttask->devicep, 1, &ttask->hostaddrs[i], NULL,
-		       &ttask->sizes[i], &ttask->kinds[i],
-		       true, GOMP_MAP_VARS_ENTER_DATA);
+	gomp_map_vars (devicep, 1, &ttask->hostaddrs[i], NULL, &ttask->sizes[i],
+		       &ttask->kinds[i], true, GOMP_MAP_VARS_ENTER_DATA);
   else
-    gomp_exit_data (ttask->devicep, ttask->mapnum, ttask->hostaddrs,
-		    ttask->sizes, ttask->kinds);
+    gomp_exit_data (devicep, ttask->mapnum, ttask->hostaddrs, ttask->sizes,
+		    ttask->kinds);
 }
 
 void
@@ -2108,6 +2153,8 @@ gomp_load_plugin_for_device (struct gomp_device_descr *device,
   if (device->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400)
     {
       DLSYM (run);
+      DLSYM (async_run);
+      DLSYM (async_is_completed);
       DLSYM (dev2dev);
     }
   if (device->capabilities & GOMP_OFFLOAD_CAP_OPENACC_200)
diff --git a/libgomp/testsuite/libgomp.c/target-tmp.c b/libgomp/testsuite/libgomp.c/target-tmp.c
new file mode 100644
index 0000000..23a739c
--- /dev/null
+++ b/libgomp/testsuite/libgomp.c/target-tmp.c
@@ -0,0 +1,40 @@
+#include <stdio.h>
+#include <unistd.h>
+
+#pragma omp declare target
+void foo (int n)
+{
+  printf ("Start tgt %d\n", n);
+  usleep (5000000);
+  printf ("End tgt %d\n", n);
+}
+#pragma omp end declare target
+
+int x, y, z;
+
+int main ()
+{
+  #pragma omp parallel
+  #pragma omp single
+    {
+      #pragma omp task depend(out: x)
+      printf ("Host task\n");
+
+      #pragma omp target nowait depend(in: x) depend(out: y)
+      foo (1);
+
+      #pragma omp target nowait depend(in: y)
+      foo (2);
+
+      #pragma omp target nowait depend(in: y)
+      foo (3);
+
+      while (1)
+	{
+	  usleep (333333);
+	  fprintf (stderr, ".");
+	}
+    }
+
+  return 0;
+}
diff --git a/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp b/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp
index 26ac6fe..c843710 100644
--- a/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp
+++ b/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp
@@ -34,6 +34,7 @@
 #include <string.h>
 #include <utility>
 #include <vector>
+#include <set>
 #include <map>
 #include "libgomp-plugin.h"
 #include "compiler_if_host.h"
@@ -76,9 +77,15 @@ static int num_images;
    second key is number of device.  Contains a vector of pointer pairs.  */
 static ImgDevAddrMap *address_table;
 
+/* Set of asynchronously running target tasks.  */
+static std::set<const void *> *async_tasks;
+
 /* Thread-safe registration of the main image.  */
 static pthread_once_t main_image_is_registered = PTHREAD_ONCE_INIT;
 
+/* Mutex for protecting async_tasks.  */
+static pthread_mutex_t async_tasks_lock = PTHREAD_MUTEX_INITIALIZER;
+
 static VarDesc vd_host2tgt = {
   { 1, 1 },		      /* dst, src			      */
   { 1, 0 },		      /* in, out			      */
@@ -156,6 +163,8 @@ init (void)
 
 out:
   address_table = new ImgDevAddrMap;
+  async_tasks = new std::set<const void *>;
+  pthread_mutex_init (&async_tasks_lock, NULL);
   num_devices = _Offload_number_of_devices ();
 }
 
@@ -192,11 +201,27 @@ GOMP_OFFLOAD_get_num_devices (void)
 
 static void
 offload (const char *file, uint64_t line, int device, const char *name,
-	 int num_vars, VarDesc *vars, VarDesc2 *vars2)
+	 int num_vars, VarDesc *vars, VarDesc2 *vars2, const void *async_data)
 {
   OFFLOAD ofld = __offload_target_acquire1 (&device, file, line);
   if (ofld)
-    __offload_offload1 (ofld, name, 0, num_vars, vars, vars2, 0, NULL, NULL);
+    {
+      if (async_data == NULL)
+	__offload_offload1 (ofld, name, 0, num_vars, vars, vars2, 0, NULL,
+			    NULL);
+      else
+	{
+	  pthread_mutex_lock (&async_tasks_lock);
+	  async_tasks->insert (async_data);
+	  pthread_mutex_unlock (&async_tasks_lock);
+
+	  OffloadFlags flags;
+	  flags.flags = 0;
+	  flags.bits.omp_async = 1;
+	  __offload_offload3 (ofld, name, 0, num_vars, vars, NULL, 0, NULL,
+			      (const void **) async_data, 0, NULL, flags, NULL);
+	}
+    }
   else
     {
       fprintf (stderr, "%s:%d: Offload target acquire failed\n", file, line);
@@ -218,7 +243,7 @@ GOMP_OFFLOAD_init_device (int device)
   TRACE ("");
   pthread_once (&main_image_is_registered, register_main_image);
   offload (__FILE__, __LINE__, device, "__offload_target_init_proc", 0,
-	   NULL, NULL);
+	   NULL, NULL, NULL);
 }
 
 extern "C" void
@@ -240,7 +265,7 @@ get_target_table (int device, int &num_funcs, int &num_vars, void **&table)
   VarDesc2 vd1g[2] = { { "num_funcs", 0 }, { "num_vars", 0 } };
 
   offload (__FILE__, __LINE__, device, "__offload_target_table_p1", 2,
-	   vd1, vd1g);
+	   vd1, vd1g, NULL);
 
   int table_size = num_funcs + 2 * num_vars;
   if (table_size > 0)
@@ -254,7 +279,7 @@ get_target_table (int device, int &num_funcs, int &num_vars, void **&table)
       VarDesc2 vd2g = { "table", 0 };
 
       offload (__FILE__, __LINE__, device, "__offload_target_table_p2", 1,
-	       &vd2, &vd2g);
+	       &vd2, &vd2g, NULL);
     }
 }
 
@@ -401,8 +426,8 @@ GOMP_OFFLOAD_alloc (int device, size_t size)
   vd1[1].size = sizeof (void *);
   VarDesc2 vd1g[2] = { { "size", 0 }, { "tgt_ptr", 0 } };
 
-  offload (__FILE__, __LINE__, device, "__offload_target_alloc", 2, vd1, vd1g);
-
+  offload (__FILE__, __LINE__, device, "__offload_target_alloc", 2, vd1, vd1g,
+	   NULL);
   return tgt_ptr;
 }
 
@@ -416,7 +441,8 @@ GOMP_OFFLOAD_free (int device, void *tgt_ptr)
   vd1.size = sizeof (void *);
   VarDesc2 vd1g = { "tgt_ptr", 0 };
 
-  offload (__FILE__, __LINE__, device, "__offload_target_free", 1, &vd1, &vd1g);
+  offload (__FILE__, __LINE__, device, "__offload_target_free", 1, &vd1, &vd1g,
+	   NULL);
 }
 
 extern "C" void *
@@ -435,7 +461,7 @@ GOMP_OFFLOAD_host2dev (int device, void *tgt_ptr, const void *host_ptr,
   VarDesc2 vd1g[2] = { { "tgt_ptr", 0 }, { "size", 0 } };
 
   offload (__FILE__, __LINE__, device, "__offload_target_host2tgt_p1", 2,
-	   vd1, vd1g);
+	   vd1, vd1g, NULL);
 
   VarDesc vd2 = vd_host2tgt;
   vd2.ptr = (void *) host_ptr;
@@ -443,7 +469,7 @@ GOMP_OFFLOAD_host2dev (int device, void *tgt_ptr, const void *host_ptr,
   VarDesc2 vd2g = { "var", 0 };
 
   offload (__FILE__, __LINE__, device, "__offload_target_host2tgt_p2", 1,
-	   &vd2, &vd2g);
+	   &vd2, &vd2g, NULL);
 
   return tgt_ptr;
 }
@@ -464,7 +490,7 @@ GOMP_OFFLOAD_dev2host (int device, void *host_ptr, const void *tgt_ptr,
   VarDesc2 vd1g[2] = { { "tgt_ptr", 0 }, { "size", 0 } };
 
   offload (__FILE__, __LINE__, device, "__offload_target_tgt2host_p1", 2,
-	   vd1, vd1g);
+	   vd1, vd1g, NULL);
 
   VarDesc vd2 = vd_tgt2host;
   vd2.ptr = (void *) host_ptr;
@@ -472,7 +498,7 @@ GOMP_OFFLOAD_dev2host (int device, void *host_ptr, const void *tgt_ptr,
   VarDesc2 vd2g = { "var", 0 };
 
   offload (__FILE__, __LINE__, device, "__offload_target_tgt2host_p2", 1,
-	   &vd2, &vd2g);
+	   &vd2, &vd2g, NULL);
 
   return host_ptr;
 }
@@ -495,22 +521,56 @@ GOMP_OFFLOAD_dev2dev (int device, void *dst_ptr, const void *src_ptr,
   VarDesc2 vd1g[3] = { { "dst_ptr", 0 }, { "src_ptr", 0 }, { "size", 0 } };
 
   offload (__FILE__, __LINE__, device, "__offload_target_tgt2tgt", 3, vd1,
-	   vd1g);
+	   vd1g, NULL);
 
   return dst_ptr;
 }
 
 extern "C" void
+GOMP_OFFLOAD_async_run (int device, void *tgt_fn, void *tgt_vars,
+			const void *async_data)
+{
+  TRACE ("(device = %d, tgt_fn = %p, tgt_vars = %p, async_data = %p)", device,
+	 tgt_fn, tgt_vars, async_data);
+
+  VarDesc vd[2] = { vd_host2tgt, vd_host2tgt };
+  vd[0].ptr = &tgt_fn;
+  vd[0].size = sizeof (void *);
+  vd[1].ptr = &tgt_vars;
+  vd[1].size = sizeof (void *);
+
+  offload (__FILE__, __LINE__, device, "__offload_target_run", 2, vd, NULL,
+	   async_data);
+}
+
+extern "C" void
 GOMP_OFFLOAD_run (int device, void *tgt_fn, void *tgt_vars)
 {
-  TRACE ("(tgt_fn = %p, tgt_vars = %p)", tgt_fn, tgt_vars);
+  TRACE ("(device = %d, tgt_fn = %p, tgt_vars = %p)", device, tgt_fn, tgt_vars);
 
-  VarDesc vd1[2] = { vd_host2tgt, vd_host2tgt };
-  vd1[0].ptr = &tgt_fn;
-  vd1[0].size = sizeof (void *);
-  vd1[1].ptr = &tgt_vars;
-  vd1[1].size = sizeof (void *);
-  VarDesc2 vd1g[2] = { { "tgt_fn", 0 }, { "tgt_vars", 0 } };
+  GOMP_OFFLOAD_async_run (device, tgt_fn, tgt_vars, NULL);
+}
+
+extern "C" bool
+GOMP_OFFLOAD_async_is_completed (int device, const void *async_data)
+{
+  TRACE ("(device = %d, async_data = %p)", device, async_data);
+
+  bool res;
+  pthread_mutex_lock (&async_tasks_lock);
+  res = async_tasks->count (async_data) == 0;
+  pthread_mutex_unlock (&async_tasks_lock);
+  return res;
+}
+
+/* Called by liboffloadmic when asynchronous function is completed.  */
+
+extern "C" void
+__gomp_offload_intelmic_async_completed (const void *async_data)
+{
+  TRACE ("(async_data = %p)", async_data);
 
-  offload (__FILE__, __LINE__, device, "__offload_target_run", 2, vd1, vd1g);
+  pthread_mutex_lock (&async_tasks_lock);
+  async_tasks->erase (async_data);
+  pthread_mutex_unlock (&async_tasks_lock);
 }
diff --git a/liboffloadmic/runtime/offload_host.cpp b/liboffloadmic/runtime/offload_host.cpp
index 08f626f..8cee12c 100644
--- a/liboffloadmic/runtime/offload_host.cpp
+++ b/liboffloadmic/runtime/offload_host.cpp
@@ -64,6 +64,9 @@ static void __offload_fini_library(void);
 #define GET_OFFLOAD_NUMBER(timer_data) \
     timer_data? timer_data->offload_number : 0
 
+extern "C" void
+__gomp_offload_intelmic_async_completed (const void *);
+
 extern "C" {
 #ifdef TARGET_WINNT
 // Windows does not support imports from libraries without actually
@@ -2507,7 +2510,7 @@ extern "C" {
         const void *info
     )
     {
-	/* TODO: Call callback function, pass info.  */
+	__gomp_offload_intelmic_async_completed (info);
     }
 }
 

  -- Ilya

^ permalink raw reply	[flat|nested] 23+ messages in thread

* Re: [gomp4.1] depend nowait support for target {update,{enter,exit} data}
  2015-10-02 19:28 ` Ilya Verbin
@ 2015-10-15 14:02   ` Jakub Jelinek
  2015-10-15 16:18     ` Alexander Monakov
                       ` (3 more replies)
  0 siblings, 4 replies; 23+ messages in thread
From: Jakub Jelinek @ 2015-10-15 14:02 UTC (permalink / raw)
  To: Ilya Verbin
  Cc: Aldy Hernandez, gcc-patches, Kirill Yukhin, Thomas Schwinge,
	Alexander Monakov, Martin Jambor

Hi!

CCing various people, because I'd like to have something that won't work on
XeonPhi only.

On Fri, Oct 02, 2015 at 10:28:01PM +0300, Ilya Verbin wrote:
> On Tue, Sep 08, 2015 at 11:20:14 +0200, Jakub Jelinek wrote:
> > nowait support for #pragma omp target is not implemented yet, supposedly we
> > need to mark those somehow (some flag) already in the struct gomp_task
> > structure, essentially it will need either 2 or 3 callbacks
> > (the current one, executed when the dependencies are resolved (it actually
> > waits until some thread schedules it after that point, I think it is
> > undesirable to run it with the tasking lock held), which would perform
> > the gomp_map_vars and initiate the running of the region, and then some
> > query routine which would poll the plugin whether the task is done or not,
> > and either perform the finalization (unmap_vars) if it is done (and in any
> > case return bool whether it should be polled again or not), and if the
> > finalization is not done there, also another callback for the finalization.
> > Also, there is the issue that if we are waiting for task that needs to be
> > polled, and we don't have any further tasks to run, we shouldn't really
> > attempt to sleep on some semaphore (e.g. in taskwait, end of
> > taskgroup, etc.) or barrier, but rather either need to keep polling it, or
> > call the query hook with some argument that it should sleep in there until
> > the work is done by the offloading device.
> > Also, there needs to be a way for the target nowait first callback to say
> > that it is using host fallback and thus acts as a normal task, therefore
> > once the task fn finishes, the task is done.
> 
> Here is my WIP patch.  target.c part is obviously incorrect, but it demonstrates
> a possible libgomp <-> plugin interface for running a target task function
> asynchronously and checking whether it is completed or not.
> (Refactored liboffloadmic/runtime/emulator from trunk is required to run
> target-tmp.c testcase.)

The difficulty is designing something that will work (if possible fast) on the
various devices we want to eventually support (at least XeonPhi, XeonPhi emul,
PTX/Cuda and HSA), ideally without too much busy waiting.

The OpenMP 4.5 spec says that there is a special "target task" on the host
side around the target region, and that the "target task" is mergeable and
if nowait is not specified is included (otherwise it may be), and that the
mapping operations (which include target device memory allocation,
refcount management and mapping data structure updates as well as the
memory copying to target device) happens only after the (optional) dependencies
are satisfied.  After the memory mapping operations are done, the offloading
kernel starts, and when it finishes, the unmapping operations are performed
(which includes memory copying from the target device, refcount management
and mapping data structure updates, and finally memory deallocation).

Right now on the OpenMP side everything is synchronous, e.g. target
enter/exit data and update are asynchronous only in that the mapping or
unmapping operation is scheduled as a task, but the whole mapping or
unmapping operations including all the above mentioned subparts are
performed while holding the particular device's lock.

To make that more asynchronous, e.g. for Cuda we might want to use Cuda
(non-default) streams, and perform the allocation, refcount management and
mapping data structure updates, and perform the data copying to device
already as part of the stream.  Except that it means that if another target
mapping/unmapping operation is enqueued at that point and it refers to any
of the affected objects, it could acquire the device lock, yet the data
copying would be still in flux.  Dunno here if it would be e.g. acceptable
to add some flags to the mapping data structures, this memory range has
either pending data transfers or has enqueued data transfers that depend on
whether the refcount will become zero or not.  When mapping if we'd want to
touch any of the regions marked with such in_flux flag, we'd need to wait
until all of the other stream's operation finish and the unmapping
operations are performed (and the device lock released again) before
continuing.  That way we could get good performance if either concurrent
async regions touch different variables, or target data or non-async target
enter data or exit data has been put around the mappings, so the streams can
be independent, but worst case we'd make them non-concurrent.

Anyway, let's put the asynchronous memory data transfers (which also implies
the ability to enqueue multiple different target regions into a stream for
the device to operate on independently from the host) on the side for now
and just discuss what we want for the actual async execution and for now
keep a device lock around all the mapping or unmapping operations.

If the "target task" has unresolved dependencies, then it will use existing
task.c waiting code first (if the above is resolved somehow, there could be
exceptions of "target task" depending on another "target task").
When the dependencies are resolved, we can run the gomp_target_task_fn
callback (but not with the team's tasking lock held), which can perform
the gomp_map_vars call and start the async execution.  For host fallback,
that is all we do, the task is at this point a normal task.
For offloading task, we now want the host to continue scheduling other tasks
if there are any, which means (not currently implemented on the task.c side)
we want to move the task somewhere that we don't consider it finished, and
that we'll need to schedule it again at some point to perform the unmapping
(perhaps requeue it again in a WAITING or some other state).

Right now, the tasking code would in that case try to schedule another task,
and if there are none or none that are runnable among the tasks of interest,
it can go to sleep and expect to be awaken when some task it is waiting for
is awaken.

And the main question is how to find out on the various devices whether
the async execution has completed already.

From what I can see in the liboffloadmic plugin, you have an extra host
thread that can run a callback function on the host.  Such a callback could
say tweak the state of the "target task", could take the team's tasking
lock, and even awake sleepers, maybe even take the device lock and perform
unmapping of vars?  The function would need to be in task.c
so that it can access everything defined in there.  Or the callback could
just change something in the "target task" state and let the tasking poll
for the change.

Looking at Cuda, for async target region kernels we'd probably use
a non-default stream and enqueue the async kernel in there.  I see
we can e.g. cudaEventRecord into the stream and then either cudaEventQuery
to busy poll the event, or cudaEventSynchronize to block until the event
occurs, plus there is cudaStreamWaitEvent that perhaps might be even used to
resolve the above mentioned mapping/unmapping async issues for Cuda
- like add an event after the mapping operations that the other target tasks
could wait for if they see any in_flux stuff, and wait for an event etc.
I don't see a possibility to have something like a callback on stream
completion though, so it has to be handled with polling.  If that is true,
it means the tasking code can't go to sleep if there are any pending target
tasks (at least for devices that can't do a callback) it wants to wait for,
it would need to call in a loop the poll methods of the plugins that it
wants to wait for (unless there are no host tasks left and only a single
device is involved, then it could call a blocking method).

For HSA I have no idea.

Now, for the polling case, the question is how the polling is expensive,
whether it can be performed with the team's lock held or not.  If XeonPhi
doesn't do the full host callback, but polling, it could just read some
memory from target_task struct and thus be fast enough to run it with the
lock held.  How expensive is cudaEventQuery?

> diff --git a/libgomp/target.c b/libgomp/target.c
> index 77bd442..31f034c 100644
> --- a/libgomp/target.c
> +++ b/libgomp/target.c
> @@ -45,6 +45,10 @@
>  #include "plugin-suffix.h"
>  #endif
>  
> +/* FIXME: TMP */
> +#include <stdio.h>
> +#include <unistd.h>

I hope you mean to remove this later on.

> @@ -1227,6 +1231,44 @@ gomp_target_fallback (void (*fn) (void *), void **hostaddrs)
>    *thr = old_thr;
>  }
>  
> +/* Host fallback with firstprivate map-type handling.  */
> +
> +static void
> +gomp_target_fallback_firstprivate (void (*fn) (void *), size_t mapnum,
> +				   void **hostaddrs, size_t *sizes,
> +				   unsigned short *kinds)
> +{
> +  size_t i, tgt_align = 0, tgt_size = 0;
> +  char *tgt = NULL;
> +  for (i = 0; i < mapnum; i++)
> +    if ((kinds[i] & 0xff) == GOMP_MAP_FIRSTPRIVATE)
> +      {
> +	size_t align = (size_t) 1 << (kinds[i] >> 8);
> +	if (tgt_align < align)
> +	  tgt_align = align;
> +	tgt_size = (tgt_size + align - 1) & ~(align - 1);
> +	tgt_size += sizes[i];
> +      }
> +  if (tgt_align)
> +    {
> +      tgt = gomp_alloca (tgt_size + tgt_align - 1);
> +      uintptr_t al = (uintptr_t) tgt & (tgt_align - 1);
> +      if (al)
> +	tgt += tgt_align - al;
> +      tgt_size = 0;
> +      for (i = 0; i < mapnum; i++)
> +	if ((kinds[i] & 0xff) == GOMP_MAP_FIRSTPRIVATE)
> +	  {
> +	    size_t align = (size_t) 1 << (kinds[i] >> 8);
> +	    tgt_size = (tgt_size + align - 1) & ~(align - 1);
> +	    memcpy (tgt + tgt_size, hostaddrs[i], sizes[i]);
> +	    hostaddrs[i] = tgt + tgt_size;
> +	    tgt_size = tgt_size + sizes[i];
> +	  }
> +    }
> +  gomp_target_fallback (fn, hostaddrs);
> +}

This is ok.

> +
>  /* Helper function of GOMP_target{,_41} routines.  */
>  
>  static void *
> @@ -1311,40 +1353,19 @@ GOMP_target_41 (int device, void (*fn) (void *), size_t mapnum,
>    if (devicep == NULL
>        || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
>      {
> -      size_t i, tgt_align = 0, tgt_size = 0;
> -      char *tgt = NULL;
> -      for (i = 0; i < mapnum; i++)
> -	if ((kinds[i] & 0xff) == GOMP_MAP_FIRSTPRIVATE)
> -	  {
> -	    size_t align = (size_t) 1 << (kinds[i] >> 8);
> -	    if (tgt_align < align)
> -	      tgt_align = align;
> -	    tgt_size = (tgt_size + align - 1) & ~(align - 1);
> -	    tgt_size += sizes[i];
> -	  }
> -      if (tgt_align)
> -	{
> -	  tgt = gomp_alloca (tgt_size + tgt_align - 1);
> -	  uintptr_t al = (uintptr_t) tgt & (tgt_align - 1);
> -	  if (al)
> -	    tgt += tgt_align - al;
> -	  tgt_size = 0;
> -	  for (i = 0; i < mapnum; i++)
> -	    if ((kinds[i] & 0xff) == GOMP_MAP_FIRSTPRIVATE)
> -	      {
> -		size_t align = (size_t) 1 << (kinds[i] >> 8);
> -		tgt_size = (tgt_size + align - 1) & ~(align - 1);
> -		memcpy (tgt + tgt_size, hostaddrs[i], sizes[i]);
> -		hostaddrs[i] = tgt + tgt_size;
> -		tgt_size = tgt_size + sizes[i];
> -	      }
> -	}
> -      gomp_target_fallback (fn, hostaddrs);
> +      gomp_target_fallback_firstprivate (fn, mapnum, hostaddrs, sizes, kinds);
>        return;
>      }

This too.

>    void *fn_addr = gomp_get_target_fn_addr (devicep, fn);
>  
> +  if (flags & GOMP_TARGET_FLAG_NOWAIT)
> +    {
> +      gomp_create_target_task (devicep, fn_addr, mapnum, hostaddrs, sizes,
> +			       kinds, flags, depend);
> +      return;
> +    }

But this is not ok.  You need to do this far earlier, already before the
if (depend != NULL) code in GOMP_target_41.  And, I think you should just
not pass fn_addr, but fn itself.

> @@ -1636,34 +1657,58 @@ void
>  gomp_target_task_fn (void *data)
>  {
>    struct gomp_target_task *ttask = (struct gomp_target_task *) data;
> +  struct gomp_device_descr *devicep = ttask->devicep;
> +
>    if (ttask->fn != NULL)
>      {
> -      /* GOMP_target_41 */
> +      if (devicep == NULL
> +	  || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
> +	{
> +	  /* FIXME: Save host fn addr into gomp_target_task?  */
> +	  gomp_target_fallback_firstprivate (NULL, ttask->mapnum,

If you pass above fn instead of fn_addr, ttask->fn is what you want
to pass to gomp_target_fallback_firstprivate here and remove the FIXME.

> +					     ttask->hostaddrs, ttask->sizes,
> +					     ttask->kinds);
> +	  return;
> +	}
> +
> +      struct target_mem_desc *tgt_vars
> +	= gomp_map_vars (devicep, ttask->mapnum, ttask->hostaddrs, NULL,
> +			 ttask->sizes, ttask->kinds, true,
> +			 GOMP_MAP_VARS_TARGET);
> +      devicep->async_run_func (devicep->target_id, ttask->fn,
> +			       (void *) tgt_vars->tgt_start, data);

You need to void *fn_addr = gomp_get_target_fn_addr (devicep, ttask->fn);
first obviously, and pass fn_addr.

> +
> +      /* FIXME: TMP example of checking for completion.
> +	 Alternatively the plugin can set some completion flag in ttask.  */
> +      while (!devicep->async_is_completed_func (devicep->target_id, data))
> +	{
> +	  fprintf (stderr, "-");
> +	  usleep (100000);
> +	}

This obviously doesn't belong here.

>    if (device->capabilities & GOMP_OFFLOAD_CAP_OPENACC_200)
> diff --git a/libgomp/testsuite/libgomp.c/target-tmp.c b/libgomp/testsuite/libgomp.c/target-tmp.c
> new file mode 100644
> index 0000000..23a739c
> --- /dev/null
> +++ b/libgomp/testsuite/libgomp.c/target-tmp.c
> @@ -0,0 +1,40 @@
> +#include <stdio.h>
> +#include <unistd.h>
> +
> +#pragma omp declare target
> +void foo (int n)
> +{
> +  printf ("Start tgt %d\n", n);
> +  usleep (5000000);

5s is too long.  Not to mention that not sure if PTX can do printf
and especially usleep.

> diff --git a/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp b/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp
> index 26ac6fe..c843710 100644
> --- a/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp
> +++ b/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp
...
> +/* Set of asynchronously running target tasks.  */
> +static std::set<const void *> *async_tasks;
> +
>  /* Thread-safe registration of the main image.  */
>  static pthread_once_t main_image_is_registered = PTHREAD_ONCE_INIT;
>  
> +/* Mutex for protecting async_tasks.  */
> +static pthread_mutex_t async_tasks_lock = PTHREAD_MUTEX_INITIALIZER;
> +
>  static VarDesc vd_host2tgt = {
>    { 1, 1 },		      /* dst, src			      */
>    { 1, 0 },		      /* in, out			      */
> @@ -156,6 +163,8 @@ init (void)
>  
>  out:
>    address_table = new ImgDevAddrMap;
> +  async_tasks = new std::set<const void *>;
> +  pthread_mutex_init (&async_tasks_lock, NULL);

PTHREAD_MUTEX_INITIALIZER should already initialize the lock.
But, do you really need async_tasks and the lock?  Better store
something into some plugin's owned field in target_task struct and
let the plugin callback be passed address of that field rather than the
whole target_task?

> diff --git a/liboffloadmic/runtime/offload_host.cpp b/liboffloadmic/runtime/offload_host.cpp
> index 08f626f..8cee12c 100644
> --- a/liboffloadmic/runtime/offload_host.cpp
> +++ b/liboffloadmic/runtime/offload_host.cpp
> @@ -64,6 +64,9 @@ static void __offload_fini_library(void);
>  #define GET_OFFLOAD_NUMBER(timer_data) \
>      timer_data? timer_data->offload_number : 0
>  
> +extern "C" void
> +__gomp_offload_intelmic_async_completed (const void *);
> +
>  extern "C" {
>  #ifdef TARGET_WINNT
>  // Windows does not support imports from libraries without actually
> @@ -2507,7 +2510,7 @@ extern "C" {
>          const void *info
>      )
>      {
> -	/* TODO: Call callback function, pass info.  */
> +	__gomp_offload_intelmic_async_completed (info);
>      }
>  }

Is this for the emul only, or KNL only, or both?
In any case, not sure how it works, this is in liboffloadmic.so and
the function defined in the plugin?

	Jakub

^ permalink raw reply	[flat|nested] 23+ messages in thread

* Re: [gomp4.1] depend nowait support for target {update,{enter,exit} data}
  2015-10-15 14:02   ` Jakub Jelinek
@ 2015-10-15 16:18     ` Alexander Monakov
  2015-10-15 17:18       ` Jakub Jelinek
  2015-10-15 16:42     ` Ilya Verbin
                       ` (2 subsequent siblings)
  3 siblings, 1 reply; 23+ messages in thread
From: Alexander Monakov @ 2015-10-15 16:18 UTC (permalink / raw)
  To: Jakub Jelinek
  Cc: Ilya Verbin, Aldy Hernandez, gcc-patches, Kirill Yukhin,
	Thomas Schwinge, Martin Jambor

On Thu, 15 Oct 2015, Jakub Jelinek wrote:
> Looking at Cuda, for async target region kernels we'd probably use
> a non-default stream and enqueue the async kernel in there.  I see
> we can e.g. cudaEventRecord into the stream and then either cudaEventQuery
> to busy poll the event, or cudaEventSynchronize to block until the event
> occurs, plus there is cudaStreamWaitEvent that perhaps might be even used to
> resolve the above mentioned mapping/unmapping async issues for Cuda
> - like add an event after the mapping operations that the other target tasks
> could wait for if they see any in_flux stuff, and wait for an event etc.
> I don't see a possibility to have something like a callback on stream
> completion though, so it has to be handled with polling.

Not sure why you say so.  There's cu[da]StreamAddCallback, which exists
exactly for registering completion callback, but there are restrictions:

  - this functionality doesn't currently work through CUDA MPS ("multi-process
    server", for funneling CUDA calls from different processes through a
    single "server" process, avoiding context-switch overhead on the device,
    sometimes used for CUDA-with-MPI applications);

  - it is explicitely forbidden to invoke CUDA API calls from the callback;
    perhaps understandable, as the callback may be running in a signal-handler
    context (unlikely), or, more plausibly, in a different thread than the one
    that registered the callback.

Ideally we'd queue all accelerator work up front via
EventRecord/StreamWaitEvent, and not rely on callbacks.  If host-side work
must be done on completion, we could spawn a helper thread waiting on
cudaEventSynchronize.

> > --- /dev/null
> > +++ b/libgomp/testsuite/libgomp.c/target-tmp.c
> > @@ -0,0 +1,40 @@
> > +#include <stdio.h>
> > +#include <unistd.h>
> > +
> > +#pragma omp declare target
> > +void foo (int n)
> > +{
> > +  printf ("Start tgt %d\n", n);
> > +  usleep (5000000);
> 
> 5s is too long.  Not to mention that not sure if PTX can do printf
> and especially usleep.

printf is available, usleep is not (but presumably use of usleep needs to be
revised anyway)

Alexander

^ permalink raw reply	[flat|nested] 23+ messages in thread

* Re: [gomp4.1] depend nowait support for target {update,{enter,exit} data}
  2015-10-15 14:02   ` Jakub Jelinek
  2015-10-15 16:18     ` Alexander Monakov
@ 2015-10-15 16:42     ` Ilya Verbin
  2015-10-16 11:50     ` Martin Jambor
  2015-10-19 19:55     ` Ilya Verbin
  3 siblings, 0 replies; 23+ messages in thread
From: Ilya Verbin @ 2015-10-15 16:42 UTC (permalink / raw)
  To: Jakub Jelinek
  Cc: Aldy Hernandez, gcc-patches, Kirill Yukhin, Thomas Schwinge,
	Alexander Monakov, Martin Jambor

On Thu, Oct 15, 2015 at 16:01:56 +0200, Jakub Jelinek wrote:
> On Fri, Oct 02, 2015 at 10:28:01PM +0300, Ilya Verbin wrote:
> > Here is my WIP patch.  target.c part is obviously incorrect, but it demonstrates
> > a possible libgomp <-> plugin interface for running a target task function
> > asynchronously and checking whether it is completed or not.
> > (Refactored liboffloadmic/runtime/emulator from trunk is required to run
> > target-tmp.c testcase.)
> 
> > diff --git a/libgomp/target.c b/libgomp/target.c
> > index 77bd442..31f034c 100644
> > --- a/libgomp/target.c
> > +++ b/libgomp/target.c
> > @@ -45,6 +45,10 @@
> >  #include "plugin-suffix.h"
> >  #endif
> >  
> > +/* FIXME: TMP */
> > +#include <stdio.h>
> > +#include <unistd.h>
> 
> I hope you mean to remove this later on.

Sure, this is just a prototype, not for committing.


> > @@ -1227,6 +1231,44 @@ gomp_target_fallback (void (*fn) (void *), void **hostaddrs)
> >    *thr = old_thr;
> >  }
> >  
> > +/* Host fallback with firstprivate map-type handling.  */
> > +
> > +static void
> > +gomp_target_fallback_firstprivate (void (*fn) (void *), size_t mapnum,
> > +				   void **hostaddrs, size_t *sizes,
> > +				   unsigned short *kinds)
> > +{
> > +  size_t i, tgt_align = 0, tgt_size = 0;
> > +  char *tgt = NULL;
> > +  for (i = 0; i < mapnum; i++)
> > +    if ((kinds[i] & 0xff) == GOMP_MAP_FIRSTPRIVATE)
> > +      {
> > +	size_t align = (size_t) 1 << (kinds[i] >> 8);
> > +	if (tgt_align < align)
> > +	  tgt_align = align;
> > +	tgt_size = (tgt_size + align - 1) & ~(align - 1);
> > +	tgt_size += sizes[i];
> > +      }
> > +  if (tgt_align)
> > +    {
> > +      tgt = gomp_alloca (tgt_size + tgt_align - 1);
> > +      uintptr_t al = (uintptr_t) tgt & (tgt_align - 1);
> > +      if (al)
> > +	tgt += tgt_align - al;
> > +      tgt_size = 0;
> > +      for (i = 0; i < mapnum; i++)
> > +	if ((kinds[i] & 0xff) == GOMP_MAP_FIRSTPRIVATE)
> > +	  {
> > +	    size_t align = (size_t) 1 << (kinds[i] >> 8);
> > +	    tgt_size = (tgt_size + align - 1) & ~(align - 1);
> > +	    memcpy (tgt + tgt_size, hostaddrs[i], sizes[i]);
> > +	    hostaddrs[i] = tgt + tgt_size;
> > +	    tgt_size = tgt_size + sizes[i];
> > +	  }
> > +    }
> > +  gomp_target_fallback (fn, hostaddrs);
> > +}
> 
> This is ok.
> 
> >  /* Helper function of GOMP_target{,_41} routines.  */
> >  
> >  static void *
> > @@ -1311,40 +1353,19 @@ GOMP_target_41 (int device, void (*fn) (void *), size_t mapnum,
> >    if (devicep == NULL
> >        || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
> >      {
> > -      size_t i, tgt_align = 0, tgt_size = 0;
> > -      char *tgt = NULL;
> > -      for (i = 0; i < mapnum; i++)
> > -	if ((kinds[i] & 0xff) == GOMP_MAP_FIRSTPRIVATE)
> > -	  {
> > -	    size_t align = (size_t) 1 << (kinds[i] >> 8);
> > -	    if (tgt_align < align)
> > -	      tgt_align = align;
> > -	    tgt_size = (tgt_size + align - 1) & ~(align - 1);
> > -	    tgt_size += sizes[i];
> > -	  }
> > -      if (tgt_align)
> > -	{
> > -	  tgt = gomp_alloca (tgt_size + tgt_align - 1);
> > -	  uintptr_t al = (uintptr_t) tgt & (tgt_align - 1);
> > -	  if (al)
> > -	    tgt += tgt_align - al;
> > -	  tgt_size = 0;
> > -	  for (i = 0; i < mapnum; i++)
> > -	    if ((kinds[i] & 0xff) == GOMP_MAP_FIRSTPRIVATE)
> > -	      {
> > -		size_t align = (size_t) 1 << (kinds[i] >> 8);
> > -		tgt_size = (tgt_size + align - 1) & ~(align - 1);
> > -		memcpy (tgt + tgt_size, hostaddrs[i], sizes[i]);
> > -		hostaddrs[i] = tgt + tgt_size;
> > -		tgt_size = tgt_size + sizes[i];
> > -	      }
> > -	}
> > -      gomp_target_fallback (fn, hostaddrs);
> > +      gomp_target_fallback_firstprivate (fn, mapnum, hostaddrs, sizes, kinds);
> >        return;
> >      }
> 
> This too.

I will commit this small part to gomp-4_5-branch separately.


> > diff --git a/libgomp/testsuite/libgomp.c/target-tmp.c b/libgomp/testsuite/libgomp.c/target-tmp.c
> > new file mode 100644
> > index 0000000..23a739c
> > --- /dev/null
> > +++ b/libgomp/testsuite/libgomp.c/target-tmp.c
> > @@ -0,0 +1,40 @@
> > +#include <stdio.h>
> > +#include <unistd.h>
> > +
> > +#pragma omp declare target
> > +void foo (int n)
> > +{
> > +  printf ("Start tgt %d\n", n);
> > +  usleep (5000000);
> 
> 5s is too long.  Not to mention that not sure if PTX can do printf
> and especially usleep.

This testcase is also for demonstration only.


> > diff --git a/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp b/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp
> > index 26ac6fe..c843710 100644
> > --- a/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp
> > +++ b/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp
> ...
> > +/* Set of asynchronously running target tasks.  */
> > +static std::set<const void *> *async_tasks;
> > +
> >  /* Thread-safe registration of the main image.  */
> >  static pthread_once_t main_image_is_registered = PTHREAD_ONCE_INIT;
> >  
> > +/* Mutex for protecting async_tasks.  */
> > +static pthread_mutex_t async_tasks_lock = PTHREAD_MUTEX_INITIALIZER;
> > +
> >  static VarDesc vd_host2tgt = {
> >    { 1, 1 },		      /* dst, src			      */
> >    { 1, 0 },		      /* in, out			      */
> > @@ -156,6 +163,8 @@ init (void)
> >  
> >  out:
> >    address_table = new ImgDevAddrMap;
> > +  async_tasks = new std::set<const void *>;
> > +  pthread_mutex_init (&async_tasks_lock, NULL);
> 
> PTHREAD_MUTEX_INITIALIZER should already initialize the lock.
> But, do you really need async_tasks and the lock?  Better store
> something into some plugin's owned field in target_task struct and
> let the plugin callback be passed address of that field rather than the
> whole target_task?

OK, that should work.


> > diff --git a/liboffloadmic/runtime/offload_host.cpp b/liboffloadmic/runtime/offload_host.cpp
> > index 08f626f..8cee12c 100644
> > --- a/liboffloadmic/runtime/offload_host.cpp
> > +++ b/liboffloadmic/runtime/offload_host.cpp
> > @@ -64,6 +64,9 @@ static void __offload_fini_library(void);
> >  #define GET_OFFLOAD_NUMBER(timer_data) \
> >      timer_data? timer_data->offload_number : 0
> >  
> > +extern "C" void
> > +__gomp_offload_intelmic_async_completed (const void *);
> > +
> >  extern "C" {
> >  #ifdef TARGET_WINNT
> >  // Windows does not support imports from libraries without actually
> > @@ -2507,7 +2510,7 @@ extern "C" {
> >          const void *info
> >      )
> >      {
> > -	/* TODO: Call callback function, pass info.  */
> > +	__gomp_offload_intelmic_async_completed (info);
> >      }
> >  }
> 
> Is this for the emul only, or KNL only, or both?

This is for both.  liboffloadmic doesn't know whether target process is running
on real KNL or on host using emul, only underlying libcoi matters.

> In any case, not sure how it works, this is in liboffloadmic.so and
> the function defined in the plugin?

Yes, this is in liboffloadmic_host.so, and the function is defined in the
plugin.  Both are loaded into the host process.
We can replace it by a callback directly into libgomp, if needed.

  -- Ilya

^ permalink raw reply	[flat|nested] 23+ messages in thread

* Re: [gomp4.1] depend nowait support for target {update,{enter,exit} data}
  2015-10-15 16:18     ` Alexander Monakov
@ 2015-10-15 17:18       ` Jakub Jelinek
  2015-10-15 18:11         ` Alexander Monakov
  0 siblings, 1 reply; 23+ messages in thread
From: Jakub Jelinek @ 2015-10-15 17:18 UTC (permalink / raw)
  To: Alexander Monakov
  Cc: Ilya Verbin, Aldy Hernandez, gcc-patches, Kirill Yukhin,
	Thomas Schwinge, Martin Jambor

On Thu, Oct 15, 2015 at 07:18:53PM +0300, Alexander Monakov wrote:
> On Thu, 15 Oct 2015, Jakub Jelinek wrote:
> > Looking at Cuda, for async target region kernels we'd probably use
> > a non-default stream and enqueue the async kernel in there.  I see
> > we can e.g. cudaEventRecord into the stream and then either cudaEventQuery
> > to busy poll the event, or cudaEventSynchronize to block until the event
> > occurs, plus there is cudaStreamWaitEvent that perhaps might be even used to
> > resolve the above mentioned mapping/unmapping async issues for Cuda
> > - like add an event after the mapping operations that the other target tasks
> > could wait for if they see any in_flux stuff, and wait for an event etc.
> > I don't see a possibility to have something like a callback on stream
> > completion though, so it has to be handled with polling.
> 
> Not sure why you say so.  There's cu[da]StreamAddCallback, which exists
> exactly for registering completion callback, but there are restrictions:

Ah, thanks.

>   - this functionality doesn't currently work through CUDA MPS ("multi-process
>     server", for funneling CUDA calls from different processes through a
>     single "server" process, avoiding context-switch overhead on the device,
>     sometimes used for CUDA-with-MPI applications);

That shouldn't be an issue for the OpenMP 4.5 / PTX offloading, right?

>   - it is explicitely forbidden to invoke CUDA API calls from the callback;
>     perhaps understandable, as the callback may be running in a signal-handler
>     context (unlikely), or, more plausibly, in a different thread than the one
>     that registered the callback.

So, is it run from async signal handlers, or just could be?
If all we need to achieve is just change some word in target_task struct,
then it should be enough to just asynchronously memcpy there the value,
or e.g. use the events.  If we need to also gomp_sem_post, then for
config/linux/ that is also something that can be done from async signal
contexts, but not for other OSes (but perhaps we could just not go to sleep
on those OSes if there are pending offloading tasks).

> Ideally we'd queue all accelerator work up front via
> EventRecord/StreamWaitEvent, and not rely on callbacks.

> If host-side work
> must be done on completion, we could spawn a helper thread waiting on
> cudaEventSynchronize.

Spawning a helper thread is very expensive and we need something to be done
upon completion pretty much always.  Perhaps we can optimize and somehow
deal with merging multiple async tasks that are waiting on each other, but
the user could have intermixed the offloading tasks with host tasks and have
dependencies in between them, plus there are all the various spots where
user wants to wait for both host and offloading tasks, or e.g. offloading
tasks from two different devices, or multiple offloading tasks from the same
devices (multiple streams), etc.

	Jakub

^ permalink raw reply	[flat|nested] 23+ messages in thread

* Re: [gomp4.1] depend nowait support for target {update,{enter,exit} data}
  2015-10-15 17:18       ` Jakub Jelinek
@ 2015-10-15 18:11         ` Alexander Monakov
  0 siblings, 0 replies; 23+ messages in thread
From: Alexander Monakov @ 2015-10-15 18:11 UTC (permalink / raw)
  To: Jakub Jelinek
  Cc: Ilya Verbin, Aldy Hernandez, gcc-patches, Kirill Yukhin,
	Thomas Schwinge, Martin Jambor

On Thu, 15 Oct 2015, Jakub Jelinek wrote:
> >   - this functionality doesn't currently work through CUDA MPS ("multi-process
> >     server", for funneling CUDA calls from different processes through a
> >     single "server" process, avoiding context-switch overhead on the device,
> >     sometimes used for CUDA-with-MPI applications);
> 
> That shouldn't be an issue for the OpenMP 4.5 / PTX offloading, right?

I think it can be an issue for applications employing MPI for (coarse-grain)
parallelism and OpenMP for simd/offloading.  It can be a non-issue if PTX
offloading conflicts with MPS in some other way, but at the moment I'm not
aware of such (as long as dynamic parallelism is not a hard requirement).

> >   - it is explicitely forbidden to invoke CUDA API calls from the callback;
> >     perhaps understandable, as the callback may be running in a signal-handler
> >     context (unlikely), or, more plausibly, in a different thread than the one
> >     that registered the callback.
> 
> So, is it run from async signal handlers, or just could be?

The documentation doesn't tell.  I could find out experimentally, but then it
would tell how the current implementation behaves; it could change in the
future.  Like I said in the quote, I expect it runs asynchronously in a
different thread, rather than in an async signal context.

> Spawning a helper thread is very expensive and we need something to be done
> upon completion pretty much always.  Perhaps we can optimize and somehow
> deal with merging multiple async tasks that are waiting on each other, but
> the user could have intermixed the offloading tasks with host tasks and have
> dependencies in between them, plus there are all the various spots where
> user wants to wait for both host and offloading tasks, or e.g. offloading
> tasks from two different devices, or multiple offloading tasks from the same
> devices (multiple streams), etc.

I think we should avoid involving the host in "reasonable" cases, and for the
rest just have something minimally acceptable (either with callbacks, or
polling).

Alexander

^ permalink raw reply	[flat|nested] 23+ messages in thread

* Re: [gomp4.1] depend nowait support for target {update,{enter,exit} data}
  2015-10-15 14:02   ` Jakub Jelinek
  2015-10-15 16:18     ` Alexander Monakov
  2015-10-15 16:42     ` Ilya Verbin
@ 2015-10-16 11:50     ` Martin Jambor
  2015-10-19 19:55     ` Ilya Verbin
  3 siblings, 0 replies; 23+ messages in thread
From: Martin Jambor @ 2015-10-16 11:50 UTC (permalink / raw)
  To: Jakub Jelinek
  Cc: Ilya Verbin, Aldy Hernandez, gcc-patches, Kirill Yukhin,
	Thomas Schwinge, Alexander Monakov

Hi,

On Thu, Oct 15, 2015 at 04:01:56PM +0200, Jakub Jelinek wrote:
> Hi!
> 
> CCing various people, because I'd like to have something that won't work on
> XeonPhi only.

thanks.  However, I have not paid too much attention to OMP tasks
yet.  Nevertheless, let me try to answer some of the questions.

> 
> On Fri, Oct 02, 2015 at 10:28:01PM +0300, Ilya Verbin wrote:
> > On Tue, Sep 08, 2015 at 11:20:14 +0200, Jakub Jelinek wrote:
> > > nowait support for #pragma omp target is not implemented yet, supposedly we
> > > need to mark those somehow (some flag) already in the struct gomp_task
> > > structure, essentially it will need either 2 or 3 callbacks
> > > (the current one, executed when the dependencies are resolved (it actually
> > > waits until some thread schedules it after that point, I think it is
> > > undesirable to run it with the tasking lock held), which would perform
> > > the gomp_map_vars and initiate the running of the region, and then some
> > > query routine which would poll the plugin whether the task is done or not,
> > > and either perform the finalization (unmap_vars) if it is done (and in any
> > > case return bool whether it should be polled again or not), and if the
> > > finalization is not done there, also another callback for the finalization.
> > > Also, there is the issue that if we are waiting for task that needs to be
> > > polled, and we don't have any further tasks to run, we shouldn't really
> > > attempt to sleep on some semaphore (e.g. in taskwait, end of
> > > taskgroup, etc.) or barrier, but rather either need to keep polling it, or
> > > call the query hook with some argument that it should sleep in there until
> > > the work is done by the offloading device.
> > > Also, there needs to be a way for the target nowait first callback to say
> > > that it is using host fallback and thus acts as a normal task, therefore
> > > once the task fn finishes, the task is done.
> > 
> > Here is my WIP patch.  target.c part is obviously incorrect, but it demonstrates
> > a possible libgomp <-> plugin interface for running a target task function
> > asynchronously and checking whether it is completed or not.
> > (Refactored liboffloadmic/runtime/emulator from trunk is required to run
> > target-tmp.c testcase.)
> 
> The difficulty is designing something that will work (if possible fast) on the
> various devices we want to eventually support (at least XeonPhi, XeonPhi emul,
> PTX/Cuda and HSA), ideally without too much busy waiting.
> 
> The OpenMP 4.5 spec says that there is a special "target task" on the host
> side around the target region, and that the "target task" is mergeable and
> if nowait is not specified is included (otherwise it may be), and that the
> mapping operations (which include target device memory allocation,
> refcount management and mapping data structure updates as well as the
> memory copying to target device) happens only after the (optional) dependencies
> are satisfied.  After the memory mapping operations are done, the offloading
> kernel starts, and when it finishes, the unmapping operations are performed
> (which includes memory copying from the target device, refcount management
> and mapping data structure updates, and finally memory deallocation).
> 
> Right now on the OpenMP side everything is synchronous, e.g. target
> enter/exit data and update are asynchronous only in that the mapping or
> unmapping operation is scheduled as a task, but the whole mapping or
> unmapping operations including all the above mentioned subparts are
> performed while holding the particular device's lock.

Memory mapping and unmapping is a no-op on HSA so this is fortunately
a concern for us.  (I'm assuming that ref-counting is also something
device specific and not part of running a task here).

> Anyway, let's put the asynchronous memory data transfers (which also implies
> the ability to enqueue multiple different target regions into a stream for
> the device to operate on independently from the host) on the side for now
> and just discuss what we want for the actual async execution and for now
> keep a device lock around all the mapping or unmapping operations.
> 
> If the "target task" has unresolved dependencies, then it will use existing
> task.c waiting code first (if the above is resolved somehow, there could be
> exceptions of "target task" depending on another "target task").
> When the dependencies are resolved, we can run the gomp_target_task_fn
> callback (but not with the team's tasking lock held), which can perform
> the gomp_map_vars call and start the async execution.  For host fallback,
> that is all we do, the task is at this point a normal task.
> For offloading task, we now want the host to continue scheduling other tasks
> if there are any, which means (not currently implemented on the task.c side)
> we want to move the task somewhere that we don't consider it finished, and
> that we'll need to schedule it again at some point to perform the unmapping
> (perhaps requeue it again in a WAITING or some other state).
> 
> Right now, the tasking code would in that case try to schedule another task,
> and if there are none or none that are runnable among the tasks of interest,
> it can go to sleep and expect to be awaken when some task it is waiting for
> is awaken.
> 
> And the main question is how to find out on the various devices whether
> the async execution has completed already.
> 
> From what I can see in the liboffloadmic plugin, you have an extra host
> thread that can run a callback function on the host.  Such a callback could
> say tweak the state of the "target task", could take the team's tasking
> lock, and even awake sleepers, maybe even take the device lock and perform
> unmapping of vars?  The function would need to be in task.c
> so that it can access everything defined in there.  Or the callback could
> just change something in the "target task" state and let the tasking poll
> for the change.
> 

...

> 
> For HSA I have no idea.
> 

In HSA, the task completion is signaled via so called "signals."
Which are basically (long) integers that you can atomically read/write
(etc) with the given API and wait until a specified condition (eq, ne,
le, gte) happens.  Atomic reading should be very cheap.  I do not see
a way to wait on multiple signals but we can arrange it so that
completions of a number of kernels are communicated with a single
signal.

At the moment we wait and do not create any special servicing threads
in our libgomp plugin and, as far as I know, run-time itself does not
offer a way of registering a call-back to announce kernel completion.

So polling is certainly a possibility, blocking wait if HSA task(s)
are the last ones we wait for is also simple.  Sleeping until either a
CPU or a HSA task completes might be tricky.

I hope this helps,

Martin

^ permalink raw reply	[flat|nested] 23+ messages in thread

* Re: [gomp4.1] depend nowait support for target {update,{enter,exit} data}
  2015-10-15 14:02   ` Jakub Jelinek
                       ` (2 preceding siblings ...)
  2015-10-16 11:50     ` Martin Jambor
@ 2015-10-19 19:55     ` Ilya Verbin
  2015-11-11 16:52       ` [gomp4.5] depend nowait support for target Jakub Jelinek
  3 siblings, 1 reply; 23+ messages in thread
From: Ilya Verbin @ 2015-10-19 19:55 UTC (permalink / raw)
  To: Jakub Jelinek
  Cc: Aldy Hernandez, gcc-patches, Kirill Yukhin, Thomas Schwinge,
	Alexander Monakov, Martin Jambor

On Thu, Oct 15, 2015 at 16:01:56 +0200, Jakub Jelinek wrote:
> >    void *fn_addr = gomp_get_target_fn_addr (devicep, fn);
> >  
> > +  if (flags & GOMP_TARGET_FLAG_NOWAIT)
> > +    {
> > +      gomp_create_target_task (devicep, fn_addr, mapnum, hostaddrs, sizes,
> > +			       kinds, flags, depend);
> > +      return;
> > +    }
> 
> But this is not ok.  You need to do this far earlier, already before the
> if (depend != NULL) code in GOMP_target_41.  And, I think you should just
> not pass fn_addr, but fn itself.
> 
> > @@ -1636,34 +1657,58 @@ void
> >  gomp_target_task_fn (void *data)
> >  {
> >    struct gomp_target_task *ttask = (struct gomp_target_task *) data;
> > +  struct gomp_device_descr *devicep = ttask->devicep;
> > +
> >    if (ttask->fn != NULL)
> >      {
> > -      /* GOMP_target_41 */
> > +      if (devicep == NULL
> > +	  || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
> > +	{
> > +	  /* FIXME: Save host fn addr into gomp_target_task?  */
> > +	  gomp_target_fallback_firstprivate (NULL, ttask->mapnum,
> 
> If you pass above fn instead of fn_addr, ttask->fn is what you want
> to pass to gomp_target_fallback_firstprivate here and remove the FIXME.
> 
> > +					     ttask->hostaddrs, ttask->sizes,
> > +					     ttask->kinds);
> > +	  return;
> > +	}
> > +
> > +      struct target_mem_desc *tgt_vars
> > +	= gomp_map_vars (devicep, ttask->mapnum, ttask->hostaddrs, NULL,
> > +			 ttask->sizes, ttask->kinds, true,
> > +			 GOMP_MAP_VARS_TARGET);
> > +      devicep->async_run_func (devicep->target_id, ttask->fn,
> > +			       (void *) tgt_vars->tgt_start, data);
> 
> You need to void *fn_addr = gomp_get_target_fn_addr (devicep, ttask->fn);
> first obviously, and pass fn_addr.
> 
> > +
> > +      /* FIXME: TMP example of checking for completion.
> > +	 Alternatively the plugin can set some completion flag in ttask.  */
> > +      while (!devicep->async_is_completed_func (devicep->target_id, data))
> > +	{
> > +	  fprintf (stderr, "-");
> > +	  usleep (100000);
> > +	}
> 
> This obviously doesn't belong here.
> 
> >    if (device->capabilities & GOMP_OFFLOAD_CAP_OPENACC_200)
> > diff --git a/libgomp/testsuite/libgomp.c/target-tmp.c b/libgomp/testsuite/libgomp.c/target-tmp.c
> > new file mode 100644
> > index 0000000..23a739c
> > --- /dev/null
> > +++ b/libgomp/testsuite/libgomp.c/target-tmp.c
> > @@ -0,0 +1,40 @@
> > +#include <stdio.h>
> > +#include <unistd.h>
> > +
> > +#pragma omp declare target
> > +void foo (int n)
> > +{
> > +  printf ("Start tgt %d\n", n);
> > +  usleep (5000000);
> 
> 5s is too long.  Not to mention that not sure if PTX can do printf
> and especially usleep.
> 
> > diff --git a/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp b/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp
> > index 26ac6fe..c843710 100644
> > --- a/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp
> > +++ b/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp
> ...
> > +/* Set of asynchronously running target tasks.  */
> > +static std::set<const void *> *async_tasks;
> > +
> >  /* Thread-safe registration of the main image.  */
> >  static pthread_once_t main_image_is_registered = PTHREAD_ONCE_INIT;
> >  
> > +/* Mutex for protecting async_tasks.  */
> > +static pthread_mutex_t async_tasks_lock = PTHREAD_MUTEX_INITIALIZER;
> > +
> >  static VarDesc vd_host2tgt = {
> >    { 1, 1 },		      /* dst, src			      */
> >    { 1, 0 },		      /* in, out			      */
> > @@ -156,6 +163,8 @@ init (void)
> >  
> >  out:
> >    address_table = new ImgDevAddrMap;
> > +  async_tasks = new std::set<const void *>;
> > +  pthread_mutex_init (&async_tasks_lock, NULL);
> 
> PTHREAD_MUTEX_INITIALIZER should already initialize the lock.
> But, do you really need async_tasks and the lock?  Better store
> something into some plugin's owned field in target_task struct and
> let the plugin callback be passed address of that field rather than the
> whole target_task?

So, here is what I have for now.  Attached target-29.c testcase works fine with
MIC emul, however I don't know how to (and where) properly check for completion
of async execution on target.  And, similarly, where to do unmapping after that?
Do we need a callback from plugin to libgomp (as far as I understood, PTX
runtime supports this, but HSA doesn't), or libgomp will just check for
ttask->is_completed in task.c?

 
diff --git a/libgomp/libgomp.h b/libgomp/libgomp.h
index 9c8b1fb..e707c80 100644
--- a/libgomp/libgomp.h
+++ b/libgomp/libgomp.h
@@ -430,6 +430,7 @@ struct gomp_target_task
   size_t *sizes;
   unsigned short *kinds;
   unsigned int flags;
+  bool is_completed;
   void *hostaddrs[];
 };
 
@@ -877,6 +878,7 @@ struct gomp_device_descr
   void *(*host2dev_func) (int, void *, const void *, size_t);
   void *(*dev2dev_func) (int, void *, const void *, size_t);
   void (*run_func) (int, void *, void *);
+  void (*async_run_func) (int, void *, void *, bool *);
 
   /* Splay tree containing information about mapped memory regions.  */
   struct splay_tree_s mem_map;
diff --git a/libgomp/target.c b/libgomp/target.c
index d4c0cef..a136fb9 100644
--- a/libgomp/target.c
+++ b/libgomp/target.c
@@ -1339,6 +1339,14 @@ GOMP_target_41 (int device, void (*fn) (void *), size_t mapnum,
 {
   struct gomp_device_descr *devicep = resolve_device (device);
 
+  /* FIXME: Check for thr->ts.team && !thr->task->final_task ?  */
+  if (flags & GOMP_TARGET_FLAG_NOWAIT)
+    {
+      gomp_create_target_task (devicep, fn, mapnum, hostaddrs, sizes, kinds,
+			       flags, depend);
+      return;
+    }
+
   /* If there are depend clauses, but nowait is not present,
      block the parent task until the dependencies are resolved
      and then just continue with the rest of the function as if it
@@ -1650,34 +1658,56 @@ void
 gomp_target_task_fn (void *data)
 {
   struct gomp_target_task *ttask = (struct gomp_target_task *) data;
+  struct gomp_device_descr *devicep = ttask->devicep;
+
   if (ttask->fn != NULL)
     {
-      /* GOMP_target_41 */
+      if (devicep == NULL
+	  || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+	{
+	  gomp_target_fallback_firstprivate (ttask->fn, ttask->mapnum,
+					     ttask->hostaddrs, ttask->sizes,
+					     ttask->kinds);
+	  return;
+	}
+
+      void *fn_addr = gomp_get_target_fn_addr (devicep, ttask->fn);
+      struct target_mem_desc *tgt_vars
+	= gomp_map_vars (devicep, ttask->mapnum, ttask->hostaddrs, NULL,
+			 ttask->sizes, ttask->kinds, true,
+			 GOMP_MAP_VARS_TARGET);
+      devicep->async_run_func (devicep->target_id, fn_addr,
+			       (void *) tgt_vars->tgt_start,
+			       &ttask->is_completed);
+
+      /* FIXME: Move the task into some sleeping state, remove this loop from
+	 here.  */
+      while (!ttask->is_completed);
+      return;
     }
-  else if (ttask->devicep == NULL
-	   || !(ttask->devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+  else if (devicep == NULL
+	   || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
     return;
 
   size_t i;
   if (ttask->flags & GOMP_TARGET_FLAG_UPDATE)
-    gomp_update (ttask->devicep, ttask->mapnum, ttask->hostaddrs, ttask->sizes,
+    gomp_update (devicep, ttask->mapnum, ttask->hostaddrs, ttask->sizes,
 		 ttask->kinds, true);
   else if ((ttask->flags & GOMP_TARGET_FLAG_EXIT_DATA) == 0)
     for (i = 0; i < ttask->mapnum; i++)
       if ((ttask->kinds[i] & 0xff) == GOMP_MAP_STRUCT)
 	{
-	  gomp_map_vars (ttask->devicep, ttask->sizes[i] + 1,
-			 &ttask->hostaddrs[i], NULL, &ttask->sizes[i],
-			 &ttask->kinds[i], true, GOMP_MAP_VARS_ENTER_DATA);
+	  gomp_map_vars (devicep, ttask->sizes[i] + 1, &ttask->hostaddrs[i],
+			 NULL, &ttask->sizes[i], &ttask->kinds[i], true,
+			 GOMP_MAP_VARS_ENTER_DATA);
 	  i += ttask->sizes[i];
 	}
       else
-	gomp_map_vars (ttask->devicep, 1, &ttask->hostaddrs[i], NULL,
-		       &ttask->sizes[i], &ttask->kinds[i],
-		       true, GOMP_MAP_VARS_ENTER_DATA);
+	gomp_map_vars (devicep, 1, &ttask->hostaddrs[i], NULL, &ttask->sizes[i],
+		       &ttask->kinds[i], true, GOMP_MAP_VARS_ENTER_DATA);
   else
-    gomp_exit_data (ttask->devicep, ttask->mapnum, ttask->hostaddrs,
-		    ttask->sizes, ttask->kinds);
+    gomp_exit_data (devicep, ttask->mapnum, ttask->hostaddrs, ttask->sizes,
+		    ttask->kinds);
 }
 
 void
@@ -2122,6 +2152,7 @@ gomp_load_plugin_for_device (struct gomp_device_descr *device,
   if (device->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400)
     {
       DLSYM (run);
+      DLSYM (async_run);
       DLSYM (dev2dev);
     }
   if (device->capabilities & GOMP_OFFLOAD_CAP_OPENACC_200)
diff --git a/libgomp/task.c b/libgomp/task.c
index 1246c6a..7e1e3ca 100644
--- a/libgomp/task.c
+++ b/libgomp/task.c
@@ -480,6 +480,7 @@ gomp_create_target_task (struct gomp_device_descr *devicep,
   task->kind = GOMP_TASK_WAITING;
   task->in_tied_task = parent->in_tied_task;
   task->taskgroup = taskgroup;
+  /* FIXME: Segmentation fault here, if there are no dependencies.  */
   ttask = (struct gomp_target_task *) &task->depend[(uintptr_t) depend[0]];
   ttask->devicep = devicep;
   ttask->fn = fn;
@@ -490,6 +491,7 @@ gomp_create_target_task (struct gomp_device_descr *devicep,
   ttask->kinds = (unsigned short *) &ttask->sizes[mapnum];
   memcpy (ttask->kinds, kinds, mapnum * sizeof (unsigned short));
   ttask->flags = flags;
+  ttask->is_completed = false;
   task->fn = gomp_target_task_fn;
   task->fn_data = ttask;
   task->final_task = 0;
diff --git a/libgomp/testsuite/libgomp.c/target-29.c b/libgomp/testsuite/libgomp.c/target-29.c
new file mode 100644
index 0000000..c532797
--- /dev/null
+++ b/libgomp/testsuite/libgomp.c/target-29.c
@@ -0,0 +1,50 @@
+#include <stdlib.h>
+#include <unistd.h>
+
+int main ()
+{
+  int a = 0, b = 0, c = 0, d[7];
+
+  #pragma omp parallel
+  #pragma omp single
+  {
+    #pragma omp task depend(out: d[0])
+      a = 2;
+
+    #pragma omp target enter data nowait map(to: a,b,c) depend(in: d[0]) depend(out: d[1])
+
+    #pragma omp target nowait map(alloc: a) depend(in: d[1]) depend(out: d[2])
+      a++;
+
+    #pragma omp target nowait map(alloc: b) depend(in: d[2]) depend(out: d[3])
+    {
+      usleep (1000);
+      b = 4;
+    }
+
+    #pragma omp target nowait map(alloc: b) depend(in: d[2]) depend(out: d[4])
+    {
+      usleep (5000);
+      b = 5;
+    }
+
+    #pragma omp target nowait map(alloc: c) depend(in: d[3], d[4]) depend(out: d[5])
+    {
+      usleep (5000);
+      c = 6;
+    }
+
+    #pragma omp target nowait map(alloc: c) depend(in: d[3], d[4]) depend(out: d[6])
+    {
+      usleep (1000);
+      c = 7;
+    }
+
+    #pragma omp target exit data map(always,from: a,b,c) depend(in: d[5], d[6])
+  }
+
+  if (a != 3 || (b != 4 && b != 5) || (c != 6 && c != 7))
+    abort ();
+
+  return 0;
+}
diff --git a/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp b/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp
index 26ac6fe..91c52ec 100644
--- a/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp
+++ b/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp
@@ -192,11 +192,23 @@ GOMP_OFFLOAD_get_num_devices (void)
 
 static void
 offload (const char *file, uint64_t line, int device, const char *name,
-	 int num_vars, VarDesc *vars, VarDesc2 *vars2)
+	 int num_vars, VarDesc *vars, VarDesc2 *vars2, const void **async_data)
 {
   OFFLOAD ofld = __offload_target_acquire1 (&device, file, line);
   if (ofld)
-    __offload_offload1 (ofld, name, 0, num_vars, vars, vars2, 0, NULL, NULL);
+    {
+      if (async_data == NULL)
+	__offload_offload1 (ofld, name, 0, num_vars, vars, vars2, 0, NULL,
+			    NULL);
+      else
+	{
+	  OffloadFlags flags;
+	  flags.flags = 0;
+	  flags.bits.omp_async = 1;
+	  __offload_offload3 (ofld, name, 0, num_vars, vars, NULL, 0, NULL,
+			      async_data, 0, NULL, flags, NULL);
+	}
+    }
   else
     {
       fprintf (stderr, "%s:%d: Offload target acquire failed\n", file, line);
@@ -218,7 +230,7 @@ GOMP_OFFLOAD_init_device (int device)
   TRACE ("");
   pthread_once (&main_image_is_registered, register_main_image);
   offload (__FILE__, __LINE__, device, "__offload_target_init_proc", 0,
-	   NULL, NULL);
+	   NULL, NULL, NULL);
 }
 
 extern "C" void
@@ -240,7 +252,7 @@ get_target_table (int device, int &num_funcs, int &num_vars, void **&table)
   VarDesc2 vd1g[2] = { { "num_funcs", 0 }, { "num_vars", 0 } };
 
   offload (__FILE__, __LINE__, device, "__offload_target_table_p1", 2,
-	   vd1, vd1g);
+	   vd1, vd1g, NULL);
 
   int table_size = num_funcs + 2 * num_vars;
   if (table_size > 0)
@@ -254,7 +266,7 @@ get_target_table (int device, int &num_funcs, int &num_vars, void **&table)
       VarDesc2 vd2g = { "table", 0 };
 
       offload (__FILE__, __LINE__, device, "__offload_target_table_p2", 1,
-	       &vd2, &vd2g);
+	       &vd2, &vd2g, NULL);
     }
 }
 
@@ -401,8 +413,8 @@ GOMP_OFFLOAD_alloc (int device, size_t size)
   vd1[1].size = sizeof (void *);
   VarDesc2 vd1g[2] = { { "size", 0 }, { "tgt_ptr", 0 } };
 
-  offload (__FILE__, __LINE__, device, "__offload_target_alloc", 2, vd1, vd1g);
-
+  offload (__FILE__, __LINE__, device, "__offload_target_alloc", 2, vd1, vd1g,
+	   NULL);
   return tgt_ptr;
 }
 
@@ -416,7 +428,8 @@ GOMP_OFFLOAD_free (int device, void *tgt_ptr)
   vd1.size = sizeof (void *);
   VarDesc2 vd1g = { "tgt_ptr", 0 };
 
-  offload (__FILE__, __LINE__, device, "__offload_target_free", 1, &vd1, &vd1g);
+  offload (__FILE__, __LINE__, device, "__offload_target_free", 1, &vd1, &vd1g,
+	   NULL);
 }
 
 extern "C" void *
@@ -435,7 +448,7 @@ GOMP_OFFLOAD_host2dev (int device, void *tgt_ptr, const void *host_ptr,
   VarDesc2 vd1g[2] = { { "tgt_ptr", 0 }, { "size", 0 } };
 
   offload (__FILE__, __LINE__, device, "__offload_target_host2tgt_p1", 2,
-	   vd1, vd1g);
+	   vd1, vd1g, NULL);
 
   VarDesc vd2 = vd_host2tgt;
   vd2.ptr = (void *) host_ptr;
@@ -443,7 +456,7 @@ GOMP_OFFLOAD_host2dev (int device, void *tgt_ptr, const void *host_ptr,
   VarDesc2 vd2g = { "var", 0 };
 
   offload (__FILE__, __LINE__, device, "__offload_target_host2tgt_p2", 1,
-	   &vd2, &vd2g);
+	   &vd2, &vd2g, NULL);
 
   return tgt_ptr;
 }
@@ -464,7 +477,7 @@ GOMP_OFFLOAD_dev2host (int device, void *host_ptr, const void *tgt_ptr,
   VarDesc2 vd1g[2] = { { "tgt_ptr", 0 }, { "size", 0 } };
 
   offload (__FILE__, __LINE__, device, "__offload_target_tgt2host_p1", 2,
-	   vd1, vd1g);
+	   vd1, vd1g, NULL);
 
   VarDesc vd2 = vd_tgt2host;
   vd2.ptr = (void *) host_ptr;
@@ -472,7 +485,7 @@ GOMP_OFFLOAD_dev2host (int device, void *host_ptr, const void *tgt_ptr,
   VarDesc2 vd2g = { "var", 0 };
 
   offload (__FILE__, __LINE__, device, "__offload_target_tgt2host_p2", 1,
-	   &vd2, &vd2g);
+	   &vd2, &vd2g, NULL);
 
   return host_ptr;
 }
@@ -495,22 +508,43 @@ GOMP_OFFLOAD_dev2dev (int device, void *dst_ptr, const void *src_ptr,
   VarDesc2 vd1g[3] = { { "dst_ptr", 0 }, { "src_ptr", 0 }, { "size", 0 } };
 
   offload (__FILE__, __LINE__, device, "__offload_target_tgt2tgt", 3, vd1,
-	   vd1g);
+	   vd1g, NULL);
 
   return dst_ptr;
 }
 
 extern "C" void
+GOMP_OFFLOAD_async_run (int device, void *tgt_fn, void *tgt_vars,
+			bool *async_data)
+{
+  TRACE ("(device = %d, tgt_fn = %p, tgt_vars = %p, async_data = %p)", device,
+	 tgt_fn, tgt_vars, async_data);
+
+  VarDesc vd[2] = { vd_host2tgt, vd_host2tgt };
+  vd[0].ptr = &tgt_fn;
+  vd[0].size = sizeof (void *);
+  vd[1].ptr = &tgt_vars;
+  vd[1].size = sizeof (void *);
+
+  offload (__FILE__, __LINE__, device, "__offload_target_run", 2, vd, NULL,
+	   (const void **) async_data);
+}
+
+extern "C" void
 GOMP_OFFLOAD_run (int device, void *tgt_fn, void *tgt_vars)
 {
-  TRACE ("(tgt_fn = %p, tgt_vars = %p)", tgt_fn, tgt_vars);
+  TRACE ("(device = %d, tgt_fn = %p, tgt_vars = %p)", device, tgt_fn, tgt_vars);
 
-  VarDesc vd1[2] = { vd_host2tgt, vd_host2tgt };
-  vd1[0].ptr = &tgt_fn;
-  vd1[0].size = sizeof (void *);
-  vd1[1].ptr = &tgt_vars;
-  vd1[1].size = sizeof (void *);
-  VarDesc2 vd1g[2] = { { "tgt_fn", 0 }, { "tgt_vars", 0 } };
+  GOMP_OFFLOAD_async_run (device, tgt_fn, tgt_vars, NULL);
+}
+
+/* Called by liboffloadmic when asynchronous function is completed.  */
+
+extern "C" void
+__gomp_offload_intelmic_async_completed (const void *async_data)
+{
+  TRACE ("(async_data = %p)", async_data);
 
-  offload (__FILE__, __LINE__, device, "__offload_target_run", 2, vd1, vd1g);
+  bool *ttask_is_completed = (bool *) async_data;
+  *ttask_is_completed = true;
 }
diff --git a/liboffloadmic/runtime/offload_host.cpp b/liboffloadmic/runtime/offload_host.cpp
index 66c2a01..fe6ec44 100644
--- a/liboffloadmic/runtime/offload_host.cpp
+++ b/liboffloadmic/runtime/offload_host.cpp
@@ -64,6 +64,9 @@ static void __offload_fini_library(void);
 #define GET_OFFLOAD_NUMBER(timer_data) \
     timer_data? timer_data->offload_number : 0
 
+extern "C" void
+__gomp_offload_intelmic_async_completed (const void *);
+
 extern "C" {
 #ifdef TARGET_WINNT
 // Windows does not support imports from libraries without actually
@@ -2508,7 +2511,7 @@ extern "C" {
         const void *info
     )
     {
-	/* TODO: Call callback function, pass info.  */
+	__gomp_offload_intelmic_async_completed (info);
     }
 }
 

Thanks,
  -- Ilya

^ permalink raw reply	[flat|nested] 23+ messages in thread

* [gomp4.5] depend nowait support for target
  2015-10-19 19:55     ` Ilya Verbin
@ 2015-11-11 16:52       ` Jakub Jelinek
  2015-11-12 17:44         ` Ilya Verbin
  2015-11-12 17:45         ` Jakub Jelinek
  0 siblings, 2 replies; 23+ messages in thread
From: Jakub Jelinek @ 2015-11-11 16:52 UTC (permalink / raw)
  To: Ilya Verbin
  Cc: Aldy Hernandez, gcc-patches, Kirill Yukhin, Thomas Schwinge,
	Alexander Monakov, Martin Jambor

Hi!

On Mon, Oct 19, 2015 at 10:47:54PM +0300, Ilya Verbin wrote:
> So, here is what I have for now.  Attached target-29.c testcase works fine with
> MIC emul, however I don't know how to (and where) properly check for completion
> of async execution on target.  And, similarly, where to do unmapping after that?
> Do we need a callback from plugin to libgomp (as far as I understood, PTX
> runtime supports this, but HSA doesn't), or libgomp will just check for
> ttask->is_completed in task.c?

Here is the patch updated to have a task.c defined function that the plugin
can call upon completion of async offloading exection.
The testsuite coverage will need to improve, the testcase is wrong
(contains data races - if you want to test parallel running of two target
regions that both touch the same var, I'd say best would be to use
#pragma omp atomic and or in 4 in one case and 1 in another case, then
test if result is 5 (and similarly for the other var).
Also, with the usleeps Alex Monakov will be unhappy because PTX newlib does
not have it, but we'll need to find some solution for that.

Another thing to work on beyond testsuite coverage (it is desirable to test
nowait target tasks (both depend and without depend) being awaited in all
the various waiting spots, i.e. end of parallel, barrier, taskwait, end of
taskgroup, or if (0) task with depend clause waiting on that.

Also, I wonder what to do if #pragma omp target nowait is used outside of
(host) parallel - when team is NULL.  All the tasking code in that case just
executes tasks undeferred, which is fine for all but target nowait - there
it is I'd say useful to be able to run a single host thread concurrently
with some async offloading tasks.  So, I wonder if in that case,
if we encounter target nowait with team == NULL, should not just create a
dummy non-active (nthreads == 1) team, as if there was #pragma omp parallel
if (0) starting above it and ending at program's end.  In OpenMP, the
program's initial thread is implicitly surrounded by inactive parallel, so
this isn't anything against the OpenMP execution model.  But we'd need to
free the team somewhere in a destructor.

Can you please try to cleanup the liboffloadmic side of this, so that
a callback instead of hardcoded __gomp_offload_intelmic_async_completed call
is used?  Can you make sure it works on XeonPhi non-emulated too?

I'll keep working on the testcase coverage and on the team == NULL case.

The patch is on top of gomp-4_5-branch - needs Aldy's priority_queue stuff.

--- liboffloadmic/runtime/offload_host.cpp.jj	2015-11-05 11:31:05.013916598 +0100
+++ liboffloadmic/runtime/offload_host.cpp	2015-11-10 12:58:55.090951303 +0100
@@ -64,6 +64,9 @@ static void __offload_fini_library(void)
 #define GET_OFFLOAD_NUMBER(timer_data) \
     timer_data? timer_data->offload_number : 0
 
+extern "C" void
+__gomp_offload_intelmic_async_completed (const void *);
+
 extern "C" {
 #ifdef TARGET_WINNT
 // Windows does not support imports from libraries without actually
@@ -2507,7 +2510,7 @@ extern "C" {
         const void *info
     )
     {
-	/* TODO: Call callback function, pass info.  */
+	__gomp_offload_intelmic_async_completed (info);
     }
 }
 
--- liboffloadmic/plugin/libgomp-plugin-intelmic.cpp.jj	2015-10-14 10:24:10.922194230 +0200
+++ liboffloadmic/plugin/libgomp-plugin-intelmic.cpp	2015-11-11 15:48:55.428967827 +0100
@@ -192,11 +192,23 @@ GOMP_OFFLOAD_get_num_devices (void)
 
 static void
 offload (const char *file, uint64_t line, int device, const char *name,
-	 int num_vars, VarDesc *vars, VarDesc2 *vars2)
+	 int num_vars, VarDesc *vars, VarDesc2 *vars2, const void **async_data)
 {
   OFFLOAD ofld = __offload_target_acquire1 (&device, file, line);
   if (ofld)
-    __offload_offload1 (ofld, name, 0, num_vars, vars, vars2, 0, NULL, NULL);
+    {
+      if (async_data == NULL)
+	__offload_offload1 (ofld, name, 0, num_vars, vars, vars2, 0, NULL,
+			    NULL);
+      else
+	{
+	  OffloadFlags flags;
+	  flags.flags = 0;
+	  flags.bits.omp_async = 1;
+	  __offload_offload3 (ofld, name, 0, num_vars, vars, NULL, 0, NULL,
+			      async_data, 0, NULL, flags, NULL);
+	}
+    }
   else
     {
       fprintf (stderr, "%s:%d: Offload target acquire failed\n", file, line);
@@ -218,7 +230,7 @@ GOMP_OFFLOAD_init_device (int device)
   TRACE ("");
   pthread_once (&main_image_is_registered, register_main_image);
   offload (__FILE__, __LINE__, device, "__offload_target_init_proc", 0,
-	   NULL, NULL);
+	   NULL, NULL, NULL);
 }
 
 extern "C" void
@@ -240,7 +252,7 @@ get_target_table (int device, int &num_f
   VarDesc2 vd1g[2] = { { "num_funcs", 0 }, { "num_vars", 0 } };
 
   offload (__FILE__, __LINE__, device, "__offload_target_table_p1", 2,
-	   vd1, vd1g);
+	   vd1, vd1g, NULL);
 
   int table_size = num_funcs + 2 * num_vars;
   if (table_size > 0)
@@ -254,7 +266,7 @@ get_target_table (int device, int &num_f
       VarDesc2 vd2g = { "table", 0 };
 
       offload (__FILE__, __LINE__, device, "__offload_target_table_p2", 1,
-	       &vd2, &vd2g);
+	       &vd2, &vd2g, NULL);
     }
 }
 
@@ -401,8 +413,8 @@ GOMP_OFFLOAD_alloc (int device, size_t s
   vd1[1].size = sizeof (void *);
   VarDesc2 vd1g[2] = { { "size", 0 }, { "tgt_ptr", 0 } };
 
-  offload (__FILE__, __LINE__, device, "__offload_target_alloc", 2, vd1, vd1g);
-
+  offload (__FILE__, __LINE__, device, "__offload_target_alloc", 2, vd1, vd1g,
+	   NULL);
   return tgt_ptr;
 }
 
@@ -416,7 +428,8 @@ GOMP_OFFLOAD_free (int device, void *tgt
   vd1.size = sizeof (void *);
   VarDesc2 vd1g = { "tgt_ptr", 0 };
 
-  offload (__FILE__, __LINE__, device, "__offload_target_free", 1, &vd1, &vd1g);
+  offload (__FILE__, __LINE__, device, "__offload_target_free", 1, &vd1, &vd1g,
+	   NULL);
 }
 
 extern "C" void *
@@ -435,7 +448,7 @@ GOMP_OFFLOAD_host2dev (int device, void
   VarDesc2 vd1g[2] = { { "tgt_ptr", 0 }, { "size", 0 } };
 
   offload (__FILE__, __LINE__, device, "__offload_target_host2tgt_p1", 2,
-	   vd1, vd1g);
+	   vd1, vd1g, NULL);
 
   VarDesc vd2 = vd_host2tgt;
   vd2.ptr = (void *) host_ptr;
@@ -443,7 +456,7 @@ GOMP_OFFLOAD_host2dev (int device, void
   VarDesc2 vd2g = { "var", 0 };
 
   offload (__FILE__, __LINE__, device, "__offload_target_host2tgt_p2", 1,
-	   &vd2, &vd2g);
+	   &vd2, &vd2g, NULL);
 
   return tgt_ptr;
 }
@@ -464,7 +477,7 @@ GOMP_OFFLOAD_dev2host (int device, void
   VarDesc2 vd1g[2] = { { "tgt_ptr", 0 }, { "size", 0 } };
 
   offload (__FILE__, __LINE__, device, "__offload_target_tgt2host_p1", 2,
-	   vd1, vd1g);
+	   vd1, vd1g, NULL);
 
   VarDesc vd2 = vd_tgt2host;
   vd2.ptr = (void *) host_ptr;
@@ -472,7 +485,7 @@ GOMP_OFFLOAD_dev2host (int device, void
   VarDesc2 vd2g = { "var", 0 };
 
   offload (__FILE__, __LINE__, device, "__offload_target_tgt2host_p2", 1,
-	   &vd2, &vd2g);
+	   &vd2, &vd2g, NULL);
 
   return host_ptr;
 }
@@ -495,22 +508,42 @@ GOMP_OFFLOAD_dev2dev (int device, void *
   VarDesc2 vd1g[3] = { { "dst_ptr", 0 }, { "src_ptr", 0 }, { "size", 0 } };
 
   offload (__FILE__, __LINE__, device, "__offload_target_tgt2tgt", 3, vd1,
-	   vd1g);
+	   vd1g, NULL);
 
   return dst_ptr;
 }
 
 extern "C" void
+GOMP_OFFLOAD_async_run (int device, void *tgt_fn, void *tgt_vars,
+			void *async_data)
+{
+  TRACE ("(device = %d, tgt_fn = %p, tgt_vars = %p, async_data = %p)", device,
+	 tgt_fn, tgt_vars, async_data);
+
+  VarDesc vd[2] = { vd_host2tgt, vd_host2tgt };
+  vd[0].ptr = &tgt_fn;
+  vd[0].size = sizeof (void *);
+  vd[1].ptr = &tgt_vars;
+  vd[1].size = sizeof (void *);
+
+  offload (__FILE__, __LINE__, device, "__offload_target_run", 2, vd, NULL,
+	   (const void **) async_data);
+}
+
+extern "C" void
 GOMP_OFFLOAD_run (int device, void *tgt_fn, void *tgt_vars)
 {
-  TRACE ("(tgt_fn = %p, tgt_vars = %p)", tgt_fn, tgt_vars);
+  TRACE ("(device = %d, tgt_fn = %p, tgt_vars = %p)", device, tgt_fn, tgt_vars);
 
-  VarDesc vd1[2] = { vd_host2tgt, vd_host2tgt };
-  vd1[0].ptr = &tgt_fn;
-  vd1[0].size = sizeof (void *);
-  vd1[1].ptr = &tgt_vars;
-  vd1[1].size = sizeof (void *);
-  VarDesc2 vd1g[2] = { { "tgt_fn", 0 }, { "tgt_vars", 0 } };
+  GOMP_OFFLOAD_async_run (device, tgt_fn, tgt_vars, NULL);
+}
+
+/* Called by liboffloadmic when asynchronous function is completed.  */
+
+extern "C" void
+__gomp_offload_intelmic_async_completed (const void *async_data)
+{
+  TRACE ("(async_data = %p)", async_data);
 
-  offload (__FILE__, __LINE__, device, "__offload_target_run", 2, vd1, vd1g);
+  GOMP_PLUGIN_target_task_completion ((void *) async_data);
 }
--- libgomp/libgomp-plugin.h.jj	2015-10-14 10:24:10.000000000 +0200
+++ libgomp/libgomp-plugin.h	2015-11-11 15:48:16.875505434 +0100
@@ -59,10 +59,20 @@ struct addr_pair
   uintptr_t end;
 };
 
+/* Various state of OpenMP async offloading tasks.  */
+enum gomp_target_task_state
+{
+  GOMP_TARGET_TASK_DATA,
+  GOMP_TARGET_TASK_BEFORE_MAP,
+  GOMP_TARGET_TASK_FALLBACK,
+  GOMP_TARGET_TASK_RUNNING
+};
+
 /* Miscellaneous functions.  */
 extern void *GOMP_PLUGIN_malloc (size_t) __attribute__ ((malloc));
 extern void *GOMP_PLUGIN_malloc_cleared (size_t) __attribute__ ((malloc));
 extern void *GOMP_PLUGIN_realloc (void *, size_t);
+void GOMP_PLUGIN_target_task_completion (void *);
 
 extern void GOMP_PLUGIN_debug (int, const char *, ...)
 	__attribute__ ((format (printf, 2, 3)));
--- libgomp/testsuite/libgomp.c/target-32.c.jj	2015-11-10 12:58:55.087951346 +0100
+++ libgomp/testsuite/libgomp.c/target-32.c	2015-11-10 12:58:55.087951346 +0100
@@ -0,0 +1,50 @@
+#include <stdlib.h>
+#include <unistd.h>
+
+int main ()
+{
+  int a = 0, b = 0, c = 0, d[7];
+
+  #pragma omp parallel
+  #pragma omp single
+  {
+    #pragma omp task depend(out: d[0])
+      a = 2;
+
+    #pragma omp target enter data nowait map(to: a,b,c) depend(in: d[0]) depend(out: d[1])
+
+    #pragma omp target nowait map(alloc: a) depend(in: d[1]) depend(out: d[2])
+      a++;
+
+    #pragma omp target nowait map(alloc: b) depend(in: d[2]) depend(out: d[3])
+    {
+      usleep (1000);
+      b = 4;
+    }
+
+    #pragma omp target nowait map(alloc: b) depend(in: d[2]) depend(out: d[4])
+    {
+      usleep (5000);
+      b = 5;
+    }
+
+    #pragma omp target nowait map(alloc: c) depend(in: d[3], d[4]) depend(out: d[5])
+    {
+      usleep (5000);
+      c = 6;
+    }
+
+    #pragma omp target nowait map(alloc: c) depend(in: d[3], d[4]) depend(out: d[6])
+    {
+      usleep (1000);
+      c = 7;
+    }
+
+    #pragma omp target exit data map(always,from: a,b,c) depend(in: d[5], d[6])
+  }
+
+  if (a != 3 || (b != 4 && b != 5) || (c != 6 && c != 7))
+    abort ();
+
+  return 0;
+}
--- libgomp/target.c.jj	2015-11-09 11:14:37.325239961 +0100
+++ libgomp/target.c	2015-11-11 16:05:52.550784608 +0100
@@ -1348,17 +1348,7 @@ GOMP_target (int device, void (*fn) (voi
   struct target_mem_desc *tgt_vars
     = gomp_map_vars (devicep, mapnum, hostaddrs, NULL, sizes, kinds, false,
 		     GOMP_MAP_VARS_TARGET);
-  struct gomp_thread old_thr, *thr = gomp_thread ();
-  old_thr = *thr;
-  memset (thr, '\0', sizeof (*thr));
-  if (gomp_places_list)
-    {
-      thr->place = old_thr.place;
-      thr->ts.place_partition_len = gomp_places_list_len;
-    }
   devicep->run_func (devicep->target_id, fn_addr, (void *) tgt_vars->tgt_start);
-  gomp_free_thread (thr);
-  *thr = old_thr;
   gomp_unmap_vars (tgt_vars, true);
 }
 
@@ -1387,10 +1377,23 @@ GOMP_target_ext (int device, void (*fn)
   (void) num_teams;
   (void) thread_limit;
 
-  /* If there are depend clauses, but nowait is not present,
-     block the parent task until the dependencies are resolved
-     and then just continue with the rest of the function as if it
-     is a merged task.  */
+  if (flags & GOMP_TARGET_FLAG_NOWAIT)
+    {
+      struct gomp_thread *thr = gomp_thread ();
+      if (thr->ts.team
+	  && !thr->task->final_task)
+	{
+	  gomp_create_target_task (devicep, fn, mapnum, hostaddrs,
+				   sizes, kinds, flags, depend,
+				   GOMP_TARGET_TASK_BEFORE_MAP);
+	  return;
+	}
+    }
+
+  /* If there are depend clauses, but nowait is not present
+     (or we are in a final task), block the parent task until the
+     dependencies are resolved and then just continue with the rest
+     of the function as if it is a merged task.  */
   if (depend != NULL)
     {
       struct gomp_thread *thr = gomp_thread ();
@@ -1410,17 +1413,7 @@ GOMP_target_ext (int device, void (*fn)
   struct target_mem_desc *tgt_vars
     = gomp_map_vars (devicep, mapnum, hostaddrs, NULL, sizes, kinds, true,
 		     GOMP_MAP_VARS_TARGET);
-  struct gomp_thread old_thr, *thr = gomp_thread ();
-  old_thr = *thr;
-  memset (thr, '\0', sizeof (*thr));
-  if (gomp_places_list)
-    {
-      thr->place = old_thr.place;
-      thr->ts.place_partition_len = gomp_places_list_len;
-    }
   devicep->run_func (devicep->target_id, fn_addr, (void *) tgt_vars->tgt_start);
-  gomp_free_thread (thr);
-  *thr = old_thr;
   gomp_unmap_vars (tgt_vars, true);
 }
 
@@ -1527,23 +1520,25 @@ GOMP_target_update_ext (int device, size
 	      && thr->ts.team
 	      && !thr->task->final_task)
 	    {
-	      gomp_create_target_task (devicep, (void (*) (void *)) NULL,
-				       mapnum, hostaddrs, sizes, kinds,
-				       flags | GOMP_TARGET_FLAG_UPDATE,
-				       depend);
-	      return;
+	      if (gomp_create_target_task (devicep, (void (*) (void *)) NULL,
+					   mapnum, hostaddrs, sizes, kinds,
+					   flags | GOMP_TARGET_FLAG_UPDATE,
+					   depend, GOMP_TARGET_TASK_DATA))
+		return;
 	    }
+	  else
+	    {
+	      struct gomp_team *team = thr->ts.team;
+	      /* If parallel or taskgroup has been cancelled, don't start new
+		 tasks.  */
+	      if (team
+		  && (gomp_team_barrier_cancelled (&team->barrier)
+		      || (thr->task->taskgroup
+			  && thr->task->taskgroup->cancelled)))
+		return;
 
-	  struct gomp_team *team = thr->ts.team;
-	  /* If parallel or taskgroup has been cancelled, don't start new
-	     tasks.  */
-	  if (team
-	      && (gomp_team_barrier_cancelled (&team->barrier)
-		  || (thr->task->taskgroup
-		      && thr->task->taskgroup->cancelled)))
-	    return;
-
-	  gomp_task_maybe_wait_for_dependencies (depend);
+	      gomp_task_maybe_wait_for_dependencies (depend);
+	    }
 	}
     }
 
@@ -1647,22 +1642,25 @@ GOMP_target_enter_exit_data (int device,
 	      && thr->ts.team
 	      && !thr->task->final_task)
 	    {
-	      gomp_create_target_task (devicep, (void (*) (void *)) NULL,
-				       mapnum, hostaddrs, sizes, kinds,
-				       flags, depend);
-	      return;
+	      if (gomp_create_target_task (devicep, (void (*) (void *)) NULL,
+					   mapnum, hostaddrs, sizes, kinds,
+					   flags, depend,
+					   GOMP_TARGET_TASK_DATA))
+		return;
 	    }
+	  else
+	    {
+	      struct gomp_team *team = thr->ts.team;
+	      /* If parallel or taskgroup has been cancelled, don't start new
+		 tasks.  */
+	      if (team
+		  && (gomp_team_barrier_cancelled (&team->barrier)
+		      || (thr->task->taskgroup
+			  && thr->task->taskgroup->cancelled)))
+		return;
 
-	  struct gomp_team *team = thr->ts.team;
-	  /* If parallel or taskgroup has been cancelled, don't start new
-	     tasks.  */
-	  if (team
-	      && (gomp_team_barrier_cancelled (&team->barrier)
-		  || (thr->task->taskgroup
-		      && thr->task->taskgroup->cancelled)))
-	    return;
-
-	  gomp_task_maybe_wait_for_dependencies (depend);
+	      gomp_task_maybe_wait_for_dependencies (depend);
+	    }
 	}
     }
 
@@ -1694,38 +1692,65 @@ GOMP_target_enter_exit_data (int device,
     gomp_exit_data (devicep, mapnum, hostaddrs, sizes, kinds);
 }
 
-void
+bool
 gomp_target_task_fn (void *data)
 {
   struct gomp_target_task *ttask = (struct gomp_target_task *) data;
+  struct gomp_device_descr *devicep = ttask->devicep;
+
   if (ttask->fn != NULL)
     {
-      /* GOMP_target_ext */
-    }
-  else if (ttask->devicep == NULL
-	   || !(ttask->devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
-    return;
+      if (devicep == NULL
+	  || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+	{
+	  ttask->state = GOMP_TARGET_TASK_FALLBACK;
+	  gomp_target_fallback_firstprivate (ttask->fn, ttask->mapnum,
+					     ttask->hostaddrs, ttask->sizes,
+					     ttask->kinds);
+	  return false;
+	}
+
+      if (ttask->state == GOMP_TARGET_TASK_RUNNING)
+	{
+	  gomp_unmap_vars (ttask->tgt, true);
+	  return false;
+	}
+
+      void *fn_addr = gomp_get_target_fn_addr (devicep, ttask->fn);
+      ttask->tgt
+	= gomp_map_vars (devicep, ttask->mapnum, ttask->hostaddrs, NULL,
+			 ttask->sizes, ttask->kinds, true,
+			 GOMP_MAP_VARS_TARGET);
+      ttask->state = GOMP_TARGET_TASK_RUNNING;
+
+      devicep->async_run_func (devicep->target_id, fn_addr,
+			       (void *) ttask->tgt->tgt_start, (void *) ttask);
+      return true;
+    }
+  else if (devicep == NULL
+	   || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+    return false;
 
   size_t i;
   if (ttask->flags & GOMP_TARGET_FLAG_UPDATE)
-    gomp_update (ttask->devicep, ttask->mapnum, ttask->hostaddrs, ttask->sizes,
+    gomp_update (devicep, ttask->mapnum, ttask->hostaddrs, ttask->sizes,
 		 ttask->kinds, true);
   else if ((ttask->flags & GOMP_TARGET_FLAG_EXIT_DATA) == 0)
     for (i = 0; i < ttask->mapnum; i++)
       if ((ttask->kinds[i] & 0xff) == GOMP_MAP_STRUCT)
 	{
-	  gomp_map_vars (ttask->devicep, ttask->sizes[i] + 1,
-			 &ttask->hostaddrs[i], NULL, &ttask->sizes[i],
-			 &ttask->kinds[i], true, GOMP_MAP_VARS_ENTER_DATA);
+	  gomp_map_vars (devicep, ttask->sizes[i] + 1, &ttask->hostaddrs[i],
+			 NULL, &ttask->sizes[i], &ttask->kinds[i], true,
+			 GOMP_MAP_VARS_ENTER_DATA);
 	  i += ttask->sizes[i];
 	}
       else
-	gomp_map_vars (ttask->devicep, 1, &ttask->hostaddrs[i], NULL,
-		       &ttask->sizes[i], &ttask->kinds[i],
-		       true, GOMP_MAP_VARS_ENTER_DATA);
+	gomp_map_vars (devicep, 1, &ttask->hostaddrs[i], NULL, &ttask->sizes[i],
+		       &ttask->kinds[i], true, GOMP_MAP_VARS_ENTER_DATA);
   else
-    gomp_exit_data (ttask->devicep, ttask->mapnum, ttask->hostaddrs,
-		    ttask->sizes, ttask->kinds);
+    gomp_exit_data (devicep, ttask->mapnum, ttask->hostaddrs, ttask->sizes,
+		    ttask->kinds);
+  return false;
 }
 
 void
@@ -2170,6 +2195,7 @@ gomp_load_plugin_for_device (struct gomp
   if (device->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400)
     {
       DLSYM (run);
+      DLSYM (async_run);
       DLSYM (dev2dev);
     }
   if (device->capabilities & GOMP_OFFLOAD_CAP_OPENACC_200)
--- libgomp/libgomp.map.jj	2015-10-26 15:38:20.000000000 +0100
+++ libgomp/libgomp.map	2015-11-11 16:15:23.807818735 +0100
@@ -406,3 +406,8 @@ GOMP_PLUGIN_1.0 {
 	GOMP_PLUGIN_async_unmap_vars;
 	GOMP_PLUGIN_acc_thread;
 };
+
+GOMP_PLUGIN_1.1 {
+  global:
+	GOMP_PLUGIN_target_task_completion;
+} GOMP_PLUGIN_1.0;
--- libgomp/task.c.jj	2015-11-09 11:14:37.332239862 +0100
+++ libgomp/task.c	2015-11-11 16:42:43.749898088 +0100
@@ -482,11 +482,12 @@ ialias (GOMP_taskgroup_end)
 
 /* Called for nowait target tasks.  */
 
-void
+bool
 gomp_create_target_task (struct gomp_device_descr *devicep,
 			 void (*fn) (void *), size_t mapnum, void **hostaddrs,
 			 size_t *sizes, unsigned short *kinds,
-			 unsigned int flags, void **depend)
+			 unsigned int flags, void **depend,
+			 enum gomp_target_task_state state)
 {
   struct gomp_thread *thr = gomp_thread ();
   struct gomp_team *team = thr->ts.team;
@@ -495,7 +496,7 @@ gomp_create_target_task (struct gomp_dev
   if (team
       && (gomp_team_barrier_cancelled (&team->barrier)
 	  || (thr->task->taskgroup && thr->task->taskgroup->cancelled)))
-    return;
+    return true;
 
   struct gomp_target_task *ttask;
   struct gomp_task *task;
@@ -503,19 +504,44 @@ gomp_create_target_task (struct gomp_dev
   struct gomp_taskgroup *taskgroup = parent->taskgroup;
   bool do_wake;
   size_t depend_size = 0;
+  uintptr_t depend_cnt = 0;
+  size_t tgt_align = 0, tgt_size = 0;
 
   if (depend != NULL)
-    depend_size = ((uintptr_t) depend[0]
-		   * sizeof (struct gomp_task_depend_entry));
+    {
+      depend_cnt = (uintptr_t) depend[0];
+      depend_size = depend_cnt * sizeof (struct gomp_task_depend_entry);
+    }
+  if (fn)
+    {
+      /* GOMP_MAP_FIRSTPRIVATE need to be copied first, as they are
+	 firstprivate on the target task.  */
+      size_t i;
+      for (i = 0; i < mapnum; i++)
+	if ((kinds[i] & 0xff) == GOMP_MAP_FIRSTPRIVATE)
+	  {
+	    size_t align = (size_t) 1 << (kinds[i] >> 8);
+	    if (tgt_align < align)
+	      tgt_align = align;
+	    tgt_size = (tgt_size + align - 1) & ~(align - 1);
+	    tgt_size += sizes[i];
+	  }
+      if (tgt_align)
+	tgt_size += tgt_align - 1;
+      else
+	tgt_size = 0;
+    }
+
   task = gomp_malloc (sizeof (*task) + depend_size
 		      + sizeof (*ttask)
 		      + mapnum * (sizeof (void *) + sizeof (size_t)
-				  + sizeof (unsigned short)));
+				  + sizeof (unsigned short))
+		      + tgt_size);
   gomp_init_task (task, parent, gomp_icv (false));
   task->kind = GOMP_TASK_WAITING;
   task->in_tied_task = parent->in_tied_task;
   task->taskgroup = taskgroup;
-  ttask = (struct gomp_target_task *) &task->depend[(uintptr_t) depend[0]];
+  ttask = (struct gomp_target_task *) &task->depend[depend_cnt];
   ttask->devicep = devicep;
   ttask->fn = fn;
   ttask->mapnum = mapnum;
@@ -524,8 +550,29 @@ gomp_create_target_task (struct gomp_dev
   memcpy (ttask->sizes, sizes, mapnum * sizeof (size_t));
   ttask->kinds = (unsigned short *) &ttask->sizes[mapnum];
   memcpy (ttask->kinds, kinds, mapnum * sizeof (unsigned short));
+  if (tgt_align)
+    {
+      char *tgt = (char *) &ttask->kinds[mapnum];
+      size_t i;
+      uintptr_t al = (uintptr_t) tgt & (tgt_align - 1);
+      if (al)
+	tgt += tgt_align - al;
+      tgt_size = 0;
+      for (i = 0; i < mapnum; i++)
+	if ((kinds[i] & 0xff) == GOMP_MAP_FIRSTPRIVATE)
+	  {
+	    size_t align = (size_t) 1 << (kinds[i] >> 8);
+	    tgt_size = (tgt_size + align - 1) & ~(align - 1);
+	    memcpy (tgt + tgt_size, hostaddrs[i], sizes[i]); 
+	    ttask->hostaddrs[i] = tgt + tgt_size;
+	    tgt_size = tgt_size + sizes[i];
+	  }
+    }
   ttask->flags = flags;
-  task->fn = gomp_target_task_fn;
+  ttask->state = state;
+  ttask->task = task;
+  ttask->team = team;
+  task->fn = NULL;
   task->fn_data = ttask;
   task->final_task = 0;
   gomp_mutex_lock (&team->task_lock);
@@ -536,19 +583,26 @@ gomp_create_target_task (struct gomp_dev
       gomp_mutex_unlock (&team->task_lock);
       gomp_finish_task (task);
       free (task);
-      return;
+      return true;
     }
-  if (taskgroup)
-    taskgroup->num_children++;
   if (depend_size)
     {
       gomp_task_handle_depend (task, parent, depend);
       if (task->num_dependees)
 	{
 	  gomp_mutex_unlock (&team->task_lock);
-	  return;
+	  return true;
 	}
     }
+  if (state == GOMP_TARGET_TASK_DATA)
+    {
+      gomp_mutex_unlock (&team->task_lock);
+      gomp_finish_task (task);
+      free (task);
+      return false;
+    }
+  if (taskgroup)
+    taskgroup->num_children++;
   priority_queue_insert (PQ_CHILDREN, &parent->children_queue, task, 0,
 			 PRIORITY_INSERT_BEGIN,
 			 /*adjust_parent_depends_on=*/false,
@@ -570,6 +624,95 @@ gomp_create_target_task (struct gomp_dev
   gomp_mutex_unlock (&team->task_lock);
   if (do_wake)
     gomp_team_barrier_wake (&team->barrier, 1);
+  return true;
+}
+
+static void inline
+priority_queue_move_task_first (enum priority_queue_type type,
+				struct priority_queue *head,
+				struct gomp_task *task)
+{
+#if _LIBGOMP_CHECKING_
+  if (!priority_queue_task_in_queue_p (type, head, task))
+    gomp_fatal ("Attempt to move first missing task %p", task);
+#endif
+  struct priority_list *list;
+  if (priority_queue_multi_p (head))
+    {
+      list = priority_queue_lookup_priority (head, task->priority);
+#if _LIBGOMP_CHECKING_
+      if (!list)
+	gomp_fatal ("Unable to find priority %d", task->priority);
+#endif
+    }
+  else
+    list = &head->l;
+  priority_list_remove (list, task_to_priority_node (type, task), 0);
+  priority_list_insert (type, list, task, task->priority,
+			PRIORITY_INSERT_BEGIN, type == PQ_CHILDREN,
+			task->parent_depends_on);
+}
+
+/* Signal that a target task TTASK has completed the asynchronously
+   running phase and should be requeued as a task to handle the
+   variable unmapping.  */
+
+void
+GOMP_PLUGIN_target_task_completion (void *data)
+{
+  struct gomp_target_task *ttask = (struct gomp_target_task *) data;
+  struct gomp_task *task = ttask->task;
+  struct gomp_team *team = ttask->team;
+
+  gomp_mutex_lock (&team->task_lock);
+  struct gomp_task *parent = task->parent;
+  if (parent)
+    priority_queue_move_task_first (PQ_CHILDREN, &parent->children_queue,
+				    task);
+
+  struct gomp_taskgroup *taskgroup = task->taskgroup;
+  if (taskgroup)
+    priority_queue_move_task_first (PQ_TASKGROUP, &taskgroup->taskgroup_queue,
+				    task);
+
+  priority_queue_insert (PQ_TEAM, &team->task_queue, task, task->priority,
+			 PRIORITY_INSERT_BEGIN, false,
+			 task->parent_depends_on);
+  task->kind = GOMP_TASK_WAITING;
+  if (parent && parent->taskwait)
+    {
+      if (parent->taskwait->in_taskwait)
+	{
+	  /* One more task has had its dependencies met.
+	     Inform any waiters.  */
+	  parent->taskwait->in_taskwait = false;
+	  gomp_sem_post (&parent->taskwait->taskwait_sem);
+	}
+      else if (parent->taskwait->in_depend_wait)
+	{
+	  /* One more task has had its dependencies met.
+	     Inform any waiters.  */
+	  parent->taskwait->in_depend_wait = false;
+	  gomp_sem_post (&parent->taskwait->taskwait_sem);
+	}
+    }
+  if (taskgroup && taskgroup->in_taskgroup_wait)
+    {
+      /* One more task has had its dependencies met.
+	 Inform any waiters.  */
+      taskgroup->in_taskgroup_wait = false;
+      gomp_sem_post (&taskgroup->taskgroup_sem);
+    }
+
+  ++team->task_queued_count;
+  gomp_team_barrier_set_task_pending (&team->barrier);
+  /* I'm afraid this can't be done after releasing team->task_lock,
+     as gomp_target_task_completion is run from unrelated thread and
+     therefore in between gomp_mutex_unlock and gomp_team_barrier_wake
+     the team could be gone already.  */
+  if (team->nthreads > team->task_running_count)
+    gomp_team_barrier_wake (&team->barrier, 1);
+  gomp_mutex_unlock (&team->task_lock);
 }
 
 /* Given a parent_depends_on task in LIST, move it to the front of its
@@ -1041,7 +1184,20 @@ gomp_barrier_handle_tasks (gomp_barrier_
       if (child_task)
 	{
 	  thr->task = child_task;
-	  child_task->fn (child_task->fn_data);
+	  if (__builtin_expect (child_task->fn == NULL, 0))
+	    {
+	      if (gomp_target_task_fn (child_task->fn_data))
+		{
+		  thr->task = task;
+		  gomp_mutex_lock (&team->task_lock);
+		  child_task->kind = GOMP_TASK_ASYNC_RUNNING;
+		  team->task_running_count--;
+		  child_task = NULL;
+		  continue;
+		}
+	    }
+	  else
+	    child_task->fn (child_task->fn_data);
 	  thr->task = task;
 	}
       else
@@ -1170,7 +1326,19 @@ GOMP_taskwait (void)
       if (child_task)
 	{
 	  thr->task = child_task;
-	  child_task->fn (child_task->fn_data);
+	  if (__builtin_expect (child_task->fn == NULL, 0))
+	    {
+	      if (gomp_target_task_fn (child_task->fn_data))
+		{
+		  thr->task = task;
+		  gomp_mutex_lock (&team->task_lock);
+		  child_task->kind = GOMP_TASK_ASYNC_RUNNING;
+		  child_task = NULL;
+		  continue;
+		}
+	    }
+	  else
+	    child_task->fn (child_task->fn_data);
 	  thr->task = task;
 	}
       else
@@ -1342,7 +1510,19 @@ gomp_task_maybe_wait_for_dependencies (v
       if (child_task)
 	{
 	  thr->task = child_task;
-	  child_task->fn (child_task->fn_data);
+	  if (__builtin_expect (child_task->fn == NULL, 0))
+	    {
+	      if (gomp_target_task_fn (child_task->fn_data))
+		{
+		  thr->task = task;
+		  gomp_mutex_lock (&team->task_lock);
+		  child_task->kind = GOMP_TASK_ASYNC_RUNNING;
+		  child_task = NULL;
+		  continue;
+		}
+	    }
+	  else
+	    child_task->fn (child_task->fn_data);
 	  thr->task = task;
 	}
       else
@@ -1450,8 +1630,8 @@ GOMP_taskgroup_end (void)
 		= priority_queue_next_task (PQ_CHILDREN, &task->children_queue,
 					    PQ_TEAM, &team->task_queue,
 					    &unused);
-            }
-          else
+	    }
+	  else
 	    {
 	      gomp_mutex_unlock (&team->task_lock);
 	      if (to_free)
@@ -1506,7 +1686,19 @@ GOMP_taskgroup_end (void)
       if (child_task)
 	{
 	  thr->task = child_task;
-	  child_task->fn (child_task->fn_data);
+	  if (__builtin_expect (child_task->fn == NULL, 0))
+	    {
+	      if (gomp_target_task_fn (child_task->fn_data))
+		{
+		  thr->task = task;
+		  gomp_mutex_lock (&team->task_lock);
+		  child_task->kind = GOMP_TASK_ASYNC_RUNNING;
+		  child_task = NULL;
+		  continue;
+		}
+	    }
+	  else
+	    child_task->fn (child_task->fn_data);
 	  thr->task = task;
 	}
       else
--- libgomp/priority_queue.c.jj	2015-11-09 11:15:33.000000000 +0100
+++ libgomp/priority_queue.c	2015-11-10 17:52:33.769414428 +0100
@@ -85,7 +85,7 @@ priority_queue_task_in_queue_p (enum pri
    order.  LIST is a priority list of type TYPE.
 
    The expected order is that GOMP_TASK_WAITING tasks come before
-   GOMP_TASK_TIED ones.
+   GOMP_TASK_TIED/GOMP_TASK_ASYNC_RUNNING ones.
 
    If CHECK_DEPS is TRUE, we also check that parent_depends_on WAITING
    tasks come before !parent_depends_on WAITING tasks.  This is only
@@ -104,7 +104,7 @@ priority_list_verify (enum priority_queu
       struct gomp_task *t = priority_node_to_task (type, p);
       if (seen_tied && t->kind == GOMP_TASK_WAITING)
 	gomp_fatal ("priority_queue_verify: WAITING task after TIED");
-      if (t->kind == GOMP_TASK_TIED)
+      if (t->kind >= GOMP_TASK_TIED)
 	seen_tied = true;
       else if (check_deps && t->kind == GOMP_TASK_WAITING)
 	{
--- libgomp/libgomp.h.jj	2015-11-09 11:14:37.326239947 +0100
+++ libgomp/libgomp.h	2015-11-11 10:46:40.143794155 +0100
@@ -373,7 +373,12 @@ enum gomp_task_kind
   /* Task created by GOMP_task and waiting to be run.  */
   GOMP_TASK_WAITING,
   /* Task currently executing or scheduled and about to execute.  */
-  GOMP_TASK_TIED
+  GOMP_TASK_TIED,
+  /* Used for target tasks that have vars mapped and async run started,
+     but not yet completed.  Once that completes, they will be readded
+     into the queues as GOMP_TASK_WAITING in order to perform the var
+     unmapping.  */
+  GOMP_TASK_ASYNC_RUNNING
 };
 
 struct gomp_task_depend_entry
@@ -453,6 +458,8 @@ struct gomp_task
   struct gomp_task_depend_entry depend[];
 };
 
+/* This structure describes a single #pragma omp taskgroup.  */
+
 struct gomp_taskgroup
 {
   struct gomp_taskgroup *prev;
@@ -464,6 +471,8 @@ struct gomp_taskgroup
   size_t num_children;
 };
 
+/* This structure describes a target task.  */
+
 struct gomp_target_task
 {
   struct gomp_device_descr *devicep;
@@ -472,6 +481,10 @@ struct gomp_target_task
   size_t *sizes;
   unsigned short *kinds;
   unsigned int flags;
+  enum gomp_target_task_state state;
+  struct target_mem_desc *tgt;
+  struct gomp_task *task;
+  struct gomp_team *team;
   void *hostaddrs[];
 };
 
@@ -723,10 +736,10 @@ extern void gomp_init_task (struct gomp_
 extern void gomp_end_task (void);
 extern void gomp_barrier_handle_tasks (gomp_barrier_state_t);
 extern void gomp_task_maybe_wait_for_dependencies (void **);
-extern void gomp_create_target_task (struct gomp_device_descr *,
+extern bool gomp_create_target_task (struct gomp_device_descr *,
 				     void (*) (void *), size_t, void **,
 				     size_t *, unsigned short *, unsigned int,
-				     void **);
+				     void **, enum gomp_target_task_state);
 
 static void inline
 gomp_finish_task (struct gomp_task *task)
@@ -747,7 +760,7 @@ extern void gomp_free_thread (void *);
 
 extern void gomp_init_targets_once (void);
 extern int gomp_get_num_devices (void);
-extern void gomp_target_task_fn (void *);
+extern bool gomp_target_task_fn (void *);
 
 /* Splay tree definitions.  */
 typedef struct splay_tree_node_s *splay_tree_node;
@@ -901,6 +914,7 @@ struct gomp_device_descr
   void *(*host2dev_func) (int, void *, const void *, size_t);
   void *(*dev2dev_func) (int, void *, const void *, size_t);
   void (*run_func) (int, void *, void *);
+  void (*async_run_func) (int, void *, void *, void *);
 
   /* Splay tree containing information about mapped memory regions.  */
   struct splay_tree_s mem_map;


	Jakub

^ permalink raw reply	[flat|nested] 23+ messages in thread

* Re: [gomp4.5] depend nowait support for target
  2015-11-11 16:52       ` [gomp4.5] depend nowait support for target Jakub Jelinek
@ 2015-11-12 17:44         ` Ilya Verbin
  2015-11-12 17:58           ` Jakub Jelinek
  2015-11-12 17:45         ` Jakub Jelinek
  1 sibling, 1 reply; 23+ messages in thread
From: Ilya Verbin @ 2015-11-12 17:44 UTC (permalink / raw)
  To: Jakub Jelinek
  Cc: Aldy Hernandez, gcc-patches, Kirill Yukhin, Thomas Schwinge,
	Alexander Monakov, Martin Jambor

On Wed, Nov 11, 2015 at 17:52:22 +0100, Jakub Jelinek wrote:
> On Mon, Oct 19, 2015 at 10:47:54PM +0300, Ilya Verbin wrote:
> > So, here is what I have for now.  Attached target-29.c testcase works fine with
> > MIC emul, however I don't know how to (and where) properly check for completion
> > of async execution on target.  And, similarly, where to do unmapping after that?
> > Do we need a callback from plugin to libgomp (as far as I understood, PTX
> > runtime supports this, but HSA doesn't), or libgomp will just check for
> > ttask->is_completed in task.c?
> 
> Here is the patch updated to have a task.c defined function that the plugin
> can call upon completion of async offloading exection.

Thanks.

> The testsuite coverage will need to improve, the testcase is wrong
> (contains data races - if you want to test parallel running of two target
> regions that both touch the same var, I'd say best would be to use
> #pragma omp atomic and or in 4 in one case and 1 in another case, then
> test if result is 5 (and similarly for the other var).
> Also, with the usleeps Alex Monakov will be unhappy because PTX newlib does
> not have it, but we'll need to find some solution for that.
> 
> Another thing to work on beyond testsuite coverage (it is desirable to test
> nowait target tasks (both depend and without depend) being awaited in all
> the various waiting spots, i.e. end of parallel, barrier, taskwait, end of
> taskgroup, or if (0) task with depend clause waiting on that.
> 
> Also, I wonder what to do if #pragma omp target nowait is used outside of
> (host) parallel - when team is NULL.  All the tasking code in that case just
> executes tasks undeferred, which is fine for all but target nowait - there
> it is I'd say useful to be able to run a single host thread concurrently
> with some async offloading tasks.  So, I wonder if in that case,
> if we encounter target nowait with team == NULL, should not just create a
> dummy non-active (nthreads == 1) team, as if there was #pragma omp parallel
> if (0) starting above it and ending at program's end.  In OpenMP, the
> program's initial thread is implicitly surrounded by inactive parallel, so
> this isn't anything against the OpenMP execution model.  But we'd need to
> free the team somewhere in a destructor.
>
> Can you please try to cleanup the liboffloadmic side of this, so that
> a callback instead of hardcoded __gomp_offload_intelmic_async_completed call
> is used?

Do you mean something like the patch bellow?  I'll discuss it with liboffloadmic
maintainers.

> Can you make sure it works on XeonPhi non-emulated too?

I'm trying to do it, but it will take some time...

Unfortunately, target-32.c fails for me using emulation mode:

Program received signal SIGSEGV, Segmentation fault.
#0  0x00007ff4ab1265ed in priority_list_remove (list=0x0, node=0x7ff49001afa0, model=MEMMODEL_RELAXED) at libgomp/priority_queue.h:422
#1  0x00007ff4ab1266d9 in priority_tree_remove (type=PQ_CHILDREN, head=0x1883138, node=0x7ff49001afa0) at libgomp/priority_queue.c:195
#2  0x00007ff4ab10fa06 in priority_queue_remove (type=PQ_CHILDREN, head=0x1883138, task=0x7ff49001af30, model=MEMMODEL_RELAXED) at libgomp/priority_queue.h:468
#3  0x00007ff4ab11570d in gomp_task_maybe_wait_for_dependencies (depend=0x7ff49b0d9de0) at libgomp/task.c:1539
#4  0x00007ff4ab11fd46 in GOMP_target_enter_exit_data (device=-1, mapnum=3, hostaddrs=0x7ff49b0d9dc0, sizes=0x6020b0 <.omp_data_sizes.38>, kinds=0x6020a0 <.omp_data_kinds.39>, flags=2, depend=0x7ff49b0d9de0) at libgomp/target.c:1662
#5  0x00000000004011f9 in main._omp_fn ()
#6  0x00007ff4ab1160f3 in gomp_thread_start (xdata=0x7fffe93766a0) at libgomp/team.c:119
#7  0x0000003b07e07ee5 in start_thread () from /lib64/libpthread.so.0
#8  0x0000003b076f4b8d in clone () from /lib64/libc.so.6

However when I manually run commands from testsuite/libgomp.log under the same
environment, it passes.  Don't know where is the difference.

Also I tried to replace 'b = 4;' and 'b = 5;' with infinite loops, but got only
100% CPU usage in offload_target_main instead of 200%, so it seems that only one
target task is running concurrently.


diff --git a/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp b/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp
index 6da09b1..772e198 100644
--- a/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp
+++ b/liboffloadmic/plugin/libgomp-plugin-intelmic.cpp
@@ -220,6 +220,10 @@ static void
 register_main_image ()
 {
   __offload_register_image (&main_target_image);
+
+  /* liboffloadmic will call GOMP_PLUGIN_target_task_completion when
+     asynchronous task on target is completed.  */
+  __offload_register_task_callback (GOMP_PLUGIN_target_task_completion);
 }
 
 /* liboffloadmic loads and runs offload_target_main on all available devices
@@ -537,13 +541,3 @@ GOMP_OFFLOAD_run (int device, void *tgt_fn, void *tgt_vars)
 
   GOMP_OFFLOAD_async_run (device, tgt_fn, tgt_vars, NULL);
 }
-
-/* Called by liboffloadmic when asynchronous function is completed.  */
-
-extern "C" void
-__gomp_offload_intelmic_async_completed (const void *async_data)
-{
-  TRACE ("(async_data = %p)", async_data);
-
-  GOMP_PLUGIN_target_task_completion ((void *) async_data);
-}
diff --git a/liboffloadmic/runtime/offload_host.cpp b/liboffloadmic/runtime/offload_host.cpp
index a150410..e52019d 100644
--- a/liboffloadmic/runtime/offload_host.cpp
+++ b/liboffloadmic/runtime/offload_host.cpp
@@ -64,8 +64,7 @@ static void __offload_fini_library(void);
 #define GET_OFFLOAD_NUMBER(timer_data) \
     timer_data? timer_data->offload_number : 0
 
-extern "C" void
-__gomp_offload_intelmic_async_completed (const void *);
+static void (*task_completion_callback)(void *);
 
 extern "C" {
 #ifdef TARGET_WINNT
@@ -2510,7 +2509,7 @@ extern "C" {
         const void *info
     )
     {
-	__gomp_offload_intelmic_async_completed (info);
+	task_completion_callback ((void *) info);
     }
 }
 
@@ -5672,6 +5671,11 @@ extern "C" void __offload_unregister_image(const void *target_image)
     }
 }
 
+extern "C" void __offload_register_task_callback(void (*cb)(void *))
+{
+    task_completion_callback = cb;
+}
+
 // Runtime trace interface for user programs
 
 void __offload_console_trace(int level)
diff --git a/liboffloadmic/runtime/offload_host.h b/liboffloadmic/runtime/offload_host.h
index afd5c99..2a43fd6 100644
--- a/liboffloadmic/runtime/offload_host.h
+++ b/liboffloadmic/runtime/offload_host.h
@@ -376,6 +376,9 @@ extern "C" bool __offload_target_image_is_executable(const void *target_image);
 extern "C" bool __offload_register_image(const void* image);
 extern "C" void __offload_unregister_image(const void* image);
 
+// Registers asynchronous task completion callback
+extern "C" void __offload_register_task_callback(void (*cb)(void *));
+
 // Initializes offload runtime library.
 DLL_LOCAL extern int __offload_init_library(void);
 

  -- Ilya

^ permalink raw reply	[flat|nested] 23+ messages in thread

* Re: [gomp4.5] depend nowait support for target
  2015-11-11 16:52       ` [gomp4.5] depend nowait support for target Jakub Jelinek
  2015-11-12 17:44         ` Ilya Verbin
@ 2015-11-12 17:45         ` Jakub Jelinek
  2015-11-12 20:52           ` Ilya Verbin
  1 sibling, 1 reply; 23+ messages in thread
From: Jakub Jelinek @ 2015-11-12 17:45 UTC (permalink / raw)
  To: Ilya Verbin
  Cc: Aldy Hernandez, gcc-patches, Kirill Yukhin, Thomas Schwinge,
	Alexander Monakov, Martin Jambor

Hi!

Here is updated patch with the team == NULL case hopefully handled.
But the testcase I wrote (target-33.c) hangs, the problem is in the
  #pragma omp target nowait map (tofrom: a, b) depend(out: d[3])
  {
    #pragma omp atomic update
    a = a + 9;
    b -= 8;
  }
  #pragma omp target nowait map (tofrom: a, c) depend(out: d[4])
  {
    #pragma omp atomic update
    a = a + 4;
    c >>= 1;
  }
  #pragma omp task if (0) depend (in: d[3], d[4])
  if (a != 50 || b != 4 || c != 20)
    abort ();
part, where (I should change that for the case of no dependencies
eventually) the task with map_vars+async_run is queued in both cases,
then we reach GOMP_task, which calls gomp_task_maybe_wait_for_dependencies
which spawns the first half task (map_vars+async_run), and then
the second half task (map_vars+async_run), but that one gets stuck somewhere
in liboffloadmic, then some other thread (from liboffloadmic) calls
GOMP_PLUGIN_target_task_completion and enqueues the second half of the first
target task (unmap_vars), but as the only normal thread in the main program
is stuck in liboffloadmic (during gomp_map_vars, trying to allocate
target memory in the plugin), there is no thread to schedule the second half
of first target task.  So, if liboffloadmic is stuck waiting for unmap_vars,
it is a deadlock.  Can you please try to debug this?
I'll try tomorrow another testcase like target-33.c, but with
#pragma omp parallel 
#pragma omp single
around everything in main, both with OMP_NUM_THREADS=16 and 1, for 1 I would
expect it would be the same though.

--- liboffloadmic/runtime/offload_host.cpp.jj	2015-11-05 11:31:05.013916598 +0100
+++ liboffloadmic/runtime/offload_host.cpp	2015-11-10 12:58:55.090951303 +0100
@@ -64,6 +64,9 @@ static void __offload_fini_library(void)
 #define GET_OFFLOAD_NUMBER(timer_data) \
     timer_data? timer_data->offload_number : 0
 
+extern "C" void
+__gomp_offload_intelmic_async_completed (const void *);
+
 extern "C" {
 #ifdef TARGET_WINNT
 // Windows does not support imports from libraries without actually
@@ -2507,7 +2510,7 @@ extern "C" {
         const void *info
     )
     {
-	/* TODO: Call callback function, pass info.  */
+	__gomp_offload_intelmic_async_completed (info);
     }
 }
 
--- liboffloadmic/plugin/libgomp-plugin-intelmic.cpp.jj	2015-10-14 10:24:10.922194230 +0200
+++ liboffloadmic/plugin/libgomp-plugin-intelmic.cpp	2015-11-11 15:48:55.428967827 +0100
@@ -192,11 +192,23 @@ GOMP_OFFLOAD_get_num_devices (void)
 
 static void
 offload (const char *file, uint64_t line, int device, const char *name,
-	 int num_vars, VarDesc *vars, VarDesc2 *vars2)
+	 int num_vars, VarDesc *vars, VarDesc2 *vars2, const void **async_data)
 {
   OFFLOAD ofld = __offload_target_acquire1 (&device, file, line);
   if (ofld)
-    __offload_offload1 (ofld, name, 0, num_vars, vars, vars2, 0, NULL, NULL);
+    {
+      if (async_data == NULL)
+	__offload_offload1 (ofld, name, 0, num_vars, vars, vars2, 0, NULL,
+			    NULL);
+      else
+	{
+	  OffloadFlags flags;
+	  flags.flags = 0;
+	  flags.bits.omp_async = 1;
+	  __offload_offload3 (ofld, name, 0, num_vars, vars, NULL, 0, NULL,
+			      async_data, 0, NULL, flags, NULL);
+	}
+    }
   else
     {
       fprintf (stderr, "%s:%d: Offload target acquire failed\n", file, line);
@@ -218,7 +230,7 @@ GOMP_OFFLOAD_init_device (int device)
   TRACE ("");
   pthread_once (&main_image_is_registered, register_main_image);
   offload (__FILE__, __LINE__, device, "__offload_target_init_proc", 0,
-	   NULL, NULL);
+	   NULL, NULL, NULL);
 }
 
 extern "C" void
@@ -240,7 +252,7 @@ get_target_table (int device, int &num_f
   VarDesc2 vd1g[2] = { { "num_funcs", 0 }, { "num_vars", 0 } };
 
   offload (__FILE__, __LINE__, device, "__offload_target_table_p1", 2,
-	   vd1, vd1g);
+	   vd1, vd1g, NULL);
 
   int table_size = num_funcs + 2 * num_vars;
   if (table_size > 0)
@@ -254,7 +266,7 @@ get_target_table (int device, int &num_f
       VarDesc2 vd2g = { "table", 0 };
 
       offload (__FILE__, __LINE__, device, "__offload_target_table_p2", 1,
-	       &vd2, &vd2g);
+	       &vd2, &vd2g, NULL);
     }
 }
 
@@ -401,8 +413,8 @@ GOMP_OFFLOAD_alloc (int device, size_t s
   vd1[1].size = sizeof (void *);
   VarDesc2 vd1g[2] = { { "size", 0 }, { "tgt_ptr", 0 } };
 
-  offload (__FILE__, __LINE__, device, "__offload_target_alloc", 2, vd1, vd1g);
-
+  offload (__FILE__, __LINE__, device, "__offload_target_alloc", 2, vd1, vd1g,
+	   NULL);
   return tgt_ptr;
 }
 
@@ -416,7 +428,8 @@ GOMP_OFFLOAD_free (int device, void *tgt
   vd1.size = sizeof (void *);
   VarDesc2 vd1g = { "tgt_ptr", 0 };
 
-  offload (__FILE__, __LINE__, device, "__offload_target_free", 1, &vd1, &vd1g);
+  offload (__FILE__, __LINE__, device, "__offload_target_free", 1, &vd1, &vd1g,
+	   NULL);
 }
 
 extern "C" void *
@@ -435,7 +448,7 @@ GOMP_OFFLOAD_host2dev (int device, void
   VarDesc2 vd1g[2] = { { "tgt_ptr", 0 }, { "size", 0 } };
 
   offload (__FILE__, __LINE__, device, "__offload_target_host2tgt_p1", 2,
-	   vd1, vd1g);
+	   vd1, vd1g, NULL);
 
   VarDesc vd2 = vd_host2tgt;
   vd2.ptr = (void *) host_ptr;
@@ -443,7 +456,7 @@ GOMP_OFFLOAD_host2dev (int device, void
   VarDesc2 vd2g = { "var", 0 };
 
   offload (__FILE__, __LINE__, device, "__offload_target_host2tgt_p2", 1,
-	   &vd2, &vd2g);
+	   &vd2, &vd2g, NULL);
 
   return tgt_ptr;
 }
@@ -464,7 +477,7 @@ GOMP_OFFLOAD_dev2host (int device, void
   VarDesc2 vd1g[2] = { { "tgt_ptr", 0 }, { "size", 0 } };
 
   offload (__FILE__, __LINE__, device, "__offload_target_tgt2host_p1", 2,
-	   vd1, vd1g);
+	   vd1, vd1g, NULL);
 
   VarDesc vd2 = vd_tgt2host;
   vd2.ptr = (void *) host_ptr;
@@ -472,7 +485,7 @@ GOMP_OFFLOAD_dev2host (int device, void
   VarDesc2 vd2g = { "var", 0 };
 
   offload (__FILE__, __LINE__, device, "__offload_target_tgt2host_p2", 1,
-	   &vd2, &vd2g);
+	   &vd2, &vd2g, NULL);
 
   return host_ptr;
 }
@@ -495,22 +508,42 @@ GOMP_OFFLOAD_dev2dev (int device, void *
   VarDesc2 vd1g[3] = { { "dst_ptr", 0 }, { "src_ptr", 0 }, { "size", 0 } };
 
   offload (__FILE__, __LINE__, device, "__offload_target_tgt2tgt", 3, vd1,
-	   vd1g);
+	   vd1g, NULL);
 
   return dst_ptr;
 }
 
 extern "C" void
+GOMP_OFFLOAD_async_run (int device, void *tgt_fn, void *tgt_vars,
+			void *async_data)
+{
+  TRACE ("(device = %d, tgt_fn = %p, tgt_vars = %p, async_data = %p)", device,
+	 tgt_fn, tgt_vars, async_data);
+
+  VarDesc vd[2] = { vd_host2tgt, vd_host2tgt };
+  vd[0].ptr = &tgt_fn;
+  vd[0].size = sizeof (void *);
+  vd[1].ptr = &tgt_vars;
+  vd[1].size = sizeof (void *);
+
+  offload (__FILE__, __LINE__, device, "__offload_target_run", 2, vd, NULL,
+	   (const void **) async_data);
+}
+
+extern "C" void
 GOMP_OFFLOAD_run (int device, void *tgt_fn, void *tgt_vars)
 {
-  TRACE ("(tgt_fn = %p, tgt_vars = %p)", tgt_fn, tgt_vars);
+  TRACE ("(device = %d, tgt_fn = %p, tgt_vars = %p)", device, tgt_fn, tgt_vars);
 
-  VarDesc vd1[2] = { vd_host2tgt, vd_host2tgt };
-  vd1[0].ptr = &tgt_fn;
-  vd1[0].size = sizeof (void *);
-  vd1[1].ptr = &tgt_vars;
-  vd1[1].size = sizeof (void *);
-  VarDesc2 vd1g[2] = { { "tgt_fn", 0 }, { "tgt_vars", 0 } };
+  GOMP_OFFLOAD_async_run (device, tgt_fn, tgt_vars, NULL);
+}
+
+/* Called by liboffloadmic when asynchronous function is completed.  */
+
+extern "C" void
+__gomp_offload_intelmic_async_completed (const void *async_data)
+{
+  TRACE ("(async_data = %p)", async_data);
 
-  offload (__FILE__, __LINE__, device, "__offload_target_run", 2, vd1, vd1g);
+  GOMP_PLUGIN_target_task_completion ((void *) async_data);
 }
--- libgomp/parallel.c.jj	2015-10-14 10:24:10.000000000 +0200
+++ libgomp/parallel.c	2015-11-12 15:12:38.349901541 +0100
@@ -85,7 +85,7 @@ gomp_resolve_num_threads (unsigned speci
      nested parallel, so there is just one thread in the
      contention group as well, no need to handle it atomically.  */
   pool = thr->thread_pool;
-  if (thr->ts.team == NULL)
+  if (thr->ts.team == NULL || pool == NULL)
     {
       num_threads = max_num_threads;
       if (num_threads > icv->thread_limit_var)
--- libgomp/libgomp-plugin.h.jj	2015-10-14 10:24:10.000000000 +0200
+++ libgomp/libgomp-plugin.h	2015-11-11 15:48:16.875505434 +0100
@@ -59,10 +59,20 @@ struct addr_pair
   uintptr_t end;
 };
 
+/* Various state of OpenMP async offloading tasks.  */
+enum gomp_target_task_state
+{
+  GOMP_TARGET_TASK_DATA,
+  GOMP_TARGET_TASK_BEFORE_MAP,
+  GOMP_TARGET_TASK_FALLBACK,
+  GOMP_TARGET_TASK_RUNNING
+};
+
 /* Miscellaneous functions.  */
 extern void *GOMP_PLUGIN_malloc (size_t) __attribute__ ((malloc));
 extern void *GOMP_PLUGIN_malloc_cleared (size_t) __attribute__ ((malloc));
 extern void *GOMP_PLUGIN_realloc (void *, size_t);
+void GOMP_PLUGIN_target_task_completion (void *);
 
 extern void GOMP_PLUGIN_debug (int, const char *, ...)
 	__attribute__ ((format (printf, 2, 3)));
--- libgomp/testsuite/libgomp.c/target-32.c.jj	2015-11-10 12:58:55.087951346 +0100
+++ libgomp/testsuite/libgomp.c/target-32.c	2015-11-12 13:28:55.053380366 +0100
@@ -0,0 +1,54 @@
+#include <stdlib.h>
+#include <unistd.h>
+
+int main ()
+{
+  int a = 0, b = 0, c = 0, d[7];
+
+  #pragma omp parallel
+  #pragma omp single
+  {
+    #pragma omp task depend(out: d[0])
+      a = 2;
+
+    #pragma omp target enter data nowait map(to: a,b,c) depend(in: d[0]) depend(out: d[1])
+
+    #pragma omp target nowait map(alloc: a) depend(in: d[1]) depend(out: d[2])
+      a++;
+
+    #pragma omp target nowait map(alloc: b) depend(in: d[2]) depend(out: d[3])
+    {
+      usleep (1000);
+      #pragma omp atomic update
+      b |= 4;
+    }
+
+    #pragma omp target nowait map(alloc: b) depend(in: d[2]) depend(out: d[4])
+    {
+      usleep (5000);
+      #pragma omp atomic update
+      b |= 1;
+    }
+
+    #pragma omp target nowait map(alloc: c) depend(in: d[3], d[4]) depend(out: d[5])
+    {
+      usleep (5000);
+      #pragma omp atomic update
+      c |= 8;
+    }
+
+    #pragma omp target nowait map(alloc: c) depend(in: d[3], d[4]) depend(out: d[6])
+    {
+      usleep (1000);
+      #pragma omp atomic update
+      c |= 2;
+    }
+
+    #pragma omp target exit data map(always,from: a,b,c) depend(in: d[5], d[6])
+  }
+
+  if (a != 3 || b != 5 || c != 10)
+    abort ();
+
+  return 0;
+}
--- libgomp/testsuite/libgomp.c/target-33.c.jj	2015-11-12 16:20:23.332860573 +0100
+++ libgomp/testsuite/libgomp.c/target-33.c	2015-11-12 16:20:14.000000000 +0100
@@ -0,0 +1,93 @@
+extern void abort (void);
+
+int
+main ()
+{
+  int a = 1, b = 2, c = 4, d[7];
+  #pragma omp taskgroup
+  {
+    #pragma omp target enter data nowait map (to: a, b, c) depend(out: d[0])
+    #pragma omp target nowait map (alloc: a, b) depend(in: d[0]) depend(out: d[1])
+    {
+      #pragma omp atomic update
+      a |= 4;
+      #pragma omp atomic update
+      b |= 8;
+    }
+    #pragma omp target nowait map (alloc: a, c) depend(in: d[0]) depend(out: d[2])
+    {
+      #pragma omp atomic update
+      a |= 16;
+      #pragma omp atomic update
+      c |= 32;
+    }
+    #pragma omp target exit data nowait map (from: a, b, c) depend(in: d[1], d[2])
+  }
+  if (a != 21 || b != 10 || c != 36)
+    abort ();
+  #pragma omp target map (tofrom: a, b) nowait
+  {
+    a &= ~16;
+    b &= ~2;
+  }
+  #pragma omp target map (tofrom: c) nowait
+  {
+    c |= 8;
+  }
+  #pragma omp barrier
+  if (a != 5 || b != 8 || c != 44)
+    abort ();
+  #pragma omp target map (tofrom: a, b) nowait
+  {
+    a |= 32;
+    b |= 4;
+  }
+  #pragma omp target map (tofrom: c) nowait
+  {
+    c &= ~4;
+  }
+  #pragma omp taskwait
+  if (a != 37 || b != 12 || c != 40)
+    abort ();
+  #pragma omp target nowait map (tofrom: a, b) depend(out: d[3])
+  {
+    #pragma omp atomic update
+    a = a + 9;
+    b -= 8;
+  }
+  #pragma omp target nowait map (tofrom: a, c) depend(out: d[4])
+  {
+    #pragma omp atomic update
+    a = a + 4;
+    c >>= 1;
+  }
+  #pragma omp task if (0) depend (in: d[3], d[4])
+  if (a != 50 || b != 4 || c != 20)
+    abort ();
+  #pragma omp task
+  a += 50;
+  #pragma omp target nowait map (tofrom: b)
+  b++;
+  #pragma omp target map (tofrom: c) nowait
+  c--;
+  #pragma omp taskwait
+  if (a != 100 || b != 5 || c != 19)
+    abort ();
+  #pragma omp target map (tofrom: a) nowait depend(out: d[5])
+  a++;
+  #pragma omp target map (tofrom: b) nowait depend(out: d[6])
+  b++;
+  #pragma omp target map (tofrom: a, b) depend(in: d[5], d[6])
+  {
+    if (a != 101 || b != 6)
+      a = -9;
+    else
+      {
+	a = 24;
+	b = 38;
+      }
+  }
+  if (a != 24 || b != 38)
+    abort ();
+  return 0;
+}
--- libgomp/team.c.jj	2015-11-09 11:14:37.000000000 +0100
+++ libgomp/team.c	2015-11-12 15:09:23.584644449 +0100
@@ -272,6 +272,8 @@ gomp_free_thread (void *arg __attribute_
       free (pool);
       thr->thread_pool = NULL;
     }
+  if (thr->ts.level == 0 && __builtin_expect (thr->ts.team != NULL, 0))
+    gomp_team_end ();
   if (thr->task != NULL)
     {
       struct gomp_task *task = thr->task;
@@ -301,7 +303,7 @@ gomp_team_start (void (*fn) (void *), vo
   struct gomp_thread **affinity_thr = NULL;
 
   thr = gomp_thread ();
-  nested = thr->ts.team != NULL;
+  nested = thr->ts.level;
   pool = thr->thread_pool;
   task = thr->task;
   icv = task ? &task->icv : &gomp_global_icv;
--- libgomp/target.c.jj	2015-11-09 11:14:37.325239961 +0100
+++ libgomp/target.c	2015-11-12 14:33:56.607481598 +0100
@@ -1348,17 +1348,7 @@ GOMP_target (int device, void (*fn) (voi
   struct target_mem_desc *tgt_vars
     = gomp_map_vars (devicep, mapnum, hostaddrs, NULL, sizes, kinds, false,
 		     GOMP_MAP_VARS_TARGET);
-  struct gomp_thread old_thr, *thr = gomp_thread ();
-  old_thr = *thr;
-  memset (thr, '\0', sizeof (*thr));
-  if (gomp_places_list)
-    {
-      thr->place = old_thr.place;
-      thr->ts.place_partition_len = gomp_places_list_len;
-    }
   devicep->run_func (devicep->target_id, fn_addr, (void *) tgt_vars->tgt_start);
-  gomp_free_thread (thr);
-  *thr = old_thr;
   gomp_unmap_vars (tgt_vars, true);
 }
 
@@ -1387,10 +1377,52 @@ GOMP_target_ext (int device, void (*fn)
   (void) num_teams;
   (void) thread_limit;
 
-  /* If there are depend clauses, but nowait is not present,
-     block the parent task until the dependencies are resolved
-     and then just continue with the rest of the function as if it
-     is a merged task.  */
+  if (flags & GOMP_TARGET_FLAG_NOWAIT)
+    {
+      struct gomp_thread *thr = gomp_thread ();
+      /* Create a team if we don't have any around, as nowait
+	 target tasks make sense to run asynchronously even when
+	 outside of any parallel.  */
+      if (__builtin_expect (thr->ts.team == NULL, 0))
+	{
+	  struct gomp_team *team = gomp_new_team (1);
+	  struct gomp_task *task = thr->task;
+	  struct gomp_task_icv *icv = task ? &task->icv : &gomp_global_icv;
+	  team->prev_ts = thr->ts;
+	  thr->ts.team = team;
+	  thr->ts.team_id = 0;
+	  thr->ts.work_share = &team->work_shares[0];
+	  thr->ts.last_work_share = NULL;
+#ifdef HAVE_SYNC_BUILTINS
+	  thr->ts.single_count = 0;
+#endif
+	  thr->ts.static_trip = 0;
+	  thr->task = &team->implicit_task[0];
+	  gomp_init_task (thr->task, NULL, icv);
+	  if (task)
+	    {
+	      thr->task = task;
+	      gomp_end_task ();
+	      free (task);
+	      thr->task = &team->implicit_task[0];
+	    }
+	  else
+	    pthread_setspecific (gomp_thread_destructor, thr);
+	}
+      if (thr->ts.team
+	  && !thr->task->final_task)
+	{
+	  gomp_create_target_task (devicep, fn, mapnum, hostaddrs,
+				   sizes, kinds, flags, depend,
+				   GOMP_TARGET_TASK_BEFORE_MAP);
+	  return;
+	}
+    }
+
+  /* If there are depend clauses, but nowait is not present
+     (or we are in a final task), block the parent task until the
+     dependencies are resolved and then just continue with the rest
+     of the function as if it is a merged task.  */
   if (depend != NULL)
     {
       struct gomp_thread *thr = gomp_thread ();
@@ -1410,17 +1442,7 @@ GOMP_target_ext (int device, void (*fn)
   struct target_mem_desc *tgt_vars
     = gomp_map_vars (devicep, mapnum, hostaddrs, NULL, sizes, kinds, true,
 		     GOMP_MAP_VARS_TARGET);
-  struct gomp_thread old_thr, *thr = gomp_thread ();
-  old_thr = *thr;
-  memset (thr, '\0', sizeof (*thr));
-  if (gomp_places_list)
-    {
-      thr->place = old_thr.place;
-      thr->ts.place_partition_len = gomp_places_list_len;
-    }
   devicep->run_func (devicep->target_id, fn_addr, (void *) tgt_vars->tgt_start);
-  gomp_free_thread (thr);
-  *thr = old_thr;
   gomp_unmap_vars (tgt_vars, true);
 }
 
@@ -1527,23 +1549,25 @@ GOMP_target_update_ext (int device, size
 	      && thr->ts.team
 	      && !thr->task->final_task)
 	    {
-	      gomp_create_target_task (devicep, (void (*) (void *)) NULL,
-				       mapnum, hostaddrs, sizes, kinds,
-				       flags | GOMP_TARGET_FLAG_UPDATE,
-				       depend);
-	      return;
+	      if (gomp_create_target_task (devicep, (void (*) (void *)) NULL,
+					   mapnum, hostaddrs, sizes, kinds,
+					   flags | GOMP_TARGET_FLAG_UPDATE,
+					   depend, GOMP_TARGET_TASK_DATA))
+		return;
 	    }
+	  else
+	    {
+	      struct gomp_team *team = thr->ts.team;
+	      /* If parallel or taskgroup has been cancelled, don't start new
+		 tasks.  */
+	      if (team
+		  && (gomp_team_barrier_cancelled (&team->barrier)
+		      || (thr->task->taskgroup
+			  && thr->task->taskgroup->cancelled)))
+		return;
 
-	  struct gomp_team *team = thr->ts.team;
-	  /* If parallel or taskgroup has been cancelled, don't start new
-	     tasks.  */
-	  if (team
-	      && (gomp_team_barrier_cancelled (&team->barrier)
-		  || (thr->task->taskgroup
-		      && thr->task->taskgroup->cancelled)))
-	    return;
-
-	  gomp_task_maybe_wait_for_dependencies (depend);
+	      gomp_task_maybe_wait_for_dependencies (depend);
+	    }
 	}
     }
 
@@ -1647,22 +1671,25 @@ GOMP_target_enter_exit_data (int device,
 	      && thr->ts.team
 	      && !thr->task->final_task)
 	    {
-	      gomp_create_target_task (devicep, (void (*) (void *)) NULL,
-				       mapnum, hostaddrs, sizes, kinds,
-				       flags, depend);
-	      return;
+	      if (gomp_create_target_task (devicep, (void (*) (void *)) NULL,
+					   mapnum, hostaddrs, sizes, kinds,
+					   flags, depend,
+					   GOMP_TARGET_TASK_DATA))
+		return;
 	    }
+	  else
+	    {
+	      struct gomp_team *team = thr->ts.team;
+	      /* If parallel or taskgroup has been cancelled, don't start new
+		 tasks.  */
+	      if (team
+		  && (gomp_team_barrier_cancelled (&team->barrier)
+		      || (thr->task->taskgroup
+			  && thr->task->taskgroup->cancelled)))
+		return;
 
-	  struct gomp_team *team = thr->ts.team;
-	  /* If parallel or taskgroup has been cancelled, don't start new
-	     tasks.  */
-	  if (team
-	      && (gomp_team_barrier_cancelled (&team->barrier)
-		  || (thr->task->taskgroup
-		      && thr->task->taskgroup->cancelled)))
-	    return;
-
-	  gomp_task_maybe_wait_for_dependencies (depend);
+	      gomp_task_maybe_wait_for_dependencies (depend);
+	    }
 	}
     }
 
@@ -1694,38 +1721,65 @@ GOMP_target_enter_exit_data (int device,
     gomp_exit_data (devicep, mapnum, hostaddrs, sizes, kinds);
 }
 
-void
+bool
 gomp_target_task_fn (void *data)
 {
   struct gomp_target_task *ttask = (struct gomp_target_task *) data;
+  struct gomp_device_descr *devicep = ttask->devicep;
+
   if (ttask->fn != NULL)
     {
-      /* GOMP_target_ext */
-    }
-  else if (ttask->devicep == NULL
-	   || !(ttask->devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
-    return;
+      if (devicep == NULL
+	  || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+	{
+	  ttask->state = GOMP_TARGET_TASK_FALLBACK;
+	  gomp_target_fallback_firstprivate (ttask->fn, ttask->mapnum,
+					     ttask->hostaddrs, ttask->sizes,
+					     ttask->kinds);
+	  return false;
+	}
+
+      if (ttask->state == GOMP_TARGET_TASK_RUNNING)
+	{
+	  gomp_unmap_vars (ttask->tgt, true);
+	  return false;
+	}
+
+      void *fn_addr = gomp_get_target_fn_addr (devicep, ttask->fn);
+      ttask->tgt
+	= gomp_map_vars (devicep, ttask->mapnum, ttask->hostaddrs, NULL,
+			 ttask->sizes, ttask->kinds, true,
+			 GOMP_MAP_VARS_TARGET);
+      ttask->state = GOMP_TARGET_TASK_RUNNING;
+
+      devicep->async_run_func (devicep->target_id, fn_addr,
+			       (void *) ttask->tgt->tgt_start, (void *) ttask);
+      return true;
+    }
+  else if (devicep == NULL
+	   || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+    return false;
 
   size_t i;
   if (ttask->flags & GOMP_TARGET_FLAG_UPDATE)
-    gomp_update (ttask->devicep, ttask->mapnum, ttask->hostaddrs, ttask->sizes,
+    gomp_update (devicep, ttask->mapnum, ttask->hostaddrs, ttask->sizes,
 		 ttask->kinds, true);
   else if ((ttask->flags & GOMP_TARGET_FLAG_EXIT_DATA) == 0)
     for (i = 0; i < ttask->mapnum; i++)
       if ((ttask->kinds[i] & 0xff) == GOMP_MAP_STRUCT)
 	{
-	  gomp_map_vars (ttask->devicep, ttask->sizes[i] + 1,
-			 &ttask->hostaddrs[i], NULL, &ttask->sizes[i],
-			 &ttask->kinds[i], true, GOMP_MAP_VARS_ENTER_DATA);
+	  gomp_map_vars (devicep, ttask->sizes[i] + 1, &ttask->hostaddrs[i],
+			 NULL, &ttask->sizes[i], &ttask->kinds[i], true,
+			 GOMP_MAP_VARS_ENTER_DATA);
 	  i += ttask->sizes[i];
 	}
       else
-	gomp_map_vars (ttask->devicep, 1, &ttask->hostaddrs[i], NULL,
-		       &ttask->sizes[i], &ttask->kinds[i],
-		       true, GOMP_MAP_VARS_ENTER_DATA);
+	gomp_map_vars (devicep, 1, &ttask->hostaddrs[i], NULL, &ttask->sizes[i],
+		       &ttask->kinds[i], true, GOMP_MAP_VARS_ENTER_DATA);
   else
-    gomp_exit_data (ttask->devicep, ttask->mapnum, ttask->hostaddrs,
-		    ttask->sizes, ttask->kinds);
+    gomp_exit_data (devicep, ttask->mapnum, ttask->hostaddrs, ttask->sizes,
+		    ttask->kinds);
+  return false;
 }
 
 void
@@ -2170,6 +2224,7 @@ gomp_load_plugin_for_device (struct gomp
   if (device->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400)
     {
       DLSYM (run);
+      DLSYM (async_run);
       DLSYM (dev2dev);
     }
   if (device->capabilities & GOMP_OFFLOAD_CAP_OPENACC_200)
--- libgomp/libgomp.map.jj	2015-10-26 15:38:20.000000000 +0100
+++ libgomp/libgomp.map	2015-11-11 16:15:23.807818735 +0100
@@ -406,3 +406,8 @@ GOMP_PLUGIN_1.0 {
 	GOMP_PLUGIN_async_unmap_vars;
 	GOMP_PLUGIN_acc_thread;
 };
+
+GOMP_PLUGIN_1.1 {
+  global:
+	GOMP_PLUGIN_target_task_completion;
+} GOMP_PLUGIN_1.0;
--- libgomp/task.c.jj	2015-11-09 11:14:37.332239862 +0100
+++ libgomp/task.c	2015-11-12 16:24:19.127548800 +0100
@@ -482,11 +482,12 @@ ialias (GOMP_taskgroup_end)
 
 /* Called for nowait target tasks.  */
 
-void
+bool
 gomp_create_target_task (struct gomp_device_descr *devicep,
 			 void (*fn) (void *), size_t mapnum, void **hostaddrs,
 			 size_t *sizes, unsigned short *kinds,
-			 unsigned int flags, void **depend)
+			 unsigned int flags, void **depend,
+			 enum gomp_target_task_state state)
 {
   struct gomp_thread *thr = gomp_thread ();
   struct gomp_team *team = thr->ts.team;
@@ -495,7 +496,7 @@ gomp_create_target_task (struct gomp_dev
   if (team
       && (gomp_team_barrier_cancelled (&team->barrier)
 	  || (thr->task->taskgroup && thr->task->taskgroup->cancelled)))
-    return;
+    return true;
 
   struct gomp_target_task *ttask;
   struct gomp_task *task;
@@ -503,19 +504,44 @@ gomp_create_target_task (struct gomp_dev
   struct gomp_taskgroup *taskgroup = parent->taskgroup;
   bool do_wake;
   size_t depend_size = 0;
+  uintptr_t depend_cnt = 0;
+  size_t tgt_align = 0, tgt_size = 0;
 
   if (depend != NULL)
-    depend_size = ((uintptr_t) depend[0]
-		   * sizeof (struct gomp_task_depend_entry));
+    {
+      depend_cnt = (uintptr_t) depend[0];
+      depend_size = depend_cnt * sizeof (struct gomp_task_depend_entry);
+    }
+  if (fn)
+    {
+      /* GOMP_MAP_FIRSTPRIVATE need to be copied first, as they are
+	 firstprivate on the target task.  */
+      size_t i;
+      for (i = 0; i < mapnum; i++)
+	if ((kinds[i] & 0xff) == GOMP_MAP_FIRSTPRIVATE)
+	  {
+	    size_t align = (size_t) 1 << (kinds[i] >> 8);
+	    if (tgt_align < align)
+	      tgt_align = align;
+	    tgt_size = (tgt_size + align - 1) & ~(align - 1);
+	    tgt_size += sizes[i];
+	  }
+      if (tgt_align)
+	tgt_size += tgt_align - 1;
+      else
+	tgt_size = 0;
+    }
+
   task = gomp_malloc (sizeof (*task) + depend_size
 		      + sizeof (*ttask)
 		      + mapnum * (sizeof (void *) + sizeof (size_t)
-				  + sizeof (unsigned short)));
+				  + sizeof (unsigned short))
+		      + tgt_size);
   gomp_init_task (task, parent, gomp_icv (false));
   task->kind = GOMP_TASK_WAITING;
   task->in_tied_task = parent->in_tied_task;
   task->taskgroup = taskgroup;
-  ttask = (struct gomp_target_task *) &task->depend[(uintptr_t) depend[0]];
+  ttask = (struct gomp_target_task *) &task->depend[depend_cnt];
   ttask->devicep = devicep;
   ttask->fn = fn;
   ttask->mapnum = mapnum;
@@ -524,8 +550,29 @@ gomp_create_target_task (struct gomp_dev
   memcpy (ttask->sizes, sizes, mapnum * sizeof (size_t));
   ttask->kinds = (unsigned short *) &ttask->sizes[mapnum];
   memcpy (ttask->kinds, kinds, mapnum * sizeof (unsigned short));
+  if (tgt_align)
+    {
+      char *tgt = (char *) &ttask->kinds[mapnum];
+      size_t i;
+      uintptr_t al = (uintptr_t) tgt & (tgt_align - 1);
+      if (al)
+	tgt += tgt_align - al;
+      tgt_size = 0;
+      for (i = 0; i < mapnum; i++)
+	if ((kinds[i] & 0xff) == GOMP_MAP_FIRSTPRIVATE)
+	  {
+	    size_t align = (size_t) 1 << (kinds[i] >> 8);
+	    tgt_size = (tgt_size + align - 1) & ~(align - 1);
+	    memcpy (tgt + tgt_size, hostaddrs[i], sizes[i]); 
+	    ttask->hostaddrs[i] = tgt + tgt_size;
+	    tgt_size = tgt_size + sizes[i];
+	  }
+    }
   ttask->flags = flags;
-  task->fn = gomp_target_task_fn;
+  ttask->state = state;
+  ttask->task = task;
+  ttask->team = team;
+  task->fn = NULL;
   task->fn_data = ttask;
   task->final_task = 0;
   gomp_mutex_lock (&team->task_lock);
@@ -536,19 +583,26 @@ gomp_create_target_task (struct gomp_dev
       gomp_mutex_unlock (&team->task_lock);
       gomp_finish_task (task);
       free (task);
-      return;
+      return true;
     }
-  if (taskgroup)
-    taskgroup->num_children++;
   if (depend_size)
     {
       gomp_task_handle_depend (task, parent, depend);
       if (task->num_dependees)
 	{
 	  gomp_mutex_unlock (&team->task_lock);
-	  return;
+	  return true;
 	}
     }
+  if (state == GOMP_TARGET_TASK_DATA)
+    {
+      gomp_mutex_unlock (&team->task_lock);
+      gomp_finish_task (task);
+      free (task);
+      return false;
+    }
+  if (taskgroup)
+    taskgroup->num_children++;
   priority_queue_insert (PQ_CHILDREN, &parent->children_queue, task, 0,
 			 PRIORITY_INSERT_BEGIN,
 			 /*adjust_parent_depends_on=*/false,
@@ -570,6 +624,95 @@ gomp_create_target_task (struct gomp_dev
   gomp_mutex_unlock (&team->task_lock);
   if (do_wake)
     gomp_team_barrier_wake (&team->barrier, 1);
+  return true;
+}
+
+static void inline
+priority_queue_move_task_first (enum priority_queue_type type,
+				struct priority_queue *head,
+				struct gomp_task *task)
+{
+#if _LIBGOMP_CHECKING_
+  if (!priority_queue_task_in_queue_p (type, head, task))
+    gomp_fatal ("Attempt to move first missing task %p", task);
+#endif
+  struct priority_list *list;
+  if (priority_queue_multi_p (head))
+    {
+      list = priority_queue_lookup_priority (head, task->priority);
+#if _LIBGOMP_CHECKING_
+      if (!list)
+	gomp_fatal ("Unable to find priority %d", task->priority);
+#endif
+    }
+  else
+    list = &head->l;
+  priority_list_remove (list, task_to_priority_node (type, task), 0);
+  priority_list_insert (type, list, task, task->priority,
+			PRIORITY_INSERT_BEGIN, type == PQ_CHILDREN,
+			task->parent_depends_on);
+}
+
+/* Signal that a target task TTASK has completed the asynchronously
+   running phase and should be requeued as a task to handle the
+   variable unmapping.  */
+
+void
+GOMP_PLUGIN_target_task_completion (void *data)
+{
+  struct gomp_target_task *ttask = (struct gomp_target_task *) data;
+  struct gomp_task *task = ttask->task;
+  struct gomp_team *team = ttask->team;
+
+  gomp_mutex_lock (&team->task_lock);
+  struct gomp_task *parent = task->parent;
+  if (parent)
+    priority_queue_move_task_first (PQ_CHILDREN, &parent->children_queue,
+				    task);
+
+  struct gomp_taskgroup *taskgroup = task->taskgroup;
+  if (taskgroup)
+    priority_queue_move_task_first (PQ_TASKGROUP, &taskgroup->taskgroup_queue,
+				    task);
+
+  priority_queue_insert (PQ_TEAM, &team->task_queue, task, task->priority,
+			 PRIORITY_INSERT_BEGIN, false,
+			 task->parent_depends_on);
+  task->kind = GOMP_TASK_WAITING;
+  if (parent && parent->taskwait)
+    {
+      if (parent->taskwait->in_taskwait)
+	{
+	  /* One more task has had its dependencies met.
+	     Inform any waiters.  */
+	  parent->taskwait->in_taskwait = false;
+	  gomp_sem_post (&parent->taskwait->taskwait_sem);
+	}
+      else if (parent->taskwait->in_depend_wait)
+	{
+	  /* One more task has had its dependencies met.
+	     Inform any waiters.  */
+	  parent->taskwait->in_depend_wait = false;
+	  gomp_sem_post (&parent->taskwait->taskwait_sem);
+	}
+    }
+  if (taskgroup && taskgroup->in_taskgroup_wait)
+    {
+      /* One more task has had its dependencies met.
+	 Inform any waiters.  */
+      taskgroup->in_taskgroup_wait = false;
+      gomp_sem_post (&taskgroup->taskgroup_sem);
+    }
+
+  ++team->task_queued_count;
+  gomp_team_barrier_set_task_pending (&team->barrier);
+  /* I'm afraid this can't be done after releasing team->task_lock,
+     as gomp_target_task_completion is run from unrelated thread and
+     therefore in between gomp_mutex_unlock and gomp_team_barrier_wake
+     the team could be gone already.  */
+  if (team->nthreads > team->task_running_count)
+    gomp_team_barrier_wake (&team->barrier, 1);
+  gomp_mutex_unlock (&team->task_lock);
 }
 
 /* Given a parent_depends_on task in LIST, move it to the front of its
@@ -1041,7 +1184,20 @@ gomp_barrier_handle_tasks (gomp_barrier_
       if (child_task)
 	{
 	  thr->task = child_task;
-	  child_task->fn (child_task->fn_data);
+	  if (__builtin_expect (child_task->fn == NULL, 0))
+	    {
+	      if (gomp_target_task_fn (child_task->fn_data))
+		{
+		  thr->task = task;
+		  gomp_mutex_lock (&team->task_lock);
+		  child_task->kind = GOMP_TASK_ASYNC_RUNNING;
+		  team->task_running_count--;
+		  child_task = NULL;
+		  continue;
+		}
+	    }
+	  else
+	    child_task->fn (child_task->fn_data);
 	  thr->task = task;
 	}
       else
@@ -1170,7 +1326,19 @@ GOMP_taskwait (void)
       if (child_task)
 	{
 	  thr->task = child_task;
-	  child_task->fn (child_task->fn_data);
+	  if (__builtin_expect (child_task->fn == NULL, 0))
+	    {
+	      if (gomp_target_task_fn (child_task->fn_data))
+		{
+		  thr->task = task;
+		  gomp_mutex_lock (&team->task_lock);
+		  child_task->kind = GOMP_TASK_ASYNC_RUNNING;
+		  child_task = NULL;
+		  continue;
+		}
+	    }
+	  else
+	    child_task->fn (child_task->fn_data);
 	  thr->task = task;
 	}
       else
@@ -1342,7 +1510,19 @@ gomp_task_maybe_wait_for_dependencies (v
       if (child_task)
 	{
 	  thr->task = child_task;
-	  child_task->fn (child_task->fn_data);
+	  if (__builtin_expect (child_task->fn == NULL, 0))
+	    {
+	      if (gomp_target_task_fn (child_task->fn_data))
+		{
+		  thr->task = task;
+		  gomp_mutex_lock (&team->task_lock);
+		  child_task->kind = GOMP_TASK_ASYNC_RUNNING;
+		  child_task = NULL;
+		  continue;
+		}
+	    }
+	  else
+	    child_task->fn (child_task->fn_data);
 	  thr->task = task;
 	}
       else
@@ -1423,6 +1603,17 @@ GOMP_taskgroup_end (void)
   if (team == NULL)
     return;
   taskgroup = task->taskgroup;
+  if (__builtin_expect (taskgroup == NULL, 0)
+      && thr->ts.level == 0)
+    {
+      /* This can happen if GOMP_taskgroup_start is called when
+	 thr->ts.team == NULL, but inside of the taskgroup there
+	 is #pragma omp target nowait that creates an implicit
+	 team with a single thread.  In this case, we want to wait
+	 for all outstanding tasks in this team.  */
+      gomp_team_barrier_wait (&team->barrier);
+      return;
+    }
 
   /* The acquire barrier on load of taskgroup->num_children here
      synchronizes with the write of 0 in gomp_task_run_post_remove_taskgroup.
@@ -1450,8 +1641,8 @@ GOMP_taskgroup_end (void)
 		= priority_queue_next_task (PQ_CHILDREN, &task->children_queue,
 					    PQ_TEAM, &team->task_queue,
 					    &unused);
-            }
-          else
+	    }
+	  else
 	    {
 	      gomp_mutex_unlock (&team->task_lock);
 	      if (to_free)
@@ -1506,7 +1697,19 @@ GOMP_taskgroup_end (void)
       if (child_task)
 	{
 	  thr->task = child_task;
-	  child_task->fn (child_task->fn_data);
+	  if (__builtin_expect (child_task->fn == NULL, 0))
+	    {
+	      if (gomp_target_task_fn (child_task->fn_data))
+		{
+		  thr->task = task;
+		  gomp_mutex_lock (&team->task_lock);
+		  child_task->kind = GOMP_TASK_ASYNC_RUNNING;
+		  child_task = NULL;
+		  continue;
+		}
+	    }
+	  else
+	    child_task->fn (child_task->fn_data);
 	  thr->task = task;
 	}
       else
--- libgomp/priority_queue.c.jj	2015-11-09 11:15:33.000000000 +0100
+++ libgomp/priority_queue.c	2015-11-10 17:52:33.769414428 +0100
@@ -85,7 +85,7 @@ priority_queue_task_in_queue_p (enum pri
    order.  LIST is a priority list of type TYPE.
 
    The expected order is that GOMP_TASK_WAITING tasks come before
-   GOMP_TASK_TIED ones.
+   GOMP_TASK_TIED/GOMP_TASK_ASYNC_RUNNING ones.
 
    If CHECK_DEPS is TRUE, we also check that parent_depends_on WAITING
    tasks come before !parent_depends_on WAITING tasks.  This is only
@@ -104,7 +104,7 @@ priority_list_verify (enum priority_queu
       struct gomp_task *t = priority_node_to_task (type, p);
       if (seen_tied && t->kind == GOMP_TASK_WAITING)
 	gomp_fatal ("priority_queue_verify: WAITING task after TIED");
-      if (t->kind == GOMP_TASK_TIED)
+      if (t->kind >= GOMP_TASK_TIED)
 	seen_tied = true;
       else if (check_deps && t->kind == GOMP_TASK_WAITING)
 	{
--- libgomp/libgomp.h.jj	2015-11-09 11:14:37.326239947 +0100
+++ libgomp/libgomp.h	2015-11-11 10:46:40.143794155 +0100
@@ -373,7 +373,12 @@ enum gomp_task_kind
   /* Task created by GOMP_task and waiting to be run.  */
   GOMP_TASK_WAITING,
   /* Task currently executing or scheduled and about to execute.  */
-  GOMP_TASK_TIED
+  GOMP_TASK_TIED,
+  /* Used for target tasks that have vars mapped and async run started,
+     but not yet completed.  Once that completes, they will be readded
+     into the queues as GOMP_TASK_WAITING in order to perform the var
+     unmapping.  */
+  GOMP_TASK_ASYNC_RUNNING
 };
 
 struct gomp_task_depend_entry
@@ -453,6 +458,8 @@ struct gomp_task
   struct gomp_task_depend_entry depend[];
 };
 
+/* This structure describes a single #pragma omp taskgroup.  */
+
 struct gomp_taskgroup
 {
   struct gomp_taskgroup *prev;
@@ -464,6 +471,8 @@ struct gomp_taskgroup
   size_t num_children;
 };
 
+/* This structure describes a target task.  */
+
 struct gomp_target_task
 {
   struct gomp_device_descr *devicep;
@@ -472,6 +481,10 @@ struct gomp_target_task
   size_t *sizes;
   unsigned short *kinds;
   unsigned int flags;
+  enum gomp_target_task_state state;
+  struct target_mem_desc *tgt;
+  struct gomp_task *task;
+  struct gomp_team *team;
   void *hostaddrs[];
 };
 
@@ -723,10 +736,10 @@ extern void gomp_init_task (struct gomp_
 extern void gomp_end_task (void);
 extern void gomp_barrier_handle_tasks (gomp_barrier_state_t);
 extern void gomp_task_maybe_wait_for_dependencies (void **);
-extern void gomp_create_target_task (struct gomp_device_descr *,
+extern bool gomp_create_target_task (struct gomp_device_descr *,
 				     void (*) (void *), size_t, void **,
 				     size_t *, unsigned short *, unsigned int,
-				     void **);
+				     void **, enum gomp_target_task_state);
 
 static void inline
 gomp_finish_task (struct gomp_task *task)
@@ -747,7 +760,7 @@ extern void gomp_free_thread (void *);
 
 extern void gomp_init_targets_once (void);
 extern int gomp_get_num_devices (void);
-extern void gomp_target_task_fn (void *);
+extern bool gomp_target_task_fn (void *);
 
 /* Splay tree definitions.  */
 typedef struct splay_tree_node_s *splay_tree_node;
@@ -901,6 +914,7 @@ struct gomp_device_descr
   void *(*host2dev_func) (int, void *, const void *, size_t);
   void *(*dev2dev_func) (int, void *, const void *, size_t);
   void (*run_func) (int, void *, void *);
+  void (*async_run_func) (int, void *, void *, void *);
 
   /* Splay tree containing information about mapped memory regions.  */
   struct splay_tree_s mem_map;


	Jakub

^ permalink raw reply	[flat|nested] 23+ messages in thread

* Re: [gomp4.5] depend nowait support for target
  2015-11-12 17:44         ` Ilya Verbin
@ 2015-11-12 17:58           ` Jakub Jelinek
  2015-11-12 18:07             ` Ilya Verbin
  0 siblings, 1 reply; 23+ messages in thread
From: Jakub Jelinek @ 2015-11-12 17:58 UTC (permalink / raw)
  To: Ilya Verbin
  Cc: Aldy Hernandez, gcc-patches, Kirill Yukhin, Thomas Schwinge,
	Alexander Monakov, Martin Jambor

On Thu, Nov 12, 2015 at 08:43:53PM +0300, Ilya Verbin wrote:
> > Can you please try to cleanup the liboffloadmic side of this, so that
> > a callback instead of hardcoded __gomp_offload_intelmic_async_completed call
> > is used?
> 
> Do you mean something like the patch bellow?  I'll discuss it with liboffloadmic
> maintainers.

Yeah; though am not 100% sure if a static variable settable by some function
is the best way, in case liboffloadmic is used by more than just libgomp
itself in the same process.

> I'm trying to do it, but it will take some time...
> 
> Unfortunately, target-32.c fails for me using emulation mode:

I haven't managed to get it stuck yet (unlike the target-33.c one, see
another mail), what OMP_NUM_THREADS you are using
and how many cores/threads?
Anyway, will try to figure out something from the backtrace you've provided.

	Jakub

^ permalink raw reply	[flat|nested] 23+ messages in thread

* Re: [gomp4.5] depend nowait support for target
  2015-11-12 17:58           ` Jakub Jelinek
@ 2015-11-12 18:07             ` Ilya Verbin
  0 siblings, 0 replies; 23+ messages in thread
From: Ilya Verbin @ 2015-11-12 18:07 UTC (permalink / raw)
  To: Jakub Jelinek; +Cc: gcc-patches

On Thu, Nov 12, 2015 at 18:58:22 +0100, Jakub Jelinek wrote:
> > Unfortunately, target-32.c fails for me using emulation mode:
> 
> I haven't managed to get it stuck yet (unlike the target-33.c one, see
> another mail), what OMP_NUM_THREADS you are using
> and how many cores/threads?

OMP_NUM_THREADS isn't set.  40 cores.

  -- Ilya

^ permalink raw reply	[flat|nested] 23+ messages in thread

* Re: [gomp4.5] depend nowait support for target
  2015-11-12 17:45         ` Jakub Jelinek
@ 2015-11-12 20:52           ` Ilya Verbin
  2015-11-13 10:18             ` Jakub Jelinek
  0 siblings, 1 reply; 23+ messages in thread
From: Ilya Verbin @ 2015-11-12 20:52 UTC (permalink / raw)
  To: Jakub Jelinek
  Cc: Aldy Hernandez, gcc-patches, Kirill Yukhin, Thomas Schwinge,
	Alexander Monakov, Martin Jambor

On Thu, Nov 12, 2015 at 18:45:09 +0100, Jakub Jelinek wrote:
> But the testcase I wrote (target-33.c) hangs, the problem is in the
>   #pragma omp target nowait map (tofrom: a, b) depend(out: d[3])
>   {
>     #pragma omp atomic update
>     a = a + 9;
>     b -= 8;
>   }
>   #pragma omp target nowait map (tofrom: a, c) depend(out: d[4])
>   {
>     #pragma omp atomic update
>     a = a + 4;
>     c >>= 1;
>   }
>   #pragma omp task if (0) depend (in: d[3], d[4])
>   if (a != 50 || b != 4 || c != 20)
>     abort ();
> part, where (I should change that for the case of no dependencies
> eventually) the task with map_vars+async_run is queued in both cases,
> then we reach GOMP_task, which calls gomp_task_maybe_wait_for_dependencies
> which spawns the first half task (map_vars+async_run), and then
> the second half task (map_vars+async_run), but that one gets stuck somewhere
> in liboffloadmic, then some other thread (from liboffloadmic) calls
> GOMP_PLUGIN_target_task_completion and enqueues the second half of the first
> target task (unmap_vars), but as the only normal thread in the main program
> is stuck in liboffloadmic (during gomp_map_vars, trying to allocate
> target memory in the plugin), there is no thread to schedule the second half
> of first target task.  So, if liboffloadmic is stuck waiting for unmap_vars,
> it is a deadlock.  Can you please try to debug this?

I'm unable to reproduce the hang (have tried various values of OMP_NUM_THREADS).
The testcase just aborts at (a != 50 || b != 4 || c != 20), because
a == 37, b == 12, c == 40.

BTW, don't know is this a bug or not:
Conditional jump or move depends on uninitialised value(s)
   at 0x4C2083D: priority_queue_insert (priority_queue.h:347)
   by 0x4C24DF9: GOMP_PLUGIN_target_task_completion (task.c:678)

  -- Ilya

^ permalink raw reply	[flat|nested] 23+ messages in thread

* Re: [gomp4.5] depend nowait support for target
  2015-11-12 20:52           ` Ilya Verbin
@ 2015-11-13 10:18             ` Jakub Jelinek
  2015-11-13 15:12               ` Jakub Jelinek
  0 siblings, 1 reply; 23+ messages in thread
From: Jakub Jelinek @ 2015-11-13 10:18 UTC (permalink / raw)
  To: Ilya Verbin
  Cc: Aldy Hernandez, gcc-patches, Kirill Yukhin, Thomas Schwinge,
	Alexander Monakov, Martin Jambor

On Thu, Nov 12, 2015 at 11:51:33PM +0300, Ilya Verbin wrote:
> I'm unable to reproduce the hang (have tried various values of OMP_NUM_THREADS).
> The testcase just aborts at (a != 50 || b != 4 || c != 20), because
> a == 37, b == 12, c == 40.

The hang has been with a fprintf (stderr, "...\n"); inside of the parallel
regions.  Anyway, still can't reproduce target-32.c crash, the target-33.c
abort is due to thinko (a, b, c were firstprivate in the explicit tasks, so
no wonder it could see the previous values).  See the following incremental
patch.

> BTW, don't know is this a bug or not:
> Conditional jump or move depends on uninitialised value(s)
>    at 0x4C2083D: priority_queue_insert (priority_queue.h:347)
>    by 0x4C24DF9: GOMP_PLUGIN_target_task_completion (task.c:678)

This is due to uninitialized task->priority for the target task.  See below.

Now I'm fighting target-34.c test hangs, strangely it hangs even with host
fallback.

For the offloading case, I actually see a problematic spot, namely that
GOMP_PLUGIN_target_task_completion could finish too early, and get the
task_lock before the thread that run the gomp_target_task_fn doing map_vars
+ async_run for it.  Bet I need to add further ttask state kinds and deal
with that case (so GOMP_PLUGIN_target_task_completion would just take the
task lock and tweak ttask state if it has not been added to the queues
yet).
Plus I think I want to improve the case where we are not waiting, in
gomp_create_target_task if not waiting for dependencies actually schedule
manually the gomp_target_task_fn.

--- libgomp/testsuite/libgomp.c/target-33.c	2015-11-12 16:20:14.000000000 +0100
+++ libgomp/testsuite/libgomp.c/target-33.c	2015-11-13 09:45:27.174427034 +0100
@@ -61,10 +61,10 @@
     a = a + 4;
     c >>= 1;
   }
-  #pragma omp task if (0) depend (in: d[3], d[4])
+  #pragma omp task if (0) depend (in: d[3], d[4]) shared (a, b, c)
   if (a != 50 || b != 4 || c != 20)
     abort ();
-  #pragma omp task
+  #pragma omp task shared (a)
   a += 50;
   #pragma omp target nowait map (tofrom: b)
   b++;
--- libgomp/task.c	2015-11-12 16:24:19.127548800 +0100
+++ libgomp/task.c	2015-11-13 10:53:19.525519366 +0100
@@ -538,6 +538,7 @@
 				  + sizeof (unsigned short))
 		      + tgt_size);
   gomp_init_task (task, parent, gomp_icv (false));
+  task->priority = 0;
   task->kind = GOMP_TASK_WAITING;
   task->in_tied_task = parent->in_tied_task;
   task->taskgroup = taskgroup;
--- libgomp/testsuite/libgomp.c/target-34.c.jj	2015-11-13 08:54:42.607799433 +0100
+++ libgomp/testsuite/libgomp.c/target-34.c	2015-11-13 08:54:37.865866795 +0100
@@ -0,0 +1,12 @@
+#define main do_main
+#include "target-33.c"
+#undef main
+
+int
+main ()
+{
+  #pragma omp parallel
+  #pragma omp single
+  do_main ();
+  return 0;
+}


	Jakub

^ permalink raw reply	[flat|nested] 23+ messages in thread

* Re: [gomp4.5] depend nowait support for target
  2015-11-13 10:18             ` Jakub Jelinek
@ 2015-11-13 15:12               ` Jakub Jelinek
  2015-11-13 16:37                 ` Ilya Verbin
  2015-11-23 14:16                 ` [hsa] " Martin Jambor
  0 siblings, 2 replies; 23+ messages in thread
From: Jakub Jelinek @ 2015-11-13 15:12 UTC (permalink / raw)
  To: Ilya Verbin
  Cc: Aldy Hernandez, gcc-patches, Kirill Yukhin, Thomas Schwinge,
	Alexander Monakov, Martin Jambor

On Fri, Nov 13, 2015 at 11:18:41AM +0100, Jakub Jelinek wrote:
> For the offloading case, I actually see a problematic spot, namely that
> GOMP_PLUGIN_target_task_completion could finish too early, and get the
> task_lock before the thread that run the gomp_target_task_fn doing map_vars
> + async_run for it.  Bet I need to add further ttask state kinds and deal
> with that case (so GOMP_PLUGIN_target_task_completion would just take the
> task lock and tweak ttask state if it has not been added to the queues
> yet).
> Plus I think I want to improve the case where we are not waiting, in
> gomp_create_target_task if not waiting for dependencies actually schedule
> manually the gomp_target_task_fn.

These two have been resolved, plus target-34.c issue resolved too (the bug
was that I've been too lazy and just put target-33.c test into #pragma omp
parallel #pragma omp single, but that is invalid OpenMP, as single is a
worksharing region and #pragma omp barrier may not be encountered in such a
region.  Fixed by rewriting the testcase.

So here is a full patch that passes for me both non-offloading and
offloading, OMP_NUM_THREADS=16 (implicit on my box) as well as
OMP_NUM_THREADS=1 (explicit).  I've incorporated your incremental patch.

One option to avoid the static variable would be to pass two pointers
instead of one (async_data), one would be the callback function pointer,
another argument to it.  Or another possibility would be to say that
the async_data argument the plugin passes to liboffloadmic would be
pointer to structure, holding a function pointer (completion callback)
and the data pointer to pass to it, and then the plugin would just
GOMP_PLUGIN_malloc 2 * sizeof (void *) for it, fill it in and
register some function in itself that would call the
GOMP_PLUGIN_target_task_completion with the second structure element
as argument and then free the structure pointer.

Do you get still crashes on any of the testcases with this?

2015-11-13  Jakub Jelinek  <jakub@redhat.com>
	    Ilya Verbin  <ilya.verbin@intel.com>

	* parallel.c (gomp_resolve_num_threads): Don't assume that
	if thr->ts.team is non-NULL, then pool must be non-NULL.
	* libgomp-plugin.h (GOMP_PLUGIN_target_task_completion): Declare.
	* team.c (gomp_free_thread): Call gomp_team_end if thr->ts.team
	is artificial team created for target nowait in implicit parallel
	region.
	(gomp_team_start): For nested check, test thr->ts.level instead of
	thr->ts.team != NULL.
	* target.c (GOMP_target): Don't adjust *thr in any way around
	running offloaded task.
	(GOMP_target_ext): Likewise.  Handle target nowait.
	(GOMP_target_update_ext, GOMP_target_enter_exit_data): Check
	return value from gomp_create_target_task, if false, fallthrough
	as if no dependencies exist.
	(gomp_target_task_fn): Change return type to bool, return true
	if the task should have another part scheduled later.  Handle
	target nowait.
	(gomp_load_plugin_for_device): Initialize async_run.
	* libgomp.map (GOMP_PLUGIN_1.1): New symbol version, export
	GOMP_PLUGIN_target_task_completion.
	* task.c (priority_queue_move_task_first,
	gomp_target_task_completion, GOMP_PLUGIN_target_task_completion):
	New functions.
	(gomp_create_target_task): Change return type to bool, add
	state argument, return false if for async {{enter,exit} data,update}
	constructs no dependencies need to be waited for, handle target
	nowait.  Set task->fn to NULL instead of gomp_target_task_fn.
	(gomp_barrier_handle_tasks, GOMP_taskwait,
	gomp_task_maybe_wait_for_dependencies): Handle target nowait target
	tasks specially.
	(GOMP_taskgroup_end): Likewise.  If taskgroup is NULL, and
	thr->ts.level is 0, act as a barrier.
	target nowait tasks specially.
	* priority_queue.c (priority_queue_task_in_queue_p,
	priority_list_verify): Adjust for addition of
	GOMP_TASK_ASYNC_RUNNING kind.
	* libgomp.h (enum gomp_task_kind): Add GOMP_TASK_ASYNC_RUNNING.
	(enum gomp_target_task_state): New enum.
	(struct gomp_target_task): Add state, tgt, task and team fields.
	(gomp_create_target_task): Change return type to bool, add
	state argument.
	(gomp_target_task_fn): Change return type to bool.
	(struct gomp_device_descr): Add async_run_func.
	* testsuite/libgomp.c/target-32.c: New test.
	* testsuite/libgomp.c/target-34.c: New test.
	* testsuite/libgomp.c/target-33.c: New test.

2015-11-13  Ilya Verbin  <ilya.verbin@intel.com>

	* runtime/offload_host.cpp (task_completion_callback): New
	variable.
	(offload_proxy_task_completed_ooo): Call task_completion_callback.
	(__offload_register_task_callback): New function.
	* runtime/offload_host.h (__offload_register_task_callback): New
	declaration.
	* plugin/libgomp-plugin-intelmic.cpp (offload): Add async_data
	argument, handle async offloading.
	(register_main_image): Call register_main_image.
	(GOMP_OFFLOAD_init_device, get_target_table, GOMP_OFFLOAD_alloc,
	GOMP_OFFLOAD_free, GOMP_OFFLOAD_host2dev, GOMP_OFFLOAD_dev2host,
	GOMP_OFFLOAD_dev2dev) Adjust offload callers.
	(GOMP_OFFLOAD_async_run): New function.
	(GOMP_OFFLOAD_run): Implement using GOMP_OFFLOAD_async_run.

--- liboffloadmic/runtime/offload_host.cpp.jj	2015-11-05 11:31:05.013916598 +0100
+++ liboffloadmic/runtime/offload_host.cpp	2015-11-13 14:23:54.469798572 +0100
@@ -64,6 +64,8 @@ static void __offload_fini_library(void)
 #define GET_OFFLOAD_NUMBER(timer_data) \
     timer_data? timer_data->offload_number : 0
 
+static void (*task_completion_callback)(void *);
+
 extern "C" {
 #ifdef TARGET_WINNT
 // Windows does not support imports from libraries without actually
@@ -2507,7 +2509,7 @@ extern "C" {
         const void *info
     )
     {
-	/* TODO: Call callback function, pass info.  */
+	task_completion_callback ((void *) info);
     }
 }
 
@@ -5669,6 +5671,11 @@ extern "C" void __offload_unregister_ima
     }
 }
 
+extern "C" void __offload_register_task_callback(void (*cb)(void *))
+{
+    task_completion_callback = cb;
+}
+
 // Runtime trace interface for user programs
 
 void __offload_console_trace(int level)
--- liboffloadmic/runtime/offload_host.h.jj	2015-10-14 10:24:10.904194499 +0200
+++ liboffloadmic/runtime/offload_host.h	2015-11-13 14:23:54.470798557 +0100
@@ -376,6 +376,9 @@ extern "C" bool __offload_target_image_i
 extern "C" bool __offload_register_image(const void* image);
 extern "C" void __offload_unregister_image(const void* image);
 
+// Registers asynchronous task completion callback
+extern "C" void __offload_register_task_callback(void (*cb)(void *));
+
 // Initializes offload runtime library.
 DLL_LOCAL extern int __offload_init_library(void);
 
--- liboffloadmic/plugin/libgomp-plugin-intelmic.cpp.jj	2015-10-14 10:24:10.922194230 +0200
+++ liboffloadmic/plugin/libgomp-plugin-intelmic.cpp	2015-11-13 14:23:54.467798600 +0100
@@ -192,11 +192,23 @@ GOMP_OFFLOAD_get_num_devices (void)
 
 static void
 offload (const char *file, uint64_t line, int device, const char *name,
-	 int num_vars, VarDesc *vars, VarDesc2 *vars2)
+	 int num_vars, VarDesc *vars, VarDesc2 *vars2, const void **async_data)
 {
   OFFLOAD ofld = __offload_target_acquire1 (&device, file, line);
   if (ofld)
-    __offload_offload1 (ofld, name, 0, num_vars, vars, vars2, 0, NULL, NULL);
+    {
+      if (async_data == NULL)
+	__offload_offload1 (ofld, name, 0, num_vars, vars, vars2, 0, NULL,
+			    NULL);
+      else
+	{
+	  OffloadFlags flags;
+	  flags.flags = 0;
+	  flags.bits.omp_async = 1;
+	  __offload_offload3 (ofld, name, 0, num_vars, vars, NULL, 0, NULL,
+			      async_data, 0, NULL, flags, NULL);
+	}
+    }
   else
     {
       fprintf (stderr, "%s:%d: Offload target acquire failed\n", file, line);
@@ -208,6 +220,10 @@ static void
 register_main_image ()
 {
   __offload_register_image (&main_target_image);
+
+  /* liboffloadmic will call GOMP_PLUGIN_target_task_completion when
+     asynchronous task on target is completed.  */
+  __offload_register_task_callback (GOMP_PLUGIN_target_task_completion);
 }
 
 /* liboffloadmic loads and runs offload_target_main on all available devices
@@ -218,7 +234,7 @@ GOMP_OFFLOAD_init_device (int device)
   TRACE ("");
   pthread_once (&main_image_is_registered, register_main_image);
   offload (__FILE__, __LINE__, device, "__offload_target_init_proc", 0,
-	   NULL, NULL);
+	   NULL, NULL, NULL);
 }
 
 extern "C" void
@@ -240,7 +256,7 @@ get_target_table (int device, int &num_f
   VarDesc2 vd1g[2] = { { "num_funcs", 0 }, { "num_vars", 0 } };
 
   offload (__FILE__, __LINE__, device, "__offload_target_table_p1", 2,
-	   vd1, vd1g);
+	   vd1, vd1g, NULL);
 
   int table_size = num_funcs + 2 * num_vars;
   if (table_size > 0)
@@ -254,7 +270,7 @@ get_target_table (int device, int &num_f
       VarDesc2 vd2g = { "table", 0 };
 
       offload (__FILE__, __LINE__, device, "__offload_target_table_p2", 1,
-	       &vd2, &vd2g);
+	       &vd2, &vd2g, NULL);
     }
 }
 
@@ -401,8 +417,8 @@ GOMP_OFFLOAD_alloc (int device, size_t s
   vd1[1].size = sizeof (void *);
   VarDesc2 vd1g[2] = { { "size", 0 }, { "tgt_ptr", 0 } };
 
-  offload (__FILE__, __LINE__, device, "__offload_target_alloc", 2, vd1, vd1g);
-
+  offload (__FILE__, __LINE__, device, "__offload_target_alloc", 2, vd1, vd1g,
+	   NULL);
   return tgt_ptr;
 }
 
@@ -416,7 +432,8 @@ GOMP_OFFLOAD_free (int device, void *tgt
   vd1.size = sizeof (void *);
   VarDesc2 vd1g = { "tgt_ptr", 0 };
 
-  offload (__FILE__, __LINE__, device, "__offload_target_free", 1, &vd1, &vd1g);
+  offload (__FILE__, __LINE__, device, "__offload_target_free", 1, &vd1, &vd1g,
+	   NULL);
 }
 
 extern "C" void *
@@ -435,7 +452,7 @@ GOMP_OFFLOAD_host2dev (int device, void
   VarDesc2 vd1g[2] = { { "tgt_ptr", 0 }, { "size", 0 } };
 
   offload (__FILE__, __LINE__, device, "__offload_target_host2tgt_p1", 2,
-	   vd1, vd1g);
+	   vd1, vd1g, NULL);
 
   VarDesc vd2 = vd_host2tgt;
   vd2.ptr = (void *) host_ptr;
@@ -443,7 +460,7 @@ GOMP_OFFLOAD_host2dev (int device, void
   VarDesc2 vd2g = { "var", 0 };
 
   offload (__FILE__, __LINE__, device, "__offload_target_host2tgt_p2", 1,
-	   &vd2, &vd2g);
+	   &vd2, &vd2g, NULL);
 
   return tgt_ptr;
 }
@@ -464,7 +481,7 @@ GOMP_OFFLOAD_dev2host (int device, void
   VarDesc2 vd1g[2] = { { "tgt_ptr", 0 }, { "size", 0 } };
 
   offload (__FILE__, __LINE__, device, "__offload_target_tgt2host_p1", 2,
-	   vd1, vd1g);
+	   vd1, vd1g, NULL);
 
   VarDesc vd2 = vd_tgt2host;
   vd2.ptr = (void *) host_ptr;
@@ -472,7 +489,7 @@ GOMP_OFFLOAD_dev2host (int device, void
   VarDesc2 vd2g = { "var", 0 };
 
   offload (__FILE__, __LINE__, device, "__offload_target_tgt2host_p2", 1,
-	   &vd2, &vd2g);
+	   &vd2, &vd2g, NULL);
 
   return host_ptr;
 }
@@ -495,22 +512,32 @@ GOMP_OFFLOAD_dev2dev (int device, void *
   VarDesc2 vd1g[3] = { { "dst_ptr", 0 }, { "src_ptr", 0 }, { "size", 0 } };
 
   offload (__FILE__, __LINE__, device, "__offload_target_tgt2tgt", 3, vd1,
-	   vd1g);
+	   vd1g, NULL);
 
   return dst_ptr;
 }
 
 extern "C" void
-GOMP_OFFLOAD_run (int device, void *tgt_fn, void *tgt_vars)
+GOMP_OFFLOAD_async_run (int device, void *tgt_fn, void *tgt_vars,
+			void *async_data)
 {
-  TRACE ("(tgt_fn = %p, tgt_vars = %p)", tgt_fn, tgt_vars);
+  TRACE ("(device = %d, tgt_fn = %p, tgt_vars = %p, async_data = %p)", device,
+	 tgt_fn, tgt_vars, async_data);
 
-  VarDesc vd1[2] = { vd_host2tgt, vd_host2tgt };
-  vd1[0].ptr = &tgt_fn;
-  vd1[0].size = sizeof (void *);
-  vd1[1].ptr = &tgt_vars;
-  vd1[1].size = sizeof (void *);
-  VarDesc2 vd1g[2] = { { "tgt_fn", 0 }, { "tgt_vars", 0 } };
+  VarDesc vd[2] = { vd_host2tgt, vd_host2tgt };
+  vd[0].ptr = &tgt_fn;
+  vd[0].size = sizeof (void *);
+  vd[1].ptr = &tgt_vars;
+  vd[1].size = sizeof (void *);
+
+  offload (__FILE__, __LINE__, device, "__offload_target_run", 2, vd, NULL,
+	   (const void **) async_data);
+}
+
+extern "C" void
+GOMP_OFFLOAD_run (int device, void *tgt_fn, void *tgt_vars)
+{
+  TRACE ("(device = %d, tgt_fn = %p, tgt_vars = %p)", device, tgt_fn, tgt_vars);
 
-  offload (__FILE__, __LINE__, device, "__offload_target_run", 2, vd1, vd1g);
+  GOMP_OFFLOAD_async_run (device, tgt_fn, tgt_vars, NULL);
 }
--- libgomp/parallel.c.jj	2015-10-14 10:24:10.000000000 +0200
+++ libgomp/parallel.c	2015-11-12 15:12:38.349901541 +0100
@@ -85,7 +85,7 @@ gomp_resolve_num_threads (unsigned speci
      nested parallel, so there is just one thread in the
      contention group as well, no need to handle it atomically.  */
   pool = thr->thread_pool;
-  if (thr->ts.team == NULL)
+  if (thr->ts.team == NULL || pool == NULL)
     {
       num_threads = max_num_threads;
       if (num_threads > icv->thread_limit_var)
--- libgomp/libgomp-plugin.h.jj	2015-10-14 10:24:10.000000000 +0200
+++ libgomp/libgomp-plugin.h	2015-11-13 11:40:03.366418330 +0100
@@ -63,6 +63,7 @@ struct addr_pair
 extern void *GOMP_PLUGIN_malloc (size_t) __attribute__ ((malloc));
 extern void *GOMP_PLUGIN_malloc_cleared (size_t) __attribute__ ((malloc));
 extern void *GOMP_PLUGIN_realloc (void *, size_t);
+void GOMP_PLUGIN_target_task_completion (void *);
 
 extern void GOMP_PLUGIN_debug (int, const char *, ...)
 	__attribute__ ((format (printf, 2, 3)));
--- libgomp/testsuite/libgomp.c/target-32.c.jj	2015-11-10 12:58:55.087951346 +0100
+++ libgomp/testsuite/libgomp.c/target-32.c	2015-11-12 13:28:55.053380366 +0100
@@ -0,0 +1,54 @@
+#include <stdlib.h>
+#include <unistd.h>
+
+int main ()
+{
+  int a = 0, b = 0, c = 0, d[7];
+
+  #pragma omp parallel
+  #pragma omp single
+  {
+    #pragma omp task depend(out: d[0])
+      a = 2;
+
+    #pragma omp target enter data nowait map(to: a,b,c) depend(in: d[0]) depend(out: d[1])
+
+    #pragma omp target nowait map(alloc: a) depend(in: d[1]) depend(out: d[2])
+      a++;
+
+    #pragma omp target nowait map(alloc: b) depend(in: d[2]) depend(out: d[3])
+    {
+      usleep (1000);
+      #pragma omp atomic update
+      b |= 4;
+    }
+
+    #pragma omp target nowait map(alloc: b) depend(in: d[2]) depend(out: d[4])
+    {
+      usleep (5000);
+      #pragma omp atomic update
+      b |= 1;
+    }
+
+    #pragma omp target nowait map(alloc: c) depend(in: d[3], d[4]) depend(out: d[5])
+    {
+      usleep (5000);
+      #pragma omp atomic update
+      c |= 8;
+    }
+
+    #pragma omp target nowait map(alloc: c) depend(in: d[3], d[4]) depend(out: d[6])
+    {
+      usleep (1000);
+      #pragma omp atomic update
+      c |= 2;
+    }
+
+    #pragma omp target exit data map(always,from: a,b,c) depend(in: d[5], d[6])
+  }
+
+  if (a != 3 || b != 5 || c != 10)
+    abort ();
+
+  return 0;
+}
--- libgomp/testsuite/libgomp.c/target-34.c.jj	2015-11-13 08:54:42.607799433 +0100
+++ libgomp/testsuite/libgomp.c/target-34.c	2015-11-13 14:16:09.104425707 +0100
@@ -0,0 +1,112 @@
+extern void abort (void);
+
+int
+main ()
+{
+  int a = 1, b = 2, c = 4, d[7];
+  #pragma omp parallel
+  {
+    #pragma omp single
+    {
+      #pragma omp taskgroup
+      {
+	#pragma omp target enter data nowait map (to: a, b, c) depend(out: d[0])
+	#pragma omp target nowait map (alloc: a, b) depend(in: d[0]) depend(out: d[1])
+	{
+	  #pragma omp atomic update
+	  a |= 4;
+	  #pragma omp atomic update
+	  b |= 8;
+	}
+	#pragma omp target nowait map (alloc: a, c) depend(in: d[0]) depend(out: d[2])
+	{
+	  #pragma omp atomic update
+	  a |= 16;
+	  #pragma omp atomic update
+	  c |= 32;
+	}
+	#pragma omp target exit data nowait map (from: a, b, c) depend(in: d[1], d[2])
+      }
+      if (a != 21 || b != 10 || c != 36)
+	abort ();
+      #pragma omp target map (tofrom: a, b) nowait
+      {
+	a &= ~16;
+	b &= ~2;
+      }
+      #pragma omp target map (tofrom: c) nowait
+      {
+	c |= 8;
+      }
+    } /* Implicit barrier here.  */
+    #pragma omp single
+    {
+      if (a != 5 || b != 8 || c != 44)
+	abort ();
+      #pragma omp target map (tofrom: a, b) nowait
+      {
+	a |= 32;
+	b |= 4;
+      }
+      #pragma omp target map (tofrom: c) nowait
+      c &= ~4;
+      #pragma omp taskwait
+      if (a != 37 || b != 12 || c != 40)
+	abort ();
+      #pragma omp target nowait map (tofrom: a, b) depend(out: d[3])
+      {
+	#pragma omp atomic update
+	a = a + 9;
+	b -= 8;
+      }
+      #pragma omp target nowait map (tofrom: a, c) depend(out: d[4])
+      {
+	#pragma omp atomic update
+	a = a + 4;
+	c >>= 1;
+      }
+      #pragma omp task if (0) depend (in: d[3], d[4]) shared (a, b, c)
+      if (a != 50 || b != 4 || c != 20)
+	abort ();
+      #pragma omp task shared (a)
+      a += 50;
+      #pragma omp target nowait map (tofrom: b)
+      b++;
+      #pragma omp target map (tofrom: c) nowait
+      c--;
+      #pragma omp taskwait
+      if (a != 100 || b != 5 || c != 19)
+	abort ();
+      #pragma omp target map (tofrom: a) nowait depend(out: d[5])
+      a++;
+      #pragma omp target map (tofrom: b) nowait depend(out: d[6])
+      b++;
+      #pragma omp target map (tofrom: a, b) depend(in: d[5], d[6])
+      {
+	if (a != 101 || b != 6)
+	  a = -9;
+	else
+	  {
+	    a = 24;
+	    b = 38;
+	  }
+      }
+      if (a != 24 || b != 38)
+	abort ();
+    } /* Implicit barrier here.  */
+    #pragma omp master
+    {
+      #pragma omp target nowait map (tofrom: a, b)
+      {
+	a *= 2;
+	b++;
+      }
+      #pragma omp target map (tofrom: c) nowait
+      c--;
+    }
+    #pragma omp barrier
+    if (a != 48 || b != 39 || c != 18)
+      abort ();
+  }
+  return 0;
+}
--- libgomp/testsuite/libgomp.c/target-33.c.jj	2015-11-12 16:20:23.332860573 +0100
+++ libgomp/testsuite/libgomp.c/target-33.c	2015-11-13 09:45:27.000000000 +0100
@@ -0,0 +1,93 @@
+extern void abort (void);
+
+int
+main ()
+{
+  int a = 1, b = 2, c = 4, d[7];
+  #pragma omp taskgroup
+  {
+    #pragma omp target enter data nowait map (to: a, b, c) depend(out: d[0])
+    #pragma omp target nowait map (alloc: a, b) depend(in: d[0]) depend(out: d[1])
+    {
+      #pragma omp atomic update
+      a |= 4;
+      #pragma omp atomic update
+      b |= 8;
+    }
+    #pragma omp target nowait map (alloc: a, c) depend(in: d[0]) depend(out: d[2])
+    {
+      #pragma omp atomic update
+      a |= 16;
+      #pragma omp atomic update
+      c |= 32;
+    }
+    #pragma omp target exit data nowait map (from: a, b, c) depend(in: d[1], d[2])
+  }
+  if (a != 21 || b != 10 || c != 36)
+    abort ();
+  #pragma omp target map (tofrom: a, b) nowait
+  {
+    a &= ~16;
+    b &= ~2;
+  }
+  #pragma omp target map (tofrom: c) nowait
+  {
+    c |= 8;
+  }
+  #pragma omp barrier
+  if (a != 5 || b != 8 || c != 44)
+    abort ();
+  #pragma omp target map (tofrom: a, b) nowait
+  {
+    a |= 32;
+    b |= 4;
+  }
+  #pragma omp target map (tofrom: c) nowait
+  {
+    c &= ~4;
+  }
+  #pragma omp taskwait
+  if (a != 37 || b != 12 || c != 40)
+    abort ();
+  #pragma omp target nowait map (tofrom: a, b) depend(out: d[3])
+  {
+    #pragma omp atomic update
+    a = a + 9;
+    b -= 8;
+  }
+  #pragma omp target nowait map (tofrom: a, c) depend(out: d[4])
+  {
+    #pragma omp atomic update
+    a = a + 4;
+    c >>= 1;
+  }
+  #pragma omp task if (0) depend (in: d[3], d[4]) shared (a, b, c)
+  if (a != 50 || b != 4 || c != 20)
+    abort ();
+  #pragma omp task shared (a)
+  a += 50;
+  #pragma omp target nowait map (tofrom: b)
+  b++;
+  #pragma omp target map (tofrom: c) nowait
+  c--;
+  #pragma omp taskwait
+  if (a != 100 || b != 5 || c != 19)
+    abort ();
+  #pragma omp target map (tofrom: a) nowait depend(out: d[5])
+  a++;
+  #pragma omp target map (tofrom: b) nowait depend(out: d[6])
+  b++;
+  #pragma omp target map (tofrom: a, b) depend(in: d[5], d[6])
+  {
+    if (a != 101 || b != 6)
+      a = -9;
+    else
+      {
+	a = 24;
+	b = 38;
+      }
+  }
+  if (a != 24 || b != 38)
+    abort ();
+  return 0;
+}
--- libgomp/team.c.jj	2015-11-09 11:14:37.000000000 +0100
+++ libgomp/team.c	2015-11-12 15:09:23.584644449 +0100
@@ -272,6 +272,8 @@ gomp_free_thread (void *arg __attribute_
       free (pool);
       thr->thread_pool = NULL;
     }
+  if (thr->ts.level == 0 && __builtin_expect (thr->ts.team != NULL, 0))
+    gomp_team_end ();
   if (thr->task != NULL)
     {
       struct gomp_task *task = thr->task;
@@ -301,7 +303,7 @@ gomp_team_start (void (*fn) (void *), vo
   struct gomp_thread **affinity_thr = NULL;
 
   thr = gomp_thread ();
-  nested = thr->ts.team != NULL;
+  nested = thr->ts.level;
   pool = thr->thread_pool;
   task = thr->task;
   icv = task ? &task->icv : &gomp_global_icv;
--- libgomp/target.c.jj	2015-11-09 11:14:37.325239961 +0100
+++ libgomp/target.c	2015-11-13 11:42:28.255345131 +0100
@@ -1348,17 +1348,7 @@ GOMP_target (int device, void (*fn) (voi
   struct target_mem_desc *tgt_vars
     = gomp_map_vars (devicep, mapnum, hostaddrs, NULL, sizes, kinds, false,
 		     GOMP_MAP_VARS_TARGET);
-  struct gomp_thread old_thr, *thr = gomp_thread ();
-  old_thr = *thr;
-  memset (thr, '\0', sizeof (*thr));
-  if (gomp_places_list)
-    {
-      thr->place = old_thr.place;
-      thr->ts.place_partition_len = gomp_places_list_len;
-    }
   devicep->run_func (devicep->target_id, fn_addr, (void *) tgt_vars->tgt_start);
-  gomp_free_thread (thr);
-  *thr = old_thr;
   gomp_unmap_vars (tgt_vars, true);
 }
 
@@ -1387,10 +1377,52 @@ GOMP_target_ext (int device, void (*fn)
   (void) num_teams;
   (void) thread_limit;
 
-  /* If there are depend clauses, but nowait is not present,
-     block the parent task until the dependencies are resolved
-     and then just continue with the rest of the function as if it
-     is a merged task.  */
+  if (flags & GOMP_TARGET_FLAG_NOWAIT)
+    {
+      struct gomp_thread *thr = gomp_thread ();
+      /* Create a team if we don't have any around, as nowait
+	 target tasks make sense to run asynchronously even when
+	 outside of any parallel.  */
+      if (__builtin_expect (thr->ts.team == NULL, 0))
+	{
+	  struct gomp_team *team = gomp_new_team (1);
+	  struct gomp_task *task = thr->task;
+	  struct gomp_task_icv *icv = task ? &task->icv : &gomp_global_icv;
+	  team->prev_ts = thr->ts;
+	  thr->ts.team = team;
+	  thr->ts.team_id = 0;
+	  thr->ts.work_share = &team->work_shares[0];
+	  thr->ts.last_work_share = NULL;
+#ifdef HAVE_SYNC_BUILTINS
+	  thr->ts.single_count = 0;
+#endif
+	  thr->ts.static_trip = 0;
+	  thr->task = &team->implicit_task[0];
+	  gomp_init_task (thr->task, NULL, icv);
+	  if (task)
+	    {
+	      thr->task = task;
+	      gomp_end_task ();
+	      free (task);
+	      thr->task = &team->implicit_task[0];
+	    }
+	  else
+	    pthread_setspecific (gomp_thread_destructor, thr);
+	}
+      if (thr->ts.team
+	  && !thr->task->final_task)
+	{
+	  gomp_create_target_task (devicep, fn, mapnum, hostaddrs,
+				   sizes, kinds, flags, depend,
+				   GOMP_TARGET_TASK_BEFORE_MAP);
+	  return;
+	}
+    }
+
+  /* If there are depend clauses, but nowait is not present
+     (or we are in a final task), block the parent task until the
+     dependencies are resolved and then just continue with the rest
+     of the function as if it is a merged task.  */
   if (depend != NULL)
     {
       struct gomp_thread *thr = gomp_thread ();
@@ -1410,17 +1442,7 @@ GOMP_target_ext (int device, void (*fn)
   struct target_mem_desc *tgt_vars
     = gomp_map_vars (devicep, mapnum, hostaddrs, NULL, sizes, kinds, true,
 		     GOMP_MAP_VARS_TARGET);
-  struct gomp_thread old_thr, *thr = gomp_thread ();
-  old_thr = *thr;
-  memset (thr, '\0', sizeof (*thr));
-  if (gomp_places_list)
-    {
-      thr->place = old_thr.place;
-      thr->ts.place_partition_len = gomp_places_list_len;
-    }
   devicep->run_func (devicep->target_id, fn_addr, (void *) tgt_vars->tgt_start);
-  gomp_free_thread (thr);
-  *thr = old_thr;
   gomp_unmap_vars (tgt_vars, true);
 }
 
@@ -1527,23 +1549,25 @@ GOMP_target_update_ext (int device, size
 	      && thr->ts.team
 	      && !thr->task->final_task)
 	    {
-	      gomp_create_target_task (devicep, (void (*) (void *)) NULL,
-				       mapnum, hostaddrs, sizes, kinds,
-				       flags | GOMP_TARGET_FLAG_UPDATE,
-				       depend);
-	      return;
+	      if (gomp_create_target_task (devicep, (void (*) (void *)) NULL,
+					   mapnum, hostaddrs, sizes, kinds,
+					   flags | GOMP_TARGET_FLAG_UPDATE,
+					   depend, GOMP_TARGET_TASK_DATA))
+		return;
 	    }
+	  else
+	    {
+	      struct gomp_team *team = thr->ts.team;
+	      /* If parallel or taskgroup has been cancelled, don't start new
+		 tasks.  */
+	      if (team
+		  && (gomp_team_barrier_cancelled (&team->barrier)
+		      || (thr->task->taskgroup
+			  && thr->task->taskgroup->cancelled)))
+		return;
 
-	  struct gomp_team *team = thr->ts.team;
-	  /* If parallel or taskgroup has been cancelled, don't start new
-	     tasks.  */
-	  if (team
-	      && (gomp_team_barrier_cancelled (&team->barrier)
-		  || (thr->task->taskgroup
-		      && thr->task->taskgroup->cancelled)))
-	    return;
-
-	  gomp_task_maybe_wait_for_dependencies (depend);
+	      gomp_task_maybe_wait_for_dependencies (depend);
+	    }
 	}
     }
 
@@ -1647,22 +1671,25 @@ GOMP_target_enter_exit_data (int device,
 	      && thr->ts.team
 	      && !thr->task->final_task)
 	    {
-	      gomp_create_target_task (devicep, (void (*) (void *)) NULL,
-				       mapnum, hostaddrs, sizes, kinds,
-				       flags, depend);
-	      return;
+	      if (gomp_create_target_task (devicep, (void (*) (void *)) NULL,
+					   mapnum, hostaddrs, sizes, kinds,
+					   flags, depend,
+					   GOMP_TARGET_TASK_DATA))
+		return;
 	    }
+	  else
+	    {
+	      struct gomp_team *team = thr->ts.team;
+	      /* If parallel or taskgroup has been cancelled, don't start new
+		 tasks.  */
+	      if (team
+		  && (gomp_team_barrier_cancelled (&team->barrier)
+		      || (thr->task->taskgroup
+			  && thr->task->taskgroup->cancelled)))
+		return;
 
-	  struct gomp_team *team = thr->ts.team;
-	  /* If parallel or taskgroup has been cancelled, don't start new
-	     tasks.  */
-	  if (team
-	      && (gomp_team_barrier_cancelled (&team->barrier)
-		  || (thr->task->taskgroup
-		      && thr->task->taskgroup->cancelled)))
-	    return;
-
-	  gomp_task_maybe_wait_for_dependencies (depend);
+	      gomp_task_maybe_wait_for_dependencies (depend);
+	    }
 	}
     }
 
@@ -1694,38 +1721,65 @@ GOMP_target_enter_exit_data (int device,
     gomp_exit_data (devicep, mapnum, hostaddrs, sizes, kinds);
 }
 
-void
+bool
 gomp_target_task_fn (void *data)
 {
   struct gomp_target_task *ttask = (struct gomp_target_task *) data;
+  struct gomp_device_descr *devicep = ttask->devicep;
+
   if (ttask->fn != NULL)
     {
-      /* GOMP_target_ext */
-    }
-  else if (ttask->devicep == NULL
-	   || !(ttask->devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
-    return;
+      if (devicep == NULL
+	  || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+	{
+	  ttask->state = GOMP_TARGET_TASK_FALLBACK;
+	  gomp_target_fallback_firstprivate (ttask->fn, ttask->mapnum,
+					     ttask->hostaddrs, ttask->sizes,
+					     ttask->kinds);
+	  return false;
+	}
+
+      if (ttask->state == GOMP_TARGET_TASK_FINISHED)
+	{
+	  gomp_unmap_vars (ttask->tgt, true);
+	  return false;
+	}
+
+      void *fn_addr = gomp_get_target_fn_addr (devicep, ttask->fn);
+      ttask->tgt
+	= gomp_map_vars (devicep, ttask->mapnum, ttask->hostaddrs, NULL,
+			 ttask->sizes, ttask->kinds, true,
+			 GOMP_MAP_VARS_TARGET);
+      ttask->state = GOMP_TARGET_TASK_READY_TO_RUN;
+
+      devicep->async_run_func (devicep->target_id, fn_addr,
+			       (void *) ttask->tgt->tgt_start, (void *) ttask);
+      return true;
+    }
+  else if (devicep == NULL
+	   || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+    return false;
 
   size_t i;
   if (ttask->flags & GOMP_TARGET_FLAG_UPDATE)
-    gomp_update (ttask->devicep, ttask->mapnum, ttask->hostaddrs, ttask->sizes,
+    gomp_update (devicep, ttask->mapnum, ttask->hostaddrs, ttask->sizes,
 		 ttask->kinds, true);
   else if ((ttask->flags & GOMP_TARGET_FLAG_EXIT_DATA) == 0)
     for (i = 0; i < ttask->mapnum; i++)
       if ((ttask->kinds[i] & 0xff) == GOMP_MAP_STRUCT)
 	{
-	  gomp_map_vars (ttask->devicep, ttask->sizes[i] + 1,
-			 &ttask->hostaddrs[i], NULL, &ttask->sizes[i],
-			 &ttask->kinds[i], true, GOMP_MAP_VARS_ENTER_DATA);
+	  gomp_map_vars (devicep, ttask->sizes[i] + 1, &ttask->hostaddrs[i],
+			 NULL, &ttask->sizes[i], &ttask->kinds[i], true,
+			 GOMP_MAP_VARS_ENTER_DATA);
 	  i += ttask->sizes[i];
 	}
       else
-	gomp_map_vars (ttask->devicep, 1, &ttask->hostaddrs[i], NULL,
-		       &ttask->sizes[i], &ttask->kinds[i],
-		       true, GOMP_MAP_VARS_ENTER_DATA);
+	gomp_map_vars (devicep, 1, &ttask->hostaddrs[i], NULL, &ttask->sizes[i],
+		       &ttask->kinds[i], true, GOMP_MAP_VARS_ENTER_DATA);
   else
-    gomp_exit_data (ttask->devicep, ttask->mapnum, ttask->hostaddrs,
-		    ttask->sizes, ttask->kinds);
+    gomp_exit_data (devicep, ttask->mapnum, ttask->hostaddrs, ttask->sizes,
+		    ttask->kinds);
+  return false;
 }
 
 void
@@ -2170,6 +2224,7 @@ gomp_load_plugin_for_device (struct gomp
   if (device->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400)
     {
       DLSYM (run);
+      DLSYM (async_run);
       DLSYM (dev2dev);
     }
   if (device->capabilities & GOMP_OFFLOAD_CAP_OPENACC_200)
--- libgomp/libgomp.map.jj	2015-10-26 15:38:20.000000000 +0100
+++ libgomp/libgomp.map	2015-11-11 16:15:23.807818735 +0100
@@ -406,3 +406,8 @@ GOMP_PLUGIN_1.0 {
 	GOMP_PLUGIN_async_unmap_vars;
 	GOMP_PLUGIN_acc_thread;
 };
+
+GOMP_PLUGIN_1.1 {
+  global:
+	GOMP_PLUGIN_target_task_completion;
+} GOMP_PLUGIN_1.0;
--- libgomp/task.c.jj	2015-11-09 11:14:37.332239862 +0100
+++ libgomp/task.c	2015-11-13 15:36:05.954411999 +0100
@@ -480,13 +480,119 @@ ialias (GOMP_taskgroup_end)
 #undef UTYPE
 #undef GOMP_taskloop
 
-/* Called for nowait target tasks.  */
+static void inline
+priority_queue_move_task_first (enum priority_queue_type type,
+				struct priority_queue *head,
+				struct gomp_task *task)
+{
+#if _LIBGOMP_CHECKING_
+  if (!priority_queue_task_in_queue_p (type, head, task))
+    gomp_fatal ("Attempt to move first missing task %p", task);
+#endif
+  struct priority_list *list;
+  if (priority_queue_multi_p (head))
+    {
+      list = priority_queue_lookup_priority (head, task->priority);
+#if _LIBGOMP_CHECKING_
+      if (!list)
+	gomp_fatal ("Unable to find priority %d", task->priority);
+#endif
+    }
+  else
+    list = &head->l;
+  priority_list_remove (list, task_to_priority_node (type, task), 0);
+  priority_list_insert (type, list, task, task->priority,
+			PRIORITY_INSERT_BEGIN, type == PQ_CHILDREN,
+			task->parent_depends_on);
+}
+
+/* Actual body of GOMP_PLUGIN_target_task_completion that is executed
+   with team->task_lock held, or is executed in the thread that called
+   gomp_target_task_fn if GOMP_PLUGIN_target_task_completion has been
+   run before it acquires team->task_lock.  */
+
+static void
+gomp_target_task_completion (struct gomp_team *team, struct gomp_task *task)
+{
+  struct gomp_task *parent = task->parent;
+  if (parent)
+    priority_queue_move_task_first (PQ_CHILDREN, &parent->children_queue,
+				    task);
+
+  struct gomp_taskgroup *taskgroup = task->taskgroup;
+  if (taskgroup)
+    priority_queue_move_task_first (PQ_TASKGROUP, &taskgroup->taskgroup_queue,
+				    task);
+
+  priority_queue_insert (PQ_TEAM, &team->task_queue, task, task->priority,
+			 PRIORITY_INSERT_BEGIN, false,
+			 task->parent_depends_on);
+  task->kind = GOMP_TASK_WAITING;
+  if (parent && parent->taskwait)
+    {
+      if (parent->taskwait->in_taskwait)
+	{
+	  /* One more task has had its dependencies met.
+	     Inform any waiters.  */
+	  parent->taskwait->in_taskwait = false;
+	  gomp_sem_post (&parent->taskwait->taskwait_sem);
+	}
+      else if (parent->taskwait->in_depend_wait)
+	{
+	  /* One more task has had its dependencies met.
+	     Inform any waiters.  */
+	  parent->taskwait->in_depend_wait = false;
+	  gomp_sem_post (&parent->taskwait->taskwait_sem);
+	}
+    }
+  if (taskgroup && taskgroup->in_taskgroup_wait)
+    {
+      /* One more task has had its dependencies met.
+	 Inform any waiters.  */
+      taskgroup->in_taskgroup_wait = false;
+      gomp_sem_post (&taskgroup->taskgroup_sem);
+    }
+
+  ++team->task_queued_count;
+  gomp_team_barrier_set_task_pending (&team->barrier);
+  /* I'm afraid this can't be done after releasing team->task_lock,
+     as gomp_target_task_completion is run from unrelated thread and
+     therefore in between gomp_mutex_unlock and gomp_team_barrier_wake
+     the team could be gone already.  */
+  if (team->nthreads > team->task_running_count)
+    gomp_team_barrier_wake (&team->barrier, 1);
+}
+
+/* Signal that a target task TTASK has completed the asynchronously
+   running phase and should be requeued as a task to handle the
+   variable unmapping.  */
 
 void
+GOMP_PLUGIN_target_task_completion (void *data)
+{
+  struct gomp_target_task *ttask = (struct gomp_target_task *) data;
+  struct gomp_task *task = ttask->task;
+  struct gomp_team *team = ttask->team;
+
+  gomp_mutex_lock (&team->task_lock);
+  if (ttask->state == GOMP_TARGET_TASK_READY_TO_RUN)
+    {
+      ttask->state = GOMP_TARGET_TASK_FINISHED;
+      gomp_mutex_unlock (&team->task_lock);
+    }
+  ttask->state = GOMP_TARGET_TASK_FINISHED;
+  gomp_target_task_completion (team, task);
+  gomp_mutex_unlock (&team->task_lock);
+}
+
+/* Called for nowait target tasks.  */
+
+bool
 gomp_create_target_task (struct gomp_device_descr *devicep,
 			 void (*fn) (void *), size_t mapnum, void **hostaddrs,
 			 size_t *sizes, unsigned short *kinds,
-			 unsigned int flags, void **depend)
+			 unsigned int flags, void **depend,
+			 enum gomp_target_task_state state)
 {
   struct gomp_thread *thr = gomp_thread ();
   struct gomp_team *team = thr->ts.team;
@@ -495,7 +601,7 @@ gomp_create_target_task (struct gomp_dev
   if (team
       && (gomp_team_barrier_cancelled (&team->barrier)
 	  || (thr->task->taskgroup && thr->task->taskgroup->cancelled)))
-    return;
+    return true;
 
   struct gomp_target_task *ttask;
   struct gomp_task *task;
@@ -503,19 +609,45 @@ gomp_create_target_task (struct gomp_dev
   struct gomp_taskgroup *taskgroup = parent->taskgroup;
   bool do_wake;
   size_t depend_size = 0;
+  uintptr_t depend_cnt = 0;
+  size_t tgt_align = 0, tgt_size = 0;
 
   if (depend != NULL)
-    depend_size = ((uintptr_t) depend[0]
-		   * sizeof (struct gomp_task_depend_entry));
+    {
+      depend_cnt = (uintptr_t) depend[0];
+      depend_size = depend_cnt * sizeof (struct gomp_task_depend_entry);
+    }
+  if (fn)
+    {
+      /* GOMP_MAP_FIRSTPRIVATE need to be copied first, as they are
+	 firstprivate on the target task.  */
+      size_t i;
+      for (i = 0; i < mapnum; i++)
+	if ((kinds[i] & 0xff) == GOMP_MAP_FIRSTPRIVATE)
+	  {
+	    size_t align = (size_t) 1 << (kinds[i] >> 8);
+	    if (tgt_align < align)
+	      tgt_align = align;
+	    tgt_size = (tgt_size + align - 1) & ~(align - 1);
+	    tgt_size += sizes[i];
+	  }
+      if (tgt_align)
+	tgt_size += tgt_align - 1;
+      else
+	tgt_size = 0;
+    }
+
   task = gomp_malloc (sizeof (*task) + depend_size
 		      + sizeof (*ttask)
 		      + mapnum * (sizeof (void *) + sizeof (size_t)
-				  + sizeof (unsigned short)));
+				  + sizeof (unsigned short))
+		      + tgt_size);
   gomp_init_task (task, parent, gomp_icv (false));
+  task->priority = 0;
   task->kind = GOMP_TASK_WAITING;
   task->in_tied_task = parent->in_tied_task;
   task->taskgroup = taskgroup;
-  ttask = (struct gomp_target_task *) &task->depend[(uintptr_t) depend[0]];
+  ttask = (struct gomp_target_task *) &task->depend[depend_cnt];
   ttask->devicep = devicep;
   ttask->fn = fn;
   ttask->mapnum = mapnum;
@@ -524,8 +656,29 @@ gomp_create_target_task (struct gomp_dev
   memcpy (ttask->sizes, sizes, mapnum * sizeof (size_t));
   ttask->kinds = (unsigned short *) &ttask->sizes[mapnum];
   memcpy (ttask->kinds, kinds, mapnum * sizeof (unsigned short));
+  if (tgt_align)
+    {
+      char *tgt = (char *) &ttask->kinds[mapnum];
+      size_t i;
+      uintptr_t al = (uintptr_t) tgt & (tgt_align - 1);
+      if (al)
+	tgt += tgt_align - al;
+      tgt_size = 0;
+      for (i = 0; i < mapnum; i++)
+	if ((kinds[i] & 0xff) == GOMP_MAP_FIRSTPRIVATE)
+	  {
+	    size_t align = (size_t) 1 << (kinds[i] >> 8);
+	    tgt_size = (tgt_size + align - 1) & ~(align - 1);
+	    memcpy (tgt + tgt_size, hostaddrs[i], sizes[i]);
+	    ttask->hostaddrs[i] = tgt + tgt_size;
+	    tgt_size = tgt_size + sizes[i];
+	  }
+    }
   ttask->flags = flags;
-  task->fn = gomp_target_task_fn;
+  ttask->state = state;
+  ttask->task = task;
+  ttask->team = team;
+  task->fn = NULL;
   task->fn_data = ttask;
   task->final_task = 0;
   gomp_mutex_lock (&team->task_lock);
@@ -536,19 +689,65 @@ gomp_create_target_task (struct gomp_dev
       gomp_mutex_unlock (&team->task_lock);
       gomp_finish_task (task);
       free (task);
-      return;
+      return true;
     }
-  if (taskgroup)
-    taskgroup->num_children++;
   if (depend_size)
     {
       gomp_task_handle_depend (task, parent, depend);
       if (task->num_dependees)
 	{
+	  if (taskgroup)
+	    taskgroup->num_children++;
 	  gomp_mutex_unlock (&team->task_lock);
-	  return;
+	  return true;
 	}
     }
+  if (state == GOMP_TARGET_TASK_DATA)
+    {
+      gomp_mutex_unlock (&team->task_lock);
+      gomp_finish_task (task);
+      free (task);
+      return false;
+    }
+  if (taskgroup)
+    taskgroup->num_children++;
+  /* For async offloading, if we don't need to wait for dependencies,
+     run the gomp_target_task_fn right away, essentially schedule the
+     mapping part of the task in the current thread.  */
+  if (devicep != NULL
+      && (devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+    {
+      priority_queue_insert (PQ_CHILDREN, &parent->children_queue, task, 0,
+			     PRIORITY_INSERT_END,
+			     /*adjust_parent_depends_on=*/false,
+			     task->parent_depends_on);
+      if (taskgroup)
+	priority_queue_insert (PQ_TASKGROUP, &taskgroup->taskgroup_queue,
+			       task, 0, PRIORITY_INSERT_END,
+			       /*adjust_parent_depends_on=*/false,
+			       task->parent_depends_on);
+      task->pnode[PQ_TEAM].next = NULL;
+      task->pnode[PQ_TEAM].prev = NULL;
+      task->kind = GOMP_TASK_TIED;
+      ++team->task_count;
+      gomp_mutex_unlock (&team->task_lock);
+
+      thr->task = task;
+      gomp_target_task_fn (task->fn_data);
+      thr->task = parent;
+
+      gomp_mutex_lock (&team->task_lock);
+      task->kind = GOMP_TASK_ASYNC_RUNNING;
+      /* If GOMP_PLUGIN_target_task_completion has run already
+	 in between gomp_target_task_fn and the mutex lock,
+	 perform the requeuing here.  */
+      if (ttask->state == GOMP_TARGET_TASK_FINISHED)
+	gomp_target_task_completion (team, task);
+      else
+	ttask->state = GOMP_TARGET_TASK_RUNNING;
+      gomp_mutex_unlock (&team->task_lock);
+      return true;
+    }
   priority_queue_insert (PQ_CHILDREN, &parent->children_queue, task, 0,
 			 PRIORITY_INSERT_BEGIN,
 			 /*adjust_parent_depends_on=*/false,
@@ -570,6 +769,7 @@ gomp_create_target_task (struct gomp_dev
   gomp_mutex_unlock (&team->task_lock);
   if (do_wake)
     gomp_team_barrier_wake (&team->barrier, 1);
+  return true;
 }
 
 /* Given a parent_depends_on task in LIST, move it to the front of its
@@ -1041,7 +1241,29 @@ gomp_barrier_handle_tasks (gomp_barrier_
       if (child_task)
 	{
 	  thr->task = child_task;
-	  child_task->fn (child_task->fn_data);
+	  if (__builtin_expect (child_task->fn == NULL, 0))
+	    {
+	      if (gomp_target_task_fn (child_task->fn_data))
+		{
+		  thr->task = task;
+		  gomp_mutex_lock (&team->task_lock);
+		  child_task->kind = GOMP_TASK_ASYNC_RUNNING;
+		  team->task_running_count--;
+		  struct gomp_target_task *ttask
+		    = (struct gomp_target_task *) child_task->fn_data;
+		  /* If GOMP_PLUGIN_target_task_completion has run already
+		     in between gomp_target_task_fn and the mutex lock,
+		     perform the requeuing here.  */
+		  if (ttask->state == GOMP_TARGET_TASK_FINISHED)
+		    gomp_target_task_completion (team, child_task);
+		  else
+		    ttask->state = GOMP_TARGET_TASK_RUNNING;
+		  child_task = NULL;
+		  continue;
+		}
+	    }
+	  else
+	    child_task->fn (child_task->fn_data);
 	  thr->task = task;
 	}
       else
@@ -1170,7 +1392,28 @@ GOMP_taskwait (void)
       if (child_task)
 	{
 	  thr->task = child_task;
-	  child_task->fn (child_task->fn_data);
+	  if (__builtin_expect (child_task->fn == NULL, 0))
+	    {
+	      if (gomp_target_task_fn (child_task->fn_data))
+		{
+		  thr->task = task;
+		  gomp_mutex_lock (&team->task_lock);
+		  child_task->kind = GOMP_TASK_ASYNC_RUNNING;
+		  struct gomp_target_task *ttask
+		    = (struct gomp_target_task *) child_task->fn_data;
+		  /* If GOMP_PLUGIN_target_task_completion has run already
+		     in between gomp_target_task_fn and the mutex lock,
+		     perform the requeuing here.  */
+		  if (ttask->state == GOMP_TARGET_TASK_FINISHED)
+		    gomp_target_task_completion (team, child_task);
+		  else
+		    ttask->state = GOMP_TARGET_TASK_RUNNING;
+		  child_task = NULL;
+		  continue;
+		}
+	    }
+	  else
+	    child_task->fn (child_task->fn_data);
 	  thr->task = task;
 	}
       else
@@ -1342,7 +1585,28 @@ gomp_task_maybe_wait_for_dependencies (v
       if (child_task)
 	{
 	  thr->task = child_task;
-	  child_task->fn (child_task->fn_data);
+	  if (__builtin_expect (child_task->fn == NULL, 0))
+	    {
+	      if (gomp_target_task_fn (child_task->fn_data))
+		{
+		  thr->task = task;
+		  gomp_mutex_lock (&team->task_lock);
+		  child_task->kind = GOMP_TASK_ASYNC_RUNNING;
+		  struct gomp_target_task *ttask
+		    = (struct gomp_target_task *) child_task->fn_data;
+		  /* If GOMP_PLUGIN_target_task_completion has run already
+		     in between gomp_target_task_fn and the mutex lock,
+		     perform the requeuing here.  */
+		  if (ttask->state == GOMP_TARGET_TASK_FINISHED)
+		    gomp_target_task_completion (team, child_task);
+		  else
+		    ttask->state = GOMP_TARGET_TASK_RUNNING;
+		  child_task = NULL;
+		  continue;
+		}
+	    }
+	  else
+	    child_task->fn (child_task->fn_data);
 	  thr->task = task;
 	}
       else
@@ -1423,6 +1687,17 @@ GOMP_taskgroup_end (void)
   if (team == NULL)
     return;
   taskgroup = task->taskgroup;
+  if (__builtin_expect (taskgroup == NULL, 0)
+      && thr->ts.level == 0)
+    {
+      /* This can happen if GOMP_taskgroup_start is called when
+	 thr->ts.team == NULL, but inside of the taskgroup there
+	 is #pragma omp target nowait that creates an implicit
+	 team with a single thread.  In this case, we want to wait
+	 for all outstanding tasks in this team.  */
+      gomp_team_barrier_wait (&team->barrier);
+      return;
+    }
 
   /* The acquire barrier on load of taskgroup->num_children here
      synchronizes with the write of 0 in gomp_task_run_post_remove_taskgroup.
@@ -1450,8 +1725,8 @@ GOMP_taskgroup_end (void)
 		= priority_queue_next_task (PQ_CHILDREN, &task->children_queue,
 					    PQ_TEAM, &team->task_queue,
 					    &unused);
-            }
-          else
+	    }
+	  else
 	    {
 	      gomp_mutex_unlock (&team->task_lock);
 	      if (to_free)
@@ -1506,7 +1781,28 @@ GOMP_taskgroup_end (void)
       if (child_task)
 	{
 	  thr->task = child_task;
-	  child_task->fn (child_task->fn_data);
+	  if (__builtin_expect (child_task->fn == NULL, 0))
+	    {
+	      if (gomp_target_task_fn (child_task->fn_data))
+		{
+		  thr->task = task;
+		  gomp_mutex_lock (&team->task_lock);
+		  child_task->kind = GOMP_TASK_ASYNC_RUNNING;
+		  struct gomp_target_task *ttask
+		    = (struct gomp_target_task *) child_task->fn_data;
+		  /* If GOMP_PLUGIN_target_task_completion has run already
+		     in between gomp_target_task_fn and the mutex lock,
+		     perform the requeuing here.  */
+		  if (ttask->state == GOMP_TARGET_TASK_FINISHED)
+		    gomp_target_task_completion (team, child_task);
+		  else
+		    ttask->state = GOMP_TARGET_TASK_RUNNING;
+		  child_task = NULL;
+		  continue;
+		}
+	    }
+	  else
+	    child_task->fn (child_task->fn_data);
 	  thr->task = task;
 	}
       else
--- libgomp/priority_queue.c.jj	2015-11-09 11:15:33.000000000 +0100
+++ libgomp/priority_queue.c	2015-11-10 17:52:33.769414428 +0100
@@ -85,7 +85,7 @@ priority_queue_task_in_queue_p (enum pri
    order.  LIST is a priority list of type TYPE.
 
    The expected order is that GOMP_TASK_WAITING tasks come before
-   GOMP_TASK_TIED ones.
+   GOMP_TASK_TIED/GOMP_TASK_ASYNC_RUNNING ones.
 
    If CHECK_DEPS is TRUE, we also check that parent_depends_on WAITING
    tasks come before !parent_depends_on WAITING tasks.  This is only
@@ -104,7 +104,7 @@ priority_list_verify (enum priority_queu
       struct gomp_task *t = priority_node_to_task (type, p);
       if (seen_tied && t->kind == GOMP_TASK_WAITING)
 	gomp_fatal ("priority_queue_verify: WAITING task after TIED");
-      if (t->kind == GOMP_TASK_TIED)
+      if (t->kind >= GOMP_TASK_TIED)
 	seen_tied = true;
       else if (check_deps && t->kind == GOMP_TASK_WAITING)
 	{
--- libgomp/libgomp.h.jj	2015-11-09 11:14:37.326239947 +0100
+++ libgomp/libgomp.h	2015-11-13 11:41:46.743939113 +0100
@@ -373,7 +373,12 @@ enum gomp_task_kind
   /* Task created by GOMP_task and waiting to be run.  */
   GOMP_TASK_WAITING,
   /* Task currently executing or scheduled and about to execute.  */
-  GOMP_TASK_TIED
+  GOMP_TASK_TIED,
+  /* Used for target tasks that have vars mapped and async run started,
+     but not yet completed.  Once that completes, they will be readded
+     into the queues as GOMP_TASK_WAITING in order to perform the var
+     unmapping.  */
+  GOMP_TASK_ASYNC_RUNNING
 };
 
 struct gomp_task_depend_entry
@@ -453,6 +458,8 @@ struct gomp_task
   struct gomp_task_depend_entry depend[];
 };
 
+/* This structure describes a single #pragma omp taskgroup.  */
+
 struct gomp_taskgroup
 {
   struct gomp_taskgroup *prev;
@@ -464,6 +471,19 @@ struct gomp_taskgroup
   size_t num_children;
 };
 
+/* Various state of OpenMP async offloading tasks.  */
+enum gomp_target_task_state
+{
+  GOMP_TARGET_TASK_DATA,
+  GOMP_TARGET_TASK_BEFORE_MAP,
+  GOMP_TARGET_TASK_FALLBACK,
+  GOMP_TARGET_TASK_READY_TO_RUN,
+  GOMP_TARGET_TASK_RUNNING,
+  GOMP_TARGET_TASK_FINISHED
+};
+
+/* This structure describes a target task.  */
+
 struct gomp_target_task
 {
   struct gomp_device_descr *devicep;
@@ -472,6 +492,10 @@ struct gomp_target_task
   size_t *sizes;
   unsigned short *kinds;
   unsigned int flags;
+  enum gomp_target_task_state state;
+  struct target_mem_desc *tgt;
+  struct gomp_task *task;
+  struct gomp_team *team;
   void *hostaddrs[];
 };
 
@@ -723,10 +747,10 @@ extern void gomp_init_task (struct gomp_
 extern void gomp_end_task (void);
 extern void gomp_barrier_handle_tasks (gomp_barrier_state_t);
 extern void gomp_task_maybe_wait_for_dependencies (void **);
-extern void gomp_create_target_task (struct gomp_device_descr *,
+extern bool gomp_create_target_task (struct gomp_device_descr *,
 				     void (*) (void *), size_t, void **,
 				     size_t *, unsigned short *, unsigned int,
-				     void **);
+				     void **, enum gomp_target_task_state);
 
 static void inline
 gomp_finish_task (struct gomp_task *task)
@@ -747,7 +771,7 @@ extern void gomp_free_thread (void *);
 
 extern void gomp_init_targets_once (void);
 extern int gomp_get_num_devices (void);
-extern void gomp_target_task_fn (void *);
+extern bool gomp_target_task_fn (void *);
 
 /* Splay tree definitions.  */
 typedef struct splay_tree_node_s *splay_tree_node;
@@ -901,6 +925,7 @@ struct gomp_device_descr
   void *(*host2dev_func) (int, void *, const void *, size_t);
   void *(*dev2dev_func) (int, void *, const void *, size_t);
   void (*run_func) (int, void *, void *);
+  void (*async_run_func) (int, void *, void *, void *);
 
   /* Splay tree containing information about mapped memory regions.  */
   struct splay_tree_s mem_map;

	Jakub

^ permalink raw reply	[flat|nested] 23+ messages in thread

* Re: [gomp4.5] depend nowait support for target
  2015-11-13 15:12               ` Jakub Jelinek
@ 2015-11-13 16:37                 ` Ilya Verbin
  2015-11-13 16:42                   ` Jakub Jelinek
  2015-11-23 14:16                 ` [hsa] " Martin Jambor
  1 sibling, 1 reply; 23+ messages in thread
From: Ilya Verbin @ 2015-11-13 16:37 UTC (permalink / raw)
  To: Jakub Jelinek
  Cc: Aldy Hernandez, gcc-patches, Kirill Yukhin, Thomas Schwinge,
	Alexander Monakov, Martin Jambor

On Fri, Nov 13, 2015 at 16:11:50 +0100, Jakub Jelinek wrote:
> On Fri, Nov 13, 2015 at 11:18:41AM +0100, Jakub Jelinek wrote:
> > For the offloading case, I actually see a problematic spot, namely that
> > GOMP_PLUGIN_target_task_completion could finish too early, and get the
> > task_lock before the thread that run the gomp_target_task_fn doing map_vars
> > + async_run for it.  Bet I need to add further ttask state kinds and deal
> > with that case (so GOMP_PLUGIN_target_task_completion would just take the
> > task lock and tweak ttask state if it has not been added to the queues
> > yet).
> > Plus I think I want to improve the case where we are not waiting, in
> > gomp_create_target_task if not waiting for dependencies actually schedule
> > manually the gomp_target_task_fn.
> 
> These two have been resolved, plus target-34.c issue resolved too (the bug
> was that I've been too lazy and just put target-33.c test into #pragma omp
> parallel #pragma omp single, but that is invalid OpenMP, as single is a
> worksharing region and #pragma omp barrier may not be encountered in such a
> region.  Fixed by rewriting the testcase.
> 
> So here is a full patch that passes for me both non-offloading and
> offloading, OMP_NUM_THREADS=16 (implicit on my box) as well as
> OMP_NUM_THREADS=1 (explicit).  I've incorporated your incremental patch.
> 
> One option to avoid the static variable would be to pass two pointers
> instead of one (async_data), one would be the callback function pointer,
> another argument to it.  Or another possibility would be to say that
> the async_data argument the plugin passes to liboffloadmic would be
> pointer to structure, holding a function pointer (completion callback)
> and the data pointer to pass to it, and then the plugin would just
> GOMP_PLUGIN_malloc 2 * sizeof (void *) for it, fill it in and
> register some function in itself that would call the
> GOMP_PLUGIN_target_task_completion with the second structure element
> as argument and then free the structure pointer.

I don't know which interface to implement to maintain compatibility in the
future.
Anyway, currently it's impossible that a process will use the same liboffloadmic
for 2 different offloading paths (say GCC's in exec and ICC's in a dso), because
in fact GCC's and ICC's libraries are not the same.  First of all, they have
different names: liboffloadmic in GCC and just liboffload in ICC.  And most
importantly, ICC's version contains some references to libiomp5, which were
removed form GCC's version.  In theory, we want to use one library with all
compilers, but I'm not sure when it will be possible.

> Do you get still crashes on any of the testcases with this?

No, all tests now pass using emul.  I'll report when I have any results on HW.

Thanks,
  -- Ilya

^ permalink raw reply	[flat|nested] 23+ messages in thread

* Re: [gomp4.5] depend nowait support for target
  2015-11-13 16:37                 ` Ilya Verbin
@ 2015-11-13 16:42                   ` Jakub Jelinek
  2015-11-13 18:37                     ` Ilya Verbin
  0 siblings, 1 reply; 23+ messages in thread
From: Jakub Jelinek @ 2015-11-13 16:42 UTC (permalink / raw)
  To: Ilya Verbin
  Cc: Aldy Hernandez, gcc-patches, Kirill Yukhin, Thomas Schwinge,
	Alexander Monakov, Martin Jambor

On Fri, Nov 13, 2015 at 07:37:17PM +0300, Ilya Verbin wrote:
> I don't know which interface to implement to maintain compatibility in the
> future.
> Anyway, currently it's impossible that a process will use the same liboffloadmic
> for 2 different offloading paths (say GCC's in exec and ICC's in a dso), because
> in fact GCC's and ICC's libraries are not the same.  First of all, they have
> different names: liboffloadmic in GCC and just liboffload in ICC.  And most
> importantly, ICC's version contains some references to libiomp5, which were
> removed form GCC's version.  In theory, we want to use one library with all
> compilers, but I'm not sure when it will be possible.

Ok, in that case it is less of a problem.

> > Do you get still crashes on any of the testcases with this?
> 
> No, all tests now pass using emul.  I'll report when I have any results on HW.

Perfect, I'll commit it to gomp-4_5-branch then.

Thanks.

	Jakub

^ permalink raw reply	[flat|nested] 23+ messages in thread

* Re: [gomp4.5] depend nowait support for target
  2015-11-13 16:42                   ` Jakub Jelinek
@ 2015-11-13 18:37                     ` Ilya Verbin
  0 siblings, 0 replies; 23+ messages in thread
From: Ilya Verbin @ 2015-11-13 18:37 UTC (permalink / raw)
  To: Jakub Jelinek
  Cc: Aldy Hernandez, gcc-patches, Kirill Yukhin, Thomas Schwinge,
	Alexander Monakov, Martin Jambor

On Fri, Nov 13, 2015 at 17:41:53 +0100, Jakub Jelinek wrote:
> On Fri, Nov 13, 2015 at 07:37:17PM +0300, Ilya Verbin wrote:
> > I don't know which interface to implement to maintain compatibility in the
> > future.
> > Anyway, currently it's impossible that a process will use the same liboffloadmic
> > for 2 different offloading paths (say GCC's in exec and ICC's in a dso), because
> > in fact GCC's and ICC's libraries are not the same.  First of all, they have
> > different names: liboffloadmic in GCC and just liboffload in ICC.  And most
> > importantly, ICC's version contains some references to libiomp5, which were
> > removed form GCC's version.  In theory, we want to use one library with all
> > compilers, but I'm not sure when it will be possible.
> 
> Ok, in that case it is less of a problem.
> 
> > > Do you get still crashes on any of the testcases with this?
> > 
> > No, all tests now pass using emul.  I'll report when I have any results on HW.
> 
> Perfect, I'll commit it to gomp-4_5-branch then.

make check-target-libgomp with offloading to HW also passed :)

And this:

+++ b/libgomp/testsuite/libgomp.c/target-32.c
@@ -3,6 +3,7 @@
 
 int main ()
 {
+  int x = 1;
   int a = 0, b = 0, c = 0, d[7];
 
   #pragma omp parallel
@@ -18,6 +19,7 @@ int main ()
 
     #pragma omp target nowait map(alloc: b) depend(in: d[2]) depend(out: d[3])
     {
+      while (x);
       usleep (1000);
       #pragma omp atomic update
       b |= 4;
@@ -25,6 +27,7 @@ int main ()
 
     #pragma omp target nowait map(alloc: b) depend(in: d[2]) depend(out: d[4])
     {
+      while (x);
       usleep (5000);
       #pragma omp atomic update
       b |= 1;

demonstrates 200% CPU usage both using emul and HW, so 2 target tasks really run
concurrently.

  -- Ilya

^ permalink raw reply	[flat|nested] 23+ messages in thread

* [hsa] depend nowait support for target
  2015-11-13 15:12               ` Jakub Jelinek
  2015-11-13 16:37                 ` Ilya Verbin
@ 2015-11-23 14:16                 ` Martin Jambor
  2015-11-23 14:25                   ` Jakub Jelinek
  1 sibling, 1 reply; 23+ messages in thread
From: Martin Jambor @ 2015-11-23 14:16 UTC (permalink / raw)
  To: Jakub Jelinek
  Cc: Ilya Verbin, Aldy Hernandez, gcc-patches, Kirill Yukhin,
	Thomas Schwinge, Alexander Monakov

On Fri, Nov 13, 2015 at 04:11:50PM +0100, Jakub Jelinek wrote:
> On Fri, Nov 13, 2015 at 11:18:41AM +0100, Jakub Jelinek wrote:
> > For the offloading case, I actually see a problematic spot, namely that
> > GOMP_PLUGIN_target_task_completion could finish too early, and get the
> > task_lock before the thread that run the gomp_target_task_fn doing map_vars
> > + async_run for it.  Bet I need to add further ttask state kinds and deal
> > with that case (so GOMP_PLUGIN_target_task_completion would just take the
> > task lock and tweak ttask state if it has not been added to the queues
> > yet).
> > Plus I think I want to improve the case where we are not waiting, in
> > gomp_create_target_task if not waiting for dependencies actually schedule
> > manually the gomp_target_task_fn.
> 
> These two have been resolved, plus target-34.c issue resolved too (the bug
> was that I've been too lazy and just put target-33.c test into #pragma omp
> parallel #pragma omp single, but that is invalid OpenMP, as single is a
> worksharing region and #pragma omp barrier may not be encountered in such a
> region.  Fixed by rewriting the testcase.
> 
> So here is a full patch that passes for me both non-offloading and
> offloading, OMP_NUM_THREADS=16 (implicit on my box) as well as
> OMP_NUM_THREADS=1 (explicit).  I've incorporated your incremental patch.
> 

I have committed the following patch to the hsa branch to implement
GOMP_OFFLOAD_async_run.  Tests target-33.c and target-34.c pass right
away.  I also do not have any usleep on HSA, so I only ran target-32.c
manually after replacing the usleeps with some pointless busy looping.

During the testing, I have come accross quite a few places where
libgomp has to treat shared memory devices like it treats host, and so
I added that to the patch too.

The hunk in gomp_create_target_task should have been in the previous
merge from trunk but I forgot to add it then.

Any feedback welcome,

Martin


2015-11-23  Martin Jambor  <mjambor@suse.cz>

libgomp/
	* plugin/plugin-hsa.c (async_run_info): New structure.
	(run_kernel_asynchronously): New function.
	(GOMP_OFFLOAD_async_run): New implementation.
	* target.c (GOMP_target_data_ext): Handle shared memory devices like
	the host.
	(GOMP_target_update): Likewise.
	(GOMP_target_update_ext): Likewise.
	(GOMP_target_enter_exit_data): Likewise.
	(omp_target_alloc): Likewise.
	(omp_target_free): Likewise.
	(omp_target_memcpy): Likewise.
	(omp_target_memcpy_rect): Likewise.
	* task.c (gomp_create_target_task): Fill in args field of ttask.
---
 libgomp/plugin/plugin-hsa.c | 61 ++++++++++++++++++++++++++++++++++++++++-----
 libgomp/target.c            | 30 ++++++++++++++--------
 libgomp/task.c              |  1 +
 3 files changed, 76 insertions(+), 16 deletions(-)

diff --git a/libgomp/plugin/plugin-hsa.c b/libgomp/plugin/plugin-hsa.c
index 40dbcde..72f5bdd 100644
--- a/libgomp/plugin/plugin-hsa.c
+++ b/libgomp/plugin/plugin-hsa.c
@@ -1127,9 +1127,9 @@ failure:
   return false;
 }
 
-/* Part of the libgomp plugin interface.  Run a kernel on a device N and pass
-   the it an array of pointers in VARS as a parameter.  The kernel is
-   identified by FN_PTR which must point to a kernel_info structure.  */
+/* Part of the libgomp plugin interface.  Run a kernel on device N and pass it
+   an array of pointers in VARS as a parameter.  The kernel is identified by
+   FN_PTR which must point to a kernel_info structure.  */
 
 void
 GOMP_OFFLOAD_run (int n, void *fn_ptr, void *vars, void** args)
@@ -1237,13 +1237,62 @@ GOMP_OFFLOAD_run (int n, void *fn_ptr, void *vars, void** args)
     GOMP_PLUGIN_fatal ("Unable to unlock an HSA agent rwlock");
 }
 
+/* Information to be passed to a thread running a kernel asycnronously.  */
+
+struct async_run_info
+{
+  int device;
+  void *tgt_fn;
+  void *tgt_vars;
+  void **args;
+  void *async_data;
+};
+
+/* Thread routine to run a kernel asynchronously.  */
+
+static void *
+run_kernel_asynchronously (void *thread_arg)
+{
+  struct async_run_info *info = (struct async_run_info *) thread_arg;
+  int device = info->device;
+  void *tgt_fn = info->tgt_fn;
+  void *tgt_vars = info->tgt_vars;
+  void **args = info->args;
+  void *async_data = info->async_data;
+
+  free (info);
+  GOMP_OFFLOAD_run (device, tgt_fn, tgt_vars, args);
+  GOMP_PLUGIN_target_task_completion (async_data);
+  return NULL;
+}
+
+/* Part of the libgomp plugin interface.  Run a kernel like GOMP_OFFLOAD_run
+   does, but asynchronously and call GOMP_PLUGIN_target_task_completion when it
+   has finished.  */
+
 void
 GOMP_OFFLOAD_async_run (int device, void *tgt_fn, void *tgt_vars,
 			void **args, void *async_data)
 {
-  /* FIXME: Implement.  */
-  GOMP_PLUGIN_fatal ("Support for HSA does not yet implement asynchronous "
-		     "execution. ");
+  pthread_t pt;
+  struct async_run_info *info;
+  HSA_DEBUG ("GOMP_OFFLOAD_async_run invoked\n")
+  info = GOMP_PLUGIN_malloc (sizeof (struct async_run_info));
+
+  info->device = device;
+  info->tgt_fn = tgt_fn;
+  info->tgt_vars = tgt_vars;
+  info->args = args;
+  info->async_data = async_data;
+
+  int err = pthread_create (&pt, NULL, &run_kernel_asynchronously, info);
+  if (err != 0)
+    GOMP_PLUGIN_fatal ("HSA asynchronous thread creation failed: %s",
+		       strerror (err));
+  err = pthread_detach (pt);
+  if (err != 0)
+    GOMP_PLUGIN_fatal ("Failed to detach a thread to run HRA kernel "
+		       "asynchronously: %s", strerror (err));
 }
 
 /* Deinitialize all information associated with MODULE and kernels within
diff --git a/libgomp/target.c b/libgomp/target.c
index a771d7d..f8a9803 100644
--- a/libgomp/target.c
+++ b/libgomp/target.c
@@ -1527,7 +1527,8 @@ GOMP_target_data_ext (int device, size_t mapnum, void **hostaddrs,
   struct gomp_device_descr *devicep = resolve_device (device);
 
   if (devicep == NULL
-      || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+      || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400)
+      || devicep->capabilities & GOMP_OFFLOAD_CAP_SHARED_MEM)
     return gomp_target_data_fallback ();
 
   struct target_mem_desc *tgt
@@ -1557,7 +1558,8 @@ GOMP_target_update (int device, const void *unused, size_t mapnum,
   struct gomp_device_descr *devicep = resolve_device (device);
 
   if (devicep == NULL
-      || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+      || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400)
+      || devicep->capabilities & GOMP_OFFLOAD_CAP_SHARED_MEM)
     return;
 
   gomp_update (devicep, mapnum, hostaddrs, sizes, kinds, false);
@@ -1608,7 +1610,8 @@ GOMP_target_update_ext (int device, size_t mapnum, void **hostaddrs,
     }
 
   if (devicep == NULL
-      || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+      || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400)
+      || devicep->capabilities & GOMP_OFFLOAD_CAP_SHARED_MEM)
     return;
 
   struct gomp_thread *thr = gomp_thread ();
@@ -1730,7 +1733,8 @@ GOMP_target_enter_exit_data (int device, size_t mapnum, void **hostaddrs,
     }
 
   if (devicep == NULL
-      || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+      || !(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400)
+      || devicep->capabilities & GOMP_OFFLOAD_CAP_SHARED_MEM)
     return;
 
   struct gomp_thread *thr = gomp_thread ();
@@ -1861,7 +1865,8 @@ omp_target_alloc (size_t size, int device_num)
   if (devicep == NULL)
     return NULL;
 
-  if (!(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+  if (!(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400)
+      || devicep->capabilities & GOMP_OFFLOAD_CAP_SHARED_MEM)
     return malloc (size);
 
   gomp_mutex_lock (&devicep->lock);
@@ -1889,7 +1894,8 @@ omp_target_free (void *device_ptr, int device_num)
   if (devicep == NULL)
     return;
 
-  if (!(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+  if (!(devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400)
+      || devicep->capabilities & GOMP_OFFLOAD_CAP_SHARED_MEM)
     {
       free (device_ptr);
       return;
@@ -1946,7 +1952,8 @@ omp_target_memcpy (void *dst, void *src, size_t length, size_t dst_offset,
       if (dst_devicep == NULL)
 	return EINVAL;
 
-      if (!(dst_devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+      if (!(dst_devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400)
+	  || dst_devicep->capabilities & GOMP_OFFLOAD_CAP_SHARED_MEM)
 	dst_devicep = NULL;
     }
   if (src_device_num != GOMP_DEVICE_HOST_FALLBACK)
@@ -1958,7 +1965,8 @@ omp_target_memcpy (void *dst, void *src, size_t length, size_t dst_offset,
       if (src_devicep == NULL)
 	return EINVAL;
 
-      if (!(src_devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+      if (!(src_devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400)
+	  || src_devicep->capabilities & GOMP_OFFLOAD_CAP_SHARED_MEM)
 	src_devicep = NULL;
     }
   if (src_devicep == NULL && dst_devicep == NULL)
@@ -2088,7 +2096,8 @@ omp_target_memcpy_rect (void *dst, void *src, size_t element_size,
       if (dst_devicep == NULL)
 	return EINVAL;
 
-      if (!(dst_devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+      if (!(dst_devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400)
+	  || dst_devicep->capabilities & GOMP_OFFLOAD_CAP_SHARED_MEM)
 	dst_devicep = NULL;
     }
   if (src_device_num != GOMP_DEVICE_HOST_FALLBACK)
@@ -2100,7 +2109,8 @@ omp_target_memcpy_rect (void *dst, void *src, size_t element_size,
       if (src_devicep == NULL)
 	return EINVAL;
 
-      if (!(src_devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400))
+      if (!(src_devicep->capabilities & GOMP_OFFLOAD_CAP_OPENMP_400)
+	  || src_devicep->capabilities & GOMP_OFFLOAD_CAP_SHARED_MEM)
 	src_devicep = NULL;
     }
 
diff --git a/libgomp/task.c b/libgomp/task.c
index 838ef1a..18d40cf 100644
--- a/libgomp/task.c
+++ b/libgomp/task.c
@@ -652,6 +652,7 @@ gomp_create_target_task (struct gomp_device_descr *devicep,
   ttask->devicep = devicep;
   ttask->fn = fn;
   ttask->mapnum = mapnum;
+  ttask->args = args;
   memcpy (ttask->hostaddrs, hostaddrs, mapnum * sizeof (void *));
   ttask->sizes = (size_t *) &ttask->hostaddrs[mapnum];
   memcpy (ttask->sizes, sizes, mapnum * sizeof (size_t));
-- 
2.6.0

^ permalink raw reply	[flat|nested] 23+ messages in thread

* Re: [hsa] depend nowait support for target
  2015-11-23 14:16                 ` [hsa] " Martin Jambor
@ 2015-11-23 14:25                   ` Jakub Jelinek
  2015-11-25 15:41                     ` Martin Jambor
  0 siblings, 1 reply; 23+ messages in thread
From: Jakub Jelinek @ 2015-11-23 14:25 UTC (permalink / raw)
  To: Ilya Verbin, Aldy Hernandez, gcc-patches, Kirill Yukhin,
	Thomas Schwinge, Alexander Monakov

On Mon, Nov 23, 2015 at 03:12:05PM +0100, Martin Jambor wrote:
> +/* Thread routine to run a kernel asynchronously.  */
> +
> +static void *
> +run_kernel_asynchronously (void *thread_arg)
> +{
> +  struct async_run_info *info = (struct async_run_info *) thread_arg;
> +  int device = info->device;
> +  void *tgt_fn = info->tgt_fn;
> +  void *tgt_vars = info->tgt_vars;
> +  void **args = info->args;
> +  void *async_data = info->async_data;
> +
> +  free (info);
> +  GOMP_OFFLOAD_run (device, tgt_fn, tgt_vars, args);
> +  GOMP_PLUGIN_target_task_completion (async_data);
> +  return NULL;

Is this just a temporary hack to work-around the missing task.c/target.c
support for plugins that need polling (calling some hook) to determine
completion of the tasks, or there is no way to tell HSA to spawn something
asynchronously?
Short term it is ok this way.

> +  int err = pthread_create (&pt, NULL, &run_kernel_asynchronously, info);
> +  if (err != 0)
> +    GOMP_PLUGIN_fatal ("HSA asynchronous thread creation failed: %s",
> +		       strerror (err));
> +  err = pthread_detach (pt);
> +  if (err != 0)
> +    GOMP_PLUGIN_fatal ("Failed to detach a thread to run HRA kernel "
> +		       "asynchronously: %s", strerror (err));

HSA instead of HRA?

	Jakub

^ permalink raw reply	[flat|nested] 23+ messages in thread

* Re: [hsa] depend nowait support for target
  2015-11-23 14:25                   ` Jakub Jelinek
@ 2015-11-25 15:41                     ` Martin Jambor
  0 siblings, 0 replies; 23+ messages in thread
From: Martin Jambor @ 2015-11-25 15:41 UTC (permalink / raw)
  To: Jakub Jelinek
  Cc: Ilya Verbin, Aldy Hernandez, gcc-patches, Kirill Yukhin,
	Thomas Schwinge, Alexander Monakov

On Mon, Nov 23, 2015 at 03:16:42PM +0100, Jakub Jelinek wrote:
> On Mon, Nov 23, 2015 at 03:12:05PM +0100, Martin Jambor wrote:
> > +/* Thread routine to run a kernel asynchronously.  */
> > +
> > +static void *
> > +run_kernel_asynchronously (void *thread_arg)
> > +{
> > +  struct async_run_info *info = (struct async_run_info *) thread_arg;
> > +  int device = info->device;
> > +  void *tgt_fn = info->tgt_fn;
> > +  void *tgt_vars = info->tgt_vars;
> > +  void **args = info->args;
> > +  void *async_data = info->async_data;
> > +
> > +  free (info);
> > +  GOMP_OFFLOAD_run (device, tgt_fn, tgt_vars, args);
> > +  GOMP_PLUGIN_target_task_completion (async_data);
> > +  return NULL;
> 
> Is this just a temporary hack to work-around the missing task.c/target.c
> support for plugins that need polling (calling some hook) to determine
> completion of the tasks, or there is no way to tell HSA to spawn something
> asynchronously?
> Short term it is ok this way.

Basically yes.  There is no way to tell HSA-run time to be notified of
kernel completion.  If libgomp provides a way to poll the device, I'll
gladly use that instead.

> 
> > +  int err = pthread_create (&pt, NULL, &run_kernel_asynchronously, info);
> > +  if (err != 0)
> > +    GOMP_PLUGIN_fatal ("HSA asynchronous thread creation failed: %s",
> > +		       strerror (err));
> > +  err = pthread_detach (pt);
> > +  if (err != 0)
> > +    GOMP_PLUGIN_fatal ("Failed to detach a thread to run HRA kernel "
> > +		       "asynchronously: %s", strerror (err));
> 
> HSA instead of HRA?
> 

Oh, thanks.  Will fix.

Martin

^ permalink raw reply	[flat|nested] 23+ messages in thread

end of thread, other threads:[~2015-11-25 15:34 UTC | newest]

Thread overview: 23+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2015-09-08  9:26 [gomp4.1] depend nowait support for target {update,{enter,exit} data} Jakub Jelinek
2015-10-02 19:28 ` Ilya Verbin
2015-10-15 14:02   ` Jakub Jelinek
2015-10-15 16:18     ` Alexander Monakov
2015-10-15 17:18       ` Jakub Jelinek
2015-10-15 18:11         ` Alexander Monakov
2015-10-15 16:42     ` Ilya Verbin
2015-10-16 11:50     ` Martin Jambor
2015-10-19 19:55     ` Ilya Verbin
2015-11-11 16:52       ` [gomp4.5] depend nowait support for target Jakub Jelinek
2015-11-12 17:44         ` Ilya Verbin
2015-11-12 17:58           ` Jakub Jelinek
2015-11-12 18:07             ` Ilya Verbin
2015-11-12 17:45         ` Jakub Jelinek
2015-11-12 20:52           ` Ilya Verbin
2015-11-13 10:18             ` Jakub Jelinek
2015-11-13 15:12               ` Jakub Jelinek
2015-11-13 16:37                 ` Ilya Verbin
2015-11-13 16:42                   ` Jakub Jelinek
2015-11-13 18:37                     ` Ilya Verbin
2015-11-23 14:16                 ` [hsa] " Martin Jambor
2015-11-23 14:25                   ` Jakub Jelinek
2015-11-25 15:41                     ` Martin Jambor

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for read-only IMAP folder(s) and NNTP newsgroup(s).