[Openmp-commits] [openmp] r346307 - Implementation of OpenMP 5.0 mutexinoutset task dependency type.

Andrey Churbanov via Openmp-commits openmp-commits at lists.llvm.org
Wed Nov 7 04:19:57 PST 2018


Author: achurbanov
Date: Wed Nov  7 04:19:57 2018
New Revision: 346307

URL: http://llvm.org/viewvc/llvm-project?rev=346307&view=rev
Log:
Implementation of OpenMP 5.0 mutexinoutset task dependency type.

Differential Revision: https://reviews.llvm.org/D53380

Added:
    openmp/trunk/runtime/test/tasking/omp50_task_depend_mtx.c   (with props)
    openmp/trunk/runtime/test/tasking/omp50_task_depend_mtx2.c   (with props)
Modified:
    openmp/trunk/runtime/src/kmp.h
    openmp/trunk/runtime/src/kmp_taskdeps.cpp
    openmp/trunk/runtime/src/kmp_taskdeps.h
    openmp/trunk/runtime/src/kmp_tasking.cpp

Modified: openmp/trunk/runtime/src/kmp.h
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp.h?rev=346307&r1=346306&r2=346307&view=diff
==============================================================================
--- openmp/trunk/runtime/src/kmp.h (original)
+++ openmp/trunk/runtime/src/kmp.h Wed Nov  7 04:19:57 2018
@@ -2160,30 +2160,35 @@ typedef union kmp_depnode kmp_depnode_t;
 typedef struct kmp_depnode_list kmp_depnode_list_t;
 typedef struct kmp_dephash_entry kmp_dephash_entry_t;
 
+// Compiler sends us this info:
 typedef struct kmp_depend_info {
   kmp_intptr_t base_addr;
   size_t len;
   struct {
     bool in : 1;
     bool out : 1;
+    bool mtx : 1;
   } flags;
 } kmp_depend_info_t;
 
+// Internal structures to work with task dependencies:
 struct kmp_depnode_list {
   kmp_depnode_t *node;
   kmp_depnode_list_t *next;
 };
 
