[flang][OpenMP] Handle pre-detemined `lastprivate` for `simd` (#129507)

This PR tries to fix `lastprivate` update issues in composite
constructs. In particular, pre-determined `lastprivate` symbols are
attached to the wrong leaf of the composite construct (the outermost
one). When using delayed privatization (should be the default mode in
the future), this results in trying to update the `lastprivate` symbol
in the wrong construct (outside the `omp.loop_nest` op).

For example, given the following input:
```fortran
!$omp target teams distribute parallel do simd collapse(2) private(y_max)
  do i=x_min,x_max
    do j=y_min,y_max
    enddo
  enddo
```

Without the fixes introduced in this PR, the `DataSharingProcessor`
tries to generate the `lastprivate` update ops in the `parallel` op
since this is the op for which the DSP instance is created.

The fix consists of 2 main parts:
1. Instead of creating a single DSP instance, one instance is created
for the leaf constructs that might need privatization (whether for
explicit, implicit, or pre-determined symbols).
2. When generating the `lastprivate` comparison ops, we don't directly
use the SSA values of the UBs and steps. Instead, we regenerated these
SSA values from the original loop bounds' expressions. We have to do
this to avoid using `host_eval` values in the `lastprivate` comparison
logic which is illegal.
diff --git a/flang/lib/Lower/OpenMP/ClauseFinder.h b/flang/lib/Lower/OpenMP/ClauseFinder.h
new file mode 100644
index 0000000..3b77f2c
--- /dev/null
+++ b/flang/lib/Lower/OpenMP/ClauseFinder.h
@@ -0,0 +1,76 @@
+//===-- Lower/OpenMP/ClauseFinder.h --------------------------*- C++ -*-===//
+//
+// Part of the LLVM Project, under the Apache License v2.0 with LLVM Exceptions.
+// See https://llvm.org/LICENSE.txt for license information.
+// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
+//
+//===----------------------------------------------------------------------===//
+//
+// Coding style: https://mlir.llvm.org/getting_started/DeveloperGuide/
+//
+//===----------------------------------------------------------------------===//
+#ifndef FORTRAN_LOWER_CLAUSEFINDER_H
+#define FORTRAN_LOWER_CLAUSEFINDER_H
+
+#include "Clauses.h"
+
+namespace Fortran {
+namespace lower {
+namespace omp {
+
+class ClauseFinder {
+  using ClauseIterator = List<Clause>::const_iterator;
+
+public:
+  /// Utility to find a clause within a range in the clause list.
+  template <typename T>
+  static ClauseIterator findClause(ClauseIterator begin, ClauseIterator end) {
+    for (ClauseIterator it = begin; it != end; ++it) {
+      if (std::get_if<T>(&it->u))
+        return it;
+    }
+
+    return end;
+  }
+
+  /// Return the first instance of the given clause found in the clause list or
+  /// `nullptr` if not present. If more than one instance is expected, use
+  /// `findRepeatableClause` instead.
+  template <typename T>
+  static const T *findUniqueClause(const List<Clause> &clauses,
+                                   const parser::CharBlock **source = nullptr) {
+    ClauseIterator it = findClause<T>(clauses.begin(), clauses.end());
+    if (it != clauses.end()) {
+      if (source)
+        *source = &it->source;
+      return &std::get<T>(it->u);
+    }
+    return nullptr;
+  }
+
+  /// Call `callbackFn` for each occurrence of the given clause. Return `true`
+  /// if at least one instance was found.
+  template <typename T>
+  static bool findRepeatableClause(
+      const List<Clause> &clauses,
+      std::function<void(const T &, const parser::CharBlock &source)>
+          callbackFn) {
+    bool found = false;
+    ClauseIterator nextIt, endIt = clauses.end();
+    for (ClauseIterator it = clauses.begin(); it != endIt; it = nextIt) {
+      nextIt = findClause<T>(it, endIt);
+
+      if (nextIt != endIt) {
+        callbackFn(std::get<T>(nextIt->u), nextIt->source);
+        found = true;
+        ++nextIt;
+      }
+    }
+    return found;
+  }
+};
+} // namespace omp
+} // namespace lower
+} // namespace Fortran
+
+#endif // FORTRAN_LOWER_CLAUSEFINDER_H
diff --git a/flang/lib/Lower/OpenMP/ClauseProcessor.cpp b/flang/lib/Lower/OpenMP/ClauseProcessor.cpp
index e21d299..98a2bb7 100644
--- a/flang/lib/Lower/OpenMP/ClauseProcessor.cpp
+++ b/flang/lib/Lower/OpenMP/ClauseProcessor.cpp
@@ -12,6 +12,7 @@
 
 #include "ClauseProcessor.h"
 #include "Clauses.h"
+#include "Utils.h"
 
 #include "flang/Lower/PFTBuilder.h"
 #include "flang/Parser/tools.h"
@@ -201,24 +202,6 @@
     useDeviceSyms.push_back(object.sym());
 }
 
-static void convertLoopBounds(lower::AbstractConverter &converter,
-                              mlir::Location loc,
-                              mlir::omp::LoopRelatedClauseOps &result,
-                              std::size_t loopVarTypeSize) {
-  fir::FirOpBuilder &firOpBuilder = converter.getFirOpBuilder();
-  // The types of lower bound, upper bound, and step are converted into the
-  // type of the loop variable if necessary.
-  mlir::Type loopVarType = getLoopVarType(converter, loopVarTypeSize);
-  for (unsigned it = 0; it < (unsigned)result.loopLowerBounds.size(); it++) {
-    result.loopLowerBounds[it] = firOpBuilder.createConvert(
-        loc, loopVarType, result.loopLowerBounds[it]);
-    result.loopUpperBounds[it] = firOpBuilder.createConvert(
-        loc, loopVarType, result.loopUpperBounds[it]);
-    result.loopSteps[it] =
-        firOpBuilder.createConvert(loc, loopVarType, result.loopSteps[it]);
-  }
-}
-
 //===----------------------------------------------------------------------===//
 // ClauseProcessor unique clauses
 //===----------------------------------------------------------------------===//
