[Mlir-commits] [mlir] 371366c - [mlir][nvgpu] add simple pipelining for shared memory copies

Alex Zinenko llvmlistbot at llvm.org
Mon Jul 17 07:29:19 PDT 2023


Author: Alex Zinenko
Date: 2023-07-17T14:29:12Z
New Revision: 371366ce27303e0b949aeb643b973a1a110da469

URL: https://github.com/llvm/llvm-project/commit/371366ce27303e0b949aeb643b973a1a110da469
DIFF: https://github.com/llvm/llvm-project/commit/371366ce27303e0b949aeb643b973a1a110da469.diff

LOG: [mlir][nvgpu] add simple pipelining for shared memory copies

Add a simple transform operation to the NVGPU extension that performs
software pipelining of copies to shared memory. The functionality is
extremely minimalistic in this version and only supports copies from
global to shared memory inside an `scf.for` loop with either
`vector.transfer` or `nvgpu.device_async_copy` operations when
pipelining preconditions are already satisfied in the IR. This is the
minimally useful version that uses the more general loop pipeliner in an
NVGPU-specific way. Further extensions and orthogonalizations will be
necessary.

This required a change to the loop pipeliner itself to properly
propagate errors should the predicate generator fail.

This is loosely inspired from the vesion in IREE, but has less unsafe
assumptions and more principled way of communicating decisions.

Reviewed By: nicolasvasilache

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

Added: 
    mlir/test/Dialect/NVGPU/transform-pipeline-shared.mlir

Modified: 
    mlir/include/mlir/Dialect/NVGPU/TransformOps/NVGPUTransformOps.h
    mlir/include/mlir/Dialect/NVGPU/TransformOps/NVGPUTransformOps.td
    mlir/include/mlir/Dialect/SCF/Transforms/Patterns.h
    mlir/include/mlir/Dialect/SCF/Transforms/Transforms.h
    mlir/lib/Dialect/NVGPU/TransformOps/CMakeLists.txt
    mlir/lib/Dialect/NVGPU/TransformOps/NVGPUTransformOps.cpp
    mlir/lib/Dialect/SCF/Transforms/LoopPipelining.cpp
    utils/bazel/llvm-project-overlay/mlir/BUILD.bazel

Removed: 
    


################################################################################
diff  --git a/mlir/include/mlir/Dialect/NVGPU/TransformOps/NVGPUTransformOps.h b/mlir/include/mlir/Dialect/NVGPU/TransformOps/NVGPUTransformOps.h
index 0c7b9d865aa246..1c30cc4a57d880 100644
--- a/mlir/include/mlir/Dialect/NVGPU/TransformOps/NVGPUTransformOps.h
+++ b/mlir/include/mlir/Dialect/NVGPU/TransformOps/NVGPUTransformOps.h
@@ -28,6 +28,10 @@ namespace linalg {
 class LinalgOp;
 } // namespace linalg
 
+namespace scf {
+class ForOp;
+} // namespace scf
+
 namespace nvgpu {
 void registerTransformDialectExtension(DialectRegistry &registry);
 } // namespace nvgpu

diff  --git a/mlir/include/mlir/Dialect/NVGPU/TransformOps/NVGPUTransformOps.td b/mlir/include/mlir/Dialect/NVGPU/TransformOps/NVGPUTransformOps.td
index 168a445b62ccf9..58cc98c937259b 100644
--- a/mlir/include/mlir/Dialect/NVGPU/TransformOps/NVGPUTransformOps.td
+++ b/mlir/include/mlir/Dialect/NVGPU/TransformOps/NVGPUTransformOps.td
@@ -15,6 +15,77 @@ include "mlir/Dialect/Transform/IR/TransformInterfaces.td"
 include "mlir/Dialect/Transform/IR/TransformTypes.td"
 include "mlir/Interfaces/SideEffectInterfaces.td"
 
