[OpenMP] Implement GOMP task reductions

Implement the remaining GOMP_* functions to support task reductions
in taskgroup, parallel, loop, and taskloop constructs.  The unused mem
argument to many of the work-sharing constructs has to do with the
scan() directive/ inscan() modifier.  If mem is set, each function
will call KMP_FATAL() and tell the user scan/inscan is unsupported.  The
GOMP reduction implementation is kept separate from our implementation
because of how GOMP presents reduction data and computes the reductions.
GOMP expects the privatized copies to be present even after a #pragma
omp parallel reduction(task:...) region has ended so the data is stored
inside GOMP's uintptr_t* data pseudo-structure.  This style is tightly
coupled with GCC compiler codegen.  There also isn't any init(),
combiner(), fini() functions in GOMP's codegen so the two
implementations were to disparate to try to wrap GOMP's around our own.

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

GitOrigin-RevId: 4457565757ea91207b7e5f2ce7b7bf173bfd2c0c
diff --git a/runtime/src/kmp.h b/runtime/src/kmp.h
index c37e1d9..5084640 100644
--- a/runtime/src/kmp.h
+++ b/runtime/src/kmp.h
@@ -2252,6 +2252,7 @@
   // Block of data to perform task reduction
   void *reduce_data; // reduction related info
   kmp_int32 reduce_num_data; // number of data items to reduce
+  uintptr_t *gomp_data; // gomp reduction data
 } kmp_taskgroup_t;
 
 // forward declarations
diff --git a/runtime/src/kmp_ftn_os.h b/runtime/src/kmp_ftn_os.h
index c122dce..5b9e396 100644
--- a/runtime/src/kmp_ftn_os.h
+++ b/runtime/src/kmp_ftn_os.h
@@ -697,5 +697,20 @@
   GOMP_parallel_loop_maybe_nonmonotonic_runtime
 #define KMP_API_NAME_GOMP_TEAMS_REG GOMP_teams_reg
 #define KMP_API_NAME_GOMP_TASKWAIT_DEPEND GOMP_taskwait_depend
+#define KMP_API_NAME_GOMP_TASKGROUP_REDUCTION_REGISTER                         \
+  GOMP_taskgroup_reduction_register
+#define KMP_API_NAME_GOMP_TASKGROUP_REDUCTION_UNREGISTER                       \
+  GOMP_taskgroup_reduction_unregister
+#define KMP_API_NAME_GOMP_TASK_REDUCTION_REMAP GOMP_task_reduction_remap
+#define KMP_API_NAME_GOMP_PARALLEL_REDUCTIONS GOMP_parallel_reductions
+#define KMP_API_NAME_GOMP_LOOP_START GOMP_loop_start
+#define KMP_API_NAME_GOMP_LOOP_ULL_START GOMP_loop_ull_start
+#define KMP_API_NAME_GOMP_LOOP_DOACROSS_START GOMP_loop_doacross_start
+#define KMP_API_NAME_GOMP_LOOP_ULL_DOACROSS_START GOMP_loop_ull_doacross_start
+#define KMP_API_NAME_GOMP_LOOP_ORDERED_START GOMP_loop_ordered_start
+#define KMP_API_NAME_GOMP_LOOP_ULL_ORDERED_START GOMP_loop_ull_ordered_start
+#define KMP_API_NAME_GOMP_SECTIONS2_START GOMP_sections2_start
+#define KMP_API_NAME_GOMP_WORKSHARE_TASK_REDUCTION_UNREGISTER                  \
+  GOMP_workshare_task_reduction_unregister
 
 #endif /* KMP_FTN_OS_H */
diff --git a/runtime/src/kmp_gsupport.cpp b/runtime/src/kmp_gsupport.cpp
index e576413..d4e0c5b 100644
--- a/runtime/src/kmp_gsupport.cpp
+++ b/runtime/src/kmp_gsupport.cpp
@@ -1688,6 +1688,9 @@
   }
 }
 
+void KMP_EXPAND_NAME(KMP_API_NAME_GOMP_TASKGROUP_REDUCTION_REGISTER)(
+    uintptr_t *);
+
 #ifdef __cplusplus
 } // extern "C"
 #endif
@@ -1707,6 +1710,7 @@
   int if_val = gomp_flags & (1u << 10);
   int nogroup = gomp_flags & (1u << 11);
   int up = gomp_flags & (1u << 8);
+  int reductions = gomp_flags & (1u << 12);
   p_task_dup_t task_dup = NULL;
   kmp_tasking_flags_t *input_flags = (kmp_tasking_flags_t *)&flags;
 #ifdef KMP_DEBUG
@@ -1778,9 +1782,31 @@
   loop_bounds = (T *)task->shareds;
   loop_bounds[0] = start;
   loop_bounds[1] = end + (up ? -1 : 1);
