Index: llvm/include/llvm/Transforms/Coroutines.h =================================================================== --- llvm/include/llvm/Transforms/Coroutines.h +++ llvm/include/llvm/Transforms/Coroutines.h @@ -23,7 +23,7 @@ Pass *createCoroEarlyLegacyPass(); /// Split up coroutines into multiple functions driving their state machines. -Pass *createCoroSplitLegacyPass(); +Pass *createCoroSplitLegacyPass(bool ReuseFrameSlot = false); /// Analyze coroutines use sites, devirtualize resume/destroy calls and elide /// heap allocation for coroutine frame where possible. Index: llvm/include/llvm/Transforms/Coroutines/CoroSplit.h =================================================================== --- llvm/include/llvm/Transforms/Coroutines/CoroSplit.h +++ llvm/include/llvm/Transforms/Coroutines/CoroSplit.h @@ -18,13 +18,18 @@ #include "llvm/Analysis/CGSCCPassManager.h" #include "llvm/Analysis/LazyCallGraph.h" #include "llvm/IR/PassManager.h" +#include "llvm/Passes/PassBuilder.h" namespace llvm { struct CoroSplitPass : PassInfoMixin { + CoroSplitPass(bool ReuseFrameSlot = false) : ReuseFrameSlot(ReuseFrameSlot) {} + PreservedAnalyses run(LazyCallGraph::SCC &C, CGSCCAnalysisManager &AM, LazyCallGraph &CG, CGSCCUpdateResult &UR); static bool isRequired() { return true; } + + bool ReuseFrameSlot; }; } // end namespace llvm Index: llvm/lib/Passes/PassBuilder.cpp =================================================================== --- llvm/lib/Passes/PassBuilder.cpp +++ llvm/lib/Passes/PassBuilder.cpp @@ -878,7 +878,7 @@ MainCGPipeline.addPass(AttributorCGSCCPass()); if (PTO.Coroutines) - MainCGPipeline.addPass(CoroSplitPass()); + MainCGPipeline.addPass(CoroSplitPass(Level != OptimizationLevel::O0)); // Now deduce any function attributes based in the current code. MainCGPipeline.addPass(PostOrderFunctionAttrsPass()); Index: llvm/lib/Transforms/Coroutines/CMakeLists.txt =================================================================== --- llvm/lib/Transforms/Coroutines/CMakeLists.txt +++ llvm/lib/Transforms/Coroutines/CMakeLists.txt @@ -8,4 +8,5 @@ DEPENDS intrinsics_gen + LLVMAnalysis ) Index: llvm/lib/Transforms/Coroutines/CoroFrame.cpp =================================================================== --- llvm/lib/Transforms/Coroutines/CoroFrame.cpp +++ llvm/lib/Transforms/Coroutines/CoroFrame.cpp @@ -20,16 +20,18 @@ #include "llvm/ADT/BitVector.h" #include "llvm/ADT/SmallString.h" #include "llvm/Analysis/PtrUseVisitor.h" +#include "llvm/Analysis/StackLifetime.h" #include "llvm/Config/llvm-config.h" #include "llvm/IR/CFG.h" #include "llvm/IR/DIBuilder.h" #include "llvm/IR/Dominators.h" #include "llvm/IR/IRBuilder.h" #include "llvm/IR/InstIterator.h" +#include "llvm/Support/CommandLine.h" #include "llvm/Support/Debug.h" #include "llvm/Support/MathExtras.h" -#include "llvm/Support/circular_raw_ostream.h" #include "llvm/Support/OptimizedStructLayout.h" +#include "llvm/Support/circular_raw_ostream.h" #include "llvm/Transforms/Utils/BasicBlockUtils.h" #include "llvm/Transforms/Utils/Local.h" #include "llvm/Transforms/Utils/PromoteMemToReg.h" @@ -41,6 +43,13 @@ // "coro-frame", which results in leaner debug spew. #define DEBUG_TYPE "coro-suspend-crossing" +static cl::opt EnableReuseStorageInFrame( + "reuse-storage-in-coroutine-frame", cl::Hidden, + cl::desc( + "Enable the optimization which would reuse the storage in the coroutine \ + frame for allocas whose liferanges are not overlapped, for testing purposes"), + llvm::cl::init(false)); + enum { SmallVectorThreshold = 32 }; // Provides two way mapping between the blocks and numbers. @@ -342,10 +351,14 @@ // differs from the natural alignment of the alloca type we will need to insert // padding. class FrameTypeBuilder { +public: + using ForSpillType = SmallVector; + +private: struct Field { uint64_t Size; uint64_t Offset; - Spill *ForSpill; + ForSpillType ForSpill; Type *Ty; unsigned FieldIndex; Align Alignment; @@ -363,7 +376,7 @@ public: FrameTypeBuilder(LLVMContext &Context, DataLayout const &DL) - : DL(DL), Context(Context) {} + : DL(DL), Context(Context) {} class FieldId { size_t Value; @@ -374,7 +387,7 @@ /// Add a field to this structure for the storage of an `alloca` /// instruction. - FieldId addFieldForAlloca(AllocaInst *AI, Spill *ForSpill = nullptr, + FieldId addFieldForAlloca(AllocaInst *AI, ForSpillType ForSpill = {}, bool IsHeader = false) { Type *Ty = AI->getAllocatedType(); @@ -389,10 +402,39 @@ return addField(Ty, AI->getAlign(), ForSpill, IsHeader); } + /// We want to put the allocas whose lifetime-ranges are not overlapped + /// into one slot of coroutine frame. + /// Consider the example at:https://bugs.llvm.org/show_bug.cgi?id=45566 + /// + /// cppcoro::task alternative_paths(bool cond) { + /// if (cond) { + /// big_structure a; + /// process(a); + /// co_await something(); + /// } else { + /// big_structure b; + /// process2(b); + /// co_await something(); + /// } + /// } + /// + /// We want to put variable a and variable b in the same slot to + /// reduce the size of coroutine frame. + /// + /// This function use StackLifetime algorithm to partition the AllocaInsts in + /// Spills to non-overlapped sets in order to put Alloca in the same + /// non-overlapped set into the same slot in the Coroutine Frame. Then add + /// field for the allocas in the same non-overlapped set by using the largest + /// type as the field type. + /// + /// Side Effects: Because We sort the allocas, the order of allocas in the + /// frame may be different with the order in the source code. + void addFieldForAllocas(const Function &F, SpillInfo &Spills, + coro::Shape &Shape); + /// Add a field to this structure. FieldId addField(Type *Ty, MaybeAlign FieldAlignment, - Spill *ForSpill = nullptr, - bool IsHeader = false) { + ForSpillType ForSpill = {}, bool IsHeader = false) { assert(!IsFinished && "adding fields to a finished builder"); assert(Ty && "must provide a type for a field"); @@ -440,6 +482,130 @@ }; } // namespace +void FrameTypeBuilder::addFieldForAllocas(const Function &F, SpillInfo &Spills, + coro::Shape &Shape) { + DenseMap AllocaIndex; + SmallVector Allocas; + DenseMap SpillOfAllocas; + using AllocaSetType = SmallVector; + SmallVector NonOverlapedAllocas; + + // We need to add field for allocas at the end of this function. However, this + // function has multiple exits, so we use this helper to avoid redundant code. + struct RTTIHelper { + std::function func; + RTTIHelper(std::function &&func) : func(func) {} + ~RTTIHelper() { func(); } + } Helper([&]() { + for (auto AllocaSet : NonOverlapedAllocas) { + ForSpillType ForSpills; + for (auto Alloca : AllocaSet) + ForSpills.push_back(SpillOfAllocas[Alloca]); + auto *LargestAI = *AllocaSet.begin(); + addFieldForAlloca(LargestAI, ForSpills); + } + }); + + for (auto &Spill : Spills) + if (AllocaInst *AI = dyn_cast(Spill.def())) + if (find(Allocas, AI) == Allocas.end()) { + SpillOfAllocas[AI] = &Spill; + Allocas.emplace_back(AI); + } + + if (!Shape.ReuseFrameSlot && !EnableReuseStorageInFrame) { + for (auto Alloca : Allocas) { + AllocaIndex[Alloca] = NonOverlapedAllocas.size(); + NonOverlapedAllocas.emplace_back(AllocaSetType(1, Alloca)); + } + return; + } + + // Because there are pathes from the lifetime.start to coro.end + // for each alloca, the liferanges for every alloca is overlaped + // in the blocks who contain coro.end and the successor blocks. + // So we choose to skip there blocks when we calculates the liferange + // for each alloca. It should be reasonable since there shouldn't be uses + // in these blocks and the coroutine frame shouldn't be used outside the + // coroutine body. + // + // Note that the user of coro.suspend may not be SwitchInst. However, this + // case seems too complex to handle. And it is harmless to skip these + // patterns since it just prevend putting the allocas to live in the same + // slot. + DenseMap DefaultSuspendDest; + for (auto CoroSuspendInst : Shape.CoroSuspends) { + for (auto U : CoroSuspendInst->users()) { + if (auto *ConstSWI = dyn_cast(U)) { + auto *SWI = const_cast(ConstSWI); + DefaultSuspendDest[SWI] = SWI->getDefaultDest(); + SWI->setDefaultDest(SWI->getSuccessor(1)); + } + } + } + + StackLifetime StackLifetimeAnalyzer(F, Allocas, + StackLifetime::LivenessType::May); + StackLifetimeAnalyzer.run(); + auto IsAllocaInferenre = [&](const AllocaInst *AI1, const AllocaInst *AI2) { + return StackLifetimeAnalyzer.getLiveRange(AI1).overlaps( + StackLifetimeAnalyzer.getLiveRange(AI2)); + }; + auto GetAllocaSize = [&](const AllocaInst *AI) { + Optional RetSize = AI->getAllocationSizeInBits(DL); + assert(RetSize && "We can't handle scalable type now.\n"); + return RetSize.getValue(); + }; + // Put larger allocas in the front. So the larger allocas have higher + // priority to merge, which can save more space potentially. Also each + // AllocaSet would be ordered. So we can get the largest Alloca in one + // AllocaSet easily. + sort(Allocas, [&](auto Iter1, auto Iter2) { + return GetAllocaSize(Iter1) > GetAllocaSize(Iter2); + }); + for (auto Alloca : Allocas) { + bool Merged = false; + // Try to find if the Alloca is not inferenced with any existing + // NonOverlappedAllocaSet. If it is true, insert the alloca to that + // NonOverlappedAllocaSet. + for (auto &AllocaSet : NonOverlapedAllocas) { + assert(!AllocaSet.empty() && "Processing Alloca Set is not empty.\n"); + bool CouldMerge = none_of(AllocaSet, [&](auto Iter) { + return IsAllocaInferenre(Alloca, Iter); + }); + if (!CouldMerge) + continue; + AllocaIndex[Alloca] = AllocaIndex[*AllocaSet.begin()]; + AllocaSet.push_back(Alloca); + Merged = true; + break; + } + if (!Merged) { + AllocaIndex[Alloca] = NonOverlapedAllocas.size(); + NonOverlapedAllocas.emplace_back(AllocaSetType(1, Alloca)); + } + } + // Recover the default target destination for each Switch statement + // reserved. + for (auto SwitchAndDefaultDest : DefaultSuspendDest) { + SwitchInst *SWI = SwitchAndDefaultDest.first; + BasicBlock *DestBB = SwitchAndDefaultDest.second; + SWI->setDefaultDest(DestBB); + } + // This Debug Info could tell us which allocas are merged into one slot. + LLVM_DEBUG(for (auto &AllocaSet + : NonOverlapedAllocas) { + if (AllocaSet.size() > 1) { + dbgs() << "In Function:" << F.getName() << "\n"; + dbgs() << "Find Union Set " + << "\n"; + dbgs() << "\tAllocas are \n"; + for (auto Alloca : AllocaSet) + dbgs() << "\t\t" << *Alloca << "\n"; + } + }); +} + void FrameTypeBuilder::finish(StructType *Ty) { assert(!IsFinished && "already finished!"); @@ -495,8 +661,8 @@ // original Spill, if there is one. F.Offset = Offset; F.FieldIndex = FieldTypes.size(); - if (F.ForSpill) { - F.ForSpill->setFieldIndex(F.FieldIndex); + for (auto Spill : F.ForSpill) { + Spill->setFieldIndex(F.FieldIndex); } FieldTypes.push_back(F.Ty); @@ -549,13 +715,12 @@ // Add header fields for the resume and destroy functions. // We can rely on these being perfectly packed. - B.addField(FnPtrTy, None, nullptr, /*header*/ true); - B.addField(FnPtrTy, None, nullptr, /*header*/ true); + B.addField(FnPtrTy, None, {}, /*header*/ true); + B.addField(FnPtrTy, None, {}, /*header*/ true); // Add a header field for the promise if there is one. if (PromiseAlloca) { - PromiseFieldId = - B.addFieldForAlloca(PromiseAlloca, nullptr, /*header*/ true); + PromiseFieldId = B.addFieldForAlloca(PromiseAlloca, {}, /*header*/ true); } // Add a field to store the suspend index. This doesn't need to @@ -568,9 +733,11 @@ assert(PromiseAlloca == nullptr && "lowering doesn't support promises"); } + // Because multiple allocas may own the same field slot, + // we add allocas to field here. + B.addFieldForAllocas(F, Spills, Shape); Value *CurrentDef = nullptr; - - // Create an entry for every spilled value. + // Create an entry for every spilled value which is not an AllocaInst. for (auto &S : Spills) { // We can have multiple entries in Spills for a single value, but // they should form a contiguous run. Ignore all but the first. @@ -582,11 +749,9 @@ assert(CurrentDef != PromiseAlloca && "recorded spill use of promise alloca?"); - if (auto *AI = dyn_cast(CurrentDef)) { - B.addFieldForAlloca(AI, &S); - } else { + if (!isa(CurrentDef)) { Type *Ty = CurrentDef->getType(); - B.addField(Ty, None, &S); + B.addField(Ty, None, {&S}); } } @@ -820,7 +985,17 @@ } } - return Builder.CreateInBoundsGEP(FrameTy, FramePtr, Indices); + auto GEP = cast( + Builder.CreateInBoundsGEP(FrameTy, FramePtr, Indices)); + if (isa(Orig)) { + // If the type of GEP is not equal to the type of AllocaInst, it implies + // that the AllocaInst may be reused in the Frame slot of other + // AllocaInst. So we cast the GEP to the type of AllocaInst. + if (GEP->getResultElementType() != Orig->getType()) + return Builder.CreateBitCast(GEP, Orig->getType(), + Orig->getName() + Twine(".cast")); + } + return GEP; }; // Create a load instruction to reload the spilled value from the coroutine Index: llvm/lib/Transforms/Coroutines/CoroInternal.h =================================================================== --- llvm/lib/Transforms/Coroutines/CoroInternal.h +++ llvm/lib/Transforms/Coroutines/CoroInternal.h @@ -115,6 +115,8 @@ Instruction *FramePtr; BasicBlock *AllocaSpillBlock; + bool ReuseFrameSlot; + struct SwitchLoweringStorage { SwitchInst *ResumeSwitch; AllocaInst *PromiseAlloca; @@ -238,12 +240,14 @@ void emitDealloc(IRBuilder<> &Builder, Value *Ptr, CallGraph *CG) const; Shape() = default; - explicit Shape(Function &F) { buildFrom(F); } + explicit Shape(Function &F, bool ReuseFrameSlot = false) + : ReuseFrameSlot(ReuseFrameSlot) { + buildFrom(F); + } void buildFrom(Function &F); }; void buildCoroutineFrame(Function &F, Shape &Shape); - } // End namespace coro. } // End namespace llvm Index: llvm/lib/Transforms/Coroutines/CoroSplit.cpp =================================================================== --- llvm/lib/Transforms/Coroutines/CoroSplit.cpp +++ llvm/lib/Transforms/Coroutines/CoroSplit.cpp @@ -53,6 +53,7 @@ #include "llvm/IR/Verifier.h" #include "llvm/InitializePasses.h" #include "llvm/Pass.h" +#include "llvm/Passes/PassBuilder.h" #include "llvm/Support/Casting.h" #include "llvm/Support/Debug.h" #include "llvm/Support/PrettyStackTrace.h" @@ -1416,14 +1417,15 @@ } static coro::Shape splitCoroutine(Function &F, - SmallVectorImpl &Clones) { + SmallVectorImpl &Clones, + bool ReuseFrameSlot) { PrettyStackTraceFunction prettyStackTrace(F); // The suspend-crossing algorithm in buildCoroutineFrame get tripped // up by uses in unreachable blocks, so remove them as a first pass. removeUnreachableBlocks(F); - coro::Shape Shape(F); + coro::Shape Shape(F, ReuseFrameSlot); if (!Shape.CoroBegin) return Shape; @@ -1699,7 +1701,7 @@ F.removeFnAttr(CORO_PRESPLIT_ATTR); SmallVector Clones; - const coro::Shape Shape = splitCoroutine(F, Clones); + const coro::Shape Shape = splitCoroutine(F, Clones, ReuseFrameSlot); updateCallGraphAfterCoroutineSplit(*N, Shape, Clones, C, CG, AM, UR, FAM); } @@ -1723,11 +1725,13 @@ struct CoroSplitLegacy : public CallGraphSCCPass { static char ID; // Pass identification, replacement for typeid - CoroSplitLegacy() : CallGraphSCCPass(ID) { + CoroSplitLegacy(bool ReuseFrameSlot = false) + : CallGraphSCCPass(ID), ReuseFrameSlot(ReuseFrameSlot) { initializeCoroSplitLegacyPass(*PassRegistry::getPassRegistry()); } bool Run = false; + bool ReuseFrameSlot; // A coroutine is identified by the presence of coro.begin intrinsic, if // we don't have any, this pass has nothing to do. @@ -1776,7 +1780,7 @@ F->removeFnAttr(CORO_PRESPLIT_ATTR); SmallVector Clones; - const coro::Shape Shape = splitCoroutine(*F, Clones); + const coro::Shape Shape = splitCoroutine(*F, Clones, ReuseFrameSlot); updateCallGraphAfterCoroutineSplit(*F, Shape, Clones, CG, SCC); } @@ -1807,4 +1811,6 @@ "Split coroutine into a set of functions driving its state machine", false, false) -Pass *llvm::createCoroSplitLegacyPass() { return new CoroSplitLegacy(); } +Pass *llvm::createCoroSplitLegacyPass(bool ReuseFrameSlot) { + return new CoroSplitLegacy(ReuseFrameSlot); +} Index: llvm/lib/Transforms/Coroutines/Coroutines.cpp =================================================================== --- llvm/lib/Transforms/Coroutines/Coroutines.cpp +++ llvm/lib/Transforms/Coroutines/Coroutines.cpp @@ -69,7 +69,7 @@ static void addCoroutineSCCPasses(const PassManagerBuilder &Builder, legacy::PassManagerBase &PM) { - PM.add(createCoroSplitLegacyPass()); + PM.add(createCoroSplitLegacyPass(Builder.OptLevel != 0)); } static void addCoroutineOptimizerLastPasses(const PassManagerBuilder &Builder, Index: llvm/test/Transforms/Coroutines/coro-frame-reuse-alloca-00.ll =================================================================== --- /dev/null +++ llvm/test/Transforms/Coroutines/coro-frame-reuse-alloca-00.ll @@ -0,0 +1,79 @@ +; Check that we can handle spills of array allocas +; RUN: opt < %s -coro-split -reuse-storage-in-coroutine-frame -S | FileCheck %s +; RUN: opt < %s -passes=coro-split -reuse-storage-in-coroutine-frame -S | FileCheck %s + +%struct.big_structure = type { [500 x i8] } +declare void @consume(%struct.big_structure*) + +; Function Attrs: noinline optnone uwtable +define i8* @f(i1 %cond) "coroutine.presplit"="1" { +entry: + %data = alloca %struct.big_structure, align 1 + %data2 = alloca %struct.big_structure, align 1 + %id = call token @llvm.coro.id(i32 0, i8* null, i8* null, i8* null) + %size = call i32 @llvm.coro.size.i32() + %alloc = call i8* @malloc(i32 %size) + %hdl = call i8* @llvm.coro.begin(token %id, i8* %alloc) + br i1 %cond, label %then, label %else + +then: + %0 = bitcast %struct.big_structure* %data to i8* + call void @llvm.lifetime.start.p0i8(i64 500, i8* nonnull %0) + call void @consume(%struct.big_structure* %data) + %suspend.value = call i8 @llvm.coro.suspend(token none, i1 false) + switch i8 %suspend.value, label %coro.ret [i8 0, label %resume + i8 1, label %cleanup1] + +resume: + call void @llvm.lifetime.end.p0i8(i64 500, i8* nonnull %0) + br label %cleanup1 + +cleanup1: + call void @llvm.lifetime.end.p0i8(i64 500, i8* nonnull %0) + br label %cleanup + +else: + %1 = bitcast %struct.big_structure* %data2 to i8* + call void @llvm.lifetime.start.p0i8(i64 500, i8* nonnull %1) + call void @consume(%struct.big_structure* %data2) + %suspend.value2 = call i8 @llvm.coro.suspend(token none, i1 false) + switch i8 %suspend.value2, label %coro.ret [i8 0, label %resume2 + i8 1, label %cleanup2] + +resume2: + call void @llvm.lifetime.end.p0i8(i64 500, i8* nonnull %1) + br label %cleanup2 + +cleanup2: + call void @llvm.lifetime.end.p0i8(i64 500, i8* nonnull %1) + br label %cleanup + +cleanup: + %mem = call i8* @llvm.coro.free(token %id, i8* %hdl) + call void @free(i8* %mem) + br label %coro.ret +coro.ret: + call i1 @llvm.coro.end(i8* %hdl, i1 0) + ret i8* %hdl +} + +; CHECK-LABEL: @f( +; CHECK: call i8* @malloc(i32 520) + +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* @malloc(i32) +declare double @print(double) +declare void @free(i8*) + +declare void @llvm.lifetime.start.p0i8(i64, i8* nocapture) +declare void @llvm.lifetime.end.p0i8(i64, i8* nocapture) \ No newline at end of file Index: llvm/test/Transforms/Coroutines/coro-frame-reuse-alloca-01.ll =================================================================== --- /dev/null +++ llvm/test/Transforms/Coroutines/coro-frame-reuse-alloca-01.ll @@ -0,0 +1,77 @@ +; Tests that variables in a Corotuine whose lifetime range is not overlapping each other +; re-use the same slot in Coroutine frame. +; RUN: opt < %s -coro-split -reuse-storage-in-coroutine-frame -S | FileCheck %s +; RUN: opt < %s -passes=coro-split -reuse-storage-in-coroutine-frame -S | FileCheck %s +%"struct.task::promise_type" = type { i8 } +%struct.awaitable = type { i8 } +%struct.big_structure = type { [500 x i8] } +declare i8* @malloc(i64) +declare void @consume(%struct.big_structure*) +define void @a(i1 zeroext %cond) "coroutine.presplit"="1" { +entry: + %__promise = alloca %"struct.task::promise_type", align 1 + %a = alloca %struct.big_structure, align 1 + %ref.tmp7 = alloca %struct.awaitable, align 1 + %b = alloca %struct.big_structure, align 1 + %ref.tmp18 = alloca %struct.awaitable, align 1 + %0 = getelementptr inbounds %"struct.task::promise_type", %"struct.task::promise_type"* %__promise, i64 0, i32 0 + %1 = call token @llvm.coro.id(i32 16, i8* nonnull %0, i8* bitcast (void (i1)* @a to i8*), i8* null) + br label %init.ready +init.ready: + %2 = call noalias nonnull i8* @llvm.coro.begin(token %1, i8* null) + call void @llvm.lifetime.start.p0i8(i64 1, i8* nonnull %0) + br i1 %cond, label %if.then, label %if.else +if.then: + %3 = getelementptr inbounds %struct.big_structure, %struct.big_structure* %a, i64 0, i32 0, i64 0 + call void @llvm.lifetime.start.p0i8(i64 500, i8* nonnull %3) + call void @consume(%struct.big_structure* nonnull %a) + %save = call token @llvm.coro.save(i8* null) + %suspend = call i8 @llvm.coro.suspend(token %save, i1 false) + switch i8 %suspend, label %coro.ret [ + i8 0, label %await.ready + i8 1, label %cleanup1 + ] +await.ready: + call void @llvm.lifetime.end.p0i8(i64 500, i8* nonnull %3) + br label %cleanup1 +if.else: + %4 = getelementptr inbounds %struct.big_structure, %struct.big_structure* %b, i64 0, i32 0, i64 0 + call void @llvm.lifetime.start.p0i8(i64 500, i8* nonnull %4) + call void @consume(%struct.big_structure* nonnull %b) + %save2 = call token @llvm.coro.save(i8* null) + %suspend2 = call i8 @llvm.coro.suspend(token %save2, i1 false) + switch i8 %suspend2, label %coro.ret [ + i8 0, label %await2.ready + i8 1, label %cleanup2 + ] +await2.ready: + call void @llvm.lifetime.end.p0i8(i64 500, i8* nonnull %4) + br label %cleanup2 +cleanup1: + call void @llvm.lifetime.end.p0i8(i64 500, i8* nonnull %3) + br label %cleanup +cleanup2: + call void @llvm.lifetime.end.p0i8(i64 500, i8* nonnull %4) + br label %cleanup +cleanup: + call i8* @llvm.coro.free(token %1, i8* %2) + br label %coro.ret +coro.ret: + call i1 @llvm.coro.end(i8* null, i1 false) + ret void +} +; CHECK-LABEL: @a.resume( +; CHECK: %a.reload.addr{{[0-9]+}} = getelementptr inbounds %a.Frame, %a.Frame* %FramePtr[[APositon:.*]] +; CHECK: %b.reload.addr{{[0-9]+}} = getelementptr inbounds %a.Frame, %a.Frame* %FramePtr[[APositon]] + +declare token @llvm.coro.id(i32, i8* readnone, i8* nocapture readonly, i8*) +declare i1 @llvm.coro.alloc(token) #3 +declare i64 @llvm.coro.size.i64() #5 +declare i8* @llvm.coro.begin(token, i8* writeonly) #3 +declare token @llvm.coro.save(i8*) #3 +declare i8* @llvm.coro.frame() #5 +declare i8 @llvm.coro.suspend(token, i1) #3 +declare i8* @llvm.coro.free(token, i8* nocapture readonly) #2 +declare i1 @llvm.coro.end(i8*, i1) #3 +declare void @llvm.lifetime.start.p0i8(i64, i8* nocapture) #4 +declare void @llvm.lifetime.end.p0i8(i64, i8* nocapture) #4 \ No newline at end of file Index: llvm/test/Transforms/Coroutines/coro-frame-reuse-alloca-02.ll =================================================================== --- /dev/null +++ llvm/test/Transforms/Coroutines/coro-frame-reuse-alloca-02.ll @@ -0,0 +1,78 @@ +; Tests that variables of different type in a Corotuine whose lifetime range is not overlapping each other +; re-use the same slot in Coroutine frame. +; RUN: opt < %s -coro-split -reuse-storage-in-coroutine-frame -S | FileCheck %s +; RUN: opt < %s -passes=coro-split -reuse-storage-in-coroutine-frame -S | FileCheck %s +%"struct.task::promise_type" = type { i8 } +%struct.awaitable = type { i8 } +%struct.big_structure = type { [500 x i8] } +%struct.big_structure.2 = type { [300 x i8] } +declare i8* @malloc(i64) +declare void @consume(%struct.big_structure*) +declare void @consume.2(%struct.big_structure.2*) +define void @a(i1 zeroext %cond) "coroutine.presplit"="1" { +entry: + %__promise = alloca %"struct.task::promise_type", align 1 + %a = alloca %struct.big_structure, align 1 + %ref.tmp7 = alloca %struct.awaitable, align 1 + %b = alloca %struct.big_structure.2, align 1 + %ref.tmp18 = alloca %struct.awaitable, align 1 + %0 = getelementptr inbounds %"struct.task::promise_type", %"struct.task::promise_type"* %__promise, i64 0, i32 0 + %1 = call token @llvm.coro.id(i32 16, i8* nonnull %0, i8* bitcast (void (i1)* @a to i8*), i8* null) + br label %init.ready +init.ready: + %2 = call noalias nonnull i8* @llvm.coro.begin(token %1, i8* null) + call void @llvm.lifetime.start.p0i8(i64 1, i8* nonnull %0) + br i1 %cond, label %if.then, label %if.else +if.then: + %3 = getelementptr inbounds %struct.big_structure, %struct.big_structure* %a, i64 0, i32 0, i64 0 + call void @llvm.lifetime.start.p0i8(i64 500, i8* nonnull %3) + call void @consume(%struct.big_structure* nonnull %a) + %save = call token @llvm.coro.save(i8* null) + %suspend = call i8 @llvm.coro.suspend(token %save, i1 false) + switch i8 %suspend, label %coro.ret [ + i8 0, label %await.ready + i8 1, label %cleanup1 + ] +await.ready: + call void @llvm.lifetime.end.p0i8(i64 500, i8* nonnull %3) + br label %cleanup1 +if.else: + %4 = getelementptr inbounds %struct.big_structure.2, %struct.big_structure.2* %b, i64 0, i32 0, i64 0 + call void @llvm.lifetime.start.p0i8(i64 300, i8* nonnull %4) + call void @consume.2(%struct.big_structure.2* nonnull %b) + %save2 = call token @llvm.coro.save(i8* null) + %suspend2 = call i8 @llvm.coro.suspend(token %save2, i1 false) + switch i8 %suspend2, label %coro.ret [ + i8 0, label %await2.ready + i8 1, label %cleanup2 + ] +await2.ready: + call void @llvm.lifetime.end.p0i8(i64 300, i8* nonnull %4) + br label %cleanup2 +cleanup1: + call void @llvm.lifetime.end.p0i8(i64 500, i8* nonnull %3) + br label %cleanup +cleanup2: + call void @llvm.lifetime.end.p0i8(i64 300, i8* nonnull %4) + br label %cleanup +cleanup: + call i8* @llvm.coro.free(token %1, i8* %2) + br label %coro.ret +coro.ret: + call i1 @llvm.coro.end(i8* null, i1 false) + ret void +} +; CHECK-LABEL: @a.resume( +; CHECK: %b.reload.addr = bitcast %struct.big_structure* %0 to %struct.big_structure.2* + +declare token @llvm.coro.id(i32, i8* readnone, i8* nocapture readonly, i8*) +declare i1 @llvm.coro.alloc(token) #3 +declare i64 @llvm.coro.size.i64() #5 +declare i8* @llvm.coro.begin(token, i8* writeonly) #3 +declare token @llvm.coro.save(i8*) #3 +declare i8* @llvm.coro.frame() #5 +declare i8 @llvm.coro.suspend(token, i1) #3 +declare i8* @llvm.coro.free(token, i8* nocapture readonly) #2 +declare i1 @llvm.coro.end(i8*, i1) #3 +declare void @llvm.lifetime.start.p0i8(i64, i8* nocapture) #4 +declare void @llvm.lifetime.end.p0i8(i64, i8* nocapture) #4 \ No newline at end of file Index: llvm/test/Transforms/Coroutines/coro-frame-reuse-alloca-03.ll =================================================================== --- /dev/null +++ llvm/test/Transforms/Coroutines/coro-frame-reuse-alloca-03.ll @@ -0,0 +1,79 @@ +; Check that we should not reuse alloca sotrage in O0. +; RUN: opt < %s -coro-split -S | FileCheck %s +; RUN: opt < %s -passes=coro-split -S | FileCheck %s + +%struct.big_structure = type { [500 x i8] } +declare void @consume(%struct.big_structure*) + +; Function Attrs: noinline optnone uwtable +define i8* @f(i1 %cond) "coroutine.presplit"="1" { +entry: + %data = alloca %struct.big_structure, align 1 + %data2 = alloca %struct.big_structure, align 1 + %id = call token @llvm.coro.id(i32 0, i8* null, i8* null, i8* null) + %size = call i32 @llvm.coro.size.i32() + %alloc = call i8* @malloc(i32 %size) + %hdl = call i8* @llvm.coro.begin(token %id, i8* %alloc) + br i1 %cond, label %then, label %else + +then: + %0 = bitcast %struct.big_structure* %data to i8* + call void @llvm.lifetime.start.p0i8(i64 500, i8* nonnull %0) + call void @consume(%struct.big_structure* %data) + %suspend.value = call i8 @llvm.coro.suspend(token none, i1 false) + switch i8 %suspend.value, label %coro.ret [i8 0, label %resume + i8 1, label %cleanup1] + +resume: + call void @llvm.lifetime.end.p0i8(i64 500, i8* nonnull %0) + br label %cleanup1 + +cleanup1: + call void @llvm.lifetime.end.p0i8(i64 500, i8* nonnull %0) + br label %cleanup + +else: + %1 = bitcast %struct.big_structure* %data2 to i8* + call void @llvm.lifetime.start.p0i8(i64 500, i8* nonnull %1) + call void @consume(%struct.big_structure* %data2) + %suspend.value2 = call i8 @llvm.coro.suspend(token none, i1 false) + switch i8 %suspend.value2, label %coro.ret [i8 0, label %resume2 + i8 1, label %cleanup2] + +resume2: + call void @llvm.lifetime.end.p0i8(i64 500, i8* nonnull %1) + br label %cleanup2 + +cleanup2: + call void @llvm.lifetime.end.p0i8(i64 500, i8* nonnull %1) + br label %cleanup + +cleanup: + %mem = call i8* @llvm.coro.free(token %id, i8* %hdl) + call void @free(i8* %mem) + br label %coro.ret +coro.ret: + call i1 @llvm.coro.end(i8* %hdl, i1 0) + ret i8* %hdl +} + +; CHECK-LABEL: @f( +; CHECK: call i8* @malloc(i32 1024) + +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* @malloc(i32) +declare double @print(double) +declare void @free(i8*) + +declare void @llvm.lifetime.start.p0i8(i64, i8* nocapture) +declare void @llvm.lifetime.end.p0i8(i64, i8* nocapture) \ No newline at end of file