[gomp4] Taskgroup library support

Jakub Jelinek jakub@redhat.com
Tue Sep 24 20:32:00 GMT 2013


Hi!

This implements taskgroups in the library and their cancellation.
The implementation has been pretty straightforward, though I had to
consolidate some operations from {gomp_barrier_handle_tasks, GOMP_taskwait}
and the new GOMP_taskgroup_end to new inlines, because it became
non-maintainable.

In addition to this, the patch disallows plain discarding of tasks for which
we've already run the copy constructors, those will be executed and will be
cancelled only if they encounter a cancellation point.  There are omp-lang
discussions about whether the standard shouldn't be changed, so that the
copy ctors would be run only in task outlined body, not earlier.

And, lastly, this patch adds various extra cancellation testcase that
revealed the omp-low.c issue fixed in the previous patch.

Will commit tomorrow unless somebody complains.

2013-09-24  Jakub Jelinek  <jakub@redhat.com>

	* parallel.c (GOMP_cancellation_point, GOMP_cancel): Handle
	GIMPLE_CANCEL_TASKGROUP cancellation.
	* libgomp.h (struct gomp_task): Add next_taskgroup, prev_taskgroup,
	taskgroup and copy_ctors_done fields.
	(struct gomp_taskgroup): New type.
	* task.c (gomp_init_task): Initialize copy_ctors_done and taskgroup
	fields.
	(GOMP_task): Don't start a new thread also if it's taskgroup has
	been cancelled.  Set copy_ctors_done field if needed.  Initialize
	taskgroup field.  If copy_ctors_done and already cancelled, don't
	discard the task.  If taskgroup is non-NULL, enqueue the task
	into taskgroup queue.
	(gomp_task_run_pre, gomp_task_run_post_remove_parent,
	gomp_task_run_post_remove_taskgroup): New inline functions.
	(gomp_barrier_handle_tasks, GOMP_taskwait): Use them.
	(GOMP_taskgroup_start, GOMP_taskgroup_end): Implement taskgroup
	support.
	* testsuite/libgomp.c++/cancel-parallel-1.C: New test.
	* testsuite/libgomp.c++/cancel-parallel-2.C: New test.
	* testsuite/libgomp.c++/cancel-parallel-3.C: New test.
	* testsuite/libgomp.c++/cancel-for-1.C: New test.
	* testsuite/libgomp.c++/cancel-for-1.C: New test.
	* testsuite/libgomp.c++/cancel-taskgroup-1.C: New test.
	* testsuite/libgomp.c++/cancel-taskgroup-2.C: New test.
	* testsuite/libgomp.c++/cancel-taskgroup-3.C: New test.
	* testsuite/libgomp.c++/cancel-test.h: New file.
	* testsuite/libgomp.c++/cancel-sections-1.C: New test.
	* testsuite/libgomp.c++/taskgroup-1.C: New test.
	* testsuite/libgomp.c/cancel-taskgroup-1.c: New test.
	* testsuite/libgomp.c/cancel-taskgroup-2.c: New test.
	* testsuite/libgomp.c/taskgroup-1.c: New test.
	* testsuite/libgomp.c/cancel-parallel-3.c (do_some_work): Use
	void return type.

--- libgomp/parallel.c.jj	2013-09-24 12:52:53.271887599 +0200
+++ libgomp/parallel.c	2013-09-24 13:10:29.345564211 +0200
@@ -147,7 +147,8 @@ GOMP_cancellation_point (int which)
   if (!gomp_cancel_var)
     return false;
 
-  struct gomp_team *team = gomp_thread ()->ts.team;
+  struct gomp_thread *thr = gomp_thread ();
+  struct gomp_team *team = thr->ts.team;
   if (which & (GOMP_CANCEL_LOOP | GOMP_CANCEL_SECTIONS))
     {
       if (team == NULL)
@@ -156,10 +157,11 @@ GOMP_cancellation_point (int which)
     }
   else if (which & GOMP_CANCEL_TASKGROUP)
     {
-      /* FIXME: Check if current taskgroup has been cancelled,
-	 then fallthru into the GOMP_CANCEL_PARALLEL case,
-	 because if the current parallel has been cancelled,
-	 all tasks should be cancelled too.  */
+      if (thr->task->taskgroup && thr->task->taskgroup->cancelled)
+	return true;
+      /* FALLTHRU into the GOMP_CANCEL_PARALLEL case,
+	 as #pragma omp cancel parallel also cancels all explicit
+	 tasks.  */
     }
   if (team)
     return gomp_team_barrier_cancelled (&team->barrier);
@@ -176,7 +178,8 @@ GOMP_cancel (int which, bool do_cancel)
   if (!do_cancel)
     return ialias_call (GOMP_cancellation_point) (which);
 
-  struct gomp_team *team = gomp_thread ()->ts.team;
+  struct gomp_thread *thr = gomp_thread ();
+  struct gomp_team *team = thr->ts.team;
   if (which & (GOMP_CANCEL_LOOP | GOMP_CANCEL_SECTIONS))
     {
       /* In orphaned worksharing region, all we want to cancel
@@ -187,7 +190,12 @@ GOMP_cancel (int which, bool do_cancel)
     }
   else if (which & GOMP_CANCEL_TASKGROUP)
     {
-      /* FIXME: Handle taskgroup cancellation.  */
+      if (thr->task->taskgroup && !thr->task->taskgroup->cancelled)
+	{
+	  gomp_mutex_lock (&team->task_lock);
+	  thr->task->taskgroup->cancelled = true;
+	  gomp_mutex_unlock (&team->task_lock);
+	}
       return true;
     }
   team->team_cancelled = 1;