+
+  if (!nogroup) {
+#if OMPT_SUPPORT && OMPT_OPTIONAL
+    OMPT_STORE_RETURN_ADDRESS(gtid);
+#endif
+    __kmpc_taskgroup(&loc, gtid);
+    if (reductions) {
+      // The data pointer points to lb, ub, then reduction data
+      struct data_t {
+        T a, b;
+        uintptr_t *d;
+      };
+      uintptr_t *d = ((data_t *)data)->d;
+      KMP_EXPAND_NAME(KMP_API_NAME_GOMP_TASKGROUP_REDUCTION_REGISTER)(d);
+    }
+  }
   __kmpc_taskloop(&loc, gtid, task, if_val, (kmp_uint64 *)&(loop_bounds[0]),
-                  (kmp_uint64 *)&(loop_bounds[1]), (kmp_int64)step, nogroup,
-                  sched, (kmp_uint64)num_tasks, (void *)task_dup);
+                  (kmp_uint64 *)&(loop_bounds[1]), (kmp_int64)step, 1, sched,
+                  (kmp_uint64)num_tasks, (void *)task_dup);
+  if (!nogroup) {
+#if OMPT_SUPPORT && OMPT_OPTIONAL
+    OMPT_STORE_RETURN_ADDRESS(gtid);
+#endif
+    __kmpc_end_taskgroup(&loc, gtid);
+  }
 }
 
 // 4 byte version of GOMP_doacross_post
@@ -1912,6 +1938,488 @@
   KA_TRACE(20, ("GOMP_taskwait_depend exit: T#%d\n", gtid));
 }
 