-typedef struct kmp_base_depnode {
-  kmp_depnode_list_t *successors;
-  kmp_task_t *task;
-
-  kmp_lock_t lock;
+// Max number of mutexinoutset dependencies per node
+#define MAX_MTX_DEPS 4
 
+typedef struct kmp_base_depnode {
+  kmp_depnode_list_t *successors; /* used under lock */
+  kmp_task_t *task; /* non-NULL if depnode is active, used under lock */
+  kmp_lock_t *mtx_locks[MAX_MTX_DEPS]; /* lock mutexinoutset dependent tasks */
+  kmp_int32 mtx_num_locks; /* number of locks in mtx_locks array */
+  kmp_lock_t lock; /* guards shared fields: task, successors */
 #if KMP_SUPPORT_GRAPH_OUTPUT
   kmp_uint32 id;
 #endif
-
   std::atomic<kmp_int32> npredecessors;
   std::atomic<kmp_int32> nrefs;
 } kmp_base_depnode_t;
@@ -2198,6 +2203,9 @@ struct kmp_dephash_entry {
   kmp_intptr_t addr;
   kmp_depnode_t *last_out;
   kmp_depnode_list_t *last_ins;
+  kmp_depnode_list_t *last_mtxs;
+  kmp_int32 last_flag;
+  kmp_lock_t *mtx_lock; /* is referenced by depnodes w/mutexinoutset dep */
   kmp_dephash_entry_t *next_in_bucket;
 };
 

Modified: openmp/trunk/runtime/src/kmp_taskdeps.cpp
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp_taskdeps.cpp?rev=346307&r1=346306&r2=346307&view=diff
==============================================================================
--- openmp/trunk/runtime/src/kmp_taskdeps.cpp (original)
+++ openmp/trunk/runtime/src/kmp_taskdeps.cpp Wed Nov  7 04:19:57 2018
@@ -37,12 +37,14 @@ static std::atomic<kmp_int32> kmp_node_i
 #endif
 
 static void __kmp_init_node(kmp_depnode_t *node) {
-  node->dn.task = NULL; // set to null initially, it will point to the right
-  // task once dependences have been processed
   node->dn.successors = NULL;
+  node->dn.task = NULL; // will point to the rigth task
+  // once dependences have been processed
+  for (int i = 0; i < MAX_MTX_DEPS; ++i)
+    node->dn.mtx_locks[i] = NULL;
+  node->dn.mtx_num_locks = 0;
   __kmp_init_lock(&node->dn.lock);
-  KMP_ATOMIC_ST_RLX(&node->dn.nrefs,
-                    1); // init creates the first reference to the node
+  KMP_ATOMIC_ST_RLX(&node->dn.nrefs, 1); // init creates the first reference
 #ifdef KMP_SUPPORT_GRAPH_OUTPUT
   node->dn.id = KMP_ATOMIC_INC(&kmp_node_id_seed);
 #endif
@@ -94,6 +96,9 @@ static kmp_dephash_t *__kmp_dephash_crea
   return h;
 }
 
+#define ENTRY_LAST_INS 0
+#define ENTRY_LAST_MTXS 1
+
 static kmp_dephash_entry *
 __kmp_dephash_find(kmp_info_t *thread, kmp_dephash_t *h, kmp_intptr_t addr) {
   kmp_int32 bucket = __kmp_dephash_hash(addr, h->size);
@@ -115,6 +120,9 @@ __kmp_dephash_find(kmp_info_t *thread, k
     entry->addr = addr;
     entry->last_out = NULL;
     entry->last_ins = NULL;
+    entry->last_mtxs = NULL;
+    entry->last_flag = ENTRY_LAST_INS;
+    entry->mtx_lock = NULL;
     entry->next_in_bucket = h->buckets[bucket];
     h->buckets[bucket] = entry;
 #ifdef KMP_DEBUG
@@ -173,6 +181,58 @@ static inline void __kmp_track_dependenc
 #endif /* OMPT_SUPPORT && OMPT_OPTIONAL */
 }
 
+static inline kmp_int32
+__kmp_depnode_link_successor(kmp_int32 gtid, kmp_info_t *thread,
+                             kmp_task_t *task, kmp_depnode_t *node,
+                             kmp_depnode_list_t *plist) {
+  if (!plist)
+    return 0;
+  kmp_int32 npredecessors = 0;
+  // link node as successor of list elements
+  for (kmp_depnode_list_t *p = plist; p; p = p->next) {
+    kmp_depnode_t *dep = p->node;
+    if (dep->dn.task) {
+      KMP_ACQUIRE_DEPNODE(gtid, dep);
+      if (dep->dn.task) {
+        __kmp_track_dependence(dep, node, task);
+        dep->dn.successors = __kmp_add_node(thread, dep->dn.successors, node);
+        KA_TRACE(40, ("__kmp_process_deps: T#%d adding dependence from %p to "
+                      "%p\n",
+                      gtid, KMP_TASK_TO_TASKDATA(dep->dn.task),
+                      KMP_TASK_TO_TASKDATA(task)));
+        npredecessors++;
+      }
+      KMP_RELEASE_DEPNODE(gtid, dep);
+    }
+  }
+  return npredecessors;
+}
+
+static inline kmp_int32 __kmp_depnode_link_successor(kmp_int32 gtid,
+                                                     kmp_info_t *thread,
+                                                     kmp_task_t *task,
+                                                     kmp_depnode_t *source,
+                                                     kmp_depnode_t *sink) {
+  if (!sink)
+    return 0;
+  kmp_int32 npredecessors = 0;
+  if (sink->dn.task) {
+    // synchronously add source to sink' list of successors
+    KMP_ACQUIRE_DEPNODE(gtid, sink);
+    if (sink->dn.task) {
+      __kmp_track_dependence(sink, source, task);
+      sink->dn.successors = __kmp_add_node(thread, sink->dn.successors, source);
+      KA_TRACE(40, ("__kmp_process_deps: T#%d adding dependence from %p to "
+                    "%p\n",
+                    gtid, KMP_TASK_TO_TASKDATA(sink->dn.task),
+                    KMP_TASK_TO_TASKDATA(task)));
+      npredecessors++;
+    }
+    KMP_RELEASE_DEPNODE(gtid, sink);
+  }
+  return npredecessors;
+}
+
 template <bool filter>
 static inline kmp_int32
 __kmp_process_deps(kmp_int32 gtid, kmp_depnode_t *node, kmp_dephash_t *hash,
@@ -187,72 +247,106 @@ __kmp_process_deps(kmp_int32 gtid, kmp_d
   for (kmp_int32 i = 0; i < ndeps; i++) {
     const kmp_depend_info_t *dep = &dep_list[i];
 
-    KMP_DEBUG_ASSERT(dep->flags.in);
-
     if (filter && dep->base_addr == 0)
       continue; // skip filtered entries
 
     kmp_dephash_entry_t *info =
         __kmp_dephash_find(thread, hash, dep->base_addr);
     kmp_depnode_t *last_out = info->last_out;
+    kmp_depnode_list_t *last_ins = info->last_ins;
+    kmp_depnode_list_t *last_mtxs = info->last_mtxs;
 
-    if (dep->flags.out && info->last_ins) {
-      for (kmp_depnode_list_t *p = info->last_ins; p; p = p->next) {
-        kmp_depnode_t *indep = p->node;
-        if (indep->dn.task) {
-          KMP_ACQUIRE_DEPNODE(gtid, indep);
-          if (indep->dn.task) {
-            __kmp_track_dependence(indep, node, task);
-            indep->dn.successors =
-                __kmp_add_node(thread, indep->dn.successors, node);
-            KA_TRACE(40, ("__kmp_process_deps<%d>: T#%d adding dependence from "
-                          "%p to %p\n",
-                          filter, gtid, KMP_TASK_TO_TASKDATA(indep->dn.task),
-                          KMP_TASK_TO_TASKDATA(task)));
-            npredecessors++;
-          }
-          KMP_RELEASE_DEPNODE(gtid, indep);
+    if (dep->flags.out) { // out --> clean lists of ins and mtxs if any
+      if (last_ins || last_mtxs) {
+        if (info->last_flag == ENTRY_LAST_INS) { // INS were last
+          npredecessors +=
+              __kmp_depnode_link_successor(gtid, thread, task, node, last_ins);
+        } else { // MTXS were last
+          npredecessors +=
+              __kmp_depnode_link_successor(gtid, thread, task, node, last_mtxs);
         }
+        __kmp_depnode_list_free(thread, last_ins);
+        __kmp_depnode_list_free(thread, last_mtxs);
+        info->last_ins = NULL;
+        info->last_mtxs = NULL;
+      } else {
+        npredecessors +=
+            __kmp_depnode_link_successor(gtid, thread, task, node, last_out);
       }
-
-      __kmp_depnode_list_free(thread, info->last_ins);
-      info->last_ins = NULL;
-
-    } else if (last_out && last_out->dn.task) {
-      KMP_ACQUIRE_DEPNODE(gtid, last_out);
-      if (last_out->dn.task) {
-        __kmp_track_dependence(last_out, node, task);
-        last_out->dn.successors =
-            __kmp_add_node(thread, last_out->dn.successors, node);
-        KA_TRACE(
-            40,
-            ("__kmp_process_deps<%d>: T#%d adding dependence from %p to %p\n",
-             filter, gtid, KMP_TASK_TO_TASKDATA(last_out->dn.task),
-             KMP_TASK_TO_TASKDATA(task)));
-
-        npredecessors++;
-      }
-      KMP_RELEASE_DEPNODE(gtid, last_out);
-    }
-
-    if (dep_barrier) {
-      // if this is a sync point in the serial sequence, then the previous
-      // outputs are guaranteed to be completed after
-      // the execution of this task so the previous output nodes can be cleared.
       __kmp_node_deref(thread, last_out);
-      info->last_out = NULL;
+      if (dep_barrier) {
+        // if this is a sync point in the serial sequence, then the previous
+        // outputs are guaranteed to be completed after the execution of this
+        // task so the previous output nodes can be cleared.
+        info->last_out = NULL;
+      } else {
+        info->last_out = __kmp_node_ref(node);
+      }
+    } else if (dep->flags.in) {
+      // in --> link node to either last_out or last_mtxs, clean earlier deps
+      if (last_mtxs) {
+        npredecessors +=
+            __kmp_depnode_link_successor(gtid, thread, task, node, last_mtxs);
+        __kmp_node_deref(thread, last_out);
+        info->last_out = NULL;
+        if (info->last_flag == ENTRY_LAST_MTXS && last_ins) { // MTXS were last
+          // clean old INS before creating new list
+          __kmp_depnode_list_free(thread, last_ins);
+          info->last_ins = NULL;
+        }
+      } else {
+        // link node as successor of the last_out if any
+        npredecessors +=
+            __kmp_depnode_link_successor(gtid, thread, task, node, last_out);
+      }
+      info->last_flag = ENTRY_LAST_INS;
+      info->last_ins = __kmp_add_node(thread, info->last_ins, node);
     } else {
-      if (dep->flags.out) {
+      KMP_DEBUG_ASSERT(dep->flags.mtx == 1);
+      // mtx --> link node to either last_out or last_ins, clean earlier deps
+      if (last_ins) {
+        npredecessors +=
+            __kmp_depnode_link_successor(gtid, thread, task, node, last_ins);
         __kmp_node_deref(thread, last_out);
-        info->last_out = __kmp_node_ref(node);
-      } else
-        info->last_ins = __kmp_add_node(thread, info->last_ins, node);
+        info->last_out = NULL;
+        if (info->last_flag == ENTRY_LAST_INS && last_mtxs) { // INS were last
+          // clean old MTXS before creating new list
+          __kmp_depnode_list_free(thread, last_mtxs);
+          info->last_mtxs = NULL;
+        }
+      } else {
+        // link node as successor of the last_out if any
+        npredecessors +=
+            __kmp_depnode_link_successor(gtid, thread, task, node, last_out);
+      }
+      info->last_flag = ENTRY_LAST_MTXS;
+      info->last_mtxs = __kmp_add_node(thread, info->last_mtxs, node);
+      if (info->mtx_lock == NULL) {
+        info->mtx_lock = (kmp_lock_t *)__kmp_allocate(sizeof(kmp_lock_t));
+        __kmp_init_lock(info->mtx_lock);
+      }
+      KMP_DEBUG_ASSERT(node->dn.mtx_num_locks < MAX_MTX_DEPS);
+      kmp_int32 m;
+      // Save lock in node's array
+      for (m = 0; m < MAX_MTX_DEPS; ++m) {
+        // sort pointers in decreasing order to avoid potential livelock
+        if (node->dn.mtx_locks[m] < info->mtx_lock) {
+          KMP_DEBUG_ASSERT(node->dn.mtx_locks[node->dn.mtx_num_locks] == NULL);
+          for (int n = node->dn.mtx_num_locks; n > m; --n) {
+            // shift right all lesser non-NULL pointers
+            KMP_DEBUG_ASSERT(node->dn.mtx_locks[n - 1] != NULL);
+            node->dn.mtx_locks[n] = node->dn.mtx_locks[n - 1];
+          }
+          node->dn.mtx_locks[m] = info->mtx_lock;
+          break;
+        }
+      }
+      KMP_DEBUG_ASSERT(m < MAX_MTX_DEPS); // must break from loop
+      node->dn.mtx_num_locks++;
     }
   }
-
   KA_TRACE(30, ("__kmp_process_deps<%d>: T#%d found %d predecessors\n", filter,
                 gtid, npredecessors));
-
   return npredecessors;
 }
 
@@ -266,8 +360,7 @@ static bool __kmp_check_deps(kmp_int32 g
                              kmp_depend_info_t *dep_list,
                              kmp_int32 ndeps_noalias,
                              kmp_depend_info_t *noalias_dep_list) {
-  int i;
-
+  int i, n_mtxs = 0;
 #if KMP_DEBUG
   kmp_taskdata_t *taskdata = KMP_TASK_TO_TASKDATA(task);
 #endif
@@ -279,13 +372,31 @@ static bool __kmp_check_deps(kmp_int32 g
   // Filter deps in dep_list
   // TODO: Different algorithm for large dep_list ( > 10 ? )
   for (i = 0; i < ndeps; i++) {
-    if (dep_list[i].base_addr != 0)
-      for (int j = i + 1; j < ndeps; j++)
+    if (dep_list[i].base_addr != 0) {
+      for (int j = i + 1; j < ndeps; j++) {
         if (dep_list[i].base_addr == dep_list[j].base_addr) {
           dep_list[i].flags.in |= dep_list[j].flags.in;
-          dep_list[i].flags.out |= dep_list[j].flags.out;
+          dep_list[i].flags.out |=
+              (dep_list[j].flags.out ||
+               (dep_list[i].flags.in && dep_list[j].flags.mtx) ||
+               (dep_list[i].flags.mtx && dep_list[j].flags.in));
+          dep_list[i].flags.mtx =
+              dep_list[i].flags.mtx | dep_list[j].flags.mtx &&
+              !dep_list[i].flags.out;
           dep_list[j].base_addr = 0; // Mark j element as void
         }
+      }
+      if (dep_list[i].flags.mtx) {
+        // limit number of mtx deps to MAX_MTX_DEPS per node
+        if (n_mtxs < MAX_MTX_DEPS && task != NULL) {
+          ++n_mtxs;
+        } else {
+          dep_list[i].flags.in = 1; // downgrade mutexinoutset to inout
+          dep_list[i].flags.out = 1;
+          dep_list[i].flags.mtx = 0;
+        }
+      }
+    }
   }
 
   // doesn't need to be atomic as no other thread is going to be accessing this
@@ -469,7 +580,7 @@ kmp_int32 __kmpc_omp_task_with_deps(iden
 
   KA_TRACE(10, ("__kmpc_omp_task_with_deps(exit): T#%d task had no blocking "
                 "dependencies : "
-                "loc=%p task=%p, transferring to __kmpc_omp_task\n",
+                "loc=%p task=%p, transferring to __kmp_omp_task\n",
                 gtid, loc_ref, new_taskdata));
 
   kmp_int32 ret = __kmp_omp_task(gtid, new_task, true);

Modified: openmp/trunk/runtime/src/kmp_taskdeps.h
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp_taskdeps.h?rev=346307&r1=346306&r2=346307&view=diff
==============================================================================
--- openmp/trunk/runtime/src/kmp_taskdeps.h (original)
+++ openmp/trunk/runtime/src/kmp_taskdeps.h Wed Nov  7 04:19:57 2018
@@ -62,7 +62,12 @@ static inline void __kmp_dephash_free_en
       for (kmp_dephash_entry_t *entry = h->buckets[i]; entry; entry = next) {
         next = entry->next_in_bucket;
         __kmp_depnode_list_free(thread, entry->last_ins);
+        __kmp_depnode_list_free(thread, entry->last_mtxs);
         __kmp_node_deref(thread, entry->last_out);
+        if (entry->mtx_lock) {
+          __kmp_destroy_lock(entry->mtx_lock);
+          __kmp_free(entry->mtx_lock);
+        }
 #if USE_FAST_MEMORY
         __kmp_fast_free(thread, entry);
 #else

Modified: openmp/trunk/runtime/src/kmp_tasking.cpp
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp_tasking.cpp?rev=346307&r1=346306&r2=346307&view=diff
==============================================================================
--- openmp/trunk/runtime/src/kmp_tasking.cpp (original)
+++ openmp/trunk/runtime/src/kmp_tasking.cpp Wed Nov  7 04:19:57 2018
@@ -32,7 +32,7 @@ static void __kmp_alloc_task_deque(kmp_i
 static int __kmp_realloc_task_threads_data(kmp_info_t *thread,
                                            kmp_task_team_t *task_team);
 
-#ifdef OMP_45_ENABLED
+#if OMP_45_ENABLED
 static void __kmp_bottom_half_finish_proxy(kmp_int32 gtid, kmp_task_t *ptask);
 #endif
 
@@ -251,6 +251,79 @@ static void __kmp_pop_task_stack(kmp_int
 }
 #endif /* BUILD_TIED_TASK_STACK */
 
+// returns 1 if new task is allowed to execute, 0 otherwise
+// checks Task Scheduling constraint (if requested) and
+// mutexinoutset dependencies if any
+static bool __kmp_task_is_allowed(int gtid, const kmp_int32 is_constrained,
+                                  const kmp_taskdata_t *tasknew,
+                                  const kmp_taskdata_t *taskcurr) {
+  if (is_constrained && (tasknew->td_flags.tiedness == TASK_TIED)) {
+    // Check if the candidate obeys the Task Scheduling Constraints (TSC)
+    // only descendant of all deferred tied tasks can be scheduled, checking
+    // the last one is enough, as it in turn is the descendant of all others
+    kmp_taskdata_t *current = taskcurr->td_last_tied;
+    KMP_DEBUG_ASSERT(current != NULL);
+    // check if the task is not suspended on barrier
+    if (current->td_flags.tasktype == TASK_EXPLICIT ||
+        current->td_taskwait_thread > 0) { // <= 0 on barrier
+      kmp_int32 level = current->td_level;
+      kmp_taskdata_t *parent = tasknew->td_parent;
+      while (parent != current && parent->td_level > level) {
+        // check generation up to the level of the current task
+        parent = parent->td_parent;
+        KMP_DEBUG_ASSERT(parent != NULL);
+      }
+      if (parent != current)
+        return false;
+    }
+  }
+  // Check mutexinoutset dependencies, acquire locks
+  kmp_depnode_t *node = tasknew->td_depnode;
+  if (node && (node->dn.mtx_num_locks > 0)) {
+    for (int i = 0; i < node->dn.mtx_num_locks; ++i) {
+      KMP_DEBUG_ASSERT(node->dn.mtx_locks[i] != NULL);
+      if (__kmp_test_lock(node->dn.mtx_locks[i], gtid))
+        continue;
+      // could not get the lock, release previous locks
+      for (int j = i - 1; j >= 0; --j)
+        __kmp_release_lock(node->dn.mtx_locks[j], gtid);
+      return false;
+    }
+    // negative num_locks means all locks acquired successfully
+    node->dn.mtx_num_locks = -node->dn.mtx_num_locks;
+  }
+  return true;
+}
+
+// __kmp_realloc_task_deque:
+// Re-allocates a task deque for a particular thread, copies the content from
+// the old deque and adjusts the necessary data structures relating to the
+// deque. This operation must be done with the deque_lock being held
+static void __kmp_realloc_task_deque(kmp_info_t *thread,
+                                     kmp_thread_data_t *thread_data) {
+  kmp_int32 size = TASK_DEQUE_SIZE(thread_data->td);
+  kmp_int32 new_size = 2 * size;
+
+  KE_TRACE(10, ("__kmp_realloc_task_deque: T#%d reallocating deque[from %d to "
+                "%d] for thread_data %p\n",
+                __kmp_gtid_from_thread(thread), size, new_size, thread_data));
+
+  kmp_taskdata_t **new_deque =
+      (kmp_taskdata_t **)__kmp_allocate(new_size * sizeof(kmp_taskdata_t *));
+
+  int i, j;
+  for (i = thread_data->td.td_deque_head, j = 0; j < size;
+       i = (i + 1) & TASK_DEQUE_MASK(thread_data->td), j++)
+    new_deque[j] = thread_data->td.td_deque[i];
+
+  __kmp_free(thread_data->td.td_deque);
+
+  thread_data->td.td_deque_head = 0;
+  thread_data->td.td_deque_tail = size;
+  thread_data->td.td_deque = new_deque;
+  thread_data->td.td_deque_size = new_size;
+}
+
 //  __kmp_push_task: Add a task to the thread's deque
 static kmp_int32 __kmp_push_task(kmp_int32 gtid, kmp_task_t *task) {
   kmp_info_t *thread = __kmp_threads[gtid];
@@ -298,33 +371,47 @@ static kmp_int32 __kmp_push_task(kmp_int
     __kmp_alloc_task_deque(thread, thread_data);
   }
 
+  int locked = 0;
   // Check if deque is full
   if (TCR_4(thread_data->td.td_deque_ntasks) >=
       TASK_DEQUE_SIZE(thread_data->td)) {
-    KA_TRACE(20, ("__kmp_push_task: T#%d deque is full; returning "
-                  "TASK_NOT_PUSHED for task %p\n",
-                  gtid, taskdata));
-    return TASK_NOT_PUSHED;
+    if (__kmp_task_is_allowed(gtid, __kmp_task_stealing_constraint, taskdata,
+                              thread->th.th_current_task)) {
+      KA_TRACE(20, ("__kmp_push_task: T#%d deque is full; returning "
+                    "TASK_NOT_PUSHED for task %p\n",
+                    gtid, taskdata));
+      return TASK_NOT_PUSHED;
+    } else {
+      __kmp_acquire_bootstrap_lock(&thread_data->td.td_deque_lock);
+      locked = 1;
+      // expand deque to push the task which is not allowed to execute
+      __kmp_realloc_task_deque(thread, thread_data);
+    }
   }
-
   // Lock the deque for the task push operation
-  __kmp_acquire_bootstrap_lock(&thread_data->td.td_deque_lock);
-
+  if (!locked) {
+    __kmp_acquire_bootstrap_lock(&thread_data->td.td_deque_lock);
 #if OMP_45_ENABLED
-  // Need to recheck as we can get a proxy task from a thread outside of OpenMP
-  if (TCR_4(thread_data->td.td_deque_ntasks) >=
-      TASK_DEQUE_SIZE(thread_data->td)) {
-    __kmp_release_bootstrap_lock(&thread_data->td.td_deque_lock);
-    KA_TRACE(20, ("__kmp_push_task: T#%d deque is full on 2nd check; returning "
-                  "TASK_NOT_PUSHED for task %p\n",
-                  gtid, taskdata));
-    return TASK_NOT_PUSHED;
+    // Need to recheck as we can get a proxy task from thread outside of OpenMP
+    if (TCR_4(thread_data->td.td_deque_ntasks) >=
+        TASK_DEQUE_SIZE(thread_data->td)) {
+      if (__kmp_task_is_allowed(gtid, __kmp_task_stealing_constraint, taskdata,
+                                thread->th.th_current_task)) {
+        __kmp_release_bootstrap_lock(&thread_data->td.td_deque_lock);
+        KA_TRACE(20, ("__kmp_push_task: T#%d deque is full on 2nd check; "
+                      "returning TASK_NOT_PUSHED for task %p\n",
+                      gtid, taskdata));
+        return TASK_NOT_PUSHED;
+      } else {
+        // expand deque to push the task which is not allowed to execute
+        __kmp_realloc_task_deque(thread, thread_data);
+      }
+    }
+#endif
   }
-#else
   // Must have room since no thread can add tasks but calling thread
   KMP_DEBUG_ASSERT(TCR_4(thread_data->td.td_deque_ntasks) <
                    TASK_DEQUE_SIZE(thread_data->td));
-#endif
 
   thread_data->td.td_deque[thread_data->td.td_deque_tail] =
       taskdata; // Push taskdata
@@ -678,11 +765,31 @@ static void __kmp_free_task_and_ancestor
 
     taskdata = parent_taskdata;
 
+    if (team_serial)
+      return;
     // Stop checking ancestors at implicit task instead of walking up ancestor
     // tree to avoid premature deallocation of ancestors.
-    if (team_serial || taskdata->td_flags.tasktype == TASK_IMPLICIT)
+    if (taskdata->td_flags.tasktype == TASK_IMPLICIT) {
+      if (taskdata->td_dephash) { // do we need to cleanup dephash?
+        int children = KMP_ATOMIC_LD_ACQ(&taskdata->td_incomplete_child_tasks);
+        kmp_tasking_flags_t flags_old = taskdata->td_flags;
+        if (children == 0 && flags_old.complete == 1) {
+          kmp_tasking_flags_t flags_new = flags_old;
+          flags_new.complete = 0;
+          if (KMP_COMPARE_AND_STORE_ACQ32(
+                  RCAST(kmp_int32 *, &taskdata->td_flags),
+                  *RCAST(kmp_int32 *, &flags_old),
+                  *RCAST(kmp_int32 *, &flags_new))) {
+            KA_TRACE(100, ("__kmp_free_task_and_ancestors: T#%d cleans "
+                           "dephash of implicit task %p\n",
+                           gtid, taskdata));
+            // cleanup dephash of finished implicit task
+            __kmp_dephash_free_entries(thread, taskdata->td_dephash);
+          }
+        }
+      }
       return;
-
+    }
     // Predecrement simulated by "- 1" calculation
     children = KMP_ATOMIC_DEC(&taskdata->td_allocated_child_tasks) - 1;
     KMP_DEBUG_ASSERT(children >= 0);
@@ -750,6 +857,17 @@ static void __kmp_task_finish(kmp_int32
     __ompt_task_finish(task, resumed_task);
 #endif
 
+  // Check mutexinoutset dependencies, release locks
+  kmp_depnode_t *node = taskdata->td_depnode;
+  if (node && (node->dn.mtx_num_locks < 0)) {
+    // negative num_locks means all locks were acquired
+    node->dn.mtx_num_locks = -node->dn.mtx_num_locks;
+    for (int i = node->dn.mtx_num_locks - 1; i >= 0; --i) {
+      KMP_DEBUG_ASSERT(node->dn.mtx_locks[i] != NULL);
+      __kmp_release_lock(node->dn.mtx_locks[i], gtid);
+    }
+  }
+
   KMP_DEBUG_ASSERT(taskdata->td_flags.complete == 0);
   taskdata->td_flags.complete = 1; // mark the task as completed
   KMP_DEBUG_ASSERT(taskdata->td_flags.started == 1);
@@ -976,8 +1094,24 @@ void __kmp_init_implicit_task(ident_t *l
 // thread:  thread data structure corresponding to implicit task
 void __kmp_finish_implicit_task(kmp_info_t *thread) {
   kmp_taskdata_t *task = thread->th.th_current_task;
-  if (task->td_dephash)
-    __kmp_dephash_free_entries(thread, task->td_dephash);
+  if (task->td_dephash) {
+    int children;
+    task->td_flags.complete = 1;
+    children = KMP_ATOMIC_LD_ACQ(&task->td_incomplete_child_tasks);
+    kmp_tasking_flags_t flags_old = task->td_flags;
+    if (children == 0 && flags_old.complete == 1) {
+      kmp_tasking_flags_t flags_new = flags_old;
+      flags_new.complete = 0;
+      if (KMP_COMPARE_AND_STORE_ACQ32(RCAST(kmp_int32 *, &task->td_flags),
+                                      *RCAST(kmp_int32 *, &flags_old),
+                                      *RCAST(kmp_int32 *, &flags_new))) {
+        KA_TRACE(100, ("__kmp_finish_implicit_task: T#%d cleans "
+                       "dephash of implicit task %p\n",
+                       thread->th.th_info.ds.ds_gtid, task));
+        __kmp_dephash_free_entries(thread, task->td_dephash);
+      }
+    }
+  }
 }
 
 // __kmp_free_implicit_task: Release resources associated to implicit tasks
@@ -2229,33 +2363,16 @@ static kmp_task_t *__kmp_remove_my_task(
          TASK_DEQUE_MASK(thread_data->td); // Wrap index.
   taskdata = thread_data->td.td_deque[tail];
 
-  if (is_constrained && (taskdata->td_flags.tiedness == TASK_TIED)) {
-    // we need to check if the candidate obeys task scheduling constraint (TSC)
-    // only descendant of all deferred tied tasks can be scheduled, checking
-    // the last one is enough, as it in turn is the descendant of all others
-    kmp_taskdata_t *current = thread->th.th_current_task->td_last_tied;
-    KMP_DEBUG_ASSERT(current != NULL);
-    // check if last tied task is not suspended on barrier
-    if (current->td_flags.tasktype == TASK_EXPLICIT ||
-        current->td_taskwait_thread > 0) { // <= 0 on barrier
-      kmp_int32 level = current->td_level;
-      kmp_taskdata_t *parent = taskdata->td_parent;
-      while (parent != current && parent->td_level > level) {
-        parent = parent->td_parent; // check generation up to the level of the
-        // current task
-        KMP_DEBUG_ASSERT(parent != NULL);
-      }
-      if (parent != current) {
-        // The TSC does not allow to steal victim task
-        __kmp_release_bootstrap_lock(&thread_data->td.td_deque_lock);
-        KA_TRACE(10, ("__kmp_remove_my_task(exit #2): T#%d No tasks to remove: "
-                      "ntasks=%d head=%u tail=%u\n",
-                      gtid, thread_data->td.td_deque_ntasks,
-                      thread_data->td.td_deque_head,
-                      thread_data->td.td_deque_tail));
-        return NULL;
-      }
-    }
+  if (!__kmp_task_is_allowed(gtid, is_constrained, taskdata,
+                             thread->th.th_current_task)) {
+    // The TSC does not allow to steal victim task
+    __kmp_release_bootstrap_lock(&thread_data->td.td_deque_lock);
+    KA_TRACE(10,
+             ("__kmp_remove_my_task(exit #3): T#%d TSC blocks tail task: "
+              "ntasks=%d head=%u tail=%u\n",
+              gtid, thread_data->td.td_deque_ntasks,
+              thread_data->td.td_deque_head, thread_data->td.td_deque_tail));
+    return NULL;
   }
 
   thread_data->td.td_deque_tail = tail;
@@ -2263,7 +2380,7 @@ static kmp_task_t *__kmp_remove_my_task(
 
   __kmp_release_bootstrap_lock(&thread_data->td.td_deque_lock);
 
-  KA_TRACE(10, ("__kmp_remove_my_task(exit #2): T#%d task %p removed: "
+  KA_TRACE(10, ("__kmp_remove_my_task(exit #4): T#%d task %p removed: "
                 "ntasks=%d head=%u tail=%u\n",
                 gtid, taskdata, thread_data->td.td_deque_ntasks,
                 thread_data->td.td_deque_head, thread_data->td.td_deque_tail));
@@ -2284,7 +2401,7 @@ static kmp_task_t *__kmp_steal_task(kmp_
   kmp_taskdata_t *taskdata;
   kmp_taskdata_t *current;
   kmp_thread_data_t *victim_td, *threads_data;
-  kmp_int32 level, target;
+  kmp_int32 target;
   kmp_int32 victim_tid;
 
   KMP_DEBUG_ASSERT(__kmp_tasking_mode != tskm_immediate_exec);
@@ -2324,69 +2441,33 @@ static kmp_task_t *__kmp_steal_task(kmp_
   }
 
   KMP_DEBUG_ASSERT(victim_td->td.td_deque != NULL);
-
+  current = __kmp_threads[gtid]->th.th_current_task;
   taskdata = victim_td->td.td_deque[victim_td->td.td_deque_head];
-  if (is_constrained && (taskdata->td_flags.tiedness == TASK_TIED)) {
-    // we need to check if the candidate obeys task scheduling constraint (TSC)
-    // only descendant of all deferred tied tasks can be scheduled, checking
-    // the last one is enough, as it in turn is the descendant of all others
-    current = __kmp_threads[gtid]->th.th_current_task->td_last_tied;
-    KMP_DEBUG_ASSERT(current != NULL);
-    // check if last tied task is not suspended on barrier
-    if (current->td_flags.tasktype == TASK_EXPLICIT ||
-        current->td_taskwait_thread > 0) { // <= 0 on barrier
-      level = current->td_level;
-      kmp_taskdata_t *parent = taskdata->td_parent;
-      while (parent != current && parent->td_level > level) {
-        parent = parent->td_parent; // check generation up to the level of the
-        // current task
-        KMP_DEBUG_ASSERT(parent != NULL);
-      }
-      if (parent != current) {
-        if (!task_team->tt.tt_untied_task_encountered) {
-          // The TSC does not allow to steal victim task
-          __kmp_release_bootstrap_lock(&victim_td->td.td_deque_lock);
-          KA_TRACE(10,
-                   ("__kmp_steal_task(exit #3): T#%d could not steal from "
-                    "T#%d: task_team=%p ntasks=%d head=%u tail=%u\n",
-                    gtid, __kmp_gtid_from_thread(victim_thr), task_team, ntasks,
-                    victim_td->td.td_deque_head, victim_td->td.td_deque_tail));
-          return NULL;
-        }
-        taskdata = NULL; // will check other tasks in victim's deque
-      }
-    }
-  }
-  if (taskdata != NULL) {
+  if (__kmp_task_is_allowed(gtid, is_constrained, taskdata, current)) {
     // Bump head pointer and Wrap.
     victim_td->td.td_deque_head =
         (victim_td->td.td_deque_head + 1) & TASK_DEQUE_MASK(victim_td->td);
   } else {
+    if (!task_team->tt.tt_untied_task_encountered) {
+      // The TSC does not allow to steal victim task
+      __kmp_release_bootstrap_lock(&victim_td->td.td_deque_lock);
+      KA_TRACE(10, ("__kmp_steal_task(exit #3): T#%d could not steal from "
+                    "T#%d: task_team=%p ntasks=%d head=%u tail=%u\n",
+                    gtid, __kmp_gtid_from_thread(victim_thr), task_team, ntasks,
+                    victim_td->td.td_deque_head, victim_td->td.td_deque_tail));
+      return NULL;
+    }
     int i;
     // walk through victim's deque trying to steal any task
     target = victim_td->td.td_deque_head;
+    taskdata = NULL;
     for (i = 1; i < ntasks; ++i) {
       target = (target + 1) & TASK_DEQUE_MASK(victim_td->td);
       taskdata = victim_td->td.td_deque[target];
-      if (taskdata->td_flags.tiedness == TASK_TIED) {
-        // check if the candidate obeys the TSC
-        kmp_taskdata_t *parent = taskdata->td_parent;
-        // check generation up to the level of the current task
-        while (parent != current && parent->td_level > level) {
-          parent = parent->td_parent;
-          KMP_DEBUG_ASSERT(parent != NULL);
-        }
-        if (parent != current) {
-          // The TSC does not allow to steal the candidate
-          taskdata = NULL;
-          continue;
-        } else {
-          // found victim tied task
-          break;
-        }
+      if (__kmp_task_is_allowed(gtid, is_constrained, taskdata, current)) {
+        break; // found victim task
       } else {
-        // found victim untied task
-        break;
+        taskdata = NULL;
       }
     }
     if (taskdata == NULL) {
@@ -2834,35 +2915,6 @@ static void __kmp_alloc_task_deque(kmp_i
   thread_data->td.td_deque_size = INITIAL_TASK_DEQUE_SIZE;
 }
 
-// __kmp_realloc_task_deque:
-// Re-allocates a task deque for a particular thread, copies the content from
-// the old deque and adjusts the necessary data structures relating to the
-// deque. This operation must be done with a the deque_lock being held
-static void __kmp_realloc_task_deque(kmp_info_t *thread,
-                                     kmp_thread_data_t *thread_data) {
-  kmp_int32 size = TASK_DEQUE_SIZE(thread_data->td);
-  kmp_int32 new_size = 2 * size;
-
-  KE_TRACE(10, ("__kmp_realloc_task_deque: T#%d reallocating deque[from %d to "
-                "%d] for thread_data %p\n",
-                __kmp_gtid_from_thread(thread), size, new_size, thread_data));
-
-  kmp_taskdata_t **new_deque =
-      (kmp_taskdata_t **)__kmp_allocate(new_size * sizeof(kmp_taskdata_t *));
-
-  int i, j;
-  for (i = thread_data->td.td_deque_head, j = 0; j < size;
-       i = (i + 1) & TASK_DEQUE_MASK(thread_data->td), j++)
-    new_deque[j] = thread_data->td.td_deque[i];
-
-  __kmp_free(thread_data->td.td_deque);
-
-  thread_data->td.td_deque_head = 0;
-  thread_data->td.td_deque_tail = size;
-  thread_data->td.td_deque = new_deque;
-  thread_data->td.td_deque_size = new_size;
-}
-
 // __kmp_free_task_deque:
 // Deallocates a task deque for a particular thread. Happens at library
 // deallocation so don't need to reset all thread data fields.
@@ -3422,7 +3474,7 @@ release_and_exit:
 
 /* The finish of the proxy tasks is divided in two pieces:
     - the top half is the one that can be done from a thread outside the team
-    - the bottom half must be run from a them within the team
+    - the bottom half must be run from a thread within the team
 
    In order to run the bottom half the task gets queued back into one of the
    threads of the team. Once the td_incomplete_child_task counter of the parent

Added: openmp/trunk/runtime/test/tasking/omp50_task_depend_mtx.c
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/test/tasking/omp50_task_depend_mtx.c?rev=346307&view=auto
==============================================================================
--- openmp/trunk/runtime/test/tasking/omp50_task_depend_mtx.c (added)
+++ openmp/trunk/runtime/test/tasking/omp50_task_depend_mtx.c Wed Nov  7 04:19:57 2018
@@ -0,0 +1,152 @@
+// RUN: %libomp-compile-and-run
+
+// Tests OMP 5.0 task dependences "mutexinoutset", emulates compiler codegen
+// Mutually exclusive tasks get same input dependency info array
+//
+// Task tree created:
+//      task0 task1
+//         \    / \
+//         task2   task5
+//           / \
+//       task3  task4
+//       /   \
+//  task6 <-->task7  (these two are mutually exclusive)
+//       \    /
+//       task8
+//
+#include <stdio.h>
+#include <omp.h>
+
+#ifdef _WIN32
+#include <windows.h>
+#define mysleep(n) Sleep(n)
+#else
+#include <unistd.h>
+#define mysleep(n) usleep((n)*1000)
+#endif
+
+static int checker = 0; // to check if two tasks run simultaneously
+static int err = 0;
+#ifndef DELAY
+#define DELAY 100
+#endif
+
+// ---------------------------------------------------------------------------
+// internal data to emulate compiler codegen
+typedef int(*entry_t)(int, int**);
+typedef struct DEP {
+  size_t addr;
+  size_t len;
+  int flags;
+} dep;
+typedef struct ID {
+  int reserved_1;
+  int flags;
+  int reserved_2;
+  int reserved_3;
+  char *psource;
+} id;
+
+int thunk(int gtid, int** pshareds) {
+  int t = **pshareds;
+  int th = omp_get_thread_num();
+  #pragma omp atomic
+    ++checker;
+  printf("task __%d, th %d\n", t, th);
+  if (checker != 1) {
+    err++;
+    printf("Error1, checker %d != 1\n", checker);
+  }
+  mysleep(DELAY);
+  if (checker != 1) {
+    err++;
+    printf("Error2, checker %d != 1\n", checker);
+  }
+  #pragma omp atomic
+    --checker;
+  return 0;
+}
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+int __kmpc_global_thread_num(id*);
+extern int** __kmpc_omp_task_alloc(id *loc, int gtid, int flags,
+                                   size_t sz, size_t shar, entry_t rtn);
+int
+__kmpc_omp_task_with_deps(id *loc, int gtid, int **task, int nd, dep *dep_lst,
+                          int nd_noalias, dep *noalias_dep_lst);
+static id loc = {0, 2, 0, 0, ";file;func;0;0;;"};
+#ifdef __cplusplus
+} // extern "C"
+#endif
+// End of internal data
+// ---------------------------------------------------------------------------
+
+int main()
+{
+  int i1,i2,i3,i4;
+  omp_set_num_threads(2);
+  #pragma omp parallel
+  {
+    #pragma omp single nowait
+    {
+      dep sdep[2];
+      int **ptr;
+      int gtid = __kmpc_global_thread_num(&loc);
+      int t = omp_get_thread_num();
+      #pragma omp task depend(in: i1, i2)
+      { int th = omp_get_thread_num();
+        printf("task 0_%d, th %d\n", t, th);
+        mysleep(DELAY); }
+      #pragma omp task depend(in: i1, i3)
+      { int th = omp_get_thread_num();
+        printf("task 1_%d, th %d\n", t, th);
+        mysleep(DELAY); }
+      #pragma omp task depend(in: i2) depend(out: i1)
+      { int th = omp_get_thread_num();
+        printf("task 2_%d, th %d\n", t, th);
+        mysleep(DELAY); }
+      #pragma omp task depend(in: i1)
+      { int th = omp_get_thread_num();
+        printf("task 3_%d, th %d\n", t, th);
+        mysleep(DELAY); }
+      #pragma omp task depend(out: i2)
+      { int th = omp_get_thread_num();
+        printf("task 4_%d, th %d\n", t, th);
+        mysleep(DELAY+5); } // wait a bit longer than task 3
+      #pragma omp task depend(out: i3)
+      { int th = omp_get_thread_num();
+        printf("task 5_%d, th %d\n", t, th);
+        mysleep(DELAY); }
+// compiler codegen start
+      // task1
+      ptr = __kmpc_omp_task_alloc(&loc, gtid, 0, 28, 16, thunk);
+      sdep[0].addr = (size_t)&i1;
+      sdep[0].len = 0;   // not used
+      sdep[0].flags = 4; // mx
+      sdep[1].addr = (size_t)&i4;
+      sdep[1].len = 0;   // not used
+      sdep[1].flags = 4; // mx
+      **ptr = t + 10; // init single shared variable
+      __kmpc_omp_task_with_deps(&loc, gtid, ptr, 2, sdep, 0, 0);
+
+      // task2
+      ptr = __kmpc_omp_task_alloc(&loc, gtid, 0, 28, 16, thunk);
+      **ptr = t + 20; // init single shared variable
+      __kmpc_omp_task_with_deps(&loc, gtid, ptr, 2, sdep, 0, 0);
+// compiler codegen end
+      #pragma omp task depend(in: i1)
+      { int th = omp_get_thread_num();
+        printf("task 8_%d, th %d\n", t, th);
+        mysleep(DELAY); }
+    } // single
+  } // parallel
+  if (err == 0) {
+    printf("passed\n");
+    return 0;
+  } else {
+    printf("failed\n");
+    return 1;
+  }
+}

Propchange: openmp/trunk/runtime/test/tasking/omp50_task_depend_mtx.c
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: openmp/trunk/runtime/test/tasking/omp50_task_depend_mtx.c
------------------------------------------------------------------------------
    svn:keywords = Author Date Id Rev URL

Propchange: openmp/trunk/runtime/test/tasking/omp50_task_depend_mtx.c
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: openmp/trunk/runtime/test/tasking/omp50_task_depend_mtx2.c
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/test/tasking/omp50_task_depend_mtx2.c?rev=346307&view=auto
==============================================================================
--- openmp/trunk/runtime/test/tasking/omp50_task_depend_mtx2.c (added)
+++ openmp/trunk/runtime/test/tasking/omp50_task_depend_mtx2.c Wed Nov  7 04:19:57 2018
@@ -0,0 +1,155 @@
+// RUN: %libomp-compile-and-run
+
+// Tests OMP 5.0 task dependences "mutexinoutset", emulates compiler codegen
+// Mutually exclusive tasks get input dependency info array sorted differently
+//
+// Task tree created:
+//      task0 task1
+//         \    / \
+//         task2   task5
+//           / \
+//       task3  task4
+//       /   \
+//  task6 <-->task7  (these two are mutually exclusive)
+//       \    /
+//       task8
+//
+#include <stdio.h>
+#include <omp.h>
+
+#ifdef _WIN32
+#include <windows.h>
+#define mysleep(n) Sleep(n)
+#else
+#include <unistd.h>
+#define mysleep(n) usleep((n)*1000)
+#endif
+
+static int checker = 0; // to check if two tasks run simultaneously
+static int err = 0;
+#ifndef DELAY
+#define DELAY 100
+#endif
+
+// ---------------------------------------------------------------------------
+// internal data to emulate compiler codegen
+typedef int(*entry_t)(int, int**);
+typedef struct DEP {
+  size_t addr;
+  size_t len;
+  int flags;
+} dep;
+typedef struct ID {
+  int reserved_1;
+  int flags;
+  int reserved_2;
+  int reserved_3;
+  char *psource;
+} id;
+
+int thunk(int gtid, int** pshareds) {
+  int t = **pshareds;
+  int th = omp_get_thread_num();
+  #pragma omp atomic
+    ++checker;
+  printf("task __%d, th %d\n", t, th);
+  if (checker != 1) {
+    err++;
+    printf("Error1, checker %d != 1\n", checker);
+  }
+  mysleep(DELAY);
+  if (checker != 1) {
+    err++;
+    printf("Error2, checker %d != 1\n", checker);
+  }
+  #pragma omp atomic
+    --checker;
+  return 0;
+}
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+int __kmpc_global_thread_num(id*);
+extern int** __kmpc_omp_task_alloc(id *loc, int gtid, int flags,
+                                   size_t sz, size_t shar, entry_t rtn);
+int
+__kmpc_omp_task_with_deps(id *loc, int gtid, int **task, int nd, dep *dep_lst,
+                          int nd_noalias, dep *noalias_dep_lst);
+static id loc = {0, 2, 0, 0, ";file;func;0;0;;"};
+#ifdef __cplusplus
+} // extern "C"
+#endif
+// End of internal data
+// ---------------------------------------------------------------------------
+
+int main()
+{
+  int i1,i2,i3,i4;
+  omp_set_num_threads(2);
+  #pragma omp parallel
+  {
+    #pragma omp single nowait
+    {
+      dep sdep[2];
+      int **ptr;
+      int gtid = __kmpc_global_thread_num(&loc);
+      int t = omp_get_thread_num();
+      #pragma omp task depend(in: i1, i2)
+      { int th = omp_get_thread_num();
+        printf("task 0_%d, th %d\n", t, th);
+        mysleep(DELAY); }
+      #pragma omp task depend(in: i1, i3)
+      { int th = omp_get_thread_num();
+        printf("task 1_%d, th %d\n", t, th);
+        mysleep(DELAY); }
+      #pragma omp task depend(in: i2) depend(out: i1)
+      { int th = omp_get_thread_num();
+        printf("task 2_%d, th %d\n", t, th);
+        mysleep(DELAY); }
+      #pragma omp task depend(in: i1)
+      { int th = omp_get_thread_num();
+        printf("task 3_%d, th %d\n", t, th);
+        mysleep(DELAY); }
+      #pragma omp task depend(out: i2)
+      { int th = omp_get_thread_num();
+        printf("task 4_%d, th %d\n", t, th);
+        mysleep(DELAY+5); } // wait a bit longer than task 3
+      #pragma omp task depend(out: i3)
+      { int th = omp_get_thread_num();
+        printf("task 5_%d, th %d\n", t, th);
+        mysleep(DELAY); }
+// compiler codegen start
+      // task1
+      ptr = __kmpc_omp_task_alloc(&loc, gtid, 0, 28, 16, thunk);
+      sdep[0].addr = (size_t)&i1;
+      sdep[0].len = 0;   // not used
+      sdep[0].flags = 4; // mx
+      sdep[1].addr = (size_t)&i4;
+      sdep[1].len = 0;   // not used
+      sdep[1].flags = 4; // mx
+      **ptr = t + 10; // init single shared variable
+      __kmpc_omp_task_with_deps(&loc, gtid, ptr, 2, sdep, 0, 0);
+
+      // task2
+      ptr = __kmpc_omp_task_alloc(&loc, gtid, 0, 28, 16, thunk);
+      // reverse pointers - library should sort them uniquely
+      sdep[0].addr = (size_t)&i4;
+      sdep[1].addr = (size_t)&i1;
+      **ptr = t + 20; // init single shared variable
+      __kmpc_omp_task_with_deps(&loc, gtid, ptr, 2, sdep, 0, 0);
+// compiler codegen end
+      #pragma omp task depend(in: i1)
+      { int th = omp_get_thread_num();
+        printf("task 8_%d, th %d\n", t, th);
+        mysleep(DELAY); }
+    } // single
+  } // parallel
+  if (err == 0) {
+    printf("passed\n");
+    return 0;
+  } else {
+    printf("failed\n");
+    return 1;
+  }
+}

Propchange: openmp/trunk/runtime/test/tasking/omp50_task_depend_mtx2.c
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: openmp/trunk/runtime/test/tasking/omp50_task_depend_mtx2.c
------------------------------------------------------------------------------
    svn:keywords = Author Date Id Rev URL

Propchange: openmp/trunk/runtime/test/tasking/omp50_task_depend_mtx2.c
------------------------------------------------------------------------------
    svn:mime-type = text/plain




More information about the Openmp-commits mailing list