[FuncDataflow] Merge LegalizeDataflow and SplitFunction into this pass; [Transforms] Add applyDataflow API for both function and loop dataflow

This commit is contained in:
Hanchen Ye 2022-03-16 15:52:39 -05:00
parent 8059ed5080
commit 47f9f0ee4c
10 changed files with 523 additions and 611 deletions

View File

@ -31,11 +31,9 @@ std::unique_ptr<Pass> createMultipleLevelDSEPass(std::string dseTargetSpec);
/// Graph optimization passes. /// Graph optimization passes.
std::unique_ptr<Pass> createFakeQuantizePass(); std::unique_ptr<Pass> createFakeQuantizePass();
std::unique_ptr<Pass> createSimplifyTosaGraphPass(); std::unique_ptr<Pass> createSimplifyTosaGraphPass();
std::unique_ptr<Pass> createLegalizeDataflowPass(); std::unique_ptr<Pass> createFuncDataflowPass();
std::unique_ptr<Pass> std::unique_ptr<Pass> createFuncDataflowPass(unsigned dataflowGran,
createLegalizeDataflowPass(unsigned dataflowGran, bool dataflowInsertCopy = true);
bool dataflowInsertCopy = true);
std::unique_ptr<Pass> createSplitFunctionPass();
std::unique_ptr<Pass> createConvertCopyToAffineLoopsPass(); std::unique_ptr<Pass> createConvertCopyToAffineLoopsPass();
/// Runtime-related passes. /// Runtime-related passes.

View File

@ -77,15 +77,17 @@ def SimplifyTosaGraph : Pass<"scalehls-simplify-tosa-graph", "FuncOp"> {
let constructor = "mlir::scalehls::createSimplifyTosaGraphPass()"; let constructor = "mlir::scalehls::createSimplifyTosaGraphPass()";
} }
def LegalizeDataflow : Pass<"scalehls-legalize-dataflow", "FuncOp"> { def FuncDataflow : Pass<"scalehls-func-dataflow", "ModuleOp"> {
let summary = "Legalize the dataflow scheduling"; let summary = "Apply dataflow to functions";
let description = [{ let description = [{
This legalize-dataflow pass will legalize the dataflow scheduling to meet This func-dataflow pass will first legalize the dataflow scheduling to meet
the requirements of the dataflow pragma: 1) single-producer single-consumer; the requirements of the dataflow pragma: 1) single-producer single-consumer;
2) no bypass paths. 2) no bypass paths. Then, it will split operations/loops scheduled at the
same dataflow level into a separate sub-function and apply the dataflow
directive to the top function.
}]; }];
let constructor = "mlir::scalehls::createLegalizeDataflowPass()"; let constructor = "mlir::scalehls::createFuncDataflowPass()";
let options = [ let options = [
Option<"insertCopy", "insert-copy", "bool", /*default=*/"true", Option<"insertCopy", "insert-copy", "bool", /*default=*/"true",
@ -95,17 +97,6 @@ def LegalizeDataflow : Pass<"scalehls-legalize-dataflow", "FuncOp"> {
]; ];
} }
def SplitFunction : Pass<"scalehls-split-function", "ModuleOp"> {
let summary = "Split function for enabling the dataflow pragma";
let description = [{
This split-function pass will split operations/loops scheduled at the same
dataflow level into a separate sub-function for applying the dataflow pragma
to the top function.
}];
let constructor = "mlir::scalehls::createSplitFunctionPass()";
}
//===----------------------------------------------------------------------===// //===----------------------------------------------------------------------===//
// Runtime-related Passes // Runtime-related Passes
//===----------------------------------------------------------------------===// //===----------------------------------------------------------------------===//

View File

@ -96,13 +96,8 @@ bool applyLegalizeToHLSCpp(FuncOp func, bool topFunc, bool axiInterf = false);
// Graph transform utils // Graph transform utils
//===----------------------------------------------------------------------===// //===----------------------------------------------------------------------===//
/// Legalize the dataflow of "block", whose parent operation must be a function /// Apply dataflow (coarse-grained pipeline) to the block.
/// or affine loop. Return false if the legalization failed, for example, the bool applyDataflow(Block &block, unsigned minGran, bool insertCopy);
/// dataflow has cycles.
bool applyLegalizeDataflow(Block &block, int64_t minGran, bool insertCopy);
/// Split each dataflow stage of "block" into a separate sub-function.
bool applySplitFunction(Block &block);
/// Apply optimization strategy to a loop band. The ancestor function is also /// Apply optimization strategy to a loop band. The ancestor function is also
/// passed in because the post-tiling optimizations have to take function as /// passed in because the post-tiling optimizations have to take function as

View File

