[Openmp-commits] [openmp] r336568 - [OpenMP] Restructure loop code for hierarchical scheduling

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


Author: jlpeyton
Date: Mon Jul  9 10:45:33 2018
New Revision: 336568

URL: http://llvm.org/viewvc/llvm-project?rev=336568&view=rev
Log:
[OpenMP] Restructure loop code for hierarchical scheduling

This patch reorganizes the loop scheduling code in order to allow hierarchical
scheduling to use it more effectively. In particular, the goal of this patch
is to separate the algorithmic parts of the scheduling from the thread
logistics code.

Moves declarations & structures to kmp_dispatch.h for easier access in
other files.  Extracts the algorithmic part of __kmp_dispatch_init() and
__kmp_dispatch_next() into __kmp_dispatch_init_algorithm() and
__kmp_dispatch_next_algorithm(). The thread bookkeeping logic is still kept in
__kmp_dispatch_init() and __kmp_dispatch_next(). This is done because the
hierarchical scheduler needs to access the scheduling logic without the
bookkeeping logic.  To prepare for new pointer in dispatch_private_info_t, a
new flags variable is created which stores the ordered and nomerge flags instead
of them being in two separate variables. This will keep the
dispatch_private_info_t structure the same size.

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

Added:
    openmp/trunk/runtime/src/kmp_dispatch.h
Modified:
    openmp/trunk/runtime/src/kmp.h
    openmp/trunk/runtime/src/kmp_dispatch.cpp

Modified: openmp/trunk/runtime/src/kmp.h
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp.h?rev=336568&r1=336567&r2=336568&view=diff
==============================================================================
--- openmp/trunk/runtime/src/kmp.h (original)
+++ openmp/trunk/runtime/src/kmp.h Mon Jul  9 10:45:33 2018
@@ -1508,6 +1508,15 @@ struct shared_table {
 
 /* ------------------------------------------------------------------------ */
 
+typedef struct kmp_sched_flags {
+  unsigned ordered : 1;
+  unsigned nomerge : 1;
+  unsigned contains_last : 1;
+  unsigned unused : 29;
+} kmp_sched_flags_t;
+
+KMP_BUILD_ASSERT(sizeof(kmp_sched_flags_t) == 4);
+
 #if KMP_STATIC_STEAL_ENABLED
 typedef struct KMP_ALIGN_CACHE dispatch_private_info32 {
   kmp_int32 count;
@@ -1625,13 +1634,12 @@ typedef struct KMP_ALIGN_CACHE dispatch_
     dispatch_private_info64_t p64;
   } u;
   enum sched_type schedule; /* scheduling algorithm */
-  kmp_int32 ordered; /* ordered clause specified */
+  kmp_sched_flags_t flags; /* flags (e.g., ordered, nomerge, etc.) */
   kmp_int32 ordered_bumped;
   // To retain the structure size after making ordered_iteration scalar
   kmp_int32 ordered_dummy[KMP_MAX_ORDERED - 3];
   // Stack of buffers for nest of serial regions
   struct dispatch_private_info *next;
-  kmp_int32 nomerge; /* don't merge iters if serialized */
   kmp_int32 type_size; /* the size of types in private_info */
   enum cons_type pushed_ws;
 } dispatch_private_info_t;

Modified: openmp/trunk/runtime/src/kmp_dispatch.cpp
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp_dispatch.cpp?rev=336568&r1=336567&r2=336568&view=diff
==============================================================================
--- openmp/trunk/runtime/src/kmp_dispatch.cpp (original)
+++ openmp/trunk/runtime/src/kmp_dispatch.cpp Mon Jul  9 10:45:33 2018
@@ -34,275 +34,17 @@
 #if KMP_OS_WINDOWS && KMP_ARCH_X86
 #include <float.h>
 #endif
+#include "kmp_lock.h"
+#include "kmp_dispatch.h"
 
 #if OMPT_SUPPORT
 #include "ompt-specific.h"
 #endif
 
 /* ------------------------------------------------------------------------ */
-
-#if KMP_STATIC_STEAL_ENABLED
-
-// replaces dispatch_private_info{32,64} structures and
-// dispatch_private_info{32,64}_t types
-template <typename T> struct dispatch_private_infoXX_template {
-  typedef typename traits_t<T>::unsigned_t UT;
-  typedef typename traits_t<T>::signed_t ST;
-  UT count; // unsigned
-  T ub;
-  /* Adding KMP_ALIGN_CACHE here doesn't help / can hurt performance */
-  T lb;
-  ST st; // signed
-  UT tc; // unsigned
-  T static_steal_counter; // for static_steal only; maybe better to put after ub
-
-  /* parm[1-4] are used in different ways by different scheduling algorithms */
-
-  // KMP_ALIGN( 32 ) ensures ( if the KMP_ALIGN macro is turned on )
-  //    a) parm3 is properly aligned and
-  //    b) all parm1-4 are in the same cache line.
-  // Because of parm1-4 are used together, performance seems to be better
-  // if they are in the same line (not measured though).
-
-  struct KMP_ALIGN(32) { // compiler does not accept sizeof(T)*4
-    T parm1;
-    T parm2;
-    T parm3;
-    T parm4;
-  };
-
-  UT ordered_lower; // unsigned
-  UT ordered_upper; // unsigned
-#if KMP_OS_WINDOWS
-  T last_upper;
-#endif /* KMP_OS_WINDOWS */
-};
-
-#else /* KMP_STATIC_STEAL_ENABLED */
-
-// replaces dispatch_private_info{32,64} structures and
-// dispatch_private_info{32,64}_t types
-template <typename T> struct dispatch_private_infoXX_template {
-  typedef typename traits_t<T>::unsigned_t UT;
-  typedef typename traits_t<T>::signed_t ST;
-  T lb;
-  T ub;
-  ST st; // signed
-  UT tc; // unsigned
-
-  T parm1;
-  T parm2;
-  T parm3;
-  T parm4;
-
-  UT count; // unsigned
-
-  UT ordered_lower; // unsigned
-  UT ordered_upper; // unsigned
-#if KMP_OS_WINDOWS
-  T last_upper;
-#endif /* KMP_OS_WINDOWS */
-};
-
-#endif /* KMP_STATIC_STEAL_ENABLED */
-
-// replaces dispatch_private_info structure and dispatch_private_info_t type
-template <typename T> struct KMP_ALIGN_CACHE dispatch_private_info_template {
-  // duplicate alignment here, otherwise size of structure is not correct in our
-  // compiler
-  union KMP_ALIGN_CACHE private_info_tmpl {
-    dispatch_private_infoXX_template<T> p;
-    dispatch_private_info64_t p64;
-  } u;
-  enum sched_type schedule; /* scheduling algorithm */
-  kmp_uint32 ordered; /* ordered clause specified */
-  kmp_uint32 ordered_bumped;
-  // To retain the structure size after making ordered_iteration scalar
-  kmp_int32 ordered_dummy[KMP_MAX_ORDERED - 3];
-  dispatch_private_info *next; /* stack of buffers for nest of serial regions */
-  kmp_uint32 nomerge; /* don't merge iters if serialized */
-  kmp_uint32 type_size;
-  enum cons_type pushed_ws;
-};
-
-// replaces dispatch_shared_info{32,64} structures and
-// dispatch_shared_info{32,64}_t types
-template <typename UT> struct dispatch_shared_infoXX_template {
-  /* chunk index under dynamic, number of idle threads under static-steal;
-     iteration index otherwise */
-  volatile UT iteration;
-  volatile UT num_done;
-  volatile UT ordered_iteration;
-  // to retain the structure size making ordered_iteration scalar
-  UT ordered_dummy[KMP_MAX_ORDERED - 3];
-};
-
-// replaces dispatch_shared_info structure and dispatch_shared_info_t type
-template <typename UT> struct dispatch_shared_info_template {
-  // we need union here to keep the structure size
-  union shared_info_tmpl {
-    dispatch_shared_infoXX_template<UT> s;
-    dispatch_shared_info64_t s64;
-  } u;
-  volatile kmp_uint32 buffer_index;
-#if OMP_45_ENABLED
-  volatile kmp_int32 doacross_buf_idx; // teamwise index
-  kmp_uint32 *doacross_flags; // array of iteration flags (0/1)
-  kmp_int32 doacross_num_done; // count finished threads
-#endif
-#if KMP_USE_HWLOC
-  // When linking with libhwloc, the ORDERED EPCC test slowsdown on big
-  // machines (> 48 cores). Performance analysis showed that a cache thrash
-  // was occurring and this padding helps alleviate the problem.
-  char padding[64];
-#endif
-};
-
-/* ------------------------------------------------------------------------ */
-
-#undef USE_TEST_LOCKS
-
-// test_then_add template (general template should NOT be used)
-template <typename T> static __forceinline T test_then_add(volatile T *p, T d);
-
-template <>
-__forceinline kmp_int32 test_then_add<kmp_int32>(volatile kmp_int32 *p,
-                                                 kmp_int32 d) {
-  kmp_int32 r;
-  r = KMP_TEST_THEN_ADD32(p, d);
-  return r;
-}
-
-template <>
-__forceinline kmp_int64 test_then_add<kmp_int64>(volatile kmp_int64 *p,
-                                                 kmp_int64 d) {
-  kmp_int64 r;
-  r = KMP_TEST_THEN_ADD64(p, d);
-  return r;
-}
-
-// test_then_inc_acq template (general template should NOT be used)
-template <typename T> static __forceinline T test_then_inc_acq(volatile T *p);
-
-template <>
-__forceinline kmp_int32 test_then_inc_acq<kmp_int32>(volatile kmp_int32 *p) {
-  kmp_int32 r;
-  r = KMP_TEST_THEN_INC_ACQ32(p);
-  return r;
-}
-
-template <>
-__forceinline kmp_int64 test_then_inc_acq<kmp_int64>(volatile kmp_int64 *p) {
-  kmp_int64 r;
-  r = KMP_TEST_THEN_INC_ACQ64(p);
-  return r;
-}
-
-// test_then_inc template (general template should NOT be used)
-template <typename T> static __forceinline T test_then_inc(volatile T *p);
-
-template <>
-__forceinline kmp_int32 test_then_inc<kmp_int32>(volatile kmp_int32 *p) {
-  kmp_int32 r;
-  r = KMP_TEST_THEN_INC32(p);
-  return r;
-}
-
-template <>
-__forceinline kmp_int64 test_then_inc<kmp_int64>(volatile kmp_int64 *p) {
-  kmp_int64 r;
-  r = KMP_TEST_THEN_INC64(p);
-  return r;
-}
-
-// compare_and_swap template (general template should NOT be used)
-template <typename T>
-static __forceinline kmp_int32 compare_and_swap(volatile T *p, T c, T s);
-
-template <>
-__forceinline kmp_int32 compare_and_swap<kmp_int32>(volatile kmp_int32 *p,
-                                                    kmp_int32 c, kmp_int32 s) {
-  return KMP_COMPARE_AND_STORE_REL32(p, c, s);
-}
-
-template <>
-__forceinline kmp_int32 compare_and_swap<kmp_int64>(volatile kmp_int64 *p,
-                                                    kmp_int64 c, kmp_int64 s) {
-  return KMP_COMPARE_AND_STORE_REL64(p, c, s);
-}
-
-/* Spin wait loop that first does pause, then yield.
-    Waits until function returns non-zero when called with *spinner and check.
-    Does NOT put threads to sleep.
-    Arguments:
-        UT is unsigned 4- or 8-byte type
-        spinner - memory location to check value
-        checker - value which spinner is >, <, ==, etc.
-        pred - predicate function to perform binary comparison of some sort
-#if USE_ITT_BUILD
-        obj -- is higher-level synchronization object to report to ittnotify.
-        It is used to report locks consistently. For example, if lock is
-        acquired immediately, its address is reported to ittnotify via
-        KMP_FSYNC_ACQUIRED(). However, it lock cannot be acquired immediately
-        and lock routine calls to KMP_WAIT_YIELD(), the later should report the
-        same address, not an address of low-level spinner.
-#endif // USE_ITT_BUILD
-    TODO: make inline function (move to header file for icl)
-*/
-template <typename UT>
-static UT __kmp_wait_yield(volatile UT *spinner, UT checker,
-                           kmp_uint32 (*pred)(UT, UT)
-                               USE_ITT_BUILD_ARG(void *obj)) {
-  // note: we may not belong to a team at this point
-  volatile UT *spin = spinner;
-  UT check = checker;
-  kmp_uint32 spins;
-  kmp_uint32 (*f)(UT, UT) = pred;
-  UT r;
-
-  KMP_FSYNC_SPIN_INIT(obj, CCAST(UT *, spin));
-  KMP_INIT_YIELD(spins);
-  // main wait spin loop
-  while (!f(r = *spin, check)) {
-    KMP_FSYNC_SPIN_PREPARE(obj);
-    /* GEH - remove this since it was accidentally introduced when kmp_wait was
-       split. It causes problems with infinite recursion because of exit lock */
-    /* if ( TCR_4(__kmp_global.g.g_done) && __kmp_global.g.g_abort)
-        __kmp_abort_thread(); */
-
-    // if we are oversubscribed, or have waited a bit (and
-    // KMP_LIBRARY=throughput, then yield. pause is in the following code
-    KMP_YIELD(TCR_4(__kmp_nth) > __kmp_avail_proc);
-    KMP_YIELD_SPIN(spins);
-  }
-  KMP_FSYNC_SPIN_ACQUIRED(obj);
-  return r;
-}
-
-template <typename UT> static kmp_uint32 __kmp_eq(UT value, UT checker) {
-  return value == checker;
-}
-
-template <typename UT> static kmp_uint32 __kmp_neq(UT value, UT checker) {
-  return value != checker;
-}
-
-template <typename UT> static kmp_uint32 __kmp_lt(UT value, UT checker) {
-  return value < checker;
-}
-
-template <typename UT> static kmp_uint32 __kmp_ge(UT value, UT checker) {
-  return value >= checker;
-}
-
-template <typename UT> static kmp_uint32 __kmp_le(UT value, UT checker) {
-  return value <= checker;
-}
-
 /* ------------------------------------------------------------------------ */
 