@@ -240,55 +223,8 @@
     mlir::Location currentLocation, lower::pft::Evaluation &eval,
     mlir::omp::LoopRelatedClauseOps &result,
     llvm::SmallVectorImpl<const semantics::Symbol *> &iv) const {
-  bool found = false;
-  fir::FirOpBuilder &firOpBuilder = converter.getFirOpBuilder();
-
-  // Collect the loops to collapse.
-  lower::pft::Evaluation *doConstructEval = &eval.getFirstNestedEvaluation();
-  if (doConstructEval->getIf<parser::DoConstruct>()->IsDoConcurrent()) {
-    TODO(currentLocation, "Do Concurrent in Worksharing loop construct");
-  }
-
-  std::int64_t collapseValue = 1l;
-  if (auto *clause = findUniqueClause<omp::clause::Collapse>()) {
-    collapseValue = evaluate::ToInt64(clause->v).value();
-    found = true;
-  }
-
-  std::size_t loopVarTypeSize = 0;
-  do {
-    lower::pft::Evaluation *doLoop =
-        &doConstructEval->getFirstNestedEvaluation();
-    auto *doStmt = doLoop->getIf<parser::NonLabelDoStmt>();
-    assert(doStmt && "Expected do loop to be in the nested evaluation");
-    const auto &loopControl =
-        std::get<std::optional<parser::LoopControl>>(doStmt->t);
-    const parser::LoopControl::Bounds *bounds =
-        std::get_if<parser::LoopControl::Bounds>(&loopControl->u);
-    assert(bounds && "Expected bounds for worksharing do loop");
-    lower::StatementContext stmtCtx;
-    result.loopLowerBounds.push_back(fir::getBase(
-        converter.genExprValue(*semantics::GetExpr(bounds->lower), stmtCtx)));
-    result.loopUpperBounds.push_back(fir::getBase(
-        converter.genExprValue(*semantics::GetExpr(bounds->upper), stmtCtx)));
-    if (bounds->step) {
-      result.loopSteps.push_back(fir::getBase(
-          converter.genExprValue(*semantics::GetExpr(bounds->step), stmtCtx)));
-    } else { // If `step` is not present, assume it as `1`.
-      result.loopSteps.push_back(firOpBuilder.createIntegerConstant(
-          currentLocation, firOpBuilder.getIntegerType(32), 1));
-    }
-    iv.push_back(bounds->name.thing.symbol);
-    loopVarTypeSize = std::max(loopVarTypeSize,
-                               bounds->name.thing.symbol->GetUltimate().size());
-    collapseValue--;
-    doConstructEval =
-        &*std::next(doConstructEval->getNestedEvaluations().begin());
-  } while (collapseValue > 0);
-
-  convertLoopBounds(converter, currentLocation, result, loopVarTypeSize);
-
-  return found;
+  return collectLoopRelatedInfo(converter, currentLocation, eval, clauses,
+                                result, iv);
 }
 
 bool ClauseProcessor::processDevice(lower::StatementContext &stmtCtx,
diff --git a/flang/lib/Lower/OpenMP/ClauseProcessor.h b/flang/lib/Lower/OpenMP/ClauseProcessor.h
index 889a09a..c2a136d 100644
--- a/flang/lib/Lower/OpenMP/ClauseProcessor.h
+++ b/flang/lib/Lower/OpenMP/ClauseProcessor.h
@@ -12,6 +12,7 @@
 #ifndef FORTRAN_LOWER_CLAUSEPROCESSOR_H
 #define FORTRAN_LOWER_CLAUSEPROCESSOR_H
 
+#include "ClauseFinder.h"
 #include "Clauses.h"
 #include "ReductionProcessor.h"
 #include "Utils.h"
@@ -148,10 +149,6 @@
 private:
   using ClauseIterator = List<Clause>::const_iterator;
 
-  /// Utility to find a clause within a range in the clause list.
-  template <typename T>
-  static ClauseIterator findClause(ClauseIterator begin, ClauseIterator end);
-
   /// Return the first instance of the given clause found in the clause list or
   /// `nullptr` if not present. If more than one instance is expected, use
   /// `findRepeatableClause` instead.
@@ -200,44 +197,16 @@
 }
 
 template <typename T>
-ClauseProcessor::ClauseIterator
-ClauseProcessor::findClause(ClauseIterator begin, ClauseIterator end) {
-  for (ClauseIterator it = begin; it != end; ++it) {
-    if (std::get_if<T>(&it->u))
-      return it;
-  }
-
-  return end;
-}
-
-template <typename T>
 const T *
 ClauseProcessor::findUniqueClause(const parser::CharBlock **source) const {
-  ClauseIterator it = findClause<T>(clauses.begin(), clauses.end());
-  if (it != clauses.end()) {
-    if (source)
-      *source = &it->source;
-    return &std::get<T>(it->u);
-  }
-  return nullptr;
+  return ClauseFinder::findUniqueClause<T>(clauses, source);
 }
 
 template <typename T>
 bool ClauseProcessor::findRepeatableClause(
     std::function<void(const T &, const parser::CharBlock &source)> callbackFn)
     const {
-  bool found = false;
-  ClauseIterator nextIt, endIt = clauses.end();
-  for (ClauseIterator it = clauses.begin(); it != endIt; it = nextIt) {
-    nextIt = findClause<T>(it, endIt);
-
-    if (nextIt != endIt) {
-      callbackFn(std::get<T>(nextIt->u), nextIt->source);
-      found = true;
-      ++nextIt;
-    }
-  }
-  return found;
+  return ClauseFinder::findRepeatableClause<T>(clauses, callbackFn);
 }
 
 template <typename T>