+//===----------------------------------------------------------------------===//
+// PipelineSharedMemoryCopiesOp
+//===----------------------------------------------------------------------===//
+
+def PipelineSharedMemoryCopiesOp :
+  Op<Transform_Dialect, "nvgpu.pipeline_shared_memory_copies",
+    [FunctionalStyleTransformOpTrait,
+     MemoryEffectsOpInterface,
+     TransformEachOpTrait,
+     TransformOpInterface,
+     ReportTrackingListenerFailuresOpTrait]> {
+  let summary =
+    "Applies software pipelining to a given loop with shared memory copies";
+
+  let description = [{
+    Applies software pipelining to a given scf.for loop. The pipelining
+    strategy will look for a load into shared memory and pipeline it to overlap
+    it with the rest of the loop.
+    
+    NOTE: It is user responsibility to ensure that there are no dependency
+    between `depth` iterations of the loop by using multi-buffering. It is
+    also user responsibility to ensure a sufficient amount of shared memory
+    is allocated to cover eventual writes by `depth-1` speculative
+    iterations.
+
+    `depth` will indicate how many stages the software pipeline should have.
+    `peel_epilogue` allows to force the epilogue to be peeled out instead of
+    potentially using predicated operations for the epilogue phase.
+
+    #### Return modes
+
+    Consumes the operand handle and produces a result handle pointing to the
+    loop, which may or may not have been pipelined. Produces a definite failure
+    if the loop pipeliner mutated the IR before failing to pipeline, in
+    particular if `peel_epilogue` is not set and the loop body doesn't support
+    predication. If failure propagation mode is set to "propagate", produces a
+    silenceable failure when pipelining preconditions, e.g., loop bound being
+    static, are not met or when the loop wasn't pipelined because due to the
+    lack of loads into shared memory. If the failure propagation mode is set
+    to "suppress" (default), succeeds in these case and associates the result
+    handle with the original loop.
+
+    TODO: the shared memory part and behavior specific to NVGPU should be
+    made orthogonal to pipelining so that `transform.loop.pipeline` becomes
+    usable here.
+  }];
+
+  let arguments = (ins TransformHandleTypeInterface:$for_op,
+                   I64Attr:$depth,
+                   UnitAttr:$peel_epilogue,
+                   DefaultValuedAttr<FailurePropagationMode,
+                      "::mlir::transform::FailurePropagationMode::Suppress">
+                     :$failure_propagation_mode);
+  let results = (outs TransformHandleTypeInterface:$result);
+
+  let assemblyFormat = [{ 
+    `failures` `(` $failure_propagation_mode `)`
+    $for_op
+    attr-dict 
+    `:` functional-type(operands, results)
+  }];
+
+  let extraClassDeclaration = [{
+    ::mlir::DiagnosedSilenceableFailure applyToOne(
+        ::mlir::transform::TransformRewriter &rewriter,
+        ::mlir::scf::ForOp forOp,
+        ::mlir::transform::ApplyToEachResultList &results,
+        ::mlir::transform::TransformState &state);
+  }];
+}
+
 //===----------------------------------------------------------------------===//
 // RewriteMatmulAsMmaSyncOp
 //===----------------------------------------------------------------------===//