--- libgomp/libgomp.h.jj	2013-09-24 12:52:53.274887599 +0200
+++ libgomp/libgomp.h	2013-09-24 13:10:29.344564253 +0200
@@ -253,6 +253,8 @@ enum gomp_task_kind
   GOMP_TASK_TIED
 };
 
+struct gomp_taskgroup;
+
 /* This structure describes a "task" to be run by a thread.  */
 
 struct gomp_task
@@ -263,6 +265,9 @@ struct gomp_task
   struct gomp_task *prev_child;
   struct gomp_task *next_queue;
   struct gomp_task *prev_queue;
+  struct gomp_task *next_taskgroup;
+  struct gomp_task *prev_taskgroup;
+  struct gomp_taskgroup *taskgroup;
   struct gomp_task_icv icv;
   void (*fn) (void *);
   void *fn_data;
@@ -270,9 +275,19 @@ struct gomp_task
   bool in_taskwait;
   bool in_tied_task;
   bool final_task;
+  bool copy_ctors_done;
   gomp_sem_t taskwait_sem;
 };
 
+struct gomp_taskgroup
+{
+  struct gomp_taskgroup *prev;
+  struct gomp_task *children;
+  bool in_taskgroup_wait;
+  bool cancelled;
+  gomp_sem_t taskgroup_sem;
+};
+
 /* 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.  */
--- libgomp/task.c.jj	2013-09-24 12:52:53.275887594 +0200
+++ libgomp/task.c	2013-09-24 14:30:54.816285356 +0200
@@ -42,7 +42,9 @@ gomp_init_task (struct gomp_task *task,
   task->in_taskwait = false;
   task->in_tied_task = false;
   task->final_task = false;
+  task->copy_ctors_done = false;
   task->children = NULL;
+  task->taskgroup = NULL;
   gomp_sem_init (&task->taskwait_sem, 0);
 }
 