diff --git a/flang/lib/Lower/OpenMP/DataSharingProcessor.cpp b/flang/lib/Lower/OpenMP/DataSharingProcessor.cpp
index 781b0df..b88454c 100644
--- a/flang/lib/Lower/OpenMP/DataSharingProcessor.cpp
+++ b/flang/lib/Lower/OpenMP/DataSharingProcessor.cpp
@@ -257,6 +257,11 @@
     return;
 
   if (mlir::isa<mlir::omp::WsloopOp>(op) || mlir::isa<mlir::omp::SimdOp>(op)) {
+    mlir::omp::LoopRelatedClauseOps result;
+    llvm::SmallVector<const semantics::Symbol *> iv;
+    collectLoopRelatedInfo(converter, converter.getCurrentLocation(), eval,
+                           clauses, result, iv);
+
     // Update the original variable just before exiting the worksharing
     // loop. Conversion as follows:
     //
@@ -280,9 +285,8 @@
     mlir::Value cmpOp;
     llvm::SmallVector<mlir::Value> vs;
     vs.reserve(loopOp.getIVs().size());
-    for (auto [iv, ub, step] :
-         llvm::zip_equal(loopOp.getIVs(), loopOp.getLoopUpperBounds(),
-                         loopOp.getLoopSteps())) {
+    for (auto [iv, ub, step] : llvm::zip_equal(
+             loopOp.getIVs(), result.loopUpperBounds, result.loopSteps)) {
       // v = iv + step
       // cmp = step < 0 ? v < ub : v > ub
       mlir::Value v = firOpBuilder.create<mlir::arith::AddIOp>(loc, iv, step);
diff --git a/flang/lib/Lower/OpenMP/OpenMP.cpp b/flang/lib/Lower/OpenMP/OpenMP.cpp
index b1568cc..ca161bc 100644
--- a/flang/lib/Lower/OpenMP/OpenMP.cpp
+++ b/flang/lib/Lower/OpenMP/OpenMP.cpp
@@ -1208,27 +1208,27 @@
     if (privatize) {
       // DataSharingProcessor::processStep2() may create operations before/after
       // the one passed as argument. We need to treat loop wrappers and their
-      // nested loop as a unit, so we need to pass the top level wrapper (if
+      // nested loop as a unit, so we need to pass the bottom level wrapper (if
       // present). Otherwise, these operations will be inserted within a
       // wrapper region.
-      mlir::Operation *privatizationTopLevelOp = &op;
+      mlir::Operation *privatizationBottomLevelOp = &op;
       if (auto loopNest = llvm::dyn_cast<mlir::omp::LoopNestOp>(op)) {
         llvm::SmallVector<mlir::omp::LoopWrapperInterface> wrappers;
         loopNest.gatherWrappers(wrappers);
         if (!wrappers.empty())
-          privatizationTopLevelOp = &*wrappers.back();
+          privatizationBottomLevelOp = &*wrappers.front();
       }
 
       if (!info.dsp) {
         assert(tempDsp.has_value());
-        tempDsp->processStep2(privatizationTopLevelOp, isLoop);
+        tempDsp->processStep2(privatizationBottomLevelOp, isLoop);
       } else {
         if (isLoop && regionArgs.size() > 0) {
           for (const auto &regionArg : regionArgs) {
             info.dsp->pushLoopIV(info.converter.getSymbolAddress(*regionArg));
           }
         }
-        info.dsp->processStep2(privatizationTopLevelOp, isLoop);
+        info.dsp->processStep2(privatizationBottomLevelOp, isLoop);
       }
     }
   }
@@ -2741,18 +2741,20 @@
   genParallelClauses(converter, semaCtx, stmtCtx, parallelItem->clauses, loc,
                      parallelClauseOps, parallelReductionSyms);
 
-  DataSharingProcessor dsp(converter, semaCtx, simdItem->clauses, eval,
-                           /*shouldCollectPreDeterminedSymbols=*/true,
-                           /*useDelayedPrivatization=*/true, symTable);
-  dsp.processStep1(&parallelClauseOps);
+  DataSharingProcessor parallelItemDSP(
+      converter, semaCtx, parallelItem->clauses, eval,
+      /*shouldCollectPreDeterminedSymbols=*/false,
+      /*useDelayedPrivatization=*/true, symTable);
+  parallelItemDSP.processStep1(&parallelClauseOps);
 
   EntryBlockArgs parallelArgs;
-  parallelArgs.priv.syms = dsp.getDelayedPrivSymbols();
+  parallelArgs.priv.syms = parallelItemDSP.getDelayedPrivSymbols();
   parallelArgs.priv.vars = parallelClauseOps.privateVars;
   parallelArgs.reduction.syms = parallelReductionSyms;
   parallelArgs.reduction.vars = parallelClauseOps.reductionVars;
   genParallelOp(converter, symTable, semaCtx, eval, loc, queue, parallelItem,
-                parallelClauseOps, parallelArgs, &dsp, /*isComposite=*/true);
+                parallelClauseOps, parallelArgs, &parallelItemDSP,
+                /*isComposite=*/true);
 
   // Clause processing.
   mlir::omp::DistributeOperands distributeClauseOps;
@@ -2769,6 +2771,11 @@
   genSimdClauses(converter, semaCtx, simdItem->clauses, loc, simdClauseOps,
                  simdReductionSyms);
 
+  DataSharingProcessor simdItemDSP(converter, semaCtx, simdItem->clauses, eval,
+                                   /*shouldCollectPreDeterminedSymbols=*/true,
+                                   /*useDelayedPrivatization=*/true, symTable);
+  simdItemDSP.processStep1(&simdClauseOps);
+
   mlir::omp::LoopNestOperands loopNestClauseOps;
   llvm::SmallVector<const semantics::Symbol *> iv;
   genLoopNestClauses(converter, semaCtx, eval, simdItem->clauses, loc,
@@ -2790,7 +2797,8 @@
   wsloopOp.setComposite(/*val=*/true);
 
   EntryBlockArgs simdArgs;
-  // TODO: Add private syms and vars.
+  simdArgs.priv.syms = simdItemDSP.getDelayedPrivSymbols();
+  simdArgs.priv.vars = simdClauseOps.privateVars;
   simdArgs.reduction.syms = simdReductionSyms;
   simdArgs.reduction.vars = simdClauseOps.reductionVars;
   auto simdOp =
@@ -2802,7 +2810,8 @@
                 {{distributeOp, distributeArgs},
                  {wsloopOp, wsloopArgs},
                  {simdOp, simdArgs}},
-                llvm::omp::Directive::OMPD_distribute_parallel_do_simd, dsp);
+                llvm::omp::Directive::OMPD_distribute_parallel_do_simd,
+                simdItemDSP);
 }
 
 static void genCompositeDistributeSimd(lower::AbstractConverter &converter,
diff --git a/flang/lib/Lower/OpenMP/Utils.cpp b/flang/lib/Lower/OpenMP/Utils.cpp
index 48bcf49..744c3bd 100644
--- a/flang/lib/Lower/OpenMP/Utils.cpp
+++ b/flang/lib/Lower/OpenMP/Utils.cpp
@@ -14,6 +14,7 @@
 
 #include "Clauses.h"
 
+#include "ClauseFinder.h"
 #include <flang/Lower/AbstractConverter.h>
 #include <flang/Lower/ConvertType.h>
 #include <flang/Lower/DirectivesCommon.h>
@@ -595,6 +596,80 @@
   }
 }
 