diff  --git a/mlir/include/mlir/Dialect/SCF/Transforms/Patterns.h b/mlir/include/mlir/Dialect/SCF/Transforms/Patterns.h
index 598e8ba6d3af1e..e6df65ae347c62 100644
--- a/mlir/include/mlir/Dialect/SCF/Transforms/Patterns.h
+++ b/mlir/include/mlir/Dialect/SCF/Transforms/Patterns.h
@@ -24,15 +24,21 @@ namespace scf {
 /// For example if we break a loop into 3 stages named S0, S1, S2 we would
 /// generate the following code with the number in parenthesis as the iteration
 /// index:
-/// S0(0)                        // Prologue
-/// S0(1) S1(0)                  // Prologue
-/// scf.for %I = %C0 to %N - 2 {
-///  S0(I+2) S1(I+1) S2(I)       // Pipelined kernel
-/// }
-/// S1(N) S2(N-1)                // Epilogue
-/// S2(N)                        // Epilogue
+///
+///   S0(0)                        // Prologue
+///   S0(1) S1(0)                  // Prologue
+///   scf.for %I = %C0 to %N - 2 {
+///     S0(I+2) S1(I+1) S2(I)       // Pipelined kernel
+///   }
+///   S1(N) S2(N-1)                // Epilogue
+///   S2(N)                        // Epilogue
+///
+/// If `modifiedIR` is provided, it will be set to a value that indicates
+/// whether pipelining modified the IR before failing, signaling to the caller
+/// whether they can proceed with 
diff erent transformations.
 FailureOr<ForOp> pipelineForLoop(RewriterBase &rewriter, ForOp forOp,
-                                 const PipeliningOption &options);
+                                 const PipeliningOption &options,
+                                 bool *modifiedIR = nullptr);
 
 // TODO: such patterns should be auto-generated.
 class ForLoopPipeliningPattern : public OpRewritePattern<ForOp> {

diff  --git a/mlir/include/mlir/Dialect/SCF/Transforms/Transforms.h b/mlir/include/mlir/Dialect/SCF/Transforms/Transforms.h
index fbe73a260b409a..cdd2b1b2d8614b 100644
--- a/mlir/include/mlir/Dialect/SCF/Transforms/Transforms.h
+++ b/mlir/include/mlir/Dialect/SCF/Transforms/Transforms.h
@@ -154,9 +154,12 @@ struct PipeliningOption {
   /// lambda to generate the predicated version of operations.
   bool peelEpilogue = true;
 
-  // Lamdba to predicate operations when the prologue or epilogue are not
+  // Callback to predicate operations when the prologue or epilogue are not
   // peeled. This takes the original operation, an i1 predicate value and the
-  // pattern rewriter.
+  // pattern rewriter. It is expected to replace the given operation with
+  // the predicated equivalent and return it, or return nullptr if the
+  // predication is impossible. In the latter case, pipelining will fail and
+  // may leave IR in a partially transformed state.
   using PredicateOpFn =
       std::function<Operation *(RewriterBase &, Operation *, Value)>;
   PredicateOpFn predicateFn = nullptr;

diff  --git a/mlir/lib/Dialect/NVGPU/TransformOps/CMakeLists.txt b/mlir/lib/Dialect/NVGPU/TransformOps/CMakeLists.txt
index 973e5268389fda..8c8b6a89b00dfb 100644
--- a/mlir/lib/Dialect/NVGPU/TransformOps/CMakeLists.txt
+++ b/mlir/lib/Dialect/NVGPU/TransformOps/CMakeLists.txt
@@ -15,6 +15,8 @@ add_mlir_dialect_library(MLIRNVGPUTransformOps
   MLIRNVGPUDialect
   MLIRParser
   MLIRSideEffectInterfaces
+  MLIRSCFDialect
+  MLIRSCFTransforms
   MLIRTransformDialect
   MLIRTransformDialectUtils
   MLIRVectorTransforms

diff  --git a/mlir/lib/Dialect/NVGPU/TransformOps/NVGPUTransformOps.cpp b/mlir/lib/Dialect/NVGPU/TransformOps/NVGPUTransformOps.cpp
index 3bc64f742c3a6f..94a61d7de87eb4 100644
--- a/mlir/lib/Dialect/NVGPU/TransformOps/NVGPUTransformOps.cpp
+++ b/mlir/lib/Dialect/NVGPU/TransformOps/NVGPUTransformOps.cpp
@@ -8,6 +8,7 @@
 
 #include "mlir/Dialect/NVGPU/TransformOps/NVGPUTransformOps.h"
 
+#include "mlir/Analysis/SliceAnalysis.h"
 #include "mlir/Dialect/Affine/IR/AffineOps.h"
 #include "mlir/Dialect/Arith/IR/Arith.h"
 #include "mlir/Dialect/Arith/Utils/Utils.h"
@@ -15,8 +16,10 @@
 #include "mlir/Dialect/Linalg/IR/Linalg.h"
 #include "mlir/Dialect/MemRef/IR/MemRef.h"
 #include "mlir/Dialect/NVGPU/IR/NVGPUDialect.h"
+#include "mlir/Dialect/SCF/IR/SCF.h"
+#include "mlir/Dialect/SCF/Transforms/Patterns.h"
+#include "mlir/Dialect/SCF/Transforms/Transforms.h"
 #include "mlir/Dialect/Utils/IndexingUtils.h"
-#include "mlir/Dialect/Utils/StaticValueUtils.h"
 #include "mlir/Dialect/Vector/IR/VectorOps.h"
 #include "mlir/IR/AffineExpr.h"
 #include "mlir/IR/BuiltinTypes.h"
@@ -27,7 +30,6 @@
 #include "mlir/Support/LogicalResult.h"
 #include "llvm/ADT/ArrayRef.h"
 #include "llvm/Support/Debug.h"
-#include "llvm/Support/ErrorHandling.h"
 
 using namespace mlir;
 using namespace mlir::linalg;
@@ -39,6 +41,281 @@ using namespace mlir::transform;
 #define DBGSNL() (llvm::dbgs() << "\n")
 #define LDBG(X) LLVM_DEBUG(DBGS() << X << "\n")
 
+//===----------------------------------------------------------------------===//
+// PipelineSharedMemoryCopiesOp
+//===----------------------------------------------------------------------===//
+
+/// Returns true if the given type has the default memory space.
+static bool hasDefaultMemorySpace(BaseMemRefType type) {
+  return !type.getMemorySpace() || type.getMemorySpaceAsInt() == 0;
+}
+
+/// Returns true if the given type has the shared (workgroup) memory space.
+static bool hasSharedMemorySpace(BaseMemRefType type) {
+  auto space =
+      dyn_cast_if_present<gpu::AddressSpaceAttr>(type.getMemorySpace());
+  return space &&
+         space.getValue() == gpu::GPUDialect::getWorkgroupAddressSpace();
+}
+
+/// Returns the value produced by a load from the default memory space. Returns
+/// null if the operation is not such a load.
+static Value getValueLoadedFromGlobal(Operation *op) {
+  // TODO: consider an interface or leveraging the memory effects interface.
+  auto load = dyn_cast<vector::TransferReadOp>(op);
+  if (!load)
+    return nullptr;
+
+  auto loadType = dyn_cast<MemRefType>(load.getSource().getType());
+  if (!loadType || !hasDefaultMemorySpace(loadType))
+    return nullptr;
+  return load;
+}
+
+/// Returns true if the operation is storing the given value into shared memory.
+static bool isStoreToShared(Operation *op, Value v) {
+  // TOD: consider an interface or leveraging the memory effects interface.
+  auto store = dyn_cast<vector::TransferWriteOp>(op);
+  if (!store || store.getVector() != v)
+    return false;
+
+  auto storeType = dyn_cast<MemRefType>(store.getSource().getType());
+  return storeType || hasSharedMemorySpace(storeType);
+}
+
+/// Returns true if the operation is a load from the default memory space the
+/// result of which is only stored into the shared memory space.
+static bool isLoadFromGlobalStoredToShared(Operation *op) {
+  Value loaded = getValueLoadedFromGlobal(op);
+  if (!loaded || !loaded.hasOneUse())
+    return false;
+
+  return isStoreToShared(*loaded.getUsers().begin(), loaded);
+}
+
+/// Populate `ops` with the set of operations that belong to the stage 0 of the
+/// pipelined version of the given loop when pipelining copies to shared memory.
+/// Specifically, this collects:
+///
+///   1. all loads from global memory, both sync and async;
+///   2. the barriers for async loads.
+///
+/// In particular, barriers are omitted if they do not dominate at least one
+/// async load for which there is not yet a barrier.
+static LogicalResult
+collectStage0PipeliningOps(scf::ForOp forOp,
+                           llvm::SmallPtrSet<Operation *, 16> &ops) {
+
+  llvm::SmallPtrSet<Operation *, 4> barriers;
+  for (Operation &op : *forOp.getBody()) {
+    // Bail on nested ops for now.
+    if (op.getNumRegions() > 0)
+      return failure();
+
+    if (isa<gpu::BarrierOp>(op)) {
+      barriers.insert(&op);
+      continue;
+    }
+
+    if (isa<nvgpu::DeviceAsyncCopyOp, nvgpu::DeviceAsyncCreateGroupOp>(op)) {
+      ops.insert(&op);
+      ops.insert(std::make_move_iterator(barriers.begin()),
+                 std::make_move_iterator(barriers.end()));
+      assert(barriers.empty() &&
+             "expected to have moved the barriers into another set");
+      continue;
+    }
+
+    if (isLoadFromGlobalStoredToShared(&op)) {
+      ops.insert(&op);
+      continue;
+    }
+  }
+
+  return success();
+}
+
+/// Hook for the loop pipeliner that sets the "num groups in flight" attribute
+/// of async wait operations corresponding to pipelined shared memory copies.
+// TODO: this currently assumes that there are no groups that could be in flight
+// in the existing code.
+static void
+setAsyncWaitGroupsInFlight(OpBuilder &builder, Operation *op,
+                           scf::PipeliningOption::PipelinerPart part,
+                           unsigned iteration, unsigned depth) {
+  // Based on the order of copies within the loop we need to set the number
+  // of copies in flight, unless it is already set.
+  auto waitOp = dyn_cast<nvgpu::DeviceAsyncWaitOp>(op);
+  if (!waitOp || waitOp.getNumGroups())
+    return;
+
+  int numGroupInFlight = 0;
+  if (part == scf::PipeliningOption::PipelinerPart::Kernel ||
+      part == scf::PipeliningOption::PipelinerPart::Prologue) {
+    numGroupInFlight = depth - 1;
+  } else {
+    // By construction there should be no wait op in the prologue as all the
+    // wait should be in the last stage.
+    assert(part == scf::PipeliningOption::PipelinerPart::Epilogue);
+    // Based on the schedule we pick we know how many groups are in flight for
+    // each iteration of the epilogue.
+    numGroupInFlight = depth - 1 - iteration;
+  }
+  waitOp.setNumGroups(numGroupInFlight);
+}
+
+/// Hook for the loop pipeliner that populates `ops` with the stage information
+/// as follows:
+///
+///   - operations in `stage0Ops` (typically loads from global memory and
+///     related barriers) are at stage 0;
+///   - operations in the backward slice of any stage0Ops are all at stage 0;
+///   - other operations are at stage `depth`;
+///   - the internal order of the pipelined loop has ops at stage `depth` first,
+///   then those at stage 0, with relative order within each group preserved.
+///
+static void getPipelineStages(
+    scf::ForOp forOp,
+    std::vector<std::pair<Operation *, unsigned>> &opsWithPipelineStages,
+    unsigned depth, llvm::SmallPtrSetImpl<Operation *> &stage0Ops) {
+  SetVector<Operation *> dependencies;
+  BackwardSliceOptions options([&](Operation *visited) {
+    return visited->getBlock() == forOp.getBody();
+  });
+  options.inclusive = true;
+  for (Operation &op : forOp.getBody()->getOperations()) {
+    if (stage0Ops.contains(&op))
+      getBackwardSlice(&op, &dependencies, options);
+  }
+
+  for (Operation &op : forOp.getBody()->getOperations()) {
+    if (!dependencies.contains(&op) && !isa<scf::YieldOp>(op))
+      opsWithPipelineStages.emplace_back(&op, depth);
+  }
+  for (Operation &op : forOp.getBody()->getOperations()) {
+    if (dependencies.contains(&op))
+      opsWithPipelineStages.emplace_back(&op, 0);
+  }
+}
+
+/// Hook for the loop pipeliner. Replaces op with a predicated version and
+/// returns the resulting operation. Returns the original op if the predication
+/// isn't necessary for the given op. Returns null if predication is needed but
+/// not supported.
+static Operation *replaceOpWithPredicatedOp(RewriterBase &rewriter,
+                                            Operation *op, Value predicate) {
+  // Some operations may be fine to execute "speculatively" more times than the
+  // original number of iterations, in particular side-effect free operations
+  // and barriers, even if they cannot be predicated.
+  if (isMemoryEffectFree(op) ||
+      isa<gpu::BarrierOp, nvgpu::DeviceAsyncCreateGroupOp,
+          nvgpu::DeviceAsyncWaitOp>(op)) {
+    return op;
+  }
+
+  // Otherwise, only async copies can currently be predicated.
+  auto asyncCopyOp = dyn_cast<nvgpu::DeviceAsyncCopyOp>(op);
+  if (!asyncCopyOp)
+    return nullptr;
+
+  // Create srcElement Value based on `predicate`. The next lines generate
+  // the following code:
+  //
+  //   srcElement = (pred) ?  prevSrcElements : 0;
+  //
+  Location loc = asyncCopyOp->getLoc();
+  Value dstElements =
+      rewriter.create<arith::ConstantOp>(loc, asyncCopyOp.getDstElementsAttr());
+  Value originalSrcElement =
+      asyncCopyOp.getSrcElements() ? asyncCopyOp.getSrcElements() : dstElements;
+  Value c0Index = rewriter.create<arith::ConstantIndexOp>(loc, 0);
+  auto srcElements = rewriter.create<arith::SelectOp>(
+      loc, predicate, originalSrcElement, c0Index);
+  auto asyncCopyZeroFillOp = rewriter.create<nvgpu::DeviceAsyncCopyOp>(
+      loc, nvgpu::DeviceAsyncTokenType::get(asyncCopyOp.getContext()),
+      asyncCopyOp.getDst(), asyncCopyOp.getDstIndices(), asyncCopyOp.getSrc(),
+      asyncCopyOp.getSrcIndices(), asyncCopyOp.getDstElements(), srcElements,
+      UnitAttr());
+  rewriter.replaceOp(asyncCopyOp, asyncCopyZeroFillOp);
+  return asyncCopyZeroFillOp;
+}
+
+/// Applies loop pipelining with the given depth to the given loop so that
+/// copies into the shared memory are pipelined. Doesn't affect other loops.
+/// Returns a pair containing the error state and the pipelined op, the latter
+/// being null in case of any failure. The error state contains a definite error
+/// if the IR has been modified and a silenceable error otherwise.
+static std::tuple<DiagnosedSilenceableFailure, scf::ForOp>
+pipelineForSharedCopies(RewriterBase &rewriter, scf::ForOp forOp, int64_t depth,
+                        bool epiloguePeeling) {
+  llvm::SmallPtrSet<Operation *, 16> stage0Ops;
+  if (failed(collectStage0PipeliningOps(forOp, stage0Ops))) {
+    return std::make_tuple(
+        emitSilenceableFailure(forOp, "cannot find stage 0 ops for pipelining"),
+        scf::ForOp());
+  }
+  if (stage0Ops.empty()) {
+    return std::make_tuple(
+        emitSilenceableFailure(forOp, "no shared memory copy"), scf::ForOp());
+  }
+
+  scf::PipeliningOption options;
+  unsigned maxDepth = depth;
+  auto setAnnotation = [&](Operation *op,
+                           scf::PipeliningOption::PipelinerPart part,
+                           unsigned iteration) {
+    return setAsyncWaitGroupsInFlight(rewriter, op, part, iteration, maxDepth);
+  };
+  options.getScheduleFn =
+      [&](scf::ForOp schedulingFor,
+          std::vector<std::pair<Operation *, unsigned>> &ops) {
+        if (schedulingFor != forOp)
+          return;
+        return getPipelineStages(forOp, ops, maxDepth, stage0Ops);
+      };
+  options.annotateFn = setAnnotation;
+  if (!epiloguePeeling) {
+    options.peelEpilogue = false;
+    options.predicateFn = replaceOpWithPredicatedOp;
+  }
+
+  OpBuilder::InsertionGuard guard(rewriter);
+  rewriter.setInsertionPoint(forOp);
+  bool modifiedIR;
+  FailureOr<scf::ForOp> maybePipelined =
+      pipelineForLoop(rewriter, forOp, options, &modifiedIR);
+  if (succeeded(maybePipelined)) {
+    return std::make_tuple(DiagnosedSilenceableFailure::success(),
+                           *maybePipelined);
+  }
+  return std::make_tuple(
+      modifiedIR
+          ? DiagnosedSilenceableFailure::definiteFailure()
+          : emitSilenceableFailure(forOp, "pipelining preconditions failed"),
+      scf::ForOp());
+}
+
+DiagnosedSilenceableFailure PipelineSharedMemoryCopiesOp::applyToOne(
+    TransformRewriter &rewriter, scf::ForOp forOp,
+    ApplyToEachResultList &results, TransformState &state) {
+  auto [diag, pipelined] = pipelineForSharedCopies(
+      rewriter, forOp, static_cast<int64_t>(getDepth()), getPeelEpilogue());
+  if (diag.succeeded()) {
+    results.push_back(pipelined);
+    return DiagnosedSilenceableFailure::success();
+  }
+  if (diag.isDefiniteFailure()) {
+    auto diag = emitDefiniteFailure("irreversible pipelining failure");
+    if (!getPeelEpilogue()) {
+      diag.attachNote(forOp->getLoc()) << "couldn't predicate?";
+      diag.attachNote(getLoc()) << "try setting " << getPeelEpilogueAttrName();
+    }
+    return diag;
+  }
+
+  return std::move(diag);
+}
+
 //===----------------------------------------------------------------------===//
 // RewriteMatmulAsMmaSyncOp
 //===----------------------------------------------------------------------===//

diff  --git a/mlir/lib/Dialect/SCF/Transforms/LoopPipelining.cpp b/mlir/lib/Dialect/SCF/Transforms/LoopPipelining.cpp
index 9b673d6f1de938..4ff7965c0858a3 100644
--- a/mlir/lib/Dialect/SCF/Transforms/LoopPipelining.cpp
+++ b/mlir/lib/Dialect/SCF/Transforms/LoopPipelining.cpp
@@ -77,7 +77,7 @@ struct LoopPipelinerInternal {
       llvm::DenseMap<std::pair<Value, unsigned>, unsigned> &loopArgMap);
   /// Emits the pipelined kernel. This clones loop operations following user
   /// order and remaps operands defined in a 
diff erent stage as their use.
-  void createKernel(
+  LogicalResult createKernel(
       scf::ForOp newForOp,
       const llvm::MapVector<Value, LiverangeInfo> &crossStageValues,
       const llvm::DenseMap<std::pair<Value, unsigned>, unsigned> &loopArgMap,
@@ -314,7 +314,7 @@ scf::ForOp LoopPipelinerInternal::createKernelLoop(
   return newForOp;
 }
 
-void LoopPipelinerInternal::createKernel(
+LogicalResult LoopPipelinerInternal::createKernel(
     scf::ForOp newForOp,
     const llvm::MapVector<Value, LoopPipelinerInternal::LiverangeInfo>
         &crossStageValues,
@@ -401,6 +401,8 @@ void LoopPipelinerInternal::createKernel(
 
     if (predicates[useStage]) {
       newOp = predicateFn(rewriter, newOp, predicates[useStage]);
+      if (!newOp)
+        return failure();
       // Remap the results to the new predicated one.
       for (auto values : llvm::zip(op->getResults(), newOp->getResults()))
         mapping.map(std::get<0>(values), std::get<1>(values));
@@ -422,9 +424,9 @@ void LoopPipelinerInternal::createKernel(
   for (auto &it : crossStageValues) {
     int64_t version = maxStage - it.second.lastUseStage + 1;
     unsigned numVersionReturned = it.second.lastUseStage - it.second.defStage;
-    // add the original verstion to yield ops.
-    // If there is a liverange spanning across more than 2 stages we need to add
-    // extra arg.
+    // add the original version to yield ops.
+    // If there is a live range spanning across more than 2 stages we need to
+    // add extra arg.
     for (unsigned i = 1; i < numVersionReturned; i++) {
       setValueMapping(it.first, newForOp->getResult(yieldOperands.size()),
                       version++);
@@ -447,6 +449,7 @@ void LoopPipelinerInternal::createKernel(
                     maxStage - defStage + 1);
   }
   rewriter.create<scf::YieldOp>(forOp.getLoc(), yieldOperands);
+  return success();
 }
 
 llvm::SmallVector<Value>
@@ -516,11 +519,17 @@ void LoopPipelinerInternal::setValueMapping(Value key, Value el, int64_t idx) {
 } // namespace
 
 FailureOr<ForOp> mlir::scf::pipelineForLoop(RewriterBase &rewriter, ForOp forOp,
-                                            const PipeliningOption &options) {
+                                            const PipeliningOption &options,
+                                            bool *modifiedIR) {
+  if (modifiedIR)
+    *modifiedIR = false;
   LoopPipelinerInternal pipeliner;
   if (!pipeliner.initializeLoopInfo(forOp, options))
     return failure();
 
+  if (modifiedIR)
+    *modifiedIR = true;
+
   // 1. Emit prologue.
   pipeliner.emitPrologue(rewriter);
 
@@ -540,7 +549,9 @@ FailureOr<ForOp> mlir::scf::pipelineForLoop(RewriterBase &rewriter, ForOp forOp,
       pipeliner.createKernelLoop(crossStageValues, rewriter, loopArgMap);
   // Create the kernel block, order ops based on user choice and remap
   // operands.
-  pipeliner.createKernel(newForOp, crossStageValues, loopArgMap, rewriter);
+  if (failed(pipeliner.createKernel(newForOp, crossStageValues, loopArgMap,
+                                    rewriter)))
+    return failure();
 
   llvm::SmallVector<Value> returnValues =
       newForOp.getResults().take_front(forOp->getNumResults());

diff  --git a/mlir/test/Dialect/NVGPU/transform-pipeline-shared.mlir b/mlir/test/Dialect/NVGPU/transform-pipeline-shared.mlir
new file mode 100644
index 00000000000000..56af734b398510
--- /dev/null
+++ b/mlir/test/Dialect/NVGPU/transform-pipeline-shared.mlir
@@ -0,0 +1,182 @@
+// RUN: mlir-opt %s --test-transform-dialect-interpreter --split-input-file --verify-diagnostics | FileCheck %s
+
+func.func @simple_depth_2_unpeeled(%global: memref<?xf32>, %result: memref<?xf32> ) {
+  %c0 = arith.constant 0 : index
+  %c100 = arith.constant 100 : index
+  %c4 = arith.constant 4 : index
+  %shared = memref.alloc(%c100) : memref<?xf32, #gpu.address_space<workgroup>>
+  %c0f = arith.constant 0.0 : f32
+  // Predication is not currently implemented for transfer_read/write, so this is expected to fail.
+  // expected-note @below {{couldn't predicate}}
+  scf.for %i = %c0 to %c100 step %c4 iter_args(%accum = %c0f) -> f32 {
+    %mem = vector.transfer_read %global[%i], %c0f : memref<?xf32>, vector<4xf32>
+    vector.transfer_write %mem, %shared[%i] : vector<4xf32>, memref<?xf32, #gpu.address_space<workgroup>>
+    %0 = arith.addf %accum, %accum : f32
+    scf.yield %0 : f32
+  }
+  return
+}
+
+!t = !transform.any_op
+
+transform.sequence failures(propagate) {
+^bb0(%arg0: !t):
+  %loop = transform.structured.match ops{["scf.for"]} in %arg0 : (!t) -> !t
+  // expected-error @below {{irreversible pipelining failure}}
+  // expected-note @below {{try setting "peel_epilogue"}}
+  transform.nvgpu.pipeline_shared_memory_copies failures(propagate) %loop { depth = 2 } : (!t) -> !t
+}
+
+// -----
+
+// Loop pipeliner is tested separately, just verify the overall shape of the IR here.
+
+func.func private @body(index, memref<?xf32, #gpu.address_space<workgroup>>)
+
+// CHECK-LABEL: @simple_depth_2_peeled
+// CHECK-SAME: %[[ARG:.+]]: memref
+func.func @simple_depth_2_peeled(%global: memref<?xf32>) {
+  %c0 = arith.constant 0 : index
+  %c100 = arith.constant 100 : index
+  %c200 = arith.constant 200 : index
+  %c4 = arith.constant 4 : index
+  // CHECK: memref.alloc
+  %shared = memref.alloc(%c200) : memref<?xf32, #gpu.address_space<workgroup>>
+  %c0f = arith.constant 0.0 : f32
+  // CHECK: %[[LOADED1:.+]] = vector.transfer_read %[[ARG]]
+  // CHECK: %[[LOADED2:.+]] = vector.transfer_read %[[ARG]]
+  // CHECK: %[[LOOP:.+]]:2 = scf.for {{.*}} iter_args(%[[IA1:.+]] = %[[LOADED1]], %[[IA2:.+]] = %[[LOADED2]])
+  // CHECK:   vector.transfer_write %[[IA1]]
+  // CHECK:   func.call @body
+  // CHECK:   %[[LOCAL_LOADED:.+]] = vector.transfer_read %[[ARG]]
+  // CHECK:   scf.yield %[[IA2]], %[[LOCAL_LOADED]]
+  scf.for %i = %c0 to %c100 step %c4 {
+    %mem = vector.transfer_read %global[%i], %c0f : memref<?xf32>, vector<4xf32>
+    vector.transfer_write %mem, %shared[%i] : vector<4xf32>, memref<?xf32, #gpu.address_space<workgroup>>
+    func.call @body(%i, %shared) : (index, memref<?xf32, #gpu.address_space<workgroup>>) -> ()
+  }
+  // CHECK: vector.transfer_write %[[LOOP]]#0
+  // CHECK: call @body
+  // CHECK: vector.transfer_write %[[LOOP]]#1
+  // CHECK: call @body
+  return
+}
+
+!t = !transform.any_op
+
+transform.sequence failures(propagate) {
+^bb0(%arg0: !t):
+  %loop = transform.structured.match ops{["scf.for"]} in %arg0 : (!t) -> !t
+  transform.nvgpu.pipeline_shared_memory_copies failures(propagate) %loop { depth = 2, peel_epilogue } : (!t) -> !t
+}
+
+// -----
+
+// CHECK-LABEL: @async_depth_2_predicated
+// CHECK-SAME: %[[GLOBAL:.+]]: memref
+func.func @async_depth_2_predicated(%global: memref<?xf32>) {
+  %c0 = arith.constant 0 : index
+  %c98 = arith.constant 98 : index
+  %c100 = arith.constant 100 : index
+  %c200 = arith.constant 200 : index
+  // CHECK: %[[C4:.+]] = arith.constant 4
+  %c4 = arith.constant 4 : index
+  // CHECK: %[[SHARED:.+]] = memref.alloc{{.*}} #gpu.address_space<workgroup>
+  %shared = memref.alloc(%c200) : memref<?xf32, #gpu.address_space<workgroup>>
+  %c0f = arith.constant 0.0 : f32
+  // CHECK: %[[TOKEN0:.+]] = nvgpu.device_async_copy
+  // CHECK: %[[TOKEN1:.+]] = nvgpu.device_async_copy
+  // CHECK: scf.for %[[I:.+]] = {{.*}} iter_args
+  // CHECK-SAME: %[[ITER_ARG0:.+]] = %[[TOKEN0]]
+  // CHECK-SAME: %[[ITER_ARG1:.+]] = %[[TOKEN1]]
+  scf.for %i = %c0 to %c98 step %c4 {
+    // Condition for the predication "select" below.
+    // CHECK:   %[[C90:.+]] = arith.constant 90
+    // CHECK:   %[[CMP0:.+]] = arith.cmpi slt, %[[I]], %[[C90]]
+    // CHECK:   nvgpu.device_async_wait %[[ITER_ARG0]] {numGroups = 1
+
+    // Original "select" with updated induction variable.
+    // CHECK:   %[[C96:.+]] = arith.constant 96
+    // CHECK:   %[[C8:.+]] = arith.constant 8
+    // CHECK:   %[[I_PLUS_8:.+]] = arith.addi %[[I]], %[[C8]]
+    // CHECK:   %[[CMP1:.+]] = arith.cmpi slt, %[[I_PLUS_8]], %[[C96]]
+    // CHECK:   %[[C2:.+]] = arith.constant 2
+    // CHECK:   %[[SELECTED0:.+]] = arith.select %[[CMP1]], %[[C4]], %[[C2]]
+    %c96 = arith.constant 96 : index
+    %cond = arith.cmpi slt, %i, %c96 : index
+    %c2 = arith.constant 2 : index
+    %read_size = arith.select %cond, %c4, %c2 : index
+
+    // Updated induction variables (two more) for the device_async_copy below.
+    // These are generated repeatedly by the pipeliner.
+    // CHECK:   %[[C8_2:.+]] = arith.constant 8
+    // CHECK:   %[[I_PLUS_8_2:.+]] = arith.addi %[[I]], %[[C8_2]]
+    // CHECK:   %[[C8_3:.+]] = arith.constant 8
+    // CHECK:   %[[I_PLUS_8_3:.+]] = arith.addi %[[I]], %[[C8_3]]
+
+    // The second "select" is generated by predication and selects 0 for
+    // the two last iterations.
+    // CHECK:   %[[C0:.+]] = arith.constant 0
+    // CHECK:   %[[SELECTED1:.+]] = arith.select %[[CMP0]], %[[SELECTED0]], %[[C0]]
+    // CHECK:   %[[ASYNC_TOKEN:.+]] = nvgpu.device_async_copy %[[GLOBAL]][%[[I_PLUS_8_3]]], %[[SHARED]][%[[I_PLUS_8_2]]], 4, %[[SELECTED1]]
+    %token = nvgpu.device_async_copy %global[%i], %shared[%i], 4, %read_size
+      : memref<?xf32> to memref<?xf32, #gpu.address_space<workgroup>>
+
+    nvgpu.device_async_wait %token
+
+    // CHECK: scf.yield %[[ITER_ARG1]], %[[ASYNC_TOKEN]]
+  }
+  // There is no need to wait for the last copies as it it was fully predicated
+  // out and doesn't load the original data.
+  // CHECK-NOT: nvgpu.device_async_wait
+  return
+}
+
+
+!t = !transform.any_op
+
+transform.sequence failures(propagate) {
+^bb0(%arg0: !t):
+  %loop = transform.structured.match ops{["scf.for"]} in %arg0 : (!t) -> !t
+  transform.nvgpu.pipeline_shared_memory_copies failures(propagate) %loop { depth = 2 } : (!t) -> !t
+}
+
+// -----
+
+// CHECK-LABEL: @async_depth_2_peeled
+func.func @async_depth_2_peeled(%global: memref<?xf32>) {
+  %c0 = arith.constant 0 : index
+  %c98 = arith.constant 98 : index
+  %c100 = arith.constant 100 : index
+  %c4 = arith.constant 4 : index
+  %shared = memref.alloc(%c100) : memref<?xf32, #gpu.address_space<workgroup>>
+  %c0f = arith.constant 0.0 : f32
+  // CHECK: nvgpu.device_async_copy
+  // CHECK: nvgpu.device_async_copy
+  // CHECK: scf.for
+  // CHECK:   nvgpu.device_async_wait %{{.*}} {numGroups = 1
+  // CHECK:   arith.select
+  // CHECK:   nvgpu.device_async_copy
+  // CHECK:   scf.yield
+  // CHECK: nvgpu.device_async_wait %{{.*}} {numGroups = 1
+  // CHEKC: nvgpu.device_async_wait %{{.*}} {numGroups = 0
+  scf.for %i = %c0 to %c98 step %c4 {
+    %c96 = arith.constant 96 : index
+    %cond = arith.cmpi slt, %i, %c96 : index
+    %c2 = arith.constant 2 : index
+    %read_size = arith.select %cond, %c4, %c2 : index
+    %token = nvgpu.device_async_copy %global[%i], %shared[%i], 4, %read_size
+      : memref<?xf32> to memref<?xf32, #gpu.address_space<workgroup>>
+    nvgpu.device_async_wait %token
+  }
+  return
+}
+
+
+!t = !transform.any_op
+
+transform.sequence failures(propagate) {
+^bb0(%arg0: !t):
+  %loop = transform.structured.match ops{["scf.for"]} in %arg0 : (!t) -> !t
+  transform.nvgpu.pipeline_shared_memory_copies failures(propagate) %loop { depth = 2, peel_epilogue } : (!t) -> !t
+}

diff  --git a/utils/bazel/llvm-project-overlay/mlir/BUILD.bazel b/utils/bazel/llvm-project-overlay/mlir/BUILD.bazel
index 2b48f8f2d40376..7e1d873572cdf3 100644
--- a/utils/bazel/llvm-project-overlay/mlir/BUILD.bazel
+++ b/utils/bazel/llvm-project-overlay/mlir/BUILD.bazel
@@ -2786,6 +2786,7 @@ cc_library(
     includes = ["include"],
     deps = [
         ":AffineDialect",
+        ":Analysis",
         ":ArithDialect",
         ":ArithUtils",
         ":DialectUtils",
@@ -2795,6 +2796,8 @@ cc_library(
         ":MemRefDialect",
         ":NVGPUDialect",
         ":NVGPUTransformOpsIncGen",
+        ":SCFDialect",
+        ":SCFTransforms",
         ":Support",
         ":TransformDialect",
         ":VectorDialect",


        


More information about the Mlir-commits mailing list