Add the initial inlining infrastructure.

This defines a set of initial utilities for inlining a region(or a FuncOp), and defines a simple inliner pass for testing purposes.
A new dialect interface is defined, DialectInlinerInterface, that allows for dialects to override hooks controlling inlining legality. The interface currently provides the following hooks, but these are just premilinary and should be changed/added to/modified as necessary:

* isLegalToInline
  - Determine if a region can be inlined into one of this dialect, *or* if an operation of this dialect can be inlined into a given region.

* shouldAnalyzeRecursively
  - Determine if an operation with regions should be analyzed recursively for legality. This allows for child operations to be closed off from the legality checks for operations like lambdas.

* handleTerminator
  - Process a terminator that has been inlined.

This cl adds support for inlining StandardOps, but other dialects will be added in followups as necessary.

PiperOrigin-RevId: 267426759
This commit is contained in:
River Riddle 2019-09-05 12:23:45 -07:00 committed by A. Unique TensorFlower
parent 7eb25cd367
commit 0ba0087887
12 changed files with 859 additions and 2 deletions

View File

@ -0,0 +1,196 @@
//===- InliningUtils.h - Inliner utilities ----------------------*- C++ -*-===//
//
// Copyright 2019 The MLIR Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
// =============================================================================
//
// This header file defines interfaces for various inlining utility methods.
//
//===----------------------------------------------------------------------===//
#ifndef MLIR_TRANSFORMS_INLINING_UTILS_H
#define MLIR_TRANSFORMS_INLINING_UTILS_H
#include "mlir/IR/DialectInterface.h"
#include "mlir/IR/Location.h"
namespace mlir {
class Block;
class BlockAndValueMapping;
class FuncOp;
class Operation;
class Region;
class Value;
//===----------------------------------------------------------------------===//
// InlinerInterface
//===----------------------------------------------------------------------===//
/// This is the interface that must be implemented by the dialects of operations
/// to be inlined. This interface should only handle the operations of the
/// given dialect.
class DialectInlinerInterface
: public DialectInterface::Base<DialectInlinerInterface> {
public:
DialectInlinerInterface(Dialect *dialect) : Base(dialect) {}
//===--------------------------------------------------------------------===//
// Analysis Hooks
//===--------------------------------------------------------------------===//
/// Returns true if the given region 'src' can be inlined into the region
/// 'dest' that is attached to an operation registered to the current dialect.
/// 'valueMapping' contains any remapped values from within the 'src' region.
/// This can be used to examine what values will replace entry arguments into
/// the 'src' region for example.
virtual bool isLegalToInline(Region *dest, Region *src,
BlockAndValueMapping &valueMapping) const {
return false;
}
/// Returns true if the given operation 'op', that is registered to this
/// dialect, can be inlined into the given region, false otherwise.
/// 'valueMapping' contains any remapped values from within the 'src' region.
/// This can be used to examine what values may potentially replace the
/// operands to 'op'.
virtual bool isLegalToInline(Operation *op, Region *dest,
BlockAndValueMapping &valueMapping) const {
return false;
}
/// This hook is invoked on an operation that contains regions. It should
/// return true if the analyzer should recurse within the regions of this
/// operation when computing legality and cost, false otherwise. The default
/// implementation returns true.
virtual bool shouldAnalyzeRecursively(Operation *op) const { return true; }
//===--------------------------------------------------------------------===//
// Transformation Hooks
//===--------------------------------------------------------------------===//
/// Handle the given inlined terminator by replacing it with a new operation
/// as necessary. This overload is called when the inlined region has more
/// than one block. The 'newDest' block represents the new final branching
/// destination of blocks within this region, i.e. operations that release
/// control to the parent operation will likely now branch to this block.
/// Its block arguments correspond to any values that need to be replaced by
/// terminators within the inlined region.
virtual void handleTerminator(Operation *op, Block *newDest) const {
llvm_unreachable("must implement handleTerminator in the case of multiple "
"inlined blocks");
}
/// Handle the given inlined terminator by replacing it with a new operation
/// as necessary. This overload is called when the inlined region only
/// contains one block. 'valuesToReplace' contains the previously returned
/// values of the call site before inlining. These values must be replaced by
/// this callback if they had any users (for example for traditional function
/// calls, these are directly replaced with the operands of the `return`
/// operation). The given 'op' will be removed by the caller, after this
/// function has been called.
virtual void handleTerminator(Operation *op,
ArrayRef<Value *> valuesToReplace) const {
llvm_unreachable(
"must implement handleTerminator in the case of one inlined block");
}
};
/// This interface provides the hooks into the inlining interface.
/// Note: this class automatically collects 'DialectInlinerInterface' objects
/// registered to each dialect within the given context.
class InlinerInterface
: public DialectInterfaceCollection<DialectInlinerInterface> {
public:
using Base::Base;
virtual ~InlinerInterface();
/// These hooks mirror the hooks for the DialectInlinerInterface, with default
/// implementations that call the hook on the handler for the dialect 'op' is
/// registered to.
//===--------------------------------------------------------------------===//
// Analysis Hooks
//===--------------------------------------------------------------------===//
virtual bool isLegalToInline(Region *dest, Region *src,
BlockAndValueMapping &valueMapping) const;
virtual bool isLegalToInline(Operation *op, Region *dest,
BlockAndValueMapping &valueMapping) const;
virtual bool shouldAnalyzeRecursively(Operation *op) const;
//===--------------------------------------------------------------------===//
// Transformation Hooks
//===--------------------------------------------------------------------===//
virtual void handleTerminator(Operation *op, Block *newDest) const;
virtual void handleTerminator(Operation *op,
ArrayRef<Value *> valuesToRepl) const;
};
//===----------------------------------------------------------------------===//
// Inline Methods.
//===----------------------------------------------------------------------===//
/// This function inlines a region, 'src', into another. This function returns
/// failure if it is not possible to inline this function. If the function
/// returned failure, then no changes to the module have been made.
///
/// The provided 'inlinePoint' must be within a region, and corresponds to the
/// location where the 'src' region should be inlined. 'mapping' contains any
/// remapped operands that are used within the region, and *must* include
/// remappings for the entry arguments to the region. 'resultsToReplace'
/// corresponds to any results that should be replaced by terminators within the
/// inlined region. 'inlineLoc' is an optional Location that, if provided, will
/// be used to update the inlined operations's location information.
/// 'shouldCloneInlinedRegion' corresponds to whether the source region should
/// be cloned into the 'inlinePoint' or spliced directly.
LogicalResult inlineRegion(InlinerInterface &interface, Region *src,
Operation *inlinePoint, BlockAndValueMapping &mapper,
ArrayRef<Value *> resultsToReplace,
llvm::Optional<Location> inlineLoc = llvm::None,
bool shouldCloneInlinedRegion = true);
/// This function is an overload of the above 'inlineRegion' that allows for
/// providing the set of operands ('inlinedOperands') that should be used
/// in-favor of the region arguments when inlining.
LogicalResult inlineRegion(InlinerInterface &interface, Region *src,
Operation *inlinePoint,
ArrayRef<Value *> inlinedOperands,
ArrayRef<Value *> resultsToReplace,
llvm::Optional<Location> inlineLoc = llvm::None,
bool shouldCloneInlinedRegion = true);
/// This function inlines a FuncOp into another. This function returns failure
/// if it is not possible to inline this FuncOp. If the function returned
/// failure, then no changes to the module have been made.
///
/// Note that this only does one level of inlining. For example, if the
/// instruction 'call B' is inlined into function 'A', and function 'B' also
/// calls 'C', then the call to 'C' now exists inside the body of 'A'. Similarly
/// this will inline a recursive FuncOp by one level.
///
/// 'callOperands' must correspond, 1-1, with the arguments to the provided
/// FuncOp. 'callResults' must correspond, 1-1, with the results of the
/// provided FuncOp. These results will be replaced by the operands of any
/// return operations that are inlined. 'inlineLoc' should refer to the location
/// that the FuncOp is being inlined into.
LogicalResult inlineFunction(InlinerInterface &interface, FuncOp callee,
Operation *inlinePoint,
ArrayRef<Value *> callOperands,
ArrayRef<Value *> callResults, Location inlineLoc);
} // end namespace mlir
#endif // MLIR_TRANSFORMS_INLINING_UTILS_H

