[Openmp-commits] [openmp] r336563 - [OpenMP] Use C++11 Atomics - barrier, tasking, and lock code

Jonathan Peyton via Openmp-commits openmp-commits at lists.llvm.org
Mon Jul 9 10:36:23 PDT 2018


Author: jlpeyton
Date: Mon Jul  9 10:36:22 2018
New Revision: 336563

URL: http://llvm.org/viewvc/llvm-project?rev=336563&view=rev
Log:
[OpenMP] Use C++11 Atomics - barrier, tasking, and lock code

These are preliminary changes that attempt to use C++11 Atomics in the runtime.
We are expecting better portability with this change across architectures/OSes.
Here is the summary of the changes.

Most variables that need synchronization operation were converted to generic
atomic variables (std::atomic<T>). Variables that are updated with combined CAS
are packed into a single atomic variable, and partial read/write is done
through unpacking/packing

Patch by Hansang Bae

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

Modified:
    openmp/trunk/runtime/src/kmp.h
    openmp/trunk/runtime/src/kmp_barrier.cpp
    openmp/trunk/runtime/src/kmp_cancel.cpp
    openmp/trunk/runtime/src/kmp_csupport.cpp
    openmp/trunk/runtime/src/kmp_debugger.cpp
    openmp/trunk/runtime/src/kmp_global.cpp
    openmp/trunk/runtime/src/kmp_io.cpp
    openmp/trunk/runtime/src/kmp_lock.cpp
    openmp/trunk/runtime/src/kmp_lock.h
    openmp/trunk/runtime/src/kmp_omp.h
    openmp/trunk/runtime/src/kmp_os.h
    openmp/trunk/runtime/src/kmp_runtime.cpp
    openmp/trunk/runtime/src/kmp_taskdeps.cpp
    openmp/trunk/runtime/src/kmp_tasking.cpp
    openmp/trunk/runtime/src/kmp_wait_release.h
    openmp/trunk/runtime/src/z_Linux_util.cpp
    openmp/trunk/runtime/src/z_Windows_NT_util.cpp

Modified: openmp/trunk/runtime/src/kmp.h
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp.h?rev=336563&r1=336562&r2=336563&view=diff
==============================================================================
--- openmp/trunk/runtime/src/kmp.h (original)
+++ openmp/trunk/runtime/src/kmp.h Mon Jul  9 10:36:22 2018
@@ -940,7 +940,7 @@ extern int __kmp_hws_abs_flag; // absolu
 // HW TSC is used to reduce overhead (clock tick instead of nanosecond).
 extern kmp_uint64 __kmp_ticks_per_msec;
 #if KMP_COMPILER_ICC
-#define KMP_NOW() _rdtsc()
+#define KMP_NOW() ((kmp_uint64)_rdtsc())
 #else
 #define KMP_NOW() __kmp_hardware_timestamp()
 #endif