+static void convertLoopBounds(lower::AbstractConverter &converter,
+                              mlir::Location loc,
+                              mlir::omp::LoopRelatedClauseOps &result,
+                              std::size_t loopVarTypeSize) {
+  fir::FirOpBuilder &firOpBuilder = converter.getFirOpBuilder();
+  // The types of lower bound, upper bound, and step are converted into the
+  // type of the loop variable if necessary.
+  mlir::Type loopVarType = getLoopVarType(converter, loopVarTypeSize);
+  for (unsigned it = 0; it < (unsigned)result.loopLowerBounds.size(); it++) {
+    result.loopLowerBounds[it] = firOpBuilder.createConvert(
+        loc, loopVarType, result.loopLowerBounds[it]);
+    result.loopUpperBounds[it] = firOpBuilder.createConvert(
+        loc, loopVarType, result.loopUpperBounds[it]);
+    result.loopSteps[it] =
+        firOpBuilder.createConvert(loc, loopVarType, result.loopSteps[it]);
+  }
+}
+
+bool collectLoopRelatedInfo(
+    lower::AbstractConverter &converter, mlir::Location currentLocation,
+    lower::pft::Evaluation &eval, const omp::List<omp::Clause> &clauses,
+    mlir::omp::LoopRelatedClauseOps &result,
+    llvm::SmallVectorImpl<const semantics::Symbol *> &iv) {
+  bool found = false;
+  fir::FirOpBuilder &firOpBuilder = converter.getFirOpBuilder();
+
+  // Collect the loops to collapse.
+  lower::pft::Evaluation *doConstructEval = &eval.getFirstNestedEvaluation();
+  if (doConstructEval->getIf<parser::DoConstruct>()->IsDoConcurrent()) {
+    TODO(currentLocation, "Do Concurrent in Worksharing loop construct");
+  }
+
+  std::int64_t collapseValue = 1l;
+  if (auto *clause =
+          ClauseFinder::findUniqueClause<omp::clause::Collapse>(clauses)) {
+    collapseValue = evaluate::ToInt64(clause->v).value();
+    found = true;
+  }
+
+  std::size_t loopVarTypeSize = 0;
+  do {
+    lower::pft::Evaluation *doLoop =
+        &doConstructEval->getFirstNestedEvaluation();
+    auto *doStmt = doLoop->getIf<parser::NonLabelDoStmt>();
+    assert(doStmt && "Expected do loop to be in the nested evaluation");
+    const auto &loopControl =
+        std::get<std::optional<parser::LoopControl>>(doStmt->t);
+    const parser::LoopControl::Bounds *bounds =
+        std::get_if<parser::LoopControl::Bounds>(&loopControl->u);
+    assert(bounds && "Expected bounds for worksharing do loop");
+    lower::StatementContext stmtCtx;
+    result.loopLowerBounds.push_back(fir::getBase(
+        converter.genExprValue(*semantics::GetExpr(bounds->lower), stmtCtx)));
+    result.loopUpperBounds.push_back(fir::getBase(
+        converter.genExprValue(*semantics::GetExpr(bounds->upper), stmtCtx)));
+    if (bounds->step) {
+      result.loopSteps.push_back(fir::getBase(
+          converter.genExprValue(*semantics::GetExpr(bounds->step), stmtCtx)));
+    } else { // If `step` is not present, assume it as `1`.
+      result.loopSteps.push_back(firOpBuilder.createIntegerConstant(
+          currentLocation, firOpBuilder.getIntegerType(32), 1));
+    }
+    iv.push_back(bounds->name.thing.symbol);
+    loopVarTypeSize = std::max(loopVarTypeSize,
+                               bounds->name.thing.symbol->GetUltimate().size());
+    collapseValue--;
+    doConstructEval =
+        &*std::next(doConstructEval->getNestedEvaluations().begin());
+  } while (collapseValue > 0);
+
+  convertLoopBounds(converter, currentLocation, result, loopVarTypeSize);
+
+  return found;
+}
 } // namespace omp
 } // namespace lower
 } // namespace Fortran
diff --git a/flang/lib/Lower/OpenMP/Utils.h b/flang/lib/Lower/OpenMP/Utils.h
index 3943eb6..30b4613 100644
--- a/flang/lib/Lower/OpenMP/Utils.h
+++ b/flang/lib/Lower/OpenMP/Utils.h
@@ -163,6 +163,11 @@
 void lastprivateModifierNotSupported(const omp::clause::Lastprivate &lastp,
                                      mlir::Location loc);
 
+bool collectLoopRelatedInfo(
+    lower::AbstractConverter &converter, mlir::Location currentLocation,
+    lower::pft::Evaluation &eval, const omp::List<omp::Clause> &clauses,
+    mlir::omp::LoopRelatedClauseOps &result,
+    llvm::SmallVectorImpl<const semantics::Symbol *> &iv);
 } // namespace omp
 } // namespace lower
 } // namespace Fortran
diff --git a/flang/test/Lower/OpenMP/distribute-parallel-do-simd.f90 b/flang/test/Lower/OpenMP/distribute-parallel-do-simd.f90
index bea7f03..142bc02a 100644
--- a/flang/test/Lower/OpenMP/distribute-parallel-do-simd.f90
+++ b/flang/test/Lower/OpenMP/distribute-parallel-do-simd.f90
@@ -8,10 +8,10 @@
 subroutine distribute_parallel_do_simd_num_threads()
   !$omp teams
 