View File

@ -29,6 +29,7 @@
#include "mlir/IR/Value.h"
#include "mlir/Support/MathExtras.h"
#include "mlir/Support/STLExtras.h"
#include "mlir/Transforms/InliningUtils.h"
#include "llvm/ADT/StringSwitch.h"
#include "llvm/Support/FormatVariadic.h"
#include "llvm/Support/raw_ostream.h"
@ -68,6 +69,54 @@ struct StdOpAsmInterface : public OpAsmDialectInterface {
}
}
};
/// This class defines the interface for handling inlining with standard
/// operations.
struct StdInlinerInterface : public DialectInlinerInterface {
using DialectInlinerInterface::DialectInlinerInterface;
//===--------------------------------------------------------------------===//
// Analysis Hooks
//===--------------------------------------------------------------------===//
/// All operations within standard ops can be inlined.
bool isLegalToInline(Operation *, Region *,
BlockAndValueMapping &) const final {
return true;
}
//===--------------------------------------------------------------------===//
// Transformation Hooks
//===--------------------------------------------------------------------===//
/// Handle the given inlined terminator by replacing it with a new operation
/// as necessary.
void handleTerminator(Operation *op, Block *newDest) const final {
// Only "std.return" needs to be handled here.
auto returnOp = dyn_cast<ReturnOp>(op);
if (!returnOp)
return;
// Replace the return with a branch to the dest.
OpBuilder builder(op);
builder.create<BranchOp>(op->getLoc(), newDest,
llvm::to_vector<4>(returnOp.getOperands()));
op->erase();
}
/// Handle the given inlined terminator by replacing it with a new operation
/// as necessary.
void handleTerminator(Operation *op,
ArrayRef<Value *> valuesToRepl) const final {
// Only "std.return" needs to be handled here.
auto returnOp = cast<ReturnOp>(op);
// Replace the values directly with the return operands.
assert(returnOp.getNumOperands() == valuesToRepl.size());
for (const auto &it : llvm::enumerate(returnOp.getOperands()))
valuesToRepl[it.index()]->replaceAllUsesWith(it.value());
}
};
} // end anonymous namespace
//===----------------------------------------------------------------------===//
@ -122,7 +171,7 @@ StandardOpsDialect::StandardOpsDialect(MLIRContext *context)
#define GET_OP_LIST
#include "mlir/Dialect/StandardOps/Ops.cpp.inc"
>();
addInterfaces<StdOpAsmInterface>();
addInterfaces<StdInlinerInterface, StdOpAsmInterface>();
}
void mlir::printDimAndSymbolList(Operation::operand_iterator begin,

View File

@ -5,6 +5,7 @@ add_llvm_library(MLIRTransforms
Canonicalizer.cpp
CSE.cpp
DialectConversion.cpp
Inliner.cpp
LoopCoalescing.cpp
LoopFusion.cpp
LoopInvariantCodeMotion.cpp

View File

@ -0,0 +1,59 @@
//===- Inliner.cpp - Pass to inline function calls ------------------------===//
//
// Copyright 2019 The MLIR Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
// =============================================================================
#include "mlir/Dialect/StandardOps/Ops.h"
#include "mlir/IR/Builders.h"
#include "mlir/IR/Function.h"
#include "mlir/IR/Module.h"
#include "mlir/Pass/Pass.h"
#include "mlir/Transforms/InliningUtils.h"
#include "mlir/Transforms/Passes.h"
#include "llvm/ADT/StringSet.h"
using namespace mlir;
// TODO(riverriddle) This pass should currently only be used for basic testing
// of inlining functionality.
namespace {
struct Inliner : public ModulePass<Inliner> {
void runOnModule() override {
auto module = getModule();
// Collect each of the direct function calls within the module.
SmallVector<CallOp, 16> callOps;
for (auto &f : module)
f.walk([&](CallOp callOp) { callOps.push_back(callOp); });
// Build the inliner interface.
InlinerInterface interface(&getContext());
// Try to inline each of the call operations.
for (auto &call : callOps) {
if (failed(inlineFunction(
interface, module.lookupSymbol<FuncOp>(call.getCallee()), call,
llvm::to_vector<8>(call.getArgOperands()),
llvm::to_vector<8>(call.getResults()), call.getLoc())))
continue;
// If the inlining was successful then erase the call.
call.erase();
}
}
};
} // end anonymous namespace
static PassRegistration<Inliner> pass("inline", "Inline function calls");

View File

@ -1,6 +1,7 @@
add_llvm_library(MLIRTransformUtils
FoldUtils.cpp
GreedyPatternRewriteDriver.cpp
InliningUtils.cpp
LoopFusionUtils.cpp
LoopUtils.cpp
RegionUtils.cpp

View File

@ -0,0 +1,287 @@
//===- InliningUtils.cpp ---- Misc utilities for inlining -----------------===//
//
// Copyright 2019 The MLIR Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
// =============================================================================
//
// This file implements miscellaneous inlining utilities.
//
//===----------------------------------------------------------------------===//
#include "mlir/Transforms/InliningUtils.h"
#include "mlir/IR/BlockAndValueMapping.h"
#include "mlir/IR/Function.h"
#include "mlir/IR/Operation.h"
#include "llvm/ADT/MapVector.h"
#include "llvm/Support/raw_ostream.h"
#define DEBUG_TYPE "inlining"
using namespace mlir;
/// Remap locations from the inlined blocks with CallSiteLoc locations with the
/// provided caller location.
static void
remapInlinedLocations(llvm::iterator_range<Region::iterator> inlinedBlocks,
Location callerLoc) {
DenseMap<Location, Location> mappedLocations;
auto remapOpLoc = [&](Operation *op) {
auto it = mappedLocations.find(op->getLoc());
if (it == mappedLocations.end()) {
auto newLoc = CallSiteLoc::get(op->getLoc(), callerLoc);
it = mappedLocations.try_emplace(op->getLoc(), newLoc).first;
}
op->setLoc(it->second);
};
for (auto &block : inlinedBlocks)
block.walk(remapOpLoc);
}
static void
remapInlinedOperands(llvm::iterator_range<Region::iterator> inlinedBlocks,
BlockAndValueMapping &mapper) {
auto remapOperands = [&](Operation *op) {
for (auto &operand : op->getOpOperands())
if (auto *mappedOp = mapper.lookupOrNull(operand.get()))
operand.set(mappedOp);
};
for (auto &block : inlinedBlocks)
block.walk(remapOperands);
}
//===----------------------------------------------------------------------===//
// InlinerInterface
//===----------------------------------------------------------------------===//
InlinerInterface::~InlinerInterface() {}
bool InlinerInterface::isLegalToInline(
Region *dest, Region *src, BlockAndValueMapping &valueMapping) const {
// Regions can always be inlined into functions.
if (isa<FuncOp>(dest->getParentOp()))
return true;
auto *handler = getInterfaceFor(dest->getParentOp());
return handler ? handler->isLegalToInline(src, dest, valueMapping) : false;
}
bool InlinerInterface::isLegalToInline(
Operation *op, Region *dest, BlockAndValueMapping &valueMapping) const {
auto *handler = getInterfaceFor(op);
return handler ? handler->isLegalToInline(op, dest, valueMapping) : false;
}
bool InlinerInterface::shouldAnalyzeRecursively(Operation *op) const {
auto *handler = getInterfaceFor(op);
return handler ? handler->shouldAnalyzeRecursively(op) : true;
}
/// Handle the given inlined terminator by replacing it with a new operation
/// as necessary.
void InlinerInterface::handleTerminator(Operation *op, Block *newDest) const {
auto *handler = getInterfaceFor(op);
assert(handler && "expected valid dialect handler");
handler->handleTerminator(op, newDest);
}
/// Handle the given inlined terminator by replacing it with a new operation
/// as necessary.
void InlinerInterface::handleTerminator(Operation *op,
ArrayRef<Value *> valuesToRepl) const {
auto *handler = getInterfaceFor(op);
assert(handler && "expected valid dialect handler");
handler->handleTerminator(op, valuesToRepl);
}
/// Utility to check that all of the operations within 'src' can be inlined.
static bool isLegalToInline(InlinerInterface &interface, Region *src,
Region *insertRegion,
BlockAndValueMapping &valueMapping) {
for (auto &block : *src) {
for (auto &op : block) {
// Check this operation.
if (!interface.isLegalToInline(&op, insertRegion, valueMapping))
return false;
// Check any nested regions.
if (interface.shouldAnalyzeRecursively(&op) &&
llvm::any_of(op.getRegions(), [&](Region &region) {
return !isLegalToInline(interface, &region, insertRegion,
valueMapping);
}))
return false;
}
}
return true;
}
//===----------------------------------------------------------------------===//
// Inline Methods
//===----------------------------------------------------------------------===//
LogicalResult mlir::inlineRegion(InlinerInterface &interface, Region *src,
Operation *inlinePoint,
BlockAndValueMapping &mapper,
ArrayRef<Value *> resultsToReplace,
llvm::Optional<Location> inlineLoc,
bool shouldCloneInlinedRegion) {
// We expect the region to have at least one block.
if (src->empty())
return failure();
// Check that all of the region arguments have been mapped.
auto *srcEntryBlock = &src->front();
if (llvm::any_of(srcEntryBlock->getArguments(),
[&](BlockArgument *arg) { return !mapper.contains(arg); }))
return failure();
// The insertion point must be within a block.
Block *insertBlock = inlinePoint->getBlock();
if (!insertBlock)
return failure();
Region *insertRegion = insertBlock->getParent();
// Check that the operations within the source region are valid to inline.
if (!interface.isLegalToInline(insertRegion, src, mapper) ||
!isLegalToInline(interface, src, insertRegion, mapper))
return failure();
// Split the insertion block.
Block *postInsertBlock =
insertBlock->splitBlock(++inlinePoint->getIterator());
// Check to see if the region is being cloned, or moved inline. In either
// case, move the new blocks after the 'insertBlock' to improve IR
// readability.
if (shouldCloneInlinedRegion)
src->cloneInto(insertRegion, postInsertBlock->getIterator(), mapper);
else
insertRegion->getBlocks().splice(postInsertBlock->getIterator(),
src->getBlocks(), src->begin(),
src->end());
// Get the range of newly inserted blocks.
auto newBlocks = llvm::make_range(std::next(insertBlock->getIterator()),
postInsertBlock->getIterator());
Block *firstNewBlock = &*newBlocks.begin();
// Remap the locations of the inlined operations if a valid source location
// was provided.
if (inlineLoc && !inlineLoc->isa<UnknownLoc>())
remapInlinedLocations(newBlocks, *inlineLoc);
// If the blocks were moved in-place, make sure to remap any necessary
// operands.
if (!shouldCloneInlinedRegion)
remapInlinedOperands(newBlocks, mapper);
// Handle the case where only a single block was inlined.
if (std::next(newBlocks.begin()) == newBlocks.end()) {
// Have the interface handle the terminator of this block.
auto *firstBlockTerminator = firstNewBlock->getTerminator();
interface.handleTerminator(firstBlockTerminator, resultsToReplace);
firstBlockTerminator->erase();
// Merge the post insert block into the cloned entry block.
firstNewBlock->getOperations().splice(firstNewBlock->end(),
postInsertBlock->getOperations());
postInsertBlock->erase();
} else {
// Otherwise, there were multiple blocks inlined. Add arguments to the post
// insertion block to represent the results to replace.
for (Value *resultToRepl : resultsToReplace) {
resultToRepl->replaceAllUsesWith(
postInsertBlock->addArgument(resultToRepl->getType()));
}
/// Handle the terminators for each of the new blocks.
for (auto &newBlock : newBlocks)
interface.handleTerminator(newBlock.getTerminator(), postInsertBlock);
}
// Splice the instructions of the inlined entry block into the insert block.
insertBlock->getOperations().splice(insertBlock->end(),
firstNewBlock->getOperations());
firstNewBlock->erase();
return success();
}
/// This function is an overload of the above 'inlineRegion' that allows for
/// providing the set of operands ('inlinedOperands') that should be used
/// in-favor of the region arguments when inlining.
LogicalResult mlir::inlineRegion(InlinerInterface &interface, Region *src,
Operation *inlinePoint,
ArrayRef<Value *> inlinedOperands,
ArrayRef<Value *> resultsToReplace,
llvm::Optional<Location> inlineLoc,
bool shouldCloneInlinedRegion) {
// We expect the region to have at least one block.
if (src->empty())
return failure();
auto *entryBlock = &src->front();
if (inlinedOperands.size() != entryBlock->getNumArguments())
return failure();
// Map the provided call operands to the arguments of the region.
BlockAndValueMapping mapper;
for (unsigned i = 0, e = inlinedOperands.size(); i != e; ++i) {
// Verify that the types of the provided values match the function argument
// types.
BlockArgument *regionArg = entryBlock->getArgument(i);
if (inlinedOperands[i]->getType() != regionArg->getType())
return failure();
mapper.map(regionArg, inlinedOperands[i]);
}
// Call into the main region inliner function.
return inlineRegion(interface, src, inlinePoint, mapper, resultsToReplace,
inlineLoc, shouldCloneInlinedRegion);
}
/// This function inlines a FuncOp into another. This function returns failure
/// if it is not possible to inline this FuncOp. If the function returned
/// failure, then no changes to the module have been made.
///
/// Note that this only does one level of inlining. For example, if the
/// instruction 'call B' is inlined, and 'B' calls 'C', then the call to 'C' now
/// exists in the instruction stream. Similarly this will inline a recursive
/// FuncOp by one level.
///
LogicalResult mlir::inlineFunction(InlinerInterface &interface, FuncOp callee,
Operation *inlinePoint,
ArrayRef<Value *> callOperands,
ArrayRef<Value *> callResults,
Location inlineLoc) {
// We don't inline if the provided callee function is a declaration.
assert(callee && "expected valid function to inline");
if (callee.isExternal())
return failure();
// Verify that the provided arguments match the function arguments.
if (callOperands.size() != callee.getNumArguments())
return failure();
// Verify that the provided values to replace match the function results.
auto funcResultTypes = callee.getType().getResults();
if (callResults.size() != funcResultTypes.size())
return failure();
for (unsigned i = 0, e = callResults.size(); i != e; ++i)
if (callResults[i]->getType() != funcResultTypes[i])
return failure();
// Call into the main region inliner function.
return inlineRegion(interface, &callee.getBody(), inlinePoint, callOperands,
callResults, inlineLoc);
}

View File

@ -0,0 +1,74 @@
// RUN: mlir-opt %s -inline | FileCheck %s
// RUN: mlir-opt %s -inline -mlir-print-debuginfo | FileCheck %s --check-prefix INLINE-LOC
// Inline a function that takes an argument.
func @func_with_arg(%c : i32) -> i32 {
%b = addi %c, %c : i32
return %b : i32
}
// CHECK-LABEL: func @inline_with_arg
func @inline_with_arg(%arg0 : i32) -> i32 {
// CHECK-NEXT: addi
// CHECK-NEXT: return
%0 = call @func_with_arg(%arg0) : (i32) -> i32
return %0 : i32
}
// Inline a function that has multiple return operations.
func @func_with_multi_return(%a : i1) -> (i32) {
cond_br %a, ^bb1, ^bb2
^bb1:
%const_0 = constant 0 : i32
return %const_0 : i32
^bb2:
%const_55 = constant 55 : i32
return %const_55 : i32
}
// CHECK-LABEL: func @inline_with_multi_return() -> i32
func @inline_with_multi_return() -> i32 {
// CHECK-NEXT: [[VAL_7:%.*]] = constant 0 : i1
// CHECK-NEXT: cond_br [[VAL_7]], ^bb1, ^bb2
// CHECK: ^bb1:
// CHECK-NEXT: [[VAL_8:%.*]] = constant 0 : i32
// CHECK-NEXT: br ^bb3([[VAL_8]] : i32)
// CHECK: ^bb2:
// CHECK-NEXT: [[VAL_9:%.*]] = constant 55 : i32
// CHECK-NEXT: br ^bb3([[VAL_9]] : i32)
// CHECK: ^bb3([[VAL_10:%.*]]: i32):
// CHECK-NEXT: return [[VAL_10]] : i32
%false = constant 0 : i1
%x = call @func_with_multi_return(%false) : (i1) -> i32
return %x : i32
}
// Check that location information is updated for inlined instructions.
func @func_with_locations(%c : i32) -> i32 {
%b = addi %c, %c : i32 loc("mysource.cc":10:8)
return %b : i32 loc("mysource.cc":11:2)
}
// INLINE-LOC-LABEL: func @inline_with_locations
func @inline_with_locations(%arg0 : i32) -> i32 {
// INLINE-LOC-NEXT: addi %{{.*}}, %{{.*}} : i32 loc(callsite("mysource.cc":10:8 at "mysource.cc":55:14))
// INLINE-LOC-NEXT: return
%0 = call @func_with_locations(%arg0) : (i32) -> i32 loc("mysource.cc":55:14)
return %0 : i32
}
// Check that external functions are not inlined.
func @func_external()
// CHECK-LABEL: func @no_inline_external
func @no_inline_external() {
// CHECK-NEXT: call @func_external()
call @func_external() : () -> ()
return
}

View File

@ -0,0 +1,67 @@
// RUN: mlir-opt %s -test-inline | FileCheck %s
// CHECK-LABEL: func @inline_with_arg
func @inline_with_arg(%arg0 : i32) -> i32 {
// CHECK-NEXT: %[[ADD:.*]] = addi %{{.*}}, %{{.*}} : i32
// CHECK-NEXT: return %[[ADD]] : i32
%fn = "test.functional_region_op"() ({
^bb0(%a : i32):
%b = addi %a, %a : i32
"test.return"(%b) : (i32) -> ()
}) : () -> ((i32) -> i32)
%0 = call_indirect %fn(%arg0) : (i32) -> i32
return %0 : i32
}
// CHECK-LABEL: func @no_inline_invalid_nested_operation
func @no_inline_invalid_nested_operation() {
// CHECK: call_indirect
// test.region is analyzed recursively, so it must not have an invalid op.
%fn = "test.functional_region_op"() ({
"test.region"() ({
"foo.noinline_operation"() : () -> ()
}) : () -> ()
"test.return"() : () -> ()
}) : () -> (() -> ())
call_indirect %fn() : () -> ()
return
}
// CHECK-LABEL: func @inline_ignore_invalid_nested_operation
func @inline_ignore_invalid_nested_operation() {
// CHECK-NOT: call_indirect
// test.functional_region_op is not analyzed recursively, so it may have an
// invalid op.
%fn = "test.functional_region_op"() ({
%internal_fn = "test.functional_region_op"() ({
"foo.noinline_operation"() : () -> ()
}) : () -> (() -> ())
"test.return"() : () -> ()
}) : () -> (() -> ())
call_indirect %fn() : () -> ()
return
}
// CHECK-LABEL: func @no_inline_invalid_dest_region
func @no_inline_invalid_dest_region() {
// CHECK: call_indirect
// foo.unknown_region is unknown, so we can't inline into it.
"foo.unknown_region"() ({
%fn = "test.functional_region_op"() ({
"test.return"() : () -> ()
}) : () -> (() -> ())
call_indirect %fn() : () -> ()
"test.return"() : () -> ()
}) : () -> ()
return
}

View File

@ -19,6 +19,7 @@
#include "mlir/IR/PatternMatch.h"
#include "mlir/IR/TypeUtilities.h"
#include "mlir/Transforms/FoldUtils.h"
#include "mlir/Transforms/InliningUtils.h"
using namespace mlir;
@ -38,6 +39,46 @@ struct TestOpFolderDialectInterface : public OpFolderDialectInterface {
return isa<OneRegionOp>(region->getParentOp());
}
};
/// This class defines the interface for handling inlining with standard
/// operations.
struct TestInlinerInterface : public DialectInlinerInterface {
using DialectInlinerInterface::DialectInlinerInterface;
//===--------------------------------------------------------------------===//
// Analysis Hooks
//===--------------------------------------------------------------------===//
bool isLegalToInline(Operation *, Region *,
BlockAndValueMapping &) const final {
return true;
}
bool shouldAnalyzeRecursively(Operation *op) const {
// Analyze recursively if this is not a functional region operation, it
// froms a separate functional scope.
return !isa<FunctionalRegionOp>(op);
}
//===--------------------------------------------------------------------===//
// Transformation Hooks
//===--------------------------------------------------------------------===//
/// Handle the given inlined terminator by replacing it with a new operation
/// as necessary.
void handleTerminator(Operation *op,
ArrayRef<Value *> valuesToRepl) const final {
// Only handle "test.return" here.
auto returnOp = dyn_cast<TestReturnOp>(op);
if (!returnOp)
return;
// Replace the values directly with the return operands.
assert(returnOp.getNumOperands() == valuesToRepl.size());
for (const auto &it : llvm::enumerate(returnOp.getOperands()))
valuesToRepl[it.index()]->replaceAllUsesWith(it.value());
}
};
} // end anonymous namespace
//===----------------------------------------------------------------------===//
@ -50,7 +91,7 @@ TestDialect::TestDialect(MLIRContext *context)
#define GET_OP_LIST
#include "TestOps.cpp.inc"
>();
addInterfaces<TestOpFolderDialectInterface>();
addInterfaces<TestOpFolderDialectInterface, TestInlinerInterface>();
allowUnknownOperations();
}