@@ -2109,8 +2109,9 @@ typedef struct kmp_task { /* GEH: Should
 
 #if OMP_40_ENABLED
 typedef struct kmp_taskgroup {
-  kmp_int32 count; // number of allocated and not yet complete tasks
-  kmp_int32 cancel_request; // request for cancellation of this taskgroup
+  std::atomic<kmp_int32> count; // number of allocated and incomplete tasks
+  std::atomic<kmp_int32>
+      cancel_request; // request for cancellation of this taskgroup
   struct kmp_taskgroup *parent; // parent taskgroup
 // TODO: change to OMP_50_ENABLED, need to change build tools for this to work
 #if OMP_45_ENABLED
@@ -2149,8 +2150,8 @@ typedef struct kmp_base_depnode {
   kmp_uint32 id;
 #endif
 
-  volatile kmp_int32 npredecessors;
-  volatile kmp_int32 nrefs;
+  std::atomic<kmp_int32> npredecessors;
+  std::atomic<kmp_int32> nrefs;
 } kmp_base_depnode_t;
 
 union KMP_ALIGN_CACHE kmp_depnode {
@@ -2242,7 +2243,7 @@ struct kmp_taskdata { /* aligned during
   /* Currently not used except for perhaps IDB */
   kmp_taskdata_t *td_parent; /* parent task                             */
   kmp_int32 td_level; /* task nesting level                      */
-  kmp_int32 td_untied_count; /* untied task active parts counter        */
+  std::atomic<kmp_int32> td_untied_count; // untied task active parts counter
   ident_t *td_ident; /* task identifier                         */
   // Taskwait data.
   ident_t *td_taskwait_ident;
@@ -2250,10 +2251,10 @@ struct kmp_taskdata { /* aligned during
   kmp_int32 td_taskwait_thread; /* gtid + 1 of thread encountered taskwait */
   KMP_ALIGN_CACHE kmp_internal_control_t
       td_icvs; /* Internal control variables for the task */
-  KMP_ALIGN_CACHE volatile kmp_int32
+  KMP_ALIGN_CACHE std::atomic<kmp_int32>
       td_allocated_child_tasks; /* Child tasks (+ current task) not yet
                                    deallocated */
-  volatile kmp_int32
+  std::atomic<kmp_int32>
       td_incomplete_child_tasks; /* Child tasks not yet complete */
 #if OMP_40_ENABLED
   kmp_taskgroup_t
@@ -2338,7 +2339,7 @@ typedef struct kmp_base_task_team {
   kmp_int32 tt_untied_task_encountered;
 
   KMP_ALIGN_CACHE
-  volatile kmp_int32 tt_unfinished_threads; /* #threads still active      */
+  std::atomic<kmp_int32> tt_unfinished_threads; /* #threads still active */
 
   KMP_ALIGN_CACHE
   volatile kmp_uint32
@@ -2561,7 +2562,7 @@ typedef struct KMP_ALIGN_CACHE kmp_base_
   // ---------------------------------------------------------------------------
   KMP_ALIGN_CACHE kmp_ordered_team_t t_ordered;
   kmp_balign_team_t t_bar[bs_last_barrier];
-  volatile int t_construct; // count of single directive encountered by team
+  std::atomic<int> t_construct; // count of single directive encountered by team
   char pad[sizeof(kmp_lock_t)]; // padding to maintain performance on big iron
 
   // Master only
@@ -2636,12 +2637,14 @@ typedef struct KMP_ALIGN_CACHE kmp_base_
 // for SERIALIZED teams nested 2 or more levels deep
 #if OMP_40_ENABLED
   // typed flag to store request state of cancellation
-  kmp_int32 t_cancel_request;
+  std::atomic<kmp_int32> t_cancel_request;
 #endif
   int t_master_active; // save on fork, restore on join
   kmp_taskq_t t_taskq; // this team's task queue
   void *t_copypriv_data; // team specific pointer to copyprivate data array
-  kmp_uint32 t_copyin_counter;
+#if KMP_OS_WINDOWS
+  std::atomic<kmp_uint32> t_copyin_counter;
+#endif
 #if USE_ITT_BUILD
   void *t_stack_id; // team specific stack stitching id (for ittnotify)
 #endif /* USE_ITT_BUILD */
@@ -2685,7 +2688,8 @@ typedef struct kmp_base_root {
   volatile int r_active; /* TRUE if some region in a nest has > 1 thread */
   // GEH: This is misnamed, should be r_in_parallel
   volatile int r_nested; // TODO: GEH - This is unused, just remove it entirely.
-  int r_in_parallel; /* keeps a count of active parallel regions per root */
+  // keeps a count of active parallel regions per root
+  std::atomic<int> r_in_parallel;
   // GEH: This is misnamed, should be r_active_levels
   kmp_team_t *r_root_team;
   kmp_team_t *r_hot_team;
@@ -2742,8 +2746,8 @@ extern int __kmp_debug_buf_atomic; /* TR
                                       entry pointer */
 
 extern char *__kmp_debug_buffer; /* Debug buffer itself */
-extern int __kmp_debug_count; /* Counter for number of lines printed in buffer
-                                 so far */
+extern std::atomic<int> __kmp_debug_count; /* Counter for number of lines
+                                              printed in buffer so far */
 extern int __kmp_debug_buf_warn_chars; /* Keep track of char increase
                                           recommended in warnings */
 /* end rotating debug buffer */
@@ -3000,7 +3004,7 @@ extern volatile int __kmp_nth;
    threads, and those in the thread pool */
 extern volatile int __kmp_all_nth;
 extern int __kmp_thread_pool_nth;
-extern volatile int __kmp_thread_pool_active_nth;
+extern std::atomic<int> __kmp_thread_pool_active_nth;
 
 extern kmp_root_t **__kmp_root; /* root of thread hierarchy */
 /* end data protected by fork/join lock */
@@ -3009,14 +3013,14 @@ extern kmp_root_t **__kmp_root; /* root
 extern kmp_global_t __kmp_global; /* global status */
 
 extern kmp_info_t __kmp_monitor;
-extern volatile kmp_uint32 __kmp_team_counter; // For Debugging Support Library
-extern volatile kmp_uint32 __kmp_task_counter; // For Debugging Support Library
+// For Debugging Support Library
+extern std::atomic<kmp_uint32> __kmp_team_counter;
+// For Debugging Support Library
+extern std::atomic<kmp_uint32> __kmp_task_counter;
 
 #if USE_DEBUGGER
-
 #define _KMP_GEN_ID(counter)                                                   \
-  (__kmp_debugging ? KMP_TEST_THEN_INC32((volatile kmp_int32 *)&counter) + 1   \
-                   : ~0)
+  (__kmp_debugging ? KMP_ATOMIC_INC(&counter) + 1 : ~0)
 #else
 #define _KMP_GEN_ID(counter) (~0)
 #endif /* USE_DEBUGGER */

Modified: openmp/trunk/runtime/src/kmp_barrier.cpp
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp_barrier.cpp?rev=336563&r1=336562&r2=336563&view=diff
==============================================================================
--- openmp/trunk/runtime/src/kmp_barrier.cpp (original)
+++ openmp/trunk/runtime/src/kmp_barrier.cpp Mon Jul  9 10:36:22 2018
@@ -956,14 +956,12 @@ static void __kmp_hierarchical_barrier_g
   // All subordinates are gathered; now release parent if not master thread
 
   if (!KMP_MASTER_TID(tid)) { // worker threads release parent in hierarchy
-    KA_TRACE(
-        20,
-        ("__kmp_hierarchical_barrier_gather: T#%d(%d:%d) releasing T#%d(%d:%d) "
-         "arrived(%p): %llu => %llu\n",
-         gtid, team->t.t_id, tid,
-         __kmp_gtid_from_tid(thr_bar->parent_tid, team), team->t.t_id,
-         thr_bar->parent_tid, &thr_bar->b_arrived, thr_bar->b_arrived,
-         thr_bar->b_arrived + KMP_BARRIER_STATE_BUMP));
+    KA_TRACE(20, ("__kmp_hierarchical_barrier_gather: T#%d(%d:%d) releasing"
+                  " T#%d(%d:%d) arrived(%p): %llu => %llu\n",
+                  gtid, team->t.t_id, tid,
+                  __kmp_gtid_from_tid(thr_bar->parent_tid, team), team->t.t_id,
+                  thr_bar->parent_tid, &thr_bar->b_arrived, thr_bar->b_arrived,
+                  thr_bar->b_arrived + KMP_BARRIER_STATE_BUMP));
     /* Mark arrival to parent: After performing this write, a worker thread may
        not assume that the team is valid any more - it could be deallocated by
        the master thread at any time. */
@@ -973,8 +971,8 @@ static void __kmp_hierarchical_barrier_g
       ANNOTATE_BARRIER_BEGIN(this_thr);
       kmp_flag_64 flag(&thr_bar->b_arrived, other_threads[thr_bar->parent_tid]);
       flag.release();
-    } else { // Leaf does special release on the "offset" bits of parent's
-      // b_arrived flag
+    } else {
+      // Leaf does special release on "offset" bits of parent's b_arrived flag
       thr_bar->b_arrived = team->t.t_bar[bt].b_arrived + KMP_BARRIER_STATE_BUMP;
       kmp_flag_oncore flag(&thr_bar->parent_bar->b_arrived, thr_bar->offset);
       flag.set_waiter(other_threads[thr_bar->parent_tid]);
@@ -1353,10 +1351,10 @@ int __kmp_barrier(enum barrier_type bt,
 #endif
 
 #if OMP_40_ENABLED
+      kmp_int32 cancel_request = KMP_ATOMIC_LD_RLX(&team->t.t_cancel_request);
       // Reset cancellation flag for worksharing constructs
-      if (team->t.t_cancel_request == cancel_loop ||
-          team->t.t_cancel_request == cancel_sections) {
-        team->t.t_cancel_request = cancel_noreq;
+      if (cancel_request == cancel_loop || cancel_request == cancel_sections) {
+        KMP_ATOMIC_ST_RLX(&team->t.t_cancel_request, cancel_noreq);
       }
 #endif
 #if USE_ITT_BUILD

Modified: openmp/trunk/runtime/src/kmp_cancel.cpp
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp_cancel.cpp?rev=336563&r1=336562&r2=336563&view=diff
==============================================================================
--- openmp/trunk/runtime/src/kmp_cancel.cpp (original)
+++ openmp/trunk/runtime/src/kmp_cancel.cpp Mon Jul  9 10:36:22 2018
@@ -51,8 +51,8 @@ kmp_int32 __kmpc_cancel(ident_t *loc_ref
       {
         kmp_team_t *this_team = this_thr->th.th_team;
         KMP_DEBUG_ASSERT(this_team);
-        kmp_int32 old = KMP_COMPARE_AND_STORE_RET32(
-            &(this_team->t.t_cancel_request), cancel_noreq, cncl_kind);
+        kmp_int32 old = cancel_noreq;
+        this_team->t.t_cancel_request.compare_exchange_strong(old, cncl_kind);
         if (old == cancel_noreq || old == cncl_kind) {
 // we do not have a cancellation request in this team or we do have
 // one that matches the current request -> cancel
@@ -89,8 +89,8 @@ kmp_int32 __kmpc_cancel(ident_t *loc_ref
 
         taskgroup = task->td_taskgroup;
         if (taskgroup) {
-          kmp_int32 old = KMP_COMPARE_AND_STORE_RET32(
-              &(taskgroup->cancel_request), cancel_noreq, cncl_kind);
+          kmp_int32 old = cancel_noreq;
+          taskgroup->cancel_request.compare_exchange_strong(old, cncl_kind);
           if (old == cancel_noreq || old == cncl_kind) {
 // we do not have a cancellation request in this taskgroup or we do
 // have one that matches the current request -> cancel
@@ -257,7 +257,7 @@ kmp_int32 __kmpc_cancel_barrier(ident_t
   if (__kmp_omp_cancellation) {
     // depending on which construct to cancel, check the flag and
     // reset the flag
-    switch (this_team->t.t_cancel_request) {
+    switch (KMP_ATOMIC_LD_RLX(&(this_team->t.t_cancel_request))) {
     case cancel_parallel:
       ret = 1;
       // ensure that threads have checked the flag, when

Modified: openmp/trunk/runtime/src/kmp_csupport.cpp
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp_csupport.cpp?rev=336563&r1=336562&r2=336563&view=diff
==============================================================================
--- openmp/trunk/runtime/src/kmp_csupport.cpp (original)
+++ openmp/trunk/runtime/src/kmp_csupport.cpp Mon Jul  9 10:36:22 2018
@@ -930,9 +930,10 @@ __kmp_init_indirect_csptr(kmp_critical_n
 #define KMP_ACQUIRE_TAS_LOCK(lock, gtid)                                       \
   {                                                                            \
     kmp_tas_lock_t *l = (kmp_tas_lock_t *)lock;                                \
-    if (l->lk.poll != KMP_LOCK_FREE(tas) ||                                    \
-        !KMP_COMPARE_AND_STORE_ACQ32(&(l->lk.poll), KMP_LOCK_FREE(tas),        \
-                                     KMP_LOCK_BUSY(gtid + 1, tas))) {          \
+    kmp_int32 tas_free = KMP_LOCK_FREE(tas);                                   \
+    kmp_int32 tas_busy = KMP_LOCK_BUSY(gtid + 1, tas);                         \
+    if (KMP_ATOMIC_LD_RLX(&l->lk.poll) != tas_free ||                          \
+        !__kmp_atomic_compare_store_acq(&l->lk.poll, tas_free, tas_busy)) {    \
       kmp_uint32 spins;                                                        \
       KMP_FSYNC_PREPARE(l);                                                    \
       KMP_INIT_YIELD(spins);                                                   \
@@ -943,9 +944,9 @@ __kmp_init_indirect_csptr(kmp_critical_n
         KMP_YIELD_SPIN(spins);                                                 \
       }                                                                        \
       kmp_backoff_t backoff = __kmp_spin_backoff_params;                       \
-      while (l->lk.poll != KMP_LOCK_FREE(tas) ||                               \
-             !KMP_COMPARE_AND_STORE_ACQ32(&(l->lk.poll), KMP_LOCK_FREE(tas),   \
-                                          KMP_LOCK_BUSY(gtid + 1, tas))) {     \
+      while (                                                                  \
+          KMP_ATOMIC_LD_RLX(&l->lk.poll) != tas_free ||                        \
+          !__kmp_atomic_compare_store_acq(&l->lk.poll, tas_free, tas_busy)) {  \
         __kmp_spin_backoff(&backoff);                                          \
         if (TCR_4(__kmp_nth) >                                                 \
             (__kmp_avail_proc ? __kmp_avail_proc : __kmp_xproc)) {             \
@@ -962,17 +963,15 @@ __kmp_init_indirect_csptr(kmp_critical_n
 #define KMP_TEST_TAS_LOCK(lock, gtid, rc)                                      \
   {                                                                            \
     kmp_tas_lock_t *l = (kmp_tas_lock_t *)lock;                                \
-    rc = l->lk.poll == KMP_LOCK_FREE(tas) &&                                   \
-         KMP_COMPARE_AND_STORE_ACQ32(&(l->lk.poll), KMP_LOCK_FREE(tas),        \
-                                     KMP_LOCK_BUSY(gtid + 1, tas));            \
+    kmp_int32 tas_free = KMP_LOCK_FREE(tas);                                   \
+    kmp_int32 tas_busy = KMP_LOCK_BUSY(gtid + 1, tas);                         \
+    rc = KMP_ATOMIC_LD_RLX(&l->lk.poll) == tas_free &&                         \
+         __kmp_atomic_compare_store_acq(&l->lk.poll, tas_free, tas_busy);      \
   }
 
 // Fast-path release tas lock
 #define KMP_RELEASE_TAS_LOCK(lock, gtid)                                       \
-  {                                                                            \
-    TCW_4(((kmp_tas_lock_t *)lock)->lk.poll, KMP_LOCK_FREE(tas));              \
-    KMP_MB();                                                                  \
-  }
+  { KMP_ATOMIC_ST_REL(&((kmp_tas_lock_t *)lock)->lk.poll, KMP_LOCK_FREE(tas)); }
 
 #if KMP_USE_FUTEX
 

Modified: openmp/trunk/runtime/src/kmp_debugger.cpp
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp_debugger.cpp?rev=336563&r1=336562&r2=336563&view=diff
==============================================================================
--- openmp/trunk/runtime/src/kmp_debugger.cpp (original)
+++ openmp/trunk/runtime/src/kmp_debugger.cpp Mon Jul  9 10:36:22 2018
@@ -68,7 +68,9 @@ kmp_omp_struct_info_t __kmp_omp_debug_st
     addr_and_size_of(__kmp_threads),
     addr_and_size_of(__kmp_root),
     addr_and_size_of(__kmp_threads_capacity),
+#if KMP_USE_MONITOR
     addr_and_size_of(__kmp_monitor),
+#endif
 #if !KMP_USE_DYNAMIC_LOCK
     addr_and_size_of(__kmp_user_lock_table),
 #endif

Modified: openmp/trunk/runtime/src/kmp_global.cpp
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp_global.cpp?rev=336563&r1=336562&r2=336563&view=diff
==============================================================================
--- openmp/trunk/runtime/src/kmp_global.cpp (original)
+++ openmp/trunk/runtime/src/kmp_global.cpp Mon Jul  9 10:36:22 2018
@@ -57,8 +57,8 @@ int __kmp_init_counter = 0;
 int __kmp_root_counter = 0;
 int __kmp_version = 0;
 
-volatile kmp_uint32 __kmp_team_counter = 0;
-volatile kmp_uint32 __kmp_task_counter = 0;
+std::atomic<kmp_uint32> __kmp_team_counter = ATOMIC_VAR_INIT(0);
+std::atomic<kmp_uint32> __kmp_task_counter = ATOMIC_VAR_INIT(0);
 
 unsigned int __kmp_init_wait =
     KMP_DEFAULT_INIT_WAIT; /* initial number of spin-tests   */
@@ -335,8 +335,8 @@ int __kmp_debug_buf_atomic =
     FALSE; /* TRUE means use atomic update of buffer entry pointer */
 
 char *__kmp_debug_buffer = NULL; /* Debug buffer itself */
-int __kmp_debug_count =
-    0; /* Counter for number of lines printed in buffer so far */
+std::atomic<int> __kmp_debug_count =
+    ATOMIC_VAR_INIT(0); /* number of lines printed in buffer so far */
 int __kmp_debug_buf_warn_chars =
     0; /* Keep track of char increase recommended in warnings */
 /* end rotating debug buffer */
@@ -402,7 +402,7 @@ volatile kmp_info_t *__kmp_thread_pool =
 volatile kmp_team_t *__kmp_team_pool = NULL;
 
 KMP_ALIGN_CACHE
-volatile int __kmp_thread_pool_active_nth = 0;
+std::atomic<int> __kmp_thread_pool_active_nth = ATOMIC_VAR_INIT(0);
 
 /* -------------------------------------------------
  * GLOBAL/ROOT STATE */
@@ -418,47 +418,47 @@ kmp_global_t __kmp_global = {{0}};
  * false sharing if the alignment is not large enough for these locks */
 KMP_ALIGN_CACHE_INTERNODE
 
-kmp_bootstrap_lock_t __kmp_initz_lock = KMP_BOOTSTRAP_LOCK_INITIALIZER(
-    __kmp_initz_lock); /* Control initializations */
+KMP_BOOTSTRAP_LOCK_INIT(__kmp_initz_lock); /* Control initializations */
 KMP_ALIGN_CACHE_INTERNODE
-kmp_bootstrap_lock_t __kmp_forkjoin_lock; /* control fork/join access */
+KMP_BOOTSTRAP_LOCK_INIT(__kmp_forkjoin_lock); /* control fork/join access */
 KMP_ALIGN_CACHE_INTERNODE
-kmp_bootstrap_lock_t __kmp_exit_lock; /* exit() is not always thread-safe */
+KMP_BOOTSTRAP_LOCK_INIT(__kmp_exit_lock); /* exit() is not always thread-safe */
 #if KMP_USE_MONITOR
+/* control monitor thread creation */
 KMP_ALIGN_CACHE_INTERNODE
-kmp_bootstrap_lock_t __kmp_monitor_lock; /* control monitor thread creation */
+KMP_BOOTSTRAP_LOCK_INIT(__kmp_monitor_lock);
 #endif
 /* used for the hack to allow threadprivate cache and __kmp_threads expansion
    to co-exist */
 KMP_ALIGN_CACHE_INTERNODE
-kmp_bootstrap_lock_t __kmp_tp_cached_lock;
+KMP_BOOTSTRAP_LOCK_INIT(__kmp_tp_cached_lock);
 
 KMP_ALIGN_CACHE_INTERNODE
-kmp_lock_t __kmp_global_lock; /* Control OS/global access */
+KMP_LOCK_INIT(__kmp_global_lock); /* Control OS/global access */
 KMP_ALIGN_CACHE_INTERNODE
 kmp_queuing_lock_t __kmp_dispatch_lock; /* Control dispatch access  */
 KMP_ALIGN_CACHE_INTERNODE
-kmp_lock_t __kmp_debug_lock; /* Control I/O access for KMP_DEBUG */
+KMP_LOCK_INIT(__kmp_debug_lock); /* Control I/O access for KMP_DEBUG */
 #else
 KMP_ALIGN_CACHE
 
-kmp_bootstrap_lock_t __kmp_initz_lock = KMP_BOOTSTRAP_LOCK_INITIALIZER(
-    __kmp_initz_lock); /* Control initializations */
-kmp_bootstrap_lock_t __kmp_forkjoin_lock; /* control fork/join access */
-kmp_bootstrap_lock_t __kmp_exit_lock; /* exit() is not always thread-safe */
+KMP_BOOTSTRAP_LOCK_INIT(__kmp_initz_lock); /* Control initializations */
+KMP_BOOTSTRAP_LOCK_INIT(__kmp_forkjoin_lock); /* control fork/join access */
+KMP_BOOTSTRAP_LOCK_INIT(__kmp_exit_lock); /* exit() is not always thread-safe */
 #if KMP_USE_MONITOR
-kmp_bootstrap_lock_t __kmp_monitor_lock; /* control monitor thread creation */
+/* control monitor thread creation */
+KMP_BOOTSTRAP_LOCK_INIT(__kmp_monitor_lock);
 #endif
 /* used for the hack to allow threadprivate cache and __kmp_threads expansion
    to co-exist */
-kmp_bootstrap_lock_t __kmp_tp_cached_lock;
+KMP_BOOTSTRAP_LOCK_INIT(__kmp_tp_cached_lock);
 
 KMP_ALIGN(128)
-kmp_lock_t __kmp_global_lock; /* Control OS/global access */
+KMP_LOCK_INIT(__kmp_global_lock); /* Control OS/global access */
 KMP_ALIGN(128)
 kmp_queuing_lock_t __kmp_dispatch_lock; /* Control dispatch access  */
 KMP_ALIGN(128)
-kmp_lock_t __kmp_debug_lock; /* Control I/O access for KMP_DEBUG */
+KMP_LOCK_INIT(__kmp_debug_lock); /* Control I/O access for KMP_DEBUG */
 #endif
 
 /* ----------------------------------------------- */

Modified: openmp/trunk/runtime/src/kmp_io.cpp
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp_io.cpp?rev=336563&r1=336562&r2=336563&view=diff
==============================================================================
--- openmp/trunk/runtime/src/kmp_io.cpp (original)
+++ openmp/trunk/runtime/src/kmp_io.cpp Mon Jul  9 10:36:22 2018
@@ -152,9 +152,7 @@ void __kmp_vprintf(enum kmp_io __kmp_io,
 
   if (__kmp_debug_buf && __kmp_debug_buffer != NULL) {
 
-    int dc = (__kmp_debug_buf_atomic ? KMP_TEST_THEN_INC32(&__kmp_debug_count)
-                                     : __kmp_debug_count++) %
-             __kmp_debug_buf_lines;
+    int dc = __kmp_debug_count++ % __kmp_debug_buf_lines;
     char *db = &__kmp_debug_buffer[dc * __kmp_debug_buf_chars];
     int chars = 0;
 

Modified: openmp/trunk/runtime/src/kmp_lock.cpp
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp_lock.cpp?rev=336563&r1=336562&r2=336563&view=diff
==============================================================================
--- openmp/trunk/runtime/src/kmp_lock.cpp (original)
+++ openmp/trunk/runtime/src/kmp_lock.cpp Mon Jul  9 10:36:22 2018
@@ -19,6 +19,7 @@
 #include "kmp_io.h"
 #include "kmp_itt.h"
 #include "kmp_lock.h"
+#include "kmp_wait_release.h"
 
 #include "tsan_annotations.h"
 
@@ -69,7 +70,7 @@ void __kmp_validate_locks(void) {
 // entire 8 bytes were allocated for nested locks on all 64-bit platforms.
 
 static kmp_int32 __kmp_get_tas_lock_owner(kmp_tas_lock_t *lck) {
-  return KMP_LOCK_STRIP(TCR_4(lck->lk.poll)) - 1;
+  return KMP_LOCK_STRIP(KMP_ATOMIC_LD_RLX(&lck->lk.poll)) - 1;
 }
 
 static inline bool __kmp_is_tas_lock_nestable(kmp_tas_lock_t *lck) {
@@ -81,15 +82,17 @@ __kmp_acquire_tas_lock_timed_template(km
   KMP_MB();
 
 #ifdef USE_LOCK_PROFILE
-  kmp_uint32 curr = KMP_LOCK_STRIP(TCR_4(lck->lk.poll));
+  kmp_uint32 curr = KMP_LOCK_STRIP(lck->lk.poll);
   if ((curr != 0) && (curr != gtid + 1))
     __kmp_printf("LOCK CONTENTION: %p\n", lck);
 /* else __kmp_printf( "." );*/
 #endif /* USE_LOCK_PROFILE */
 
-  if ((lck->lk.poll == KMP_LOCK_FREE(tas)) &&
-      KMP_COMPARE_AND_STORE_ACQ32(&(lck->lk.poll), KMP_LOCK_FREE(tas),
-                                  KMP_LOCK_BUSY(gtid + 1, tas))) {
+  kmp_int32 tas_free = KMP_LOCK_FREE(tas);
+  kmp_int32 tas_busy = KMP_LOCK_BUSY(gtid + 1, tas);
+
+  if (KMP_ATOMIC_LD_RLX(&lck->lk.poll) == tas_free &&
+      __kmp_atomic_compare_store_acq(&lck->lk.poll, tas_free, tas_busy)) {
     KMP_FSYNC_ACQUIRED(lck);
     return KMP_LOCK_ACQUIRED_FIRST;
   }
@@ -104,10 +107,8 @@ __kmp_acquire_tas_lock_timed_template(km
   }
 
   kmp_backoff_t backoff = __kmp_spin_backoff_params;
-  while ((lck->lk.poll != KMP_LOCK_FREE(tas)) ||
-         (!KMP_COMPARE_AND_STORE_ACQ32(&(lck->lk.poll), KMP_LOCK_FREE(tas),
-                                       KMP_LOCK_BUSY(gtid + 1, tas)))) {
-
+  while (KMP_ATOMIC_LD_RLX(&lck->lk.poll) != tas_free ||
+         !__kmp_atomic_compare_store_acq(&lck->lk.poll, tas_free, tas_busy)) {
     __kmp_spin_backoff(&backoff);
     if (TCR_4(__kmp_nth) >
         (__kmp_avail_proc ? __kmp_avail_proc : __kmp_xproc)) {
@@ -140,9 +141,10 @@ static int __kmp_acquire_tas_lock_with_c
 }
 
 int __kmp_test_tas_lock(kmp_tas_lock_t *lck, kmp_int32 gtid) {
-  if ((lck->lk.poll == KMP_LOCK_FREE(tas)) &&
-      KMP_COMPARE_AND_STORE_ACQ32(&(lck->lk.poll), KMP_LOCK_FREE(tas),
-                                  KMP_LOCK_BUSY(gtid + 1, tas))) {
+  kmp_int32 tas_free = KMP_LOCK_FREE(tas);
+  kmp_int32 tas_busy = KMP_LOCK_BUSY(gtid + 1, tas);
+  if (KMP_ATOMIC_LD_RLX(&lck->lk.poll) == tas_free &&
+      __kmp_atomic_compare_store_acq(&lck->lk.poll, tas_free, tas_busy)) {
     KMP_FSYNC_ACQUIRED(lck);
     return TRUE;
   }
@@ -164,7 +166,7 @@ int __kmp_release_tas_lock(kmp_tas_lock_
 
   KMP_FSYNC_RELEASING(lck);
   ANNOTATE_TAS_RELEASED(lck);
-  KMP_ST_REL32(&(lck->lk.poll), KMP_LOCK_FREE(tas));
+  KMP_ATOMIC_ST_REL(&lck->lk.poll, KMP_LOCK_FREE(tas));
   KMP_MB(); /* Flush all pending memory write invalidates.  */
 
   KMP_YIELD(TCR_4(__kmp_nth) >
@@ -191,7 +193,7 @@ static int __kmp_release_tas_lock_with_c
 }
 
 void __kmp_init_tas_lock(kmp_tas_lock_t *lck) {
-  TCW_4(lck->lk.poll, KMP_LOCK_FREE(tas));
+  lck->lk.poll = KMP_LOCK_FREE(tas);
 }
 
 static void __kmp_init_tas_lock_with_checks(kmp_tas_lock_t *lck) {
@@ -2279,7 +2281,7 @@ static void __kmp_destroy_adaptive_lock_
 /* "DRDPA" means Dynamically Reconfigurable Distributed Polling Area */
 
 static kmp_int32 __kmp_get_drdpa_lock_owner(kmp_drdpa_lock_t *lck) {
-  return TCR_4(lck->lk.owner_id) - 1;
+  return lck->lk.owner_id - 1;
 }
 
 static inline bool __kmp_is_drdpa_lock_nestable(kmp_drdpa_lock_t *lck) {
@@ -2288,13 +2290,12 @@ static inline bool __kmp_is_drdpa_lock_n
 
 __forceinline static int
 __kmp_acquire_drdpa_lock_timed_template(kmp_drdpa_lock_t *lck, kmp_int32 gtid) {
-  kmp_uint64 ticket =
-      KMP_TEST_THEN_INC64(RCAST(volatile kmp_int64 *, &lck->lk.next_ticket));
-  kmp_uint64 mask = TCR_8(lck->lk.mask); // volatile load
-  volatile struct kmp_base_drdpa_lock::kmp_lock_poll *polls = lck->lk.polls;
+  kmp_uint64 ticket = KMP_ATOMIC_INC(&lck->lk.next_ticket);
+  kmp_uint64 mask = lck->lk.mask; // atomic load
+  std::atomic<kmp_uint64> *polls = lck->lk.polls;
 
 #ifdef USE_LOCK_PROFILE
-  if (TCR_8(polls[ticket & mask].poll) != ticket)
+  if (polls[ticket & mask] != ticket)
     __kmp_printf("LOCK CONTENTION: %p\n", lck);
 /* else __kmp_printf( "." );*/
 #endif /* USE_LOCK_PROFILE */
@@ -2311,7 +2312,7 @@ __kmp_acquire_drdpa_lock_timed_template(
 
   KMP_FSYNC_PREPARE(lck);
   KMP_INIT_YIELD(spins);
-  while (TCR_8(polls[ticket & mask].poll) < ticket) { // volatile load
+  while (polls[ticket & mask] < ticket) { // atomic load
     // If we are oversubscribed,
     // or have waited a bit (and KMP_LIBRARY=turnaround), then yield.
     // CPU Pause is in the macros for yield.
@@ -2327,8 +2328,8 @@ __kmp_acquire_drdpa_lock_timed_template(
     // If another thread picks reconfigures the polling area and updates their
     // values, and we get the new value of mask and the old polls pointer, we
     // could access memory beyond the end of the old polling area.
-    mask = TCR_8(lck->lk.mask); // volatile load
-    polls = lck->lk.polls; // volatile load
+    mask = lck->lk.mask; // atomic load
+    polls = lck->lk.polls; // atomic load
   }
 
   // Critical section starts here
@@ -2343,7 +2344,7 @@ __kmp_acquire_drdpa_lock_timed_template(
   // The >= check is in case __kmp_test_drdpa_lock() allocated the cleanup
   // ticket.
   if ((lck->lk.old_polls != NULL) && (ticket >= lck->lk.cleanup_ticket)) {
-    __kmp_free(CCAST(kmp_base_drdpa_lock::kmp_lock_poll *, lck->lk.old_polls));
+    __kmp_free(lck->lk.old_polls);
     lck->lk.old_polls = NULL;
     lck->lk.cleanup_ticket = 0;
   }
@@ -2353,7 +2354,7 @@ __kmp_acquire_drdpa_lock_timed_template(
   // previous reconfiguration, let a later thread reconfigure it.
   if (lck->lk.old_polls == NULL) {
     bool reconfigure = false;
-    volatile struct kmp_base_drdpa_lock::kmp_lock_poll *old_polls = polls;
+    std::atomic<kmp_uint64> *old_polls = polls;
     kmp_uint32 num_polls = TCR_4(lck->lk.num_polls);
 
     if (TCR_4(__kmp_nth) >
@@ -2365,9 +2366,9 @@ __kmp_acquire_drdpa_lock_timed_template(
         num_polls = TCR_4(lck->lk.num_polls);
         mask = 0;
         num_polls = 1;
-        polls = (volatile struct kmp_base_drdpa_lock::kmp_lock_poll *)
-            __kmp_allocate(num_polls * sizeof(*polls));
-        polls[0].poll = ticket;
+        polls = (std::atomic<kmp_uint64> *)__kmp_allocate(num_polls *
+                                                          sizeof(*polls));
+        polls[0] = ticket;
       }
     } else {
       // We are in under/fully subscribed mode.  Check the number of
@@ -2386,11 +2387,11 @@ __kmp_acquire_drdpa_lock_timed_template(
         // of the old polling area to the new area.  __kmp_allocate()
         // zeroes the memory it allocates, and most of the old area is
         // just zero padding, so we only copy the release counters.
-        polls = (volatile struct kmp_base_drdpa_lock::kmp_lock_poll *)
-            __kmp_allocate(num_polls * sizeof(*polls));
+        polls = (std::atomic<kmp_uint64> *)__kmp_allocate(num_polls *
+                                                          sizeof(*polls));
         kmp_uint32 i;
         for (i = 0; i < old_num_polls; i++) {
-          polls[i].poll = old_polls[i].poll;
+          polls[i].store(old_polls[i]);
         }
       }
     }
@@ -2409,13 +2410,13 @@ __kmp_acquire_drdpa_lock_timed_template(
                       "lock %p to %d polls\n",
                       ticket, lck, num_polls));
 
-      lck->lk.old_polls = old_polls; // non-volatile store
-      lck->lk.polls = polls; // volatile store
+      lck->lk.old_polls = old_polls;
+      lck->lk.polls = polls; // atomic store
 
       KMP_MB();
 
-      lck->lk.num_polls = num_polls; // non-volatile store
-      lck->lk.mask = mask; // volatile store
+      lck->lk.num_polls = num_polls;
+      lck->lk.mask = mask; // atomic store
 
       KMP_MB();
 
@@ -2423,7 +2424,7 @@ __kmp_acquire_drdpa_lock_timed_template(
       // to main memory can we update the cleanup ticket field.
       //
       // volatile load / non-volatile store
-      lck->lk.cleanup_ticket = TCR_8(lck->lk.next_ticket);
+      lck->lk.cleanup_ticket = lck->lk.next_ticket;
     }
   }
   return KMP_LOCK_ACQUIRED_FIRST;
@@ -2457,13 +2458,13 @@ static int __kmp_acquire_drdpa_lock_with
 int __kmp_test_drdpa_lock(kmp_drdpa_lock_t *lck, kmp_int32 gtid) {
   // First get a ticket, then read the polls pointer and the mask.
   // The polls pointer must be read before the mask!!! (See above)
-  kmp_uint64 ticket = TCR_8(lck->lk.next_ticket); // volatile load
-  volatile struct kmp_base_drdpa_lock::kmp_lock_poll *polls = lck->lk.polls;
-  kmp_uint64 mask = TCR_8(lck->lk.mask); // volatile load
-  if (TCR_8(polls[ticket & mask].poll) == ticket) {
+  kmp_uint64 ticket = lck->lk.next_ticket; // atomic load
+  std::atomic<kmp_uint64> *polls = lck->lk.polls;
+  kmp_uint64 mask = lck->lk.mask; // atomic load
+  if (polls[ticket & mask] == ticket) {
     kmp_uint64 next_ticket = ticket + 1;
-    if (KMP_COMPARE_AND_STORE_ACQ64(&lck->lk.next_ticket, ticket,
-                                    next_ticket)) {
+    if (__kmp_atomic_compare_store_acq(&lck->lk.next_ticket, ticket,
+                                       next_ticket)) {
       KMP_FSYNC_ACQUIRED(lck);
       KA_TRACE(1000, ("__kmp_test_drdpa_lock: ticket #%lld acquired lock %p\n",
                       ticket, lck));
@@ -2502,14 +2503,14 @@ static int __kmp_test_drdpa_lock_with_ch
 int __kmp_release_drdpa_lock(kmp_drdpa_lock_t *lck, kmp_int32 gtid) {
   // Read the ticket value from the lock data struct, then the polls pointer and
   // the mask.  The polls pointer must be read before the mask!!! (See above)
-  kmp_uint64 ticket = lck->lk.now_serving + 1; // non-volatile load
-  volatile struct kmp_base_drdpa_lock::kmp_lock_poll *polls = lck->lk.polls;
-  kmp_uint64 mask = TCR_8(lck->lk.mask); // volatile load
+  kmp_uint64 ticket = lck->lk.now_serving + 1; // non-atomic load
+  std::atomic<kmp_uint64> *polls = lck->lk.polls; // atomic load
+  kmp_uint64 mask = lck->lk.mask; // atomic load
   KA_TRACE(1000, ("__kmp_release_drdpa_lock: ticket #%lld released lock %p\n",
                   ticket - 1, lck));
   KMP_FSYNC_RELEASING(lck);
   ANNOTATE_DRDPA_RELEASED(lck);
-  KMP_ST_REL64(&(polls[ticket & mask].poll), ticket); // volatile store
+  polls[ticket & mask] = ticket; // atomic store
   return KMP_LOCK_RELEASED;
 }
 
@@ -2538,9 +2539,8 @@ void __kmp_init_drdpa_lock(kmp_drdpa_loc
   lck->lk.location = NULL;
   lck->lk.mask = 0;
   lck->lk.num_polls = 1;
-  lck->lk.polls =
-      (volatile struct kmp_base_drdpa_lock::kmp_lock_poll *)__kmp_allocate(
-          lck->lk.num_polls * sizeof(*(lck->lk.polls)));
+  lck->lk.polls = (std::atomic<kmp_uint64> *)__kmp_allocate(
+      lck->lk.num_polls * sizeof(*(lck->lk.polls)));
   lck->lk.cleanup_ticket = 0;
   lck->lk.old_polls = NULL;
   lck->lk.next_ticket = 0;
@@ -2559,12 +2559,12 @@ static void __kmp_init_drdpa_lock_with_c
 void __kmp_destroy_drdpa_lock(kmp_drdpa_lock_t *lck) {
   lck->lk.initialized = NULL;
   lck->lk.location = NULL;
-  if (lck->lk.polls != NULL) {
-    __kmp_free(CCAST(kmp_base_drdpa_lock::kmp_lock_poll *, lck->lk.polls));
+  if (lck->lk.polls.load() != NULL) {
+    __kmp_free(lck->lk.polls.load());
     lck->lk.polls = NULL;
   }
   if (lck->lk.old_polls != NULL) {
-    __kmp_free(CCAST(kmp_base_drdpa_lock::kmp_lock_poll *, lck->lk.old_polls));
+    __kmp_free(lck->lk.old_polls);
     lck->lk.old_polls = NULL;
   }
   lck->lk.mask = 0;

Modified: openmp/trunk/runtime/src/kmp_lock.h
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp_lock.h?rev=336563&r1=336562&r2=336563&view=diff
==============================================================================
--- openmp/trunk/runtime/src/kmp_lock.h (original)
+++ openmp/trunk/runtime/src/kmp_lock.h Mon Jul  9 10:36:22 2018
@@ -121,7 +121,7 @@ extern void __kmp_validate_locks(void);
 
 struct kmp_base_tas_lock {
   // KMP_LOCK_FREE(tas) => unlocked; locked: (gtid+1) of owning thread
-  volatile kmp_int32 poll;
+  std::atomic<kmp_int32> poll;
   kmp_int32 depth_locked; // depth locked, for nested locks only
 };
 
@@ -139,7 +139,7 @@ typedef union kmp_tas_lock kmp_tas_lock_
 //    kmp_tas_lock_t xlock = KMP_TAS_LOCK_INITIALIZER( xlock );
 #define KMP_TAS_LOCK_INITIALIZER(lock)                                         \
   {                                                                            \
-    { KMP_LOCK_FREE(tas), 0 }                                                  \
+    { ATOMIC_VAR_INIT(KMP_LOCK_FREE(tas)), 0 }                                 \
   }
 
 extern int __kmp_acquire_tas_lock(kmp_tas_lock_t *lck, kmp_int32 gtid);
@@ -446,17 +446,17 @@ struct kmp_base_drdpa_lock {
   volatile union kmp_drdpa_lock
       *initialized; // points to the lock union if in initialized state
   ident_t const *location; // Source code location of omp_init_lock().
-  volatile struct kmp_lock_poll { kmp_uint64 poll; } * volatile polls;
-  volatile kmp_uint64 mask; // is 2**num_polls-1 for mod op
+  std::atomic<std::atomic<kmp_uint64> *> polls;
+  std::atomic<kmp_uint64> mask; // is 2**num_polls-1 for mod op
   kmp_uint64 cleanup_ticket; // thread with cleanup ticket
-  volatile struct kmp_lock_poll *old_polls; // will deallocate old_polls
+  std::atomic<kmp_uint64> *old_polls; // will deallocate old_polls
   kmp_uint32 num_polls; // must be power of 2
 
   // next_ticket it needs to exist in a separate cache line, as it is
   // invalidated every time a thread takes a new ticket.
   KMP_ALIGN_CACHE
 
-  volatile kmp_uint64 next_ticket;
+  std::atomic<kmp_uint64> next_ticket;
 
   // now_serving is used to store our ticket value while we hold the lock. It
   // has a slightly different meaning in the DRDPA ticket locks (where it is
@@ -518,6 +518,8 @@ extern void __kmp_destroy_nested_drdpa_l
 typedef kmp_ticket_lock_t kmp_bootstrap_lock_t;
 
 #define KMP_BOOTSTRAP_LOCK_INITIALIZER(lock) KMP_TICKET_LOCK_INITIALIZER((lock))
+#define KMP_BOOTSTRAP_LOCK_INIT(lock)                                          \
+  kmp_bootstrap_lock_t lock = KMP_TICKET_LOCK_INITIALIZER(lock)
 
 static inline int __kmp_acquire_bootstrap_lock(kmp_bootstrap_lock_t *lck) {
   return __kmp_acquire_ticket_lock(lck, KMP_GTID_DNE);
@@ -549,6 +551,8 @@ static inline void __kmp_destroy_bootstr
 
 typedef kmp_ticket_lock_t kmp_lock_t;
 
+#define KMP_LOCK_INIT(lock) kmp_lock_t lock = KMP_TICKET_LOCK_INITIALIZER(lock)
+
 static inline int __kmp_acquire_lock(kmp_lock_t *lck, kmp_int32 gtid) {
   return __kmp_acquire_ticket_lock(lck, gtid);
 }
@@ -644,8 +648,8 @@ extern int (*__kmp_acquire_user_lock_wit
         KMP_FATAL(LockIsAlreadyOwned, func);                                   \
       }                                                                        \
     }                                                                          \
-    if ((lck->tas.lk.poll != 0) ||                                             \
-        (!KMP_COMPARE_AND_STORE_ACQ32(&(lck->tas.lk.poll), 0, gtid + 1))) {    \
+    if (lck->tas.lk.poll != 0 ||                                               \
+        !__kmp_compare_and_store_acq(&lck->tas.lk.poll, 0, gtid + 1)) {        \
       kmp_uint32 spins;                                                        \
       KMP_FSYNC_PREPARE(lck);                                                  \
       KMP_INIT_YIELD(spins);                                                   \
@@ -655,9 +659,8 @@ extern int (*__kmp_acquire_user_lock_wit
       } else {                                                                 \
         KMP_YIELD_SPIN(spins);                                                 \
       }                                                                        \
-      while (                                                                  \
-          (lck->tas.lk.poll != 0) ||                                           \
-          (!KMP_COMPARE_AND_STORE_ACQ32(&(lck->tas.lk.poll), 0, gtid + 1))) {  \
+      while (lck->tas.lk.poll != 0 ||                                          \
+             !__kmp_compare_and_store_acq(&lck->tas.lk.poll, 0, gtid + 1)) {   \
         if (TCR_4(__kmp_nth) >                                                 \
             (__kmp_avail_proc ? __kmp_avail_proc : __kmp_xproc)) {             \
           KMP_YIELD(TRUE);                                                     \
@@ -699,7 +702,7 @@ static inline int __kmp_test_user_lock_w
       }
     }
     return ((lck->tas.lk.poll == 0) &&
-            KMP_COMPARE_AND_STORE_ACQ32(&(lck->tas.lk.poll), 0, gtid + 1));
+            __kmp_compare_and_store_acq(&lck->tas.lk.poll, 0, gtid + 1));
   } else {
     KMP_DEBUG_ASSERT(__kmp_test_user_lock_with_checks_ != NULL);
     return (*__kmp_test_user_lock_with_checks_)(lck, gtid);
@@ -764,7 +767,7 @@ extern int (*__kmp_acquire_nested_user_l
       *depth = KMP_LOCK_ACQUIRED_NEXT;                                         \
     } else {                                                                   \
       if ((lck->tas.lk.poll != 0) ||                                           \
-          (!KMP_COMPARE_AND_STORE_ACQ32(&(lck->tas.lk.poll), 0, gtid + 1))) {  \
+          !__kmp_compare_and_store_acq(&lck->tas.lk.poll, 0, gtid + 1)) {      \
         kmp_uint32 spins;                                                      \
         KMP_FSYNC_PREPARE(lck);                                                \
         KMP_INIT_YIELD(spins);                                                 \
@@ -775,8 +778,7 @@ extern int (*__kmp_acquire_nested_user_l
           KMP_YIELD_SPIN(spins);                                               \
         }                                                                      \
         while ((lck->tas.lk.poll != 0) ||                                      \
-               (!KMP_COMPARE_AND_STORE_ACQ32(&(lck->tas.lk.poll), 0,           \
-                                             gtid + 1))) {                     \
+               !__kmp_compare_and_store_acq(&lck->tas.lk.poll, 0, gtid + 1)) { \
           if (TCR_4(__kmp_nth) >                                               \
               (__kmp_avail_proc ? __kmp_avail_proc : __kmp_xproc)) {           \
             KMP_YIELD(TRUE);                                                   \
@@ -824,7 +826,7 @@ static inline int __kmp_test_nested_user
       return ++lck->tas.lk.depth_locked; /* same owner, depth increased */
     }
     retval = ((lck->tas.lk.poll == 0) &&
-              KMP_COMPARE_AND_STORE_ACQ32(&(lck->tas.lk.poll), 0, gtid + 1));
+              __kmp_compare_and_store_acq(&lck->tas.lk.poll, 0, gtid + 1));
     if (retval) {
       KMP_MB();
       lck->tas.lk.depth_locked = 1;

Modified: openmp/trunk/runtime/src/kmp_omp.h
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp_omp.h?rev=336563&r1=336562&r2=336563&view=diff
==============================================================================
--- openmp/trunk/runtime/src/kmp_omp.h (original)
+++ openmp/trunk/runtime/src/kmp_omp.h Mon Jul  9 10:36:22 2018
@@ -74,7 +74,9 @@ typedef struct {
   addr_and_size_t threads; // Pointer to __kmp_threads.
   addr_and_size_t roots; // Pointer to __kmp_root.
   addr_and_size_t capacity; // Pointer to __kmp_threads_capacity.
+#if KMP_USE_MONITOR
   addr_and_size_t monitor; // Pointer to __kmp_monitor.
+#endif
 #if !KMP_USE_DYNAMIC_LOCK
   addr_and_size_t lock_table; // Pointer to __kmp_lock_table.
 #endif

Modified: openmp/trunk/runtime/src/kmp_os.h
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp_os.h?rev=336563&r1=336562&r2=336563&view=diff
==============================================================================
--- openmp/trunk/runtime/src/kmp_os.h (original)
+++ openmp/trunk/runtime/src/kmp_os.h Mon Jul  9 10:36:22 2018
@@ -16,6 +16,7 @@
 
 #include "kmp_config.h"
 #include <stdlib.h>
+#include <atomic>
 
 #define KMP_FTN_PLAIN 1
 #define KMP_FTN_APPEND 2
@@ -905,6 +906,45 @@ enum kmp_warnings_level {
 } // extern "C"
 #endif // __cplusplus
 
+// Macros for C++11 atomic functions
+#define KMP_ATOMIC_LD(p, order) (p)->load(std::memory_order_##order)
+#define KMP_ATOMIC_OP(op, p, v, order) (p)->op(v, std::memory_order_##order)
+
+// For non-default load/store
+#define KMP_ATOMIC_LD_ACQ(p) KMP_ATOMIC_LD(p, acquire)
+#define KMP_ATOMIC_LD_RLX(p) KMP_ATOMIC_LD(p, relaxed)
+#define KMP_ATOMIC_ST_REL(p, v) KMP_ATOMIC_OP(store, p, v, release)
+#define KMP_ATOMIC_ST_RLX(p, v) KMP_ATOMIC_OP(store, p, v, relaxed)
+
+// For non-default fetch_<op>
+#define KMP_ATOMIC_ADD(p, v) KMP_ATOMIC_OP(fetch_add, p, v, acq_rel)
+#define KMP_ATOMIC_SUB(p, v) KMP_ATOMIC_OP(fetch_sub, p, v, acq_rel)
+#define KMP_ATOMIC_AND(p, v) KMP_ATOMIC_OP(fetch_and, p, v, acq_rel)
+#define KMP_ATOMIC_OR(p, v) KMP_ATOMIC_OP(fetch_or, p, v, acq_rel)
+#define KMP_ATOMIC_INC(p) KMP_ATOMIC_OP(fetch_add, p, 1, acq_rel)
+#define KMP_ATOMIC_DEC(p) KMP_ATOMIC_OP(fetch_sub, p, 1, acq_rel)
+#define KMP_ATOMIC_ADD_RLX(p, v) KMP_ATOMIC_OP(fetch_add, p, v, relaxed)
+#define KMP_ATOMIC_INC_RLX(p) KMP_ATOMIC_OP(fetch_add, p, 1, relaxed)
+
+// Callers of the following functions cannot see the side effect on "expected".
+template <typename T>
+bool __kmp_atomic_compare_store(std::atomic<T> *p, T expected, T desired) {
+  return p->compare_exchange_strong(
+      expected, desired, std::memory_order_acq_rel, std::memory_order_relaxed);
+}
+
+template <typename T>
+bool __kmp_atomic_compare_store_acq(std::atomic<T> *p, T expected, T desired) {
+  return p->compare_exchange_strong(
+      expected, desired, std::memory_order_acquire, std::memory_order_relaxed);
+}
+
+template <typename T>
+bool __kmp_atomic_compare_store_rel(std::atomic<T> *p, T expected, T desired) {
+  return p->compare_exchange_strong(
+      expected, desired, std::memory_order_release, std::memory_order_relaxed);
+}
+
 #endif /* KMP_OS_H */
 // Safe C API
 #include "kmp_safe_c_api.h"

Modified: openmp/trunk/runtime/src/kmp_runtime.cpp
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp_runtime.cpp?rev=336563&r1=336562&r2=336563&view=diff
==============================================================================
--- openmp/trunk/runtime/src/kmp_runtime.cpp (original)
+++ openmp/trunk/runtime/src/kmp_runtime.cpp Mon Jul  9 10:36:22 2018
@@ -63,7 +63,9 @@ char const __kmp_version_lock[] =
 
 /* ------------------------------------------------------------------------ */
 
+#if KMP_USE_MONITOR
 kmp_info_t __kmp_monitor;
+#endif
 
 /* Forward declarations */
 
@@ -754,8 +756,8 @@ int __kmp_enter_single(int gtid, ident_t
        single block */
     /* TODO: Should this be acquire or release? */
     if (team->t.t_construct == old_this) {
-      status = KMP_COMPARE_AND_STORE_ACQ32(&team->t.t_construct, old_this,
-                                           th->th.th_local.this_construct);
+      status = __kmp_atomic_compare_store_acq(&team->t.t_construct, old_this,
+                                              th->th.th_local.this_construct);
     }
 #if USE_ITT_BUILD
     if (__itt_metadata_add_ptr && __kmp_forkjoin_frames_mode == 3 &&
@@ -1599,7 +1601,7 @@ int __kmp_fork_call(ident_t *loc, int gt
 
       parent_team->t.t_pkfn = microtask;
       parent_team->t.t_invoke = invoker;
-      KMP_TEST_THEN_INC32((kmp_int32 *)&root->r.r_in_parallel);
+      KMP_ATOMIC_INC(&root->r.r_in_parallel);
       parent_team->t.t_active_level++;
       parent_team->t.t_level++;
 
@@ -1956,7 +1958,7 @@ int __kmp_fork_call(ident_t *loc, int gt
 #endif /* OMP_40_ENABLED */
     {
       /* Increment our nested depth level */
-      KMP_TEST_THEN_INC32((kmp_int32 *)&root->r.r_in_parallel);
+      KMP_ATOMIC_INC(&root->r.r_in_parallel);
     }
 
     // See if we need to make a copy of the ICVs.
@@ -2433,7 +2435,7 @@ void __kmp_join_call(ident_t *loc, int g
     /* Decrement our nested depth level */
     team->t.t_level--;
     team->t.t_active_level--;
-    KMP_TEST_THEN_DEC32((kmp_int32 *)&root->r.r_in_parallel);
+    KMP_ATOMIC_DEC(&root->r.r_in_parallel);
 
     /* Restore number of threads in the team if needed */
     if (master_th->th.th_team_nproc < master_th->th.th_teams_size.nth) {
@@ -2491,7 +2493,7 @@ void __kmp_join_call(ident_t *loc, int g
 #endif /* OMP_40_ENABLED */
   {
     /* Decrement our nested depth level */
-    KMP_TEST_THEN_DEC32((kmp_int32 *)&root->r.r_in_parallel);
+    KMP_ATOMIC_DEC(&root->r.r_in_parallel);
   }
   KMP_DEBUG_ASSERT(root->r.r_in_parallel >= 0);
 
@@ -3388,7 +3390,7 @@ void __kmp_print_structure(void) {
                                      root->r.r_uber_thread);
         __kmp_printf("    Active?:      %2d\n", root->r.r_active);
         __kmp_printf("    Nested?:      %2d\n", root->r.r_nested);
-        __kmp_printf("    In Parallel:  %2d\n", root->r.r_in_parallel);
+        __kmp_printf("    In Parallel:  %2d\n", KMP_ATOMIC_LD_RLX(&root->r.r_in_parallel));
         __kmp_printf("\n");
         __kmp_print_structure_team_accum(list, root->r.r_root_team);
         __kmp_print_structure_team_accum(list, root->r.r_hot_team);
@@ -4448,7 +4450,9 @@ static void __kmp_initialize_team(kmp_te
 #ifdef KMP_DEBUG
   team->t.t_copypriv_data = NULL; /* not necessary, but nice for debugging */
 #endif
+#if KMP_OS_WINDOWS
   team->t.t_copyin_counter = 0; /* for barrier-free copyin implementation */
+#endif
 
   team->t.t_control_stack_top = NULL;
 
@@ -5346,7 +5350,9 @@ void __kmp_free_team(kmp_root_t *root,
   /* team is done working */
   TCW_SYNC_PTR(team->t.t_pkfn,
                NULL); // Important for Debugging Support Library.
+#if KMP_OS_WINDOWS
   team->t.t_copyin_counter = 0; // init counter for possible reuse
+#endif
   // Do not reset pointer to parent team to NULL for hot teams.
 
   /* if we are non-hot team, release our threads */
@@ -5783,8 +5789,8 @@ static void __kmp_reap_thread(kmp_info_t
     // so there are no harmful side effects.
     if (thread->th.th_active_in_pool) {
       thread->th.th_active_in_pool = FALSE;
-      KMP_TEST_THEN_DEC32(&__kmp_thread_pool_active_nth);
-      KMP_DEBUG_ASSERT(TCR_4(__kmp_thread_pool_active_nth) >= 0);
+      KMP_ATOMIC_DEC(&__kmp_thread_pool_active_nth);
+      KMP_DEBUG_ASSERT(__kmp_thread_pool_active_nth >= 0);
     }
 
     // Decrement # of [worker] threads in the pool.
@@ -7240,7 +7246,7 @@ static int __kmp_load_balance_nproc(kmp_
   // executing thread (to become the master) are available to add to the new
   // team, but are currently contributing to the system load, and must be
   // accounted for.
-  pool_active = TCR_4(__kmp_thread_pool_active_nth);
+  pool_active = __kmp_thread_pool_active_nth;
   hot_team_active = __kmp_active_hot_team_nproc(root);
   team_curr_active = pool_active + hot_team_active + 1;
 

Modified: openmp/trunk/runtime/src/kmp_taskdeps.cpp
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp_taskdeps.cpp?rev=336563&r1=336562&r2=336563&view=diff
==============================================================================
--- openmp/trunk/runtime/src/kmp_taskdeps.cpp (original)
+++ openmp/trunk/runtime/src/kmp_taskdeps.cpp Mon Jul  9 10:36:22 2018
@@ -32,7 +32,7 @@
 // TODO: Any ITT support needed?
 
 #ifdef KMP_SUPPORT_GRAPH_OUTPUT
-static kmp_int32 kmp_node_id_seed = 0;
+static std::atomic<kmp_int32> kmp_node_id_seed = ATOMIC_VAR_INIT(0);
 #endif
 
 static void __kmp_init_node(kmp_depnode_t *node) {
@@ -40,14 +40,14 @@ static void __kmp_init_node(kmp_depnode_
   // task once dependences have been processed
   node->dn.successors = NULL;
   __kmp_init_lock(&node->dn.lock);
-  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 to the node
 #ifdef KMP_SUPPORT_GRAPH_OUTPUT
-  node->dn.id = KMP_TEST_THEN_INC32(&kmp_node_id_seed);
+  node->dn.id = KMP_ATOMIC_INC(&kmp_node_id_seed);
 #endif
 }
 
 static inline kmp_depnode_t *__kmp_node_ref(kmp_depnode_t *node) {
-  KMP_TEST_THEN_INC32(CCAST(kmp_int32 *, &node->dn.nrefs));
+  KMP_ATOMIC_INC(&node->dn.nrefs);
   return node;
 }
 
@@ -55,7 +55,7 @@ static inline void __kmp_node_deref(kmp_
   if (!node)
     return;
 
-  kmp_int32 n = KMP_TEST_THEN_DEC32(CCAST(kmp_int32 *, &node->dn.nrefs)) - 1;
+  kmp_int32 n = KMP_ATOMIC_DEC(&node->dn.nrefs) - 1;
   if (n == 0) {
     KMP_ASSERT(node->dn.nrefs == 0);
 #if USE_FAST_MEMORY
@@ -375,9 +375,7 @@ static bool __kmp_check_deps(kmp_int32 g
   // any outstandig dependences (some tasks may have finished while we processed
   // the dependences)
   npredecessors =
-      KMP_TEST_THEN_ADD32(CCAST(kmp_int32 *, &node->dn.npredecessors),
-                          npredecessors) +
-      npredecessors;
+      node->dn.npredecessors.fetch_add(npredecessors) + npredecessors;
 
   KA_TRACE(20, ("__kmp_check_deps: T#%d found %d predecessors for task %p \n",
                 gtid, npredecessors, taskdata));
@@ -413,9 +411,8 @@ void __kmp_release_deps(kmp_int32 gtid,
   kmp_depnode_list_t *next;
   for (kmp_depnode_list_t *p = node->dn.successors; p; p = next) {
     kmp_depnode_t *successor = p->node;
-    kmp_int32 npredecessors =
-        KMP_TEST_THEN_DEC32(CCAST(kmp_int32 *, &successor->dn.npredecessors)) -
-        1;
+    kmp_int32 npredecessors = KMP_ATOMIC_DEC(&successor->dn.npredecessors) - 1;
+
     // successor task can be NULL for wait_depends or because deps are still
     // being processed
     if (npredecessors == 0) {
@@ -648,7 +645,7 @@ void __kmpc_omp_wait_deps(ident_t *loc_r
     return;
   }
 
-  kmp_depnode_t node;
+  kmp_depnode_t node = {0};
   __kmp_init_node(&node);
 
   if (!__kmp_check_deps(gtid, &node, NULL, current_task->td_dephash,
@@ -661,12 +658,10 @@ void __kmpc_omp_wait_deps(ident_t *loc_r
   }
 
   int thread_finished = FALSE;
-  kmp_flag_32 flag((volatile kmp_uint32 *)&(node.dn.npredecessors), 0U);
+  kmp_flag_32 flag((std::atomic<kmp_uint32> *)&node.dn.npredecessors, 0U);
   while (node.dn.npredecessors > 0) {
-    flag.execute_tasks(thread, gtid, FALSE, &thread_finished,
-#if USE_ITT_BUILD
-                       NULL,
-#endif
+    flag.execute_tasks(thread, gtid, FALSE,
+                       &thread_finished USE_ITT_BUILD_ARG(NULL),
                        __kmp_task_stealing_constraint);
   }
 

Modified: openmp/trunk/runtime/src/kmp_tasking.cpp
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp_tasking.cpp?rev=336563&r1=336562&r2=336563&view=diff
==============================================================================
--- openmp/trunk/runtime/src/kmp_tasking.cpp (original)
+++ openmp/trunk/runtime/src/kmp_tasking.cpp Mon Jul  9 10:36:22 2018
@@ -264,7 +264,7 @@ static kmp_int32 __kmp_push_task(kmp_int
   if (taskdata->td_flags.tiedness == TASK_UNTIED) {
     // untied task needs to increment counter so that the task structure is not
     // freed prematurely
-    kmp_int32 counter = 1 + KMP_TEST_THEN_INC32(&taskdata->td_untied_count);
+    kmp_int32 counter = 1 + KMP_ATOMIC_INC(&taskdata->td_untied_count);
     KA_TRACE(
         20,
         ("__kmp_push_task: T#%d untied_count (%d) incremented for task %p\n",
@@ -528,7 +528,7 @@ static void __kmpc_omp_task_begin_if0_te
   if (taskdata->td_flags.tiedness == TASK_UNTIED) {
     // untied task needs to increment counter so that the task structure is not
     // freed prematurely
-    kmp_int32 counter = 1 + KMP_TEST_THEN_INC32(&taskdata->td_untied_count);
+    kmp_int32 counter = 1 + KMP_ATOMIC_INC(&taskdata->td_untied_count);
     KA_TRACE(20, ("__kmpc_omp_task_begin_if0: T#%d untied_count (%d) "
                   "incremented for task %p\n",
                   gtid, counter, taskdata));
@@ -625,9 +625,9 @@ static void __kmp_free_task(kmp_int32 gt
   KMP_DEBUG_ASSERT(taskdata->td_flags.executing == 0);
   KMP_DEBUG_ASSERT(taskdata->td_flags.complete == 1);
   KMP_DEBUG_ASSERT(taskdata->td_flags.freed == 0);
-  KMP_DEBUG_ASSERT(TCR_4(taskdata->td_allocated_child_tasks) == 0 ||
+  KMP_DEBUG_ASSERT(taskdata->td_allocated_child_tasks == 0 ||
                    taskdata->td_flags.task_serial == 1);
-  KMP_DEBUG_ASSERT(TCR_4(taskdata->td_incomplete_child_tasks) == 0);
+  KMP_DEBUG_ASSERT(taskdata->td_incomplete_child_tasks == 0);
 
   taskdata->td_flags.freed = 1;
   ANNOTATE_HAPPENS_BEFORE(taskdata);
@@ -662,8 +662,7 @@ static void __kmp_free_task_and_ancestor
 #endif
   KMP_DEBUG_ASSERT(taskdata->td_flags.tasktype == TASK_EXPLICIT);
 
-  kmp_int32 children =
-      KMP_TEST_THEN_DEC32(&taskdata->td_allocated_child_tasks) - 1;
+  kmp_int32 children = KMP_ATOMIC_DEC(&taskdata->td_allocated_child_tasks) - 1;
   KMP_DEBUG_ASSERT(children >= 0);
 
   // Now, go up the ancestor tree to see if any ancestors can now be freed.
@@ -685,7 +684,7 @@ static void __kmp_free_task_and_ancestor
       return;
 
     // Predecrement simulated by "- 1" calculation
-    children = KMP_TEST_THEN_DEC32(&taskdata->td_allocated_child_tasks) - 1;
+    children = KMP_ATOMIC_DEC(&taskdata->td_allocated_child_tasks) - 1;
     KMP_DEBUG_ASSERT(children >= 0);
   }
 
@@ -724,7 +723,7 @@ static void __kmp_task_finish(kmp_int32
   if (taskdata->td_flags.tiedness == TASK_UNTIED) {
     // untied task needs to check the counter so that the task structure is not
     // freed prematurely
-    kmp_int32 counter = KMP_TEST_THEN_DEC32(&taskdata->td_untied_count) - 1;
+    kmp_int32 counter = KMP_ATOMIC_DEC(&taskdata->td_untied_count) - 1;
     KA_TRACE(
         20,
         ("__kmp_task_finish: T#%d untied_count (%d) decremented for task %p\n",
@@ -756,12 +755,11 @@ static void __kmp_task_finish(kmp_int32
   if (!(taskdata->td_flags.team_serial || taskdata->td_flags.tasking_ser)) {
     // Predecrement simulated by "- 1" calculation
     children =
-        KMP_TEST_THEN_DEC32(&taskdata->td_parent->td_incomplete_child_tasks) -
-        1;
+        KMP_ATOMIC_DEC(&taskdata->td_parent->td_incomplete_child_tasks) - 1;
     KMP_DEBUG_ASSERT(children >= 0);
 #if OMP_40_ENABLED
     if (taskdata->td_taskgroup)
-      KMP_TEST_THEN_DEC32((kmp_int32 *)(&taskdata->td_taskgroup->count));
+      KMP_ATOMIC_DEC(&taskdata->td_taskgroup->count);
 #if OMP_45_ENABLED
   }
   // if we found proxy tasks there could exist a dependency chain
@@ -945,9 +943,9 @@ void __kmp_init_implicit_task(ident_t *l
   task->td_last_tied = task;
 
   if (set_curr_task) { // only do this init first time thread is created
-    task->td_incomplete_child_tasks = 0;
+    KMP_ATOMIC_ST_REL(&task->td_incomplete_child_tasks, 0);
     // Not used: don't need to deallocate implicit task
-    task->td_allocated_child_tasks = 0;
+    KMP_ATOMIC_ST_REL(&task->td_allocated_child_tasks, 0);
 #if OMP_40_ENABLED
     task->td_taskgroup = NULL; // An implicit task does not have taskgroup
     task->td_dephash = NULL;
@@ -1130,7 +1128,7 @@ kmp_task_t *__kmp_task_alloc(ident_t *lo
   taskdata->td_alloc_thread = thread;
   taskdata->td_parent = parent_task;
   taskdata->td_level = parent_task->td_level + 1; // increment nesting level
-  taskdata->td_untied_count = 0;
+  KMP_ATOMIC_ST_RLX(&taskdata->td_untied_count, 0);
   taskdata->td_ident = loc_ref;
   taskdata->td_taskwait_ident = NULL;
   taskdata->td_taskwait_counter = 0;
@@ -1176,9 +1174,9 @@ kmp_task_t *__kmp_task_alloc(ident_t *lo
 
   taskdata->td_flags.native = flags->native;
 
-  taskdata->td_incomplete_child_tasks = 0;
-  taskdata->td_allocated_child_tasks = 1; // start at one because counts current
-// task and children
+  KMP_ATOMIC_ST_RLX(&taskdata->td_incomplete_child_tasks, 0);
+  // start at one because counts current task and children
+  KMP_ATOMIC_ST_RLX(&taskdata->td_allocated_child_tasks, 1);
 #if OMP_40_ENABLED
   taskdata->td_taskgroup =
       parent_task->td_taskgroup; // task inherits taskgroup from the parent task
@@ -1199,15 +1197,15 @@ kmp_task_t *__kmp_task_alloc(ident_t *lo
   if (!(taskdata->td_flags.team_serial || taskdata->td_flags.tasking_ser))
 #endif
   {
-    KMP_TEST_THEN_INC32(&parent_task->td_incomplete_child_tasks);
+    KMP_ATOMIC_INC(&parent_task->td_incomplete_child_tasks);
 #if OMP_40_ENABLED
     if (parent_task->td_taskgroup)
-      KMP_TEST_THEN_INC32((kmp_int32 *)(&parent_task->td_taskgroup->count));
+      KMP_ATOMIC_INC(&parent_task->td_taskgroup->count);
 #endif
     // Only need to keep track of allocated child tasks for explicit tasks since
     // implicit not deallocated
     if (taskdata->td_parent->td_flags.tasktype == TASK_EXPLICIT) {
-      KMP_TEST_THEN_INC32(&taskdata->td_parent->td_allocated_child_tasks);
+      KMP_ATOMIC_INC(&taskdata->td_parent->td_allocated_child_tasks);
     }
   }
 
@@ -1648,10 +1646,10 @@ static kmp_int32 __kmpc_omp_taskwait_tem
                               thread->th.th_task_team->tt.tt_found_proxy_tasks);
 #endif
     if (must_wait) {
-      kmp_flag_32 flag(
-          RCAST(volatile kmp_uint32 *, &taskdata->td_incomplete_child_tasks),
-          0U);
-      while (TCR_4(taskdata->td_incomplete_child_tasks) != 0) {
+      kmp_flag_32 flag(RCAST(std::atomic<kmp_uint32> *,
+                             &(taskdata->td_incomplete_child_tasks)),
+                       0U);
+      while (KMP_ATOMIC_LD_ACQ(&taskdata->td_incomplete_child_tasks) != 0) {
         flag.execute_tasks(thread, gtid, FALSE,
                            &thread_finished USE_ITT_BUILD_ARG(itt_sync_obj),
                            __kmp_task_stealing_constraint);
@@ -1982,8 +1980,8 @@ void __kmpc_taskgroup(ident_t *loc, int
   kmp_taskgroup_t *tg_new =
       (kmp_taskgroup_t *)__kmp_thread_malloc(thread, sizeof(kmp_taskgroup_t));
   KA_TRACE(10, ("__kmpc_taskgroup: T#%d loc=%p group=%p\n", gtid, loc, tg_new));
-  tg_new->count = 0;
-  tg_new->cancel_request = cancel_noreq;
+  KMP_ATOMIC_ST_RLX(&tg_new->count, 0);
+  KMP_ATOMIC_ST_RLX(&tg_new->cancel_request, cancel_noreq);
   tg_new->parent = taskdata->td_taskgroup;
 // TODO: change to OMP_50_ENABLED, need to change build tools for this to work
 #if OMP_45_ENABLED
@@ -2066,8 +2064,9 @@ void __kmpc_end_taskgroup(ident_t *loc,
     if (!taskdata->td_flags.team_serial)
 #endif
     {
-      kmp_flag_32 flag(RCAST(kmp_uint32 *, &taskgroup->count), 0U);
-      while (TCR_4(taskgroup->count) != 0) {
+      kmp_flag_32 flag(RCAST(std::atomic<kmp_uint32> *, &(taskgroup->count)),
+                       0U);
+      while (KMP_ATOMIC_LD_ACQ(&taskgroup->count) != 0) {
         flag.execute_tasks(thread, gtid, FALSE,
                            &thread_finished USE_ITT_BUILD_ARG(itt_sync_obj),
                            __kmp_task_stealing_constraint);
@@ -2205,7 +2204,7 @@ static kmp_task_t *__kmp_remove_my_task(
 // task_team thread_data before calling this routine.
 static kmp_task_t *__kmp_steal_task(kmp_info_t *victim_thr, kmp_int32 gtid,
                                     kmp_task_team_t *task_team,
-                                    volatile kmp_int32 *unfinished_threads,
+                                    std::atomic<kmp_int32> *unfinished_threads,
                                     int *thread_finished,
                                     kmp_int32 is_constrained) {
   kmp_task_t *task;
@@ -2343,7 +2342,7 @@ static kmp_task_t *__kmp_steal_task(kmp_
     // master victim) might be prematurely released from the barrier!!!
     kmp_int32 count;
 
-    count = KMP_TEST_THEN_INC32(unfinished_threads);
+    count = KMP_ATOMIC_INC(unfinished_threads);
 
     KA_TRACE(
         20,
@@ -2386,7 +2385,7 @@ static inline int __kmp_execute_tasks_te
   kmp_task_t *task;
   kmp_info_t *other_thread;
   kmp_taskdata_t *current_task = thread->th.th_current_task;
-  volatile kmp_int32 *unfinished_threads;
+  std::atomic<kmp_int32> *unfinished_threads;
   kmp_int32 nthreads, victim_tid = -2, use_own_tasks = 1, new_victim = 0,
                       tid = thread->th.th_info.ds.ds_tid;
 
@@ -2411,7 +2410,7 @@ static inline int __kmp_execute_tasks_te
 #else
   KMP_DEBUG_ASSERT(nthreads > 1);
 #endif
-  KMP_DEBUG_ASSERT(TCR_4(*unfinished_threads) >= 0);
+  KMP_DEBUG_ASSERT(*unfinished_threads >= 0);
 
   while (1) { // Outer loop keeps trying to find tasks in case of single thread
     // getting tasks from target constructs
@@ -2541,7 +2540,8 @@ static inline int __kmp_execute_tasks_te
 #if OMP_45_ENABLED
     // The work queue may be empty but there might be proxy tasks still
     // executing
-    if (final_spin && TCR_4(current_task->td_incomplete_child_tasks) == 0)
+    if (final_spin &&
+        KMP_ATOMIC_LD_ACQ(&current_task->td_incomplete_child_tasks) == 0)
 #else
     if (final_spin)
 #endif
@@ -2552,7 +2552,7 @@ static inline int __kmp_execute_tasks_te
       if (!*thread_finished) {
         kmp_int32 count;
 
-        count = KMP_TEST_THEN_DEC32(unfinished_threads) - 1;
+        count = KMP_ATOMIC_DEC(unfinished_threads) - 1;
         KA_TRACE(20, ("__kmp_execute_tasks_template: T#%d dec "
                       "unfinished_threads to %d task_team=%p\n",
                       gtid, count, task_team));
@@ -2979,13 +2979,13 @@ static kmp_task_team_t *__kmp_allocate_t
 #endif
   task_team->tt.tt_nproc = nthreads = team->t.t_nproc;
 
-  TCW_4(task_team->tt.tt_unfinished_threads, nthreads);
+  KMP_ATOMIC_ST_REL(&task_team->tt.tt_unfinished_threads, nthreads);
   TCW_4(task_team->tt.tt_active, TRUE);
 
   KA_TRACE(20, ("__kmp_allocate_task_team: T#%d exiting; task_team = %p "
                 "unfinished_threads init'd to %d\n",
                 (thread ? __kmp_gtid_from_thread(thread) : -1), task_team,
-                task_team->tt.tt_unfinished_threads));
+                KMP_ATOMIC_LD_RLX(&task_team->tt.tt_unfinished_threads)));
   return task_team;
 }
 
@@ -3143,7 +3143,8 @@ void __kmp_task_team_setup(kmp_info_t *t
 #if OMP_45_ENABLED
         TCW_4(task_team->tt.tt_found_proxy_tasks, FALSE);
 #endif
-        TCW_4(task_team->tt.tt_unfinished_threads, team->t.t_nproc);
+        KMP_ATOMIC_ST_REL(&task_team->tt.tt_unfinished_threads,
+                          team->t.t_nproc);
         TCW_4(task_team->tt.tt_active, TRUE);
       }
       // if team size has changed, the first thread to enable tasking will
@@ -3200,9 +3201,9 @@ void __kmp_task_team_wait(
       // Worker threads may have dropped through to release phase, but could
       // still be executing tasks. Wait here for tasks to complete. To avoid
       // memory contention, only master thread checks termination condition.
-      kmp_flag_32 flag(
-          RCAST(volatile kmp_uint32 *, &task_team->tt.tt_unfinished_threads),
-          0U);
+      kmp_flag_32 flag(RCAST(std::atomic<kmp_uint32> *,
+                             &task_team->tt.tt_unfinished_threads),
+                       0U);
       flag.wait(this_thr, TRUE USE_ITT_BUILD_ARG(itt_sync_obj));
     }
     // Deactivate the old task team, so that the worker threads will stop
@@ -3233,21 +3234,21 @@ void __kmp_task_team_wait(
 // barrier. It is a full barrier itself, which unfortunately turns regular
 // barriers into double barriers and join barriers into 1 1/2 barriers.
 void __kmp_tasking_barrier(kmp_team_t *team, kmp_info_t *thread, int gtid) {
-  volatile kmp_uint32 *spin = RCAST(
-      volatile kmp_uint32 *,
+  std::atomic<kmp_uint32> *spin = RCAST(
+      std::atomic<kmp_uint32> *,
       &team->t.t_task_team[thread->th.th_task_state]->tt.tt_unfinished_threads);
   int flag = FALSE;
   KMP_DEBUG_ASSERT(__kmp_tasking_mode == tskm_extra_barrier);
 
 #if USE_ITT_BUILD
-  KMP_FSYNC_SPIN_INIT(spin, (kmp_uint32 *)NULL);
+  KMP_FSYNC_SPIN_INIT(spin, NULL);
 #endif /* USE_ITT_BUILD */
   kmp_flag_32 spin_flag(spin, 0U);
   while (!spin_flag.execute_tasks(thread, gtid, TRUE,
                                   &flag USE_ITT_BUILD_ARG(NULL), 0)) {
 #if USE_ITT_BUILD
     // TODO: What about itt_sync_obj??
-    KMP_FSYNC_SPIN_PREPARE(CCAST(kmp_uint32 *, spin));
+    KMP_FSYNC_SPIN_PREPARE(RCAST(void *, spin));
 #endif /* USE_ITT_BUILD */
 
     if (TCR_4(__kmp_global.g.g_done)) {
@@ -3258,7 +3259,7 @@ void __kmp_tasking_barrier(kmp_team_t *t
     KMP_YIELD(TRUE); // GH: We always yield here
   }
 #if USE_ITT_BUILD
-  KMP_FSYNC_SPIN_ACQUIRED(CCAST(kmp_uint32 *, spin));
+  KMP_FSYNC_SPIN_ACQUIRED(RCAST(void *, spin));
 #endif /* USE_ITT_BUILD */
 }
 
@@ -3370,11 +3371,11 @@ static void __kmp_first_top_half_finish_
   taskdata->td_flags.complete = 1; // mark the task as completed
 
   if (taskdata->td_taskgroup)
-    KMP_TEST_THEN_DEC32(&taskdata->td_taskgroup->count);
+    KMP_ATOMIC_DEC(&taskdata->td_taskgroup->count);
 
   // Create an imaginary children for this task so the bottom half cannot
   // release the task before we have completed the second top half
-  TCI_4(taskdata->td_incomplete_child_tasks);
+  KMP_ATOMIC_INC(&taskdata->td_incomplete_child_tasks);
 }
 
 static void __kmp_second_top_half_finish_proxy(kmp_taskdata_t *taskdata) {
@@ -3382,11 +3383,11 @@ static void __kmp_second_top_half_finish
 
   // Predecrement simulated by "- 1" calculation
   children =
-      KMP_TEST_THEN_DEC32(&taskdata->td_parent->td_incomplete_child_tasks) - 1;
+      KMP_ATOMIC_DEC(&taskdata->td_parent->td_incomplete_child_tasks) - 1;
   KMP_DEBUG_ASSERT(children >= 0);
 
   // Remove the imaginary children
-  TCD_4(taskdata->td_incomplete_child_tasks);
+  KMP_ATOMIC_DEC(&taskdata->td_incomplete_child_tasks);
 }
 
 static void __kmp_bottom_half_finish_proxy(kmp_int32 gtid, kmp_task_t *ptask) {
@@ -3399,7 +3400,7 @@ static void __kmp_bottom_half_finish_pro
 
   // We need to wait to make sure the top half is finished
   // Spinning here should be ok as this should happen quickly
-  while (TCR_4(taskdata->td_incomplete_child_tasks) > 0)
+  while (KMP_ATOMIC_LD_ACQ(&taskdata->td_incomplete_child_tasks) > 0)
     ;
 
   __kmp_release_deps(gtid, taskdata);
@@ -3534,13 +3535,13 @@ kmp_task_t *__kmp_task_dup_alloc(kmp_inf
   // Only need to keep track of child task counts if team parallel and tasking
   // not serialized
   if (!(taskdata->td_flags.team_serial || taskdata->td_flags.tasking_ser)) {
-    KMP_TEST_THEN_INC32(&parent_task->td_incomplete_child_tasks);
+    KMP_ATOMIC_INC(&parent_task->td_incomplete_child_tasks);
     if (parent_task->td_taskgroup)
-      KMP_TEST_THEN_INC32(&parent_task->td_taskgroup->count);
+      KMP_ATOMIC_INC(&parent_task->td_taskgroup->count);
     // Only need to keep track of allocated child tasks for explicit tasks since
     // implicit not deallocated
     if (taskdata->td_parent->td_flags.tasktype == TASK_EXPLICIT)
-      KMP_TEST_THEN_INC32(&taskdata->td_parent->td_allocated_child_tasks);
+      KMP_ATOMIC_INC(&taskdata->td_parent->td_allocated_child_tasks);
   }
 
   KA_TRACE(20,

Modified: openmp/trunk/runtime/src/kmp_wait_release.h
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp_wait_release.h?rev=336563&r1=336562&r2=336563&view=diff
==============================================================================
--- openmp/trunk/runtime/src/kmp_wait_release.h (original)
+++ openmp/trunk/runtime/src/kmp_wait_release.h Mon Jul  9 10:36:22 2018
@@ -46,26 +46,51 @@ enum flag_type {
 /*!
  * Base class for wait/release volatile flag
  */
+template <typename P> class kmp_flag_native {
+  volatile P *loc;
+  flag_type t;
+
+public:
+  typedef P flag_t;
+  kmp_flag_native(volatile P *p, flag_type ft) : loc(p), t(ft) {}
+  volatile P *get() { return loc; }
+  void set(volatile P *new_loc) { loc = new_loc; }
+  flag_type get_type() { return t; }
+  P load() { return *loc; }
+  void store(P val) { *loc = val; }
+};
+
+/*!
+ * Base class for wait/release atomic flag
+ */
 template <typename P> class kmp_flag {
-  volatile P
+  std::atomic<P>
       *loc; /**< Pointer to the flag storage that is modified by another thread
              */
   flag_type t; /**< "Type" of the flag in loc */
 public:
   typedef P flag_t;
-  kmp_flag(volatile P *p, flag_type ft) : loc(p), t(ft) {}
+  kmp_flag(std::atomic<P> *p, flag_type ft) : loc(p), t(ft) {}
   /*!
    * @result the pointer to the actual flag
    */
-  volatile P *get() { return loc; }
+  std::atomic<P> *get() { return loc; }
   /*!
    * @param new_loc in   set loc to point at new_loc
    */
-  void set(volatile P *new_loc) { loc = new_loc; }
+  void set(std::atomic<P> *new_loc) { loc = new_loc; }
   /*!
    * @result the flag_type
    */
   flag_type get_type() { return t; }
+  /*!
+   * @result flag value
+   */
+  P load() { return loc->load(std::memory_order_acquire); }
+  /*!
+   * @param val the new flag value to be stored
+   */
+  void store(P val) { loc->store(val, std::memory_order_release); }
   // Derived classes must provide the following:
   /*
   kmp_info_t * get_waiter(kmp_uint32 i);
@@ -134,7 +159,7 @@ static inline void
 __kmp_wait_template(kmp_info_t *this_thr, C *flag,
                     int final_spin USE_ITT_BUILD_ARG(void *itt_sync_obj)) {
   // NOTE: We may not belong to a team at this point.
-  volatile typename C::flag_t *spin = flag->get();
+  volatile void *spin = flag->get();
   kmp_uint32 spins;
   kmp_uint32 hibernate;
   int th_gtid;
@@ -147,7 +172,7 @@ __kmp_wait_template(kmp_info_t *this_thr
 
   KMP_FSYNC_SPIN_INIT(spin, NULL);
   if (flag->done_check()) {
-    KMP_FSYNC_SPIN_ACQUIRED(CCAST(typename C::flag_t *, spin));
+    KMP_FSYNC_SPIN_ACQUIRED(spin);
     return;
   }
   th_gtid = this_thr->th.th_info.ds.ds_gtid;
@@ -318,7 +343,7 @@ final_spin=FALSE)
       } // if
     } // if
 
-    KMP_FSYNC_SPIN_PREPARE(CCAST(typename C::flag_t *, spin));
+    KMP_FSYNC_SPIN_PREPARE(spin);
     if (TCR_4(__kmp_global.g.g_done)) {
       if (__kmp_global.g.g_abort)
         __kmp_abort_thread();
@@ -340,7 +365,7 @@ final_spin=FALSE)
     in_pool = !!TCR_4(this_thr->th.th_in_pool);
     if (in_pool != !!this_thr->th.th_active_in_pool) {
       if (in_pool) { // Recently transferred from team to pool
-        KMP_TEST_THEN_INC32(&__kmp_thread_pool_active_nth);
+        KMP_ATOMIC_INC(&__kmp_thread_pool_active_nth);
         this_thr->th.th_active_in_pool = TRUE;
         /* Here, we cannot assert that:
            KMP_DEBUG_ASSERT(TCR_4(__kmp_thread_pool_active_nth) <=
@@ -350,7 +375,7 @@ final_spin=FALSE)
            inc/dec'd asynchronously by the workers. The two can get out of sync
            for brief periods of time.  */
       } else { // Recently transferred from pool to team
-        KMP_TEST_THEN_DEC32(&__kmp_thread_pool_active_nth);
+        KMP_ATOMIC_DEC(&__kmp_thread_pool_active_nth);
         KMP_DEBUG_ASSERT(TCR_4(__kmp_thread_pool_active_nth) >= 0);
         this_thr->th.th_active_in_pool = FALSE;
       }
@@ -425,7 +450,7 @@ final_spin=FALSE)
   }
 #endif
 
-  KMP_FSYNC_SPIN_ACQUIRED(CCAST(typename C::flag_t *, spin));
+  KMP_FSYNC_SPIN_ACQUIRED(spin);
 }
 
 /* Release any threads specified as waiting on the flag by releasing the flag
@@ -438,12 +463,12 @@ template <class C> static inline void __
 #endif
   KF_TRACE(20, ("__kmp_release: T#%d releasing flag(%x)\n", gtid, flag->get()));
   KMP_DEBUG_ASSERT(flag->get());
-  KMP_FSYNC_RELEASING(CCAST(typename C::flag_t *, flag->get()));
+  KMP_FSYNC_RELEASING(flag->get());
 
   flag->internal_release();
 
   KF_TRACE(100, ("__kmp_release: T#%d set new spin=%d\n", gtid, flag->get(),
-                 *(flag->get())));
+                 flag->load()));
 
   if (__kmp_dflt_blocktime != KMP_MAX_BLOCKTIME) {
     // Only need to check sleep stuff if infinite block time not set.
@@ -497,7 +522,9 @@ template <> struct flag_traits<kmp_uint6
   }
 };
 
-template <typename FlagType> class kmp_basic_flag : public kmp_flag<FlagType> {
+// Basic flag that does not use C11 Atomics
+template <typename FlagType>
+class kmp_basic_flag_native : public kmp_flag_native<FlagType> {
   typedef flag_traits<FlagType> traits_type;
   FlagType checker; /**< Value to compare flag to to check if flag has been
                        released. */
@@ -506,14 +533,14 @@ template <typename FlagType> class kmp_b
   kmp_uint32
       num_waiting_threads; /**< Number of threads sleeping on this thread. */
 public:
-  kmp_basic_flag(volatile FlagType *p)
-      : kmp_flag<FlagType>(p, traits_type::t), num_waiting_threads(0) {}
-  kmp_basic_flag(volatile FlagType *p, kmp_info_t *thr)
-      : kmp_flag<FlagType>(p, traits_type::t), num_waiting_threads(1) {
+  kmp_basic_flag_native(volatile FlagType *p)
+      : kmp_flag_native<FlagType>(p, traits_type::t), num_waiting_threads(0) {}
+  kmp_basic_flag_native(volatile FlagType *p, kmp_info_t *thr)
+      : kmp_flag_native<FlagType>(p, traits_type::t), num_waiting_threads(1) {
     waiting_threads[0] = thr;
   }
-  kmp_basic_flag(volatile FlagType *p, FlagType c)
-      : kmp_flag<FlagType>(p, traits_type::t), checker(c),
+  kmp_basic_flag_native(volatile FlagType *p, FlagType c)
+      : kmp_flag_native<FlagType>(p, traits_type::t), checker(c),
         num_waiting_threads(0) {}
   /*!
    * param i in   index into waiting_threads
@@ -594,12 +621,105 @@ public:
   enum barrier_type get_bt() { return bs_last_barrier; }
 };
 
+template <typename FlagType> class kmp_basic_flag : public kmp_flag<FlagType> {
+  typedef flag_traits<FlagType> traits_type;
+  FlagType checker; /**< Value to compare flag to to check if flag has been
+                       released. */
+  kmp_info_t
+      *waiting_threads[1]; /**< Array of threads sleeping on this thread. */
+  kmp_uint32
+      num_waiting_threads; /**< Number of threads sleeping on this thread. */
+public:
+  kmp_basic_flag(std::atomic<FlagType> *p)
+      : kmp_flag<FlagType>(p, traits_type::t), num_waiting_threads(0) {}
+  kmp_basic_flag(std::atomic<FlagType> *p, kmp_info_t *thr)
+      : kmp_flag<FlagType>(p, traits_type::t), num_waiting_threads(1) {
+    waiting_threads[0] = thr;
+  }
+  kmp_basic_flag(std::atomic<FlagType> *p, FlagType c)
+      : kmp_flag<FlagType>(p, traits_type::t), checker(c),
+        num_waiting_threads(0) {}
+  /*!
+   * param i in   index into waiting_threads
+   * @result the thread that is waiting at index i
+   */
+  kmp_info_t *get_waiter(kmp_uint32 i) {
+    KMP_DEBUG_ASSERT(i < num_waiting_threads);
+    return waiting_threads[i];
+  }
+  /*!
+   * @result num_waiting_threads
+   */
+  kmp_uint32 get_num_waiters() { return num_waiting_threads; }
+  /*!
+   * @param thr in   the thread which is now waiting
+   *
+   * Insert a waiting thread at index 0.
+   */
+  void set_waiter(kmp_info_t *thr) {
+    waiting_threads[0] = thr;
+    num_waiting_threads = 1;
+  }
+  /*!
+   * @result true if the flag object has been released.
+   */
+  bool done_check() { return this->load() == checker; }
+  /*!
+   * @param old_loc in   old value of flag
+   * @result true if the flag's old value indicates it was released.
+   */
+  bool done_check_val(FlagType old_loc) { return old_loc == checker; }
+  /*!
+   * @result true if the flag object is not yet released.
+   * Used in __kmp_wait_template like:
+   * @code
+   * while (flag.notdone_check()) { pause(); }
+   * @endcode
+   */
+  bool notdone_check() { return this->load() != checker; }
+  /*!
+   * @result Actual flag value before release was applied.
+   * Trigger all waiting threads to run by modifying flag to release state.
+   */
+  void internal_release() { KMP_ATOMIC_ADD(this->get(), 4); }
+  /*!
+   * @result Actual flag value before sleep bit(s) set.
+   * Notes that there is at least one thread sleeping on the flag by setting
+   * sleep bit(s).
+   */
+  FlagType set_sleeping() {
+    return KMP_ATOMIC_OR(this->get(), KMP_BARRIER_SLEEP_STATE);
+  }
+  /*!
+   * @result Actual flag value before sleep bit(s) cleared.
+   * Notes that there are no longer threads sleeping on the flag by clearing
+   * sleep bit(s).
+   */
+  FlagType unset_sleeping() {
+    return KMP_ATOMIC_AND(this->get(), ~KMP_BARRIER_SLEEP_STATE);
+  }
+  /*!
+   * @param old_loc in   old value of flag
+   * Test whether there are threads sleeping on the flag's old value in old_loc.
+   */
+  bool is_sleeping_val(FlagType old_loc) {
+    return old_loc & KMP_BARRIER_SLEEP_STATE;
+  }
+  /*!
+   * Test whether there are threads sleeping on the flag.
+   */
+  bool is_sleeping() { return is_sleeping_val(this->load()); }
+  bool is_any_sleeping() { return is_sleeping_val(this->load()); }
+  kmp_uint8 *get_stolen() { return NULL; }
+  enum barrier_type get_bt() { return bs_last_barrier; }
+};
+
 class kmp_flag_32 : public kmp_basic_flag<kmp_uint32> {
 public:
-  kmp_flag_32(volatile kmp_uint32 *p) : kmp_basic_flag<kmp_uint32>(p) {}
-  kmp_flag_32(volatile kmp_uint32 *p, kmp_info_t *thr)
+  kmp_flag_32(std::atomic<kmp_uint32> *p) : kmp_basic_flag<kmp_uint32>(p) {}
+  kmp_flag_32(std::atomic<kmp_uint32> *p, kmp_info_t *thr)
       : kmp_basic_flag<kmp_uint32>(p, thr) {}
-  kmp_flag_32(volatile kmp_uint32 *p, kmp_uint32 c)
+  kmp_flag_32(std::atomic<kmp_uint32> *p, kmp_uint32 c)
       : kmp_basic_flag<kmp_uint32>(p, c) {}
   void suspend(int th_gtid) { __kmp_suspend_32(th_gtid, this); }
   void resume(int th_gtid) { __kmp_resume_32(th_gtid, this); }
@@ -619,13 +739,13 @@ public:
   flag_type get_ptr_type() { return flag32; }
 };
 
-class kmp_flag_64 : public kmp_basic_flag<kmp_uint64> {
+class kmp_flag_64 : public kmp_basic_flag_native<kmp_uint64> {
 public:
-  kmp_flag_64(volatile kmp_uint64 *p) : kmp_basic_flag<kmp_uint64>(p) {}
+  kmp_flag_64(volatile kmp_uint64 *p) : kmp_basic_flag_native<kmp_uint64>(p) {}
   kmp_flag_64(volatile kmp_uint64 *p, kmp_info_t *thr)
-      : kmp_basic_flag<kmp_uint64>(p, thr) {}
+      : kmp_basic_flag_native<kmp_uint64>(p, thr) {}
   kmp_flag_64(volatile kmp_uint64 *p, kmp_uint64 c)
-      : kmp_basic_flag<kmp_uint64>(p, c) {}
+      : kmp_basic_flag_native<kmp_uint64>(p, c) {}
   void suspend(int th_gtid) { __kmp_suspend_64(th_gtid, this); }
   void resume(int th_gtid) { __kmp_resume_64(th_gtid, this); }
   int execute_tasks(kmp_info_t *this_thr, kmp_int32 gtid, int final_spin,
@@ -645,7 +765,7 @@ public:
 };
 
 // Hierarchical 64-bit on-core barrier instantiation
-class kmp_flag_oncore : public kmp_flag<kmp_uint64> {
+class kmp_flag_oncore : public kmp_flag_native<kmp_uint64> {
   kmp_uint64 checker;
   kmp_info_t *waiting_threads[1];
   kmp_uint32 num_waiting_threads;
@@ -665,27 +785,17 @@ class kmp_flag_oncore : public kmp_flag<
 
 public:
   kmp_flag_oncore(volatile kmp_uint64 *p)
-      : kmp_flag<kmp_uint64>(p, flag_oncore), num_waiting_threads(0),
+      : kmp_flag_native<kmp_uint64>(p, flag_oncore), num_waiting_threads(0),
         flag_switch(false) {}
   kmp_flag_oncore(volatile kmp_uint64 *p, kmp_uint32 idx)
-      : kmp_flag<kmp_uint64>(p, flag_oncore), num_waiting_threads(0),
+      : kmp_flag_native<kmp_uint64>(p, flag_oncore), num_waiting_threads(0),
         offset(idx), flag_switch(false) {}
   kmp_flag_oncore(volatile kmp_uint64 *p, kmp_uint64 c, kmp_uint32 idx,
-                  enum barrier_type bar_t, kmp_info_t *thr
-#if USE_ITT_BUILD
-                  ,
-                  void *itt
-#endif
-                  )
-      : kmp_flag<kmp_uint64>(p, flag_oncore), checker(c),
+                  enum barrier_type bar_t,
+                  kmp_info_t *thr USE_ITT_BUILD_ARG(void *itt))
+      : kmp_flag_native<kmp_uint64>(p, flag_oncore), checker(c),
         num_waiting_threads(0), offset(idx), flag_switch(false), bt(bar_t),
-        this_thr(thr)
-#if USE_ITT_BUILD
-        ,
-        itt_sync_obj(itt)
-#endif
-  {
-  }
+        this_thr(thr) USE_ITT_BUILD_ARG(itt_sync_obj(itt)) {}
   kmp_info_t *get_waiter(kmp_uint32 i) {
     KMP_DEBUG_ASSERT(i < num_waiting_threads);
     return waiting_threads[i];
@@ -709,16 +819,12 @@ public:
       this_thr->th.th_bar[bt].bb.wait_flag = KMP_BARRIER_SWITCHING;
       kmp_flag_64 flag(&this_thr->th.th_bar[bt].bb.b_go,
                        (kmp_uint64)KMP_BARRIER_STATE_BUMP);
-      __kmp_wait_64(this_thr, &flag, TRUE
-#if USE_ITT_BUILD
-                    ,
-                    itt_sync_obj
-#endif
-                    );
+      __kmp_wait_64(this_thr, &flag, TRUE USE_ITT_BUILD_ARG(itt_sync_obj));
     }
     return false;
   }
   void internal_release() {
+    // Other threads can write their own bytes simultaneously.
     if (__kmp_dflt_blocktime == KMP_MAX_BLOCKTIME) {
       byteref(get(), offset) = 1;
     } else {

Modified: openmp/trunk/runtime/src/z_Linux_util.cpp
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/z_Linux_util.cpp?rev=336563&r1=336562&r2=336563&view=diff
==============================================================================
--- openmp/trunk/runtime/src/z_Linux_util.cpp (original)
+++ openmp/trunk/runtime/src/z_Linux_util.cpp Mon Jul  9 10:36:22 2018
@@ -1434,7 +1434,7 @@ static inline void __kmp_suspend_templat
 
   KF_TRACE(5, ("__kmp_suspend_template: T#%d set sleep bit for spin(%p)==%x,"
                " was %x\n",
-               th_gtid, flag->get(), *(flag->get()), old_spin));
+               th_gtid, flag->get(), flag->load(), old_spin));
 
   if (flag->done_check_val(old_spin)) {
     old_spin = flag->unset_sleeping();
@@ -1462,7 +1462,7 @@ static inline void __kmp_suspend_templat
         th->th.th_active = FALSE;
         if (th->th.th_active_in_pool) {
           th->th.th_active_in_pool = FALSE;
-          KMP_TEST_THEN_DEC32(&__kmp_thread_pool_active_nth);
+          KMP_ATOMIC_DEC(&__kmp_thread_pool_active_nth);
           KMP_DEBUG_ASSERT(TCR_4(__kmp_thread_pool_active_nth) >= 0);
         }
         deactivated = TRUE;
@@ -1518,7 +1518,7 @@ static inline void __kmp_suspend_templat
     if (deactivated) {
       th->th.th_active = TRUE;
       if (TCR_4(th->th.th_in_pool)) {
-        KMP_TEST_THEN_INC32(&__kmp_thread_pool_active_nth);
+        KMP_ATOMIC_INC(&__kmp_thread_pool_active_nth);
         th->th.th_active_in_pool = TRUE;
       }
     }
@@ -1591,7 +1591,7 @@ static inline void __kmp_resume_template
       KF_TRACE(5, ("__kmp_resume_template: T#%d exiting, thread T#%d already "
                    "awake: flag(%p): "
                    "%u => %u\n",
-                   gtid, target_gtid, flag->get(), old_spin, *flag->get()));
+                   gtid, target_gtid, flag->get(), old_spin, flag->load()));
       status = pthread_mutex_unlock(&th->th.th_suspend_mx.m_mutex);
       KMP_CHECK_SYSFAIL("pthread_mutex_unlock", status);
       return;
@@ -1599,7 +1599,7 @@ static inline void __kmp_resume_template
     KF_TRACE(5, ("__kmp_resume_template: T#%d about to wakeup T#%d, reset "
                  "sleep bit for flag's loc(%p): "
                  "%u => %u\n",
-                 gtid, target_gtid, flag->get(), old_spin, *flag->get()));
+                 gtid, target_gtid, flag->get(), old_spin, flag->load()));
   }
   TCW_PTR(th->th.th_sleep_loc, NULL);
 

Modified: openmp/trunk/runtime/src/z_Windows_NT_util.cpp
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/z_Windows_NT_util.cpp?rev=336563&r1=336562&r2=336563&view=diff
==============================================================================
--- openmp/trunk/runtime/src/z_Windows_NT_util.cpp (original)
+++ openmp/trunk/runtime/src/z_Windows_NT_util.cpp Mon Jul  9 10:36:22 2018
@@ -350,7 +350,7 @@ static inline void __kmp_suspend_templat
         th->th.th_active = FALSE;
         if (th->th.th_active_in_pool) {
           th->th.th_active_in_pool = FALSE;
-          KMP_TEST_THEN_DEC32((kmp_int32 *)&__kmp_thread_pool_active_nth);
+          KMP_ATOMIC_DEC(&__kmp_thread_pool_active_nth);
           KMP_DEBUG_ASSERT(TCR_4(__kmp_thread_pool_active_nth) >= 0);
         }
         deactivated = TRUE;
@@ -375,7 +375,7 @@ static inline void __kmp_suspend_templat
     if (deactivated) {
       th->th.th_active = TRUE;
       if (TCR_4(th->th.th_in_pool)) {
-        KMP_TEST_THEN_INC32((kmp_int32 *)&__kmp_thread_pool_active_nth);
+        KMP_ATOMIC_INC(&__kmp_thread_pool_active_nth);
         th->th.th_active_in_pool = TRUE;
       }
     }




More information about the Openmp-commits mailing list