[Mlir-commits] [mlir] [mlir][inliner] Refactor MLIR inliner pass and utils. (PR #84059)
Slava Zakharin
llvmlistbot at llvm.org
Tue Mar 5 13:54:45 PST 2024
https://github.com/vzakhari updated https://github.com/llvm/llvm-project/pull/84059
>From dfc97ebd050c494da5b6ee04e084c8de8526a516 Mon Sep 17 00:00:00 2001
From: Slava Zakharin <szakharin at nvidia.com>
Date: Thu, 22 Feb 2024 13:29:12 -0800
Subject: [PATCH 1/3] [mlir][inliner] Refactor MLIR inliner pass and utils.
This is just code refactoring done as a preparation for adding
MLIR inliner cost model hook(s).
Related discussion: https://discourse.llvm.org/t/inliner-cost-model/2992
The logic of SCC-based MLIR inliner is separated into the Inliner
implementation. The MLIR inliner pass becomes, well, just a pass
that invokes the SCC-based MLIR inliner.
---
mlir/include/mlir/Transforms/Inliner.h | 124 ++++
mlir/include/mlir/Transforms/Passes.td | 6 +-
mlir/lib/Transforms/CMakeLists.txt | 2 +-
mlir/lib/Transforms/InlinerPass.cpp | 158 ++++++
mlir/lib/Transforms/Utils/CMakeLists.txt | 1 +
mlir/lib/Transforms/{ => Utils}/Inliner.cpp | 534 ++++++++----------
mlir/test/Dialect/Affine/inlining.mlir | 2 +-
.../Dialect/SPIRV/Transforms/inlining.mlir | 2 +-
.../inlining-dump-default-pipeline.mlir | 2 +-
mlir/test/Transforms/inlining-recursive.mlir | 4 +-
mlir/test/Transforms/inlining.mlir | 6 +-
11 files changed, 517 insertions(+), 324 deletions(-)
create mode 100644 mlir/include/mlir/Transforms/Inliner.h
create mode 100644 mlir/lib/Transforms/InlinerPass.cpp
rename mlir/lib/Transforms/{ => Utils}/Inliner.cpp (76%)
diff --git a/mlir/include/mlir/Transforms/Inliner.h b/mlir/include/mlir/Transforms/Inliner.h
new file mode 100644
index 00000000000000..a74a32fcfec430
--- /dev/null
+++ b/mlir/include/mlir/Transforms/Inliner.h
@@ -0,0 +1,124 @@
+//===- Inliner.h - Inliner pass utilities -----------------------*- 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
+//
+//===----------------------------------------------------------------------===//
+//
+// This header file declares utility structures for the inliner pass.
+//
+//===----------------------------------------------------------------------===//
+
+#ifndef MLIR_TRANSFORMS_INLINER_H
+#define MLIR_TRANSFORMS_INLINER_H
+
+#include "mlir/Analysis/CallGraph.h"
+#include "mlir/Interfaces/CallInterfaces.h"
+#include "mlir/Pass/AnalysisManager.h"
+#include "mlir/Pass/PassManager.h"
+#include "mlir/Support/LogicalResult.h"
+#include "llvm/ADT/StringMap.h"
+
+namespace mlir {
+class OpPassManager;
+class Operation;
+
+class InlinerConfig {
+public:
+ using PreInlineCallableOptPipelineTy = std::function<void(OpPassManager &)>;
+ using OpPipelinesTy = llvm::StringMap<OpPassManager>;
+
+ InlinerConfig() = default;
+ InlinerConfig(PreInlineCallableOptPipelineTy preInlineCallableOptPipeline,
+ unsigned maxInliningIterations)
+ : preInlineCallableOptPipeline(std::move(preInlineCallableOptPipeline)),
+ maxInliningIterations(maxInliningIterations) {}
+
+ const PreInlineCallableOptPipelineTy &
+ getPreInlineCallableOptPipeline() const {
+ return preInlineCallableOptPipeline;
+ }
+ const OpPipelinesTy &getOpPipelines() const { return opPipelines; }
+ unsigned getMaxInliningIterations() const { return maxInliningIterations; }
+ void
+ setPreInlineCallableOptPipeline(PreInlineCallableOptPipelineTy pipeline) {
+ preInlineCallableOptPipeline = std::move(pipeline);
+ }
+ void setOpPipelines(OpPipelinesTy pipelines) {
+ opPipelines = std::move(pipelines);
+ }
+ void setMaxInliningIterations(unsigned max) { maxInliningIterations = max; }
+
+private:
+ /// An optional function that constructs an optimization pipeline for
+ /// a given operation.
+ PreInlineCallableOptPipelineTy preInlineCallableOptPipeline;
+ /// A map of operation names to pass pipelines to use when optimizing
+ /// callable operations of these types. This provides a specialized pipeline
+ /// instead of the one produced by preInlineCallableOptPipeline.
+ OpPipelinesTy opPipelines;
+ /// For SCC-based inlining algorithms, specifies maximum number of iterations
+ /// when inlining within an SCC.
+ unsigned maxInliningIterations{0};
+};
+
+/// This is an implementation of the inliner
+/// that operates bottom up over the Strongly Connected Components(SCCs)
+/// of the CallGraph. This enables a more incremental propagation
+/// of inlining decisions from the leafs to the roots of the callgraph.
+///
+/// Derived implementations may rely on the same algorithm, but override
+/// the provided hooks to tune various algorithm aspects.
+class Inliner {
+public:
+ using RunPipelineHelperTy = std::function<LogicalResult(
+ Pass &pass, OpPassManager &pipeline, Operation *op)>;
+
+ virtual ~Inliner() {}
+ Inliner(Operation *op, CallGraph &cg, Pass &pass, AnalysisManager am,
+ RunPipelineHelperTy runPipelineHelper, const InlinerConfig &config)
+ : op(op), cg(cg), pass(pass), am(am),
+ runPipelineHelper(std::move(runPipelineHelper)), config(config) {}
+ Inliner(Inliner &) = delete;
+ void operator=(const Inliner &) = delete;
+
+ /// Perform inlining on a OpTrait::SymbolTable operation.
+ LogicalResult doInlining();
+
+ /// This struct represents a resolved call to a given callgraph node. Given
+ /// that the call does not actually contain a direct reference to the
+ /// Region(CallGraphNode) that it is dispatching to, we need to resolve them
+ /// explicitly.
+ struct ResolvedCall {
+ ResolvedCall(CallOpInterface call, CallGraphNode *sourceNode,
+ CallGraphNode *targetNode)
+ : call(call), sourceNode(sourceNode), targetNode(targetNode) {}
+ CallOpInterface call;
+ CallGraphNode *sourceNode, *targetNode;
+ };
+
+protected:
+ /// An OpTrait::SymbolTable operation to run the inlining on.
+ Operation *op;
+ /// A CallGraph analysis for the given operation.
+ CallGraph &cg;
+ /// A reference to the pass using this inliner.
+ Pass &pass;
+ /// Analysis manager for the given operation instance.
+ AnalysisManager am;
+ /// A callback for running a nested pass pipeline on the operation
+ /// contained within the main operation.
+ const RunPipelineHelperTy runPipelineHelper;
+ /// The inliner configuration parameters.
+ const InlinerConfig &config;
+
+private:
+ /// Forward declaration of the class providing the actual implementation.
+ class Impl;
+
+public:
+};
+} // namespace mlir
+
+#endif // MLIR_TRANSFORMS_INLINER_H
diff --git a/mlir/include/mlir/Transforms/Passes.td b/mlir/include/mlir/Transforms/Passes.td
index 2d2d54fb8fb5ea..1e61026d7fc3c6 100644
--- a/mlir/include/mlir/Transforms/Passes.td
+++ b/mlir/include/mlir/Transforms/Passes.td
@@ -268,8 +268,10 @@ def Inliner : Pass<"inline"> {
let summary = "Inline function calls";
let constructor = "mlir::createInlinerPass()";
let options = [
- Option<"defaultPipelineStr", "default-pipeline", "std::string",
- /*default=*/"\"canonicalize\"", "The default optimizer pipeline used for callables">,
+ Option<"preInlineCallableOptPipelineStr", "pre-inline-pipeline",
+ "std::string", /*default=*/"\"canonicalize\"",
+ "The optimizer pipeline used for callables that do not have "
+ "a dedicated optimizer pipeline in opPipelineList">,
ListOption<"opPipelineList", "op-pipelines", "OpPassManager",
"Callable operation specific optimizer pipelines (in the form "
"of `dialect.op(pipeline)`)">,
diff --git a/mlir/lib/Transforms/CMakeLists.txt b/mlir/lib/Transforms/CMakeLists.txt
index af51a4ab1157f1..6c32ecf8a2a2f1 100644
--- a/mlir/lib/Transforms/CMakeLists.txt
+++ b/mlir/lib/Transforms/CMakeLists.txt
@@ -5,7 +5,7 @@ add_mlir_library(MLIRTransforms
ControlFlowSink.cpp
CSE.cpp
GenerateRuntimeVerification.cpp
- Inliner.cpp
+ InlinerPass.cpp
LocationSnapshot.cpp
LoopInvariantCodeMotion.cpp
Mem2Reg.cpp
diff --git a/mlir/lib/Transforms/InlinerPass.cpp b/mlir/lib/Transforms/InlinerPass.cpp
new file mode 100644
index 00000000000000..5df598f1526b80
--- /dev/null
+++ b/mlir/lib/Transforms/InlinerPass.cpp
@@ -0,0 +1,158 @@
+//===- InlinerPass.cpp - Pass to inline function calls --------------------===//
+//
+// 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
+//
+//===----------------------------------------------------------------------===//
+//
+// This file implements a basic inlining algorithm that operates bottom up over
+// the Strongly Connect Components(SCCs) of the CallGraph. This enables a more
+// incremental propagation of inlining decisions from the leafs to the roots of
+// the callgraph.
+//
+//===----------------------------------------------------------------------===//
+
+#include "mlir/Transforms/Passes.h"
+
+#include "mlir/Analysis/CallGraph.h"
+#include "mlir/Pass/PassManager.h"
+#include "mlir/Transforms/Inliner.h"
+
+namespace mlir {
+#define GEN_PASS_DEF_INLINER
+#include "mlir/Transforms/Passes.h.inc"
+} // namespace mlir
+
+using namespace mlir;
+
+/// This function implements the inliner optimization pipeline.
+static void defaultInlinerOptPipeline(OpPassManager &pm) {
+ pm.addPass(createCanonicalizerPass());
+}
+
+//===----------------------------------------------------------------------===//
+// InlinerPass
+//===----------------------------------------------------------------------===//
+
+namespace {
+class InlinerPass : public impl::InlinerBase<InlinerPass> {
+public:
+ InlinerPass();
+ InlinerPass(const InlinerPass &) = default;
+ InlinerPass(
+ std::function<void(OpPassManager &)> preInlineCallableOptPipeline);
+ InlinerPass(std::function<void(OpPassManager &)> preInlineCallableOptPipeline,
+ llvm::StringMap<OpPassManager> opPipelines);
+ void runOnOperation() override;
+
+ /// A callback provided to the inliner driver to execute
+ /// the specified pass pipeline on the given operation
+ /// within the context of the current inliner pass,
+ /// which is passed as the first argument.
+ /// runPipeline API is protected within the Pass class,
+ /// so this helper is required to call it from the foreign
+ /// inliner driver.
+ static LogicalResult runPipelineHelper(Pass &pass, OpPassManager &pipeline,
+ Operation *op) {
+ return mlir::cast<InlinerPass>(pass).runPipeline(pipeline, op);
+ }
+
+private:
+ /// Attempt to initialize the options of this pass from the given string.
+ /// Derived classes may override this method to hook into the point at which
+ /// options are initialized, but should generally always invoke this base
+ /// class variant.
+ LogicalResult initializeOptions(StringRef options) override;
+
+ /// Inliner configuration parameters created from the pass options.
+ InlinerConfig config;
+};
+} // namespace
+
+InlinerPass::InlinerPass() : InlinerPass(defaultInlinerOptPipeline) {}
+
+InlinerPass::InlinerPass(
+ std::function<void(OpPassManager &)> preInlineCallableOptPipelineArg)
+ : InlinerPass(std::move(preInlineCallableOptPipelineArg),
+ llvm::StringMap<OpPassManager>{}) {}
+
+InlinerPass::InlinerPass(
+ std::function<void(OpPassManager &)> preInlineCallableOptPipeline,
+ llvm::StringMap<OpPassManager> opPipelines)
+ : config(std::move(preInlineCallableOptPipeline), maxInliningIterations) {
+ if (opPipelines.empty())
+ return;
+
+ // Update the option for the op specific optimization pipelines.
+ for (auto &it : opPipelines)
+ opPipelineList.addValue(it.second);
+ config.setOpPipelines(std::move(opPipelines));
+}
+
+void InlinerPass::runOnOperation() {
+ CallGraph &cg = getAnalysis<CallGraph>();
+
+ // The inliner should only be run on operations that define a symbol table,
+ // as the callgraph will need to resolve references.
+ Operation *op = getOperation();
+ if (!op->hasTrait<OpTrait::SymbolTable>()) {
+ op->emitOpError() << " was scheduled to run under the inliner, but does "
+ "not define a symbol table";
+ return signalPassFailure();
+ }
+
+ // Get an instance of the inliner.
+ Inliner inliner(op, cg, *this, getAnalysisManager(), runPipelineHelper,
+ config);
+
+ // Run the inlining.
+ if (failed(inliner.doInlining()))
+ signalPassFailure();
+ return;
+}
+
+LogicalResult InlinerPass::initializeOptions(StringRef options) {
+ if (failed(Pass::initializeOptions(options)))
+ return failure();
+
+ // Initialize the pipeline builder for operations without the dedicated
+ // optimization pipeline in opPipelineList to use the option string.
+ // TODO: Use a generic pass manager for the pre-inline pipeline, and remove
+ // this.
+ if (!preInlineCallableOptPipelineStr.empty()) {
+ std::string preInlineCallableOptPipelineCopy =
+ preInlineCallableOptPipelineStr;
+ config.setPreInlineCallableOptPipeline([=](OpPassManager &pm) {
+ (void)parsePassPipeline(preInlineCallableOptPipelineCopy, pm);
+ });
+ } else if (preInlineCallableOptPipelineStr.getNumOccurrences()) {
+ config.setPreInlineCallableOptPipeline(nullptr);
+ }
+
+ // Initialize the op specific pass pipelines.
+ llvm::StringMap<OpPassManager> pipelines;
+ for (OpPassManager pipeline : opPipelineList)
+ if (!pipeline.empty())
+ pipelines.try_emplace(pipeline.getOpAnchorName(), pipeline);
+ config.setOpPipelines(std::move(pipelines));
+
+ config.setMaxInliningIterations(maxInliningIterations);
+
+ return success();
+}
+
+std::unique_ptr<Pass> mlir::createInlinerPass() {
+ return std::make_unique<InlinerPass>();
+}
+std::unique_ptr<Pass>
+mlir::createInlinerPass(llvm::StringMap<OpPassManager> opPipelines) {
+ return std::make_unique<InlinerPass>(defaultInlinerOptPipeline,
+ std::move(opPipelines));
+}
+std::unique_ptr<Pass> mlir::createInlinerPass(
+ llvm::StringMap<OpPassManager> opPipelines,
+ std::function<void(OpPassManager &)> preInlineCallableOptPipelineBuilder) {
+ return std::make_unique<InlinerPass>(
+ std::move(preInlineCallableOptPipelineBuilder), std::move(opPipelines));
+}
diff --git a/mlir/lib/Transforms/Utils/CMakeLists.txt b/mlir/lib/Transforms/Utils/CMakeLists.txt
index 1c608e0634a67e..d6aac0e2da4f5a 100644
--- a/mlir/lib/Transforms/Utils/CMakeLists.txt
+++ b/mlir/lib/Transforms/Utils/CMakeLists.txt
@@ -5,6 +5,7 @@ add_mlir_library(MLIRTransformUtils
DialectConversion.cpp
FoldUtils.cpp
GreedyPatternRewriteDriver.cpp
+ Inliner.cpp
InliningUtils.cpp
LoopInvariantCodeMotionUtils.cpp
OneToNTypeConversion.cpp
diff --git a/mlir/lib/Transforms/Inliner.cpp b/mlir/lib/Transforms/Utils/Inliner.cpp
similarity index 76%
rename from mlir/lib/Transforms/Inliner.cpp
rename to mlir/lib/Transforms/Utils/Inliner.cpp
index b32b0fc28c78b0..be30159c9ac976 100644
--- a/mlir/lib/Transforms/Inliner.cpp
+++ b/mlir/lib/Transforms/Utils/Inliner.cpp
@@ -1,4 +1,4 @@
-//===- Inliner.cpp - Pass to inline function calls ------------------------===//
+//===- Inliner.cpp ---- SCC-based inliner ---------------------------------===//
//
// Part of the LLVM Project, under the Apache License v2.0 with LLVM Exceptions.
// See https://llvm.org/LICENSE.txt for license information.
@@ -6,38 +6,29 @@
//
//===----------------------------------------------------------------------===//
//
-// This file implements a basic inlining algorithm that operates bottom up over
-// the Strongly Connect Components(SCCs) of the CallGraph. This enables a more
-// incremental propagation of inlining decisions from the leafs to the roots of
-// the callgraph.
+// This file implements Inliner that uses a basic inlining
+// algorithm that operates bottom up over the Strongly Connect Components(SCCs)
+// of the CallGraph. This enables a more incremental propagation of inlining
+// decisions from the leafs to the roots of the callgraph.
//
//===----------------------------------------------------------------------===//
-#include "mlir/Transforms/Passes.h"
-
-#include "mlir/Analysis/CallGraph.h"
+#include "mlir/Transforms/Inliner.h"
#include "mlir/IR/Threading.h"
#include "mlir/Interfaces/CallInterfaces.h"
#include "mlir/Interfaces/SideEffectInterfaces.h"
-#include "mlir/Pass/PassManager.h"
+#include "mlir/Pass/Pass.h"
#include "mlir/Support/DebugStringHelper.h"
#include "mlir/Transforms/InliningUtils.h"
#include "llvm/ADT/SCCIterator.h"
+#include "llvm/ADT/SmallPtrSet.h"
#include "llvm/Support/Debug.h"
-namespace mlir {
-#define GEN_PASS_DEF_INLINER
-#include "mlir/Transforms/Passes.h.inc"
-} // namespace mlir
-
#define DEBUG_TYPE "inlining"
using namespace mlir;
-/// This function implements the default inliner optimization pipeline.
-static void defaultInlinerOptPipeline(OpPassManager &pm) {
- pm.addPass(createCanonicalizerPass());
-}
+using ResolvedCall = Inliner::ResolvedCall;
//===----------------------------------------------------------------------===//
// Symbol Use Tracking
@@ -306,20 +297,6 @@ static LogicalResult runTransformOnCGSCCs(
return success();
}
-namespace {
-/// This struct represents a resolved call to a given callgraph node. Given that
-/// the call does not actually contain a direct reference to the
-/// Region(CallGraphNode) that it is dispatching to, we need to resolve them
-/// explicitly.
-struct ResolvedCall {
- ResolvedCall(CallOpInterface call, CallGraphNode *sourceNode,
- CallGraphNode *targetNode)
- : call(call), sourceNode(sourceNode), targetNode(targetNode) {}
- CallOpInterface call;
- CallGraphNode *sourceNode, *targetNode;
-};
-} // namespace
-
/// Collect all of the callable operations within the given range of blocks. If
/// `traverseNestedCGNodes` is true, this will also collect call operations
/// inside of nested callgraph nodes.
@@ -368,7 +345,7 @@ static void collectCallOps(iterator_range<Region::iterator> blocks,
}
//===----------------------------------------------------------------------===//
-// Inliner
+// InlinerInterfaceImpl
//===----------------------------------------------------------------------===//
#ifndef NDEBUG
@@ -397,9 +374,9 @@ static bool inlineHistoryIncludes(
namespace {
/// This class provides a specialization of the main inlining interface.
-struct Inliner : public InlinerInterface {
- Inliner(MLIRContext *context, CallGraph &cg,
- SymbolTableCollection &symbolTable)
+struct InlinerInterfaceImpl : public InlinerInterface {
+ InlinerInterfaceImpl(MLIRContext *context, CallGraph &cg,
+ SymbolTableCollection &symbolTable)
: InlinerInterface(context), cg(cg), symbolTable(symbolTable) {}
/// Process a set of blocks that have been inlined. This callback is invoked
@@ -442,45 +419,173 @@ struct Inliner : public InlinerInterface {
};
} // namespace
-/// Returns true if the given call should be inlined.
-static bool shouldInline(ResolvedCall &resolvedCall) {
- // Don't allow inlining terminator calls. We currently don't support this
- // case.
- if (resolvedCall.call->hasTrait<OpTrait::IsTerminator>())
- return false;
+namespace mlir {
- // Don't allow inlining if the target is an ancestor of the call. This
- // prevents inlining recursively.
- Region *callableRegion = resolvedCall.targetNode->getCallableRegion();
- if (callableRegion->isAncestor(resolvedCall.call->getParentRegion()))
- return false;
+class Inliner::Impl {
+public:
+ Impl(Inliner &inliner) : inliner(inliner) {}
- // Don't allow inlining if the callee has multiple blocks (unstructured
- // control flow) but we cannot be sure that the caller region supports that.
- bool calleeHasMultipleBlocks =
- llvm::hasNItemsOrMore(*callableRegion, /*N=*/2);
- // If both parent ops have the same type, it is safe to inline. Otherwise,
- // decide based on whether the op has the SingleBlock trait or not.
- // Note: This check does currently not account for SizedRegion/MaxSizedRegion.
- auto callerRegionSupportsMultipleBlocks = [&]() {
- return callableRegion->getParentOp()->getName() ==
- resolvedCall.call->getParentOp()->getName() ||
- !resolvedCall.call->getParentOp()
- ->mightHaveTrait<OpTrait::SingleBlock>();
- };
- if (calleeHasMultipleBlocks && !callerRegionSupportsMultipleBlocks())
- return false;
+ /// Attempt to inline calls within the given scc, and run simplifications,
+ /// until a fixed point is reached. This allows for the inlining of newly
+ /// devirtualized calls. Returns failure if there was a fatal error during
+ /// inlining.
+ LogicalResult inlineSCC(InlinerInterfaceImpl &inlinerIface,
+ CGUseList &useList, CallGraphSCC ¤tSCC,
+ MLIRContext *context);
- // Otherwise, inline.
- return true;
+private:
+ /// Optimize the nodes within the given SCC with one of the held optimization
+ /// pass pipelines. Returns failure if an error occurred during the
+ /// optimization of the SCC, success otherwise.
+ LogicalResult optimizeSCC(CallGraph &cg, CGUseList &useList,
+ CallGraphSCC ¤tSCC, MLIRContext *context);
+
+ /// Optimize the nodes within the given SCC in parallel. Returns failure if an
+ /// error occurred during the optimization of the SCC, success otherwise.
+ LogicalResult optimizeSCCAsync(MutableArrayRef<CallGraphNode *> nodesToVisit,
+ MLIRContext *context);
+
+ /// Optimize the given callable node with one of the pass managers provided
+ /// with `pipelines`, or the generic pre-inline pipeline. Returns failure if
+ /// an error occurred during the optimization of the callable, success
+ /// otherwise.
+ LogicalResult optimizeCallable(CallGraphNode *node,
+ llvm::StringMap<OpPassManager> &pipelines);
+
+ /// Attempt to inline calls within the given scc. This function returns
+ /// success if any calls were inlined, failure otherwise.
+ LogicalResult inlineCallsInSCC(InlinerInterfaceImpl &inlinerIface,
+ CGUseList &useList, CallGraphSCC ¤tSCC);
+
+ /// Returns true if the given call should be inlined.
+ bool shouldInline(ResolvedCall &resolvedCall);
+
+private:
+ Inliner &inliner;
+ llvm::SmallVector<llvm::StringMap<OpPassManager>> pipelines;
+};
+
+LogicalResult Inliner::Impl::inlineSCC(InlinerInterfaceImpl &inlinerIface,
+ CGUseList &useList,
+ CallGraphSCC ¤tSCC,
+ MLIRContext *context) {
+ // Continuously simplify and inline until we either reach a fixed point, or
+ // hit the maximum iteration count. Simplifying early helps to refine the cost
+ // model, and in future iterations may devirtualize new calls.
+ unsigned iterationCount = 0;
+ do {
+ if (failed(optimizeSCC(inlinerIface.cg, useList, currentSCC, context)))
+ return failure();
+ if (failed(inlineCallsInSCC(inlinerIface, useList, currentSCC)))
+ break;
+ } while (++iterationCount < inliner.config.getMaxInliningIterations());
+ return success();
+}
+
+LogicalResult Inliner::Impl::optimizeSCC(CallGraph &cg, CGUseList &useList,
+ CallGraphSCC ¤tSCC,
+ MLIRContext *context) {
+ // Collect the sets of nodes to simplify.
+ SmallVector<CallGraphNode *, 4> nodesToVisit;
+ for (auto *node : currentSCC) {
+ if (node->isExternal())
+ continue;
+
+ // Don't simplify nodes with children. Nodes with children require special
+ // handling as we may remove the node during simplification. In the future,
+ // we should be able to handle this case with proper node deletion tracking.
+ if (node->hasChildren())
+ continue;
+
+ // We also won't apply simplifications to nodes that can't have passes
+ // scheduled on them.
+ auto *region = node->getCallableRegion();
+ if (!region->getParentOp()->hasTrait<OpTrait::IsIsolatedFromAbove>())
+ continue;
+ nodesToVisit.push_back(node);
+ }
+ if (nodesToVisit.empty())
+ return success();
+
+ // Optimize each of the nodes within the SCC in parallel.
+ if (failed(optimizeSCCAsync(nodesToVisit, context)))
+ return failure();
+
+ // Recompute the uses held by each of the nodes.
+ for (CallGraphNode *node : nodesToVisit)
+ useList.recomputeUses(node, cg);
+ return success();
+}
+
+LogicalResult
+Inliner::Impl::optimizeSCCAsync(MutableArrayRef<CallGraphNode *> nodesToVisit,
+ MLIRContext *ctx) {
+ // We must maintain a fixed pool of pass managers which is at least as large
+ // as the maximum parallelism of the failableParallelForEach below.
+ // Note: The number of pass managers here needs to remain constant
+ // to prevent issues with pass instrumentations that rely on having the same
+ // pass manager for the main thread.
+ size_t numThreads = ctx->getNumThreads();
+ const auto &opPipelines = inliner.config.getOpPipelines();
+ if (pipelines.size() < numThreads) {
+ pipelines.reserve(numThreads);
+ pipelines.resize(numThreads, opPipelines);
+ }
+
+ // Ensure an analysis manager has been constructed for each of the nodes.
+ // This prevents thread races when running the nested pipelines.
+ for (CallGraphNode *node : nodesToVisit)
+ inliner.am.nest(node->getCallableRegion()->getParentOp());
+
+ // An atomic failure variable for the async executors.
+ std::vector<std::atomic<bool>> activePMs(pipelines.size());
+ std::fill(activePMs.begin(), activePMs.end(), false);
+ return failableParallelForEach(ctx, nodesToVisit, [&](CallGraphNode *node) {
+ // Find a pass manager for this operation.
+ auto it = llvm::find_if(activePMs, [](std::atomic<bool> &isActive) {
+ bool expectedInactive = false;
+ return isActive.compare_exchange_strong(expectedInactive, true);
+ });
+ assert(it != activePMs.end() &&
+ "could not find inactive pass manager for thread");
+ unsigned pmIndex = it - activePMs.begin();
+
+ // Optimize this callable node.
+ LogicalResult result = optimizeCallable(node, pipelines[pmIndex]);
+
+ // Reset the active bit for this pass manager.
+ activePMs[pmIndex].store(false);
+ return result;
+ });
+}
+
+LogicalResult
+Inliner::Impl::optimizeCallable(CallGraphNode *node,
+ llvm::StringMap<OpPassManager> &pipelines) {
+ Operation *callable = node->getCallableRegion()->getParentOp();
+ StringRef opName = callable->getName().getStringRef();
+ auto pipelineIt = pipelines.find(opName);
+ const auto &preInlineCallableOptPipeline =
+ inliner.config.getPreInlineCallableOptPipeline();
+ if (pipelineIt == pipelines.end()) {
+ // If a pipeline didn't exist, use the generic pipeline if possible.
+ if (!preInlineCallableOptPipeline)
+ return success();
+
+ OpPassManager pm(opName);
+ preInlineCallableOptPipeline(pm);
+ pipelineIt = pipelines.try_emplace(opName, std::move(pm)).first;
+ }
+ return inliner.runPipelineHelper(inliner.pass, pipelineIt->second, callable);
}
/// Attempt to inline calls within the given scc. This function returns
/// success if any calls were inlined, failure otherwise.
-static LogicalResult inlineCallsInSCC(Inliner &inliner, CGUseList &useList,
- CallGraphSCC ¤tSCC) {
- CallGraph &cg = inliner.cg;
- auto &calls = inliner.calls;
+LogicalResult
+Inliner::Impl::inlineCallsInSCC(InlinerInterfaceImpl &inlinerIface,
+ CGUseList &useList, CallGraphSCC ¤tSCC) {
+ CallGraph &cg = inlinerIface.cg;
+ auto &calls = inlinerIface.calls;
// A set of dead nodes to remove after inlining.
llvm::SmallSetVector<CallGraphNode *, 1> deadNodes;
@@ -496,8 +601,9 @@ static LogicalResult inlineCallsInSCC(Inliner &inliner, CGUseList &useList,
if (useList.isDead(node)) {
deadNodes.insert(node);
} else {
- collectCallOps(*node->getCallableRegion(), node, cg, inliner.symbolTable,
- calls, /*traverseNestedCGNodes=*/false);
+ collectCallOps(*node->getCallableRegion(), node, cg,
+ inlinerIface.symbolTable, calls,
+ /*traverseNestedCGNodes=*/false);
}
}
@@ -544,9 +650,10 @@ static LogicalResult inlineCallsInSCC(Inliner &inliner, CGUseList &useList,
// then inline it in-place and delete the node if successful.
bool inlineInPlace = useList.hasOneUseAndDiscardable(it.targetNode);
- LogicalResult inlineResult = inlineCall(
- inliner, call, cast<CallableOpInterface>(targetRegion->getParentOp()),
- targetRegion, /*shouldCloneInlinedRegion=*/!inlineInPlace);
+ LogicalResult inlineResult =
+ inlineCall(inlinerIface, call,
+ cast<CallableOpInterface>(targetRegion->getParentOp()),
+ targetRegion, /*shouldCloneInlinedRegion=*/!inlineInPlace);
if (failed(inlineResult)) {
LLVM_DEBUG(llvm::dbgs() << "** Failed to inline\n");
continue;
@@ -592,262 +699,63 @@ static LogicalResult inlineCallsInSCC(Inliner &inliner, CGUseList &useList,
for (CallGraphNode *node : deadNodes) {
currentSCC.remove(node);
- inliner.markForDeletion(node);
+ inlinerIface.markForDeletion(node);
}
calls.clear();
return success(inlinedAnyCalls);
}
-//===----------------------------------------------------------------------===//
-// InlinerPass
-//===----------------------------------------------------------------------===//
-
-namespace {
-class InlinerPass : public impl::InlinerBase<InlinerPass> {
-public:
- InlinerPass();
- InlinerPass(const InlinerPass &) = default;
- InlinerPass(std::function<void(OpPassManager &)> defaultPipeline);
- InlinerPass(std::function<void(OpPassManager &)> defaultPipeline,
- llvm::StringMap<OpPassManager> opPipelines);
- void runOnOperation() override;
-
-private:
- /// Attempt to inline calls within the given scc, and run simplifications,
- /// until a fixed point is reached. This allows for the inlining of newly
- /// devirtualized calls. Returns failure if there was a fatal error during
- /// inlining.
- LogicalResult inlineSCC(Inliner &inliner, CGUseList &useList,
- CallGraphSCC ¤tSCC, MLIRContext *context);
-
- /// Optimize the nodes within the given SCC with one of the held optimization
- /// pass pipelines. Returns failure if an error occurred during the
- /// optimization of the SCC, success otherwise.
- LogicalResult optimizeSCC(CallGraph &cg, CGUseList &useList,
- CallGraphSCC ¤tSCC, MLIRContext *context);
-
- /// Optimize the nodes within the given SCC in parallel. Returns failure if an
- /// error occurred during the optimization of the SCC, success otherwise.
- LogicalResult optimizeSCCAsync(MutableArrayRef<CallGraphNode *> nodesToVisit,
- MLIRContext *context);
-
- /// Optimize the given callable node with one of the pass managers provided
- /// with `pipelines`, or the default pipeline. Returns failure if an error
- /// occurred during the optimization of the callable, success otherwise.
- LogicalResult optimizeCallable(CallGraphNode *node,
- llvm::StringMap<OpPassManager> &pipelines);
-
- /// Attempt to initialize the options of this pass from the given string.
- /// Derived classes may override this method to hook into the point at which
- /// options are initialized, but should generally always invoke this base
- /// class variant.
- LogicalResult initializeOptions(StringRef options) override;
-
- /// An optional function that constructs a default optimization pipeline for
- /// a given operation.
- std::function<void(OpPassManager &)> defaultPipeline;
- /// A map of operation names to pass pipelines to use when optimizing
- /// callable operations of these types. This provides a specialized pipeline
- /// instead of the default. The vector size is the number of threads used
- /// during optimization.
- SmallVector<llvm::StringMap<OpPassManager>, 8> opPipelines;
-};
-} // namespace
+/// Returns true if the given call should be inlined.
+bool Inliner::Impl::shouldInline(ResolvedCall &resolvedCall) {
+ // Don't allow inlining terminator calls. We currently don't support this
+ // case.
+ if (resolvedCall.call->hasTrait<OpTrait::IsTerminator>())
+ return false;
-InlinerPass::InlinerPass() : InlinerPass(defaultInlinerOptPipeline) {}
-InlinerPass::InlinerPass(
- std::function<void(OpPassManager &)> defaultPipelineArg)
- : defaultPipeline(std::move(defaultPipelineArg)) {
- opPipelines.push_back({});
-}
+ // Don't allow inlining if the target is an ancestor of the call. This
+ // prevents inlining recursively.
+ Region *callableRegion = resolvedCall.targetNode->getCallableRegion();
+ if (callableRegion->isAncestor(resolvedCall.call->getParentRegion()))
+ return false;
-InlinerPass::InlinerPass(std::function<void(OpPassManager &)> defaultPipeline,
- llvm::StringMap<OpPassManager> opPipelines)
- : InlinerPass(std::move(defaultPipeline)) {
- if (opPipelines.empty())
- return;
+ // Don't allow inlining if the callee has multiple blocks (unstructured
+ // control flow) but we cannot be sure that the caller region supports that.
+ bool calleeHasMultipleBlocks =
+ llvm::hasNItemsOrMore(*callableRegion, /*N=*/2);
+ // If both parent ops have the same type, it is safe to inline. Otherwise,
+ // decide based on whether the op has the SingleBlock trait or not.
+ // Note: This check does currently not account for SizedRegion/MaxSizedRegion.
+ auto callerRegionSupportsMultipleBlocks = [&]() {
+ return callableRegion->getParentOp()->getName() ==
+ resolvedCall.call->getParentOp()->getName() ||
+ !resolvedCall.call->getParentOp()
+ ->mightHaveTrait<OpTrait::SingleBlock>();
+ };
+ if (calleeHasMultipleBlocks && !callerRegionSupportsMultipleBlocks())
+ return false;
- // Update the option for the op specific optimization pipelines.
- for (auto &it : opPipelines)
- opPipelineList.addValue(it.second);
- this->opPipelines.emplace_back(std::move(opPipelines));
+ // Otherwise, inline.
+ return true;
}
-void InlinerPass::runOnOperation() {
- CallGraph &cg = getAnalysis<CallGraph>();
- auto *context = &getContext();
-
- // The inliner should only be run on operations that define a symbol table,
- // as the callgraph will need to resolve references.
- Operation *op = getOperation();
- if (!op->hasTrait<OpTrait::SymbolTable>()) {
- op->emitOpError() << " was scheduled to run under the inliner, but does "
- "not define a symbol table";
- return signalPassFailure();
- }
-
+LogicalResult Inliner::doInlining() {
+ Impl impl(*this);
+ auto *context = op->getContext();
// Run the inline transform in post-order over the SCCs in the callgraph.
SymbolTableCollection symbolTable;
- Inliner inliner(context, cg, symbolTable);
- CGUseList useList(getOperation(), cg, symbolTable);
+ // FIXME: some clean-up can be done for the arguments
+ // of the Impl's methods, if the inlinerIface and useList
+ // become the states of the Impl.
+ InlinerInterfaceImpl inlinerIface(context, cg, symbolTable);
+ CGUseList useList(op, cg, symbolTable);
LogicalResult result = runTransformOnCGSCCs(cg, [&](CallGraphSCC &scc) {
- return inlineSCC(inliner, useList, scc, context);
+ return impl.inlineSCC(inlinerIface, useList, scc, context);
});
if (failed(result))
- return signalPassFailure();
-
- // After inlining, make sure to erase any callables proven to be dead.
- inliner.eraseDeadCallables();
-}
-
-LogicalResult InlinerPass::inlineSCC(Inliner &inliner, CGUseList &useList,
- CallGraphSCC ¤tSCC,
- MLIRContext *context) {
- // Continuously simplify and inline until we either reach a fixed point, or
- // hit the maximum iteration count. Simplifying early helps to refine the cost
- // model, and in future iterations may devirtualize new calls.
- unsigned iterationCount = 0;
- do {
- if (failed(optimizeSCC(inliner.cg, useList, currentSCC, context)))
- return failure();
- if (failed(inlineCallsInSCC(inliner, useList, currentSCC)))
- break;
- } while (++iterationCount < maxInliningIterations);
- return success();
-}
-
-LogicalResult InlinerPass::optimizeSCC(CallGraph &cg, CGUseList &useList,
- CallGraphSCC ¤tSCC,
- MLIRContext *context) {
- // Collect the sets of nodes to simplify.
- SmallVector<CallGraphNode *, 4> nodesToVisit;
- for (auto *node : currentSCC) {
- if (node->isExternal())
- continue;
-
- // Don't simplify nodes with children. Nodes with children require special
- // handling as we may remove the node during simplification. In the future,
- // we should be able to handle this case with proper node deletion tracking.
- if (node->hasChildren())
- continue;
-
- // We also won't apply simplifications to nodes that can't have passes
- // scheduled on them.
- auto *region = node->getCallableRegion();
- if (!region->getParentOp()->hasTrait<OpTrait::IsIsolatedFromAbove>())
- continue;
- nodesToVisit.push_back(node);
- }
- if (nodesToVisit.empty())
- return success();
-
- // Optimize each of the nodes within the SCC in parallel.
- if (failed(optimizeSCCAsync(nodesToVisit, context)))
- return failure();
-
- // Recompute the uses held by each of the nodes.
- for (CallGraphNode *node : nodesToVisit)
- useList.recomputeUses(node, cg);
- return success();
-}
-
-LogicalResult
-InlinerPass::optimizeSCCAsync(MutableArrayRef<CallGraphNode *> nodesToVisit,
- MLIRContext *ctx) {
- // We must maintain a fixed pool of pass managers which is at least as large
- // as the maximum parallelism of the failableParallelForEach below.
- // Note: The number of pass managers here needs to remain constant
- // to prevent issues with pass instrumentations that rely on having the same
- // pass manager for the main thread.
- size_t numThreads = ctx->getNumThreads();
- if (opPipelines.size() < numThreads) {
- // Reserve before resizing so that we can use a reference to the first
- // element.
- opPipelines.reserve(numThreads);
- opPipelines.resize(numThreads, opPipelines.front());
- }
-
- // Ensure an analysis manager has been constructed for each of the nodes.
- // This prevents thread races when running the nested pipelines.
- for (CallGraphNode *node : nodesToVisit)
- getAnalysisManager().nest(node->getCallableRegion()->getParentOp());
-
- // An atomic failure variable for the async executors.
- std::vector<std::atomic<bool>> activePMs(opPipelines.size());
- std::fill(activePMs.begin(), activePMs.end(), false);
- return failableParallelForEach(ctx, nodesToVisit, [&](CallGraphNode *node) {
- // Find a pass manager for this operation.
- auto it = llvm::find_if(activePMs, [](std::atomic<bool> &isActive) {
- bool expectedInactive = false;
- return isActive.compare_exchange_strong(expectedInactive, true);
- });
- assert(it != activePMs.end() &&
- "could not find inactive pass manager for thread");
- unsigned pmIndex = it - activePMs.begin();
-
- // Optimize this callable node.
- LogicalResult result = optimizeCallable(node, opPipelines[pmIndex]);
-
- // Reset the active bit for this pass manager.
- activePMs[pmIndex].store(false);
return result;
- });
-}
-
-LogicalResult
-InlinerPass::optimizeCallable(CallGraphNode *node,
- llvm::StringMap<OpPassManager> &pipelines) {
- Operation *callable = node->getCallableRegion()->getParentOp();
- StringRef opName = callable->getName().getStringRef();
- auto pipelineIt = pipelines.find(opName);
- if (pipelineIt == pipelines.end()) {
- // If a pipeline didn't exist, use the default if possible.
- if (!defaultPipeline)
- return success();
-
- OpPassManager defaultPM(opName);
- defaultPipeline(defaultPM);
- pipelineIt = pipelines.try_emplace(opName, std::move(defaultPM)).first;
- }
- return runPipeline(pipelineIt->second, callable);
-}
-
-LogicalResult InlinerPass::initializeOptions(StringRef options) {
- if (failed(Pass::initializeOptions(options)))
- return failure();
-
- // Initialize the default pipeline builder to use the option string.
- // TODO: Use a generic pass manager for default pipelines, and remove this.
- if (!defaultPipelineStr.empty()) {
- std::string defaultPipelineCopy = defaultPipelineStr;
- defaultPipeline = [=](OpPassManager &pm) {
- (void)parsePassPipeline(defaultPipelineCopy, pm);
- };
- } else if (defaultPipelineStr.getNumOccurrences()) {
- defaultPipeline = nullptr;
- }
-
- // Initialize the op specific pass pipelines.
- llvm::StringMap<OpPassManager> pipelines;
- for (OpPassManager pipeline : opPipelineList)
- if (!pipeline.empty())
- pipelines.try_emplace(pipeline.getOpAnchorName(), pipeline);
- opPipelines.assign({std::move(pipelines)});
+ // After inlining, make sure to erase any callables proven to be dead.
+ inlinerIface.eraseDeadCallables();
return success();
}
-
-std::unique_ptr<Pass> mlir::createInlinerPass() {
- return std::make_unique<InlinerPass>();
-}
-std::unique_ptr<Pass>
-mlir::createInlinerPass(llvm::StringMap<OpPassManager> opPipelines) {
- return std::make_unique<InlinerPass>(defaultInlinerOptPipeline,
- std::move(opPipelines));
-}
-std::unique_ptr<Pass> mlir::createInlinerPass(
- llvm::StringMap<OpPassManager> opPipelines,
- std::function<void(OpPassManager &)> defaultPipelineBuilder) {
- return std::make_unique<InlinerPass>(std::move(defaultPipelineBuilder),
- std::move(opPipelines));
-}
+} // namespace mlir
diff --git a/mlir/test/Dialect/Affine/inlining.mlir b/mlir/test/Dialect/Affine/inlining.mlir
index b98d00a4a63783..41938037cc0218 100644
--- a/mlir/test/Dialect/Affine/inlining.mlir
+++ b/mlir/test/Dialect/Affine/inlining.mlir
@@ -1,4 +1,4 @@
-// RUN: mlir-opt -allow-unregistered-dialect %s -inline="default-pipeline=''" | FileCheck %s
+// RUN: mlir-opt -allow-unregistered-dialect %s -inline="pre-inline-pipeline=''" | FileCheck %s
// Basic test that functions within affine operations are inlined.
func.func @func_with_affine_ops(%N: index) {
diff --git a/mlir/test/Dialect/SPIRV/Transforms/inlining.mlir b/mlir/test/Dialect/SPIRV/Transforms/inlining.mlir
index 3aadb19ec15829..4cb8638fb2903d 100644
--- a/mlir/test/Dialect/SPIRV/Transforms/inlining.mlir
+++ b/mlir/test/Dialect/SPIRV/Transforms/inlining.mlir
@@ -1,4 +1,4 @@
-// RUN: mlir-opt %s -split-input-file -pass-pipeline='builtin.module(spirv.module(inline{default-pipeline=''}))' | FileCheck %s
+// RUN: mlir-opt %s -split-input-file -pass-pipeline='builtin.module(spirv.module(inline{pre-inline-pipeline=''}))' | FileCheck %s
spirv.module Logical GLSL450 {
spirv.func @callee() "None" {
diff --git a/mlir/test/Transforms/inlining-dump-default-pipeline.mlir b/mlir/test/Transforms/inlining-dump-default-pipeline.mlir
index e2c31867a8e045..def7e955bd9d1a 100644
--- a/mlir/test/Transforms/inlining-dump-default-pipeline.mlir
+++ b/mlir/test/Transforms/inlining-dump-default-pipeline.mlir
@@ -1,2 +1,2 @@
// RUN: mlir-opt %s -pass-pipeline="builtin.module(inline)" -dump-pass-pipeline 2>&1 | FileCheck %s
-// CHECK: builtin.module(inline{default-pipeline=canonicalize max-iterations=4 })
+// CHECK: builtin.module(inline{max-iterations=4 pre-inline-pipeline=canonicalize})
diff --git a/mlir/test/Transforms/inlining-recursive.mlir b/mlir/test/Transforms/inlining-recursive.mlir
index a02fe69133ad87..15b95097503439 100644
--- a/mlir/test/Transforms/inlining-recursive.mlir
+++ b/mlir/test/Transforms/inlining-recursive.mlir
@@ -1,5 +1,5 @@
-// RUN: mlir-opt %s -inline='default-pipeline=''' | FileCheck %s
-// RUN: mlir-opt %s --mlir-disable-threading -inline='default-pipeline=''' | FileCheck %s
+// RUN: mlir-opt %s -inline='pre-inline-pipeline=''' | FileCheck %s
+// RUN: mlir-opt %s --mlir-disable-threading -inline='pre-inline-pipeline=''' | FileCheck %s
// CHECK-LABEL: func.func @foo0
func.func @foo0(%arg0 : i32) -> i32 {
diff --git a/mlir/test/Transforms/inlining.mlir b/mlir/test/Transforms/inlining.mlir
index 2a08e625ba79e2..0717d07317b23c 100644
--- a/mlir/test/Transforms/inlining.mlir
+++ b/mlir/test/Transforms/inlining.mlir
@@ -1,6 +1,6 @@
-// RUN: mlir-opt %s -inline='default-pipeline=''' | FileCheck %s
-// RUN: mlir-opt %s --mlir-disable-threading -inline='default-pipeline=''' | FileCheck %s
-// RUN: mlir-opt %s -inline='default-pipeline=''' -mlir-print-debuginfo -mlir-print-local-scope | FileCheck %s --check-prefix INLINE-LOC
+// RUN: mlir-opt %s -inline='pre-inline-pipeline=''' | FileCheck %s
+// RUN: mlir-opt %s --mlir-disable-threading -inline='pre-inline-pipeline=''' | FileCheck %s
+// RUN: mlir-opt %s -inline='pre-inline-pipeline=''' -mlir-print-debuginfo -mlir-print-local-scope | FileCheck %s --check-prefix INLINE-LOC
// RUN: mlir-opt %s -inline | FileCheck %s --check-prefix INLINE_SIMPLIFY
// RUN: mlir-opt %s -inline='op-pipelines=func.func(canonicalize,cse)' | FileCheck %s --check-prefix INLINE_SIMPLIFY
>From 01de518ea81ac1dfb94e07484e9733a4c6b38378 Mon Sep 17 00:00:00 2001
From: Slava Zakharin <szakharin at nvidia.com>
Date: Tue, 5 Mar 2024 12:12:56 -0800
Subject: [PATCH 2/3] Removed incorrect comment and removed virtual from the
destructor.
---
mlir/include/mlir/Transforms/Inliner.h | 4 ----
1 file changed, 4 deletions(-)
diff --git a/mlir/include/mlir/Transforms/Inliner.h b/mlir/include/mlir/Transforms/Inliner.h
index a74a32fcfec430..c1db2ebd73682a 100644
--- a/mlir/include/mlir/Transforms/Inliner.h
+++ b/mlir/include/mlir/Transforms/Inliner.h
@@ -67,15 +67,11 @@ class InlinerConfig {
/// that operates bottom up over the Strongly Connected Components(SCCs)
/// of the CallGraph. This enables a more incremental propagation
/// of inlining decisions from the leafs to the roots of the callgraph.
-///
-/// Derived implementations may rely on the same algorithm, but override
-/// the provided hooks to tune various algorithm aspects.
class Inliner {
public:
using RunPipelineHelperTy = std::function<LogicalResult(
Pass &pass, OpPassManager &pipeline, Operation *op)>;
- virtual ~Inliner() {}
Inliner(Operation *op, CallGraph &cg, Pass &pass, AnalysisManager am,
RunPipelineHelperTy runPipelineHelper, const InlinerConfig &config)
: op(op), cg(cg), pass(pass), am(am),
>From 329d3905e12f8ef7687b035f9300ed9973aa6699 Mon Sep 17 00:00:00 2001
From: Slava Zakharin <szakharin at nvidia.com>
Date: Tue, 5 Mar 2024 13:36:59 -0800
Subject: [PATCH 3/3] Brought back the defaultPipeline.
---
mlir/include/mlir/Transforms/Inliner.h | 24 ++++++-------
mlir/include/mlir/Transforms/Passes.td | 4 +--
mlir/lib/Transforms/InlinerPass.cpp | 35 +++++++++----------
mlir/lib/Transforms/Utils/Inliner.cpp | 11 +++---
mlir/test/Dialect/Affine/inlining.mlir | 2 +-
.../Dialect/SPIRV/Transforms/inlining.mlir | 2 +-
.../inlining-dump-default-pipeline.mlir | 2 +-
mlir/test/Transforms/inlining-recursive.mlir | 4 +--
mlir/test/Transforms/inlining.mlir | 6 ++--
9 files changed, 43 insertions(+), 47 deletions(-)
diff --git a/mlir/include/mlir/Transforms/Inliner.h b/mlir/include/mlir/Transforms/Inliner.h
index c1db2ebd73682a..1fe61fb4bbe7d9 100644
--- a/mlir/include/mlir/Transforms/Inliner.h
+++ b/mlir/include/mlir/Transforms/Inliner.h
@@ -26,24 +26,22 @@ class Operation;
class InlinerConfig {
public:
- using PreInlineCallableOptPipelineTy = std::function<void(OpPassManager &)>;
+ using DefaultPipelineTy = std::function<void(OpPassManager &)>;
using OpPipelinesTy = llvm::StringMap<OpPassManager>;
InlinerConfig() = default;
- InlinerConfig(PreInlineCallableOptPipelineTy preInlineCallableOptPipeline,
+ InlinerConfig(DefaultPipelineTy defaultPipeline,
unsigned maxInliningIterations)
- : preInlineCallableOptPipeline(std::move(preInlineCallableOptPipeline)),
+ : defaultPipeline(std::move(defaultPipeline)),
maxInliningIterations(maxInliningIterations) {}
- const PreInlineCallableOptPipelineTy &
- getPreInlineCallableOptPipeline() const {
- return preInlineCallableOptPipeline;
+ const DefaultPipelineTy &getDefaultPipeline() const {
+ return defaultPipeline;
}
const OpPipelinesTy &getOpPipelines() const { return opPipelines; }
unsigned getMaxInliningIterations() const { return maxInliningIterations; }
- void
- setPreInlineCallableOptPipeline(PreInlineCallableOptPipelineTy pipeline) {
- preInlineCallableOptPipeline = std::move(pipeline);
+ void setDefaultPipeline(DefaultPipelineTy pipeline) {
+ defaultPipeline = std::move(pipeline);
}
void setOpPipelines(OpPipelinesTy pipelines) {
opPipelines = std::move(pipelines);
@@ -52,11 +50,13 @@ class InlinerConfig {
private:
/// An optional function that constructs an optimization pipeline for
- /// a given operation.
- PreInlineCallableOptPipelineTy preInlineCallableOptPipeline;
+ /// a given operation. This optimization pipeline is applied
+ /// only to those callable operations that do not have dedicated
+ /// optimization pipeline in opPipelines (based on the operation name).
+ DefaultPipelineTy defaultPipeline;
/// A map of operation names to pass pipelines to use when optimizing
/// callable operations of these types. This provides a specialized pipeline
- /// instead of the one produced by preInlineCallableOptPipeline.
+ /// instead of the one produced by defaultPipeline.
OpPipelinesTy opPipelines;
/// For SCC-based inlining algorithms, specifies maximum number of iterations
/// when inlining within an SCC.
diff --git a/mlir/include/mlir/Transforms/Passes.td b/mlir/include/mlir/Transforms/Passes.td
index 1e61026d7fc3c6..b8fdf7a580476e 100644
--- a/mlir/include/mlir/Transforms/Passes.td
+++ b/mlir/include/mlir/Transforms/Passes.td
@@ -268,8 +268,8 @@ def Inliner : Pass<"inline"> {
let summary = "Inline function calls";
let constructor = "mlir::createInlinerPass()";
let options = [
- Option<"preInlineCallableOptPipelineStr", "pre-inline-pipeline",
- "std::string", /*default=*/"\"canonicalize\"",
+ Option<"defaultPipelineStr", "default-pipeline", "std::string",
+ /*default=*/"\"canonicalize\"",
"The optimizer pipeline used for callables that do not have "
"a dedicated optimizer pipeline in opPipelineList">,
ListOption<"opPipelineList", "op-pipelines", "OpPassManager",
diff --git a/mlir/lib/Transforms/InlinerPass.cpp b/mlir/lib/Transforms/InlinerPass.cpp
index 5df598f1526b80..c058e8050cd199 100644
--- a/mlir/lib/Transforms/InlinerPass.cpp
+++ b/mlir/lib/Transforms/InlinerPass.cpp
@@ -40,9 +40,8 @@ class InlinerPass : public impl::InlinerBase<InlinerPass> {
public:
InlinerPass();
InlinerPass(const InlinerPass &) = default;
- InlinerPass(
- std::function<void(OpPassManager &)> preInlineCallableOptPipeline);
- InlinerPass(std::function<void(OpPassManager &)> preInlineCallableOptPipeline,
+ InlinerPass(std::function<void(OpPassManager &)> defaultPipeline);
+ InlinerPass(std::function<void(OpPassManager &)> defaultPipeline,
llvm::StringMap<OpPassManager> opPipelines);
void runOnOperation() override;
@@ -73,14 +72,13 @@ class InlinerPass : public impl::InlinerBase<InlinerPass> {
InlinerPass::InlinerPass() : InlinerPass(defaultInlinerOptPipeline) {}
InlinerPass::InlinerPass(
- std::function<void(OpPassManager &)> preInlineCallableOptPipelineArg)
- : InlinerPass(std::move(preInlineCallableOptPipelineArg),
+ std::function<void(OpPassManager &)> defaultPipelineArg)
+ : InlinerPass(std::move(defaultPipelineArg),
llvm::StringMap<OpPassManager>{}) {}
-InlinerPass::InlinerPass(
- std::function<void(OpPassManager &)> preInlineCallableOptPipeline,
- llvm::StringMap<OpPassManager> opPipelines)
- : config(std::move(preInlineCallableOptPipeline), maxInliningIterations) {
+InlinerPass::InlinerPass(std::function<void(OpPassManager &)> defaultPipeline,
+ llvm::StringMap<OpPassManager> opPipelines)
+ : config(std::move(defaultPipeline), maxInliningIterations) {
if (opPipelines.empty())
return;
@@ -120,14 +118,13 @@ LogicalResult InlinerPass::initializeOptions(StringRef options) {
// optimization pipeline in opPipelineList to use the option string.
// TODO: Use a generic pass manager for the pre-inline pipeline, and remove
// this.
- if (!preInlineCallableOptPipelineStr.empty()) {
- std::string preInlineCallableOptPipelineCopy =
- preInlineCallableOptPipelineStr;
- config.setPreInlineCallableOptPipeline([=](OpPassManager &pm) {
- (void)parsePassPipeline(preInlineCallableOptPipelineCopy, pm);
+ if (!defaultPipelineStr.empty()) {
+ std::string defaultPipelineCopy = defaultPipelineStr;
+ config.setDefaultPipeline([=](OpPassManager &pm) {
+ (void)parsePassPipeline(defaultPipelineCopy, pm);
});
- } else if (preInlineCallableOptPipelineStr.getNumOccurrences()) {
- config.setPreInlineCallableOptPipeline(nullptr);
+ } else if (defaultPipelineStr.getNumOccurrences()) {
+ config.setDefaultPipeline(nullptr);
}
// Initialize the op specific pass pipelines.
@@ -152,7 +149,7 @@ mlir::createInlinerPass(llvm::StringMap<OpPassManager> opPipelines) {
}
std::unique_ptr<Pass> mlir::createInlinerPass(
llvm::StringMap<OpPassManager> opPipelines,
- std::function<void(OpPassManager &)> preInlineCallableOptPipelineBuilder) {
- return std::make_unique<InlinerPass>(
- std::move(preInlineCallableOptPipelineBuilder), std::move(opPipelines));
+ std::function<void(OpPassManager &)> defaultPipelineBuilder) {
+ return std::make_unique<InlinerPass>(std::move(defaultPipelineBuilder),
+ std::move(opPipelines));
}
diff --git a/mlir/lib/Transforms/Utils/Inliner.cpp b/mlir/lib/Transforms/Utils/Inliner.cpp
index be30159c9ac976..74776a73db9aaa 100644
--- a/mlir/lib/Transforms/Utils/Inliner.cpp
+++ b/mlir/lib/Transforms/Utils/Inliner.cpp
@@ -565,16 +565,15 @@ Inliner::Impl::optimizeCallable(CallGraphNode *node,
Operation *callable = node->getCallableRegion()->getParentOp();
StringRef opName = callable->getName().getStringRef();
auto pipelineIt = pipelines.find(opName);
- const auto &preInlineCallableOptPipeline =
- inliner.config.getPreInlineCallableOptPipeline();
+ const auto &defaultPipeline = inliner.config.getDefaultPipeline();
if (pipelineIt == pipelines.end()) {
// If a pipeline didn't exist, use the generic pipeline if possible.
- if (!preInlineCallableOptPipeline)
+ if (!defaultPipeline)
return success();
- OpPassManager pm(opName);
- preInlineCallableOptPipeline(pm);
- pipelineIt = pipelines.try_emplace(opName, std::move(pm)).first;
+ OpPassManager defaultPM(opName);
+ defaultPipeline(defaultPM);
+ pipelineIt = pipelines.try_emplace(opName, std::move(defaultPM)).first;
}
return inliner.runPipelineHelper(inliner.pass, pipelineIt->second, callable);
}
diff --git a/mlir/test/Dialect/Affine/inlining.mlir b/mlir/test/Dialect/Affine/inlining.mlir
index 41938037cc0218..b98d00a4a63783 100644
--- a/mlir/test/Dialect/Affine/inlining.mlir
+++ b/mlir/test/Dialect/Affine/inlining.mlir
@@ -1,4 +1,4 @@
-// RUN: mlir-opt -allow-unregistered-dialect %s -inline="pre-inline-pipeline=''" | FileCheck %s
+// RUN: mlir-opt -allow-unregistered-dialect %s -inline="default-pipeline=''" | FileCheck %s
// Basic test that functions within affine operations are inlined.
func.func @func_with_affine_ops(%N: index) {
diff --git a/mlir/test/Dialect/SPIRV/Transforms/inlining.mlir b/mlir/test/Dialect/SPIRV/Transforms/inlining.mlir
index 4cb8638fb2903d..3aadb19ec15829 100644
--- a/mlir/test/Dialect/SPIRV/Transforms/inlining.mlir
+++ b/mlir/test/Dialect/SPIRV/Transforms/inlining.mlir
@@ -1,4 +1,4 @@
-// RUN: mlir-opt %s -split-input-file -pass-pipeline='builtin.module(spirv.module(inline{pre-inline-pipeline=''}))' | FileCheck %s
+// RUN: mlir-opt %s -split-input-file -pass-pipeline='builtin.module(spirv.module(inline{default-pipeline=''}))' | FileCheck %s
spirv.module Logical GLSL450 {
spirv.func @callee() "None" {
diff --git a/mlir/test/Transforms/inlining-dump-default-pipeline.mlir b/mlir/test/Transforms/inlining-dump-default-pipeline.mlir
index def7e955bd9d1a..e2c31867a8e045 100644
--- a/mlir/test/Transforms/inlining-dump-default-pipeline.mlir
+++ b/mlir/test/Transforms/inlining-dump-default-pipeline.mlir
@@ -1,2 +1,2 @@
// RUN: mlir-opt %s -pass-pipeline="builtin.module(inline)" -dump-pass-pipeline 2>&1 | FileCheck %s
-// CHECK: builtin.module(inline{max-iterations=4 pre-inline-pipeline=canonicalize})
+// CHECK: builtin.module(inline{default-pipeline=canonicalize max-iterations=4 })
diff --git a/mlir/test/Transforms/inlining-recursive.mlir b/mlir/test/Transforms/inlining-recursive.mlir
index 15b95097503439..a02fe69133ad87 100644
--- a/mlir/test/Transforms/inlining-recursive.mlir
+++ b/mlir/test/Transforms/inlining-recursive.mlir
@@ -1,5 +1,5 @@
-// RUN: mlir-opt %s -inline='pre-inline-pipeline=''' | FileCheck %s
-// RUN: mlir-opt %s --mlir-disable-threading -inline='pre-inline-pipeline=''' | FileCheck %s
+// RUN: mlir-opt %s -inline='default-pipeline=''' | FileCheck %s
+// RUN: mlir-opt %s --mlir-disable-threading -inline='default-pipeline=''' | FileCheck %s
// CHECK-LABEL: func.func @foo0
func.func @foo0(%arg0 : i32) -> i32 {
diff --git a/mlir/test/Transforms/inlining.mlir b/mlir/test/Transforms/inlining.mlir
index 0717d07317b23c..2a08e625ba79e2 100644
--- a/mlir/test/Transforms/inlining.mlir
+++ b/mlir/test/Transforms/inlining.mlir
@@ -1,6 +1,6 @@
-// RUN: mlir-opt %s -inline='pre-inline-pipeline=''' | FileCheck %s
-// RUN: mlir-opt %s --mlir-disable-threading -inline='pre-inline-pipeline=''' | FileCheck %s
-// RUN: mlir-opt %s -inline='pre-inline-pipeline=''' -mlir-print-debuginfo -mlir-print-local-scope | FileCheck %s --check-prefix INLINE-LOC
+// RUN: mlir-opt %s -inline='default-pipeline=''' | FileCheck %s
+// RUN: mlir-opt %s --mlir-disable-threading -inline='default-pipeline=''' | FileCheck %s
+// RUN: mlir-opt %s -inline='default-pipeline=''' -mlir-print-debuginfo -mlir-print-local-scope | FileCheck %s --check-prefix INLINE-LOC
// RUN: mlir-opt %s -inline | FileCheck %s --check-prefix INLINE_SIMPLIFY
// RUN: mlir-opt %s -inline='op-pipelines=func.func(canonicalize,cse)' | FileCheck %s --check-prefix INLINE_SIMPLIFY
More information about the Mlir-commits
mailing list