-static void __kmp_dispatch_deo_error(int *gtid_ref, int *cid_ref,
-                                     ident_t *loc_ref) {
+void __kmp_dispatch_deo_error(int *gtid_ref, int *cid_ref, ident_t *loc_ref) {
   kmp_info_t *th;
 
   KMP_DEBUG_ASSERT(gtid_ref);
@@ -320,85 +62,7 @@ static void __kmp_dispatch_deo_error(int
   }
 }
 
-template <typename UT>
-static void __kmp_dispatch_deo(int *gtid_ref, int *cid_ref, ident_t *loc_ref) {
-  typedef typename traits_t<UT>::signed_t ST;
-  dispatch_private_info_template<UT> *pr;
-
-  int gtid = *gtid_ref;
-  //    int  cid = *cid_ref;
-  kmp_info_t *th = __kmp_threads[gtid];
-  KMP_DEBUG_ASSERT(th->th.th_dispatch);
-
-  KD_TRACE(100, ("__kmp_dispatch_deo: T#%d called\n", gtid));
-  if (__kmp_env_consistency_check) {
-    pr = reinterpret_cast<dispatch_private_info_template<UT> *>(
-        th->th.th_dispatch->th_dispatch_pr_current);
-    if (pr->pushed_ws != ct_none) {
-#if KMP_USE_DYNAMIC_LOCK
-      __kmp_push_sync(gtid, ct_ordered_in_pdo, loc_ref, NULL, 0);
-#else
-      __kmp_push_sync(gtid, ct_ordered_in_pdo, loc_ref, NULL);
-#endif
-    }
-  }
-
-  if (!th->th.th_team->t.t_serialized) {
-    dispatch_shared_info_template<UT> *sh =
-        reinterpret_cast<dispatch_shared_info_template<UT> *>(
-            th->th.th_dispatch->th_dispatch_sh_current);
-    UT lower;
-
-    if (!__kmp_env_consistency_check) {
-      pr = reinterpret_cast<dispatch_private_info_template<UT> *>(
-          th->th.th_dispatch->th_dispatch_pr_current);
-    }
-    lower = pr->u.p.ordered_lower;
-
-#if !defined(KMP_GOMP_COMPAT)
-    if (__kmp_env_consistency_check) {
-      if (pr->ordered_bumped) {
-        struct cons_header *p = __kmp_threads[gtid]->th.th_cons;
-        __kmp_error_construct2(kmp_i18n_msg_CnsMultipleNesting,
-                               ct_ordered_in_pdo, loc_ref,
-                               &p->stack_data[p->w_top]);
-      }
-    }
-#endif /* !defined(KMP_GOMP_COMPAT) */
-
-    KMP_MB();
-#ifdef KMP_DEBUG
-    {
-      char *buff;
-      // create format specifiers before the debug output
-      buff = __kmp_str_format("__kmp_dispatch_deo: T#%%d before wait: "
-                              "ordered_iter:%%%s lower:%%%s\n",
-                              traits_t<UT>::spec, traits_t<UT>::spec);
-      KD_TRACE(1000, (buff, gtid, sh->u.s.ordered_iteration, lower));
-      __kmp_str_free(&buff);
-    }
-#endif
-
-    __kmp_wait_yield<UT>(&sh->u.s.ordered_iteration, lower,
-                         __kmp_ge<UT> USE_ITT_BUILD_ARG(NULL));
-    KMP_MB(); /* is this necessary? */
-#ifdef KMP_DEBUG
-    {
-      char *buff;
-      // create format specifiers before the debug output
-      buff = __kmp_str_format("__kmp_dispatch_deo: T#%%d after wait: "
-                              "ordered_iter:%%%s lower:%%%s\n",
-                              traits_t<UT>::spec, traits_t<UT>::spec);
-      KD_TRACE(1000, (buff, gtid, sh->u.s.ordered_iteration, lower));
-      __kmp_str_free(&buff);
-    }
-#endif
-  }
-  KD_TRACE(100, ("__kmp_dispatch_deo: T#%d returned\n", gtid));
-}
-
-static void __kmp_dispatch_dxo_error(int *gtid_ref, int *cid_ref,
-                                     ident_t *loc_ref) {
+void __kmp_dispatch_dxo_error(int *gtid_ref, int *cid_ref, ident_t *loc_ref) {
   kmp_info_t *th;
 
   if (__kmp_env_consistency_check) {
@@ -409,121 +73,26 @@ static void __kmp_dispatch_dxo_error(int
   }
 }
 
-template <typename UT>
-static void __kmp_dispatch_dxo(int *gtid_ref, int *cid_ref, ident_t *loc_ref) {
-  typedef typename traits_t<UT>::signed_t ST;
-  dispatch_private_info_template<UT> *pr;
-
-  int gtid = *gtid_ref;
-  //    int  cid = *cid_ref;
-  kmp_info_t *th = __kmp_threads[gtid];
-  KMP_DEBUG_ASSERT(th->th.th_dispatch);
-
-  KD_TRACE(100, ("__kmp_dispatch_dxo: T#%d called\n", gtid));
-  if (__kmp_env_consistency_check) {
-    pr = reinterpret_cast<dispatch_private_info_template<UT> *>(
-        th->th.th_dispatch->th_dispatch_pr_current);
-    if (pr->pushed_ws != ct_none) {
-      __kmp_pop_sync(gtid, ct_ordered_in_pdo, loc_ref);
-    }
-  }
-
-  if (!th->th.th_team->t.t_serialized) {
-    dispatch_shared_info_template<UT> *sh =
-        reinterpret_cast<dispatch_shared_info_template<UT> *>(
-            th->th.th_dispatch->th_dispatch_sh_current);
-
-    if (!__kmp_env_consistency_check) {
-      pr = reinterpret_cast<dispatch_private_info_template<UT> *>(
-          th->th.th_dispatch->th_dispatch_pr_current);
-    }
-
-    KMP_FSYNC_RELEASING(CCAST(UT *, &sh->u.s.ordered_iteration));
-#if !defined(KMP_GOMP_COMPAT)
-    if (__kmp_env_consistency_check) {
-      if (pr->ordered_bumped != 0) {
-        struct cons_header *p = __kmp_threads[gtid]->th.th_cons;
-        /* How to test it? - OM */
-        __kmp_error_construct2(kmp_i18n_msg_CnsMultipleNesting,
-                               ct_ordered_in_pdo, loc_ref,
-                               &p->stack_data[p->w_top]);
-      }
-    }
-#endif /* !defined(KMP_GOMP_COMPAT) */
-
-    KMP_MB(); /* Flush all pending memory write invalidates.  */
-
-    pr->ordered_bumped += 1;
-
-    KD_TRACE(1000,
-             ("__kmp_dispatch_dxo: T#%d bumping ordered ordered_bumped=%d\n",
-              gtid, pr->ordered_bumped));
-
-    KMP_MB(); /* Flush all pending memory write invalidates.  */
-
-    /* TODO use general release procedure? */
-    test_then_inc<ST>((volatile ST *)&sh->u.s.ordered_iteration);
-
-    KMP_MB(); /* Flush all pending memory write invalidates.  */
-  }
-  KD_TRACE(100, ("__kmp_dispatch_dxo: T#%d returned\n", gtid));
-}
-
-// Computes and returns x to the power of y, where y must a non-negative integer
-template <typename UT>
-static __forceinline long double __kmp_pow(long double x, UT y) {
-  long double s = 1.0L;
-
-  KMP_DEBUG_ASSERT(x > 0.0 && x < 1.0);
-  // KMP_DEBUG_ASSERT(y >= 0); // y is unsigned
-  while (y) {
-    if (y & 1)
-      s *= x;
-    x *= x;
-    y >>= 1;
-  }
-  return s;
-}
-
-/* Computes and returns the number of unassigned iterations after idx chunks
-   have been assigned (the total number of unassigned iterations in chunks with
-   index greater than or equal to idx). __forceinline seems to be broken so that
-   if we __forceinline this function, the behavior is wrong
-   (one of the unit tests, sch_guided_analytical_basic.cpp, fails) */
-template <typename T>
-static __inline typename traits_t<T>::unsigned_t
-__kmp_dispatch_guided_remaining(T tc, typename traits_t<T>::floating_t base,
-                                typename traits_t<T>::unsigned_t idx) {
-  /* Note: On Windows* OS on IA-32 architecture and Intel(R) 64, at least for
-     ICL 8.1, long double arithmetic may not really have long double precision,
-     even with /Qlong_double.  Currently, we workaround that in the caller code,
-     by manipulating the FPCW for Windows* OS on IA-32 architecture.  The lack
-     of precision is not expected to be a correctness issue, though. */
-  typedef typename traits_t<T>::unsigned_t UT;
-
-  long double x = tc * __kmp_pow<UT>(base, idx);
-  UT r = (UT)x;
-  if (x == r)
-    return r;
-  return r + 1;
-}
-
-// Parameters of the guided-iterative algorithm:
-//   p2 = n * nproc * ( chunk + 1 )  // point of switching to dynamic
-//   p3 = 1 / ( n * nproc )          // remaining iterations multiplier
-// by default n = 2. For example with n = 3 the chunks distribution will be more
-// flat.
-// With n = 1 first chunk is the same as for static schedule, e.g. trip / nproc.
-static int guided_int_param = 2;
-static double guided_flt_param = 0.5; // = 1.0 / guided_int_param;
-
-// UT - unsigned flavor of T, ST - signed flavor of T,
-// DBL - double if sizeof(T)==4, or long double if sizeof(T)==8
+// Initialize a dispatch_private_info_template<T> buffer for a particular
+// type of schedule,chunk.  The loop description is found in lb (lower bound),
+// ub (upper bound), and st (stride).  nproc is the number of threads relevant
+// to the scheduling (often the number of threads in a team, but not always if
+// hierarchical scheduling is used).  tid is the id of the thread calling
+// the function within the group of nproc threads.  It will have a value
+// between 0 and nproc - 1.  This is often just the thread id within a team, but
+// is not necessarily the case when using hierarchical scheduling.
+// loc is the source file location of the corresponding loop
+// gtid is the global thread id
 template <typename T>
-static void
-__kmp_dispatch_init(ident_t *loc, int gtid, enum sched_type schedule, T lb,
-                    T ub, typename traits_t<T>::signed_t st,
-                    typename traits_t<T>::signed_t chunk, int push_ws) {
+void __kmp_dispatch_init_algorithm(ident_t *loc, int gtid,
+                                   dispatch_private_info_template<T> *pr,
+                                   enum sched_type schedule, T lb, T ub,
+                                   typename traits_t<T>::signed_t st,
+#if USE_ITT_BUILD
+                                   kmp_uint64 *cur_chunk,
+#endif
+                                   typename traits_t<T>::signed_t chunk,
+                                   T nproc, T tid) {
   typedef typename traits_t<T>::unsigned_t UT;
   typedef typename traits_t<T>::signed_t ST;
   typedef typename traits_t<T>::floating_t DBL;
@@ -532,30 +101,18 @@ __kmp_dispatch_init(ident_t *loc, int gt
   T tc;
   kmp_info_t *th;
   kmp_team_t *team;
-  kmp_uint32 my_buffer_index;
-  dispatch_private_info_template<T> *pr;
-  dispatch_shared_info_template<UT> volatile *sh;
-
-  KMP_BUILD_ASSERT(sizeof(dispatch_private_info_template<T>) ==
-                   sizeof(dispatch_private_info));
-  KMP_BUILD_ASSERT(sizeof(dispatch_shared_info_template<UT>) ==
-                   sizeof(dispatch_shared_info));
-
-  if (!TCR_4(__kmp_init_parallel))
-    __kmp_parallel_initialize();
 
-#if INCLUDE_SSC_MARKS
-  SSC_MARK_DISPATCH_INIT();
-#endif
 #ifdef KMP_DEBUG
   {
     char *buff;
     // create format specifiers before the debug output
-    buff = __kmp_str_format("__kmp_dispatch_init: T#%%d called: schedule:%%d "
-                            "chunk:%%%s lb:%%%s ub:%%%s st:%%%s\n",
-                            traits_t<ST>::spec, traits_t<T>::spec,
-                            traits_t<T>::spec, traits_t<ST>::spec);
-    KD_TRACE(10, (buff, gtid, schedule, chunk, lb, ub, st));
+    buff = __kmp_str_format("__kmp_dispatch_init_algorithm: T#%%d called "
+                            "pr:%%p lb:%%%s ub:%%%s st:%%%s "
+                            "schedule:%%d chunk:%%%s nproc:%%%s tid:%%%s\n",
+                            traits_t<T>::spec, traits_t<T>::spec,
+                            traits_t<ST>::spec, traits_t<ST>::spec,
+                            traits_t<T>::spec, traits_t<T>::spec);
+    KD_TRACE(10, (buff, gtid, pr, lb, ub, st, schedule, chunk, nproc, tid));
     __kmp_str_free(&buff);
   }
 #endif
@@ -563,10 +120,8 @@ __kmp_dispatch_init(ident_t *loc, int gt
   th = __kmp_threads[gtid];
   team = th->th.th_team;
   active = !team->t.t_serialized;
-  th->th.th_ident = loc;
 
 #if USE_ITT_BUILD
-  kmp_uint64 cur_chunk = chunk;
   int itt_need_metadata_reporting = __itt_metadata_add_ptr &&
                                     __kmp_forkjoin_frames_mode == 3 &&
                                     KMP_MASTER_GTID(gtid) &&
@@ -575,23 +130,6 @@ __kmp_dispatch_init(ident_t *loc, int gt
 #endif
                                     team->t.t_active_level == 1;
 #endif
-  if (!active) {
-    pr = reinterpret_cast<dispatch_private_info_template<T> *>(
-        th->th.th_dispatch->th_disp_buffer); /* top of the stack */
-  } else {
-    KMP_DEBUG_ASSERT(th->th.th_dispatch ==
-                     &th->th.th_team->t.t_dispatch[th->th.th_info.ds.ds_tid]);
-
-    my_buffer_index = th->th.th_dispatch->th_disp_index++;
-
-    /* What happens when number of threads changes, need to resize buffer? */
-    pr = reinterpret_cast<dispatch_private_info_template<T> *>(
-        &th->th.th_dispatch
-             ->th_disp_buffer[my_buffer_index % __kmp_dispatch_num_buffers]);
-    sh = reinterpret_cast<dispatch_shared_info_template<UT> volatile *>(
-        &team->t.t_disp_buffer[my_buffer_index % __kmp_dispatch_num_buffers]);
-  }
-
 #if (KMP_STATIC_STEAL_ENABLED)
   if (SCHEDULE_HAS_NONMONOTONIC(schedule))
     // AC: we now have only one implementation of stealing, so use it
@@ -602,19 +140,19 @@ __kmp_dispatch_init(ident_t *loc, int gt
 
   /* Pick up the nomerge/ordered bits from the scheduling type */
   if ((schedule >= kmp_nm_lower) && (schedule < kmp_nm_upper)) {
-    pr->nomerge = TRUE;
+    pr->flags.nomerge = TRUE;
     schedule =
         (enum sched_type)(((int)schedule) - (kmp_nm_lower - kmp_sch_lower));
   } else {
-    pr->nomerge = FALSE;
+    pr->flags.nomerge = FALSE;
   }
   pr->type_size = traits_t<T>::type_size; // remember the size of variables
   if (kmp_ord_lower & schedule) {
-    pr->ordered = TRUE;
+    pr->flags.ordered = TRUE;
     schedule =
         (enum sched_type)(((int)schedule) - (kmp_ord_lower - kmp_sch_lower));
   } else {
-    pr->ordered = FALSE;
+    pr->flags.ordered = FALSE;
   }
 
   if (schedule == kmp_sch_static) {
@@ -635,15 +173,16 @@ __kmp_dispatch_init(ident_t *loc, int gt
       // specified)
       chunk = team->t.t_sched.chunk;
 #if USE_ITT_BUILD
-      cur_chunk = chunk;
+      if (cur_chunk)
+        *cur_chunk = chunk;
 #endif
 #ifdef KMP_DEBUG
       {
         char *buff;
         // create format specifiers before the debug output
-        buff = __kmp_str_format(
-            "__kmp_dispatch_init: T#%%d new: schedule:%%d chunk:%%%s\n",
-            traits_t<ST>::spec);
+        buff = __kmp_str_format("__kmp_dispatch_init_algorithm: T#%%d new: "
+                                "schedule:%%d chunk:%%%s\n",
+                                traits_t<ST>::spec);
         KD_TRACE(10, (buff, gtid, schedule, chunk));
         __kmp_str_free(&buff);
       }
@@ -664,9 +203,10 @@ __kmp_dispatch_init(ident_t *loc, int gt
       {
         char *buff;
         // create format specifiers before the debug output
-        buff = __kmp_str_format("__kmp_dispatch_init: kmp_sch_auto: T#%%d new: "
-                                "schedule:%%d chunk:%%%s\n",
-                                traits_t<ST>::spec);
+        buff = __kmp_str_format(
+            "__kmp_dispatch_init_algorithm: kmp_sch_auto: T#%%d new: "
+            "schedule:%%d chunk:%%%s\n",
+            traits_t<ST>::spec);
         KD_TRACE(10, (buff, gtid, schedule, chunk));
         __kmp_str_free(&buff);
       }
@@ -674,11 +214,11 @@ __kmp_dispatch_init(ident_t *loc, int gt
     }
 
     /* guided analytical not safe for too many threads */
-    if (schedule == kmp_sch_guided_analytical_chunked &&
-        th->th.th_team_nproc > 1 << 20) {
+    if (schedule == kmp_sch_guided_analytical_chunked && nproc > 1 << 20) {
       schedule = kmp_sch_guided_iterative_chunked;
       KMP_WARNING(DispatchManyThreads);
     }
+#if OMP_45_ENABLED
     if (schedule == kmp_sch_runtime_simd) {
       // compiler provides simd_width in the chunk parameter
       schedule = team->t.t_sched.r_sched_type;
@@ -694,7 +234,8 @@ __kmp_dispatch_init(ident_t *loc, int gt
         chunk = team->t.t_sched.chunk * chunk;
       }
 #if USE_ITT_BUILD
-      cur_chunk = chunk;
+      if (cur_chunk)
+        *cur_chunk = chunk;
 #endif
 #ifdef KMP_DEBUG
       {
@@ -708,6 +249,7 @@ __kmp_dispatch_init(ident_t *loc, int gt
       }
 #endif
     }
+#endif // OMP_45_ENABLED
     pr->u.p.parm1 = chunk;
   }
   KMP_ASSERT2((kmp_sch_lower < schedule && schedule < kmp_sch_upper),
@@ -718,7 +260,7 @@ __kmp_dispatch_init(ident_t *loc, int gt
   if (__kmp_env_consistency_check) {
     if (st == 0) {
       __kmp_error_construct(kmp_i18n_msg_CnsLoopIncrZeroProhibited,
-                            (pr->ordered ? ct_pdo_ordered : ct_pdo), loc);
+                            (pr->flags.ordered ? ct_pdo_ordered : ct_pdo), loc);
     }
   }
   // compute trip count
@@ -746,16 +288,6 @@ __kmp_dispatch_init(ident_t *loc, int gt
     }
   }
 
-  // Any half-decent optimizer will remove this test when the blocks are empty
-  // since the macros expand to nothing when statistics are disabled.
-  if (schedule == __kmp_static) {
-    KMP_COUNT_BLOCK(OMP_FOR_static);
-    KMP_COUNT_VALUE(FOR_static_iterations, tc);
-  } else {
-    KMP_COUNT_BLOCK(OMP_FOR_dynamic);
-    KMP_COUNT_VALUE(FOR_dynamic_iterations, tc);
-  }
-
   pr->u.p.lb = lb;
   pr->u.p.ub = ub;
   pr->u.p.st = st;
@@ -768,44 +300,26 @@ __kmp_dispatch_init(ident_t *loc, int gt
   /* NOTE: only the active parallel region(s) has active ordered sections */
 
   if (active) {
-    if (pr->ordered == 0) {
-      th->th.th_dispatch->th_deo_fcn = __kmp_dispatch_deo_error;
-      th->th.th_dispatch->th_dxo_fcn = __kmp_dispatch_dxo_error;
-    } else {
+    if (pr->flags.ordered) {
       pr->ordered_bumped = 0;
-
       pr->u.p.ordered_lower = 1;
       pr->u.p.ordered_upper = 0;
-
-      th->th.th_dispatch->th_deo_fcn = __kmp_dispatch_deo<UT>;
-      th->th.th_dispatch->th_dxo_fcn = __kmp_dispatch_dxo<UT>;
-    }
-  }
-
-  if (__kmp_env_consistency_check) {
-    enum cons_type ws = pr->ordered ? ct_pdo_ordered : ct_pdo;
-    if (push_ws) {
-      __kmp_push_workshare(gtid, ws, loc);
-      pr->pushed_ws = ws;
-    } else {
-      __kmp_check_workshare(gtid, ws, loc);
-      pr->pushed_ws = ct_none;
     }
   }
 
   switch (schedule) {
 #if (KMP_STATIC_STEAL_ENABLED)
   case kmp_sch_static_steal: {
-    T nproc = th->th.th_team_nproc;
     T ntc, init;
 
     KD_TRACE(100,
-             ("__kmp_dispatch_init: T#%d kmp_sch_static_steal case\n", gtid));
+             ("__kmp_dispatch_init_algorithm: T#%d kmp_sch_static_steal case\n",
+              gtid));
 
     ntc = (tc % chunk ? 1 : 0) + tc / chunk;
     if (nproc > 1 && ntc >= nproc) {
       KMP_COUNT_BLOCK(OMP_FOR_static_steal);
-      T id = __kmp_tid_from_gtid(gtid);
+      T id = tid;
       T small_chunk, extras;
 
       small_chunk = ntc / nproc;
@@ -832,7 +346,7 @@ __kmp_dispatch_init(ident_t *loc, int gt
       }
       break;
     } else {
-      KD_TRACE(100, ("__kmp_dispatch_init: T#%d falling-through to "
+      KD_TRACE(100, ("__kmp_dispatch_init_algorithm: T#%d falling-through to "
                      "kmp_sch_static_balanced\n",
                      gtid));
       schedule = kmp_sch_static_balanced;
@@ -842,14 +356,15 @@ __kmp_dispatch_init(ident_t *loc, int gt
   } // case
 #endif
   case kmp_sch_static_balanced: {
-    T nproc = th->th.th_team_nproc;
     T init, limit;
 
-    KD_TRACE(100, ("__kmp_dispatch_init: T#%d kmp_sch_static_balanced case\n",
-                   gtid));
+    KD_TRACE(
+        100,
+        ("__kmp_dispatch_init_algorithm: T#%d kmp_sch_static_balanced case\n",
+         gtid));
 
     if (nproc > 1) {
-      T id = __kmp_tid_from_gtid(gtid);
+      T id = tid;
 
       if (tc < nproc) {
         if (id < tc) {
@@ -873,7 +388,8 @@ __kmp_dispatch_init(ident_t *loc, int gt
         init = 0;
         limit = tc - 1;
         pr->u.p.parm1 = TRUE;
-      } else { // zero trip count
+      } else {
+        // zero trip count
         pr->u.p.count = 1; /* means no more chunks to execute */
         pr->u.p.parm1 = FALSE;
         break;
@@ -882,7 +398,8 @@ __kmp_dispatch_init(ident_t *loc, int gt
 #if USE_ITT_BUILD
     // Calculate chunk for metadata report
     if (itt_need_metadata_reporting)
-      cur_chunk = limit - init + 1;
+      if (cur_chunk)
+        *cur_chunk = limit - init + 1;
 #endif
     if (st == 1) {
       pr->u.p.lb = lb + init;
@@ -899,16 +416,17 @@ __kmp_dispatch_init(ident_t *loc, int gt
         pr->u.p.ub = (ub_tmp + st < ub ? ub : ub_tmp);
       }
     }
-    if (pr->ordered) {
+    if (pr->flags.ordered) {
       pr->u.p.ordered_lower = init;
       pr->u.p.ordered_upper = limit;
     }
     break;
   } // case
+#if OMP_45_ENABLED
   case kmp_sch_static_balanced_chunked: {
     // similar to balanced, but chunk adjusted to multiple of simd width
-    T nth = th->th.th_team_nproc;
-    KD_TRACE(100, ("__kmp_dispatch_init: T#%d runtime(simd:static)"
+    T nth = nproc;
+    KD_TRACE(100, ("__kmp_dispatch_init_algorithm: T#%d runtime(simd:static)"
                    " -> falling-through to static_greedy\n",
                    gtid));
     schedule = kmp_sch_static_greedy;
@@ -918,12 +436,14 @@ __kmp_dispatch_init(ident_t *loc, int gt
       pr->u.p.parm1 = tc;
     break;
   } // case
-  case kmp_sch_guided_iterative_chunked:
-  case kmp_sch_guided_simd: {
-    T nproc = th->th.th_team_nproc;
-    KD_TRACE(100, ("__kmp_dispatch_init: T#%d kmp_sch_guided_iterative_chunked"
-                   " case\n",
-                   gtid));
+  case kmp_sch_guided_simd:
+#endif // OMP_45_ENABLED
+  case kmp_sch_guided_iterative_chunked: {
+    KD_TRACE(
+        100,
+        ("__kmp_dispatch_init_algorithm: T#%d kmp_sch_guided_iterative_chunked"
+         " case\n",
+         gtid));
 
     if (nproc > 1) {
       if ((2L * chunk + 1) * nproc >= tc) {
@@ -936,22 +456,24 @@ __kmp_dispatch_init(ident_t *loc, int gt
             guided_flt_param / nproc; // may occupy parm3 and parm4
       }
     } else {
-      KD_TRACE(100, ("__kmp_dispatch_init: T#%d falling-through to "
+      KD_TRACE(100, ("__kmp_dispatch_init_algorithm: T#%d falling-through to "
                      "kmp_sch_static_greedy\n",
                      gtid));
       schedule = kmp_sch_static_greedy;
       /* team->t.t_nproc == 1: fall-through to kmp_sch_static_greedy */
-      KD_TRACE(100, ("__kmp_dispatch_init: T#%d kmp_sch_static_greedy case\n",
-                     gtid));
+      KD_TRACE(
+          100,
+          ("__kmp_dispatch_init_algorithm: T#%d kmp_sch_static_greedy case\n",
+           gtid));
       pr->u.p.parm1 = tc;
     } // if
   } // case
   break;
   case kmp_sch_guided_analytical_chunked: {
-    T nproc = th->th.th_team_nproc;
-    KD_TRACE(100, ("__kmp_dispatch_init: T#%d kmp_sch_guided_analytical_chunked"
-                   " case\n",
+    KD_TRACE(100, ("__kmp_dispatch_init_algorithm: T#%d "
+                   "kmp_sch_guided_analytical_chunked case\n",
                    gtid));
+
     if (nproc > 1) {
       if ((2L * chunk + 1) * nproc >= tc) {
         /* chunk size too large, switch to dynamic */
@@ -1061,7 +583,7 @@ __kmp_dispatch_init(ident_t *loc, int gt
 #endif
       } // if
     } else {
-      KD_TRACE(100, ("__kmp_dispatch_init: T#%d falling-through to "
+      KD_TRACE(100, ("__kmp_dispatch_init_algorithm: T#%d falling-through to "
                      "kmp_sch_static_greedy\n",
                      gtid));
       schedule = kmp_sch_static_greedy;
@@ -1071,18 +593,18 @@ __kmp_dispatch_init(ident_t *loc, int gt
   } // case
   break;
   case kmp_sch_static_greedy:
-    KD_TRACE(100,
-             ("__kmp_dispatch_init: T#%d kmp_sch_static_greedy case\n", gtid));
-    pr->u.p.parm1 = (th->th.th_team_nproc > 1)
-                        ? (tc + th->th.th_team_nproc - 1) / th->th.th_team_nproc
-                        : tc;
+    KD_TRACE(
+        100,
+        ("__kmp_dispatch_init_algorithm: T#%d kmp_sch_static_greedy case\n",
+         gtid));
+    pr->u.p.parm1 = (nproc > 1) ? (tc + nproc - 1) / nproc : tc;
     break;
   case kmp_sch_static_chunked:
   case kmp_sch_dynamic_chunked:
     if (pr->u.p.parm1 <= 0) {
       pr->u.p.parm1 = KMP_DEFAULT_CHUNK;
     }
-    KD_TRACE(100, ("__kmp_dispatch_init: T#%d "
+    KD_TRACE(100, ("__kmp_dispatch_init_algorithm: T#%d "
                    "kmp_sch_static_chunked/kmp_sch_dynamic_chunked cases\n",
                    gtid));
     break;
@@ -1091,12 +613,13 @@ __kmp_dispatch_init(ident_t *loc, int gt
 
     T parm1, parm2, parm3, parm4;
     KD_TRACE(100,
-             ("__kmp_dispatch_init: T#%d kmp_sch_trapezoidal case\n", gtid));
+             ("__kmp_dispatch_init_algorithm: T#%d kmp_sch_trapezoidal case\n",
+              gtid));
 
     parm1 = chunk;
 
     /* F : size of the first cycle */
-    parm2 = (tc / (2 * th->th.th_team_nproc));
+    parm2 = (tc / (2 * nproc));
 
     if (parm2 < 1) {
       parm2 = 1;
@@ -1142,33 +665,138 @@ __kmp_dispatch_init(ident_t *loc, int gt
   } break;
   } // switch
   pr->schedule = schedule;
-  if (active) {
-    /* The name of this buffer should be my_buffer_index when it's free to use
-     * it */
+}
 
-    KD_TRACE(100, ("__kmp_dispatch_init: T#%d before wait: my_buffer_index:%d "
-                   "sh->buffer_index:%d\n",
-                   gtid, my_buffer_index, sh->buffer_index));
-    __kmp_wait_yield<kmp_uint32>(&sh->buffer_index, my_buffer_index,
-                                 __kmp_eq<kmp_uint32> USE_ITT_BUILD_ARG(NULL));
-    // Note: KMP_WAIT_YIELD() cannot be used there: buffer index and
-    // my_buffer_index are *always* 32-bit integers.
-    KMP_MB(); /* is this necessary? */
-    KD_TRACE(100, ("__kmp_dispatch_init: T#%d after wait: my_buffer_index:%d "
-                   "sh->buffer_index:%d\n",
-                   gtid, my_buffer_index, sh->buffer_index));
+// UT - unsigned flavor of T, ST - signed flavor of T,
+// DBL - double if sizeof(T)==4, or long double if sizeof(T)==8
+template <typename T>
+static void
+__kmp_dispatch_init(ident_t *loc, int gtid, enum sched_type schedule, T lb,
+                    T ub, typename traits_t<T>::signed_t st,
+                    typename traits_t<T>::signed_t chunk, int push_ws) {
+  typedef typename traits_t<T>::unsigned_t UT;
+  typedef typename traits_t<T>::signed_t ST;
+  typedef typename traits_t<T>::floating_t DBL;
 
-    th->th.th_dispatch->th_dispatch_pr_current = (dispatch_private_info_t *)pr;
-    th->th.th_dispatch->th_dispatch_sh_current =
-        CCAST(dispatch_shared_info_t *, (volatile dispatch_shared_info_t *)sh);
-#if USE_ITT_BUILD
-    if (pr->ordered) {
-      __kmp_itt_ordered_init(gtid);
-    }
-    // Report loop metadata
-    if (itt_need_metadata_reporting) {
-      // Only report metadata by master of active team at level 1
-      kmp_uint64 schedtype = 0;
+  int active;
+  kmp_info_t *th;
+  kmp_team_t *team;
+  kmp_uint32 my_buffer_index;
+  dispatch_private_info_template<T> *pr;
+  dispatch_shared_info_template<T> volatile *sh;
+
+  KMP_BUILD_ASSERT(sizeof(dispatch_private_info_template<T>) ==
+                   sizeof(dispatch_private_info));
+  KMP_BUILD_ASSERT(sizeof(dispatch_shared_info_template<UT>) ==
+                   sizeof(dispatch_shared_info));
+
+  if (!TCR_4(__kmp_init_parallel))
+    __kmp_parallel_initialize();
+
+#if INCLUDE_SSC_MARKS
+  SSC_MARK_DISPATCH_INIT();
+#endif
+#ifdef KMP_DEBUG
+  {
+    char *buff;
+    // create format specifiers before the debug output
+    buff = __kmp_str_format("__kmp_dispatch_init: T#%%d called: schedule:%%d "
+                            "chunk:%%%s lb:%%%s ub:%%%s st:%%%s\n",
+                            traits_t<ST>::spec, traits_t<T>::spec,
+                            traits_t<T>::spec, traits_t<ST>::spec);
+    KD_TRACE(10, (buff, gtid, schedule, chunk, lb, ub, st));
+    __kmp_str_free(&buff);
+  }
+#endif
+  /* setup data */
+  th = __kmp_threads[gtid];
+  team = th->th.th_team;
+  active = !team->t.t_serialized;
+  th->th.th_ident = loc;
+
+#if USE_ITT_BUILD
+  kmp_uint64 cur_chunk = chunk;
+  int itt_need_metadata_reporting = __itt_metadata_add_ptr &&
+                                    __kmp_forkjoin_frames_mode == 3 &&
+                                    KMP_MASTER_GTID(gtid) &&
+#if OMP_40_ENABLED
+                                    th->th.th_teams_microtask == NULL &&
+#endif
+                                    team->t.t_active_level == 1;
+#endif
+  if (!active) {
+    pr = reinterpret_cast<dispatch_private_info_template<T> *>(
+        th->th.th_dispatch->th_disp_buffer); /* top of the stack */
+  } else {
+    KMP_DEBUG_ASSERT(th->th.th_dispatch ==
+                     &th->th.th_team->t.t_dispatch[th->th.th_info.ds.ds_tid]);
+
+    my_buffer_index = th->th.th_dispatch->th_disp_index++;
+
+    /* What happens when number of threads changes, need to resize buffer? */
+    pr = reinterpret_cast<dispatch_private_info_template<T> *>(
+        &th->th.th_dispatch
+             ->th_disp_buffer[my_buffer_index % __kmp_dispatch_num_buffers]);
+    sh = reinterpret_cast<dispatch_shared_info_template<T> volatile *>(
+        &team->t.t_disp_buffer[my_buffer_index % __kmp_dispatch_num_buffers]);
+    KD_TRACE(10, ("__kmp_dispatch_init: T#%d my_buffer_index:%d\n", gtid,
+                  my_buffer_index));
+  }
+
+  __kmp_dispatch_init_algorithm(loc, gtid, pr, schedule, lb, ub, st,
+#if USE_ITT_BUILD
+                                &cur_chunk,
+#endif
+                                chunk, (T)th->th.th_team_nproc,
+                                (T)th->th.th_info.ds.ds_tid);
+  if (active) {
+    if (pr->flags.ordered == 0) {
+      th->th.th_dispatch->th_deo_fcn = __kmp_dispatch_deo_error;
+      th->th.th_dispatch->th_dxo_fcn = __kmp_dispatch_dxo_error;
+    } else {
+      th->th.th_dispatch->th_deo_fcn = __kmp_dispatch_deo<UT>;
+      th->th.th_dispatch->th_dxo_fcn = __kmp_dispatch_dxo<UT>;
+    }
+  }
+
+  // Any half-decent optimizer will remove this test when the blocks are empty
+  // since the macros expand to nothing
+  // when statistics are disabled.
+  if (schedule == __kmp_static) {
+    KMP_COUNT_BLOCK(OMP_FOR_static);
+    KMP_COUNT_VALUE(FOR_static_iterations, pr->u.p.tc);
+  } else {
+    KMP_COUNT_BLOCK(OMP_FOR_dynamic);
+    KMP_COUNT_VALUE(FOR_dynamic_iterations, pr->u.p.tc);
+  }
+
+  if (active) {
+    /* The name of this buffer should be my_buffer_index when it's free to use
+     * it */
+
+    KD_TRACE(100, ("__kmp_dispatch_init: T#%d before wait: my_buffer_index:%d "
+                   "sh->buffer_index:%d\n",
+                   gtid, my_buffer_index, sh->buffer_index));
+    __kmp_wait_yield<kmp_uint32>(&sh->buffer_index, my_buffer_index,
+                                 __kmp_eq<kmp_uint32> USE_ITT_BUILD_ARG(NULL));
+    // Note: KMP_WAIT_YIELD() cannot be used there: buffer index and
+    // my_buffer_index are *always* 32-bit integers.
+    KMP_MB(); /* is this necessary? */
+    KD_TRACE(100, ("__kmp_dispatch_init: T#%d after wait: my_buffer_index:%d "
+                   "sh->buffer_index:%d\n",
+                   gtid, my_buffer_index, sh->buffer_index));
+
+    th->th.th_dispatch->th_dispatch_pr_current = (dispatch_private_info_t *)pr;
+    th->th.th_dispatch->th_dispatch_sh_current =
+        CCAST(dispatch_shared_info_t *, (volatile dispatch_shared_info_t *)sh);
+#if USE_ITT_BUILD
+    if (pr->flags.ordered) {
+      __kmp_itt_ordered_init(gtid);
+    }
+    // Report loop metadata
+    if (itt_need_metadata_reporting) {
+      // Only report metadata by master of active team at level 1
+      kmp_uint64 schedtype = 0;
       switch (schedule) {
       case kmp_sch_static_chunked:
       case kmp_sch_static_balanced: // Chunk is calculated in the switch above
@@ -1181,7 +809,9 @@ __kmp_dispatch_init(ident_t *loc, int gt
         break;
       case kmp_sch_guided_iterative_chunked:
       case kmp_sch_guided_analytical_chunked:
+#if OMP_45_ENABLED
       case kmp_sch_guided_simd:
+#endif
         schedtype = 2;
         break;
       default:
@@ -1190,7 +820,7 @@ __kmp_dispatch_init(ident_t *loc, int gt
         schedtype = 3;
         break;
       }
-      __kmp_itt_metadata_loop(loc, schedtype, tc, cur_chunk);
+      __kmp_itt_metadata_loop(loc, schedtype, pr->u.p.tc, cur_chunk);
     }
 #endif /* USE_ITT_BUILD */
   }
@@ -1208,10 +838,10 @@ __kmp_dispatch_init(ident_t *loc, int gt
         traits_t<ST>::spec, traits_t<UT>::spec, traits_t<UT>::spec,
         traits_t<UT>::spec, traits_t<UT>::spec, traits_t<T>::spec,
         traits_t<T>::spec, traits_t<T>::spec, traits_t<T>::spec);
-    KD_TRACE(10, (buff, gtid, pr->schedule, pr->ordered, pr->u.p.lb, pr->u.p.ub,
-                  pr->u.p.st, pr->u.p.tc, pr->u.p.count, pr->u.p.ordered_lower,
-                  pr->u.p.ordered_upper, pr->u.p.parm1, pr->u.p.parm2,
-                  pr->u.p.parm3, pr->u.p.parm4));
+    KD_TRACE(10, (buff, gtid, pr->schedule, pr->flags.ordered, pr->u.p.lb,
+                  pr->u.p.ub, pr->u.p.st, pr->u.p.tc, pr->u.p.count,
+                  pr->u.p.ordered_lower, pr->u.p.ordered_upper, pr->u.p.parm1,
+                  pr->u.p.parm2, pr->u.p.parm3, pr->u.p.parm4));
     __kmp_str_free(&buff);
   }
 #endif
@@ -1234,10 +864,9 @@ __kmp_dispatch_init(ident_t *loc, int gt
   if (ompt_enabled.ompt_callback_work) {
     ompt_team_info_t *team_info = __ompt_get_teaminfo(0, NULL);
     ompt_task_info_t *task_info = __ompt_get_task_info_object(0);
-    kmp_info_t *thr = __kmp_threads[gtid];
     ompt_callbacks.ompt_callback(ompt_callback_work)(
         ompt_work_loop, ompt_scope_begin, &(team_info->parallel_data),
-        &(task_info->task_data), tc, OMPT_LOAD_RETURN_ADDRESS(gtid));
+        &(task_info->task_data), pr->u.p.tc, OMPT_LOAD_RETURN_ADDRESS(gtid));
   }
 #endif
 }
@@ -1389,6 +1018,699 @@ static void __kmp_dispatch_finish_chunk(
 
 #endif /* KMP_GOMP_COMPAT */
 
+template <typename T>
+int __kmp_dispatch_next_algorithm(int gtid,
+                                  dispatch_private_info_template<T> *pr,
+                                  dispatch_shared_info_template<T> volatile *sh,
+                                  kmp_int32 *p_last, T *p_lb, T *p_ub,
+                                  typename traits_t<T>::signed_t *p_st, T nproc,
+                                  T tid) {
+  typedef typename traits_t<T>::unsigned_t UT;
+  typedef typename traits_t<T>::signed_t ST;
+  typedef typename traits_t<T>::floating_t DBL;
+  int status = 0;
+  kmp_int32 last = 0;
+  T start;
+  ST incr;
+  UT limit, trip, init;
+  kmp_info_t *th = __kmp_threads[gtid];
+  kmp_team_t *team = th->th.th_team;
+
+  KMP_DEBUG_ASSERT(th->th.th_dispatch ==
+                   &th->th.th_team->t.t_dispatch[th->th.th_info.ds.ds_tid]);
+  KMP_DEBUG_ASSERT(pr);
+  KMP_DEBUG_ASSERT(sh);
+  KMP_DEBUG_ASSERT(tid >= 0 && tid < nproc);
+#ifdef KMP_DEBUG
+  {
+    char *buff;
+    // create format specifiers before the debug output
+    buff =
+        __kmp_str_format("__kmp_dispatch_next_algorithm: T#%%d called pr:%%p "
+                         "sh:%%p nproc:%%%s tid:%%%s\n",
+                         traits_t<T>::spec, traits_t<T>::spec);
+    KD_TRACE(10, (buff, gtid, pr, sh, nproc, tid));
+    __kmp_str_free(&buff);
+  }
+#endif
+
+  // zero trip count
+  if (pr->u.p.tc == 0) {
+    KD_TRACE(10,
+             ("__kmp_dispatch_next_algorithm: T#%d early exit trip count is "
+              "zero status:%d\n",
+              gtid, status));
+    return 0;
+  }
+
+  switch (pr->schedule) {
+#if (KMP_STATIC_STEAL_ENABLED)
+  case kmp_sch_static_steal: {
+    T chunk = pr->u.p.parm1;
+
+    KD_TRACE(100,
+             ("__kmp_dispatch_next_algorithm: T#%d kmp_sch_static_steal case\n",
+              gtid));
+
+    trip = pr->u.p.tc - 1;
+
+    if (traits_t<T>::type_size > 4) {
+      // use lock for 8-byte and CAS for 4-byte induction
+      // variable. TODO (optional): check and use 16-byte CAS
+      kmp_lock_t *lck = th->th.th_dispatch->th_steal_lock;
+      KMP_DEBUG_ASSERT(lck != NULL);
+      if (pr->u.p.count < (UT)pr->u.p.ub) {
+        __kmp_acquire_lock(lck, gtid);
+        // try to get own chunk of iterations
+        init = (pr->u.p.count)++;
+        status = (init < (UT)pr->u.p.ub);
+        __kmp_release_lock(lck, gtid);
+      } else {
+        status = 0; // no own chunks
+      }
+      if (!status) { // try to steal
+        kmp_info_t **other_threads = team->t.t_threads;
+        int while_limit = nproc; // nproc attempts to find a victim
+        int while_index = 0;
+        // TODO: algorithm of searching for a victim
+        // should be cleaned up and measured
+        while ((!status) && (while_limit != ++while_index)) {
+          T remaining;
+          T victimIdx = pr->u.p.parm4;
+          T oldVictimIdx = victimIdx ? victimIdx - 1 : nproc - 1;
+          dispatch_private_info_template<T> *victim =
+              reinterpret_cast<dispatch_private_info_template<T> *>(
+                  other_threads[victimIdx]
+                      ->th.th_dispatch->th_dispatch_pr_current);
+          while ((victim == NULL || victim == pr ||
+                  (*(volatile T *)&victim->u.p.static_steal_counter !=
+                   *(volatile T *)&pr->u.p.static_steal_counter)) &&
+                 oldVictimIdx != victimIdx) {
+            victimIdx = (victimIdx + 1) % nproc;
+            victim = reinterpret_cast<dispatch_private_info_template<T> *>(
+                other_threads[victimIdx]
+                    ->th.th_dispatch->th_dispatch_pr_current);
+          }
+          if (!victim || (*(volatile T *)&victim->u.p.static_steal_counter !=
+                          *(volatile T *)&pr->u.p.static_steal_counter)) {
+            continue; // try once more (nproc attempts in total)
+            // no victim is ready yet to participate in stealing
+            // because all victims are still in kmp_init_dispatch
+          }
+          if (victim->u.p.count + 2 > (UT)victim->u.p.ub) {
+            pr->u.p.parm4 = (victimIdx + 1) % nproc; // shift start tid
+            continue; // not enough chunks to steal, goto next victim
+          }
+
+          lck = other_threads[victimIdx]->th.th_dispatch->th_steal_lock;
+          KMP_ASSERT(lck != NULL);
+          __kmp_acquire_lock(lck, gtid);
+          limit = victim->u.p.ub; // keep initial ub
+          if (victim->u.p.count >= limit ||
+              (remaining = limit - victim->u.p.count) < 2) {
+            __kmp_release_lock(lck, gtid);
+            pr->u.p.parm4 = (victimIdx + 1) % nproc; // next victim
+            continue; // not enough chunks to steal
+          }
+          // stealing succeded, reduce victim's ub by 1/4 of undone chunks or
+          // by 1
+          if (remaining > 3) {
+            // steal 1/4 of remaining
+            KMP_COUNT_VALUE(FOR_static_steal_stolen, remaining >> 2);
+            init = (victim->u.p.ub -= (remaining >> 2));
+          } else {
+            // steal 1 chunk of 2 or 3 remaining
+            KMP_COUNT_VALUE(FOR_static_steal_stolen, 1);
+            init = (victim->u.p.ub -= 1);
+          }
+          __kmp_release_lock(lck, gtid);
+
+          KMP_DEBUG_ASSERT(init + 1 <= limit);
+          pr->u.p.parm4 = victimIdx; // remember victim to steal from
+          status = 1;
+          while_index = 0;
+          // now update own count and ub with stolen range but init chunk
+          __kmp_acquire_lock(th->th.th_dispatch->th_steal_lock, gtid);
+          pr->u.p.count = init + 1;
+          pr->u.p.ub = limit;
+          __kmp_release_lock(th->th.th_dispatch->th_steal_lock, gtid);
+        } // while (search for victim)
+      } // if (try to find victim and steal)
+    } else {
+      // 4-byte induction variable, use 8-byte CAS for pair (count, ub)
+      typedef union {
+        struct {
+          UT count;
+          T ub;
+        } p;
+        kmp_int64 b;
+      } union_i4;
+      // All operations on 'count' or 'ub' must be combined atomically
+      // together.
+      {
+        union_i4 vold, vnew;
+        vold.b = *(volatile kmp_int64 *)(&pr->u.p.count);
+        vnew = vold;
+        vnew.p.count++;
+        while (!KMP_COMPARE_AND_STORE_ACQ64(
+            (volatile kmp_int64 *)&pr->u.p.count,
+            *VOLATILE_CAST(kmp_int64 *) & vold.b,
+            *VOLATILE_CAST(kmp_int64 *) & vnew.b)) {
+          KMP_CPU_PAUSE();
+          vold.b = *(volatile kmp_int64 *)(&pr->u.p.count);
+          vnew = vold;
+          vnew.p.count++;
+        }
+        vnew = vold;
+        init = vnew.p.count;
+        status = (init < (UT)vnew.p.ub);
+      }
+
+      if (!status) {
+        kmp_info_t **other_threads = team->t.t_threads;
+        int while_limit = nproc; // nproc attempts to find a victim
+        int while_index = 0;
+
+        // TODO: algorithm of searching for a victim
+        // should be cleaned up and measured
+        while ((!status) && (while_limit != ++while_index)) {
+          union_i4 vold, vnew;
+          kmp_int32 remaining;
+          T victimIdx = pr->u.p.parm4;
+          T oldVictimIdx = victimIdx ? victimIdx - 1 : nproc - 1;
+          dispatch_private_info_template<T> *victim =
+              reinterpret_cast<dispatch_private_info_template<T> *>(
+                  other_threads[victimIdx]
+                      ->th.th_dispatch->th_dispatch_pr_current);
+          while ((victim == NULL || victim == pr ||
+                  (*(volatile T *)&victim->u.p.static_steal_counter !=
+                   *(volatile T *)&pr->u.p.static_steal_counter)) &&
+                 oldVictimIdx != victimIdx) {
+            victimIdx = (victimIdx + 1) % nproc;
+            victim = reinterpret_cast<dispatch_private_info_template<T> *>(
+                other_threads[victimIdx]
+                    ->th.th_dispatch->th_dispatch_pr_current);
+          }
+          if (!victim || (*(volatile T *)&victim->u.p.static_steal_counter !=
+                          *(volatile T *)&pr->u.p.static_steal_counter)) {
+            continue; // try once more (nproc attempts in total)
+            // no victim is ready yet to participate in stealing
+            // because all victims are still in kmp_init_dispatch
+          }
+          pr->u.p.parm4 = victimIdx; // new victim found
+          while (1) { // CAS loop if victim has enough chunks to steal
+            vold.b = *(volatile kmp_int64 *)(&victim->u.p.count);
+            vnew = vold;
+
+            KMP_DEBUG_ASSERT((vnew.p.ub - 1) * (UT)chunk <= trip);
+            if (vnew.p.count >= (UT)vnew.p.ub ||
+                (remaining = vnew.p.ub - vnew.p.count) < 2) {
+              pr->u.p.parm4 = (victimIdx + 1) % nproc; // shift start victim id
+              break; // not enough chunks to steal, goto next victim
+            }
+            if (remaining > 3) {
+              vnew.p.ub -= (remaining >> 2); // try to steal 1/4 of remaining
+            } else {
+              vnew.p.ub -= 1; // steal 1 chunk of 2 or 3 remaining
+            }
+            KMP_DEBUG_ASSERT((vnew.p.ub - 1) * (UT)chunk <= trip);
+            // TODO: Should this be acquire or release?
+            if (KMP_COMPARE_AND_STORE_ACQ64(
+                    (volatile kmp_int64 *)&victim->u.p.count,
+                    *VOLATILE_CAST(kmp_int64 *) & vold.b,
+                    *VOLATILE_CAST(kmp_int64 *) & vnew.b)) {
+              // stealing succedded
+              KMP_COUNT_VALUE(FOR_static_steal_stolen, vold.p.ub - vnew.p.ub);
+              status = 1;
+              while_index = 0;
+              // now update own count and ub
+              init = vnew.p.ub;
+              vold.p.count = init + 1;
+#if KMP_ARCH_X86
+              KMP_XCHG_FIXED64((volatile kmp_int64 *)(&pr->u.p.count), vold.b);
+#else
+              *(volatile kmp_int64 *)(&pr->u.p.count) = vold.b;
+#endif
+              break;
+            } // if (check CAS result)
+            KMP_CPU_PAUSE(); // CAS failed, repeate attempt
+          } // while (try to steal from particular victim)
+        } // while (search for victim)
+      } // if (try to find victim and steal)
+    } // if (4-byte induction variable)
+    if (!status) {
+      *p_lb = 0;
+      *p_ub = 0;
+      if (p_st != NULL)
+        *p_st = 0;
+    } else {
+      start = pr->u.p.parm2;
+      init *= chunk;
+      limit = chunk + init - 1;
+      incr = pr->u.p.st;
+      KMP_COUNT_VALUE(FOR_static_steal_chunks, 1);
+
+      KMP_DEBUG_ASSERT(init <= trip);
+      if ((last = (limit >= trip)) != 0)
+        limit = trip;
+      if (p_st != NULL)
+        *p_st = incr;
+
+      if (incr == 1) {
+        *p_lb = start + init;
+        *p_ub = start + limit;
+      } else {
+        *p_lb = start + init * incr;
+        *p_ub = start + limit * incr;
+      }
+
+      if (pr->flags.ordered) {
+        pr->u.p.ordered_lower = init;
+        pr->u.p.ordered_upper = limit;
+      } // if
+    } // if
+    break;
+  } // case
+#endif // ( KMP_STATIC_STEAL_ENABLED )
+  case kmp_sch_static_balanced: {
+    KD_TRACE(
+        10,
+        ("__kmp_dispatch_next_algorithm: T#%d kmp_sch_static_balanced case\n",
+         gtid));
+    /* check if thread has any iteration to do */
+    if ((status = !pr->u.p.count) != 0) {
+      pr->u.p.count = 1;
+      *p_lb = pr->u.p.lb;
+      *p_ub = pr->u.p.ub;
+      last = pr->u.p.parm1;
+      if (p_st != NULL)
+        *p_st = pr->u.p.st;
+    } else { /* no iterations to do */
+      pr->u.p.lb = pr->u.p.ub + pr->u.p.st;
+    }
+  } // case
+  break;
+  case kmp_sch_static_greedy: /* original code for kmp_sch_static_greedy was
+                                 merged here */
+  case kmp_sch_static_chunked: {
+    T parm1;
+
+    KD_TRACE(100, ("__kmp_dispatch_next_algorithm: T#%d "
+                   "kmp_sch_static_[affinity|chunked] case\n",
+                   gtid));
+    parm1 = pr->u.p.parm1;
+
+    trip = pr->u.p.tc - 1;
+    init = parm1 * (pr->u.p.count + tid);
+
+    if ((status = (init <= trip)) != 0) {
+      start = pr->u.p.lb;
+      incr = pr->u.p.st;
+      limit = parm1 + init - 1;
+
+      if ((last = (limit >= trip)) != 0)
+        limit = trip;
+
+      if (p_st != NULL)
+        *p_st = incr;
+
+      pr->u.p.count += nproc;
+
+      if (incr == 1) {
+        *p_lb = start + init;
+        *p_ub = start + limit;
+      } else {
+        *p_lb = start + init * incr;
+        *p_ub = start + limit * incr;
+      }
+
+      if (pr->flags.ordered) {
+        pr->u.p.ordered_lower = init;
+        pr->u.p.ordered_upper = limit;
+      } // if
+    } // if
+  } // case
+  break;
+
+  case kmp_sch_dynamic_chunked: {
+    T chunk = pr->u.p.parm1;
+
+    KD_TRACE(
+        100,
+        ("__kmp_dispatch_next_algorithm: T#%d kmp_sch_dynamic_chunked case\n",
+         gtid));
+
+    init = chunk * test_then_inc_acq<ST>((volatile ST *)&sh->u.s.iteration);
+    trip = pr->u.p.tc - 1;
+
+    if ((status = (init <= trip)) == 0) {
+      *p_lb = 0;
+      *p_ub = 0;
+      if (p_st != NULL)
+        *p_st = 0;
+    } else {
+      start = pr->u.p.lb;
+      limit = chunk + init - 1;
+      incr = pr->u.p.st;
+
+      if ((last = (limit >= trip)) != 0)
+        limit = trip;
+
+      if (p_st != NULL)
+        *p_st = incr;
+
+      if (incr == 1) {
+        *p_lb = start + init;
+        *p_ub = start + limit;
+      } else {
+        *p_lb = start + init * incr;
+        *p_ub = start + limit * incr;
+      }
+
+      if (pr->flags.ordered) {
+        pr->u.p.ordered_lower = init;
+        pr->u.p.ordered_upper = limit;
+      } // if
+    } // if
+  } // case
+  break;
+
+  case kmp_sch_guided_iterative_chunked: {
+    T chunkspec = pr->u.p.parm1;
+    KD_TRACE(100, ("__kmp_dispatch_next_algorithm: T#%d kmp_sch_guided_chunked "
+                   "iterative case\n",
+                   gtid));
+    trip = pr->u.p.tc;
+    // Start atomic part of calculations
+    while (1) {
+      ST remaining; // signed, because can be < 0
+      init = sh->u.s.iteration; // shared value
+      remaining = trip - init;
+      if (remaining <= 0) { // AC: need to compare with 0 first
+        // nothing to do, don't try atomic op
+        status = 0;
+        break;
+      }
+      if ((T)remaining <
+          pr->u.p.parm2) { // compare with K*nproc*(chunk+1), K=2 by default
+        // use dynamic-style shcedule
+        // atomically inrement iterations, get old value
+        init = test_then_add<ST>(RCAST(volatile ST *, &sh->u.s.iteration),
+                                 (ST)chunkspec);
+        remaining = trip - init;
+        if (remaining <= 0) {
+          status = 0; // all iterations got by other threads
+        } else {
+          // got some iterations to work on
+          status = 1;
+          if ((T)remaining > chunkspec) {
+            limit = init + chunkspec - 1;
+          } else {
+            last = 1; // the last chunk
+            limit = init + remaining - 1;
+          } // if
+        } // if
+        break;
+      } // if
+      limit = init +
+              (UT)(remaining * *(double *)&pr->u.p.parm3); // divide by K*nproc
+      if (compare_and_swap<ST>(RCAST(volatile ST *, &sh->u.s.iteration),
+                               (ST)init, (ST)limit)) {
+        // CAS was successful, chunk obtained
+        status = 1;
+        --limit;
+        break;
+      } // if
+    } // while
+    if (status != 0) {
+      start = pr->u.p.lb;
+      incr = pr->u.p.st;
+      if (p_st != NULL)
+        *p_st = incr;
+      *p_lb = start + init * incr;
+      *p_ub = start + limit * incr;
+      if (pr->flags.ordered) {
+        pr->u.p.ordered_lower = init;
+        pr->u.p.ordered_upper = limit;
+      } // if
+    } else {
+      *p_lb = 0;
+      *p_ub = 0;
+      if (p_st != NULL)
+        *p_st = 0;
+    } // if
+  } // case
+  break;
+
+#if OMP_45_ENABLED
+  case kmp_sch_guided_simd: {
+    // same as iterative but curr-chunk adjusted to be multiple of given
+    // chunk
+    T chunk = pr->u.p.parm1;
+    KD_TRACE(100,
+             ("__kmp_dispatch_next_algorithm: T#%d kmp_sch_guided_simd case\n",
+              gtid));
+    trip = pr->u.p.tc;
+    // Start atomic part of calculations
+    while (1) {
+      ST remaining; // signed, because can be < 0
+      init = sh->u.s.iteration; // shared value
+      remaining = trip - init;
+      if (remaining <= 0) { // AC: need to compare with 0 first
+        status = 0; // nothing to do, don't try atomic op
+        break;
+      }
+      KMP_DEBUG_ASSERT(init % chunk == 0);
+      // compare with K*nproc*(chunk+1), K=2 by default
+      if ((T)remaining < pr->u.p.parm2) {
+        // use dynamic-style shcedule
+        // atomically inrement iterations, get old value
+        init = test_then_add<ST>(RCAST(volatile ST *, &sh->u.s.iteration),
+                                 (ST)chunk);
+        remaining = trip - init;
+        if (remaining <= 0) {
+          status = 0; // all iterations got by other threads
+        } else {
+          // got some iterations to work on
+          status = 1;
+          if ((T)remaining > chunk) {
+            limit = init + chunk - 1;
+          } else {
+            last = 1; // the last chunk
+            limit = init + remaining - 1;
+          } // if
+        } // if
+        break;
+      } // if
+      // divide by K*nproc
+      UT span = remaining * (*(double *)&pr->u.p.parm3);
+      UT rem = span % chunk;
+      if (rem) // adjust so that span%chunk == 0
+        span += chunk - rem;
+      limit = init + span;
+      if (compare_and_swap<ST>(RCAST(volatile ST *, &sh->u.s.iteration),
+                               (ST)init, (ST)limit)) {
+        // CAS was successful, chunk obtained
+        status = 1;
+        --limit;
+        break;
+      } // if
+    } // while
+    if (status != 0) {
+      start = pr->u.p.lb;
+      incr = pr->u.p.st;
+      if (p_st != NULL)
+        *p_st = incr;
+      *p_lb = start + init * incr;
+      *p_ub = start + limit * incr;
+      if (pr->flags.ordered) {
+        pr->u.p.ordered_lower = init;
+        pr->u.p.ordered_upper = limit;
+      } // if
+    } else {
+      *p_lb = 0;
+      *p_ub = 0;
+      if (p_st != NULL)
+        *p_st = 0;
+    } // if
+  } // case
+  break;
+#endif // OMP_45_ENABLED
+
+  case kmp_sch_guided_analytical_chunked: {
+    T chunkspec = pr->u.p.parm1;
+    UT chunkIdx;
+#if KMP_OS_WINDOWS && KMP_ARCH_X86
+    /* for storing original FPCW value for Windows* OS on
+       IA-32 architecture 8-byte version */
+    unsigned int oldFpcw;
+    unsigned int fpcwSet = 0;
+#endif
+    KD_TRACE(100, ("__kmp_dispatch_next_algorithm: T#%d "
+                   "kmp_sch_guided_analytical_chunked case\n",
+                   gtid));
+
+    trip = pr->u.p.tc;
+
+    KMP_DEBUG_ASSERT(nproc > 1);
+    KMP_DEBUG_ASSERT((2UL * chunkspec + 1) * (UT)nproc < trip);
+
+    while (1) { /* this while loop is a safeguard against unexpected zero
+                   chunk sizes */
+      chunkIdx = test_then_inc_acq<ST>((volatile ST *)&sh->u.s.iteration);
+      if (chunkIdx >= (UT)pr->u.p.parm2) {
+        --trip;
+        /* use dynamic-style scheduling */
+        init = chunkIdx * chunkspec + pr->u.p.count;
+        /* need to verify init > 0 in case of overflow in the above
+         * calculation */
+        if ((status = (init > 0 && init <= trip)) != 0) {
+          limit = init + chunkspec - 1;
+
+          if ((last = (limit >= trip)) != 0)
+            limit = trip;
+        }
+        break;
+      } else {
+/* use exponential-style scheduling */
+/* The following check is to workaround the lack of long double precision on
+   Windows* OS.
+   This check works around the possible effect that init != 0 for chunkIdx == 0.
+ */
+#if KMP_OS_WINDOWS && KMP_ARCH_X86
+        /* If we haven't already done so, save original
+           FPCW and set precision to 64-bit, as Windows* OS
+           on IA-32 architecture defaults to 53-bit */
+        if (!fpcwSet) {
+          oldFpcw = _control87(0, 0);
+          _control87(_PC_64, _MCW_PC);
+          fpcwSet = 0x30000;
+        }
+#endif
+        if (chunkIdx) {
+          init = __kmp_dispatch_guided_remaining<T>(
+              trip, *(DBL *)&pr->u.p.parm3, chunkIdx);
+          KMP_DEBUG_ASSERT(init);
+          init = trip - init;
+        } else
+          init = 0;
+        limit = trip - __kmp_dispatch_guided_remaining<T>(
+                           trip, *(DBL *)&pr->u.p.parm3, chunkIdx + 1);
+        KMP_ASSERT(init <= limit);
+        if (init < limit) {
+          KMP_DEBUG_ASSERT(limit <= trip);
+          --limit;
+          status = 1;
+          break;
+        } // if
+      } // if
+    } // while (1)
+#if KMP_OS_WINDOWS && KMP_ARCH_X86
+    /* restore FPCW if necessary
+       AC: check fpcwSet flag first because oldFpcw can be uninitialized here
+    */
+    if (fpcwSet && (oldFpcw & fpcwSet))
+      _control87(oldFpcw, _MCW_PC);
+#endif
+    if (status != 0) {
+      start = pr->u.p.lb;
+      incr = pr->u.p.st;
+      if (p_st != NULL)
+        *p_st = incr;
+      *p_lb = start + init * incr;
+      *p_ub = start + limit * incr;
+      if (pr->flags.ordered) {
+        pr->u.p.ordered_lower = init;
+        pr->u.p.ordered_upper = limit;
+      }
+    } else {
+      *p_lb = 0;
+      *p_ub = 0;
+      if (p_st != NULL)
+        *p_st = 0;
+    }
+  } // case
+  break;
+
+  case kmp_sch_trapezoidal: {
+    UT index;
+    T parm2 = pr->u.p.parm2;
+    T parm3 = pr->u.p.parm3;
+    T parm4 = pr->u.p.parm4;
+    KD_TRACE(100,
+             ("__kmp_dispatch_next_algorithm: T#%d kmp_sch_trapezoidal case\n",
+              gtid));
+
+    index = test_then_inc<ST>((volatile ST *)&sh->u.s.iteration);
+
+    init = (index * ((2 * parm2) - (index - 1) * parm4)) / 2;
+    trip = pr->u.p.tc - 1;
+
+    if ((status = ((T)index < parm3 && init <= trip)) == 0) {
+      *p_lb = 0;
+      *p_ub = 0;
+      if (p_st != NULL)
+        *p_st = 0;
+    } else {
+      start = pr->u.p.lb;
+      limit = ((index + 1) * (2 * parm2 - index * parm4)) / 2 - 1;
+      incr = pr->u.p.st;
+
+      if ((last = (limit >= trip)) != 0)
+        limit = trip;
+
+      if (p_st != NULL)
+        *p_st = incr;
+
+      if (incr == 1) {
+        *p_lb = start + init;
+        *p_ub = start + limit;
+      } else {
+        *p_lb = start + init * incr;
+        *p_ub = start + limit * incr;
+      }
+
+      if (pr->flags.ordered) {
+        pr->u.p.ordered_lower = init;
+        pr->u.p.ordered_upper = limit;
+      } // if
+    } // if
+  } // case
+  break;
+  default: {
+    status = 0; // to avoid complaints on uninitialized variable use
+    __kmp_fatal(KMP_MSG(UnknownSchedTypeDetected), // Primary message
+                KMP_HNT(GetNewerLibrary), // Hint
+                __kmp_msg_null // Variadic argument list terminator
+                );
+  } break;
+  } // switch
+  if (p_last)
+    *p_last = last;
+#ifdef KMP_DEBUG
+  if (pr->flags.ordered) {
+    char *buff;
+    // create format specifiers before the debug output
+    buff = __kmp_str_format("__kmp_dispatch_next_algorithm: T#%%d "
+                            "ordered_lower:%%%s ordered_upper:%%%s\n",
+                            traits_t<UT>::spec, traits_t<UT>::spec);
+    KD_TRACE(1000, (buff, gtid, pr->u.p.ordered_lower, pr->u.p.ordered_upper));
+    __kmp_str_free(&buff);
+  }
+  {
+    char *buff;
+    // create format specifiers before the debug output
+    buff = __kmp_str_format(
+        "__kmp_dispatch_next_algorithm: T#%%d exit status:%%d p_last:%%d "
+        "p_lb:%%%s p_ub:%%%s p_st:%%%s\n",
+        traits_t<T>::spec, traits_t<T>::spec, traits_t<ST>::spec);
+    KD_TRACE(10, (buff, gtid, status, *p_last, *p_lb, *p_ub, *p_st));
+    __kmp_str_free(&buff);
+  }
+#endif
+  return status;
+}
+
 /* Define a macro for exiting __kmp_dispatch_next(). If status is 0 (no more
    work), then tell OMPT the loop is over. In some cases kmp_dispatch_fini()
    is not called. */
@@ -1421,7 +1743,6 @@ static int __kmp_dispatch_next(ident_t *
   typedef typename traits_t<T>::unsigned_t UT;
   typedef typename traits_t<T>::signed_t ST;
   typedef typename traits_t<T>::floating_t DBL;
-
   // This is potentially slightly misleading, schedule(runtime) will appear here
   // even if the actual runtme schedule is static. (Which points out a
   // disadavantage of schedule(runtime): even when static scheduling is used it
@@ -1434,18 +1755,10 @@ static int __kmp_dispatch_next(ident_t *
   kmp_team_t *team = th->th.th_team;
 
   KMP_DEBUG_ASSERT(p_lb && p_ub && p_st); // AC: these cannot be NULL
-#ifdef KMP_DEBUG
-  {
-    char *buff;
-    // create format specifiers before the debug output
-    buff = __kmp_str_format("__kmp_dispatch_next: T#%%d called p_lb:%%%s "
-                            "p_ub:%%%s p_st:%%%s p_last: %%p\n",
-                            traits_t<T>::spec, traits_t<T>::spec,
-                            traits_t<ST>::spec);
-    KD_TRACE(1000, (buff, gtid, *p_lb, *p_ub, p_st ? *p_st : 0, p_last));
-    __kmp_str_free(&buff);
-  }
-#endif
+  KD_TRACE(
+      1000,
+      ("__kmp_dispatch_next: T#%d called p_lb:%p p_ub:%p p_st:%p p_last: %p\n",
+       gtid, p_lb, p_ub, p_st, p_last));
 
   if (team->t.t_serialized) {
     /* NOTE: serialize this dispatch becase we are not at the active level */
@@ -1465,7 +1778,7 @@ static int __kmp_dispatch_next(ident_t *
           pr->pushed_ws = __kmp_pop_workshare(gtid, pr->pushed_ws, loc);
         }
       }
-    } else if (pr->nomerge) {
+    } else if (pr->flags.nomerge) {
       kmp_int32 last;
       T start;
       UT limit, trip, init;
@@ -1513,7 +1826,7 @@ static int __kmp_dispatch_next(ident_t *
           *p_ub = start + limit * incr;
         }
 
-        if (pr->ordered) {
+        if (pr->flags.ordered) {
           pr->u.p.ordered_lower = init;
           pr->u.p.ordered_upper = limit;
 #ifdef KMP_DEBUG
@@ -1561,10 +1874,7 @@ static int __kmp_dispatch_next(ident_t *
     return status;
   } else {
     kmp_int32 last = 0;
-    dispatch_shared_info_template<UT> *sh;
-    T start;
-    ST incr;
-    UT limit, trip, init;
+    dispatch_shared_info_template<T> volatile *sh;
 
     KMP_DEBUG_ASSERT(th->th.th_dispatch ==
                      &th->th.th_team->t.t_dispatch[th->th.th_info.ds.ds_tid]);
@@ -1572,735 +1882,14 @@ static int __kmp_dispatch_next(ident_t *
     pr = reinterpret_cast<dispatch_private_info_template<T> *>(
         th->th.th_dispatch->th_dispatch_pr_current);
     KMP_DEBUG_ASSERT(pr);
-    sh = reinterpret_cast<dispatch_shared_info_template<UT> *>(
+    sh = reinterpret_cast<dispatch_shared_info_template<T> volatile *>(
         th->th.th_dispatch->th_dispatch_sh_current);
     KMP_DEBUG_ASSERT(sh);
 
-    if (pr->u.p.tc == 0) {
-      // zero trip count
-      status = 0;
-    } else {
-      switch (pr->schedule) {
-#if (KMP_STATIC_STEAL_ENABLED)
-      case kmp_sch_static_steal: {
-        T chunk = pr->u.p.parm1;
-        int nproc = th->th.th_team_nproc;
-
-        KD_TRACE(100, ("__kmp_dispatch_next: T#%d kmp_sch_static_steal case\n",
-                       gtid));
-
-        trip = pr->u.p.tc - 1;
-
-        if (traits_t<T>::type_size > 4) {
-          // use lock for 8-byte and CAS for 4-byte induction
-          // variable. TODO (optional): check and use 16-byte CAS
-          kmp_lock_t *lck = th->th.th_dispatch->th_steal_lock;
-          KMP_DEBUG_ASSERT(lck != NULL);
-          if (pr->u.p.count < (UT)pr->u.p.ub) {
-            __kmp_acquire_lock(lck, gtid);
-            // try to get own chunk of iterations
-            init = (pr->u.p.count)++;
-            status = (init < (UT)pr->u.p.ub);
-            __kmp_release_lock(lck, gtid);
-          } else {
-            status = 0; // no own chunks
-          }
-          if (!status) { // try to steal
-            kmp_info_t **other_threads = team->t.t_threads;
-            int while_limit = nproc; // nproc attempts to find a victim
-            int while_index = 0;
-            // TODO: algorithm of searching for a victim
-            // should be cleaned up and measured
-            while ((!status) && (while_limit != ++while_index)) {
-              T remaining;
-              T victimIdx = pr->u.p.parm4;
-              T oldVictimIdx = victimIdx ? victimIdx - 1 : nproc - 1;
-              dispatch_private_info_template<T> *victim =
-                  reinterpret_cast<dispatch_private_info_template<T> *>(
-                      other_threads[victimIdx]
-                          ->th.th_dispatch->th_dispatch_pr_current);
-              while ((victim == NULL || victim == pr ||
-                      (*(volatile T *)&victim->u.p.static_steal_counter !=
-                       *(volatile T *)&pr->u.p.static_steal_counter)) &&
-                     oldVictimIdx != victimIdx) {
-                victimIdx = (victimIdx + 1) % nproc;
-                victim = reinterpret_cast<dispatch_private_info_template<T> *>(
-                    other_threads[victimIdx]
-                        ->th.th_dispatch->th_dispatch_pr_current);
-              }
-              if (!victim ||
-                  (*(volatile T *)&victim->u.p.static_steal_counter !=
-                   *(volatile T *)&pr->u.p.static_steal_counter)) {
-                continue; // try once more (nproc attempts in total)
-                // no victim is ready yet to participate in stealing
-                // because all victims are still in kmp_init_dispatch
-              }
-              if (victim->u.p.count + 2 > (UT)victim->u.p.ub) {
-                pr->u.p.parm4 = (victimIdx + 1) % nproc; // shift start tid
-                continue; // not enough chunks to steal, goto next victim
-              }
-
-              lck = other_threads[victimIdx]->th.th_dispatch->th_steal_lock;
-              KMP_ASSERT(lck != NULL);
-              __kmp_acquire_lock(lck, gtid);
-              limit = victim->u.p.ub; // keep initial ub
-              if (victim->u.p.count >= limit ||
-                  (remaining = limit - victim->u.p.count) < 2) {
-                __kmp_release_lock(lck, gtid);
-                pr->u.p.parm4 = (victimIdx + 1) % nproc; // next victim
-                continue; // not enough chunks to steal
-              }
-              // stealing succeded, reduce victim's ub by 1/4 of undone chunks
-              // or by 1
-              if (remaining > 3) {
-                KMP_COUNT_VALUE(FOR_static_steal_stolen, remaining >> 2);
-                init = (victim->u.p.ub -=
-                        (remaining >> 2)); // steal 1/4 of remaining
-              } else {
-                KMP_COUNT_VALUE(FOR_static_steal_stolen, 1);
-                init =
-                    (victim->u.p.ub -= 1); // steal 1 chunk of 2 or 3 remaining
-              }
-              __kmp_release_lock(lck, gtid);
-
-              KMP_DEBUG_ASSERT(init + 1 <= limit);
-              pr->u.p.parm4 = victimIdx; // remember victim to steal from
-              status = 1;
-              while_index = 0;
-              // now update own count and ub with stolen range but init chunk
-              __kmp_acquire_lock(th->th.th_dispatch->th_steal_lock, gtid);
-              pr->u.p.count = init + 1;
-              pr->u.p.ub = limit;
-              __kmp_release_lock(th->th.th_dispatch->th_steal_lock, gtid);
-            } // while (search for victim)
-          } // if (try to find victim and steal)
-        } else {
-          // 4-byte induction variable, use 8-byte CAS for pair (count, ub)
-          typedef union {
-            struct {
-              UT count;
-              T ub;
-            } p;
-            kmp_int64 b;
-          } union_i4;
-          // All operations on 'count' or 'ub' must be combined atomically
-          // together.
-          {
-            union_i4 vold, vnew;
-            vold.b = *(volatile kmp_int64 *)(&pr->u.p.count);
-            vnew = vold;
-            vnew.p.count++;
-            while (!KMP_COMPARE_AND_STORE_ACQ64(
-                (volatile kmp_int64 *)&pr->u.p.count,
-                *VOLATILE_CAST(kmp_int64 *) & vold.b,
-                *VOLATILE_CAST(kmp_int64 *) & vnew.b)) {
-              KMP_CPU_PAUSE();
-              vold.b = *(volatile kmp_int64 *)(&pr->u.p.count);
-              vnew = vold;
-              vnew.p.count++;
-            }
-            vnew = vold;
-            init = vnew.p.count;
-            status = (init < (UT)vnew.p.ub);
-          }
-
-          if (!status) {
-            kmp_info_t **other_threads = team->t.t_threads;
-            int while_limit = nproc; // nproc attempts to find a victim
-            int while_index = 0;
-
-            // TODO: algorithm of searching for a victim
-            // should be cleaned up and measured
-            while ((!status) && (while_limit != ++while_index)) {
-              union_i4 vold, vnew;
-              kmp_int32 remaining;
-              T victimIdx = pr->u.p.parm4;
-              T oldVictimIdx = victimIdx ? victimIdx - 1 : nproc - 1;
-              dispatch_private_info_template<T> *victim =
-                  reinterpret_cast<dispatch_private_info_template<T> *>(
-                      other_threads[victimIdx]
-                          ->th.th_dispatch->th_dispatch_pr_current);
-              while ((victim == NULL || victim == pr ||
-                      (*(volatile T *)&victim->u.p.static_steal_counter !=
-                       *(volatile T *)&pr->u.p.static_steal_counter)) &&
-                     oldVictimIdx != victimIdx) {
-                victimIdx = (victimIdx + 1) % nproc;
-                victim = reinterpret_cast<dispatch_private_info_template<T> *>(
-                    other_threads[victimIdx]
-                        ->th.th_dispatch->th_dispatch_pr_current);
-              }
-              if (!victim ||
-                  (*(volatile T *)&victim->u.p.static_steal_counter !=
-                   *(volatile T *)&pr->u.p.static_steal_counter)) {
-                continue; // try once more (nproc attempts in total)
-                // no victim is ready yet to participate in stealing
-                // because all victims are still in kmp_init_dispatch
-              }
-              pr->u.p.parm4 = victimIdx; // new victim found
-              while (1) { // CAS loop if victim has enough chunks to steal
-                vold.b = *(volatile kmp_int64 *)(&victim->u.p.count);
-                vnew = vold;
-
-                KMP_DEBUG_ASSERT((vnew.p.ub - 1) * (UT)chunk <= trip);
-                if (vnew.p.count >= (UT)vnew.p.ub ||
-                    (remaining = vnew.p.ub - vnew.p.count) < 2) {
-                  pr->u.p.parm4 =
-                      (victimIdx + 1) % nproc; // shift start victim id
-                  break; // not enough chunks to steal, goto next victim
-                }
-                if (remaining > 3) {
-                  vnew.p.ub -= (remaining >> 2); // try to steal 1/4 remaining
-                } else {
-                  vnew.p.ub -= 1; // steal 1 chunk of 2 or 3 remaining
-                }
-                KMP_DEBUG_ASSERT((vnew.p.ub - 1) * (UT)chunk <= trip);
-                // TODO: Should this be acquire or release?
-                if (KMP_COMPARE_AND_STORE_ACQ64(
-                        (volatile kmp_int64 *)&victim->u.p.count,
-                        *VOLATILE_CAST(kmp_int64 *) & vold.b,
-                        *VOLATILE_CAST(kmp_int64 *) & vnew.b)) {
-                  // stealing succeeded
-                  KMP_COUNT_VALUE(FOR_static_steal_stolen,
-                                  vold.p.ub - vnew.p.ub);
-                  status = 1;
-                  while_index = 0;
-                  // now update own count and ub
-                  init = vnew.p.ub;
-                  vold.p.count = init + 1;
-#if KMP_ARCH_X86
-                  KMP_XCHG_FIXED64((volatile kmp_int64 *)(&pr->u.p.count),
-                                   vold.b);
-#else
-                  *(volatile kmp_int64 *)(&pr->u.p.count) = vold.b;
-#endif
-                  break;
-                } // if (check CAS result)
-                KMP_CPU_PAUSE(); // CAS failed, repeat attempt
-              } // while (try to steal from particular victim)
-            } // while (search for victim)
-          } // if (try to find victim and steal)
-        } // if (4-byte induction variable)
-        if (!status) {
-          *p_lb = 0;
-          *p_ub = 0;
-          if (p_st != NULL)
-            *p_st = 0;
-        } else {
-          start = pr->u.p.parm2;
-          init *= chunk;
-          limit = chunk + init - 1;
-          incr = pr->u.p.st;
-          KMP_COUNT_VALUE(FOR_static_steal_chunks, 1);
-
-          KMP_DEBUG_ASSERT(init <= trip);
-          if ((last = (limit >= trip)) != 0)
-            limit = trip;
-          if (p_st != NULL)
-            *p_st = incr;
-
-          if (incr == 1) {
-            *p_lb = start + init;
-            *p_ub = start + limit;
-          } else {
-            *p_lb = start + init * incr;
-            *p_ub = start + limit * incr;
-          }
-
-          if (pr->ordered) {
-            pr->u.p.ordered_lower = init;
-            pr->u.p.ordered_upper = limit;
-#ifdef KMP_DEBUG
-            {
-              char *buff;
-              // create format specifiers before the debug output
-              buff = __kmp_str_format("__kmp_dispatch_next: T#%%d "
-                                      "ordered_lower:%%%s ordered_upper:%%%s\n",
-                                      traits_t<UT>::spec, traits_t<UT>::spec);
-              KD_TRACE(1000, (buff, gtid, pr->u.p.ordered_lower,
-                              pr->u.p.ordered_upper));
-              __kmp_str_free(&buff);
-            }
-#endif
-          } // if
-        } // if
-        break;
-      } // case
-#endif // ( KMP_STATIC_STEAL_ENABLED )
-      case kmp_sch_static_balanced: {
-        KD_TRACE(
-            100,
-            ("__kmp_dispatch_next: T#%d kmp_sch_static_balanced case\n", gtid));
-        if ((status = !pr->u.p.count) !=
-            0) { /* check if thread has any iteration to do */
-          pr->u.p.count = 1;
-          *p_lb = pr->u.p.lb;
-          *p_ub = pr->u.p.ub;
-          last = pr->u.p.parm1;
-          if (p_st != NULL)
-            *p_st = pr->u.p.st;
-        } else { /* no iterations to do */
-          pr->u.p.lb = pr->u.p.ub + pr->u.p.st;
-        }
-        if (pr->ordered) {
-#ifdef KMP_DEBUG
-          {
-            char *buff;
-            // create format specifiers before the debug output
-            buff = __kmp_str_format("__kmp_dispatch_next: T#%%d "
-                                    "ordered_lower:%%%s ordered_upper:%%%s\n",
-                                    traits_t<UT>::spec, traits_t<UT>::spec);
-            KD_TRACE(1000, (buff, gtid, pr->u.p.ordered_lower,
-                            pr->u.p.ordered_upper));
-            __kmp_str_free(&buff);
-          }
-#endif
-        } // if
-      } // case
-      break;
-      case kmp_sch_static_greedy: /* original code for kmp_sch_static_greedy was
-                                     merged here */
-      case kmp_sch_static_chunked: {
-        T parm1;
-
-        KD_TRACE(100, ("__kmp_dispatch_next: T#%d "
-                       "kmp_sch_static_[affinity|chunked] case\n",
-                       gtid));
-        parm1 = pr->u.p.parm1;
-
-        trip = pr->u.p.tc - 1;
-        init = parm1 * (pr->u.p.count + __kmp_tid_from_gtid(gtid));
-
-        if ((status = (init <= trip)) != 0) {
-          start = pr->u.p.lb;
-          incr = pr->u.p.st;
-          limit = parm1 + init - 1;
-
-          if ((last = (limit >= trip)) != 0)
-            limit = trip;
-
-          if (p_st != NULL)
-            *p_st = incr;
-
-          pr->u.p.count += th->th.th_team_nproc;
-
-          if (incr == 1) {
-            *p_lb = start + init;
-            *p_ub = start + limit;
-          } else {
-            *p_lb = start + init * incr;
-            *p_ub = start + limit * incr;
-          }
-
-          if (pr->ordered) {
-            pr->u.p.ordered_lower = init;
-            pr->u.p.ordered_upper = limit;
-#ifdef KMP_DEBUG
-            {
-              char *buff;
-              // create format specifiers before the debug output
-              buff = __kmp_str_format("__kmp_dispatch_next: T#%%d "
-                                      "ordered_lower:%%%s ordered_upper:%%%s\n",
-                                      traits_t<UT>::spec, traits_t<UT>::spec);
-              KD_TRACE(1000, (buff, gtid, pr->u.p.ordered_lower,
-                              pr->u.p.ordered_upper));
-              __kmp_str_free(&buff);
-            }
-#endif
-          } // if
-        } // if
-      } // case
-      break;
-
-      case kmp_sch_dynamic_chunked: {
-        T chunk = pr->u.p.parm1;
-
-        KD_TRACE(
-            100,
-            ("__kmp_dispatch_next: T#%d kmp_sch_dynamic_chunked case\n", gtid));
-
-        init = chunk * test_then_inc_acq<ST>((volatile ST *)&sh->u.s.iteration);
-        trip = pr->u.p.tc - 1;
-
-        if ((status = (init <= trip)) == 0) {
-          *p_lb = 0;
-          *p_ub = 0;
-          if (p_st != NULL)
-            *p_st = 0;
-        } else {
-          start = pr->u.p.lb;
-          limit = chunk + init - 1;
-          incr = pr->u.p.st;
-
-          if ((last = (limit >= trip)) != 0)
-            limit = trip;
-
-          if (p_st != NULL)
-            *p_st = incr;
-
-          if (incr == 1) {
-            *p_lb = start + init;
-            *p_ub = start + limit;
-          } else {
-            *p_lb = start + init * incr;
-            *p_ub = start + limit * incr;
-          }
-
-          if (pr->ordered) {
-            pr->u.p.ordered_lower = init;
-            pr->u.p.ordered_upper = limit;
-#ifdef KMP_DEBUG
-            {
-              char *buff;
-              // create format specifiers before the debug output
-              buff = __kmp_str_format("__kmp_dispatch_next: T#%%d "
-                                      "ordered_lower:%%%s ordered_upper:%%%s\n",
-                                      traits_t<UT>::spec, traits_t<UT>::spec);
-              KD_TRACE(1000, (buff, gtid, pr->u.p.ordered_lower,
-                              pr->u.p.ordered_upper));
-              __kmp_str_free(&buff);
-            }
-#endif
-          } // if
-        } // if
-      } // case
-      break;
-
-      case kmp_sch_guided_iterative_chunked: {
-        T chunkspec = pr->u.p.parm1;
-        KD_TRACE(100, ("__kmp_dispatch_next: T#%d kmp_sch_guided_chunked "
-                       "iterative case\n",
-                       gtid));
-        trip = pr->u.p.tc;
-        // Start atomic part of calculations
-        while (1) {
-          ST remaining; // signed, because can be < 0
-          init = sh->u.s.iteration; // shared value
-          remaining = trip - init;
-          if (remaining <= 0) { // AC: need to compare with 0 first
-            // nothing to do, don't try atomic op
-            status = 0;
-            break;
-          }
-          if ((T)remaining <
-              pr->u.p.parm2) { // compare with K*nproc*(chunk+1), K=2 by default
-            // use dynamic-style shcedule
-            // atomically inrement iterations, get old value
-            init = test_then_add<ST>(RCAST(volatile ST *, &sh->u.s.iteration),
-                                     (ST)chunkspec);
-            remaining = trip - init;
-            if (remaining <= 0) {
-              status = 0; // all iterations got by other threads
-            } else { // got some iterations to work on
-              status = 1;
-              if ((T)remaining > chunkspec) {
-                limit = init + chunkspec - 1;
-              } else {
-                last = 1; // the last chunk
-                limit = init + remaining - 1;
-              } // if
-            } // if
-            break;
-          } // if
-          limit = init + (UT)(remaining *
-                              *(double *)&pr->u.p.parm3); // divide by K*nproc
-          if (compare_and_swap<ST>(RCAST(volatile ST *, &sh->u.s.iteration),
-                                   (ST)init, (ST)limit)) {
-            // CAS was successful, chunk obtained
-            status = 1;
-            --limit;
-            break;
-          } // if
-        } // while
-        if (status != 0) {
-          start = pr->u.p.lb;
-          incr = pr->u.p.st;
-          if (p_st != NULL)
-            *p_st = incr;
-          *p_lb = start + init * incr;
-          *p_ub = start + limit * incr;
-          if (pr->ordered) {
-            pr->u.p.ordered_lower = init;
-            pr->u.p.ordered_upper = limit;
-#ifdef KMP_DEBUG
-            {
-              char *buff;
-              // create format specifiers before the debug output
-              buff = __kmp_str_format("__kmp_dispatch_next: T#%%d "
-                                      "ordered_lower:%%%s ordered_upper:%%%s\n",
-                                      traits_t<UT>::spec, traits_t<UT>::spec);
-              KD_TRACE(1000, (buff, gtid, pr->u.p.ordered_lower,
-                              pr->u.p.ordered_upper));
-              __kmp_str_free(&buff);
-            }
-#endif
-          } // if
-        } else {
-          *p_lb = 0;
-          *p_ub = 0;
-          if (p_st != NULL)
-            *p_st = 0;
-        } // if
-      } // case
-      break;
-
-      case kmp_sch_guided_simd: {
-        // same as iterative but curr-chunk adjusted to be multiple of given
-        // chunk
-        T chunk = pr->u.p.parm1;
-        KD_TRACE(100, ("__kmp_dispatch_next: T#%d kmp_sch_guided_simd case\n",
-                       gtid));
-        trip = pr->u.p.tc;
-        // Start atomic part of calculations
-        while (1) {
-          ST remaining; // signed, because can be < 0
-          init = sh->u.s.iteration; // shared value
-          remaining = trip - init;
-          if (remaining <= 0) { // AC: need to compare with 0 first
-            status = 0; // nothing to do, don't try atomic op
-            break;
-          }
-          KMP_DEBUG_ASSERT(init % chunk == 0);
-          // compare with K*nproc*(chunk+1), K=2 by default
-          if ((T)remaining < pr->u.p.parm2) {
-            // use dynamic-style shcedule
-            // atomically inrement iterations, get old value
-            init = test_then_add<ST>(RCAST(volatile ST *, &sh->u.s.iteration),
-                                     (ST)chunk);
-            remaining = trip - init;
-            if (remaining <= 0) {
-              status = 0; // all iterations got by other threads
-            } else {
-              // got some iterations to work on
-              status = 1;
-              if ((T)remaining > chunk) {
-                limit = init + chunk - 1;
-              } else {
-                last = 1; // the last chunk
-                limit = init + remaining - 1;
-              } // if
-            } // if
-            break;
-          } // if
-          // divide by K*nproc
-          UT span = remaining * (*(double *)&pr->u.p.parm3);
-          UT rem = span % chunk;
-          if (rem) // adjust so that span%chunk == 0
-            span += chunk - rem;
-          limit = init + span;
-          if (compare_and_swap<ST>(RCAST(volatile ST *, &sh->u.s.iteration),
-                                   (ST)init, (ST)limit)) {
-            // CAS was successful, chunk obtained
-            status = 1;
-            --limit;
-            break;
-          } // if
-        } // while
-        if (status != 0) {
-          start = pr->u.p.lb;
-          incr = pr->u.p.st;
-          if (p_st != NULL)
-            *p_st = incr;
-          *p_lb = start + init * incr;
-          *p_ub = start + limit * incr;
-          if (pr->ordered) {
-            pr->u.p.ordered_lower = init;
-            pr->u.p.ordered_upper = limit;
-#ifdef KMP_DEBUG
-            {
-              char *buff;
-              // create format specifiers before the debug output
-              buff = __kmp_str_format("__kmp_dispatch_next: T#%%d "
-                                      "ordered_lower:%%%s ordered_upper:%%%s\n",
-                                      traits_t<UT>::spec, traits_t<UT>::spec);
-              KD_TRACE(1000, (buff, gtid, pr->u.p.ordered_lower,
-                              pr->u.p.ordered_upper));
-              __kmp_str_free(&buff);
-            }
-#endif
-          } // if
-        } else {
-          *p_lb = 0;
-          *p_ub = 0;
-          if (p_st != NULL)
-            *p_st = 0;
-        } // if
-      } // case
-      break;
-
-      case kmp_sch_guided_analytical_chunked: {
-        T chunkspec = pr->u.p.parm1;
-        UT chunkIdx;
-#if KMP_OS_WINDOWS && KMP_ARCH_X86
-        /* for storing original FPCW value for Windows* OS on
-           IA-32 architecture 8-byte version */
-        unsigned int oldFpcw;
-        unsigned int fpcwSet = 0;
-#endif
-        KD_TRACE(100, ("__kmp_dispatch_next: T#%d kmp_sch_guided_chunked "
-                       "analytical case\n",
-                       gtid));
-
-        trip = pr->u.p.tc;
-
-        KMP_DEBUG_ASSERT(th->th.th_team_nproc > 1);
-        KMP_DEBUG_ASSERT((2UL * chunkspec + 1) * (UT)th->th.th_team_nproc <
-                         trip);
-
-        while (1) { /* this while loop is a safeguard against unexpected zero
-                       chunk sizes */
-          chunkIdx = test_then_inc_acq<ST>((volatile ST *)&sh->u.s.iteration);
-          if (chunkIdx >= (UT)pr->u.p.parm2) {
-            --trip;
-            /* use dynamic-style scheduling */
-            init = chunkIdx * chunkspec + pr->u.p.count;
-            /* need to verify init > 0 in case of overflow in the above
-             * calculation */
-            if ((status = (init > 0 && init <= trip)) != 0) {
-              limit = init + chunkspec - 1;
-
-              if ((last = (limit >= trip)) != 0)
-                limit = trip;
-            }
-            break;
-          } else {
-/* use exponential-style scheduling */
-/* The following check is to workaround the lack of long double precision on
-   Windows* OS.
-   This check works around the possible effect that init != 0 for chunkIdx == 0.
- */
-#if KMP_OS_WINDOWS && KMP_ARCH_X86
-            /* If we haven't already done so, save original FPCW and set
-               precision to 64-bit, as Windows* OS on IA-32 architecture
-               defaults to 53-bit */
-            if (!fpcwSet) {
-              oldFpcw = _control87(0, 0);
-              _control87(_PC_64, _MCW_PC);
-              fpcwSet = 0x30000;
-            }
-#endif
-            if (chunkIdx) {
-              init = __kmp_dispatch_guided_remaining<T>(
-                  trip, *(DBL *)&pr->u.p.parm3, chunkIdx);
-              KMP_DEBUG_ASSERT(init);
-              init = trip - init;
-            } else
-              init = 0;
-            limit = trip - __kmp_dispatch_guided_remaining<T>(
-                               trip, *(DBL *)&pr->u.p.parm3, chunkIdx + 1);
-            KMP_ASSERT(init <= limit);
-            if (init < limit) {
-              KMP_DEBUG_ASSERT(limit <= trip);
-              --limit;
-              status = 1;
-              break;
-            } // if
-          } // if
-        } // while (1)
-#if KMP_OS_WINDOWS && KMP_ARCH_X86
-        /* restore FPCW if necessary
-           AC: check fpcwSet flag first because oldFpcw can be uninitialized
-           here */
-        if (fpcwSet && (oldFpcw & fpcwSet))
-          _control87(oldFpcw, _MCW_PC);
-#endif
-        if (status != 0) {
-          start = pr->u.p.lb;
-          incr = pr->u.p.st;
-          if (p_st != NULL)
-            *p_st = incr;
-          *p_lb = start + init * incr;
-          *p_ub = start + limit * incr;
-          if (pr->ordered) {
-            pr->u.p.ordered_lower = init;
-            pr->u.p.ordered_upper = limit;
-#ifdef KMP_DEBUG
-            {
-              char *buff;
-              // create format specifiers before the debug output
-              buff = __kmp_str_format("__kmp_dispatch_next: T#%%d "
-                                      "ordered_lower:%%%s ordered_upper:%%%s\n",
-                                      traits_t<UT>::spec, traits_t<UT>::spec);
-              KD_TRACE(1000, (buff, gtid, pr->u.p.ordered_lower,
-                              pr->u.p.ordered_upper));
-              __kmp_str_free(&buff);
-            }
-#endif
-          }
-        } else {
-          *p_lb = 0;
-          *p_ub = 0;
-          if (p_st != NULL)
-            *p_st = 0;
-        }
-      } // case
-      break;
-
-      case kmp_sch_trapezoidal: {
-        UT index;
-        T parm2 = pr->u.p.parm2;
-        T parm3 = pr->u.p.parm3;
-        T parm4 = pr->u.p.parm4;
-        KD_TRACE(100, ("__kmp_dispatch_next: T#%d kmp_sch_trapezoidal case\n",
-                       gtid));
-
-        index = test_then_inc<ST>((volatile ST *)&sh->u.s.iteration);
-
-        init = (index * ((2 * parm2) - (index - 1) * parm4)) / 2;
-        trip = pr->u.p.tc - 1;
-
-        if ((status = ((T)index < parm3 && init <= trip)) == 0) {
-          *p_lb = 0;
-          *p_ub = 0;
-          if (p_st != NULL)
-            *p_st = 0;
-        } else {
-          start = pr->u.p.lb;
-          limit = ((index + 1) * (2 * parm2 - index * parm4)) / 2 - 1;
-          incr = pr->u.p.st;
-
-          if ((last = (limit >= trip)) != 0)
-            limit = trip;
-
-          if (p_st != NULL)
-            *p_st = incr;
-
-          if (incr == 1) {
-            *p_lb = start + init;
-            *p_ub = start + limit;
-          } else {
-            *p_lb = start + init * incr;
-            *p_ub = start + limit * incr;
-          }
-
-          if (pr->ordered) {
-            pr->u.p.ordered_lower = init;
-            pr->u.p.ordered_upper = limit;
-#ifdef KMP_DEBUG
-            {
-              char *buff;
-              // create format specifiers before the debug output
-              buff = __kmp_str_format("__kmp_dispatch_next: T#%%d "
-                                      "ordered_lower:%%%s ordered_upper:%%%s\n",
-                                      traits_t<UT>::spec, traits_t<UT>::spec);
-              KD_TRACE(1000, (buff, gtid, pr->u.p.ordered_lower,
-                              pr->u.p.ordered_upper));
-              __kmp_str_free(&buff);
-            }
-#endif
-          } // if
-        } // if
-      } // case
-      break;
-      default: {
-        status = 0; // to avoid complaints on uninitialized variable use
-        __kmp_fatal(KMP_MSG(UnknownSchedTypeDetected), // Primary message
-                    KMP_HNT(GetNewerLibrary), // Hint
-                    __kmp_msg_null // Variadic argument list terminator
-                    );
-      } break;
-      } // switch
-    } // if tc == 0;
-
+    status = __kmp_dispatch_next_algorithm<T>(gtid, pr, sh, &last, p_lb, p_ub,
+                                              p_st, th->th.th_team_nproc,
+                                              th->th.th_info.ds.ds_tid);
+    // status == 0: no more iterations to execute
     if (status == 0) {
       UT num_done;
 
@@ -2312,7 +1901,7 @@ static int __kmp_dispatch_next(ident_t *
         buff = __kmp_str_format(
             "__kmp_dispatch_next: T#%%d increment num_done:%%%s\n",
             traits_t<UT>::spec);
-        KD_TRACE(100, (buff, gtid, sh->u.s.num_done));
+        KD_TRACE(10, (buff, gtid, sh->u.s.num_done));
         __kmp_str_free(&buff);
       }
 #endif
@@ -2341,7 +1930,7 @@ static int __kmp_dispatch_next(ident_t *
         sh->u.s.iteration = 0;
 
         /* TODO replace with general release procedure? */
-        if (pr->ordered) {
+        if (pr->flags.ordered) {
           sh->u.s.ordered_iteration = 0;
         }
 
@@ -2380,9 +1969,10 @@ static int __kmp_dispatch_next(ident_t *
     // create format specifiers before the debug output
     buff = __kmp_str_format(
         "__kmp_dispatch_next: T#%%d normal case: "
-        "p_lb:%%%s p_ub:%%%s p_st:%%%s p_last:%%p  returning:%%d\n",
+        "p_lb:%%%s p_ub:%%%s p_st:%%%s p_last:%%p (%%d) returning:%%d\n",
         traits_t<T>::spec, traits_t<T>::spec, traits_t<ST>::spec);
-    KD_TRACE(10, (buff, gtid, *p_lb, *p_ub, p_st ? *p_st : 0, p_last, status));
+    KD_TRACE(10, (buff, gtid, *p_lb, *p_ub, p_st ? *p_st : 0, p_last,
+                  (p_last ? *p_last : 0), status));
     __kmp_str_free(&buff);
   }
 #endif

Added: openmp/trunk/runtime/src/kmp_dispatch.h
URL: http://llvm.org/viewvc/llvm-project/openmp/trunk/runtime/src/kmp_dispatch.h?rev=336568&view=auto
==============================================================================
--- openmp/trunk/runtime/src/kmp_dispatch.h (added)
+++ openmp/trunk/runtime/src/kmp_dispatch.h Mon Jul  9 10:45:33 2018
@@ -0,0 +1,507 @@
+/*
+ * kmp_dispatch.h: dynamic scheduling - iteration initialization and dispatch.
+ */
+
+//===----------------------------------------------------------------------===//
+//
+//                     The LLVM Compiler Infrastructure
+//
+// This file is dual licensed under the MIT and the University of Illinois Open
+// Source Licenses. See LICENSE.txt for details.
+//
+//===----------------------------------------------------------------------===//
+
+#ifndef KMP_DISPATCH_H
+#define KMP_DISPATCH_H
+
+/* ------------------------------------------------------------------------ */
+/* ------------------------------------------------------------------------ */
+
+// Need to raise Win version from XP to Vista here for support of
+// InterlockedExchange64
+#if defined(_WIN32_WINNT) && defined(_M_IX86)
+#undef _WIN32_WINNT
+#define _WIN32_WINNT 0x0502
+#endif
+
+#include "kmp.h"
+#include "kmp_error.h"
+#include "kmp_i18n.h"
+#include "kmp_itt.h"
+#include "kmp_stats.h"
+#include "kmp_str.h"
+#if KMP_OS_WINDOWS && KMP_ARCH_X86
+#include <float.h>
+#endif
+
+#if OMPT_SUPPORT
+#include "ompt-internal.h"
+#include "ompt-specific.h"
+#endif
+
+/* ------------------------------------------------------------------------ */
+/* ------------------------------------------------------------------------ */
+
+template <typename T> struct dispatch_shared_info_template;
+template <typename T> struct dispatch_private_info_template;
+
+template <typename T>
+extern void __kmp_dispatch_init_algorithm(ident_t *loc, int gtid,
+                                          dispatch_private_info_template<T> *pr,
+                                          enum sched_type schedule, T lb, T ub,
+                                          typename traits_t<T>::signed_t st,
+#if USE_ITT_BUILD
+                                          kmp_uint64 *cur_chunk,
+#endif
+                                          typename traits_t<T>::signed_t chunk,
+                                          T nproc, T unit_id);
+template <typename T>
+extern int __kmp_dispatch_next_algorithm(
+    int gtid, dispatch_private_info_template<T> *pr,
+    dispatch_shared_info_template<T> volatile *sh, kmp_int32 *p_last, T *p_lb,
+    T *p_ub, typename traits_t<T>::signed_t *p_st, T nproc, T unit_id);
+
+void __kmp_dispatch_dxo_error(int *gtid_ref, int *cid_ref, ident_t *loc_ref);
+void __kmp_dispatch_deo_error(int *gtid_ref, int *cid_ref, ident_t *loc_ref);
+
+#if KMP_STATIC_STEAL_ENABLED
+
+// replaces dispatch_private_info{32,64} structures and
+// dispatch_private_info{32,64}_t types
+template <typename T> struct dispatch_private_infoXX_template {
+  typedef typename traits_t<T>::unsigned_t UT;
+  typedef typename traits_t<T>::signed_t ST;
+  UT count; // unsigned
+  T ub;
+  /* Adding KMP_ALIGN_CACHE here doesn't help / can hurt performance */
+  T lb;
+  ST st; // signed
+  UT tc; // unsigned
+  T static_steal_counter; // for static_steal only; maybe better to put after ub
+
+  /* parm[1-4] are used in different ways by different scheduling algorithms */
+
+  // KMP_ALIGN( 32 ) ensures ( if the KMP_ALIGN macro is turned on )
+  //    a) parm3 is properly aligned and
+  //    b) all parm1-4 are in the same cache line.
+  // Because of parm1-4 are used together, performance seems to be better
+  // if they are in the same line (not measured though).
+
+  struct KMP_ALIGN(32) { // compiler does not accept sizeof(T)*4
+    T parm1;
+    T parm2;
+    T parm3;
+    T parm4;
+  };
+
+  UT ordered_lower; // unsigned
+  UT ordered_upper; // unsigned
+#if KMP_OS_WINDOWS
+  T last_upper;
+#endif /* KMP_OS_WINDOWS */
+};
+
+#else /* KMP_STATIC_STEAL_ENABLED */
+
+// replaces dispatch_private_info{32,64} structures and
+// dispatch_private_info{32,64}_t types
+template <typename T> struct dispatch_private_infoXX_template {
+  typedef typename traits_t<T>::unsigned_t UT;
+  typedef typename traits_t<T>::signed_t ST;
+  T lb;
+  T ub;
+  ST st; // signed
+  UT tc; // unsigned
+
+  T parm1;
+  T parm2;
+  T parm3;
+  T parm4;
+
+  UT count; // unsigned
+
+  UT ordered_lower; // unsigned
+  UT ordered_upper; // unsigned
+#if KMP_OS_WINDOWS
+  T last_upper;
+#endif /* KMP_OS_WINDOWS */
+};
+#endif /* KMP_STATIC_STEAL_ENABLED */
+
+template <typename T> struct KMP_ALIGN_CACHE dispatch_private_info_template {
+  // duplicate alignment here, otherwise size of structure is not correct in our
+  // compiler
+  union KMP_ALIGN_CACHE private_info_tmpl {
+    dispatch_private_infoXX_template<T> p;
+    dispatch_private_info64_t p64;
+  } u;
+  enum sched_type schedule; /* scheduling algorithm */
+  kmp_sched_flags_t flags; /* flags (e.g., ordered, nomerge, etc.) */
+  kmp_uint32 ordered_bumped;
+  // to retain the structure size after making order
+  kmp_int32 ordered_dummy[KMP_MAX_ORDERED - 3];
+  dispatch_private_info *next; /* stack of buffers for nest of serial regions */
+  kmp_uint32 type_size;
+  enum cons_type pushed_ws;
+};
+
+// replaces dispatch_shared_info{32,64} structures and
+// dispatch_shared_info{32,64}_t types
+template <typename T> struct dispatch_shared_infoXX_template {
+  typedef typename traits_t<T>::unsigned_t UT;
+  /* chunk index under dynamic, number of idle threads under static-steal;
+     iteration index otherwise */
+  volatile UT iteration;
+  volatile UT num_done;
+  volatile UT ordered_iteration;
+  // to retain the structure size making ordered_iteration scalar
+  UT ordered_dummy[KMP_MAX_ORDERED - 3];
+};
+
+// replaces dispatch_shared_info structure and dispatch_shared_info_t type
+template <typename T> struct dispatch_shared_info_template {
+  typedef typename traits_t<T>::unsigned_t UT;
+  // we need union here to keep the structure size
+  union shared_info_tmpl {
+    dispatch_shared_infoXX_template<UT> s;
+    dispatch_shared_info64_t s64;
+  } u;
+  volatile kmp_uint32 buffer_index;
+#if OMP_45_ENABLED
+  volatile kmp_int32 doacross_buf_idx; // teamwise index
+  kmp_uint32 *doacross_flags; // array of iteration flags (0/1)
+  kmp_int32 doacross_num_done; // count finished threads
+#endif
+#if KMP_USE_HWLOC
+  // When linking with libhwloc, the ORDERED EPCC test slowsdown on big
+  // machines (> 48 cores). Performance analysis showed that a cache thrash
+  // was occurring and this padding helps alleviate the problem.
+  char padding[64];
+#endif
+};
+
+/* ------------------------------------------------------------------------ */
+/* ------------------------------------------------------------------------ */
+
+#undef USE_TEST_LOCKS
+
+// test_then_add template (general template should NOT be used)
+template <typename T> static __forceinline T test_then_add(volatile T *p, T d);
+
+template <>
+__forceinline kmp_int32 test_then_add<kmp_int32>(volatile kmp_int32 *p,
+                                                 kmp_int32 d) {
+  kmp_int32 r;
+  r = KMP_TEST_THEN_ADD32(p, d);
+  return r;
+}
+
+template <>
+__forceinline kmp_int64 test_then_add<kmp_int64>(volatile kmp_int64 *p,
+                                                 kmp_int64 d) {
+  kmp_int64 r;
+  r = KMP_TEST_THEN_ADD64(p, d);
+  return r;
+}
+
+// test_then_inc_acq template (general template should NOT be used)
+template <typename T> static __forceinline T test_then_inc_acq(volatile T *p);
+
+template <>
+__forceinline kmp_int32 test_then_inc_acq<kmp_int32>(volatile kmp_int32 *p) {
+  kmp_int32 r;
+  r = KMP_TEST_THEN_INC_ACQ32(p);
+  return r;
+}
+
+template <>
+__forceinline kmp_int64 test_then_inc_acq<kmp_int64>(volatile kmp_int64 *p) {
+  kmp_int64 r;
+  r = KMP_TEST_THEN_INC_ACQ64(p);
+  return r;
+}
+
+// test_then_inc template (general template should NOT be used)
+template <typename T> static __forceinline T test_then_inc(volatile T *p);
+
+template <>
+__forceinline kmp_int32 test_then_inc<kmp_int32>(volatile kmp_int32 *p) {
+  kmp_int32 r;
+  r = KMP_TEST_THEN_INC32(p);
+  return r;
+}
+
+template <>
+__forceinline kmp_int64 test_then_inc<kmp_int64>(volatile kmp_int64 *p) {
+  kmp_int64 r;
+  r = KMP_TEST_THEN_INC64(p);
+  return r;
+}
+
+// compare_and_swap template (general template should NOT be used)
+template <typename T>
+static __forceinline kmp_int32 compare_and_swap(volatile T *p, T c, T s);
+
+template <>
+__forceinline kmp_int32 compare_and_swap<kmp_int32>(volatile kmp_int32 *p,
+                                                    kmp_int32 c, kmp_int32 s) {
+  return KMP_COMPARE_AND_STORE_REL32(p, c, s);
+}
+
+template <>
+__forceinline kmp_int32 compare_and_swap<kmp_int64>(volatile kmp_int64 *p,
+                                                    kmp_int64 c, kmp_int64 s) {
+  return KMP_COMPARE_AND_STORE_REL64(p, c, s);
+}
+
+template <typename T> kmp_uint32 __kmp_ge(T value, T checker) {
+  return value >= checker;
+}
+template <typename T> kmp_uint32 __kmp_eq(T value, T checker) {
+  return value == checker;
+}
+
+/*
+    Spin wait loop that first does pause, then yield.
+    Waits until function returns non-zero when called with *spinner and check.
+    Does NOT put threads to sleep.
+    Arguments:
+        UT is unsigned 4- or 8-byte type
+        spinner - memory location to check value
+        checker - value which spinner is >, <, ==, etc.
+        pred - predicate function to perform binary comparison of some sort
+#if USE_ITT_BUILD
+        obj -- is higher-level synchronization object to report to ittnotify. It
+        is used to report locks consistently. For example, if lock is acquired
+        immediately, its address is reported to ittnotify via
+        KMP_FSYNC_ACQUIRED(). However, it lock cannot be acquired immediately
+        and lock routine calls to KMP_WAIT_YIELD(), the later should report the
+        same address, not an address of low-level spinner.
+#endif // USE_ITT_BUILD
+    TODO: make inline function (move to header file for icl)
+*/
+template <typename UT>
+static UT __kmp_wait_yield(volatile UT *spinner, UT checker,
+                           kmp_uint32 (*pred)(UT, UT)
+                               USE_ITT_BUILD_ARG(void *obj)) {
+  // note: we may not belong to a team at this point
+  volatile UT *spin = spinner;
+  UT check = checker;
+  kmp_uint32 spins;
+  kmp_uint32 (*f)(UT, UT) = pred;
+  UT r;
+
+  KMP_FSYNC_SPIN_INIT(obj, CCAST(UT *, spin));
+  KMP_INIT_YIELD(spins);
+  // main wait spin loop
+  while (!f(r = *spin, check)) {
+    KMP_FSYNC_SPIN_PREPARE(obj);
+    /* GEH - remove this since it was accidentally introduced when kmp_wait was
+       split.
+       It causes problems with infinite recursion because of exit lock */
+    /* if ( TCR_4(__kmp_global.g.g_done) && __kmp_global.g.g_abort)
+        __kmp_abort_thread(); */
+
+    // if we are oversubscribed,
+    // or have waited a bit (and KMP_LIBRARY=throughput, then yield
+    // pause is in the following code
+    KMP_YIELD(TCR_4(__kmp_nth) > __kmp_avail_proc);
+    KMP_YIELD_SPIN(spins);
+  }
+  KMP_FSYNC_SPIN_ACQUIRED(obj);
+  return r;
+}
+
+/* ------------------------------------------------------------------------ */
+/* ------------------------------------------------------------------------ */
+
+template <typename UT>
+void __kmp_dispatch_deo(int *gtid_ref, int *cid_ref, ident_t *loc_ref) {
+  typedef typename traits_t<UT>::signed_t ST;
+  dispatch_private_info_template<UT> *pr;
+
+  int gtid = *gtid_ref;
+  //    int  cid = *cid_ref;
+  kmp_info_t *th = __kmp_threads[gtid];
+  KMP_DEBUG_ASSERT(th->th.th_dispatch);
+
+  KD_TRACE(100, ("__kmp_dispatch_deo: T#%d called\n", gtid));
+  if (__kmp_env_consistency_check) {
+    pr = reinterpret_cast<dispatch_private_info_template<UT> *>(
+        th->th.th_dispatch->th_dispatch_pr_current);
+    if (pr->pushed_ws != ct_none) {
+#if KMP_USE_DYNAMIC_LOCK
+      __kmp_push_sync(gtid, ct_ordered_in_pdo, loc_ref, NULL, 0);
+#else
+      __kmp_push_sync(gtid, ct_ordered_in_pdo, loc_ref, NULL);
+#endif
+    }
+  }
+
+  if (!th->th.th_team->t.t_serialized) {
+    dispatch_shared_info_template<UT> *sh =
+        reinterpret_cast<dispatch_shared_info_template<UT> *>(
+            th->th.th_dispatch->th_dispatch_sh_current);
+    UT lower;
+
+    if (!__kmp_env_consistency_check) {
+      pr = reinterpret_cast<dispatch_private_info_template<UT> *>(
+          th->th.th_dispatch->th_dispatch_pr_current);
+    }
+    lower = pr->u.p.ordered_lower;
+
+#if !defined(KMP_GOMP_COMPAT)
+    if (__kmp_env_consistency_check) {
+      if (pr->ordered_bumped) {
+        struct cons_header *p = __kmp_threads[gtid]->th.th_cons;
+        __kmp_error_construct2(kmp_i18n_msg_CnsMultipleNesting,
+                               ct_ordered_in_pdo, loc_ref,
+                               &p->stack_data[p->w_top]);
+      }
+    }
+#endif /* !defined(KMP_GOMP_COMPAT) */
+
+    KMP_MB();
+#ifdef KMP_DEBUG
+    {
+      char *buff;
+      // create format specifiers before the debug output
+      buff = __kmp_str_format("__kmp_dispatch_deo: T#%%d before wait: "
+                              "ordered_iter:%%%s lower:%%%s\n",
+                              traits_t<UT>::spec, traits_t<UT>::spec);
+      KD_TRACE(1000, (buff, gtid, sh->u.s.ordered_iteration, lower));
+      __kmp_str_free(&buff);
+    }
+#endif
+    __kmp_wait_yield<UT>(&sh->u.s.ordered_iteration, lower,
+                         __kmp_ge<UT> USE_ITT_BUILD_ARG(NULL));
+    KMP_MB(); /* is this necessary? */
+#ifdef KMP_DEBUG
+    {
+      char *buff;
+      // create format specifiers before the debug output
+      buff = __kmp_str_format("__kmp_dispatch_deo: T#%%d after wait: "
+                              "ordered_iter:%%%s lower:%%%s\n",
+                              traits_t<UT>::spec, traits_t<UT>::spec);
+      KD_TRACE(1000, (buff, gtid, sh->u.s.ordered_iteration, lower));
+      __kmp_str_free(&buff);
+    }
+#endif
+  }
+  KD_TRACE(100, ("__kmp_dispatch_deo: T#%d returned\n", gtid));
+}
+
+template <typename UT>
+void __kmp_dispatch_dxo(int *gtid_ref, int *cid_ref, ident_t *loc_ref) {
+  typedef typename traits_t<UT>::signed_t ST;
+  dispatch_private_info_template<UT> *pr;
+
+  int gtid = *gtid_ref;
+  //    int  cid = *cid_ref;
+  kmp_info_t *th = __kmp_threads[gtid];
+  KMP_DEBUG_ASSERT(th->th.th_dispatch);
+
+  KD_TRACE(100, ("__kmp_dispatch_dxo: T#%d called\n", gtid));
+  if (__kmp_env_consistency_check) {
+    pr = reinterpret_cast<dispatch_private_info_template<UT> *>(
+        th->th.th_dispatch->th_dispatch_pr_current);
+    if (pr->pushed_ws != ct_none) {
+      __kmp_pop_sync(gtid, ct_ordered_in_pdo, loc_ref);
+    }
+  }
+
+  if (!th->th.th_team->t.t_serialized) {
+    dispatch_shared_info_template<UT> *sh =
+        reinterpret_cast<dispatch_shared_info_template<UT> *>(
+            th->th.th_dispatch->th_dispatch_sh_current);
+
+    if (!__kmp_env_consistency_check) {
+      pr = reinterpret_cast<dispatch_private_info_template<UT> *>(
+          th->th.th_dispatch->th_dispatch_pr_current);
+    }
+
+    KMP_FSYNC_RELEASING(CCAST(UT *, &sh->u.s.ordered_iteration));
+#if !defined(KMP_GOMP_COMPAT)
+    if (__kmp_env_consistency_check) {
+      if (pr->ordered_bumped != 0) {
+        struct cons_header *p = __kmp_threads[gtid]->th.th_cons;
+        /* How to test it? - OM */
+        __kmp_error_construct2(kmp_i18n_msg_CnsMultipleNesting,
+                               ct_ordered_in_pdo, loc_ref,
+                               &p->stack_data[p->w_top]);
+      }
+    }
+#endif /* !defined(KMP_GOMP_COMPAT) */
+
+    KMP_MB(); /* Flush all pending memory write invalidates.  */
+
+    pr->ordered_bumped += 1;
+
+    KD_TRACE(1000,
+             ("__kmp_dispatch_dxo: T#%d bumping ordered ordered_bumped=%d\n",
+              gtid, pr->ordered_bumped));
+
+    KMP_MB(); /* Flush all pending memory write invalidates.  */
+
+    /* TODO use general release procedure? */
+    test_then_inc<ST>((volatile ST *)&sh->u.s.ordered_iteration);
+
+    KMP_MB(); /* Flush all pending memory write invalidates.  */
+  }
+  KD_TRACE(100, ("__kmp_dispatch_dxo: T#%d returned\n", gtid));
+}
+
+/* Computes and returns x to the power of y, where y must a non-negative integer
+ */
+template <typename UT>
+static __forceinline long double __kmp_pow(long double x, UT y) {
+  long double s = 1.0L;
+
+  KMP_DEBUG_ASSERT(x > 0.0 && x < 1.0);
+  // KMP_DEBUG_ASSERT(y >= 0); // y is unsigned
+  while (y) {
+    if (y & 1)
+      s *= x;
+    x *= x;
+    y >>= 1;
+  }
+  return s;
+}
+
+/* Computes and returns the number of unassigned iterations after idx chunks
+   have been assigned
+   (the total number of unassigned iterations in chunks with index greater than
+   or equal to idx).
+   __forceinline seems to be broken so that if we __forceinline this function,
+   the behavior is wrong
+   (one of the unit tests, sch_guided_analytical_basic.cpp, fails)
+*/
+template <typename T>
+static __inline typename traits_t<T>::unsigned_t
+__kmp_dispatch_guided_remaining(T tc, typename traits_t<T>::floating_t base,
+                                typename traits_t<T>::unsigned_t idx) {
+  /* Note: On Windows* OS on IA-32 architecture and Intel(R) 64, at
+     least for ICL 8.1, long double arithmetic may not really have
+     long double precision, even with /Qlong_double.  Currently, we
+     workaround that in the caller code, by manipulating the FPCW for
+     Windows* OS on IA-32 architecture.  The lack of precision is not
+     expected to be a correctness issue, though.
+  */
+  typedef typename traits_t<T>::unsigned_t UT;
+
+  long double x = tc * __kmp_pow<UT>(base, idx);
+  UT r = (UT)x;
+  if (x == r)
+    return r;
+  return r + 1;
+}
+
+// Parameters of the guided-iterative algorithm:
+//   p2 = n * nproc * ( chunk + 1 )  // point of switching to dynamic
+//   p3 = 1 / ( n * nproc )          // remaining iterations multiplier
+// by default n = 2. For example with n = 3 the chunks distribution will be more
+// flat.
+// With n = 1 first chunk is the same as for static schedule, e.g. trip / nproc.
+static const int guided_int_param = 2;
+static const double guided_flt_param = 0.5; // = 1.0 / guided_int_param;
+#endif // KMP_DISPATCH_H




More information about the Openmp-commits mailing list