+static inline void
+__kmp_GOMP_taskgroup_reduction_register(uintptr_t *data, kmp_taskgroup_t *tg,
+                                        int nthreads,
+                                        uintptr_t *allocated = nullptr) {
+  KMP_ASSERT(data);
+  KMP_ASSERT(nthreads > 0);
+  // Have private copy pointers point to previously allocated
+  // reduction data or allocate new data here
+  if (allocated) {
+    data[2] = allocated[2];
+    data[6] = allocated[6];
+  } else {
+    data[2] = (uintptr_t)__kmp_allocate(nthreads * data[1]);
+    data[6] = data[2] + (nthreads * data[1]);
+  }
+  if (tg)
+    tg->gomp_data = data;
+}
+
+void KMP_EXPAND_NAME(KMP_API_NAME_GOMP_TASKGROUP_REDUCTION_REGISTER)(
+    uintptr_t *data) {
+  int gtid = __kmp_entry_gtid();
+  KA_TRACE(20, ("GOMP_taskgroup_reduction_register: T#%d\n", gtid));
+  kmp_info_t *thread = __kmp_threads[gtid];
+  kmp_taskgroup_t *tg = thread->th.th_current_task->td_taskgroup;
+  int nthreads = thread->th.th_team_nproc;
+  __kmp_GOMP_taskgroup_reduction_register(data, tg, nthreads);
+}
+
+void KMP_EXPAND_NAME(KMP_API_NAME_GOMP_TASKGROUP_REDUCTION_UNREGISTER)(
+    uintptr_t *data) {
+  KA_TRACE(20,
+           ("GOMP_taskgroup_reduction_unregister: T#%d\n", __kmp_get_gtid()));
+  KMP_ASSERT(data && data[2]);
+  __kmp_free((void *)data[2]);
+}
+
+// Search through reduction data and set ptrs[] elements
+// to proper privatized copy address
+void KMP_EXPAND_NAME(KMP_API_NAME_GOMP_TASK_REDUCTION_REMAP)(size_t cnt,
+                                                             size_t cntorig,
+                                                             void **ptrs) {
+  int gtid = __kmp_entry_gtid();
+  KA_TRACE(20, ("GOMP_task_reduction_remap: T#%d\n", gtid));
+  kmp_info_t *thread = __kmp_threads[gtid];
+  kmp_int32 tid = __kmp_get_tid();
+  for (size_t i = 0; i < cnt; ++i) {
+    uintptr_t address = (uintptr_t)ptrs[i];
+    void *propagated_address = NULL;
+    void *mapped_address = NULL;
+    // Check taskgroups reduce data
+    kmp_taskgroup_t *tg = thread->th.th_current_task->td_taskgroup;
+    while (tg) {
+      uintptr_t *gomp_data = tg->gomp_data;
+      if (!gomp_data) {
+        tg = tg->parent;
+        continue;
+      }
+      // Check the shared addresses list
+      size_t num_vars = (size_t)gomp_data[0];
+      uintptr_t per_thread_size = gomp_data[1];
+      uintptr_t reduce_data = gomp_data[2];
+      uintptr_t end_reduce_data = gomp_data[6];
+      for (size_t j = 0; j < num_vars; ++j) {
+        uintptr_t *entry = gomp_data + 7 + 3 * j;
+        if (entry[0] == address) {
+          uintptr_t offset = entry[1];
+          mapped_address =
+              (void *)(reduce_data + tid * per_thread_size + offset);
+          if (i < cntorig)
+            propagated_address = (void *)entry[0];
+          break;
+        }
+      }
+      if (mapped_address)
+        break;
+      // Check if address is within privatized copies range
+      if (!mapped_address && address >= reduce_data &&
+          address < end_reduce_data) {
+        uintptr_t offset = (address - reduce_data) % per_thread_size;
+        mapped_address = (void *)(reduce_data + tid * per_thread_size + offset);
+        if (i < cntorig) {
+          for (size_t j = 0; j < num_vars; ++j) {
+            uintptr_t *entry = gomp_data + 7 + 3 * j;
+            if (entry[1] == offset) {
+              propagated_address = (void *)entry[0];
+              break;
+            }
+          }
+        }
+      }
+      if (mapped_address)
+        break;
+      tg = tg->parent;
+    }
+    KMP_ASSERT(mapped_address);
+    ptrs[i] = mapped_address;
+    if (i < cntorig) {
+      KMP_ASSERT(propagated_address);
+      ptrs[cnt + i] = propagated_address;
+    }
+  }
+}
+
+static void __kmp_GOMP_init_reductions(int gtid, uintptr_t *data, int is_ws) {
+  kmp_info_t *thr = __kmp_threads[gtid];
+  kmp_team_t *team = thr->th.th_team;
+  // First start a taskgroup
+  __kmpc_taskgroup(NULL, gtid);
+  // Then setup reduction data
+  void *reduce_data = KMP_ATOMIC_LD_RLX(&team->t.t_tg_reduce_data[is_ws]);
+  if (reduce_data == NULL &&
+      __kmp_atomic_compare_store(&team->t.t_tg_reduce_data[is_ws], reduce_data,
+                                 (void *)1)) {
+    // Single thread enters this block to initialize common reduction data
+    KMP_DEBUG_ASSERT(reduce_data == NULL);
+    __kmp_GOMP_taskgroup_reduction_register(data, NULL, thr->th.th_team_nproc);
+    KMP_ATOMIC_ST_REL(&team->t.t_tg_fini_counter[is_ws], 0);
+    KMP_ATOMIC_ST_REL(&team->t.t_tg_reduce_data[is_ws], (void *)data);
+  } else {
+    // Wait for task reduction initialization
+    while ((reduce_data = KMP_ATOMIC_LD_ACQ(
+                &team->t.t_tg_reduce_data[is_ws])) == (void *)1) {
+      KMP_CPU_PAUSE();
+    }
+    KMP_DEBUG_ASSERT(reduce_data > (void *)1); // should be valid pointer here
+  }
+  // For worksharing constructs, each thread has its own reduction structure.
+  // Have each reduction structure point to same privatized copies of vars.
+  // For parallel, each thread points to same reduction structure and privatized
+  // copies of vars
+  if (is_ws) {
+    __kmp_GOMP_taskgroup_reduction_register(
+        data, NULL, thr->th.th_team_nproc,
+        (uintptr_t *)KMP_ATOMIC_LD_ACQ(&team->t.t_tg_reduce_data[is_ws]));
+  }
+  kmp_taskgroup_t *tg = thr->th.th_current_task->td_taskgroup;
+  tg->gomp_data = data;
+}
+
+static unsigned
+__kmp_GOMP_par_reductions_microtask_wrapper(int *gtid, int *npr,
+                                            void (*task)(void *), void *data) {
+  kmp_info_t *thr = __kmp_threads[*gtid];
+  kmp_team_t *team = thr->th.th_team;
+  uintptr_t *reduce_data = *(uintptr_t **)data;
+  __kmp_GOMP_init_reductions(*gtid, reduce_data, 0);
+
+#if OMPT_SUPPORT
+  ompt_frame_t *ompt_frame;
+  ompt_state_t enclosing_state;
+
+  if (ompt_enabled.enabled) {
+    // save enclosing task state; set current state for task
+    enclosing_state = thr->th.ompt_thread_info.state;
+    thr->th.ompt_thread_info.state = ompt_state_work_parallel;
+
+    // set task frame
+    __ompt_get_task_info_internal(0, NULL, NULL, &ompt_frame, NULL, NULL);
+    ompt_frame->exit_frame.ptr = OMPT_GET_FRAME_ADDRESS(0);
+  }
+#endif
+
+  task(data);
+
+#if OMPT_SUPPORT
+  if (ompt_enabled.enabled) {
+    // clear task frame
+    ompt_frame->exit_frame = ompt_data_none;
+
+    // restore enclosing state
+    thr->th.ompt_thread_info.state = enclosing_state;
+  }
+#endif
+  __kmpc_end_taskgroup(NULL, *gtid);
+  // if last thread out, then reset the team's reduce data
+  // the GOMP_taskgroup_reduction_unregister() function will deallocate
+  // private copies after reduction calculations take place.
+  int count = KMP_ATOMIC_INC(&team->t.t_tg_fini_counter[0]);
+  if (count == thr->th.th_team_nproc - 1) {
+    KMP_ATOMIC_ST_REL(&team->t.t_tg_reduce_data[0], NULL);
+    KMP_ATOMIC_ST_REL(&team->t.t_tg_fini_counter[0], 0);
+  }
+  return (unsigned)thr->th.th_team_nproc;
+}
+
+unsigned KMP_EXPAND_NAME(KMP_API_NAME_GOMP_PARALLEL_REDUCTIONS)(
+    void (*task)(void *), void *data, unsigned num_threads,
+    unsigned int flags) {
+  MKLOC(loc, "GOMP_parallel_reductions");
+  int gtid = __kmp_entry_gtid();
+  KA_TRACE(20, ("GOMP_parallel_reductions: T#%d\n", gtid));
+  __kmp_GOMP_fork_call(&loc, gtid, num_threads, flags, task,
+                       (microtask_t)__kmp_GOMP_par_reductions_microtask_wrapper,
+                       2, task, data);
+  unsigned retval =
+      __kmp_GOMP_par_reductions_microtask_wrapper(&gtid, NULL, task, data);
+  KMP_EXPAND_NAME(KMP_API_NAME_GOMP_PARALLEL_END)();
+  KA_TRACE(20, ("GOMP_parallel_reductions exit: T#%d\n", gtid));
+  return retval;
+}
+
+bool KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_START)(
+    long start, long end, long incr, long sched, long chunk_size, long *istart,
+    long *iend, uintptr_t *reductions, void **mem) {
+  int status = 0;
+  int gtid = __kmp_entry_gtid();
+  KA_TRACE(20, ("GOMP_loop_start: T#%d, reductions: %p\n", gtid, reductions));
+  if (reductions)
+    __kmp_GOMP_init_reductions(gtid, reductions, 1);
+  if (mem)
+    KMP_FATAL(GompFeatureNotSupported, "scan");
+  if (istart == NULL)
+    return true;
+  const long MONOTONIC_FLAG = (long)(kmp_sched_monotonic);
+  long monotonic = sched & MONOTONIC_FLAG;
+  sched &= ~MONOTONIC_FLAG;
+  if (sched == 0) {
+    if (monotonic)
+      status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_RUNTIME_START)(
+          start, end, incr, istart, iend);
+    else
+      status = KMP_EXPAND_NAME(
+          KMP_API_NAME_GOMP_LOOP_MAYBE_NONMONOTONIC_RUNTIME_START)(
+          start, end, incr, istart, iend);
+  } else if (sched == 1) {
+    status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_STATIC_START)(
+        start, end, incr, chunk_size, istart, iend);
+  } else if (sched == 2) {
+    if (monotonic)
+      status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_DYNAMIC_START)(
+          start, end, incr, chunk_size, istart, iend);
+    else
+      status =
+          KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_NONMONOTONIC_DYNAMIC_START)(
+              start, end, incr, chunk_size, istart, iend);
+  } else if (sched == 3) {
+    if (monotonic)
+      status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_GUIDED_START)(
+          start, end, incr, chunk_size, istart, iend);
+    else
+      status =
+          KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_NONMONOTONIC_GUIDED_START)(
+              start, end, incr, chunk_size, istart, iend);
+  } else if (sched == 4) {
+    status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_NONMONOTONIC_RUNTIME_START)(
+        start, end, incr, istart, iend);
+  } else {
+    KMP_ASSERT(0);
+  }
+  return status;
+}
+
+bool KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ULL_START)(
+    bool up, unsigned long long start, unsigned long long end,
+    unsigned long long incr, long sched, unsigned long long chunk_size,
+    unsigned long long *istart, unsigned long long *iend, uintptr_t *reductions,
+    void **mem) {
+  int status = 0;
+  int gtid = __kmp_entry_gtid();
+  KA_TRACE(20,
+           ("GOMP_loop_ull_start: T#%d, reductions: %p\n", gtid, reductions));
+  if (reductions)
+    __kmp_GOMP_init_reductions(gtid, reductions, 1);
+  if (mem)
+    KMP_FATAL(GompFeatureNotSupported, "scan");
+  if (istart == NULL)
+    return true;
+  const long MONOTONIC_FLAG = (long)(kmp_sched_monotonic);
+  long monotonic = sched & MONOTONIC_FLAG;
+  sched &= ~MONOTONIC_FLAG;
+  if (sched == 0) {
+    if (monotonic)
+      status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ULL_RUNTIME_START)(
+          up, start, end, incr, istart, iend);
+    else
+      status = KMP_EXPAND_NAME(
+          KMP_API_NAME_GOMP_LOOP_ULL_MAYBE_NONMONOTONIC_RUNTIME_START)(
+          up, start, end, incr, istart, iend);
+  } else if (sched == 1) {
+    status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ULL_STATIC_START)(
+        up, start, end, incr, chunk_size, istart, iend);
+  } else if (sched == 2) {
+    if (monotonic)
+      status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ULL_DYNAMIC_START)(
+          up, start, end, incr, chunk_size, istart, iend);
+    else
+      status = KMP_EXPAND_NAME(
+          KMP_API_NAME_GOMP_LOOP_ULL_NONMONOTONIC_DYNAMIC_START)(
+          up, start, end, incr, chunk_size, istart, iend);
+  } else if (sched == 3) {
+    if (monotonic)
+      status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ULL_GUIDED_START)(
+          up, start, end, incr, chunk_size, istart, iend);
+    else
+      status =
+          KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ULL_NONMONOTONIC_GUIDED_START)(
+              up, start, end, incr, chunk_size, istart, iend);
+  } else if (sched == 4) {
+    status =
+        KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ULL_NONMONOTONIC_RUNTIME_START)(
+            up, start, end, incr, istart, iend);
+  } else {
+    KMP_ASSERT(0);
+  }
+  return status;
+}
+
+bool KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_DOACROSS_START)(
+    unsigned ncounts, long *counts, long sched, long chunk_size, long *istart,
+    long *iend, uintptr_t *reductions, void **mem) {
+  int status = 0;
+  int gtid = __kmp_entry_gtid();
+  KA_TRACE(20, ("GOMP_loop_doacross_start: T#%d, reductions: %p\n", gtid,
+                reductions));
+  if (reductions)
+    __kmp_GOMP_init_reductions(gtid, reductions, 1);
+  if (mem)
+    KMP_FATAL(GompFeatureNotSupported, "scan");
+  if (istart == NULL)
+    return true;
+  // Ignore any monotonic flag
+  const long MONOTONIC_FLAG = (long)(kmp_sched_monotonic);
+  sched &= ~MONOTONIC_FLAG;
+  if (sched == 0) {
+    status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_DOACROSS_RUNTIME_START)(
+        ncounts, counts, istart, iend);
+  } else if (sched == 1) {
+    status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_DOACROSS_STATIC_START)(
+        ncounts, counts, chunk_size, istart, iend);
+  } else if (sched == 2) {
+    status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_DOACROSS_DYNAMIC_START)(
+        ncounts, counts, chunk_size, istart, iend);
+  } else if (sched == 3) {
+    status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_DOACROSS_GUIDED_START)(
+        ncounts, counts, chunk_size, istart, iend);
+  } else {
+    KMP_ASSERT(0);
+  }
+  return status;
+}
+
+bool KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ULL_DOACROSS_START)(
+    unsigned ncounts, unsigned long long *counts, long sched,
+    unsigned long long chunk_size, unsigned long long *istart,
+    unsigned long long *iend, uintptr_t *reductions, void **mem) {
+  int status = 0;
+  int gtid = __kmp_entry_gtid();
+  KA_TRACE(20, ("GOMP_loop_ull_doacross_start: T#%d, reductions: %p\n", gtid,
+                reductions));
+  if (reductions)
+    __kmp_GOMP_init_reductions(gtid, reductions, 1);
+  if (mem)
+    KMP_FATAL(GompFeatureNotSupported, "scan");
+  if (istart == NULL)
+    return true;
+  // Ignore any monotonic flag
+  const long MONOTONIC_FLAG = (long)(kmp_sched_monotonic);
+  sched &= ~MONOTONIC_FLAG;
+  if (sched == 0) {
+    status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ULL_DOACROSS_RUNTIME_START)(
+        ncounts, counts, istart, iend);
+  } else if (sched == 1) {
+    status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ULL_DOACROSS_STATIC_START)(
+        ncounts, counts, chunk_size, istart, iend);
+  } else if (sched == 2) {
+    status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ULL_DOACROSS_DYNAMIC_START)(
+        ncounts, counts, chunk_size, istart, iend);
+  } else if (sched == 3) {
+    status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ULL_DOACROSS_GUIDED_START)(
+        ncounts, counts, chunk_size, istart, iend);
+  } else {
+    KMP_ASSERT(0);
+  }
+  return status;
+}
+
+bool KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ORDERED_START)(
+    long start, long end, long incr, long sched, long chunk_size, long *istart,
+    long *iend, uintptr_t *reductions, void **mem) {
+  int status = 0;
+  int gtid = __kmp_entry_gtid();
+  KA_TRACE(20, ("GOMP_loop_ordered_start: T#%d, reductions: %p\n", gtid,
+                reductions));
+  if (reductions)
+    __kmp_GOMP_init_reductions(gtid, reductions, 1);
+  if (mem)
+    KMP_FATAL(GompFeatureNotSupported, "scan");
+  if (istart == NULL)
+    return true;
+  // Ignore any monotonic flag
+  const long MONOTONIC_FLAG = (long)(kmp_sched_monotonic);
+  sched &= ~MONOTONIC_FLAG;
+  if (sched == 0) {
+    status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ORDERED_RUNTIME_START)(
+        start, end, incr, istart, iend);
+  } else if (sched == 1) {
+    status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ORDERED_STATIC_START)(
+        start, end, incr, chunk_size, istart, iend);
+  } else if (sched == 2) {
+    status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ORDERED_DYNAMIC_START)(
+        start, end, incr, chunk_size, istart, iend);
+  } else if (sched == 3) {
+    status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ORDERED_GUIDED_START)(
+        start, end, incr, chunk_size, istart, iend);
+  } else {
+    KMP_ASSERT(0);
+  }
+  return status;
+}
+
+bool KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ULL_ORDERED_START)(
+    bool up, unsigned long long start, unsigned long long end,
+    unsigned long long incr, long sched, unsigned long long chunk_size,
+    unsigned long long *istart, unsigned long long *iend, uintptr_t *reductions,
+    void **mem) {
+  int status = 0;
+  int gtid = __kmp_entry_gtid();
+  KA_TRACE(20, ("GOMP_loop_ull_ordered_start: T#%d, reductions: %p\n", gtid,
+                reductions));
+  if (reductions)
+    __kmp_GOMP_init_reductions(gtid, reductions, 1);
+  if (mem)
+    KMP_FATAL(GompFeatureNotSupported, "scan");
+  if (istart == NULL)
+    return true;
+  // Ignore any monotonic flag
+  const long MONOTONIC_FLAG = (long)(kmp_sched_monotonic);
+  sched &= ~MONOTONIC_FLAG;
+  if (sched == 0) {
+    status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ULL_ORDERED_RUNTIME_START)(
+        up, start, end, incr, istart, iend);
+  } else if (sched == 1) {
+    status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ULL_ORDERED_STATIC_START)(
+        up, start, end, incr, chunk_size, istart, iend);
+  } else if (sched == 2) {
+    status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ULL_ORDERED_DYNAMIC_START)(
+        up, start, end, incr, chunk_size, istart, iend);
+  } else if (sched == 3) {
+    status = KMP_EXPAND_NAME(KMP_API_NAME_GOMP_LOOP_ULL_ORDERED_GUIDED_START)(
+        up, start, end, incr, chunk_size, istart, iend);
+  } else {
+    KMP_ASSERT(0);
+  }
+  return status;
+}
+
+unsigned KMP_EXPAND_NAME(KMP_API_NAME_GOMP_SECTIONS2_START)(
+    unsigned count, uintptr_t *reductions, void **mem) {
+  int gtid = __kmp_entry_gtid();
+  KA_TRACE(20,
+           ("GOMP_sections2_start: T#%d, reductions: %p\n", gtid, reductions));
+  if (reductions)
+    __kmp_GOMP_init_reductions(gtid, reductions, 1);
+  if (mem)
+    KMP_FATAL(GompFeatureNotSupported, "scan");
+  return KMP_EXPAND_NAME(KMP_API_NAME_GOMP_SECTIONS_START)(count);
+}
+
+void KMP_EXPAND_NAME(KMP_API_NAME_GOMP_WORKSHARE_TASK_REDUCTION_UNREGISTER)(
+    bool cancelled) {
+  int gtid = __kmp_get_gtid();
+  MKLOC(loc, "GOMP_workshare_task_reduction_unregister");
+  KA_TRACE(20, ("GOMP_workshare_task_reduction_unregister: T#%d\n", gtid));
+  kmp_info_t *thr = __kmp_threads[gtid];
+  kmp_team_t *team = thr->th.th_team;
+  __kmpc_end_taskgroup(NULL, gtid);
+  // If last thread out of workshare, then reset the team's reduce data
+  // the GOMP_taskgroup_reduction_unregister() function will deallocate
+  // private copies after reduction calculations take place.
+  int count = KMP_ATOMIC_INC(&team->t.t_tg_fini_counter[1]);
+  if (count == thr->th.th_team_nproc - 1) {
+    KMP_EXPAND_NAME(KMP_API_NAME_GOMP_TASKGROUP_REDUCTION_UNREGISTER)
+    ((uintptr_t *)KMP_ATOMIC_LD_RLX(&team->t.t_tg_reduce_data[1]));
+    KMP_ATOMIC_ST_REL(&team->t.t_tg_reduce_data[1], NULL);
+    KMP_ATOMIC_ST_REL(&team->t.t_tg_fini_counter[1], 0);
+  }
+  if (!cancelled) {
+    __kmpc_barrier(&loc, gtid);
+  }
+}
+
 /* The following sections of code create aliases for the GOMP_* functions, then
    create versioned symbols using the assembler directive .symver. This is only
    pertinent for ELF .so library. The KMP_VERSION_SYMBOL macro is defined in
@@ -2085,7 +2593,21 @@
                    50, "GOMP_5.0");
 KMP_VERSION_SYMBOL(KMP_API_NAME_GOMP_TEAMS_REG, 50, "GOMP_5.0");
 KMP_VERSION_SYMBOL(KMP_API_NAME_GOMP_TASKWAIT_DEPEND, 50, "GOMP_5.0");
-
+KMP_VERSION_SYMBOL(KMP_API_NAME_GOMP_TASKGROUP_REDUCTION_REGISTER, 50,
+                   "GOMP_5.0");
+KMP_VERSION_SYMBOL(KMP_API_NAME_GOMP_TASKGROUP_REDUCTION_UNREGISTER, 50,
+                   "GOMP_5.0");
+KMP_VERSION_SYMBOL(KMP_API_NAME_GOMP_TASK_REDUCTION_REMAP, 50, "GOMP_5.0");
+KMP_VERSION_SYMBOL(KMP_API_NAME_GOMP_PARALLEL_REDUCTIONS, 50, "GOMP_5.0");
+KMP_VERSION_SYMBOL(KMP_API_NAME_GOMP_LOOP_START, 50, "GOMP_5.0");
+KMP_VERSION_SYMBOL(KMP_API_NAME_GOMP_LOOP_ULL_START, 50, "GOMP_5.0");
+KMP_VERSION_SYMBOL(KMP_API_NAME_GOMP_LOOP_DOACROSS_START, 50, "GOMP_5.0");
+KMP_VERSION_SYMBOL(KMP_API_NAME_GOMP_LOOP_ULL_DOACROSS_START, 50, "GOMP_5.0");
+KMP_VERSION_SYMBOL(KMP_API_NAME_GOMP_LOOP_ORDERED_START, 50, "GOMP_5.0");
+KMP_VERSION_SYMBOL(KMP_API_NAME_GOMP_LOOP_ULL_ORDERED_START, 50, "GOMP_5.0");
+KMP_VERSION_SYMBOL(KMP_API_NAME_GOMP_SECTIONS2_START, 50, "GOMP_5.0");
+KMP_VERSION_SYMBOL(KMP_API_NAME_GOMP_WORKSHARE_TASK_REDUCTION_UNREGISTER, 50,
+                   "GOMP_5.0");
 #endif // KMP_USE_VERSION_SYMBOLS
 
 #ifdef __cplusplus
diff --git a/runtime/src/kmp_tasking.cpp b/runtime/src/kmp_tasking.cpp
index d6409f7..8964dec 100644
--- a/runtime/src/kmp_tasking.cpp
+++ b/runtime/src/kmp_tasking.cpp
@@ -2497,6 +2497,7 @@
   tg_new->parent = taskdata->td_taskgroup;
   tg_new->reduce_data = NULL;
   tg_new->reduce_num_data = 0;
+  tg_new->gomp_data = NULL;
   taskdata->td_taskgroup = tg_new;
 
 #if OMPT_SUPPORT && OMPT_OPTIONAL
@@ -2595,7 +2596,8 @@
   }
   KMP_DEBUG_ASSERT(taskgroup->count == 0);
 
-  if (taskgroup->reduce_data != NULL) { // need to reduce?
+  if (taskgroup->reduce_data != NULL &&
+      !taskgroup->gomp_data) { // need to reduce?
     int cnt;
     void *reduce_data;
     kmp_team_t *t = thread->th.th_team;
diff --git a/runtime/test/tasking/omp_task_red_taskloop.c b/runtime/test/tasking/omp_task_red_taskloop.c
index 6683ab6..17130f4 100644
--- a/runtime/test/tasking/omp_task_red_taskloop.c
+++ b/runtime/test/tasking/omp_task_red_taskloop.c
@@ -6,9 +6,8 @@
 // Parsing error until clang11:
 // UNSUPPORTED: clang-10, clang-9, clang-8, clang-7
 
-// Missing GOMP_taskgroup_reduction_(un)register in LLVM/OpenMP
-// Should be removed once the functions are implemented
-// XFAIL: gcc-9, gcc-10
+// No icc compiler support yet
+// XFAIL: icc
 
 #include <stdio.h>
 #include <omp.h>
diff --git a/runtime/test/tasking/task_reduction1.c b/runtime/test/tasking/task_reduction1.c
new file mode 100644
index 0000000..39712ea
--- /dev/null
+++ b/runtime/test/tasking/task_reduction1.c
@@ -0,0 +1,42 @@
+// RUN: %libomp-compile-and-run
+
+// UNSUPPORTED: gcc-4, gcc-5, gcc-6, gcc-7, gcc-8
+
+#include <stdio.h>
+#include <stdlib.h>
+
+int a = 0, b = 1;
+
+int main(int argc, char **argv) {
+
+  #pragma omp parallel
+  #pragma omp single
+  {
+    #pragma omp taskgroup task_reduction(+: a) task_reduction(*: b)
+    {
+      int i;
+      for (i = 1; i <= 5; ++i) {
+        #pragma omp task in_reduction(+: a) in_reduction(*: b)
+        {
+          a += i;
+          b *= i;
+          #pragma omp task in_reduction(+: a)
+          {
+            a += i;
+          }
+        }
+      }
+    }
+  }
+
+  if (a != 30) {
+    fprintf(stderr, "error: a != 30. Instead a = %d\n", a);
+    exit(EXIT_FAILURE);
+  }
+  if (b != 120) {
+    fprintf(stderr, "error: b != 120. Instead b = %d\n", b);
+    exit(EXIT_FAILURE);
+  }
+
+  return EXIT_SUCCESS;
+}
diff --git a/runtime/test/tasking/task_reduction2.c b/runtime/test/tasking/task_reduction2.c
new file mode 100644
index 0000000..06a93b4
--- /dev/null
+++ b/runtime/test/tasking/task_reduction2.c
@@ -0,0 +1,37 @@
+// RUN: %libomp-compile-and-run
+//
+// XFAIL: icc
+// UNSUPPORTED: clang-4, clang-5, clang-6, clang-7, clang-8, clang-9, clang-10
+// UNSUPPORTED: gcc-4, gcc-5, gcc-6, gcc-7, gcc-8
+
+#include <stdio.h>
+#include <stdlib.h>
+
+int a = 0, b = 1;
+
+int main(int argc, char **argv) {
+
+  #pragma omp parallel
+  {
+    int i;
+    #pragma omp for reduction(task, +: a) reduction(task, *: b)
+    for (i = 1; i <= 5; ++i) {
+      #pragma omp task in_reduction(+: a) in_reduction(*: b)
+      {
+        a += i;
+        b *= i;
+      }
+    }
+  }
+
+  if (a != 15) {
+    fprintf(stderr, "error: a != 15. Instead a = %d\n", a);
+    exit(EXIT_FAILURE);
+  }
+  if (b != 120) {
+    fprintf(stderr, "error: b != 120. Instead b = %d\n", b);
+    exit(EXIT_FAILURE);
+  }
+
+  return EXIT_SUCCESS;
+}
diff --git a/runtime/test/tasking/task_reduction3.c b/runtime/test/tasking/task_reduction3.c
new file mode 100644
index 0000000..b125e3f
--- /dev/null
+++ b/runtime/test/tasking/task_reduction3.c
@@ -0,0 +1,71 @@
+// RUN: %libomp-compile-and-run
+
+// XFAIL: icc
+// UNSUPPORTED: clang-4, clang-5, clang-6, clang-7, clang-8, clang-9, clang-10
+// UNSUPPORTED: gcc-4, gcc-5, gcc-6, gcc-7, gcc-8
+
+#include <stdio.h>
+#include <stdlib.h>
+
+int a = 0, b = 1;
+
+int main(int argc, char **argv) {
+
+  #pragma omp parallel
+  {
+    #pragma omp sections reduction(task, +: a) reduction(task, *: b)
+    {
+      #pragma omp section
+      {
+        #pragma omp task in_reduction(+: a) in_reduction(*: b)
+        {
+          a += 1;
+          b *= 1;
+        }
+      }
+      #pragma omp section
+      {
+        #pragma omp task in_reduction(+: a) in_reduction(*: b)
+        {
+          a += 2;
+          b *= 2;
+        }
+      }
+      #pragma omp section
+      {
+        #pragma omp task in_reduction(+: a) in_reduction(*: b)
+        {
+          a += 3;
+          b *= 3;
+        }
+      }
+      #pragma omp section
+      {
+        #pragma omp task in_reduction(+: a) in_reduction(*: b)
+        {
+          a += 4;
+          b *= 4;
+        }
+      }
+      #pragma omp section
+      {
+        #pragma omp task in_reduction(+: a) in_reduction(*: b)
+        {
+          a += 5;
+          b *= 5;
+        }
+      }
+    }
+  }
+
+  if (a != 15) {
+    fprintf(stderr, "error: a != 15. Instead a = %d\n", a);
+    exit(EXIT_FAILURE);
+  }
+  if (b != 120) {
+    fprintf(stderr, "error: b != 120. Instead b = %d\n", b);
+    exit(EXIT_FAILURE);
+  }
+
+  return EXIT_SUCCESS;
+}
diff --git a/runtime/test/tasking/task_reduction4.c b/runtime/test/tasking/task_reduction4.c
new file mode 100644
index 0000000..9b686ab
--- /dev/null
+++ b/runtime/test/tasking/task_reduction4.c
@@ -0,0 +1,39 @@
+// RUN: %libomp-compile-and-run
+
+// XFAIL: icc
+// UNSUPPORTED: clang-4, clang-5, clang-6, clang-7, clang-8, clang-9, clang-10
+// UNSUPPORTED: gcc-4, gcc-5, gcc-6, gcc-7, gcc-8
+
+#include <stdio.h>
+#include <stdlib.h>
+
+int a = 0, b = 1;
+
+int main(int argc, char **argv) {
+
+  #pragma omp parallel reduction(task, +:a) reduction(task, *:b)
+  {
+    #pragma omp single
+    {
+      int i;
+      for (i = 1; i <= 5; ++i) {
+        #pragma omp task in_reduction(+: a) in_reduction(*: b)
+        {
+          a += i;
+          b *= i;
+        }
+      }
+    }
+  }
+
+  if (a != 15) {
+    fprintf(stderr, "error: a != 15. Instead a = %d\n", a);
+    exit(EXIT_FAILURE);
+  }
+  if (b != 120) {
+    fprintf(stderr, "error: b != 120. Instead b = %d\n", b);
+    exit(EXIT_FAILURE);
+  }
+
+  return EXIT_SUCCESS;
+}