-  ! CHECK:      omp.parallel num_threads({{.*}}) private({{.*}}) {
+  ! CHECK:      omp.parallel num_threads({{.*}}) {
   ! CHECK:      omp.distribute {
   ! CHECK-NEXT: omp.wsloop {
-  ! CHECK-NEXT: omp.simd {
+  ! CHECK-NEXT: omp.simd private({{.*}}) {
   ! CHECK-NEXT: omp.loop_nest
   !$omp distribute parallel do simd num_threads(10)
   do index_ = 1, 10
@@ -25,10 +25,10 @@
 subroutine distribute_parallel_do_simd_dist_schedule()
   !$omp teams
 
-  ! CHECK:      omp.parallel private({{.*}}) {
+  ! CHECK:      omp.parallel  {
   ! CHECK:      omp.distribute dist_schedule_static dist_schedule_chunk_size({{.*}}) {
   ! CHECK-NEXT: omp.wsloop {
-  ! CHECK-NEXT: omp.simd {
+  ! CHECK-NEXT: omp.simd private({{.*}}) {
   ! CHECK-NEXT: omp.loop_nest
   !$omp distribute parallel do simd dist_schedule(static, 4)
   do index_ = 1, 10
@@ -42,10 +42,10 @@
 subroutine distribute_parallel_do_simd_schedule()
   !$omp teams
 
-  ! CHECK:      omp.parallel private({{.*}}) {
+  ! CHECK:      omp.parallel {
   ! CHECK:      omp.distribute {
   ! CHECK-NEXT: omp.wsloop schedule(static = {{.*}}) {
-  ! CHECK-NEXT: omp.simd {
+  ! CHECK-NEXT: omp.simd private({{.*}}) {
   ! CHECK-NEXT: omp.loop_nest
   !$omp distribute parallel do simd schedule(static, 4)
   do index_ = 1, 10
@@ -59,10 +59,10 @@
 subroutine distribute_parallel_do_simd_simdlen()
   !$omp teams
 
-  ! CHECK:      omp.parallel private({{.*}}) {
+  ! CHECK:      omp.parallel {
   ! CHECK:      omp.distribute {
   ! CHECK-NEXT: omp.wsloop {
-  ! CHECK-NEXT: omp.simd simdlen(4) {
+  ! CHECK-NEXT: omp.simd simdlen(4) private({{.*}}) {
   ! CHECK-NEXT: omp.loop_nest
   !$omp distribute parallel do simd simdlen(4)
   do index_ = 1, 10
@@ -83,14 +83,14 @@
   ! CHECK: omp.teams {
   !$omp teams
 
-  ! CHECK:      omp.parallel private(@{{.*}} %[[X]]#0 -> %[[X_ARG:[^,]+]],
-  ! CHECK-SAME:                      @{{.*}} %[[INDEX]]#0 -> %[[INDEX_ARG:.*]] : !fir.ref<i64>, !fir.ref<i32>) {
-  ! CHECK:      %[[X_PRIV:.*]]:2 = hlfir.declare %[[X_ARG]]
-  ! CHECK:      %[[INDEX_PRIV:.*]]:2 = hlfir.declare %[[INDEX_ARG]]
+  ! CHECK:      omp.parallel {
   ! CHECK:      omp.distribute {
   ! CHECK-NEXT: omp.wsloop {
-  ! CHECK-NEXT: omp.simd {
+  ! CHECK-NEXT: omp.simd private(@{{.*}} %[[X]]#0 -> %[[X_ARG:[^,]+]],
+  ! CHECK-SAME:                  @{{.*}} %[[INDEX]]#0 -> %[[INDEX_ARG:.*]] : !fir.ref<i64>, !fir.ref<i32>) {
   ! CHECK-NEXT: omp.loop_nest
+  ! CHECK:      %[[X_PRIV:.*]]:2 = hlfir.declare %[[X_ARG]]
+  ! CHECK:      %[[INDEX_PRIV:.*]]:2 = hlfir.declare %[[INDEX_ARG]]
   !$omp distribute parallel do simd private(x)
   do index_ = 1, 10
   end do
@@ -98,3 +98,48 @@
 
   !$omp end teams
 end subroutine distribute_parallel_do_simd_private
+
+! CHECK-LABEL:   func.func @_QPlastprivate_cond_in_composite_construct
+subroutine lastprivate_cond_in_composite_construct(x_min, x_max, y_min, y_max)
+implicit none
+integer :: x_min,x_max,y_min,y_max
+integer :: i,j
+
+! CHECK:           omp.target {{.*}} {
+! CHECK:             %[[X_MAX_MAPPED:.*]]:2 = hlfir.declare %{{.*}} {uniq_name = "{{.*}}x_max"}
+! CHECK:             omp.teams {
+! CHECK:               omp.parallel {
+! CHECK:                 omp.distribute {
+! CHECK:                   omp.wsloop {
+! CHECK:                     omp.simd private({{.*}}) {
+! CHECK:                       omp.loop_nest (%[[I_IV:.*]], %[[J_IV:.*]]) : i32 = ({{.*}}) to ({{.*}}) inclusive step ({{.*}}) {
+! CHECK:                         %[[Y_MAX_PRIV:.*]]:2 = hlfir.declare %{{.*}} {uniq_name = "{{.*}}y_max"}
+
+! CHECK:                         %[[I_UB:.*]] = fir.load %[[X_MAX_MAPPED]]#0 : !fir.ref<i32>
+! CHECK:                         %[[I_STEP:.*]] = arith.constant 1 : i32
+! CHECK:                         %[[J_UB:.*]] = fir.load %[[Y_MAX_PRIV]]#0 : !fir.ref<i32>
+! CHECK:                         %[[J_STEP:.*]] = arith.constant 1 : i32
+
+! CHECK:                         %[[VAL_55:.*]] = arith.addi %[[I_IV]], %[[I_STEP]] : i32
+! CHECK:                         %[[VAL_56:.*]] = arith.constant 0 : i32
+! CHECK:                         %[[VAL_57:.*]] = arith.cmpi slt, %[[I_STEP]], %[[VAL_56]] : i32
+! CHECK:                         %[[VAL_58:.*]] = arith.cmpi slt, %[[VAL_55]], %[[I_UB]] : i32
+! CHECK:                         %[[VAL_59:.*]] = arith.cmpi sgt, %[[VAL_55]], %[[I_UB]] : i32
+! CHECK:                         %[[VAL_60:.*]] = arith.select %[[VAL_57]], %[[VAL_58]], %[[VAL_59]] : i1
+
+! CHECK:                         %[[VAL_61:.*]] = arith.addi %[[J_IV]], %[[J_STEP]] : i32
+! CHECK:                         %[[VAL_62:.*]] = arith.constant 0 : i32
+! CHECK:                         %[[VAL_63:.*]] = arith.cmpi slt, %[[J_STEP]], %[[VAL_62]] : i32
+! CHECK:                         %[[VAL_64:.*]] = arith.cmpi slt, %[[VAL_61]], %[[J_UB]] : i32
+! CHECK:                         %[[VAL_65:.*]] = arith.cmpi sgt, %[[VAL_61]], %[[J_UB]] : i32
+! CHECK:                         %[[VAL_66:.*]] = arith.select %[[VAL_63]], %[[VAL_64]], %[[VAL_65]] : i1
+
+! CHECK:                         %[[LASTPRIV_CMP:.*]] = arith.andi %[[VAL_60]], %[[VAL_66]] : i1
+! CHECK:                         fir.if %[[LASTPRIV_CMP]] {
+
+!$omp target teams distribute parallel do simd collapse(2) private(y_max)
+  do i=x_min,x_max
+    do j=y_min,y_max
+    enddo
+  enddo
+end subroutine
diff --git a/flang/test/Lower/OpenMP/lastprivate-iv.f90 b/flang/test/Lower/OpenMP/lastprivate-iv.f90
index 7918b47..e90c7e2 100644
--- a/flang/test/Lower/OpenMP/lastprivate-iv.f90
+++ b/flang/test/Lower/OpenMP/lastprivate-iv.f90
@@ -14,11 +14,13 @@
 !CHECK-NEXT:   omp.loop_nest (%[[IV:.*]]) : i32 = (%[[LB]]) to (%[[UB]]) inclusive step (%[[STEP]]) {
 !CHECK:          %[[I:.*]]:2 = hlfir.declare %[[I_MEM]] {uniq_name = "_QFlastprivate_iv_incEi"} : (!fir.ref<i32>) -> (!fir.ref<i32>, !fir.ref<i32>)
 !CHECK:          hlfir.assign %[[IV]] to %[[I]]#1 : i32, !fir.ref<i32>
-!CHECK:          %[[V:.*]] = arith.addi %[[IV]], %[[STEP]] : i32
+!CHECK:          %[[UB_2:.*]] = arith.constant 10 : i32
+!CHECK:          %[[STEP_2:.*]]  = arith.constant 3 : i32
+!CHECK:          %[[V:.*]] = arith.addi %[[IV]], %[[STEP_2]] : i32
 !CHECK:          %[[C0:.*]] = arith.constant 0 : i32
-!CHECK:          %[[STEP_NEG:.*]] = arith.cmpi slt, %[[STEP]], %[[C0]] : i32
-!CHECK:          %[[V_LT:.*]] = arith.cmpi slt, %[[V]], %[[UB]] : i32
-!CHECK:          %[[V_GT:.*]] = arith.cmpi sgt, %[[V]], %[[UB]] : i32
+!CHECK:          %[[STEP_NEG:.*]] = arith.cmpi slt, %[[STEP_2]], %[[C0]] : i32
+!CHECK:          %[[V_LT:.*]] = arith.cmpi slt, %[[V]], %[[UB_2]] : i32
+!CHECK:          %[[V_GT:.*]] = arith.cmpi sgt, %[[V]], %[[UB_2]] : i32
 !CHECK:          %[[CMP:.*]] = arith.select %[[STEP_NEG]], %[[V_LT]], %[[V_GT]] : i1
 !CHECK:          fir.if %[[CMP]] {
 !CHECK:            hlfir.assign %[[V]] to %[[I]]#1 : i32, !fir.ref<i32>
@@ -48,11 +50,13 @@
 !CHECK-NEXT:   omp.loop_nest (%[[IV:.*]]) : i32 = (%[[LB]]) to (%[[UB]]) inclusive step (%[[STEP]]) {
 !CHECK:          %[[I:.*]]:2 = hlfir.declare %[[I_MEM]] {uniq_name = "_QFlastprivate_iv_decEi"} : (!fir.ref<i32>) -> (!fir.ref<i32>, !fir.ref<i32>)
 !CHECK:          hlfir.assign %[[IV]] to %[[I]]#1 : i32, !fir.ref<i32>
-!CHECK:          %[[V:.*]] = arith.addi %[[IV]], %[[STEP]] : i32
+!CHECK:          %[[UB_2:.*]] = arith.constant 1 : i32
+!CHECK:          %[[STEP_2:.*]]  = arith.constant -3 : i32
+!CHECK:          %[[V:.*]] = arith.addi %[[IV]], %[[STEP_2]] : i32
 !CHECK:          %[[C0:.*]] = arith.constant 0 : i32
-!CHECK:          %[[STEP_NEG:.*]] = arith.cmpi slt, %[[STEP]], %[[C0]] : i32
-!CHECK:          %[[V_LT:.*]] = arith.cmpi slt, %[[V]], %[[UB]] : i32
-!CHECK:          %[[V_GT:.*]] = arith.cmpi sgt, %[[V]], %[[UB]] : i32
+!CHECK:          %[[STEP_NEG:.*]] = arith.cmpi slt, %[[STEP_2]], %[[C0]] : i32
+!CHECK:          %[[V_LT:.*]] = arith.cmpi slt, %[[V]], %[[UB_2]] : i32
+!CHECK:          %[[V_GT:.*]] = arith.cmpi sgt, %[[V]], %[[UB_2]] : i32
 !CHECK:          %[[CMP:.*]] = arith.select %[[STEP_NEG]], %[[V_LT]], %[[V_GT]] : i1
 !CHECK:          fir.if %[[CMP]] {
 !CHECK:            hlfir.assign %[[V]] to %[[I]]#1 : i32, !fir.ref<i32>
diff --git a/flang/test/Lower/OpenMP/lastprivate-simd.f90 b/flang/test/Lower/OpenMP/lastprivate-simd.f90
index df42b35..c542f1f 100644
--- a/flang/test/Lower/OpenMP/lastprivate-simd.f90
+++ b/flang/test/Lower/OpenMP/lastprivate-simd.f90
@@ -38,7 +38,7 @@
 ! CHECK:         %[[IDO2_PRIV_DECL:.*]]:2 = hlfir.declare %[[IDO2_PRIV_ARG]] {uniq_name = "{{.*}}Eido2"}
 ! CHECK:         %[[IDO3_PRIV_DECL:.*]]:2 = hlfir.declare %[[IDO3_PRIV_ARG]] {uniq_name = "{{.*}}Eido3"}
 
-! CHECK:         fir.if %33 {
+! CHECK:         fir.if %{{.*}} {
 ! CHECK:           hlfir.assign %{{.*}} to %[[IDO1_PRIV_DECL]]#1
 ! CHECK:           hlfir.assign %{{.*}} to %[[IDO2_PRIV_DECL]]#1
 ! CHECK:           hlfir.assign %{{.*}} to %[[IDO3_PRIV_DECL]]#1
diff --git a/flang/test/Lower/OpenMP/parallel-wsloop-lastpriv.f90 b/flang/test/Lower/OpenMP/parallel-wsloop-lastpriv.f90
index 72482fc..14b83ce 100644
--- a/flang/test/Lower/OpenMP/parallel-wsloop-lastpriv.f90
+++ b/flang/test/Lower/OpenMP/parallel-wsloop-lastpriv.f90
@@ -23,11 +23,13 @@
 
   ! CHECK-NEXT: hlfir.assign %[[ARG1]] to %[[I_PVT_DECL]]#1 : i32, !fir.ref<i32>
   ! CHECK-NEXT: fir.call @_QPfoo(%[[I_PVT_DECL]]#1, %[[A_PVT_DECL]]#1) {{.*}}: (!fir.ref<i32>, !fir.ref<i32>) -> ()
-  ! CHECK:      %[[NEXT_ARG1:.*]] = arith.addi %[[ARG1]], %[[STEP]] : i32
+  ! CHECK:      %[[UB_2:.*]] = fir.load %[[A_PVT_DECL]]#0 : !fir.ref<i32>
+  ! CHECK:      %[[STEP_2:.*]] = arith.constant 1 : i32
+  ! CHECK:      %[[NEXT_ARG1:.*]] = arith.addi %[[ARG1]], %[[STEP_2]] : i32
   ! CHECK:      %[[ZERO:.*]] = arith.constant 0 : i32
-  ! CHECK:      %[[STEP_DIR:.*]] = arith.cmpi slt, %[[STEP]], %[[ZERO]] : i32
-  ! CHECK:      %[[LT_UB:.*]] = arith.cmpi slt, %[[NEXT_ARG1]], %[[UB]] : i32
-  ! CHECK:      %[[GT_UB:.*]] = arith.cmpi sgt, %[[NEXT_ARG1]], %[[UB]] : i32
+  ! CHECK:      %[[STEP_DIR:.*]] = arith.cmpi slt, %[[STEP_2]], %[[ZERO]] : i32
+  ! CHECK:      %[[LT_UB:.*]] = arith.cmpi slt, %[[NEXT_ARG1]], %[[UB_2]] : i32
+  ! CHECK:      %[[GT_UB:.*]] = arith.cmpi sgt, %[[NEXT_ARG1]], %[[UB_2]] : i32
   ! CHECK:      %[[SEL:.*]] = arith.select %[[STEP_DIR]], %[[LT_UB]], %[[GT_UB]] : i1
   ! CHECK:      fir.if %[[SEL]] {
   ! CHECK:        hlfir.assign %[[NEXT_ARG1]] to %[[I_PVT_DECL]]#1 : i32, !fir.ref<i32>
@@ -67,11 +69,13 @@
 
   ! CHECK: hlfir.assign %[[ARG2]] to %[[I_PVT_DECL]]#1 : i32, !fir.ref<i32>
   ! CHECK: fir.call @_QPfoo(%[[I_PVT_DECL]]#1, %[[A_PVT_DECL]]#1) {{.*}}: (!fir.ref<i32>, !fir.ref<i32>) -> ()
-  ! CHECK: %[[NEXT_ARG2:.*]] = arith.addi %[[ARG2]], %[[STEP]] : i32
+  ! CHECK: %[[UB_2:.*]] = fir.load %[[N_PVT_DECL]]#0 : !fir.ref<i32>
+  ! CHECK: %[[STEP_2:.*]] = arith.constant 1 : i32
+  ! CHECK: %[[NEXT_ARG2:.*]] = arith.addi %[[ARG2]], %[[STEP_2]] : i32
   ! CHECK: %[[ZERO:.*]] = arith.constant 0 : i32
-  ! CHECK: %[[STEP_DIR:.*]] = arith.cmpi slt, %[[STEP]], %[[ZERO]] : i32
-  ! CHECK: %[[LT_UB:.*]] = arith.cmpi slt, %[[NEXT_ARG2]], %[[UB]] : i32
-  ! CHECK: %[[GT_UB:.*]] = arith.cmpi sgt, %[[NEXT_ARG2]], %[[UB]] : i32
+  ! CHECK: %[[STEP_DIR:.*]] = arith.cmpi slt, %[[STEP_2]], %[[ZERO]] : i32
+  ! CHECK: %[[LT_UB:.*]] = arith.cmpi slt, %[[NEXT_ARG2]], %[[UB_2]] : i32
+  ! CHECK: %[[GT_UB:.*]] = arith.cmpi sgt, %[[NEXT_ARG2]], %[[UB_2]] : i32
   ! CHECK: %[[SEL:.*]] = arith.select %[[STEP_DIR]], %[[LT_UB]], %[[GT_UB]] : i1
   ! CHECK: fir.if %[[SEL]] {
   ! CHECK:   hlfir.assign %[[NEXT_ARG2]] to %[[I_PVT_DECL]]#1 : i32, !fir.ref<i32>
@@ -112,17 +116,24 @@
   ! CHECK-NEXT: hlfir.assign %[[ARG1]] to %[[I_PVT_DECL]]#1 : i32, !fir.ref<i32>
   ! CHECK-NEXT: hlfir.assign %[[ARG2]] to %[[J_PVT_DECL]]#1 : i32, !fir.ref<i32>
   ! CHECK-NEXT: fir.call @_QPfoo(%[[I_PVT_DECL]]#1, %[[A_PVT_DECL]]#1) {{.*}}: (!fir.ref<i32>, !fir.ref<i32>) -> ()
-  ! CHECK:      %[[NEXT_ARG1:.*]] = arith.addi %[[ARG1]], %[[STEP1]] : i32
+
+  ! CHECK:      %[[UB1_2:.*]] = fir.load %[[A_PVT_DECL]]#0 : !fir.ref<i32>
+  ! CHECK:      %[[STEP1_2:.*]] = arith.constant 1 : i32
+  ! CHECK:      %[[UB2_2:.*]] = fir.load %[[A_PVT_DECL]]#0 : !fir.ref<i32>
+  ! CHECK:      %[[STEP2_2:.*]] = arith.constant 1 : i32
+
+  ! CHECK:      %[[NEXT_ARG1:.*]] = arith.addi %[[ARG1]], %[[STEP1_2]] : i32
   ! CHECK:      %[[ZERO1:.*]] = arith.constant 0 : i32
-  ! CHECK:      %[[STEP1_END:.*]] = arith.cmpi slt, %[[STEP1]], %[[ZERO1]] : i32
-  ! CHECK:      %[[LT_UB1:.*]] = arith.cmpi slt, %[[NEXT_ARG1]], %[[UB1]] : i32
-  ! CHECK:      %[[GT_UB1:.*]] = arith.cmpi sgt, %[[NEXT_ARG1]], %[[UB1]] : i32
+  ! CHECK:      %[[STEP1_END:.*]] = arith.cmpi slt, %[[STEP1_2]], %[[ZERO1]] : i32
+  ! CHECK:      %[[LT_UB1:.*]] = arith.cmpi slt, %[[NEXT_ARG1]], %[[UB1_2]] : i32
+  ! CHECK:      %[[GT_UB1:.*]] = arith.cmpi sgt, %[[NEXT_ARG1]], %[[UB1_2]] : i32
   ! CHECK:      %[[SEL1:.*]] = arith.select %[[STEP1_END]], %[[LT_UB1]], %[[GT_UB1]] : i1
-  ! CHECK:      %[[NEXT_ARG2:.*]] = arith.addi %[[ARG2]], %[[STEP2]] : i32
+
+  ! CHECK:      %[[NEXT_ARG2:.*]] = arith.addi %[[ARG2]], %[[STEP2_2]] : i32
   ! CHECK:      %[[ZERO2:.*]] = arith.constant 0 : i32
-  ! CHECK:      %[[STEP2_END:.*]] = arith.cmpi slt, %[[STEP2]], %[[ZERO2]] : i32
-  ! CHECK:      %[[LT_UB2:.*]] = arith.cmpi slt, %[[NEXT_ARG2]], %[[UB2]] : i32
-  ! CHECK:      %[[GT_UB2:.*]] = arith.cmpi sgt, %[[NEXT_ARG2]], %[[UB2]] : i32
+  ! CHECK:      %[[STEP2_END:.*]] = arith.cmpi slt, %[[STEP2_2]], %[[ZERO2]] : i32
+  ! CHECK:      %[[LT_UB2:.*]] = arith.cmpi slt, %[[NEXT_ARG2]], %[[UB2_2]] : i32
+  ! CHECK:      %[[GT_UB2:.*]] = arith.cmpi sgt, %[[NEXT_ARG2]], %[[UB2_2]] : i32
   ! CHECK:      %[[SEL2:.*]] = arith.select %[[STEP2_END]], %[[LT_UB2]], %[[GT_UB2]] : i1
   ! CHECK:      %[[AND:.*]] = arith.andi %[[SEL1]], %[[SEL2]] : i1
   ! CHECK:      fir.if %[[AND]] {
@@ -173,24 +184,32 @@
   ! CHECK-NEXT: hlfir.assign %[[ARG2]] to %[[J_PVT_DECL]]#1 : i32, !fir.ref<i32>
   ! CHECK-NEXT: hlfir.assign %[[ARG3]] to %[[K_PVT_DECL]]#1 : i32, !fir.ref<i32>
   ! CHECK-NEXT: fir.call @_QPfoo(%[[I_PVT_DECL]]#1, %[[A_PVT_DECL]]#1) {{.*}}: (!fir.ref<i32>, !fir.ref<i32>) -> ()
-  ! CHECK:      %[[NEXT_ARG1:.*]] = arith.addi %[[ARG1]], %[[STEP1]] : i32
+
+  ! CHECK:      %[[UB1_2:.*]] = fir.load %[[A_PVT_DECL]]#0 : !fir.ref<i32>
+  ! CHECK:      %[[STEP1_2:.*]] = arith.constant 1 : i32
+  ! CHECK:      %[[UB2_2:.*]] = fir.load %[[A_PVT_DECL]]#0 : !fir.ref<i32>
+  ! CHECK:      %[[STEP2_2:.*]] = arith.constant 1 : i32
+  ! CHECK:      %[[UB3_2:.*]] = fir.load %[[A_PVT_DECL]]#0 : !fir.ref<i32>
+  ! CHECK:      %[[STEP3_2:.*]] = arith.constant 1 : i32
+
+  ! CHECK:      %[[NEXT_ARG1:.*]] = arith.addi %[[ARG1]], %[[STEP1_2]] : i32
   ! CHECK:      %[[ZERO1:.*]] = arith.constant 0 : i32
-  ! CHECK:      %[[STEP1_END:.*]] = arith.cmpi slt, %[[STEP1]], %[[ZERO1]] : i32
-  ! CHECK:      %[[LT_UB1:.*]] = arith.cmpi slt, %[[NEXT_ARG1]], %[[UB1]] : i32
-  ! CHECK:      %[[GT_UB1:.*]] = arith.cmpi sgt, %[[NEXT_ARG1]], %[[UB1]] : i32
+  ! CHECK:      %[[STEP1_END:.*]] = arith.cmpi slt, %[[STEP1_2]], %[[ZERO1]] : i32
+  ! CHECK:      %[[LT_UB1:.*]] = arith.cmpi slt, %[[NEXT_ARG1]], %[[UB1_2]] : i32
+  ! CHECK:      %[[GT_UB1:.*]] = arith.cmpi sgt, %[[NEXT_ARG1]], %[[UB1_2]] : i32
   ! CHECK:      %[[SEL1:.*]] = arith.select %[[STEP1_END]], %[[LT_UB1]], %[[GT_UB1]] : i1
-  ! CHECK:      %[[NEXT_ARG2:.*]] = arith.addi %[[ARG2]], %[[STEP2]] : i32
+  ! CHECK:      %[[NEXT_ARG2:.*]] = arith.addi %[[ARG2]], %[[STEP2_2]] : i32
   ! CHECK:      %[[ZERO2:.*]] = arith.constant 0 : i32
-  ! CHECK:      %[[STEP2_END:.*]] = arith.cmpi slt, %[[STEP2]], %[[ZERO2]] : i32
-  ! CHECK:      %[[LT_UB2:.*]] = arith.cmpi slt, %[[NEXT_ARG2]], %[[UB2]] : i32
-  ! CHECK:      %[[GT_UB2:.*]] = arith.cmpi sgt, %[[NEXT_ARG2]], %[[UB2]] : i32
+  ! CHECK:      %[[STEP2_END:.*]] = arith.cmpi slt, %[[STEP2_2]], %[[ZERO2]] : i32
+  ! CHECK:      %[[LT_UB2:.*]] = arith.cmpi slt, %[[NEXT_ARG2]], %[[UB2_2]] : i32
+  ! CHECK:      %[[GT_UB2:.*]] = arith.cmpi sgt, %[[NEXT_ARG2]], %[[UB2_2]] : i32
   ! CHECK:      %[[SEL2:.*]] = arith.select %[[STEP2_END]], %[[LT_UB2]], %[[GT_UB2]] : i1
   ! CHECK:      %[[AND1:.*]] = arith.andi %[[SEL1]], %[[SEL2]] : i1
-  ! CHECK:      %[[NEXT_ARG3:.*]] = arith.addi %[[ARG3]], %[[STEP3]] : i32
+  ! CHECK:      %[[NEXT_ARG3:.*]] = arith.addi %[[ARG3]], %[[STEP3_2]] : i32
   ! CHECK:      %[[ZERO3:.*]] = arith.constant 0 : i32
-  ! CHECK:      %[[STEP3_END:.*]] = arith.cmpi slt, %[[STEP3]], %[[ZERO3]] : i32
-  ! CHECK:      %[[LT_UB3:.*]] = arith.cmpi slt, %[[NEXT_ARG3]], %[[UB3]] : i32
-  ! CHECK:      %[[GT_UB3:.*]] = arith.cmpi sgt, %[[NEXT_ARG3]], %[[UB3]] : i32
+  ! CHECK:      %[[STEP3_END:.*]] = arith.cmpi slt, %[[STEP3_2]], %[[ZERO3]] : i32
+  ! CHECK:      %[[LT_UB3:.*]] = arith.cmpi slt, %[[NEXT_ARG3]], %[[UB3_2]] : i32
+  ! CHECK:      %[[GT_UB3:.*]] = arith.cmpi sgt, %[[NEXT_ARG3]], %[[UB3_2]] : i32
   ! CHECK:      %[[SEL3:.*]] = arith.select %[[STEP3_END]], %[[LT_UB3]], %[[GT_UB3]] : i1
   ! CHECK:      %[[AND2:.*]] = arith.andi %[[AND1]], %[[SEL3]] : i1
   ! CHECK:      fir.if %[[AND2]] {