@@ -94,8 +96,10 @@ GOMP_task (void (*fn) (void *), void *da
     flags &= ~1;
 #endif
 
-  /* If parallel has been cancelled, don't start new tasks.  */
-  if (team && gomp_team_barrier_cancelled (&team->barrier))
+  /* 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;
 
   if (!if_clause || team == NULL
@@ -108,7 +112,10 @@ GOMP_task (void (*fn) (void *), void *da
       task.kind = GOMP_TASK_IFFALSE;
       task.final_task = (thr->task && thr->task->final_task) || (flags & 2);
       if (thr->task)
-	task.in_tied_task = thr->task->in_tied_task;
+	{
+	  task.in_tied_task = thr->task->in_tied_task;
+	  task.taskgroup = thr->task->taskgroup;
+	}
       thr->task = &task;
       if (__builtin_expect (cpyfn != NULL, 0))
 	{
@@ -141,6 +148,7 @@ GOMP_task (void (*fn) (void *), void *da
     {
       struct gomp_task *task;
       struct gomp_task *parent = thr->task;
+      struct gomp_taskgroup *taskgroup = parent->taskgroup;
       char *arg;
       bool do_wake;
 
@@ -150,9 +158,13 @@ GOMP_task (void (*fn) (void *), void *da
       gomp_init_task (task, parent, gomp_icv (false));
       task->kind = GOMP_TASK_IFFALSE;
       task->in_tied_task = parent->in_tied_task;
+      task->taskgroup = taskgroup;
       thr->task = task;
       if (cpyfn)
-	cpyfn (arg, data);
+	{
+	  cpyfn (arg, data);
+	  task->copy_ctors_done = true;
+	}
       else
 	memcpy (arg, data, arg_size);
       thr->task = parent;
@@ -162,8 +174,11 @@ GOMP_task (void (*fn) (void *), void *da
       task->in_tied_task = true;
       task->final_task = (flags & 2) >> 1;
       gomp_mutex_lock (&team->task_lock);
-      /* If parallel has been cancelled, don't start new tasks.  */
-      if (gomp_team_barrier_cancelled (&team->barrier))
+      /* If parallel or taskgroup has been cancelled, don't start new
+	 tasks.  */
+      if (__builtin_expect ((gomp_team_barrier_cancelled (&team->barrier)
+			     || (taskgroup && taskgroup->cancelled))
+			    && !task->copy_ctors_done, 0))
 	{
 	  gomp_mutex_unlock (&team->task_lock);
 	  gomp_finish_task (task);
@@ -183,6 +198,22 @@ GOMP_task (void (*fn) (void *), void *da
 	  task->prev_child = task;
 	}
       parent->children = task;
+      if (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;
@@ -206,6 +237,84 @@ GOMP_task (void (*fn) (void *), void *da
     }
 }
 
+static inline bool
+gomp_task_run_pre (struct gomp_task *child_task, struct gomp_task *parent,
+		   struct gomp_taskgroup *taskgroup, struct gomp_team *team)
+{
+  if (parent && parent->children == child_task)
+    parent->children = child_task->next_child;
+  if (taskgroup && taskgroup->children == child_task)
+    taskgroup->children = child_task->next_taskgroup;
+  child_task->prev_queue->next_queue = child_task->next_queue;
+  child_task->next_queue->prev_queue = child_task->prev_queue;
+  if (team->task_queue == child_task)
+    {
+      if (child_task->next_queue != child_task)
+	team->task_queue = child_task->next_queue;
+      else
+	team->task_queue = NULL;
+    }
+  child_task->kind = GOMP_TASK_TIED;
+  if ((gomp_team_barrier_cancelled (&team->barrier)
+       || (taskgroup && taskgroup->cancelled))
+      && !child_task->copy_ctors_done)
+    return true;
+  team->task_running_count++;
+  if (team->task_count == team->task_running_count)
+    gomp_team_barrier_clear_task_pending (&team->barrier);
+  return false;
+}
+
+static inline void
+gomp_task_run_post_remove_parent (struct gomp_task *child_task)
+{
+  struct gomp_task *parent = child_task->parent;
+  if (parent == NULL)
+    return;
+  child_task->prev_child->next_child = child_task->next_child;
+  child_task->next_child->prev_child = child_task->prev_child;
+  if (parent->children != child_task)
+    return;
+  if (child_task->next_child != child_task)
+    parent->children = child_task->next_child;
+  else
+    {
+      /* We access task->children in GOMP_taskwait
+	 outside of the task lock mutex region, so
+	 need a release barrier here to ensure memory
+	 written by child_task->fn above is flushed
+	 before the NULL is written.  */
+      __atomic_store_n (&parent->children, NULL, MEMMODEL_RELEASE);
+      if (parent->in_taskwait)
+	gomp_sem_post (&parent->taskwait_sem);
+    }
+}
+
+static inline void
+gomp_task_run_post_remove_taskgroup (struct gomp_task *child_task)
+{
+  struct gomp_taskgroup *taskgroup = child_task->taskgroup;
+  if (taskgroup == NULL)
+    return;
+  child_task->prev_taskgroup->next_taskgroup = child_task->next_taskgroup;
+  child_task->next_taskgroup->prev_taskgroup = child_task->prev_taskgroup;
+  if (taskgroup->children != child_task)
+    return;
+  if (child_task->next_taskgroup != child_task)
+    taskgroup->children = child_task->next_taskgroup;
+  else
+    {
+      /* We access task->children in GOMP_taskgroup_end
+	 outside of the task lock mutex region, so
+	 need a release barrier here to ensure memory
+	 written by child_task->fn above is flushed
+	 before the NULL is written.  */
+      __atomic_store_n (&taskgroup->children, NULL, MEMMODEL_RELEASE);
+      if (taskgroup->in_taskgroup_wait)
+	gomp_sem_post (&taskgroup->taskgroup_sem);
+    }
+}
+
 void
 gomp_barrier_handle_tasks (gomp_barrier_state_t state)
 {
@@ -214,7 +323,6 @@ gomp_barrier_handle_tasks (gomp_barrier_
   struct gomp_task *task = thr->task;
   struct gomp_task *child_task = NULL;
   struct gomp_task *to_free = NULL;
-  bool cancelled = false;
 
   gomp_mutex_lock (&team->task_lock);
   if (gomp_barrier_last_thread (state))
@@ -231,22 +339,12 @@ gomp_barrier_handle_tasks (gomp_barrier_
 
   while (1)
     {
+      bool cancelled = false;
       if (team->task_queue != NULL)
 	{
-	  struct gomp_task *parent;
-
 	  child_task = team->task_queue;
-	  parent = child_task->parent;
-	  if (parent && parent->children == child_task)
-	    parent->children = child_task->next_child;
-	  child_task->prev_queue->next_queue = child_task->next_queue;
-	  child_task->next_queue->prev_queue = child_task->prev_queue;
-	  if (child_task->next_queue != child_task)
-	    team->task_queue = child_task->next_queue;
-	  else
-	    team->task_queue = NULL;
-	  child_task->kind = GOMP_TASK_TIED;
-	  cancelled |= gomp_team_barrier_cancelled (&team->barrier);
+	  cancelled = gomp_task_run_pre (child_task, child_task->parent,
+					 child_task->taskgroup, team);
 	  if (__builtin_expect (cancelled, 0))
 	    {
 	      if (to_free)
@@ -257,9 +355,6 @@ gomp_barrier_handle_tasks (gomp_barrier_
 		}
 	      goto finish_cancelled;
 	    }
-	  team->task_running_count++;
-	  if (team->task_count == team->task_running_count)
-	    gomp_team_barrier_clear_task_pending (&team->barrier);
 	}
       gomp_mutex_unlock (&team->task_lock);
       if (to_free)
@@ -277,33 +372,12 @@ gomp_barrier_handle_tasks (gomp_barrier_
       else
 	return;
       gomp_mutex_lock (&team->task_lock);
-     finish_cancelled:
       if (child_task)
 	{
-	  struct gomp_task *parent = child_task->parent;
-	  if (parent)
-	    {
-	      child_task->prev_child->next_child = child_task->next_child;
-	      child_task->next_child->prev_child = child_task->prev_child;
-	      if (parent->children == child_task)
-		{
-		  if (child_task->next_child != child_task)
-		    parent->children = child_task->next_child;
-		  else
-		    {
-		      /* We access task->children in GOMP_taskwait
-			 outside of the task lock mutex region, so
-			 need a release barrier here to ensure memory
-			 written by child_task->fn above is flushed
-			 before the NULL is written.  */
-		      __atomic_store_n (&parent->children, NULL,
-					MEMMODEL_RELEASE);
-		      if (parent->in_taskwait)
-			gomp_sem_post (&parent->taskwait_sem);
-		    }
-		}
-	    }
+	 finish_cancelled:
+	  gomp_task_run_post_remove_parent (child_task);
 	  gomp_clear_parent (child_task->children);
+	  gomp_task_run_post_remove_taskgroup (child_task);
 	  to_free = child_task;
 	  child_task = NULL;
 	  if (!cancelled)
@@ -344,6 +418,7 @@ GOMP_taskwait (void)
   gomp_mutex_lock (&team->task_lock);
   while (1)
     {
+      bool cancelled = false;
       if (task->children == NULL)
 	{
 	  gomp_mutex_unlock (&team->task_lock);
@@ -357,20 +432,19 @@ GOMP_taskwait (void)
       if (task->children->kind == GOMP_TASK_WAITING)
 	{
 	  child_task = task->children;
-	  task->children = child_task->next_child;
-	  child_task->prev_queue->next_queue = child_task->next_queue;
-	  child_task->next_queue->prev_queue = child_task->prev_queue;
-	  if (team->task_queue == child_task)
+	  cancelled
+	    = gomp_task_run_pre (child_task, task, child_task->taskgroup,
+				 team);
+	  if (__builtin_expect (cancelled, 0))
 	    {
-	      if (child_task->next_queue != child_task)
-		team->task_queue = child_task->next_queue;
-	      else
-		team->task_queue = NULL;
+	      if (to_free)
+		{
+		  gomp_finish_task (to_free);
+		  free (to_free);
+		  to_free = NULL;
+		}
+	      goto finish_cancelled;
 	    }
-	  child_task->kind = GOMP_TASK_TIED;
-	  team->task_running_count++;
-	  if (team->task_count == team->task_running_count)
-	    gomp_team_barrier_clear_task_pending (&team->barrier);
 	}
       else
 	/* All tasks we are waiting for are already running
@@ -398,6 +472,7 @@ GOMP_taskwait (void)
       gomp_mutex_lock (&team->task_lock);
       if (child_task)
 	{
+	 finish_cancelled:
 	  child_task->prev_child->next_child = child_task->next_child;
 	  child_task->next_child->prev_child = child_task->prev_child;
 	  if (task->children == child_task)
@@ -408,6 +483,7 @@ GOMP_taskwait (void)
 		task->children = NULL;
 	    }
 	  gomp_clear_parent (child_task->children);
+	  gomp_task_run_post_remove_taskgroup (child_task);
 	  to_free = child_task;
 	  child_task = NULL;
 	  team->task_count--;
@@ -427,11 +503,123 @@ GOMP_taskyield (void)
 void
 GOMP_taskgroup_start (void)
 {
+  struct gomp_thread *thr = gomp_thread ();
+  struct gomp_team *team = thr->ts.team;
+  struct gomp_task *task = thr->task;
+  struct gomp_taskgroup *taskgroup;
+
+  /* If team is NULL, all tasks are executed as
+     GOMP_TASK_IFFALSE tasks and thus all children tasks of
+     taskgroup and their descendant tasks will be finished
+     by the time GOMP_taskgroup_end is called.  */
+  if (team == NULL)
+    return;
+  taskgroup = gomp_malloc (sizeof (struct gomp_taskgroup));
+  taskgroup->prev = task->taskgroup;
+  taskgroup->children = NULL;
+  taskgroup->in_taskgroup_wait = false;
+  taskgroup->cancelled = false;
+  gomp_sem_init (&taskgroup->taskgroup_sem, 0);
+  task->taskgroup = taskgroup;
 }
 
 void
 GOMP_taskgroup_end (void)
 {
+  struct gomp_thread *thr = gomp_thread ();
+  struct gomp_team *team = thr->ts.team;
+  struct gomp_task *task = thr->task;
+  struct gomp_taskgroup *taskgroup;
+  struct gomp_task *child_task = NULL;
+  struct gomp_task *to_free = NULL;
+
+  if (team == NULL)
+    return;
+  taskgroup = task->taskgroup;
+  if (__atomic_load_n (&taskgroup->children, MEMMODEL_ACQUIRE) == NULL)
+    goto finish;
+  gomp_mutex_lock (&team->task_lock);
+  while (1)
+    {
+      bool cancelled = false;
+      if (taskgroup->children == NULL)
+	{
+	  gomp_mutex_unlock (&team->task_lock);
+	  if (to_free)
+	    {
+	      gomp_finish_task (to_free);
+	      free (to_free);
+	    }
+	  goto finish;
+	}
+      if (taskgroup->children->kind == GOMP_TASK_WAITING)
+	{
+	  child_task = taskgroup->children;
+	  cancelled
+	    = gomp_task_run_pre (child_task, child_task->parent, taskgroup,
+				 team);
+	  if (__builtin_expect (cancelled, 0))
+	    {
+	      if (to_free)
+		{
+		  gomp_finish_task (to_free);
+		  free (to_free);
+		  to_free = NULL;
+		}
+	      goto finish_cancelled;
+	    }
+	}
+      else
+	/* All tasks we are waiting for are already running
+	   in other threads.  Wait for them.  */
+	taskgroup->in_taskgroup_wait = true;
+      gomp_mutex_unlock (&team->task_lock);
+      if (to_free)
+	{
+	  gomp_finish_task (to_free);
+	  free (to_free);
+	  to_free = NULL;
+	}
+      if (child_task)
+	{
+	  thr->task = child_task;
+	  child_task->fn (child_task->fn_data);
+	  thr->task = task;
+	}
+      else
+	{
+	  gomp_sem_wait (&taskgroup->taskgroup_sem);
+	  taskgroup->in_taskgroup_wait = false;
+	  goto finish;
+	}
+      gomp_mutex_lock (&team->task_lock);
+      if (child_task)
+	{
+	 finish_cancelled:
+	  child_task->prev_taskgroup->next_taskgroup
+	    = child_task->next_taskgroup;
+	  child_task->next_taskgroup->prev_taskgroup
+	    = child_task->prev_taskgroup;
+	  if (taskgroup->children == child_task)
+	    {
+	      if (child_task->next_taskgroup != child_task)
+		taskgroup->children = child_task->next_taskgroup;
+	      else
+		taskgroup->children = NULL;
+	    }
+	  gomp_task_run_post_remove_parent (child_task);
+	  gomp_clear_parent (child_task->children);
+	  to_free = child_task;
+	  child_task = NULL;
+	  team->task_count--;
+	  team->task_running_count--;
+	}
+    }
+
+ finish:
+  task->taskgroup = taskgroup->prev;
+  gomp_sem_destroy (&taskgroup->taskgroup_sem);
+  free (taskgroup);
 }
 
 int
--- libgomp/testsuite/libgomp.c++/cancel-parallel-2.C.jj	2013-09-24 16:29:45.391438911 +0200
+++ libgomp/testsuite/libgomp.c++/cancel-parallel-2.C	2013-09-24 16:44:53.965871096 +0200
@@ -0,0 +1,57 @@
+// { dg-do run }
+// { dg-set-target-env-var OMP_CANCELLATION "true" }
+
+#include <unistd.h>
+#include <omp.h>
+#include "cancel-test.h"
+
+static void
+foo (int *x)
+{
+  S a, b, c;
+  #pragma omp parallel firstprivate(x, c) num_threads (32) private (b)
+  {
+    S d;
+    b.bump ();
+    c.bump ();
+    int thr = omp_get_thread_num ();
+    switch (x[thr])
+      {
+      case 4:
+	#pragma omp cancel parallel
+	break;
+      case 3:
+	#pragma omp task
+	usleep (1000);
+	#pragma omp task
+	usleep (2000);
+	#pragma omp task
+	usleep (4000);
+	break;
+      case 2:
+	usleep (1000);
+	/* FALLTHRU */
+      case 1:
+	#pragma omp cancellation point parallel
+	break;
+      }
+    #pragma omp barrier
+    if (omp_get_cancellation ())
+      abort ();
+  }
+}
+
+int
+main ()
+{
+  int i, j, x[32] = { 0, 1, 2, 4, 2, 2, 1, 0 };
+  foo (x);
+  for (i = 0; i < 32; i++)
+    {
+      for (j = 0; j < 32; j++)
+	x[j] = rand () & 3;
+      x[rand () & 31] = 4;
+      foo (x);
+    }
+  S::verify ();
+}
--- libgomp/testsuite/libgomp.c++/cancel-parallel-3.C.jj	2013-09-24 16:29:45.392438935 +0200
+++ libgomp/testsuite/libgomp.c++/cancel-parallel-3.C	2013-09-24 17:15:29.794631875 +0200
@@ -0,0 +1,50 @@
+// { dg-do run }
+// { dg-set-target-env-var OMP_CANCELLATION "true" }
+
+#include <omp.h>
+#include <unistd.h>
+#include "cancel-test.h"
+
+static inline void
+do_some_work (void)
+{
+  asm volatile ("" : : : "memory");
+}
+
+void
+foo ()
+{
+  S a, b, c;
+  omp_set_dynamic (0);
+  omp_set_schedule (omp_sched_static, 1);
+  #pragma omp parallel num_threads (16) private (b) firstprivate (c)
+  {
+    S d;
+    int i, j;
+    b.bump ();
+    c.bump ();
+    do_some_work ();
+    #pragma omp barrier
+    if (omp_get_thread_num () == 1)
+      {
+	sleep (2);
+	#pragma omp cancellation point parallel
+      }
+    for (j = 3; j <= 16; j++)
+      #pragma omp for schedule (runtime) nowait
+      for (i = 0; i < j; i++)
+	do_some_work ();
+    if (omp_get_thread_num () == 0)
+      {
+	sleep (1);
+	#pragma omp cancel parallel
+      }
+  }
+}
+
+int
+main ()
+{
+  foo ();
+  S::verify ();
+}
--- libgomp/testsuite/libgomp.c++/cancel-parallel-1.C.jj	2013-09-24 16:29:45.390438885 +0200
+++ libgomp/testsuite/libgomp.c++/cancel-parallel-1.C	2013-09-24 16:44:50.905888765 +0200
@@ -0,0 +1,18 @@
+// { dg-do run }
+// { dg-set-target-env-var OMP_CANCELLATION "true" }
+
+#include <omp.h>
+#include "cancel-test.h"
+
+int
+main ()
+{
+  #pragma omp parallel num_threads (32)
+  {
+    S a;
+    #pragma omp cancel parallel
+    if (omp_get_cancellation ())
+      abort ();
+  }
+  S::verify ();
+}
--- libgomp/testsuite/libgomp.c++/cancel-for-2.C.jj	2013-09-24 16:29:45.389438859 +0200
+++ libgomp/testsuite/libgomp.c++/cancel-for-2.C	2013-09-24 17:13:56.799105718 +0200
@@ -0,0 +1,126 @@
+// { dg-do run }
+// { dg-set-target-env-var OMP_CANCELLATION "true" }
+
+#include <omp.h>
+#include "cancel-test.h"
+
+__attribute__((noinline, noclone)) int
+foo (int *x)
+{
+  S a, b, c, d, e;
+  int v = 0, w = 0;
+  #pragma omp parallel num_threads (32) shared (v, w) private (c, d) firstprivate (e)
+  {
+    S g;
+    int i;
+    c.bump ();
+    e.bump ();
+    #pragma omp for private (d, g) firstprivate (b)
+    for (i = 0; i < 1000; ++i)
+      {
+	b.bump ();
+	d.bump ();
+	g.bump ();
+	#pragma omp cancel for if (x[0])
+	abort ();
+      }
+    #pragma omp for private (d, g) firstprivate (b)
+    for (i = 0; i < 1000; ++i)
+      {
+	b.bump ();
+	d.bump ();
+	g.bump ();
+	#pragma omp cancel for if (x[1])
+	#pragma omp atomic
+	v++;
+      }
+    #pragma omp for private (d, g) firstprivate (b)
+    for (i = 0; i < 1000; ++i)
+      {
+	b.bump ();
+	d.bump ();
+	g.bump ();
+	#pragma omp cancel for if (x[2])
+	#pragma omp atomic
+	w += 8;
+      }
+    #pragma omp for private (d, g) firstprivate (b)
+    for (i = 0; i < 1000; ++i)
+      {
+	b.bump ();
+	d.bump ();
+	g.bump ();
+	#pragma omp cancel for if (x[3])
+	#pragma omp atomic
+	v += 2;
+      }
+  }
+  if (v != 3000 || w != 0)
+    abort ();
+  #pragma omp parallel num_threads (32) shared (v, w) private (c, d) firstprivate (e)
+  {
+    S g, h;
+    int i;
+    c.bump ();
+    e.bump ();
+    /* None of these cancel directives should actually cancel anything,
+       but the compiler shouldn't know that and thus should use cancellable
+       barriers at the end of all the workshares.  */
+    #pragma omp cancel parallel if (omp_get_thread_num () == 1 && x[4])
+    #pragma omp for private (d, g) firstprivate (b)
+    for (i = 0; i < 1000; ++i)
+      {
+	b.bump ();
+	d.bump ();
+	g.bump ();
+	#pragma omp cancel for if (x[0])
+	abort ();
+      }
+    #pragma omp cancel parallel if (omp_get_thread_num () == 2 && x[4])
+    #pragma omp for private (d, g) firstprivate (b)
+    for (i = 0; i < 1000; ++i)
+      {
+	b.bump ();
+	d.bump ();
+	g.bump ();
+	#pragma omp cancel for if (x[1])
+	#pragma omp atomic
+	v++;
+      }
+    #pragma omp cancel parallel if (omp_get_thread_num () == 3 && x[4])
+    #pragma omp for private (d, g) firstprivate (b)
+    for (i = 0; i < 1000; ++i)
+      {
+	b.bump ();
+	d.bump ();
+	g.bump ();
+	#pragma omp cancel for if (x[2])
+	#pragma omp atomic
+	w += 8;
+      }
+    #pragma omp cancel parallel if (omp_get_thread_num () == 4 && x[4])
+    #pragma omp for private (d, g) firstprivate (b)
+    for (i = 0; i < 1000; ++i)
+      {
+	b.bump ();
+	d.bump ();
+	g.bump ();
+	#pragma omp cancel for if (x[3])
+	#pragma omp atomic
+	v += 2;
+      }
+    #pragma omp cancel parallel if (omp_get_thread_num () == 5 && x[4])
+  }
+  if (v != 6000 || w != 0)
+    abort ();
+  return 0;
+}
+
+int
+main ()
+{
+  int x[] = { 1, 0, 1, 0, 0 };
+  if (omp_get_cancellation ())
+    foo (x);
+  S::verify ();
+}
--- libgomp/testsuite/libgomp.c++/cancel-taskgroup-1.C.jj	2013-09-24 16:29:45.393438957 +0200
+++ libgomp/testsuite/libgomp.c++/cancel-taskgroup-1.C	2013-09-24 16:46:27.560400571 +0200
@@ -0,0 +1,4 @@
+// { dg-do run }
+// { dg-set-target-env-var OMP_CANCELLATION "true" }
+
+#include "../libgomp.c/cancel-taskgroup-1.c"
--- libgomp/testsuite/libgomp.c++/cancel-for-1.C.jj	2013-09-24 16:29:45.388438835 +0200
+++ libgomp/testsuite/libgomp.c++/cancel-for-1.C	2013-09-24 17:11:45.488759105 +0200
@@ -0,0 +1,29 @@
+// { dg-do run }
+// { dg-set-target-env-var OMP_CANCELLATION "true" }
+
+#include <omp.h>
+#include "cancel-test.h"
+
+int
+main ()
+{
+  {
+    S c;
+    #pragma omp parallel num_threads (32)
+    {
+      S a, b;
+      int i;
+      #pragma omp for private (b) firstprivate (c)
+      for (i = 0; i < 1000; ++i)
+	{
+	  S d;
+	  #pragma omp cancel for
+	  if (omp_get_cancellation ())
+	    abort ();
+	  b.bump ();
+	  c.bump ();
+	}
+    }
+  }
+  S::verify ();
+}
--- libgomp/testsuite/libgomp.c++/taskgroup-1.C.jj	2013-09-24 16:26:39.090383292 +0200
+++ libgomp/testsuite/libgomp.c++/taskgroup-1.C	2013-09-24 16:26:46.407338971 +0200
@@ -0,0 +1 @@
+#include "../libgomp.c/taskgroup-1.c"
--- libgomp/testsuite/libgomp.c++/cancel-test.h.jj	2013-09-24 16:32:48.082518857 +0200
+++ libgomp/testsuite/libgomp.c++/cancel-test.h	2013-09-24 17:14:35.032912877 +0200
@@ -0,0 +1,47 @@
+#include <stdlib.h>
+#include <omp.h>
+
+struct S
+{
+  static int s;
+  int v;
+  S ()
+  {
+    #pragma omp atomic
+    s++;
+  }
+
+  S (int x)
+  {
+    #pragma omp atomic
+    s++;
+    v = x;
+  }
+
+  ~S ()
+  {
+    #pragma omp atomic
+    s--;
+  }
+
+  S (const S &x)
+  {
+    #pragma omp atomic
+    s++;
+    v = x.v;
+  }
+
+  static void
+  verify ()
+  {
+    if (s) abort ();
+  }
+
+  void
+  bump ()
+  {
+    v++;
+  }
+};
+
+int S::s = 0;
--- libgomp/testsuite/libgomp.c++/cancel-sections-1.C.jj	2013-09-24 16:29:45.393438957 +0200
+++ libgomp/testsuite/libgomp.c++/cancel-sections-1.C	2013-09-24 16:45:44.489616705 +0200
@@ -0,0 +1,43 @@
+// { dg-do run }
+// { dg-set-target-env-var OMP_CANCELLATION "true" }
+
+#include <omp.h>
+#include "cancel-test.h"
+
+int
+main ()
+{
+  if (!omp_get_cancellation ())
+    return 0;
+  #pragma omp parallel num_threads (32)
+  {
+    S a;
+    #pragma omp sections
+      {
+	{
+	  S b;
+	  #pragma omp cancel sections
+	  abort ();
+	}
+      #pragma omp section
+	{
+	  S c;
+	  #pragma omp cancel sections
+	  abort ();
+	}
+      #pragma omp section
+	{
+	  S d;
+	  #pragma omp cancel sections
+	  abort ();
+	}
+      #pragma omp section
+	{
+	  S e;
+	  #pragma omp cancel sections
+	  abort ();
+	}
+      }
+  }
+  S::verify ();
+}
--- libgomp/testsuite/libgomp.c++/cancel-taskgroup-2.C.jj	2013-09-24 16:29:45.394438978 +0200
+++ libgomp/testsuite/libgomp.c++/cancel-taskgroup-2.C	2013-09-24 16:53:28.077286199 +0200
@@ -0,0 +1,4 @@
+// { dg-do run }
+// { dg-set-target-env-var OMP_CANCELLATION "true" }
+
+#include "../libgomp.c/cancel-taskgroup-2.c"
--- libgomp/testsuite/libgomp.c++/cancel-taskgroup-3.C.jj	2013-09-24 16:53:00.297425570 +0200
+++ libgomp/testsuite/libgomp.c++/cancel-taskgroup-3.C	2013-09-24 20:28:00.157209836 +0200
@@ -0,0 +1,58 @@
+// { dg-do run }
+// { dg-set-target-env-var OMP_CANCELLATION "true" }
+
+#include <unistd.h>
+#include <omp.h>
+#include "cancel-test.h"
+
+void
+foo ()
+{
+  S a, b, c, d, e, f;
+  #pragma omp parallel private (c, d) firstprivate (e, f)
+  #pragma omp taskgroup
+  {
+    c.bump ();
+    e.bump ();
+    #pragma omp task firstprivate (b, f) private (d)
+    {
+      S h;
+      b.bump ();
+      d.bump ();
+      f.bump ();
+      #pragma omp cancel taskgroup
+      if (omp_get_cancellation ())
+	abort ();
+    }
+  }
+  #pragma omp parallel private (c, d) firstprivate (e, f)
+  {
+    #pragma omp barrier
+    #pragma omp single
+    #pragma omp taskgroup
+    {
+      int i;
+      c.bump ();
+      e.bump ();
+      for (i = 0; i < 50; i++)
+	#pragma omp task firstprivate (b, f) private (d)
+	{
+	  S h;
+	  b.bump ();
+	  d.bump ();
+	  f.bump ();
+	  #pragma omp cancellation point taskgroup
+	  usleep (30);
+	  #pragma omp cancel taskgroup if (i > 5)
+	}
+    }
+    usleep (10);
+  }
+}
+
+int
+main ()
+{
+  foo ();
+  S::verify ();
+}
--- libgomp/testsuite/libgomp.c/cancel-taskgroup-1.c.jj	2013-09-24 15:59:41.482515570 +0200
+++ libgomp/testsuite/libgomp.c/cancel-taskgroup-1.c	2013-09-24 20:12:34.269799985 +0200
@@ -0,0 +1,70 @@
+/* { dg-do run } */
+/* { dg-set-target-env-var OMP_CANCELLATION "true" } */
+
+#include <stdlib.h>
+#include <omp.h>
+
+struct T { struct T *children[2]; int val; };
+
+struct T *
+search (struct T *tree, int val, int lvl)
+{
+  if (tree == NULL || tree->val == val)
+    return tree;
+  struct T *ret = NULL;
+  int i;
+  for (i = 0; i < 2; i++)
+    #pragma omp task shared(ret) if(lvl < 10)
+    {
+      struct T *r = search (tree->children[i], val, lvl + 1);
+      if (r)
+	{
+	  #pragma omp atomic write
+	  ret = r;
+	  #pragma omp cancel taskgroup
+	}
+    }
+  #pragma omp taskwait
+  return ret;
+}
+
+struct T *
+searchp (struct T *tree, int val)
+{
+  struct T *ret;
+  #pragma omp parallel shared(ret) firstprivate (tree, val)
+  #pragma omp single
+  #pragma omp taskgroup
+  ret = search (tree, val, 0);
+  return ret;
+}
+
+int
+main ()
+{
+  /* Must be power of two minus 1.  */
+  int size = 0x7ffff;
+  struct T *trees = (struct T *) malloc (size * sizeof (struct T));
+  if (trees == NULL)
+    return 0;
+  int i, l = 1, b = 0;
+  for (i = 0; i < size; i++)
+    {
+      if (i == l)
+	{
+	  b = l;
+	  l = l * 2 + 1;
+	}
+      trees[i].val = i;
+      trees[i].children[0] = l == size ? NULL : &trees[l + (i - b) * 2];
+      trees[i].children[1] = l == size ? NULL : &trees[l + (i - b) * 2 + 1];
+    }
+  for (i = 0; i < 50; i++)
+    {
+      int v = random () & size;
+      if (searchp (&trees[0], v) != &trees[v])
+	abort ();
+    }
+  free (trees);
+  return 0;
+}
--- libgomp/testsuite/libgomp.c/cancel-parallel-3.c.jj	2013-09-24 12:52:53.000000000 +0200
+++ libgomp/testsuite/libgomp.c/cancel-parallel-3.c	2013-09-24 17:15:43.472569223 +0200
@@ -4,7 +4,7 @@
 #include <omp.h>
 #include <unistd.h>
 
-static inline
+static inline void
 do_some_work (void)
 {
   asm volatile ("" : : : "memory");
--- libgomp/testsuite/libgomp.c/cancel-taskgroup-2.c.jj	2013-09-24 16:15:27.489754589 +0200
+++ libgomp/testsuite/libgomp.c/cancel-taskgroup-2.c	2013-09-24 16:25:54.664599316 +0200
@@ -0,0 +1,37 @@
+/* { dg-do run } */
+/* { dg-set-target-env-var OMP_CANCELLATION "true" } */
+
+#include <stdlib.h>
+#include <unistd.h>
+#include <omp.h>
+
+int
+main ()
+{
+  #pragma omp parallel
+  #pragma omp taskgroup
+  #pragma omp task
+  {
+    #pragma omp cancel taskgroup
+    if (omp_get_cancellation ())
+      abort ();
+  }
+  #pragma omp parallel
+  {
+    #pragma omp barrier
+    #pragma omp single
+    #pragma omp taskgroup
+    {
+      int i;
+      for (i = 0; i < 50; i++)
+	#pragma omp task
+	{
+	  #pragma omp cancellation point taskgroup
+	  usleep (30);
+	  #pragma omp cancel taskgroup if (i > 5)
+	}
+    }
+    usleep (10);
+  }
+  return 0;
+}
--- libgomp/testsuite/libgomp.c/taskgroup-1.c.jj	2013-09-24 13:10:29.346564173 +0200
+++ libgomp/testsuite/libgomp.c/taskgroup-1.c	2013-09-24 13:14:03.882481554 +0200
@@ -0,0 +1,83 @@
+extern
+#ifdef __cplusplus
+"C"
+#endif
+void abort (void);
+int v[16] = { 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16 };
+
+int
+main ()
+{
+  #pragma omp parallel num_threads (4)
+  #pragma omp single
+    {
+      int i;
+      #pragma omp taskgroup
+      {
+	for (i = 0; i < 16; i += 2)
+	  #pragma omp task
+	    {
+	      #pragma omp task
+		v[i]++;
+	      #pragma omp task
+		v[i + 1]++;
+	    }
+      }
+      for (i = 0; i < 16; i++)
+	if (v[i] != i + 2)
+	  abort ();
+      #pragma omp taskgroup
+      {
+	for (i = 0; i < 16; i += 2)
+	  #pragma omp task
+	    {
+	      #pragma omp task
+		v[i]++;
+	      #pragma omp task
+		v[i + 1]++;
+	      #pragma omp taskwait
+	    }
+      }
+      for (i = 0; i < 16; i++)
+	if (v[i] != i + 3)
+	  abort ();
+      #pragma omp taskgroup
+      {
+	for (i = 0; i < 16; i += 2)
+	  #pragma omp task
+	    {
+	      #pragma omp task
+		v[i]++;
+	      v[i + 1]++;
+	    }
+	#pragma omp taskwait
+	for (i = 0; i < 16; i += 2)
+	  #pragma omp task
+	    v[i + 1]++;
+      }
+      for (i = 0; i < 16; i++)
+	if (v[i] != i + 4 + (i & 1))
+	  abort ();
+      #pragma omp taskgroup
+      {
+	for (i = 0; i < 16; i += 2)
+	  {
+	    #pragma omp taskgroup
+	      {
+		#pragma omp task
+		  v[i]++;
+		#pragma omp task
+		  v[i + 1]++;
+	      }
+	    if (v[i] != i + 5 || v[i + 1] != i + 7)
+	      abort ();
+	    #pragma omp task
+	    v[i]++;
+	  }
+      }
+      for (i = 0; i < 16; i++)
+	if (v[i] != i + 6)
+	  abort ();
+    }
+  return 0;
+}

	Jakub



More information about the Gcc-patches mailing list