View File

@ -174,6 +174,11 @@ def SizedRegionOp : TEST_Op<"sized_region_op", []> {
let regions = (region SizedRegion<2>:$my_region, SizedRegion<1>);
}
def FunctionalRegionOp : TEST_Op<"functional_region_op", []> {
let regions = (region AnyRegion:$body);
let results = (outs FunctionType);
}
//===----------------------------------------------------------------------===//
// Test Traits
//===----------------------------------------------------------------------===//

View File

@ -1,6 +1,7 @@
add_llvm_library(MLIRTestTransforms
TestConstantFold.cpp
TestLoopFusion.cpp
TestInlining.cpp
TestLoopMapping.cpp
TestLoopParametricTiling.cpp
TestVectorizationUtils.cpp
@ -8,11 +9,14 @@ add_llvm_library(MLIRTestTransforms
ADDITIONAL_HEADER_DIRS
${MLIR_MAIN_INCLUDE_DIR}/mlir/Transforms
)
include_directories(${CMAKE_CURRENT_SOURCE_DIR}/../TestDialect)
include_directories(${CMAKE_CURRENT_BINARY_DIR}/../TestDialect)
add_dependencies(MLIRTestTransforms MLIRStandardOpsIncGen)
target_link_libraries(MLIRTestTransforms
MLIRAffineOps
MLIRAnalysis
MLIRLoopOps
MLIRPass
MLIRTestDialect
MLIRVectorOps
)

View File

@ -0,0 +1,73 @@
//===- TestInlining.cpp - Pass to inline calls in the test dialect --------===//
//
// Copyright 2019 The MLIR Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
// =============================================================================
//
// TODO(riverriddle) This pass is only necessary because the main inlining pass
// has no abstracted away the call+callee relationship. When the inlining
// interface has this support, this pass should be removed.
//
//===----------------------------------------------------------------------===//
#include "TestDialect.h"
#include "mlir/Dialect/StandardOps/Ops.h"
#include "mlir/IR/Function.h"
#include "mlir/Pass/Pass.h"
#include "mlir/Transforms/InliningUtils.h"
#include "mlir/Transforms/Passes.h"
#include "llvm/ADT/StringSet.h"
using namespace mlir;
namespace {
struct Inliner : public FunctionPass<Inliner> {
void runOnFunction() override {
auto function = getFunction();
// Collect each of the direct function calls within the module.
SmallVector<CallIndirectOp, 16> callers;
function.walk([&](CallIndirectOp caller) { callers.push_back(caller); });
// Build the inliner interface.
InlinerInterface interface(&getContext());
// Try to inline each of the call operations.
for (auto caller : callers) {
auto callee = dyn_cast_or_null<FunctionalRegionOp>(
caller.getCallee()->getDefiningOp());
if (!callee)
continue;
// Inline the functional region operation, but only clone the internal
// region if there is more than one use.
if (failed(inlineRegion(
interface, &callee.body(), caller,
llvm::to_vector<8>(caller.getArgOperands()),
llvm::to_vector<8>(caller.getResults()), caller.getLoc(),
/*shouldCloneInlinedRegion=*/!callee.getResult()->hasOneUse())))
continue;
// If the inlining was successful then erase the call and callee if
// possible.
caller.erase();
if (callee.use_empty())
callee.erase();
}
}
};
} // end anonymous namespace
static PassRegistration<Inliner> pass("test-inline",
"Test inlining region calls");