From 669664054dccf604d8758da437bf546eed258feb Mon Sep 17 00:00:00 2001 From: Hanchen Ye Date: Sat, 23 Mar 2024 23:14:17 -0500 Subject: [PATCH] Support to bufferize task op with new destination-style semantics --- include/scalehls/Dialect/HLS/IR/HLSOps.td | 13 +- lib/Dialect/HLS/IR/HLSOps.cpp | 10 +- .../BufferizableOpInterfaceImpl.cpp | 509 ++++++++++++++---- .../HLS/Transforms/ConvertDataflowToFunc.cpp | 2 +- .../HLS/Transforms/ScheduleDataflow.cpp | 11 +- 5 files changed, 418 insertions(+), 127 deletions(-) diff --git a/include/scalehls/Dialect/HLS/IR/HLSOps.td b/include/scalehls/Dialect/HLS/IR/HLSOps.td index 1a16e069..aa46f021 100644 --- a/include/scalehls/Dialect/HLS/IR/HLSOps.td +++ b/include/scalehls/Dialect/HLS/IR/HLSOps.td @@ -379,15 +379,22 @@ def TaskOp : HLSOp<"task", [SingleBlockImplicitTerminator<"YieldOp">]> { let hasCanonicalizer = 1; let extraClassDeclaration = [{ - /// Return true if this task op contains nested sub-tasks. - bool hasHierarchy() { - return walk([&](Operation *op) { + /// Return whether this task op is a leaf task. + bool isLeafTask() { + return !walk([&](Operation *op) { if (isa(op) && op != *this) return WalkResult::interrupt(); return WalkResult::advance(); }).wasInterrupted(); } + /// Return whether this task is the single task in the parent block. + bool isSingleTask() { + return llvm::hasSingleElement( + llvm::make_filter_range((*this)->getBlock()->getOperations(), + [](Operation &op) { return isa(op); })); + } + /// Return the yield op of this task op. YieldOp getYieldOp(); diff --git a/lib/Dialect/HLS/IR/HLSOps.cpp b/lib/Dialect/HLS/IR/HLSOps.cpp index 40de3060..95a82b71 100644 --- a/lib/Dialect/HLS/IR/HLSOps.cpp +++ b/lib/Dialect/HLS/IR/HLSOps.cpp @@ -435,7 +435,8 @@ struct FoldTaskIterArgs : public OpRewritePattern { task.getLoc(), TypeRange(newIterArgs), newIterArgs); newtask->setAttrs(task->getAttrs()); Block *newBlock = rewriter.createBlock( - &newtask.getBody(), newtask.getBody().begin(), TypeRange(newIterArgs)); + &newtask.getBody(), newtask.getBody().begin(), TypeRange(newIterArgs), + llvm::map_to_vector(newIterArgs, [&](Value v) { return v.getLoc(); })); rewriter.setInsertionPointToEnd(newBlock); rewriter.create(task.getLoc(), newYieldValues); @@ -518,11 +519,8 @@ struct InlineTask : public OpRewritePattern { void TaskOp::getCanonicalizationPatterns(RewritePatternSet &results, MLIRContext *context) { results.add(context); - results.add(context, [](TaskOp task) { - return llvm::hasSingleElement( - llvm::make_filter_range(task->getBlock()->getOperations(), - [](Operation &op) { return isa(op); })); - }); + results.add(context, + [](TaskOp task) { return task.isSingleTask(); }); } LogicalResult TaskOp::verify() { diff --git a/lib/Dialect/HLS/Transforms/BufferizableOpInterfaceImpl.cpp b/lib/Dialect/HLS/Transforms/BufferizableOpInterfaceImpl.cpp index a9444850..d71cb2e9 100644 --- a/lib/Dialect/HLS/Transforms/BufferizableOpInterfaceImpl.cpp +++ b/lib/Dialect/HLS/Transforms/BufferizableOpInterfaceImpl.cpp @@ -7,6 +7,7 @@ #include "scalehls/Dialect/HLS/Transforms/BufferizableOpInterfaceImpl.h" #include "mlir/Dialect/Bufferization/IR/BufferizableOpInterface.h" #include "mlir/Dialect/Bufferization/IR/Bufferization.h" +#include "mlir/Dialect/Bufferization/Transforms/OneShotAnalysis.h" #include "mlir/Dialect/MemRef/IR/MemRef.h" #include "mlir/Interfaces/ViewLikeInterface.h" #include "scalehls/Dialect/HLS/IR/HLS.h" @@ -16,87 +17,390 @@ using namespace bufferization; using namespace scalehls; using namespace hls; -/// Bufferization of schedule/task operation. Replace with a new schedule/task -/// that yields memrefs. +/// Helper function for task bufferization. Return the indices of all values +/// that have a tensor type. +static DenseSet getTensorIndices(ValueRange values) { + DenseSet result; + for (const auto &it : llvm::enumerate(values)) + if (isa(it.value().getType())) + result.insert(it.index()); + return result; +} + +/// Helper function for task bufferization. Return "true" if the given value +/// is guaranteed to not alias with an external tensor apart from values in +/// `exceptions`. A value is external if it is defined outside of the given +/// region or if it is an entry block argument of the region. +static bool doesNotAliasExternalValue(Value value, Region *region, + ValueRange exceptions, + const OneShotAnalysisState &state) { + assert(region->getBlocks().size() == 1 && + "expected region with single block"); + bool result = true; + state.applyOnAliases(value, [&](Value alias) { + if (llvm::is_contained(exceptions, alias)) + return; + Region *aliasRegion = alias.getParentRegion(); + if (isa(alias) && !region->isProperAncestor(aliasRegion)) + result = false; + if (isa(alias) && !region->isAncestor(aliasRegion)) + result = false; + }); + return result; +} + +/// Compute the bufferized type of a task init_arg. This type must be equal to +/// the bufferized type of the corresponding init_arg and the bufferized type of +/// the corresponding yielded value. +/// +/// This function uses bufferization::getBufferType to compute the bufferized +/// type of the init_arg and of the yielded value. (The computation of the +/// bufferized yielded value type usually requires computing the bufferized type +/// of the `iter_arg` again; the implementation of getBufferType traces back the +/// use-def chain of the given value and computes a buffer type along the way.) +/// If both buffer types are equal, no casts are needed the computed buffer type +/// can be used directly. Otherwise, the buffer types can only differ in their +/// layout map and a cast must be inserted. +static FailureOr computeTaskRegionIterArgBufferType( + Operation *task, BlockArgument iterArg, Value initArg, Value yieldedValue, + const BufferizationOptions &options, SmallVector &invocationStack) { + // Determine the buffer type of the init_arg. + auto initArgBufferType = + bufferization::getBufferType(initArg, options, invocationStack); + if (failed(initArgBufferType)) + return failure(); + + if (llvm::count(invocationStack, iterArg) >= 2) { + // If the iter_arg is already twice on the invocation stack, just take the + // type of the init_arg. This is to avoid infinite tasks when calculating + // the buffer type. This will most likely result in computing a memref type + // with a fully dynamic layout map. + + // Note: For more precise layout map computation, a fixpoint iteration could + // be done (i.e., re-computing the yielded buffer type until the bufferized + // iter_arg type no longer changes). This current implementation immediately + // switches to a fully dynamic layout map when a mismatch between bufferized + // init_arg type and bufferized yield value type is detected. + return *initArgBufferType; + } + + // Compute the buffer type of the yielded value. + BaseMemRefType yieldedValueBufferType; + if (isa(yieldedValue.getType())) { + // Task yield was already bufferized. + yieldedValueBufferType = cast(yieldedValue.getType()); + } else { + // Note: This typically triggers a recursive call for the buffer type of + // the iter_arg. + auto maybeBufferType = + bufferization::getBufferType(yieldedValue, options, invocationStack); + if (failed(maybeBufferType)) + return failure(); + yieldedValueBufferType = *maybeBufferType; + } + + // If yielded type and init_arg type are the same, use that type directly. + if (*initArgBufferType == yieldedValueBufferType) + return yieldedValueBufferType; + + // If there is a mismatch between the yielded buffer type and the init_arg + // buffer type, the buffer type must be promoted to a fully dynamic layout + // map. + auto yieldedBufferType = cast(yieldedValueBufferType); + auto iterTensorType = cast(iterArg.getType()); + auto initBufferType = llvm::cast(*initArgBufferType); + if (initBufferType.getMemorySpace() != yieldedBufferType.getMemorySpace()) + return task->emitOpError( + "init_arg and yielded value bufferize to inconsistent memory spaces"); + + if (auto yieldedRankedBufferType = dyn_cast(yieldedBufferType)) { + assert( + llvm::all_equal({yieldedRankedBufferType.getShape(), + cast(initBufferType).getShape(), + cast(iterTensorType).getShape()}) && + "expected same shape"); + } + + return getMemRefTypeWithFullyDynamicLayout( + iterTensorType, yieldedBufferType.getMemorySpace()); +} + +/// Helper function for task bufferization. Return the bufferized values of the +/// given OpOperands. If an operand is not a tensor, return the original value. +static FailureOr> +getBuffers(RewriterBase &rewriter, MutableOperandRange operands, + const BufferizationOptions &options) { + SmallVector result; + for (OpOperand &opOperand : operands) { + if (isa(opOperand.get().getType())) { + FailureOr resultBuffer = + getBuffer(rewriter, opOperand.get(), options); + if (failed(resultBuffer)) + return failure(); + result.push_back(*resultBuffer); + } else { + result.push_back(opOperand.get()); + } + } + return result; +} + +/// Helper function for task bufferization. Cast the given buffer to the given +/// memref type. +static Value castBuffer(OpBuilder &b, Value buffer, Type type) { + assert(isa(type) && "expected BaseMemRefType"); + assert(isa(buffer.getType()) && "expected BaseMemRefType"); + // If the buffer already has the correct type, no cast is needed. + if (buffer.getType() == type) + return buffer; + // TODO: In case `type` has a layout map that is not the fully dynamic + // one, we may not be able to cast the buffer. In that case, the task + // iter_arg's layout map must be changed (see uses of `castBuffer`). + assert(memref::CastOp::areCastCompatible(buffer.getType(), type) && + "buffer cast incompatible in bufferization"); + return b.create(buffer.getLoc(), type, buffer).getResult(); +} + +/// Helper function for task bufferization. Given a list of bbArgs of the new +/// (bufferized) task op, wrap the bufferized tensor args (now memrefs) into +/// ToTensorOps, so that the block body can be moved over to the new op. +static SmallVector +getBbArgReplacements(RewriterBase &rewriter, Block::BlockArgListType bbArgs, + const DenseSet &tensorIndices) { + SmallVector result; + for (const auto &it : llvm::enumerate(bbArgs)) { + size_t idx = it.index(); + Value val = it.value(); + if (tensorIndices.contains(idx)) { + result.push_back( + rewriter.create(val.getLoc(), val) + .getResult()); + } else { + result.push_back(val); + } + } + return result; +} + struct TaskOpInterface - : public BufferizableOpInterface::ExternalModel { - /// Schedule/task do not have tensor OpOperands. Thus, no OpOperand will be - /// bufferized to memory read/write or be aliased to any returned values. + : public BufferizableOpInterface::ExternalModel { + bool bufferizesToMemoryRead(Operation *op, OpOperand &opOperand, + const AnalysisState &state) const { + auto task = cast(op); + + // TaskOp alone doesn't bufferize to a memory read, one of the uses of its + // matching bbArg may. + return state.isValueRead( + task.getBody().getArgument(opOperand.getOperandNumber())); + } + + bool bufferizesToMemoryWrite(Operation *op, OpOperand &opOperand, + const AnalysisState &state) const { + // Tensor `inits` of TaskOps are always considered as a write. + return true; + } + AliasingValueList getAliasingValues(Operation *op, OpOperand &opOperand, const AnalysisState &state) const { - return {}; + auto task = cast(op); + OpResult opResult = task->getOpResult(opOperand.getOperandNumber()); + BufferRelation relation = bufferRelation(op, opResult, state); + return {{opResult, relation, + /*isDefinite=*/relation == BufferRelation::Equivalent}}; } - // Schedule/task do not have tensor OpOperands. The yielded value can be any - // SSA value that is in scope. To allow for use-def chain traversal in the - // analysis, the yielded value is aliasing with the result. - AliasingOpOperandList - getAliasingOpOperands(Operation *op, Value value, - const AnalysisState &state) const { - OpOperand *operand = &cast(op).getYieldOp()->getOpOperand( - cast(value).getResultNumber()); - return {{operand, BufferRelation::Equivalent}}; + BufferRelation bufferRelation(Operation *op, OpResult opResult, + const AnalysisState &state) const { + // ForOp results are equivalent to their corresponding init_args if the + // corresponding iter_args and yield values are equivalent. + auto task = cast(op); + BlockArgument bbArg = + task.getBody().getArgument(opResult.getResultNumber()); + bool equivalentYield = state.areEquivalentBufferizedValues( + bbArg, task.getYieldOp().getOperand(bbArg.getArgNumber())); + return equivalentYield ? BufferRelation::Equivalent + : BufferRelation::Unknown; } - LogicalResult bufferize(Operation *op, RewriterBase &rewriter, - const BufferizationOptions &options) const { - OpBuilder::InsertionGuard g(rewriter); - auto concreteOp = cast(op); + bool isWritable(Operation *op, Value value, + const AnalysisState &state) const { + // Interestingly, TaskOp's bbArg can **always** be viewed inplace from the + // perspective of ops nested under: + // 1. Either the matching iter operand is not bufferized inplace and an + // alloc + optional copy makes the bbArg itself inplaceable. + // 2. Or the matching iter operand is bufferized inplace and bbArg just + // bufferizes to that too. + return true; + } + + LogicalResult resolveConflicts(Operation *op, RewriterBase &rewriter, + const AnalysisState &state) const { + auto bufferizableOp = cast(op); + if (failed(bufferizableOp.resolveTensorOpOperandConflicts(rewriter, state))) + return failure(); + + if (!state.getOptions().enforceAliasingInvariants) + return success(); - // Compute bufferized result types. - SmallVector newTypes; - for (Value result : concreteOp.getResults()) { - if (!result.getType().isa()) { - newTypes.push_back(result.getType()); + // According to the `getAliasing...` implementations, a bufferized OpResult + // may alias only with the corresponding bufferized init_arg (or with a + // newly allocated buffer) and not with other buffers defined outside of the + // task. I.e., the i-th OpResult may alias with the i-th init_arg; + // but not with any other OpOperand. + auto task = cast(op); + auto yieldOp = task.getYieldOp(); + OpBuilder::InsertionGuard g(rewriter); + rewriter.setInsertionPoint(yieldOp); + + // Indices of all iter_args that have tensor type. These are the ones that + // are bufferized. + DenseSet indices = getTensorIndices(task.getInits()); + // For every yielded value, does it alias with something defined outside of + // the task? + SmallVector yieldValues; + for (const auto it : llvm::enumerate(yieldOp.getResults())) { + // Note: `state` is guaranteed to be a `OneShotAnalysisState`, but this + // type cannot be used in the signature of `resolveConflicts` because the + // op interface is in the "IR" build unit and the `OneShotAnalysisState` + // is defined in the "Transforms" build unit. + if (!indices.contains(it.index()) || + doesNotAliasExternalValue( + it.value(), &task.getBody(), + /*exceptions=*/task.getBody().getArgument(it.index()), + static_cast(state))) { + yieldValues.push_back(it.value()); continue; } - auto bufferType = bufferization::getBufferType(result, options); - if (failed(bufferType)) + FailureOr alloc = allocateTensorForShapedValue( + rewriter, yieldOp.getLoc(), it.value(), state.getOptions()); + if (failed(alloc)) return failure(); - newTypes.push_back(*bufferType); + yieldValues.push_back(*alloc); } - // Create new schedule/task op. - rewriter.setInsertionPoint(concreteOp); - auto newOp = rewriter.create(concreteOp.getLoc(), newTypes, - ValueRange()); - rewriter.inlineRegionBefore(concreteOp.getBody(), newOp.getBody(), - newOp.getBody().end()); - - // Replace schedule/task op results. - replaceOpWithBufferizedValues(rewriter, concreteOp, newOp->getResults()); + rewriter.updateRootInPlace( + yieldOp, [&]() { yieldOp.getResultsMutable().assign(yieldValues); }); return success(); } FailureOr getBufferType(Operation *op, Value value, const BufferizationOptions &options, SmallVector &invocationStack) const { - assert(value.getDefiningOp() == op && "invalid value"); - auto yieldedValue = cast(op).getYieldOp()->getOperand( - value.cast().getResultNumber()); + auto task = cast(op); + assert(getOwnerOfValue(value) == op && "invalid value"); + assert(isa(value.getType()) && "expected tensor type"); + + if (auto opResult = dyn_cast(value)) { + // The type of an OpResult must match the corresponding iter_arg type. + BlockArgument bbArg = + task.getBody().getArgument(opResult.getResultNumber()); + return bufferization::getBufferType(bbArg, options, invocationStack); + } - if (auto bufferType = - yieldedValue.getType().template dyn_cast()) - return bufferType; + // Compute result/argument number. + BlockArgument bbArg = cast(value); + unsigned resultNum = bbArg.getArgNumber(); + + // Compute the bufferized type. + auto yieldOp = task.getYieldOp(); + Value yieldedValue = yieldOp.getOperand(resultNum); + BlockArgument iterArg = task.getBody().getArgument(resultNum); + Value initArg = task.getInits()[resultNum]; + return computeTaskRegionIterArgBufferType( + op, iterArg, initArg, yieldedValue, options, invocationStack); + } - auto maybeBufferType = - bufferization::getBufferType(yieldedValue, options, invocationStack); - if (failed(maybeBufferType)) + LogicalResult bufferize(Operation *op, RewriterBase &rewriter, + const BufferizationOptions &options) const { + auto task = cast(op); + Block *taskBlock = &task.getBody().front(); + + // Indices of all iter_args that have tensor type. These are the ones that + // are bufferized. + DenseSet indices = getTensorIndices(task.getInits()); + + // The new memref init_args of the task. + FailureOr> maybeInitArgs = + getBuffers(rewriter, task.getInitsMutable(), options); + if (failed(maybeInitArgs)) return failure(); - return *maybeBufferType; + SmallVector initArgs = *maybeInitArgs; + + // Cast init_args if necessary. + SmallVector castedInitArgs; + for (const auto &it : llvm::enumerate(initArgs)) { + Value initArg = it.value(); + Value result = task->getResult(it.index()); + // If the type is not a tensor, bufferization doesn't need to touch it. + if (!isa(result.getType())) { + castedInitArgs.push_back(initArg); + continue; + } + auto targetType = bufferization::getBufferType(result, options); + if (failed(targetType)) + return failure(); + castedInitArgs.push_back(castBuffer(rewriter, initArg, *targetType)); + } + + // Construct a new task op with memref instead of tensor values. + auto castedInitTypes = TypeRange(castedInitArgs); + auto newTask = + rewriter.create(task.getLoc(), castedInitTypes, castedInitArgs); + newTask->setAttrs(task->getAttrs()); + Block *newTaskBlock = rewriter.createBlock( + &newTask.getBody(), newTask.getBody().begin(), castedInitTypes, + llvm::map_to_vector(castedInitArgs, + [&](Value v) { return v.getLoc(); })); + + // Set up new iter_args. The task body uses tensors, so wrap the (memref) + // iter_args of the new task in ToTensorOps. + rewriter.setInsertionPointToStart(newTaskBlock); + SmallVector iterArgs = getBbArgReplacements( + rewriter, newTask.getBody().getArguments(), indices); + + // Move task body to new task. + rewriter.mergeBlocks(taskBlock, newTaskBlock, iterArgs); + + // Replace task results. + replaceOpWithBufferizedValues(rewriter, op, newTask->getResults()); + return success(); + } + + /// Assert that yielded values of an task op are equivalent to their + /// corresponding bbArgs. In that case, the buffer relations of the + /// corresponding OpResults are "Equivalent". + /// + /// If this is not the case, an allocs+copies are inserted and yielded from + /// the task. This could be a performance problem, so it must be explicitly + /// activated with `alloc-return-allocs`. + LogicalResult verifyAnalysis(Operation *op, + const AnalysisState &state) const { + auto task = cast(op); + auto yieldOp = task.getYieldOp(); + for (OpResult opResult : op->getOpResults()) { + if (!isa(opResult.getType())) + continue; + + // Note: This is overly strict. We should check for aliasing bufferized + // values. But we don't have a "must-alias" analysis yet. + if (bufferRelation(op, opResult, state) != BufferRelation::Equivalent) + return yieldOp->emitError() + << "Yield operand #" << opResult.getResultNumber() + << " is not equivalent to the corresponding iter bbArg"; + } + + return success(); } }; -/// Bufferization of fdf.yield operation. struct YieldOpInterface - : public BufferizableOpInterface::ExternalModel { - bool bufferizesToAllocation(Operation *op, Value value) const { return true; } - + : public BufferizableOpInterface::ExternalModel { bool bufferizesToMemoryRead(Operation *op, OpOperand &opOperand, const AnalysisState &state) const { return true; } + bool bufferizesToMemoryWrite(Operation *op, OpOperand &opOperand, const AnalysisState &state) const { return false; @@ -104,67 +408,46 @@ struct YieldOpInterface AliasingValueList getAliasingValues(Operation *op, OpOperand &opOperand, const AnalysisState &state) const { - if (isa(op->getParentOp())) - return {{op->getParentOp()->getResult(opOperand.getOperandNumber()), - BufferRelation::Equivalent}}; return {}; } bool mustBufferizeInPlace(Operation *op, OpOperand &opOperand, const AnalysisState &state) const { - // Yield operands always bufferize inplace. Otherwise, an alloc + copy may - // be generated inside the block. We should not return/yield allocations + // Yield operands always bufferize inplace. Otherwise, an alloc + copy + // may be generated inside the block. We should not return/yield allocations // when possible. return true; } LogicalResult bufferize(Operation *op, RewriterBase &rewriter, const BufferizationOptions &options) const { - OpBuilder::InsertionGuard g(rewriter); - auto yield = cast(op); - auto parent = yield->getParentOp(); - - // Traverse and bufferize each operand of the yield operation. - for (auto operand : yield.getOperands()) { - if (!operand.getType().isa()) - continue; - - auto maybeBuffer = getBuffer(rewriter, operand, options); - auto maybeType = bufferization::getBufferType(operand, options); - if (failed(maybeBuffer) || failed(maybeType)) - continue; - - // For now, we always generate an explicit copy to handle view-like - // operations. This is not efficient but it's safe. - if (auto view = maybeBuffer->getDefiningOp()) { - rewriter.setInsertionPoint(parent); - auto localBuffer = options.createAlloc( - rewriter, yield.getLoc(), maybeType->cast(), {}); - if (failed(localBuffer)) + auto yieldOp = cast(op); + + SmallVector newResults; + for (const auto &it : llvm::enumerate(yieldOp.getResults())) { + Value value = it.value(); + if (isa(value.getType())) { + FailureOr maybeBuffer = getBuffer(rewriter, value, options); + if (failed(maybeBuffer)) return failure(); + Value buffer = *maybeBuffer; - rewriter.setInsertionPoint(yield); - if (failed(options.createMemCpy(rewriter, yield.getLoc(), *maybeBuffer, - *localBuffer))) + FailureOr resultType = bufferization::getBufferType( + yieldOp->getParentOp()->getResult(it.index()), options); + if (failed(resultType)) return failure(); - - rewriter.replaceUsesWithIf(operand, *localBuffer, [&](OpOperand &use) { - return use.getOwner() == yield; - }); + buffer = castBuffer(rewriter, buffer, *resultType); + newResults.push_back(buffer); } else { - rewriter.setInsertionPoint(yield); - auto replacement = rewriter.create( - yield.getLoc(), *maybeType, operand); - rewriter.replaceUsesWithIf(operand, replacement, [&](OpOperand &use) { - return use.getOwner() == yield; - }); + newResults.push_back(value); } } + + replaceOpWithNewBufferizedOp(rewriter, op, newResults); return success(); } }; -/// Bufferization of fdf.tensor_init operation. struct TensorInitOpInterface : public BufferizableOpInterface::ExternalModel { @@ -172,15 +455,10 @@ struct TensorInitOpInterface bool resultBufferizesToMemoryWrite(Operation *op, OpResult opResult, const AnalysisState &state) const { + // The returned tensor does not have specified contents. return false; } - AliasingValueList getAliasingValues(Operation *op, OpOperand &opOperand, - const AnalysisState &state) const { - // This is a new allocation. It does not alias with any other buffer. - return {}; - } - LogicalResult bufferize(Operation *op, RewriterBase &rewriter, const BufferizationOptions &options) const { OpBuilder::InsertionGuard g(rewriter); @@ -198,22 +476,27 @@ struct TensorInitOpInterface if (failed(maybeType)) return failure(); - FailureOr buffer = options.createAlloc( - rewriter, tensorInit.getLoc(), maybeType->cast(), {}); - if (failed(buffer)) - return failure(); - - // Handle initial value. - if (auto initValue = tensorInit.getInitValue()) { - auto initValueOp = initValue.getDefiningOp(); - auto bufferOp = buffer->getDefiningOp(); - if (!initValueOp || !bufferOp) + for (auto &use : llvm::make_early_inc_range(tensorInit->getUses())) { + rewriter.setInsertionPoint(use.getOwner()); + FailureOr buffer = options.createAlloc( + rewriter, tensorInit.getLoc(), maybeType->cast(), {}); + if (failed(buffer)) return failure(); - bufferOp.setInitValueAttr(initValueOp.getValue()); - } - // Replace op. - replaceOpWithBufferizedValues(rewriter, tensorInit, *buffer); + // Handle initial value. + if (auto initValue = tensorInit.getInitValue()) { + auto initValueOp = initValue.getDefiningOp(); + auto bufferOp = buffer->getDefiningOp(); + if (!initValueOp || !bufferOp) + return failure(); + bufferOp.setInitValueAttr(initValueOp.getValue()); + } + + auto repl = rewriter.create( + tensorInit.getLoc(), *buffer); + rewriter.updateRootInPlace(use.getOwner(), [&]() { use.set(repl); }); + } + rewriter.eraseOp(tensorInit); return success(); } @@ -238,8 +521,8 @@ struct TensorInitOpInterface void mlir::scalehls::hls::registerBufferizableOpInterfaceExternalModels( DialectRegistry ®istry) { registry.addExtension(+[](MLIRContext *ctx, HLSDialect *dialect) { - TaskOp::attachInterface(*ctx); - YieldOp::attachInterface(*ctx); + hls::TaskOp::attachInterface(*ctx); + hls::YieldOp::attachInterface(*ctx); hls::TensorInitOp::attachInterface(*ctx); }); } diff --git a/lib/Dialect/HLS/Transforms/ConvertDataflowToFunc.cpp b/lib/Dialect/HLS/Transforms/ConvertDataflowToFunc.cpp index a058b166..f25424e6 100644 --- a/lib/Dialect/HLS/Transforms/ConvertDataflowToFunc.cpp +++ b/lib/Dialect/HLS/Transforms/ConvertDataflowToFunc.cpp @@ -92,7 +92,7 @@ struct ConvertTaskToFunc : public OpRewritePattern { subFunc->setAttrs(task->getAttrs()); // FIXME: A better method to judge whether to inline the node. - if (!task.hasHierarchy() && + if (task.isLeafTask() && (llvm::hasSingleElement(task.getOps()) || llvm::hasSingleElement(task.getOps()))) subFunc->setAttr("__inline__", rewriter.getUnitAttr()); diff --git a/lib/Dialect/HLS/Transforms/ScheduleDataflow.cpp b/lib/Dialect/HLS/Transforms/ScheduleDataflow.cpp index d6a04a87..02295d5a 100644 --- a/lib/Dialect/HLS/Transforms/ScheduleDataflow.cpp +++ b/lib/Dialect/HLS/Transforms/ScheduleDataflow.cpp @@ -17,14 +17,13 @@ using namespace hls; hls::TaskOp wrapOpIntoTask(Operation *op, StringRef taskName, ValueRange destOperands, OpBuilder &builder) { auto destTypes = TypeRange(destOperands); - auto destLocs = llvm::to_vector(llvm::map_range( - destOperands, [](Value destOperand) { return destOperand.getLoc(); })); builder.setInsertionPoint(op); auto task = builder.create(op->getLoc(), destTypes, destOperands); task->setAttr(taskName, builder.getUnitAttr()); - auto taskBlock = builder.createBlock(&task.getBody(), task.getBody().end(), - destTypes, destLocs); + auto taskBlock = builder.createBlock( + &task.getBody(), task.getBody().end(), destTypes, + llvm::map_to_vector(destOperands, [&](Value v) { return v.getLoc(); })); IRMapping mapper; for (auto [destOperand, taskBlockArg] : llvm::zip(destOperands, taskBlock->getArguments())) @@ -48,10 +47,14 @@ static LogicalResult scheduleBlock(StringRef prefix, Block *block, std::string taskName = prefix.str() + "_" + std::to_string(taskId); ValueRange destOperands; + if (auto loop = dyn_cast(op)) { if (failed(scheduleBlock(taskName, loop.getBody(), builder))) return failure(); destOperands = loop.getInitArgs(); + } else if (isa(op)) { + continue; } else if (auto destStyleOp = dyn_cast(op)) { destOperands = destStyleOp.getDpsInits(); } else if (auto writeOp = dyn_cast(op)) {