#include "CoroInternal.h"
#include "llvm/ADT/BitVector.h"
+#include "llvm/Analysis/PtrUseVisitor.h"
#include "llvm/Transforms/Utils/Local.h"
#include "llvm/Config/llvm-config.h"
#include "llvm/IR/CFG.h"
return FrameTy;
}
+// We use a pointer use visitor to discover if there are any writes into an
+// alloca that dominates CoroBegin. If that is the case, insertSpills will copy
+// the value from the alloca into the coroutine frame spill slot corresponding
+// to that alloca.
+namespace {
+struct AllocaUseVisitor : PtrUseVisitor<AllocaUseVisitor> {
+ using Base = PtrUseVisitor<AllocaUseVisitor>;
+ AllocaUseVisitor(const DataLayout &DL, const DominatorTree &DT,
+ const CoroBeginInst &CB)
+ : PtrUseVisitor(DL), DT(DT), CoroBegin(CB) {}
+
+ // We are only interested in uses that dominate coro.begin.
+ void visit(Instruction &I) {
+ if (DT.dominates(&I, &CoroBegin))
+ Base::visit(I);
+ }
+ // We need to provide this overload as PtrUseVisitor uses a pointer based
+ // visiting function.
+ void visit(Instruction *I) { return visit(*I); }
+
+ void visitLoadInst(LoadInst &) {} // Good. Nothing to do.
+
+ // If the use is an operand, the pointer escaped and anything can write into
+ // that memory. If the use is the pointer, we are definitely writing into the
+ // alloca and therefore we need to copy.
+ void visitStoreInst(StoreInst &SI) { PI.setAborted(&SI); }
+
+ // Any other instruction that is not filtered out by PtrUseVisitor, will
+ // result in the copy.
+ void visitInstruction(Instruction &I) { PI.setAborted(&I); }
+
+private:
+ const DominatorTree &DT;
+ const CoroBeginInst &CoroBegin;
+};
+} // namespace
+static bool mightWriteIntoAllocaPtr(AllocaInst &A, const DominatorTree &DT,
+ const CoroBeginInst &CB) {
+ const DataLayout &DL = A.getModule()->getDataLayout();
+ AllocaUseVisitor Visitor(DL, DT, CB);
+ auto PtrI = Visitor.visitPtr(A);
+ if (PtrI.isEscaped() || PtrI.isAborted()) {
+ auto *PointerEscapingInstr = PtrI.getEscapingInst()
+ ? PtrI.getEscapingInst()
+ : PtrI.getAbortingInst();
+ if (PointerEscapingInstr) {
+ LLVM_DEBUG(
+ dbgs() << "AllocaInst copy was triggered by instruction: "
+ << *PointerEscapingInstr << "\n");
+ }
+ return true;
+ }
+ return false;
+}
+
// We need to make room to insert a spill after initial PHIs, but before
// catchswitch instruction. Placing it before violates the requirement that
// catchswitch, like all other EHPads must be the first nonPHI in a block.
PointerType *FramePtrTy = FrameTy->getPointerTo();
auto *FramePtr =
cast<Instruction>(Builder.CreateBitCast(CB, FramePtrTy, "FramePtr"));
+ DominatorTree DT(*CB->getFunction());
Value *CurrentValue = nullptr;
BasicBlock *CurrentBlock = nullptr;
// that the suspend will be followed by a branch.
InsertPt = CSI->getParent()->getSingleSuccessor()->getFirstNonPHI();
} else {
+ auto *I = cast<Instruction>(E.def());
+ assert(!I->isTerminator() && "unexpected terminator");
// For all other values, the spill is placed immediately after
// the definition.
- assert(!cast<Instruction>(E.def())->isTerminator() &&
- "unexpected terminator");
- InsertPt = cast<Instruction>(E.def())->getNextNode();
+ if (DT.dominates(CB, I)) {
+ InsertPt = I->getNextNode();
+ } else {
+ // Unless, it is not dominated by CoroBegin, then it will be
+ // inserted immediately after CoroFrame is computed.
+ InsertPt = FramePtr->getNextNode();
+ }
}
Builder.SetInsertPoint(InsertPt);
FramePtrBB->splitBasicBlock(FramePtr->getNextNode(), "AllocaSpillBB");
SpillBlock->splitBasicBlock(&SpillBlock->front(), "PostSpill");
Shape.AllocaSpillBlock = SpillBlock;
-
// If we found any allocas, replace all of their remaining uses with Geps.
- Builder.SetInsertPoint(&SpillBlock->front());
+ // Note: we cannot do it indiscriminately as some of the uses may not be
+ // dominated by CoroBegin.
+ bool MightNeedToCopy = false;
+ Builder.SetInsertPoint(&Shape.AllocaSpillBlock->front());
+ SmallVector<Instruction *, 4> UsersToUpdate;
for (auto &P : Allocas) {
- auto *G = GetFramePointer(P.second, P.first);
+ AllocaInst *const A = P.first;
+ UsersToUpdate.clear();
+ for (User *U : A->users()) {
+ auto *I = cast<Instruction>(U);
+ if (DT.dominates(CB, I))
+ UsersToUpdate.push_back(I);
+ else
+ MightNeedToCopy = true;
+ }
+ if (!UsersToUpdate.empty()) {
+ auto *G = GetFramePointer(P.second, A);
+ G->takeName(A);
+ for (Instruction *I : UsersToUpdate)
+ I->replaceUsesOfWith(A, G);
+ }
+ }
+ // If we discovered such uses not dominated by CoroBegin, see if any of them
+ // preceed coro begin and have instructions that can modify the
+ // value of the alloca and therefore would require a copying the value into
+ // the spill slot in the coroutine frame.
+ if (MightNeedToCopy) {
+ Builder.SetInsertPoint(FramePtr->getNextNode());
+
+ for (auto &P : Allocas) {
+ AllocaInst *const A = P.first;
+ if (mightWriteIntoAllocaPtr(*A, DT, *CB)) {
+ if (A->isArrayAllocation())
+ report_fatal_error(
+ "Coroutines cannot handle copying of array allocas yet");
- // We are not using ReplaceInstWithInst(P.first, cast<Instruction>(G)) here,
- // as we are changing location of the instruction.
- G->takeName(P.first);
- P.first->replaceAllUsesWith(G);
- P.first->eraseFromParent();
+ auto *G = GetFramePointer(P.second, A);
+ auto *Value = Builder.CreateLoad(A);
+ Builder.CreateStore(Value, G);
+ }
+ }
}
return FramePtr;
}
}
}
-// Move early uses of spilled variable after CoroBegin.
-// For example, if a parameter had address taken, we may end up with the code
-// like:
-// define @f(i32 %n) {
-// %n.addr = alloca i32
-// store %n, %n.addr
-// ...
-// call @coro.begin
-// we need to move the store after coro.begin
-static void moveSpillUsesAfterCoroBegin(Function &F, SpillInfo const &Spills,
- CoroBeginInst *CoroBegin) {
- DominatorTree DT(F);
- SmallVector<Instruction *, 8> NeedsMoving;
-
- Value *CurrentValue = nullptr;
-
- for (auto const &E : Spills) {
- if (CurrentValue == E.def())
- continue;
-
- CurrentValue = E.def();
-
- for (User *U : CurrentValue->users()) {
- Instruction *I = cast<Instruction>(U);
- if (!DT.dominates(CoroBegin, I)) {
- LLVM_DEBUG(dbgs() << "will move: " << *I << "\n");
-
- // TODO: Make this more robust. Currently if we run into a situation
- // where simple instruction move won't work we panic and
- // report_fatal_error.
- for (User *UI : I->users()) {
- if (!DT.dominates(CoroBegin, cast<Instruction>(UI)))
- report_fatal_error("cannot move instruction since its users are not"
- " dominated by CoroBegin");
- }
-
- NeedsMoving.push_back(I);
- }
- }
- }
-
- Instruction *InsertPt = CoroBegin->getNextNode();
- for (Instruction *I : NeedsMoving)
- I->moveBefore(InsertPt);
-}
-
// Splits the block at a particular instruction unless it is the first
// instruction in the block with a single predecessor.
static BasicBlock *splitBlockIfNotFirst(Instruction *I, const Twine &Name) {
}
}
LLVM_DEBUG(dump("Spills", Spills));
- moveSpillUsesAfterCoroBegin(F, Spills, Shape.CoroBegin);
Shape.FrameTy = buildFrameType(F, Shape, Spills);
Shape.FramePtr = insertSpills(Spills, Shape);
lowerLocalAllocas(LocalAllocas, DeadInstructions);
S.resize(N);
}
-static SmallPtrSet<BasicBlock *, 4> getCoroBeginPredBlocks(CoroBeginInst *CB) {
- // Collect all blocks that we need to look for instructions to relocate.
- SmallPtrSet<BasicBlock *, 4> RelocBlocks;
- SmallVector<BasicBlock *, 4> Work;
- Work.push_back(CB->getParent());
-
- do {
- BasicBlock *Current = Work.pop_back_val();
- for (BasicBlock *BB : predecessors(Current))
- if (RelocBlocks.count(BB) == 0) {
- RelocBlocks.insert(BB);
- Work.push_back(BB);
- }
- } while (!Work.empty());
- return RelocBlocks;
-}
-
-static SmallPtrSet<Instruction *, 8>
-getNotRelocatableInstructions(CoroBeginInst *CoroBegin,
- SmallPtrSetImpl<BasicBlock *> &RelocBlocks) {
- SmallPtrSet<Instruction *, 8> DoNotRelocate;
- // Collect all instructions that we should not relocate
- SmallVector<Instruction *, 8> Work;
-
- // Start with CoroBegin and terminators of all preceding blocks.
- Work.push_back(CoroBegin);
- BasicBlock *CoroBeginBB = CoroBegin->getParent();
- for (BasicBlock *BB : RelocBlocks)
- if (BB != CoroBeginBB)
- Work.push_back(BB->getTerminator());
-
- // For every instruction in the Work list, place its operands in DoNotRelocate
- // set.
- do {
- Instruction *Current = Work.pop_back_val();
- LLVM_DEBUG(dbgs() << "CoroSplit: Will not relocate: " << *Current << "\n");
- DoNotRelocate.insert(Current);
- for (Value *U : Current->operands()) {
- auto *I = dyn_cast<Instruction>(U);
- if (!I)
- continue;
-
- if (auto *A = dyn_cast<AllocaInst>(I)) {
- // Stores to alloca instructions that occur before the coroutine frame
- // is allocated should not be moved; the stored values may be used by
- // the coroutine frame allocator. The operands to those stores must also
- // remain in place.
- for (const auto &User : A->users())
- if (auto *SI = dyn_cast<llvm::StoreInst>(User))
- if (RelocBlocks.count(SI->getParent()) != 0 &&
- DoNotRelocate.count(SI) == 0) {
- Work.push_back(SI);
- DoNotRelocate.insert(SI);
- }
- continue;
- }
-
- if (DoNotRelocate.count(I) == 0) {
- Work.push_back(I);
- DoNotRelocate.insert(I);
- }
- }
- } while (!Work.empty());
- return DoNotRelocate;
-}
-
-static void relocateInstructionBefore(CoroBeginInst *CoroBegin, Function &F) {
- // Analyze which non-alloca instructions are needed for allocation and
- // relocate the rest to after coro.begin. We need to do it, since some of the
- // targets of those instructions may be placed into coroutine frame memory
- // for which becomes available after coro.begin intrinsic.
-
- auto BlockSet = getCoroBeginPredBlocks(CoroBegin);
- auto DoNotRelocateSet = getNotRelocatableInstructions(CoroBegin, BlockSet);
-
- Instruction *InsertPt = CoroBegin->getNextNode();
- BasicBlock &BB = F.getEntryBlock(); // TODO: Look at other blocks as well.
- for (auto B = BB.begin(), E = BB.end(); B != E;) {
- Instruction &I = *B++;
- if (isa<AllocaInst>(&I))
- continue;
- if (&I == CoroBegin)
- break;
- if (DoNotRelocateSet.count(&I))
- continue;
- I.moveBefore(InsertPt);
- }
-}
-
static void splitSwitchCoroutine(Function &F, coro::Shape &Shape,
SmallVectorImpl<Function *> &Clones) {
assert(Shape.ABI == coro::ABI::Switch);
return;
simplifySuspendPoints(Shape);
- relocateInstructionBefore(Shape.CoroBegin, F);
buildCoroutineFrame(F, Shape);
replaceFrameSize(Shape);
--- /dev/null
+; Check that we can handle the case when both alloc function and
+; the user body consume the same argument.
+; RUN: opt < %s -coro-split -S | FileCheck %s
+
+; using this directly (as it would happen under -O2)
+define i8* @f_direct(i64 %this) "coroutine.presplit"="1" {
+entry:
+ %id = call token @llvm.coro.id(i32 0, i8* null, i8* null, i8* null)
+ %size = call i32 @llvm.coro.size.i32()
+ %alloc = call i8* @myAlloc(i64 %this, i32 %size)
+ %hdl = call i8* @llvm.coro.begin(token %id, i8* %alloc)
+ %0 = call i8 @llvm.coro.suspend(token none, i1 false)
+ switch i8 %0, label %suspend [i8 0, label %resume
+ i8 1, label %cleanup]
+resume:
+ call void @print2(i64 %this)
+ br label %cleanup
+
+cleanup:
+ %mem = call i8* @llvm.coro.free(token %id, i8* %hdl)
+ call void @free(i8* %mem)
+ br label %suspend
+suspend:
+ call i1 @llvm.coro.end(i8* %hdl, i1 0)
+ ret i8* %hdl
+}
+
+; using copy of this (as it would happen under -O0)
+define i8* @f_copy(i64 %this_arg) "coroutine.presplit"="1" {
+entry:
+ %this.addr = alloca i64
+ store i64 %this_arg, i64* %this.addr
+ %this = load i64, i64* %this.addr
+ %id = call token @llvm.coro.id(i32 0, i8* null, i8* null, i8* null)
+ %size = call i32 @llvm.coro.size.i32()
+ %alloc = call i8* @myAlloc(i64 %this, i32 %size)
+ %hdl = call i8* @llvm.coro.begin(token %id, i8* %alloc)
+ %0 = call i8 @llvm.coro.suspend(token none, i1 false)
+ switch i8 %0, label %suspend [i8 0, label %resume
+ i8 1, label %cleanup]
+resume:
+ call void @print2(i64 %this)
+ br label %cleanup
+
+cleanup:
+ %mem = call i8* @llvm.coro.free(token %id, i8* %hdl)
+ call void @free(i8* %mem)
+ br label %suspend
+suspend:
+ call i1 @llvm.coro.end(i8* %hdl, i1 0)
+ ret i8* %hdl
+}
+
+; See if %this was added to the frame
+; CHECK: %f_direct.Frame = type { void (%f_direct.Frame*)*, void (%f_direct.Frame*)*, i1, i1, i64 }
+; CHECK: %f_copy.Frame = type { void (%f_copy.Frame*)*, void (%f_copy.Frame*)*, i1, i1, i64 }
+
+; See that %this is spilled into the frame
+; CHECK-LABEL: define i8* @f_direct(i64 %this)
+; CHECK: %this.spill.addr = getelementptr inbounds %f_direct.Frame, %f_direct.Frame* %FramePtr, i32 0, i32 4
+; CHECK: store i64 %this, i64* %this.spill.addr
+; CHECK: ret i8* %hdl
+
+; See that %this is spilled into the frame
+; CHECK-LABEL: define i8* @f_copy(i64 %this_arg)
+; CHECK: %this.spill.addr = getelementptr inbounds %f_copy.Frame, %f_copy.Frame* %FramePtr, i32 0, i32 4
+; CHECK: store i64 %this_arg, i64* %this.spill.addr
+; CHECK: ret i8* %hdl
+
+; See that %this was loaded from the frame
+; CHECK-LABEL: @f_direct.resume(
+; CHECK: %this.reload = load i64, i64* %this.reload.addr
+; CHECK: call void @print2(i64 %this.reload)
+; CHECK: ret void
+
+; See that %this was loaded from the frame
+; CHECK-LABEL: @f_copy.resume(
+; CHECK: %this.reload = load i64, i64* %this.reload.addr
+; CHECK: call void @print2(i64 %this.reload)
+; CHECK: ret void
+
+declare i8* @llvm.coro.free(token, i8*)
+declare i32 @llvm.coro.size.i32()
+declare i8 @llvm.coro.suspend(token, i1)
+declare void @llvm.coro.resume(i8*)
+declare void @llvm.coro.destroy(i8*)
+
+declare token @llvm.coro.id(i32, i8*, i8*, i8*)
+declare i1 @llvm.coro.alloc(token)
+declare i8* @llvm.coro.begin(token, i8*)
+declare i1 @llvm.coro.end(i8*, i1)
+
+declare noalias i8* @myAlloc(i64, i32)
+declare double @print(double)
+declare void @print2(i64)
+declare void @free(i8*)
--- /dev/null
+; Check that we create copy the data from the alloca into the coroutine
+; frame slot if it was written to.
+; RUN: opt < %s -coro-split -S | FileCheck %s
+
+define i8* @f() "coroutine.presplit"="1" {
+entry:
+ %x.addr = alloca i64
+ call void @use(i64* %x.addr) ; might write to %x
+ %y.addr = alloca i64
+ %y = load i64, i64* %y.addr ; cannot modify the value, don't need to copy
+ call void @print(i64 %y)
+
+ %id = call token @llvm.coro.id(i32 0, i8* null, i8* null, i8* null)
+ %size = call i32 @llvm.coro.size.i32()
+ %alloc = call i8* @myAlloc(i64 %y, i32 %size)
+ %hdl = call i8* @llvm.coro.begin(token %id, i8* %alloc)
+ %0 = call i8 @llvm.coro.suspend(token none, i1 false)
+ switch i8 %0, label %suspend [i8 0, label %resume
+ i8 1, label %cleanup]
+resume:
+ call void @use(i64* %x.addr)
+ call void @use(i64* %y.addr)
+ br label %cleanup
+
+cleanup:
+ %mem = call i8* @llvm.coro.free(token %id, i8* %hdl)
+ call void @free(i8* %mem)
+ br label %suspend
+suspend:
+ call i1 @llvm.coro.end(i8* %hdl, i1 0)
+ ret i8* %hdl
+}
+
+; See that we added both x and y to the frame.
+; CHECK: %f.Frame = type { void (%f.Frame*)*, void (%f.Frame*)*, i1, i1, i64, i64 }
+
+; See that all of the uses prior to coro-begin stays put.
+; CHECK-LABEL: define i8* @f() {
+; CHECK-NEXT: entry:
+; CHECK-NEXT: %x.addr = alloca i64
+; CHECK-NEXT: call void @use(i64* %x.addr)
+; CHECK-NEXT: %y.addr = alloca i64
+; CHECK-NEXT: %y = load i64, i64* %y.addr
+; CHECK-NEXT: call void @print(i64 %y)
+
+; See that we only copy the x as y was not modified prior to coro.begin.
+; CHECK: store void (%f.Frame*)* @f.destroy, void (%f.Frame*)** %destroy.addr
+; CHECK-NEXT: %0 = getelementptr inbounds %f.Frame, %f.Frame* %FramePtr, i32 0, i32 4
+; CHECK-NEXT: %1 = load i64, i64* %x.addr
+; CHECK-NEXT: store i64 %1, i64* %0
+; CHECK-NEXT: %index.addr1 = getelementptr inbounds %f.Frame, %f.Frame* %FramePtr, i32 0, i32 3
+; CHECK-NEXT: store i1 false, i1* %index.addr1
+; CHECK-NEXT: ret i8* %hdl
+
+declare i8* @llvm.coro.free(token, i8*)
+declare i32 @llvm.coro.size.i32()
+declare i8 @llvm.coro.suspend(token, i1)
+declare void @llvm.coro.resume(i8*)
+declare void @llvm.coro.destroy(i8*)
+
+declare token @llvm.coro.id(i32, i8*, i8*, i8*)
+declare i1 @llvm.coro.alloc(token)
+declare i8* @llvm.coro.begin(token, i8*)
+declare i1 @llvm.coro.end(i8*, i1)
+
+declare noalias i8* @myAlloc(i64, i32)
+declare void @print(i64)
+declare void @use(i64*)
+declare void @free(i8*)