@ -4,9 +4,8 @@ add_mlir_library(MLIRScaleHLSTransforms
Directive/FuncPipelining.cpp Directive/FuncPipelining.cpp
Directive/LoopPipelining.cpp Directive/LoopPipelining.cpp
Graph/FakeQuantize.cpp Graph/FakeQuantize.cpp
Graph/LegalizeDataflow.cpp Graph/FuncDataflow.cpp
Graph/SimplifyTosaGraph.cpp Graph/SimplifyTosaGraph.cpp
Graph/SplitFunction.cpp
Loop/AffineLoopOrderOpt.cpp Loop/AffineLoopOrderOpt.cpp
Loop/AffineLoopPerfection.cpp Loop/AffineLoopPerfection.cpp
Loop/AffineLoopTile.cpp Loop/AffineLoopTile.cpp

View File

@ -0,0 +1,456 @@
//===----------------------------------------------------------------------===//
//
// Copyright 2020-2021 The ScaleHLS Authors.
//
//===----------------------------------------------------------------------===//
#include "mlir/Analysis/Liveness.h"
#include "mlir/Dialect/Bufferization/IR/Bufferization.h"
#include "mlir/Dialect/Linalg/IR/Linalg.h"
#include "mlir/Dialect/MemRef/IR/MemRef.h"
#include "mlir/Dialect/Tosa/IR/TosaOps.h"
#include "mlir/IR/Dominance.h"
#include "mlir/Transforms/GreedyPatternRewriteDriver.h"
#include "scalehls/Dialect/HLSCpp/HLSCpp.h"
#include "scalehls/Transforms/Passes.h"
#include "scalehls/Transforms/Utils.h"
using namespace mlir;
using namespace scalehls;
/// A dataflow use includes the intermediate value and the user operation, which
/// is similar to the concept of OpOperand in the SSA graph.
using DataflowUse = std::pair<Value, Operation *>;
using DataflowUses = SmallVector<DataflowUse, 4>;
/// A mapping from an operation to all its dataflow uses.
using DataflowUsesMap = llvm::SmallDenseMap<Operation *, DataflowUses, 64>;
namespace {
struct DataflowGraph {
DataflowGraph(Block &block);
bool hasNode(Operation *node) const { return nodes.count(node); }
DataflowUses getNodeUses(Operation *node) const {
return usesMap.lookup(node);
}
private:
// Hold all nodes in the dataflow graph.
llvm::SmallDenseSet<Operation *, 64> nodes;
// Hold the uses mapping.
DataflowUsesMap usesMap;
};
} // namespace
DataflowGraph::DataflowGraph(Block &block) {
// Results map of each operation.
DenseMap<Operation *, llvm::SmallDenseSet<Value, 2>> resultsMap;
for (auto &op : block) {
// Handle Linalg dialect operations.
if (isa<linalg::LinalgDialect>(op.getDialect())) {
auto generic = dyn_cast<linalg::GenericOp>(op);
if (!generic || !generic.hasBufferSemantics()) {
op.emitOpError("found ungeneralized or unbufferized linalg ops");
return;
}
for (auto result : generic.getOutputOperands())
resultsMap[&op].insert(result->get());
continue;
}
// Handle copy operations.
if (auto copy = dyn_cast<memref::CopyOp>(op))
resultsMap[&op].insert(copy.getTarget());
// Handle memory stores. Child regions are recursively traversed, such that
// for and if operations are considered as a node of the dataflow.
op.walk([&](Operation *child) {
// TODO: Support transfer write?
if (auto affineStore = dyn_cast<mlir::AffineWriteOpInterface>(child)) {
resultsMap[&op].insert(affineStore.getMemRef());
} else if (auto store = dyn_cast<memref::StoreOp>(child))
resultsMap[&op].insert(store.getMemRef());
});
// Handle normal SSA results.
for (auto result : op.getResults())
resultsMap[&op].insert(result);
}
// Get the dominace tree for later use.
DominanceInfo DT(block.getParentOp());
// Find successors of all operations.
for (auto &op : block) {
// TODO: Some operations are dataflow source/sink/call node, which will not
// be scheduled. Any other operations should appear here?
if (isa<memref::GetGlobalOp, memref::AllocOp, memref::AllocaOp,
bufferization::ToMemrefOp, tosa::ConstOp, arith::ConstantOp,
linalg::InitTensorOp, CallOp, ReturnOp>(op))
continue;
nodes.insert(&op);
for (auto result : resultsMap.lookup(&op)) {
for (auto user : result.getUsers()) {
// If the same block user doesn't exist, or is not properly dominated,
// or is also an updater of the result, continue.
auto sameBlockUser = block.findAncestorOpInBlock(*user);
if (!sameBlockUser || isa<ReturnOp>(sameBlockUser) ||
!DT.properlyDominates(&op, sameBlockUser))
continue;
// Only push back non-exist uses.
// TODO: Create a DenseMapInfo struct to make use SmallDenseSet.
auto &uses = usesMap[&op];
auto newUse = DataflowUse({result, sameBlockUser});
if (llvm::find(uses, newUse) == uses.end())
uses.push_back(newUse);
}
}
}
}
/// Legalize the dataflow of "block", whose parent operation must be a function
/// or affine loop. Return false if the legalization failed, for example, the
/// dataflow has cycles.
static bool applyLegalizeDataflow(Block &block, int64_t minGran,
bool insertCopy) {
auto builder = OpBuilder(block.getParentOp());
DataflowGraph graph(block);
llvm::SmallDenseMap<Operation *, int64_t, 32> map;
llvm::SmallDenseMap<int64_t, int64_t, 16> dataflowToMerge;
// Walk through all dataflow operations in a reversed order for establishing
// a ALAP scheduling.
for (auto it = block.rbegin(); it != block.rend(); ++it) {
auto op = &*it;
if (!graph.hasNode(op))
continue;
// Walk through all uses and schedule the dataflow level.
int64_t dataflowLevel = 0;
for (auto use : graph.getNodeUses(op)) {
if (!map.count(use.second))
return op->emitOpError("has unexpected use, legalize failed"), false;
dataflowLevel = std::max(dataflowLevel, map.lookup(use.second));
}
map[op] = dataflowLevel + 1;
// Eliminate bypass paths if detected.
for (auto use : graph.getNodeUses(op)) {
auto value = use.first;
auto successor = use.second;
// Continue if bypass path does not exist.
auto successorDataflowLevel = map.lookup(successor);
if (dataflowLevel == successorDataflowLevel)
continue;
// If insert-copy is set, insert CopyOp to the bypass path. Otherwise,
// record all the bypass paths in dataflowToMerge.
if (insertCopy) {
// Insert CopyOps if required.
SmallVector<Value, 4> values;
values.push_back(value);
builder.setInsertionPoint(successor);
for (auto i = dataflowLevel; i > successorDataflowLevel; --i) {
// Create and set the dataflow level of CopyOp.
Value newValue;
Operation *copyOp;
if (auto type = value.getType().dyn_cast<MemRefType>()) {
newValue = builder.create<memref::AllocOp>(op->getLoc(), type);
copyOp = builder.create<memref::CopyOp>(op->getLoc(), values.back(),
newValue);
} else {
copyOp = builder.create<hlscpp::AssignOp>(
op->getLoc(), value.getType(), values.back());
newValue = copyOp->getResult(0);
}
map[copyOp] = i;
// Chain created CopyOps.
if (i == successorDataflowLevel + 1)
value.replaceUsesWithIf(newValue, [&](OpOperand &use) {
return successor->isAncestor(use.getOwner());
});
else
values.push_back(newValue);
}
} else {
// Always retain the longest merge path.
auto dst = dataflowToMerge.lookup(successorDataflowLevel);
dataflowToMerge[successorDataflowLevel] = std::max(dst, dataflowLevel);
}
}
}
// Merge dataflow levels according to the bypasses and minimum granularity.
if (minGran != 1 || !insertCopy) {
// Collect all operations in each dataflow level.
DenseMap<int64_t, SmallVector<Operation *, 8>> dataflowOps;
for (auto &op : block.getOperations())
if (map.count(&op))
dataflowOps[map.lookup(&op)].push_back(&op);
unsigned newLevel = 1;
unsigned toMerge = minGran;
for (unsigned i = 1, e = dataflowOps.size(); i <= e; ++i) {
// If the current level is the start point of a bypass, refresh toMerge.
// Otherwise, decrease toMerge by 1.
if (auto dst = dataflowToMerge.lookup(i))
toMerge = dst - i;
else
toMerge--;
// Annotate all ops in the current level to the new level.
for (auto op : dataflowOps[i])
op->setAttr("dataflow_level",
builder.getIntegerAttr(builder.getI64Type(), newLevel));
// Update toMerge and newLevel if required.
if (toMerge == 0) {
toMerge = minGran;
++newLevel;
}
}
} else {
for (auto pair : map)
pair.first->setAttr(
"dataflow_level",
builder.getIntegerAttr(builder.getI64Type(), pair.second));
}
return true;
}
static bool createSubFunction(Block &block, ArrayRef<Operation *> ops,
StringRef name, OpBuilder &builder) {
Liveness liveness(block.getParentOp());
// A helper that checks whether a value is a liveout value.
auto isLiveOut = [&](Value value) {
return any_of(value.getUsers(), [&](auto user) {
return all_of(ops, [&](auto op) { return !op->isAncestor(user); });
});
};
// Output types and values of the sub-function.
SmallVector<Type, 8> outputTypes;
SmallVector<Value, 8> outputValues;
// Internal values of the sub-function.
llvm::SmallDenseSet<Value, 16> internalValues;
for (auto op : ops)
for (auto result : op->getResults()) {
internalValues.insert(result);
if (isLiveOut(result)) {
outputTypes.push_back(result.getType());
outputValues.push_back(result);
}
}
// Input types and values of the sub-function.
SmallVector<Type, 8> inputTypes;
SmallVector<Value, 8> inputValues;
// Local buffers of the sub-function.
llvm::SmallDenseSet<Operation *, 8> localOps;
for (auto op : ops) {
// Push back all operands and liveins as candidates.
SmallVector<Value, 8> inputCandidates(op->getOperands());
for (auto &region : op->getRegions()) {
auto entryBlock = &region.front();
auto args = entryBlock->getArguments();
for (auto liveIn : liveness.getLiveIn(entryBlock))
if (llvm::find(args, liveIn) == args.end())
inputCandidates.push_back(liveIn);
}
for (auto input : inputCandidates) {
// If the current input is a induction variable or internal value, it
// doesn't needs to be passed in as argument.
if (isForInductionVar(input) || internalValues.count(input))
continue;
if (auto defOp = input.getDefiningOp()) {
// If the current input is not a liveout and it's defined by an memref
// alloc/alloca/get_global or tensor_init op, it is a local buffer and
// can be localized later.
if (!isLiveOut(input) &&
isa<memref::AllocOp, memref::AllocaOp>(defOp)) {
localOps.insert(defOp);
continue;
}
// Since we have localized all tosa constant operations, we can safely
// insert a constant as a local op here.
if (isa<tosa::ConstOp>(defOp)) {
localOps.insert(defOp);
continue;
}
}
// Only unique inputs will be added.
if (llvm::find(inputValues, input) != inputValues.end())
continue;
inputTypes.push_back(input.getType());
inputValues.push_back(input);
}
}
// Create a new function for the current dataflow level.
auto loc = builder.getUnknownLoc();
builder.setInsertionPoint(block.getParent()->getParentOfType<FuncOp>());
auto subFunc = builder.create<FuncOp>(
loc, name, builder.getFunctionType(inputTypes, outputTypes));
// Create a function call and reconnect all inputs and outputs.
builder.setInsertionPointAfter(ops.back());
auto call = builder.create<CallOp>(loc, subFunc, inputValues);
unsigned outputIdx = 0;
for (auto result : call.getResults())
outputValues[outputIdx++].replaceAllUsesWith(result);
// Create new return operation in the new created function.
auto entry = subFunc.addEntryBlock();
builder.setInsertionPointToEnd(entry);
auto returnOp = builder.create<ReturnOp>(loc, outputValues);
// Move local buffers into the new created function.
for (auto localOp : localOps)
localOp->moveBefore(&subFunc.front().front());
// Move same level operations into the new created function.
for (auto op : ops) {
op->moveBefore(returnOp);
op->removeAttr("dataflow_level");
}
// Connect operands to the arguments of the new created function.
for (unsigned i = 0, e = inputValues.size(); i < e; ++i)
inputValues[i].replaceUsesWithIf(
entry->getArgument(i),
[&](OpOperand &use) { return subFunc->isAncestor(use.getOwner()); });
return true;
}
/// Split each dataflow stage of "block" into a separate sub-function.
static bool applySplitFunction(Block &block) {
auto builder = OpBuilder(block.getParentOp());
// Collect all constants that have more than one use.
SmallVector<tosa::ConstOp, 16> constants;
block.walk([&](tosa::ConstOp constant) {
if (!constant->hasOneUse())
constants.push_back(constant);
});
// Localize constants to each of its use.
for (auto constant : constants) {
for (auto &use : llvm::make_early_inc_range(constant->getUses())) {
auto cloneConstant = constant->clone();
builder.setInsertionPoint(use.getOwner());
builder.insert(cloneConstant);
use.set(cloneConstant->getResult(0));
}
}
// Split sub-functions.
DenseMap<int64_t, SmallVector<Operation *, 8>> dataflowOps;
for (auto &op : block)
if (auto attr = op.getAttrOfType<IntegerAttr>("dataflow_level"))
dataflowOps[attr.getInt()].push_back(&op);
for (auto pair : dataflowOps) {
auto name = "dataflow" + std::to_string(pair.first);
if (!createSubFunction(block, pair.second, name, builder))
return false;
}
return true;
}
namespace {
/// The tosa reshape to tensor reshape conversion.
struct ReshapeOpRewritePattern : public OpRewritePattern<tosa::ReshapeOp> {
using OpRewritePattern<tosa::ReshapeOp>::OpRewritePattern;
LogicalResult matchAndRewrite(tosa::ReshapeOp reshape,
PatternRewriter &rewriter) const override {
rewriter.setInsertionPoint(reshape);
auto newShapeType = RankedTensorType::get(
{(int64_t)reshape.new_shape().size()}, rewriter.getI32Type());
auto newShapeArray = llvm::to_vector<8>(
llvm::map_range(reshape.new_shape(), [&](Attribute attr) {
return APInt(32, attr.cast<IntegerAttr>().getInt());
}));
auto newShapeAttr = DenseIntElementsAttr::get(newShapeType, newShapeArray);
auto newShape =
rewriter.create<arith::ConstantOp>(reshape.getLoc(), newShapeAttr);
rewriter.replaceOpWithNewOp<tensor::ReshapeOp>(reshape, reshape.getType(),
reshape.input1(), newShape);
return success();
}
};
} // namespace
/// Apply dataflow (coarse-grained pipeline) to the block.
bool scalehls::applyDataflow(Block &block, unsigned minGran, bool insertCopy) {
if (!applyLegalizeDataflow(block, minGran, insertCopy))
return false;
if (!applySplitFunction(block))
return false;
auto parentOp = block.getParentOp();
if (isa<FuncOp>(parentOp))
setFuncDirective(parentOp, false, 1, true);
else if (isa<AffineForOp>(parentOp))
setLoopDirective(parentOp, false, 1, true, false);
else
return false;
return true;
}
namespace {
struct FuncDataflow : public FuncDataflowBase<FuncDataflow> {
FuncDataflow() = default;
FuncDataflow(unsigned dataflowGran, bool dataflowInsertCopy) {
minGran = dataflowGran;
insertCopy = dataflowInsertCopy;
}
void runOnOperation() override {
auto module = getOperation();
// Split each functions in the module.
for (auto func : llvm::make_early_inc_range(module.getOps<FuncOp>()))
applyDataflow(func.front(), minGran, insertCopy);
// Simplify copy and assign operations generated by LegalizeDataflow.
auto context = module.getContext();
mlir::RewritePatternSet patterns(context);
patterns.add<ReshapeOpRewritePattern>(context);
hlscpp::AssignOp::getCanonicalizationPatterns(patterns, context);
(void)applyPatternsAndFoldGreedily(module, std::move(patterns));
}
};
} // namespace
std::unique_ptr<Pass> scalehls::createFuncDataflowPass() {
return std::make_unique<FuncDataflow>();
}
std::unique_ptr<Pass>
scalehls::createFuncDataflowPass(unsigned dataflowGran,
bool dataflowInsertCopy) {
return std::make_unique<FuncDataflow>(dataflowGran, dataflowInsertCopy);
}

View File

@ -1,252 +0,0 @@
//===----------------------------------------------------------------------===//
//
// Copyright 2020-2021 The ScaleHLS Authors.
//
//===----------------------------------------------------------------------===//
#include "mlir/Dialect/Bufferization/IR/Bufferization.h"
#include "mlir/Dialect/Linalg/IR/Linalg.h"
#include "mlir/Dialect/MemRef/IR/MemRef.h"
#include "mlir/Dialect/StandardOps/IR/Ops.h"
#include "mlir/Dialect/Tosa/IR/TosaOps.h"
#include "mlir/IR/Dominance.h"
#include "scalehls/Transforms/Passes.h"
#include "scalehls/Transforms/Utils.h"
using namespace mlir;
using namespace scalehls;
// A dataflow use includes the intermediate value and the user operation, which
// is similar to the concept of OpOperand in the SSA graph.
using DataflowUse = std::pair<Value, Operation *>;
using DataflowUses = SmallVector<DataflowUse, 4>;
// A mapping from an operation to all its dataflow uses.
using DataflowUsesMap = llvm::SmallDenseMap<Operation *, DataflowUses, 64>;
namespace {
struct DataflowGraph {
DataflowGraph(Block &block);
bool hasNode(Operation *node) const { return nodes.count(node); }
DataflowUses getNodeUses(Operation *node) const {
return usesMap.lookup(node);
}
private:
// Hold all nodes in the dataflow graph.
llvm::SmallDenseSet<Operation *, 64> nodes;
// Hold the uses mapping.
DataflowUsesMap usesMap;
};
} // namespace
DataflowGraph::DataflowGraph(Block &block) {
// Results map of each operation.
DenseMap<Operation *, llvm::SmallDenseSet<Value, 2>> resultsMap;
for (auto &op : block) {
// Handle Linalg dialect operations.
if (isa<linalg::LinalgDialect>(op.getDialect())) {
auto generic = dyn_cast<linalg::GenericOp>(op);
if (!generic || !generic.hasBufferSemantics()) {
op.emitOpError("found ungeneralized or unbufferized linalg ops");
return;
}
for (auto result : generic.getOutputOperands())
resultsMap[&op].insert(result->get());
continue;
}
// Handle copy operations.
if (auto copy = dyn_cast<memref::CopyOp>(op))
resultsMap[&op].insert(copy.getTarget());
// Handle memory stores. Child regions are recursively traversed, such that
// for and if operations are considered as a node of the dataflow.
op.walk([&](Operation *child) {
// TODO: Support transfer write?
if (auto affineStore = dyn_cast<mlir::AffineWriteOpInterface>(child)) {
resultsMap[&op].insert(affineStore.getMemRef());
} else if (auto store = dyn_cast<memref::StoreOp>(child))
resultsMap[&op].insert(store.getMemRef());
});
// Handle normal SSA results.
for (auto result : op.getResults())
resultsMap[&op].insert(result);
}
// Get the dominace tree for later use.
DominanceInfo DT(block.getParentOp());
// Find successors of all operations.
for (auto &op : block) {
// TODO: Some operations are dataflow source/sink/call node, which will not
// be scheduled. Any other operations should appear here?
if (isa<memref::GetGlobalOp, memref::AllocOp, memref::AllocaOp,
bufferization::ToMemrefOp, tosa::ConstOp, arith::ConstantOp,
linalg::InitTensorOp, CallOp, ReturnOp>(op))
continue;
nodes.insert(&op);
for (auto result : resultsMap.lookup(&op)) {
for (auto user : result.getUsers()) {
// If the same block user doesn't exist, or is not properly dominated,
// or is also an updater of the result, continue.
auto sameBlockUser = block.findAncestorOpInBlock(*user);
if (!sameBlockUser || isa<ReturnOp>(sameBlockUser) ||
!DT.properlyDominates(&op, sameBlockUser))
continue;
// Only push back non-exist uses.
// TODO: Create a DenseMapInfo struct to make use SmallDenseSet.
auto &uses = usesMap[&op];
auto newUse = DataflowUse({result, sameBlockUser});
if (llvm::find(uses, newUse) == uses.end())
uses.push_back(newUse);
}
}
}
}
/// Legalize the dataflow of "block", whose parent operation must be a function
/// or affine loop. Return false if the legalization failed, for example, the
/// dataflow has cycles.
bool scalehls::applyLegalizeDataflow(Block &block, int64_t minGran,
bool insertCopy) {
auto builder = OpBuilder(block.getParentOp());
DataflowGraph graph(block);
llvm::SmallDenseMap<Operation *, int64_t, 32> map;
llvm::SmallDenseMap<int64_t, int64_t, 16> dataflowToMerge;
// Walk through all dataflow operations in a reversed order for establishing
// a ALAP scheduling.
for (auto it = block.rbegin(); it != block.rend(); ++it) {
auto op = &*it;
if (!graph.hasNode(op))
continue;
// Walk through all uses and schedule the dataflow level.
int64_t dataflowLevel = 0;
for (auto use : graph.getNodeUses(op)) {
if (!map.count(use.second))
return op->emitOpError("has unexpected use, legalize failed"), false;
dataflowLevel = std::max(dataflowLevel, map.lookup(use.second));
}
map[op] = dataflowLevel + 1;
// Eliminate bypass paths if detected.
for (auto use : graph.getNodeUses(op)) {
auto value = use.first;
auto successor = use.second;
// Continue if bypass path does not exist.
auto successorDataflowLevel = map.lookup(successor);
if (dataflowLevel == successorDataflowLevel)
continue;
// If insert-copy is set, insert CopyOp to the bypass path. Otherwise,
// record all the bypass paths in dataflowToMerge.
if (insertCopy) {
// Insert CopyOps if required.
SmallVector<Value, 4> values;
values.push_back(value);
builder.setInsertionPoint(successor);
for (auto i = dataflowLevel; i > successorDataflowLevel; --i) {
// Create and set the dataflow level of CopyOp.
Value newValue;
Operation *copyOp;
if (auto type = value.getType().dyn_cast<MemRefType>()) {
newValue = builder.create<memref::AllocOp>(op->getLoc(), type);
copyOp = builder.create<memref::CopyOp>(op->getLoc(), values.back(),
newValue);
} else {
copyOp = builder.create<hlscpp::AssignOp>(
op->getLoc(), value.getType(), values.back());
newValue = copyOp->getResult(0);
}
map[copyOp] = i;
// Chain created CopyOps.
if (i == successorDataflowLevel + 1)
value.replaceUsesWithIf(newValue, [&](OpOperand &use) {
return successor->isAncestor(use.getOwner());
});
else
values.push_back(newValue);
}
} else {
// Always retain the longest merge path.
auto dst = dataflowToMerge.lookup(successorDataflowLevel);
dataflowToMerge[successorDataflowLevel] = std::max(dst, dataflowLevel);
}
}
}
// Merge dataflow levels according to the bypasses and minimum granularity.
if (minGran != 1 || !insertCopy) {
// Collect all operations in each dataflow level.
DenseMap<int64_t, SmallVector<Operation *, 8>> dataflowOps;
for (auto &op : block.getOperations())
if (map.count(&op))
dataflowOps[map.lookup(&op)].push_back(&op);
unsigned newLevel = 1;
unsigned toMerge = minGran;
for (unsigned i = 1, e = dataflowOps.size(); i <= e; ++i) {
// If the current level is the start point of a bypass, refresh toMerge.
// Otherwise, decrease toMerge by 1.
if (auto dst = dataflowToMerge.lookup(i))
toMerge = dst - i;
else
toMerge--;
// Annotate all ops in the current level to the new level.
for (auto op : dataflowOps[i])
op->setAttr("dataflow_level",
builder.getIntegerAttr(builder.getI64Type(), newLevel));
// Update toMerge and newLevel if required.
if (toMerge == 0) {
toMerge = minGran;
++newLevel;
}
}
} else {
for (auto pair : map)
pair.first->setAttr(
"dataflow_level",
builder.getIntegerAttr(builder.getI64Type(), pair.second));
}
return true;
}
namespace {
struct LegalizeDataflow : public LegalizeDataflowBase<LegalizeDataflow> {
LegalizeDataflow() = default;
LegalizeDataflow(unsigned dataflowGran, bool dataflowInsertCopy) {
minGran = dataflowGran;
insertCopy = dataflowInsertCopy;
}
void runOnOperation() override {
auto func = getOperation();
applyLegalizeDataflow(func.front(), minGran, insertCopy);
setFuncDirective(func, false, 1, true);
}
};
} // namespace
std::unique_ptr<Pass> scalehls::createLegalizeDataflowPass() {
return std::make_unique<LegalizeDataflow>();
}
std::unique_ptr<Pass>
scalehls::createLegalizeDataflowPass(unsigned dataflowGran,
bool dataflowInsertCopy) {
return std::make_unique<LegalizeDataflow>(dataflowGran, dataflowInsertCopy);
}

View File

@ -1,216 +0,0 @@
//===----------------------------------------------------------------------===//
//
// Copyright 2020-2021 The ScaleHLS Authors.
//
//===----------------------------------------------------------------------===//
#include "mlir/Analysis/Liveness.h"
#include "mlir/Dialect/MemRef/IR/MemRef.h"
#include "mlir/Dialect/Tosa/IR/TosaOps.h"
#include "mlir/Transforms/GreedyPatternRewriteDriver.h"
#include "scalehls/Dialect/HLSCpp/HLSCpp.h"
#include "scalehls/Transforms/Passes.h"
#include "scalehls/Transforms/Utils.h"
using namespace mlir;
using namespace scalehls;
static bool createSubFunction(Block &block, ArrayRef<Operation *> ops,
StringRef name, OpBuilder &builder) {
Liveness liveness(block.getParentOp());
// A helper that checks whether a value is a liveout value.
auto isLiveOut = [&](Value value) {
return any_of(value.getUsers(), [&](auto user) {
return all_of(ops, [&](auto op) { return !op->isAncestor(user); });
});
};
// Output types and values of the sub-function.
SmallVector<Type, 8> outputTypes;
SmallVector<Value, 8> outputValues;
// Internal values of the sub-function.
llvm::SmallDenseSet<Value, 16> internalValues;
for (auto op : ops)
for (auto result : op->getResults()) {
internalValues.insert(result);
if (isLiveOut(result)) {
outputTypes.push_back(result.getType());
outputValues.push_back(result);
}
}
// Input types and values of the sub-function.
SmallVector<Type, 8> inputTypes;
SmallVector<Value, 8> inputValues;
// Local buffers of the sub-function.
llvm::SmallDenseSet<Operation *, 8> localOps;
for (auto op : ops) {
// Push back all operands and liveins as candidates.
SmallVector<Value, 8> inputCandidates(op->getOperands());
for (auto &region : op->getRegions()) {
auto entryBlock = &region.front();
auto args = entryBlock->getArguments();
for (auto liveIn : liveness.getLiveIn(entryBlock))
if (llvm::find(args, liveIn) == args.end())
inputCandidates.push_back(liveIn);
}
for (auto input : inputCandidates) {
// If the current input is a induction variable or internal value, it
// doesn't needs to be passed in as argument.
if (isForInductionVar(input) || internalValues.count(input))
continue;
if (auto defOp = input.getDefiningOp()) {
// If the current input is not a liveout and it's defined by an memref
// alloc/alloca/get_global or tensor_init op, it is a local buffer and
// can be localized later.
if (!isLiveOut(input) &&
isa<memref::AllocOp, memref::AllocaOp>(defOp)) {
localOps.insert(defOp);
continue;
}
// Since we have localized all tosa constant operations, we can safely
// insert a constant as a local op here.
if (isa<tosa::ConstOp>(defOp)) {
localOps.insert(defOp);
continue;
}
}
// Only unique inputs will be added.
if (llvm::find(inputValues, input) != inputValues.end())
continue;
inputTypes.push_back(input.getType());
inputValues.push_back(input);
}
}
// Create a new function for the current dataflow level.
auto loc = builder.getUnknownLoc();
builder.setInsertionPoint(block.getParent()->getParentOfType<FuncOp>());
auto subFunc = builder.create<FuncOp>(
loc, name, builder.getFunctionType(inputTypes, outputTypes));
// Create a function call and reconnect all inputs and outputs.
builder.setInsertionPointAfter(ops.back());
auto call = builder.create<CallOp>(loc, subFunc, inputValues);
unsigned outputIdx = 0;
for (auto result : call.getResults())
outputValues[outputIdx++].replaceAllUsesWith(result);
// Create new return operation in the new created function.
auto entry = subFunc.addEntryBlock();
builder.setInsertionPointToEnd(entry);
auto returnOp = builder.create<ReturnOp>(loc, outputValues);
// Move local buffers into the new created function.
for (auto localOp : localOps)
localOp->moveBefore(&subFunc.front().front());
// Move same level operations into the new created function.
for (auto op : ops) {
op->moveBefore(returnOp);
op->removeAttr("dataflow_level");
}
// Connect operands to the arguments of the new created function.
for (unsigned i = 0, e = inputValues.size(); i < e; ++i)
inputValues[i].replaceUsesWithIf(
entry->getArgument(i),
[&](OpOperand &use) { return subFunc->isAncestor(use.getOwner()); });
return true;
}
/// Split each dataflow stage of "block" into a separate sub-function.
bool scalehls::applySplitFunction(Block &block) {
auto builder = OpBuilder(block.getParentOp());
// Collect all constants that have more than one use.
SmallVector<tosa::ConstOp, 16> constants;
block.walk([&](tosa::ConstOp constant) {
if (!constant->hasOneUse())
constants.push_back(constant);
});
// Localize constants to each of its use.
for (auto constant : constants) {
for (auto &use : llvm::make_early_inc_range(constant->getUses())) {
auto cloneConstant = constant->clone();
builder.setInsertionPoint(use.getOwner());
builder.insert(cloneConstant);
use.set(cloneConstant->getResult(0));
}
}
// Split sub-functions.
DenseMap<int64_t, SmallVector<Operation *, 8>> dataflowOps;
for (auto &op : block)
if (auto attr = op.getAttrOfType<IntegerAttr>("dataflow_level"))
dataflowOps[attr.getInt()].push_back(&op);
for (auto pair : dataflowOps) {
auto name = "dataflow" + std::to_string(pair.first);
if (!createSubFunction(block, pair.second, name, builder))
return false;
}
return true;
}
namespace {
/// The tosa reshape to tensor reshape conversion.
struct ReshapeOpRewritePattern : public OpRewritePattern<tosa::ReshapeOp> {
using OpRewritePattern<tosa::ReshapeOp>::OpRewritePattern;
LogicalResult matchAndRewrite(tosa::ReshapeOp reshape,
PatternRewriter &rewriter) const override {
rewriter.setInsertionPoint(reshape);
auto newShapeType = RankedTensorType::get(
{(int64_t)reshape.new_shape().size()}, rewriter.getI32Type());
auto newShapeArray = llvm::to_vector<8>(
llvm::map_range(reshape.new_shape(), [&](Attribute attr) {
return APInt(32, attr.cast<IntegerAttr>().getInt());
}));
auto newShapeAttr = DenseIntElementsAttr::get(newShapeType, newShapeArray);
auto newShape =
rewriter.create<arith::ConstantOp>(reshape.getLoc(), newShapeAttr);
rewriter.replaceOpWithNewOp<tensor::ReshapeOp>(reshape, reshape.getType(),
reshape.input1(), newShape);
return success();
}
};
} // namespace
namespace {
struct SplitFunction : public SplitFunctionBase<SplitFunction> {
void runOnOperation() override {
auto module = getOperation();
auto context = module.getContext();
// Split each functions in the module.
for (auto func : llvm::make_early_inc_range(module.getOps<FuncOp>()))
applySplitFunction(func.front());
// Simplify copy and assign operations generated by LegalizeDataflow.
mlir::RewritePatternSet patterns(context);
// TODO: This reshape op rewriting should be factored out! It's quite weird
// to see this as a part of SplitFunction.
patterns.add<ReshapeOpRewritePattern>(context);
hlscpp::AssignOp::getCanonicalizationPatterns(patterns, context);
(void)applyPatternsAndFoldGreedily(module, std::move(patterns));
}
};
} // namespace
std::unique_ptr<Pass> scalehls::createSplitFunctionPass() {
return std::make_unique<SplitFunction>();
}

View File

@ -62,9 +62,8 @@ void scalehls::registerScaleHLSDSEPipeline() {
// If AXI interfaces are created, we need to dataflow the program to // If AXI interfaces are created, we need to dataflow the program to
// hide the latency of data load/store from/to external memories. // hide the latency of data load/store from/to external memories.
if (opts.hlsAxiInterf) { if (opts.hlsAxiInterf) {
pm.addPass(scalehls::createLegalizeDataflowPass( pm.addPass(scalehls::createFuncDataflowPass(
/*dataflowGran=*/(unsigned)1, /*dataflowInsertCopy=*/false)); /*dataflowGran=*/(unsigned)1, /*dataflowInsertCopy=*/false));
pm.addPass(scalehls::createSplitFunctionPass());
pm.addPass(scalehls::createConvertCopyToAffineLoopsPass()); pm.addPass(scalehls::createConvertCopyToAffineLoopsPass());
} }
@ -129,8 +128,7 @@ void scalehls::registerScaleHLSPyTorchPipeline() {
pm.addPass(mlir::createCanonicalizerPass()); pm.addPass(mlir::createCanonicalizerPass());
pm.addPass(scalehls::createSimplifyTosaGraphPass()); pm.addPass(scalehls::createSimplifyTosaGraphPass());
if (dataflowGran) if (dataflowGran)
pm.addPass(scalehls::createLegalizeDataflowPass(dataflowGran)); pm.addPass(scalehls::createFuncDataflowPass(dataflowGran));
pm.addPass(scalehls::createSplitFunctionPass());
pm.addPass(tosa::createTosaToLinalgNamed()); pm.addPass(tosa::createTosaToLinalgNamed());
pm.addPass(mlir::createCanonicalizerPass()); pm.addPass(mlir::createCanonicalizerPass());
pm.addPass(tosa::createTosaToLinalg()); pm.addPass(tosa::createTosaToLinalg());

View File

@ -1,8 +1,52 @@
// RUN: scalehls-opt -scalehls-legalize-dataflow="min-gran=3 insert-copy=true" %s | FileCheck %s // RUN: scalehls-opt -scalehls-func-dataflow="min-gran=3 insert-copy=true" %s | FileCheck %s
module { module {
// CHECK: func @dataflow2(%arg0: tensor<1x32x32x64xi8>) -> tensor<1x1x64xi8> {
// CHECK: %1 = "tosa.avg_pool2d"
// CHECK: %2 = "tosa.transpose"
// CHECK: %3 = tensor.reshape
// CHECK: return %3 : tensor<1x1x64xi8>
// CHECK: }
// CHECK: func @dataflow4(%arg0: tensor<1x32x32x64xi8>) -> (tensor<1x32x32x64xi8>, tensor<1x32x32x64xi8>) {
// CHECK: %2 = "tosa.clamp"
// CHECK: %3 = "tosa.conv2d"
// CHECK: %4 = "tosa.clamp"
// CHECK: %5 = "hlscpp.assign"
// CHECK: return %4, %5
// CHECK: }
// CHECK: func @dataflow1(%arg0: tensor<1x1x64xi8>) -> tensor<1x10xi8> {
// CHECK: %2 = "tosa.matmul"
// CHECK: %3 = tensor.reshape
// CHECK: %4 = "tosa.add"
// CHECK: return %4
// CHECK: }
// CHECK: func @dataflow3(%arg0: tensor<1x32x32x64xi8>, %arg1: tensor<1x32x32x64xi8>) -> tensor<1x32x32x64xi8> {
// CHECK: %2 = "tosa.conv2d"
// CHECK: %3 = "hlscpp.assign"
// CHECK: %4 = "tosa.add"
// CHECK: %5 = "tosa.clamp"
// CHECK: return %5
// CHECK: }
// CHECK: func @dataflow5(%arg0: tensor<1x3x32x32xi8>) -> tensor<1x32x32x64xi8> {
// CHECK: %3 = "tosa.transpose"
// CHECK: %4 = "tosa.conv2d"
// CHECK: return %4
// CHECK: }
// CHECK: func @forward(%arg0: tensor<1x3x32x32xi8>) -> tensor<1x10xi8> attributes {func_directive = #hlscpp.fd<pipeline=false, targetInterval=1, dataflow=true>} { // CHECK: func @forward(%arg0: tensor<1x3x32x32xi8>) -> tensor<1x10xi8> attributes {func_directive = #hlscpp.fd<pipeline=false, targetInterval=1, dataflow=true>} {
func @forward(%arg0: tensor<1x3x32x32xi8>) -> tensor<1x10xi8> { func @forward(%arg0: tensor<1x3x32x32xi8>) -> tensor<1x10xi8> {
// CHECK-NOT: %0 = "tosa.const"() {value = dense<0> : tensor<1x10xi8>} : () -> tensor<1x10xi8>
// CHECK-NOT: %1 = "tosa.const"() {value = dense<1> : tensor<1x64x10xi8>} : () -> tensor<1x64x10xi8>
// CHECK-NOT: %2 = "tosa.const"() {value = dense<2> : tensor<64x3x3x64xi8>} : () -> tensor<64x3x3x64xi8>
// CHECK-NOT: %3 = "tosa.const"() {value = dense<3> : tensor<64x3x3x64xi8>} : () -> tensor<64x3x3x64xi8>
// CHECK-NOT: %4 = "tosa.const"() {value = dense<4> : tensor<64x3x3x3xi8>} : () -> tensor<64x3x3x3xi8>
// CHECK-NOT: %5 = "tosa.const"() {value = dense<[0, 3, 1, 2]> : tensor<4xi32>} : () -> tensor<4xi32>
// CHECK-NOT: %6 = "tosa.const"() {value = dense<[0, 2, 3, 1]> : tensor<4xi32>} : () -> tensor<4xi32>
// CHECK-NOT: %7 = "tosa.const"() {value = dense<5> : tensor<64xi8>} : () -> tensor<64xi8>
%0 = "tosa.const"() {value = dense<0> : tensor<1x10xi8>} : () -> tensor<1x10xi8> %0 = "tosa.const"() {value = dense<0> : tensor<1x10xi8>} : () -> tensor<1x10xi8>
%1 = "tosa.const"() {value = dense<1> : tensor<1x64x10xi8>} : () -> tensor<1x64x10xi8> %1 = "tosa.const"() {value = dense<1> : tensor<1x64x10xi8>} : () -> tensor<1x64x10xi8>
%2 = "tosa.const"() {value = dense<2> : tensor<64x3x3x64xi8>} : () -> tensor<64x3x3x64xi8> %2 = "tosa.const"() {value = dense<2> : tensor<64x3x3x64xi8>} : () -> tensor<64x3x3x64xi8>
@ -12,43 +56,31 @@ module {
%6 = "tosa.const"() {value = dense<[0, 2, 3, 1]> : tensor<4xi32>} : () -> tensor<4xi32> %6 = "tosa.const"() {value = dense<[0, 2, 3, 1]> : tensor<4xi32>} : () -> tensor<4xi32>
%7 = "tosa.const"() {value = dense<5> : tensor<64xi8>} : () -> tensor<64xi8> %7 = "tosa.const"() {value = dense<5> : tensor<64xi8>} : () -> tensor<64xi8>
// CHECK: %8 = "tosa.transpose"(%arg0, %6) // CHECK: %0 = call @dataflow5(%arg0) : (tensor<1x3x32x32xi8>) -> tensor<1x32x32x64xi8>
// CHECK-SAME: dataflow_level = 5
%8 = "tosa.transpose"(%arg0, %6) : (tensor<1x3x32x32xi8>, tensor<4xi32>) -> tensor<1x32x32x3xi8> %8 = "tosa.transpose"(%arg0, %6) : (tensor<1x3x32x32xi8>, tensor<4xi32>) -> tensor<1x32x32x3xi8>
%9 = "tosa.conv2d"(%8, %4, %7) {dilation = [1, 1], pad = [1, 1, 1, 1], quantization_info = {input_zp = 0 : i32, weight_zp = 0 : i32}, stride = [1, 1]} : (tensor<1x32x32x3xi8>, tensor<64x3x3x3xi8>, tensor<64xi8>) -> tensor<1x32x32x64xi8> %9 = "tosa.conv2d"(%8, %4, %7) {dilation = [1, 1], pad = [1, 1, 1, 1], quantization_info = {input_zp = 0 : i32, weight_zp = 0 : i32}, stride = [1, 1]} : (tensor<1x32x32x3xi8>, tensor<64x3x3x3xi8>, tensor<64xi8>) -> tensor<1x32x32x64xi8>
// CHECK: %10 = "tosa.clamp"(%9) // CHECK: %1:2 = call @dataflow4(%0) : (tensor<1x32x32x64xi8>) -> (tensor<1x32x32x64xi8>, tensor<1x32x32x64xi8>)
// CHECK-SAME: dataflow_level = 4
%10 = "tosa.clamp"(%9) {max_fp = 3.40282347E+38 : f32, max_int = 2147483647 : i64, min_fp = 0.000000e+00 : f32, min_int = 0 : i64} : (tensor<1x32x32x64xi8>) -> tensor<1x32x32x64xi8> %10 = "tosa.clamp"(%9) {max_fp = 3.40282347E+38 : f32, max_int = 2147483647 : i64, min_fp = 0.000000e+00 : f32, min_int = 0 : i64} : (tensor<1x32x32x64xi8>) -> tensor<1x32x32x64xi8>
%11 = "tosa.conv2d"(%10, %3, %7) {dilation = [1, 1], pad = [1, 1, 1, 1], quantization_info = {input_zp = 0 : i32, weight_zp = 0 : i32}, stride = [1, 1]} : (tensor<1x32x32x64xi8>, tensor<64x3x3x64xi8>, tensor<64xi8>) -> tensor<1x32x32x64xi8> %11 = "tosa.conv2d"(%10, %3, %7) {dilation = [1, 1], pad = [1, 1, 1, 1], quantization_info = {input_zp = 0 : i32, weight_zp = 0 : i32}, stride = [1, 1]} : (tensor<1x32x32x64xi8>, tensor<64x3x3x64xi8>, tensor<64xi8>) -> tensor<1x32x32x64xi8>
%12 = "tosa.clamp"(%11) {max_fp = 3.40282347E+38 : f32, max_int = 2147483647 : i64, min_fp = 0.000000e+00 : f32, min_int = 0 : i64} : (tensor<1x32x32x64xi8>) -> tensor<1x32x32x64xi8> %12 = "tosa.clamp"(%11) {max_fp = 3.40282347E+38 : f32, max_int = 2147483647 : i64, min_fp = 0.000000e+00 : f32, min_int = 0 : i64} : (tensor<1x32x32x64xi8>) -> tensor<1x32x32x64xi8>
// CHECK: %13 = "tosa.conv2d"(%12, %2, %7)
// CHECK-SAME: dataflow_level = 3
%13 = "tosa.conv2d"(%12, %2, %7) {dilation = [1, 1], pad = [1, 1, 1, 1], quantization_info = {input_zp = 0 : i32, weight_zp = 0 : i32}, stride = [1, 1]} : (tensor<1x32x32x64xi8>, tensor<64x3x3x64xi8>, tensor<64xi8>) -> tensor<1x32x32x64xi8> %13 = "tosa.conv2d"(%12, %2, %7) {dilation = [1, 1], pad = [1, 1, 1, 1], quantization_info = {input_zp = 0 : i32, weight_zp = 0 : i32}, stride = [1, 1]} : (tensor<1x32x32x64xi8>, tensor<64x3x3x64xi8>, tensor<64xi8>) -> tensor<1x32x32x64xi8>
// CHECK: %14 = "hlscpp.assign"(%10) // CHECK: %2 = call @dataflow3(%1#0, %1#1) : (tensor<1x32x32x64xi8>, tensor<1x32x32x64xi8>) -> tensor<1x32x32x64xi8>
// CHECK-SAME: dataflow_level = 4
// CHECK: %15 = "hlscpp.assign"(%14)
// CHECK-SAME: dataflow_level = 4
// CHECK: %16 = "hlscpp.assign"(%15)
// CHECK-SAME: dataflow_level = 3
// CHECK: %17 = "tosa.add"(%13, %16)
// CHECK-SAME: dataflow_level = 3
%14 = "tosa.add"(%13, %10) : (tensor<1x32x32x64xi8>, tensor<1x32x32x64xi8>) -> tensor<1x32x32x64xi8> %14 = "tosa.add"(%13, %10) : (tensor<1x32x32x64xi8>, tensor<1x32x32x64xi8>) -> tensor<1x32x32x64xi8>
%15 = "tosa.clamp"(%14) {max_fp = 3.40282347E+38 : f32, max_int = 2147483647 : i64, min_fp = 0.000000e+00 : f32, min_int = 0 : i64} : (tensor<1x32x32x64xi8>) -> tensor<1x32x32x64xi8> %15 = "tosa.clamp"(%14) {max_fp = 3.40282347E+38 : f32, max_int = 2147483647 : i64, min_fp = 0.000000e+00 : f32, min_int = 0 : i64} : (tensor<1x32x32x64xi8>) -> tensor<1x32x32x64xi8>
// CHECK: %19 = "tosa.avg_pool2d"(%18) // CHECK: %3 = call @dataflow2(%2) : (tensor<1x32x32x64xi8>) -> tensor<1x1x64xi8>
// CHECK-SAME: dataflow_level = 2
%16 = "tosa.avg_pool2d"(%15) {kernel = [32, 32], pad = [0, 0, 0, 0], quantization_info = {input_zp = 0 : i32, output_zp = 0 : i32}, stride = [32, 32]} : (tensor<1x32x32x64xi8>) -> tensor<1x1x1x64xi8> %16 = "tosa.avg_pool2d"(%15) {kernel = [32, 32], pad = [0, 0, 0, 0], quantization_info = {input_zp = 0 : i32, output_zp = 0 : i32}, stride = [32, 32]} : (tensor<1x32x32x64xi8>) -> tensor<1x1x1x64xi8>
%17 = "tosa.transpose"(%16, %5) : (tensor<1x1x1x64xi8>, tensor<4xi32>) -> tensor<1x64x1x1xi8> %17 = "tosa.transpose"(%16, %5) : (tensor<1x1x1x64xi8>, tensor<4xi32>) -> tensor<1x64x1x1xi8>
%18 = "tosa.reshape"(%17) {new_shape = [1, 1, 64]} : (tensor<1x64x1x1xi8>) -> tensor<1x1x64xi8> %18 = "tosa.reshape"(%17) {new_shape = [1, 1, 64]} : (tensor<1x64x1x1xi8>) -> tensor<1x1x64xi8>
// CHECK: %22 = "tosa.matmul"(%21, %1) // CHECK: %4 = call @dataflow1(%3) : (tensor<1x1x64xi8>) -> tensor<1x10xi8>
// CHECK-SAME: dataflow_level = 1
%19 = "tosa.matmul"(%18, %1) {quantization_info = {a_zp = 0 : i32, b_zp = 0 : i32}} : (tensor<1x1x64xi8>, tensor<1x64x10xi8>) -> tensor<1x1x10xi8> %19 = "tosa.matmul"(%18, %1) {quantization_info = {a_zp = 0 : i32, b_zp = 0 : i32}} : (tensor<1x1x64xi8>, tensor<1x64x10xi8>) -> tensor<1x1x10xi8>
%20 = "tosa.reshape"(%19) {new_shape = [1, 10]} : (tensor<1x1x10xi8>) -> tensor<1x10xi8> %20 = "tosa.reshape"(%19) {new_shape = [1, 10]} : (tensor<1x1x10xi8>) -> tensor<1x10xi8>
%21 = "tosa.add"(%20, %0) : (tensor<1x10xi8>, tensor<1x10xi8>) -> tensor<1x10xi8> %21 = "tosa.add"(%20, %0) : (tensor<1x10xi8>, tensor<1x10xi8>) -> tensor<1x10xi8>
// CHECK: return %4 : tensor<1x10xi8>
return %21 : tensor<1x10xi8> return %21 : tensor<1x10xi8>
} }
} }

View File

@ -1,89 +0,0 @@
// RUN: scalehls-opt -scalehls-split-function %s | FileCheck %s
module {
// CHECK: func @dataflow2(%arg0: tensor<1x32x32x64xi8>) -> tensor<1x1x64xi8> {
// CHECK: %1 = "tosa.avg_pool2d"
// CHECK: %2 = "tosa.transpose"
// CHECK: %3 = tensor.reshape
// CHECK: return %3 : tensor<1x1x64xi8>
// CHECK: }
// CHECK: func @dataflow4(%arg0: tensor<1x32x32x64xi8>) -> (tensor<1x32x32x64xi8>, tensor<1x32x32x64xi8>) {
// CHECK: %2 = "tosa.clamp"
// CHECK: %3 = "tosa.conv2d"
// CHECK: %4 = "tosa.clamp"
// CHECK: %5 = "hlscpp.assign"
// CHECK: return %4, %5
// CHECK: }
// CHECK: func @dataflow1(%arg0: tensor<1x1x64xi8>) -> tensor<1x10xi8> {
// CHECK: %2 = "tosa.matmul"
// CHECK: %3 = tensor.reshape
// CHECK: %4 = "tosa.add"
// CHECK: return %4
// CHECK: }
// CHECK: func @dataflow3(%arg0: tensor<1x32x32x64xi8>, %arg1: tensor<1x32x32x64xi8>) -> tensor<1x32x32x64xi8> {
// CHECK: %2 = "tosa.conv2d"
// CHECK: %3 = "hlscpp.assign"
// CHECK: %4 = "tosa.add"
// CHECK: %5 = "tosa.clamp"
// CHECK: return %5
// CHECK: }
// CHECK: func @dataflow5(%arg0: tensor<1x3x32x32xi8>) -> tensor<1x32x32x64xi8> {
// CHECK: %3 = "tosa.transpose"
// CHECK: %4 = "tosa.conv2d"
// CHECK: return %4
// CHECK: }
// CHECK: func @forward(%arg0: tensor<1x3x32x32xi8>) -> tensor<1x10xi8> attributes {func_directive = #hlscpp.fd<pipeline=false, targetInterval=1, dataflow=true>} {
func @forward(%arg0: tensor<1x3x32x32xi8>) -> tensor<1x10xi8> attributes {func_directive = #hlscpp.fd<pipeline=false, targetInterval=1, dataflow=true>} {
// CHECK-NOT: %0 = "tosa.const"() {value = dense<0> : tensor<1x10xi8>} : () -> tensor<1x10xi8>
// CHECK-NOT: %1 = "tosa.const"() {value = dense<1> : tensor<1x64x10xi8>} : () -> tensor<1x64x10xi8>
// CHECK-NOT: %2 = "tosa.const"() {value = dense<2> : tensor<64x3x3x64xi8>} : () -> tensor<64x3x3x64xi8>
// CHECK-NOT: %3 = "tosa.const"() {value = dense<3> : tensor<64x3x3x64xi8>} : () -> tensor<64x3x3x64xi8>
// CHECK-NOT: %4 = "tosa.const"() {value = dense<4> : tensor<64x3x3x3xi8>} : () -> tensor<64x3x3x3xi8>
// CHECK-NOT: %5 = "tosa.const"() {value = dense<[0, 3, 1, 2]> : tensor<4xi32>} : () -> tensor<4xi32>
// CHECK-NOT: %6 = "tosa.const"() {value = dense<[0, 2, 3, 1]> : tensor<4xi32>} : () -> tensor<4xi32>
// CHECK-NOT: %7 = "tosa.const"() {value = dense<5> : tensor<64xi8>} : () -> tensor<64xi8>
%0 = "tosa.const"() {value = dense<0> : tensor<1x10xi8>} : () -> tensor<1x10xi8>
%1 = "tosa.const"() {value = dense<1> : tensor<1x64x10xi8>} : () -> tensor<1x64x10xi8>
%2 = "tosa.const"() {value = dense<2> : tensor<64x3x3x64xi8>} : () -> tensor<64x3x3x64xi8>
%3 = "tosa.const"() {value = dense<3> : tensor<64x3x3x64xi8>} : () -> tensor<64x3x3x64xi8>
%4 = "tosa.const"() {value = dense<4> : tensor<64x3x3x3xi8>} : () -> tensor<64x3x3x3xi8>
%5 = "tosa.const"() {value = dense<[0, 3, 1, 2]> : tensor<4xi32>} : () -> tensor<4xi32>
%6 = "tosa.const"() {value = dense<[0, 2, 3, 1]> : tensor<4xi32>} : () -> tensor<4xi32>
%7 = "tosa.const"() {value = dense<5> : tensor<64xi8>} : () -> tensor<64xi8>
// CHECK: %0 = call @dataflow5(%arg0) : (tensor<1x3x32x32xi8>) -> tensor<1x32x32x64xi8>
%8 = "tosa.transpose"(%arg0, %6) {dataflow_level = 5 : i64} : (tensor<1x3x32x32xi8>, tensor<4xi32>) -> tensor<1x32x32x3xi8>
%9 = "tosa.conv2d"(%8, %4, %7) {dataflow_level = 5 : i64, dilation = [1, 1], pad = [1, 1, 1, 1], quantization_info = {input_zp = 0 : i32, weight_zp = 0 : i32}, stride = [1, 1]} : (tensor<1x32x32x3xi8>, tensor<64x3x3x3xi8>, tensor<64xi8>) -> tensor<1x32x32x64xi8>
// CHECK: %1:2 = call @dataflow4(%0) : (tensor<1x32x32x64xi8>) -> (tensor<1x32x32x64xi8>, tensor<1x32x32x64xi8>)
%10 = "tosa.clamp"(%9) {dataflow_level = 4 : i64, max_fp = 3.40282347E+38 : f32, max_int = 2147483647 : i64, min_fp = 0.000000e+00 : f32, min_int = 0 : i64} : (tensor<1x32x32x64xi8>) -> tensor<1x32x32x64xi8>
%11 = "tosa.conv2d"(%10, %3, %7) {dataflow_level = 4 : i64, dilation = [1, 1], pad = [1, 1, 1, 1], quantization_info = {input_zp = 0 : i32, weight_zp = 0 : i32}, stride = [1, 1]} : (tensor<1x32x32x64xi8>, tensor<64x3x3x64xi8>, tensor<64xi8>) -> tensor<1x32x32x64xi8>
%12 = "tosa.clamp"(%11) {dataflow_level = 4 : i64, max_fp = 3.40282347E+38 : f32, max_int = 2147483647 : i64, min_fp = 0.000000e+00 : f32, min_int = 0 : i64} : (tensor<1x32x32x64xi8>) -> tensor<1x32x32x64xi8>
%13 = "tosa.conv2d"(%12, %2, %7) {dataflow_level = 3 : i64, dilation = [1, 1], pad = [1, 1, 1, 1], quantization_info = {input_zp = 0 : i32, weight_zp = 0 : i32}, stride = [1, 1]} : (tensor<1x32x32x64xi8>, tensor<64x3x3x64xi8>, tensor<64xi8>) -> tensor<1x32x32x64xi8>
%14 = "hlscpp.assign"(%10) {dataflow_level = 4 : i64} : (tensor<1x32x32x64xi8>) -> tensor<1x32x32x64xi8>
%15 = "hlscpp.assign"(%14) {dataflow_level = 4 : i64} : (tensor<1x32x32x64xi8>) -> tensor<1x32x32x64xi8>
// CHECK: %2 = call @dataflow3(%1#0, %1#1) : (tensor<1x32x32x64xi8>, tensor<1x32x32x64xi8>) -> tensor<1x32x32x64xi8>
%16 = "hlscpp.assign"(%15) {dataflow_level = 3 : i64} : (tensor<1x32x32x64xi8>) -> tensor<1x32x32x64xi8>
%17 = "tosa.add"(%13, %16) {dataflow_level = 3 : i64} : (tensor<1x32x32x64xi8>, tensor<1x32x32x64xi8>) -> tensor<1x32x32x64xi8>
%18 = "tosa.clamp"(%17) {dataflow_level = 3 : i64, max_fp = 3.40282347E+38 : f32, max_int = 2147483647 : i64, min_fp = 0.000000e+00 : f32, min_int = 0 : i64} : (tensor<1x32x32x64xi8>) -> tensor<1x32x32x64xi8>
// CHECK: %3 = call @dataflow2(%2) : (tensor<1x32x32x64xi8>) -> tensor<1x1x64xi8>
%19 = "tosa.avg_pool2d"(%18) {dataflow_level = 2 : i64, kernel = [32, 32], pad = [0, 0, 0, 0], quantization_info = {input_zp = 0 : i32, output_zp = 0 : i32}, stride = [32, 32]} : (tensor<1x32x32x64xi8>) -> tensor<1x1x1x64xi8>
%20 = "tosa.transpose"(%19, %5) {dataflow_level = 2 : i64} : (tensor<1x1x1x64xi8>, tensor<4xi32>) -> tensor<1x64x1x1xi8>
%21 = "tosa.reshape"(%20) {dataflow_level = 2 : i64, new_shape = [1, 1, 64]} : (tensor<1x64x1x1xi8>) -> tensor<1x1x64xi8>
// CHECK: %4 = call @dataflow1(%3) : (tensor<1x1x64xi8>) -> tensor<1x10xi8>
%22 = "tosa.matmul"(%21, %1) {dataflow_level = 1 : i64, quantization_info = {a_zp = 0 : i32, b_zp = 0 : i32}} : (tensor<1x1x64xi8>, tensor<1x64x10xi8>) -> tensor<1x1x10xi8>
%23 = "tosa.reshape"(%22) {dataflow_level = 1 : i64, new_shape = [1, 10]} : (tensor<1x1x10xi8>) -> tensor<1x10xi8>
%24 = "tosa.add"(%23, %0) {dataflow_level = 1 : i64} : (tensor<1x10xi8>, tensor<1x10xi8>) -> tensor<1x10xi8>
// CHECK: return %4 : tensor<1x10xi8>
return %24 : tensor<1x10xi8>
}
}