Merge pull request #893 from swaroop-sridhar/gca

Precise GC: Handle GcAggregates
This commit is contained in:
Swaroop Sridhar 2015-10-19 17:53:30 -07:00
Родитель bac4bd1ee2 90265ac37e
Коммит 34f779e2f6
10 изменённых файлов: 773 добавлений и 354 удалений

Просмотреть файл

@ -19,13 +19,91 @@
#include "gcinfoencoder.h"
#include "jitpch.h"
#include "LLILCJit.h"
#include "llvm/ADT/SmallVector.h"
#include "llvm/ADT/DenseMap.h"
#include "llvm/IR/ValueMap.h"
#include "llvm/CodeGen/MachineFunction.h"
#include "llvm/CodeGen/MachineFunctionPass.h"
#include <sstream>
class GcInfoAllocator;
class GcInfoEncoder;
/// \brief Per Function GcInfo
// This Data-structure records the frame-offsets for certain special
// allocations in each function generated by LLILC. This includes:
// -> Special Symbols: GsCookie, SecurityObject, GenericsContext
// which must be identified to the runtime.
// -> Pinned pointers: Whose location must be reported to the runtime.
// Pinned pointers are reported as untracked
// -> GC Aggregates: Location of all GC-aggregates allocated on
// the stack, also reported as untracked.
//
// The stackmap generated via Statepoints only handles SSA values,
// and therefore does not track pointers contained within GC-aggregates.
// To circumvent this problem, we report all pointers within
// GC-aggregates as untracked. The Gc-aggregate stack allocations are
// known to be live throughout the function, because the reader marks
// them as frame-escaped.
class GcFuncInfo {
public:
GcFuncInfo(const llvm::Function *F);
void recordPinnedSlot(llvm::AllocaInst *Alloca);
void recordGcAggregate(llvm::AllocaInst *Alloca);
void getEscapingLocations(llvm::SmallVector<llvm::Value *, 4> &EscapingLocs);
// Function for which GcInfo is recorded
const llvm::Function *Function;
// Allocation to frame-offset Mappings
llvm::ValueMap<const llvm::AllocaInst *, int32_t> PinnedSlots;
llvm::ValueMap<const llvm::AllocaInst *, int32_t> GcAggregates;
llvm::AllocaInst *GsCookie;
int32_t GsCookieOffset;
uint32_t GsCkValidRangeStart;
uint32_t GsCkValidRangeEnd;
llvm::AllocaInst *SecurityObject;
int32_t SecurityObjectOffset;
llvm::AllocaInst *GenericsContext;
int32_t GenericsContextOffset;
GENERIC_CONTEXTPARAM_TYPE GenericsContextParamType;
};
/// \brief Per Module / Jit Invocation GcInfo
// GcFuncInfo Map for all functions in a Module.
class GcInfo {
public:
static const uint32_t UnmanagedAddressSpace = 0;
static const uint32_t ManagedAddressSpace = 1;
static const int32_t InvalidPointerOffset = -1;
static bool isGcPointer(const llvm::Type *Type);
static bool isGcAggregate(const llvm::Type *Type);
static bool isGcType(const llvm::Type *Type) {
return isGcPointer(Type) || isGcAggregate(Type);
}
static bool isUnmanagedPointer(const llvm::Type *Type) {
return Type->isPointerTy() && !isGcPointer(Type);
}
static bool isGcFunction(const llvm::Function *F);
static bool isFPBasedFunction(const llvm::Function *F);
static void getGcPointers(llvm::StructType *StructTy,
const llvm::DataLayout &DataLayout,
llvm::SmallVector<uint32_t, 4> &GcPtrOffsets);
GcFuncInfo *newGcInfo(const llvm::Function *F);
GcFuncInfo *getGcInfo(const llvm::Function *F);
llvm::ValueMap<const llvm::Function *, GcFuncInfo *> GcInfoMap;
};
/// \brief This is the translator from LLVM's GC StackMaps
/// to CoreCLR's GcInfo encoding.
class GCInfo {
class GcInfoEmitter {
public:
/// Construct a GCInfo object
/// \param JitCtx Context record for the method's jit request.
@ -33,25 +111,33 @@ public:
/// loaded in memory
/// \param Allocator The allocator to be used by GcInfo encoder
/// \param OffsetCorrection FunctionStart - CodeBlockStart difference
GCInfo(LLILCJitContext *JitCtx, uint8_t *StackMapData,
GcInfoAllocator *Allocator, size_t OffsetCorrection);
GcInfoEmitter(LLILCJitContext *JitCtx, uint8_t *StackMapData,
GcInfoAllocator *Allocator, size_t OffsetCorrection);
/// Emit GC Info to the EE using GcInfoEncoder.
void emitGCInfo();
static bool isGCFunction(const llvm::Function &F);
/// Destructor -- delete allocated memory
~GCInfo();
~GcInfoEmitter();
private:
void emitGCInfo(const llvm::Function &F);
void encodeHeader(const llvm::Function &F);
void encodeLiveness(const llvm::Function &F);
void emitGCInfo(const GcFuncInfo *GcFuncInfo);
void encodeHeader(const GcFuncInfo *GcFuncInfo);
void encodeTrackedPointers(const GcFuncInfo *GcFuncInfo);
void encodePinnedPointers(const GcFuncInfo *GcFuncInfo);
void encodeSpecialSlots(const GcFuncInfo *GcFuncInfo);
void encodeGcAggregates(const GcFuncInfo *GcFuncInfo);
void finalizeEncoding();
void emitEncoding();
bool shouldEmitGCInfo(const llvm::Function &F);
bool isStackBaseFramePointer(const llvm::Function &F);
bool needsGCInfo(const llvm::Function *F);
bool needsPointerReporting(const llvm::Function *F);
bool isTrackedSlot(GcSlotId SlotID);
GcSlotId getSlot(int32_t Offset, GcSlotFlags Flags);
GcSlotId getTrackedSlot(int32_t Offset);
GcSlotId getAggregateSlot(int32_t Offset);
GcSlotId getPinnedSlot(int32_t Offset);
const LLILCJitContext *JitContext;
const uint8_t *LLVMStackMapData;
@ -68,8 +154,20 @@ private:
// code like the gc.statepoint_poll() method.
size_t OffsetCorrection;
// Offset to SlotID Map
// Currently, the base pointer for all slots is the current function's SP.
// If this changes, we need to change SlotMap
// from {Offset -> SlotID} mapping
// to {(base, offset) -> SlotID) mapping.
llvm::DenseMap<int32_t, uint32_t> SlotMap;
GcSlotId FirstTrackedSlot;
size_t NumTrackedSlots;
#if !defined(NDEBUG)
bool EmitLogs;
std::ostringstream SlotStream;
std::ostringstream LiveStream;
#endif // !NDEBUG
#if defined(PARTIALLY_INTERRUPTIBLE_GC_SUPPORTED)
@ -79,4 +177,18 @@ private:
#endif // defined(PARTIALLY_INTERRUPTIBLE_GC_SUPPORTED)
};
/// \brief MachineFunctionPass to record frame information
/// for special allocations in GcFuncInfo
class GcInfoRecorder : public llvm::MachineFunctionPass {
public:
explicit GcInfoRecorder(bool TrackGcAggrs) : MachineFunctionPass(ID) {
TrackGcAggregates = TrackGcAggrs;
}
bool runOnMachineFunction(llvm::MachineFunction &MF) override;
private:
static char ID;
bool TrackGcAggregates;
};
#endif // GCINFO_H

Просмотреть файл

@ -26,7 +26,7 @@
#if defined(_TARGET_X86_)
#define REGNUM_FPBASE ICorDebugInfo::RegNum::REGNUM_EBP
#elif (defined(_TARGET_AMD64_) || defined(_TARGET_X64_))
#elif(defined(_TARGET_AMD64_) || defined(_TARGET_X64_))
#define REGNUM_FPBASE ICorDebugInfo::RegNum::REGNUM_RBP
#elif defined(_TARGET_ARM64_)
#define REGNUM_FPBASE ICorDebugInfo::RegNum::REGNUM_FP
@ -55,7 +55,6 @@
#define DW_R14 14
#define DW_R15 15
#define DW_FRAME_POINTER DW_RBP
#define DW_STACK_POINTER DW_RSP
#elif defined(_TARGET_ARM64_)

Просмотреть файл

@ -28,6 +28,7 @@
#include "llvm/Config/config.h"
class ABIInfo;
class GcInfo;
struct LLILCJitPerThreadState;
/// \brief This struct holds per-jit request state.
@ -110,6 +111,9 @@ public:
uintptr_t ReadOnlyDataSize = 0; ///< Size of readonly data ref'd from code.
uintptr_t StackMapSize = 0; ///< Size of readonly Stackmap section.
//@}
/// \name GC Information
GcInfo *GcInfo; ///< GcInfo for functions in CurrentModule
};
/// \brief This struct holds per-thread Jit state.

Просмотреть файл

@ -17,6 +17,8 @@
#ifndef COMPILER_H
#define COMPILER_H
#include "GcInfo.h"
#include "LLILCJit.h"
#include "llvm/ExecutionEngine/ObjectMemoryBuffer.h"
#include "llvm/IR/LegacyPassManager.h"
#include "llvm/MC/MCContext.h"
@ -42,6 +44,9 @@ public:
MCContext *Ctx;
if (TM.addPassesToEmitMC(PM, Ctx, ObjStream))
llvm_unreachable("Target does not support MC emission.");
bool TrackGcAggrs =
LLILCJit::TheJit->getLLILCJitContext()->Options->DoInsertStatepoints;
PM.add(new GcInfoRecorder(TrackGcAggrs));
PM.run(M);
std::unique_ptr<MemoryBuffer> ObjBuffer(
new ObjectMemoryBuffer(std::move(ObjBufferSV)));

Просмотреть файл

@ -308,7 +308,6 @@ protected:
std::vector<IRNode *> Stack;
public:
/// \brief Destructor
virtual ~ReaderStack() {}

Просмотреть файл

@ -27,6 +27,7 @@
#include "llvm/IR/CallSite.h"
#include "llvm/IR/DIBuilder.h"
#include "llvm/IR/DebugInfoMetadata.h"
#include "GcInfo.h"
#include "reader.h"
#include "abi.h"
#include "abisignature.h"
@ -780,6 +781,10 @@ public:
IRNode *genNullCheck(IRNode *Node) override;
llvm::AllocaInst *createAlloca(llvm::Type *T,
llvm::Value *ArraySize = nullptr,
const llvm::Twine &Name = "");
void
createSym(uint32_t Num, bool IsAuto, CorInfoType CorType,
CORINFO_CLASS_HANDLE Class, bool IsPinned,
@ -1288,16 +1293,6 @@ private:
llvm::PointerType *getUnmanagedPointerType(llvm::Type *ElementType);
bool isManagedType(llvm::Type *Type);
bool isManagedPointerType(llvm::Type *Type);
bool isManagedAggregateType(llvm::Type *Type);
/// \brief Check whether Type is an unmanaged pointer type.
///
/// \param Type Type to check.
/// \returns true iff \p Type is an unmanaged pointer type.
bool isUnmanagedPointerType(llvm::Type *Type);
llvm::StoreInst *makeStore(llvm::Value *ValueToStore, llvm::Value *Address,
bool IsVolatile, bool AddressMayBeNull = true);
llvm::StoreInst *makeStoreNonNull(llvm::Value *ValueToStore,
@ -1722,7 +1717,8 @@ private:
ABIInfo *TheABIInfo;
ReaderMethodSignature MethodSignature;
ABIMethodSignature ABIMethodSig;
llvm::Function *Function;
llvm::Function *Function; // The current function being read
GcFuncInfo *GcFuncInfo; // GcInfo for the above function
// The LLVMBuilder has a notion of a current insertion point. During the
// first-pass flow-graph construction, each method sets the insertion point
// explicitly before inserting IR (the fg- methods typically take an
@ -1780,8 +1776,6 @@ private:
///< entered. It is set and checked by monitor
///< helpers.
uint32_t TargetPointerSizeInBits;
const uint32_t UnmanagedAddressSpace = 0;
const uint32_t ManagedAddressSpace = 1;
llvm::Type *BuiltinObjectType; ///< Cached LLVM representation of object.
/// \brief Map from Element types to the LLVM representation of the

Просмотреть файл

@ -15,19 +15,296 @@
#include "earlyincludes.h"
#include "GcInfo.h"
#include "LLILCJit.h"
#include "Target.h"
#include "llvm/ADT/DenseMap.h"
#include "llvm/ADT/SmallBitVector.h"
#include "llvm/Object/StackMapParser.h"
#include <sstream>
#include "llvm/CodeGen/MachineFrameInfo.h"
using namespace llvm;
GCInfo::GCInfo(LLILCJitContext *JitCtx, uint8_t *StackMapData,
GcInfoAllocator *Allocator, size_t OffsetCor)
//-------------------------------GcInfo------------------------------------------
bool GcInfo::isGcPointer(const Type *Type) {
const PointerType *PtrType = dyn_cast<PointerType>(Type);
if (PtrType != nullptr) {
return PtrType->getAddressSpace() == GcInfo::ManagedAddressSpace;
}
return false;
}
bool GcInfo::isGcAggregate(const Type *AggType) {
const VectorType *VecType = dyn_cast<VectorType>(AggType);
if (VecType != nullptr) {
return isGcPointer(VecType->getScalarType());
}
const ArrayType *ArrType = dyn_cast<ArrayType>(AggType);
if (ArrType != nullptr) {
return isGcPointer(ArrType->getElementType());
}
const StructType *StType = dyn_cast<StructType>(AggType);
if (StType != nullptr) {
for (Type *SubType : StType->subtypes()) {
if (isGcType(SubType)) {
return true;
}
}
}
return false;
}
bool GcInfo::isGcFunction(const llvm::Function *F) {
if (!F->hasGC()) {
return false;
}
const StringRef CoreCLRName("coreclr");
return (CoreCLRName == F->getGC());
}
bool GcInfo::isFPBasedFunction(const Function *F) {
Attribute Attribute = F->getFnAttribute("no-frame-pointer-elim");
return (Attribute.getValueAsString() == "true");
}
void GcInfo::getGcPointers(StructType *StructTy, const DataLayout &DataLayout,
SmallVector<uint32_t, 4> &Pointers) {
assert(StructTy->isSized());
const uint32_t PointerSize = DataLayout.getPointerSize();
const uint32_t TypeSize = DataLayout.getTypeStoreSize(StructTy);
const StructLayout *MainStructLayout = DataLayout.getStructLayout(StructTy);
// Walk through the type in pointer-sized jumps.
for (uint32_t GcOffset = 0; GcOffset < TypeSize; GcOffset += PointerSize) {
const uint32_t FieldIndex =
MainStructLayout->getElementContainingOffset(GcOffset);
Type *FieldTy = StructTy->getStructElementType(FieldIndex);
// If the field is a value class we need to dive in
// to its fields and so on, until we reach a primitive type.
if (FieldTy->isStructTy()) {
// Prepare to loop through the nesting.
const StructLayout *OuterStructLayout = MainStructLayout;
uint32_t OuterOffset = GcOffset;
uint32_t OuterIndex = FieldIndex;
while (FieldTy->isStructTy()) {
// Offset of the Inner class within the outer class
const uint32_t InnerBaseOffset =
OuterStructLayout->getElementOffset(OuterIndex);
// Inner class should start at or before the outer offset
assert(InnerBaseOffset <= OuterOffset);
// Determine target offset relative to this inner class.
const uint32_t InnerOffset = OuterOffset - InnerBaseOffset;
// Get the inner class layout
StructType *InnerStructTy = cast<StructType>(FieldTy);
const StructLayout *InnerStructLayout =
DataLayout.getStructLayout(InnerStructTy);
// Find the field at that target offset.
const uint32_t InnerIndex =
InnerStructLayout->getElementContainingOffset(InnerOffset);
// Update for next iteration.
FieldTy = InnerStructTy->getStructElementType(InnerIndex);
OuterStructLayout = InnerStructLayout;
OuterOffset = InnerOffset;
OuterIndex = InnerIndex;
}
}
if (GcInfo::isGcPointer(FieldTy)) {
Pointers.push_back(GcOffset);
}
}
}
GcFuncInfo *GcInfo::newGcInfo(const llvm::Function *F) {
assert(getGcInfo(F) == nullptr && "Duplicate GcInfo");
GcFuncInfo *GcFInfo = new GcFuncInfo(F);
GcInfoMap[F] = GcFInfo;
return GcFInfo;
}
GcFuncInfo *GcInfo::getGcInfo(const llvm::Function *F) {
auto Iterator = GcInfoMap.find(F);
if (Iterator == GcInfoMap.end()) {
return nullptr;
}
GcFuncInfo *GcFInfo = Iterator->second;
assert(F == GcFInfo->Function && "Function mismatch");
return GcFInfo;
}
//-------------------------------GcFuncInfo------------------------------------------
GcFuncInfo::GcFuncInfo(const llvm::Function *F)
: GsCookie(nullptr), SecurityObject(nullptr), GenericsContext(nullptr) {
Function = F;
GsCookieOffset = GcInfo::InvalidPointerOffset;
SecurityObjectOffset = GcInfo::InvalidPointerOffset;
GenericsContextOffset = GcInfo::InvalidPointerOffset;
GsCkValidRangeStart = 0;
GsCkValidRangeEnd = 0;
GenericsContextParamType = GENERIC_CONTEXTPARAM_NONE;
}
void GcFuncInfo::recordPinnedSlot(AllocaInst *Alloca) {
assert(PinnedSlots.find(Alloca) == PinnedSlots.end());
PinnedSlots[Alloca] = GcInfo::InvalidPointerOffset;
}
void GcFuncInfo::recordGcAggregate(AllocaInst *Alloca) {
assert(GcAggregates.find(Alloca) == GcAggregates.end());
GcAggregates[Alloca] = GcInfo::InvalidPointerOffset;
}
void GcFuncInfo::getEscapingLocations(SmallVector<Value *, 4> &EscapingLocs) {
if (GsCookie != nullptr) {
EscapingLocs.push_back(GsCookie);
}
if (SecurityObject != nullptr) {
EscapingLocs.push_back(SecurityObject);
}
if (GenericsContext != nullptr) {
EscapingLocs.push_back(GenericsContext);
}
for (auto Pin : PinnedSlots) {
EscapingLocs.push_back((Value *)Pin.first);
}
for (auto GcAggregate : GcAggregates) {
EscapingLocs.push_back((Value *)GcAggregate.first);
}
}
//-------------------------------GcInfoRecorder-----------------------------------
char GcInfoRecorder::ID = 0;
bool GcInfoRecorder::runOnMachineFunction(MachineFunction &MF) {
const Function *F = MF.getFunction();
if (!GcInfo::isGcFunction(F)) {
return false;
}
LLILCJitContext *Context = LLILCJit::TheJit->getLLILCJitContext();
GcFuncInfo *GcFuncInfo = Context->GcInfo->getGcInfo(F);
ValueMap<const AllocaInst *, int32_t> &GcAggregates =
GcFuncInfo->GcAggregates;
ValueMap<const AllocaInst *, int32_t> &PinnedSlots = GcFuncInfo->PinnedSlots;
#if !defined(NDEBUG)
bool EmitLogs = Context->Options->LogGcInfo;
if (EmitLogs) {
dbgs() << "GcInfoRecorder: " << MF.getFunction()->getName() << "\n";
}
#endif // !NDEBUG
const MachineFrameInfo *FrameInfo = MF.getFrameInfo();
int ObjectIndexBegin = FrameInfo->getObjectIndexBegin();
int ObjectIndexEnd = FrameInfo->getObjectIndexEnd();
// FrameInfo reports the allocation offsets in terms of the
// incoming (caller's) StackPointer. Convert these in terms of the
// current (callee's) StackPointer.
uint64_t StackPointerSize = MF.getDataLayout().getPointerSize();
uint64_t SpOffset = FrameInfo->getStackSize() + StackPointerSize;
for (int Idx = ObjectIndexBegin; Idx < ObjectIndexEnd; Idx++) {
const AllocaInst *Alloca = FrameInfo->getObjectAllocation(Idx);
if (Alloca == nullptr) {
continue;
}
int32_t SlotOffset = SpOffset + FrameInfo->getObjectOffset(Idx);
if (GcFuncInfo->GsCookie == Alloca) {
GcFuncInfo->GsCookieOffset = SlotOffset;
#if !defined(NDEBUG)
if (EmitLogs) {
dbgs() << "GSCookie: @" << SlotOffset << "\n";
}
#endif // !NDEBUG
} else if (GcFuncInfo->SecurityObject == Alloca) {
GcFuncInfo->SecurityObjectOffset = SlotOffset;
#if !defined(NDEBUG)
if (EmitLogs) {
dbgs() << "SecurityObjectOffset: @" << SlotOffset << "\n";
}
#endif // !NDEBUG
} else if (GcFuncInfo->GenericsContext == Alloca) {
GcFuncInfo->GenericsContextOffset = SlotOffset;
#if !defined(NDEBUG)
if (EmitLogs) {
dbgs() << "GenericsContext: @" << SlotOffset << "\n";
}
#endif // !NDEBUG
} else if (PinnedSlots.find(Alloca) != PinnedSlots.end()) {
assert(PinnedSlots[Alloca] == GcInfo::InvalidPointerOffset &&
"Two allocations for the same pointer!");
PinnedSlots[Alloca] = SlotOffset;
#if !defined(NDEBUG)
if (EmitLogs) {
dbgs() << "Pinned Pointer: @" << SlotOffset << "\n";
}
#endif // !NDEBUG
}
if (TrackGcAggregates) {
Type *AllocatedType = Alloca->getAllocatedType();
if (GcInfo::isGcAggregate(AllocatedType)) {
assert(isa<StructType>(AllocatedType) && "Unexpected GcAggregate");
assert(GcAggregates.find(Alloca) != GcAggregates.end());
assert(GcAggregates[Alloca] == GcInfo::InvalidPointerOffset &&
"Two allocations for the same aggregate!");
GcAggregates[Alloca] = SlotOffset;
#if !defined(NDEBUG)
if (EmitLogs) {
dbgs() << "GC Aggregate: @" << SlotOffset << "\n";
}
#endif // !NDEBUG
}
}
}
#if !defined(NDEBUG)
if (EmitLogs) {
dbgs() << "\n";
}
#endif // !NDEBUG
return false; // success
}
//-------------------------------GcInfoEmitter-----------------------------------
GcInfoEmitter::GcInfoEmitter(LLILCJitContext *JitCtx, uint8_t *StackMapData,
GcInfoAllocator *Allocator, size_t OffsetCor)
: JitContext(JitCtx), LLVMStackMapData(StackMapData),
Encoder(JitContext->JitInfo, JitContext->MethodInfo, Allocator),
OffsetCorrection(OffsetCor) {
OffsetCorrection(OffsetCor), SlotMap(), FirstTrackedSlot(0),
NumTrackedSlots(0) {
#if !defined(NDEBUG)
this->EmitLogs = JitContext->Options->LogGcInfo;
#endif // !NDEBUG
@ -37,10 +314,12 @@ GCInfo::GCInfo(LLILCJitContext *JitCtx, uint8_t *StackMapData,
#endif // defined(PARTIALLY_INTERRUPTIBLE_GC_SUPPORTED)
}
void GCInfo::encodeHeader(const Function &F) {
void GcInfoEmitter::encodeHeader(const GcFuncInfo *GcFuncInfo) {
const Function *F = GcFuncInfo->Function;
#if !defined(NDEBUG)
if (EmitLogs) {
dbgs() << "GcTable for Function: " << F.getName() << "\n";
dbgs() << "GcTable for Function: " << F->getName() << "\n";
}
#endif // !NDEBUG
@ -55,7 +334,7 @@ void GCInfo::encodeHeader(const Function &F) {
}
#endif // !NDEBUG
if (isStackBaseFramePointer(F)) {
if (GcInfo::isFPBasedFunction(F)) {
Encoder.SetStackBaseRegister(REGNUM_FPBASE);
#if !defined(NDEBUG)
if (EmitLogs) {
@ -70,6 +349,10 @@ void GCInfo::encodeHeader(const Function &F) {
#endif // !NDEBUG
}
if (GcFuncInfo->Function->isVarArg()) {
Encoder.SetIsVarArg();
}
#if defined(FIXED_STACK_PARAMETER_SCRATCH_AREA)
// TODO: Set size of outgoing/scratch area accurately
// https://github.com/dotnet/llilc/issues/681
@ -83,11 +366,7 @@ void GCInfo::encodeHeader(const Function &F) {
#endif // defined(FIXED_STACK_PARAMETER_SCRATCH_AREA)
}
void GCInfo::encodeLiveness(const Function &F) {
if (LLVMStackMapData == nullptr) {
return;
}
void GcInfoEmitter::encodeTrackedPointers(const GcFuncInfo *GcFuncInfo) {
ArrayRef<uint8_t> StackMapContentsArray(LLVMStackMapData,
JitContext->StackMapSize);
@ -139,33 +418,21 @@ void GCInfo::encodeLiveness(const Function &F) {
const uint8_t CallSiteSize = 2;
DenseMap<int32_t, GcSlotId> SlotMap;
size_t NumSlots = 0;
// LLVM StackMap records all live-pointers per Safepoint, whereas
// CoreCLR's GCTables record pointer birth/deaths per Safepoint.
// So, we do the translation using old/new live-pointer-sets
// using bit-sets for recording the liveness -- one bit per slot.
//
// Pinned locations must be allocated before tracked ones, so
// that the slots are correctly marked as Pinned and Untracked.
// Since Pinned pointers are rare, we let go of the first few
// bits in the LiveSet, instead of complicating the logic in
// this method with offset calculations.
size_t LiveBitSetSize = 25;
SmallBitVector OldLiveSet(LiveBitSetSize);
SmallBitVector NewLiveSet(LiveBitSetSize);
// TODO: Identify Object and Managed pointers differently
// https://github.com/dotnet/llilc/issues/28
// We currently conservatively describe all slots as containing
// interior pointers
const GcSlotFlags SlotFlags = (GcSlotFlags)GC_SLOT_INTERIOR;
#if !defined(NDEBUG)
if (EmitLogs) {
dbgs() << " #Safepoints: " << StackMapParser.getNumRecords() << "\n";
}
std::ostringstream SlotStream;
std::ostringstream LiveStream;
#endif // !NDEBUG
size_t RecordIndex = 0;
for (const auto &R : StackMapParser.records()) {
@ -192,10 +459,6 @@ void GCInfo::encodeLiveness(const Function &F) {
}
#endif // !NDEBUG
// We don't generate GC_CALLER_SP_REL locatons, just
// using this as a default value other than SP/FP REL.
GcStackSlotBase SpBase = GC_CALLER_SP_REL;
for (const auto &Loc : R.locations()) {
switch (Loc.getKind()) {
@ -211,32 +474,19 @@ void GCInfo::encodeLiveness(const Function &F) {
break;
case StackMapParserType::LocationKind::Direct: {
uint16_t DwReg = Loc.getDwarfRegNum();
switch (DwReg) {
case DW_FRAME_POINTER:
assert(SpBase != GC_SP_REL && "Mixed SP/FP based Locations");
SpBase = GC_FRAMEREG_REL;
break;
case DW_STACK_POINTER:
assert(SpBase != GC_FRAMEREG_REL && "Mixed SP/FP based Locations");
SpBase = GC_SP_REL;
break;
default:
assert(false && "Unexpected stack base-pointer");
}
// __LLVM_Stackmap reports the liveness of pointers wrt SP even for
// methods which have a FP.
assert(Loc.getDwarfRegNum() == DW_STACK_POINTER &&
"Expect Stack Pointer to be the base");
GcSlotId SlotID;
int32_t Offset = Loc.getOffset();
DenseMap<int32_t, GcSlotId>::const_iterator ExistingSlot =
SlotMap.find(Offset);
if (ExistingSlot == SlotMap.end()) {
SlotID = Encoder.GetStackSlotId(Offset, SlotFlags, SpBase);
SlotMap[Offset] = SlotID;
SlotID = getTrackedSlot(Offset);
assert(SlotID == NumSlots && "SlotIDs dis-contiguous");
NumSlots++;
if (NumSlots > LiveBitSetSize) {
if (SlotMap.size() > LiveBitSetSize) {
LiveBitSetSize += LiveBitSetSize;
assert(LiveBitSetSize > OldLiveSet.size() &&
@ -245,19 +495,16 @@ void GCInfo::encodeLiveness(const Function &F) {
OldLiveSet.resize(LiveBitSetSize);
NewLiveSet.resize(LiveBitSetSize);
}
#if !defined(NDEBUG)
if (EmitLogs) {
SlotStream << " [" << SlotID
<< "]: " << ((SpBase == GC_SP_REL) ? "sp+" : "fp+")
<< Offset << "\n";
}
#endif // !NDEBUG
} else {
SlotID = ExistingSlot->second;
}
NewLiveSet[SlotID] = true;
// No need to report liveness if a slot is untracked.
// This may be true of pinned pointers, since statepoint's
// liveness tracking includes all managed pointers.
if (isTrackedSlot(SlotID)) {
NewLiveSet[SlotID] = true;
}
break;
}
@ -267,12 +514,7 @@ void GCInfo::encodeLiveness(const Function &F) {
}
}
// __LLVM_Stackmap reports the liveness of pointers wrt SP even for
// certain methods which have a FP. So, we cannot
// assert((SpBase == GC_FRAMEREG_REL) ?
// isStackBaseFramePointer(F) : !isStackBaseFramePointer(F));
for (GcSlotId SlotID = 0; SlotID < NumSlots; SlotID++) {
for (GcSlotId SlotID = 0; SlotID < SlotMap.size(); SlotID++) {
if (!OldLiveSet[SlotID] && NewLiveSet[SlotID]) {
#if !defined(NDEBUG)
if (EmitLogs) {
@ -301,6 +543,106 @@ void GCInfo::encodeLiveness(const Function &F) {
}
#endif // !NDEBUG
}
}
void GcInfoEmitter::encodePinnedPointers(const GcFuncInfo *GcFuncInfo) {
assert(SlotMap.size() == 0 && "Expect to allocate Pinned Slots first");
for (auto Pin : GcFuncInfo->PinnedSlots) {
int32_t Offset = Pin.second;
assert(Offset != GcInfo::InvalidPointerOffset && "Pinned Slot Not Found!");
assert(SlotMap.find(Offset) == SlotMap.end() &&
"Pinned slot already allocated");
getPinnedSlot(Offset);
}
}
void GcInfoEmitter::encodeSpecialSlots(const GcFuncInfo *GcFuncInfo) {
// The Followig information is not yet reported to the Runtime
// for the want of additional information.
// -> Prolog Size
// -> GS Cookie validity range
if (GcFuncInfo->SecurityObject != nullptr) {
// TODO: GC: Report SecurityObjectStackSlot #767
// Encoder.SetSecurityObjectStackSlot(GcFuncInfo->SecurityObjectOffset);
#if !defined(NDEBUG)
if (EmitLogs) {
dbgs() << " SecurityObjectStackSlot @"
<< GcFuncInfo->SecurityObjectOffset << " Not Reported\n";
}
#endif // !NDEBUG
}
if (GcFuncInfo->GsCookie != nullptr) {
// TODO: GC: Report GS Cookie #768
// Encoder.SetGSCookieStackSlot(GcFuncInfo->GsCookieOffset,
// GcFuncInfo->GsCkValidRangeStart,
// GcFuncInfo->GsCkValidRangeEnd);
#if !defined(NDEBUG)
if (EmitLogs) {
dbgs() << " GSCookieStackSlot @" << GcFuncInfo->GsCookieOffset << " ["
<< GcFuncInfo->GsCkValidRangeStart << " - "
<< GcFuncInfo->GsCkValidRangeEnd << "] Not Reported\n";
}
#endif // !NDEBUG
}
if (GcFuncInfo->GenericsContext != nullptr) {
// TODO: GC: Report GenericsInstContextStackSlot #766
// Encoder.SetGenericsInstContextStackSlot(
// GcFuncInfo->GenericsContextOffset,
// GcFuncInfo->GenericsContextParamType);
#if !defined(NDEBUG)
if (EmitLogs) {
dbgs() << " GenericsInstContextStackSlot @"
<< GcFuncInfo->GenericsContextOffset << " ["
<< GcFuncInfo->GenericsContextParamType << "] Not Reported\n";
}
#endif // !NDEBUG
}
}
void GcInfoEmitter::encodeGcAggregates(const GcFuncInfo *GcFuncInfo) {
for (auto Aggregate : GcFuncInfo->GcAggregates) {
const AllocaInst *Alloca = Aggregate.first;
Type *Type = Alloca->getAllocatedType();
assert(isa<StructType>(Type) && "GcAggregate is not a struct");
StructType *StructTy = cast<StructType>(Type);
int32_t AggregateOffset = Aggregate.second;
assert(AggregateOffset != GcInfo::InvalidPointerOffset &&
"GcAggregate Not Found!");
SmallVector<uint32_t, 4> GcPtrOffsets;
const DataLayout &DataLayout = JitContext->CurrentModule->getDataLayout();
GcInfo::getGcPointers(StructTy, DataLayout, GcPtrOffsets);
assert(GcPtrOffsets.size() > 0 && "GC Aggregate without GC pointers!");
for (uint32_t GcPtrOffset : GcPtrOffsets) {
getAggregateSlot(AggregateOffset + GcPtrOffset);
}
}
}
void GcInfoEmitter::finalizeEncoding() {
// Finalize Slot IDs to enable compact representation
Encoder.FinalizeSlotIds();
#if defined(PARTIALLY_INTERRUPTIBLE_GC_SUPPORTED)
// Encode Call-sites
assert(CallSiteSizes != nullptr);
assert(CallSites != nullptr);
assert(NumCallSites > 0);
Encoder.DefineCallSites(CallSites, CallSiteSizes, NumCallSites);
#endif // defined(PARTIALLY_INTERRUPTIBLE_GC_SUPPORTED)
}
void GcInfoEmitter::emitEncoding() {
Encoder.Build();
Encoder.Emit();
#if !defined(NDEBUG)
if (EmitLogs) {
@ -308,62 +650,128 @@ void GCInfo::encodeLiveness(const Function &F) {
dbgs() << " Safepoints:\n" << LiveStream.str() << "\n";
}
#endif // !NDEBUG
// Finalize Slot IDs to enable compact representation
Encoder.FinalizeSlotIds();
#if defined(PARTIALLY_INTERRUPTIBLE_GC_SUPPORTED)
// Encode Call-sites
Encoder.DefineCallSites(CallSites, CallSiteSizes, NumCallSites);
#endif // defined(PARTIALLY_INTERRUPTIBLE_GC_SUPPORTED)
}
void GCInfo::emitEncoding() {
Encoder.Build();
Encoder.Emit();
}
GCInfo::~GCInfo() {
GcInfoEmitter::~GcInfoEmitter() {
#if defined(PARTIALLY_INTERRUPTIBLE_GC_SUPPORTED)
delete CallSites;
delete CallSiteSizes;
#endif // defined(PARTIALLY_INTERRUPTIBLE_GC_SUPPORTED)
}
void GCInfo::emitGCInfo(const Function &F) {
encodeHeader(F);
encodeLiveness(F);
void GcInfoEmitter::emitGCInfo(const GcFuncInfo *GcFuncInfo) {
assert((GcFuncInfo != nullptr) && "Function missing GcInfo");
encodeHeader(GcFuncInfo);
encodeSpecialSlots(GcFuncInfo);
if (needsPointerReporting(GcFuncInfo->Function)) {
// Pinned slots must be allocated before Tracked Slots.
// Pinned pointers are included in the set of pointers tracked
// within statepoints. Untracked reporting is prefered for
// Pinned pointers. So, allocate them first before allocating
// Tracked pointer slots.
encodePinnedPointers(GcFuncInfo);
// Assign Slots for Tracked pointers and report their liveness
encodeTrackedPointers(GcFuncInfo);
// Aggregate slots should be allocated after Live Slots
// There is no overlap between slots created by encodeGcAggregates()
// and encodeTrackedPointers(). encoding GcAggregates last
// helps save some bits in the data-structures used by
// encodeTrackedPointers()
encodeGcAggregates(GcFuncInfo);
// Finalization must be done after all encodings
finalizeEncoding();
}
emitEncoding();
}
void GCInfo::emitGCInfo() {
const Module::FunctionListType &FunctionList =
JitContext->CurrentModule->getFunctionList();
Module::const_iterator Iterator = FunctionList.begin();
Module::const_iterator End = FunctionList.end();
for (; Iterator != End; ++Iterator) {
const Function &F = *Iterator;
if (shouldEmitGCInfo(F)) {
emitGCInfo(F);
void GcInfoEmitter::emitGCInfo() {
for (auto GcInfoIterator : JitContext->GcInfo->GcInfoMap) {
GcFuncInfo *GcFuncInfo = GcInfoIterator->second;
if (needsGCInfo(GcFuncInfo->Function)) {
emitGCInfo(GcFuncInfo);
}
}
}
bool GCInfo::shouldEmitGCInfo(const Function &F) {
return !F.isDeclaration() && isGCFunction(F);
bool GcInfoEmitter::needsGCInfo(const Function *F) {
return !F->isDeclaration() && GcInfo::isGcFunction(F);
}
bool GCInfo::isGCFunction(const llvm::Function &F) {
if (!F.hasGC()) {
return false;
bool GcInfoEmitter::needsPointerReporting(const Function *F) {
bool TargetPreciseGcRuntime = JitContext->Options->DoInsertStatepoints;
bool HasGcSafePoints = (LLVMStackMapData != nullptr);
return TargetPreciseGcRuntime && HasGcSafePoints;
}
bool GcInfoEmitter::isTrackedSlot(GcSlotId SlotID) {
// This function requires that all tracked slots be allocated
// contiguously. If this property doesn't hold, SlotMap should
// be changed:
// from Offset -> SlotID map
// to Offset -> {SlotId, SlotFlags, SpBase} map
return (NumTrackedSlots > 0) && (SlotID >= FirstTrackedSlot) &&
(SlotID < FirstTrackedSlot + NumTrackedSlots);
}
GcSlotId GcInfoEmitter::getSlot(int32_t Offset, GcSlotFlags Flags) {
GcSlotId SlotID = Encoder.GetStackSlotId(Offset, Flags, GC_SP_REL);
SlotMap[Offset] = SlotID;
assert(SlotID == (SlotMap.size() - 1) && "SlotIDs dis-contiguous");
#if !defined(NDEBUG)
if (EmitLogs) {
SlotStream << " [" << SlotID << "]: "
<< "sp+" << Offset << " (" << ((Flags & GC_SLOT_BASE) ? "O" : "")
<< ((Flags & GC_SLOT_INTERIOR) ? "M" : "")
<< ((Flags & GC_SLOT_UNTRACKED) ? "U" : "")
<< ((Flags & GC_SLOT_PINNED) ? "P" : "") << ")\n";
}
#endif // !NDEBUG
return SlotID;
}
GcSlotId GcInfoEmitter::getTrackedSlot(int32_t Offset) {
// TODO: Identify Object and Managed pointers differently
// https://github.com/dotnet/llilc/issues/28
// We currently conservatively describe all slots as containing
// interior pointers
const GcSlotFlags ManagedPointerFlags = (GcSlotFlags)GC_SLOT_INTERIOR;
GcSlotId SlotID = getSlot(Offset, ManagedPointerFlags);
NumTrackedSlots++;
if (NumTrackedSlots == 1) {
FirstTrackedSlot = SlotID;
}
const StringRef CoreCLRName("coreclr");
return (CoreCLRName == F.getGC());
return SlotID;
}
bool GCInfo::isStackBaseFramePointer(const llvm::Function &F) {
Attribute Attribute = F.getFnAttribute("no-frame-pointer-elim");
return (Attribute.getValueAsString() == "true");
GcSlotId GcInfoEmitter::getAggregateSlot(int32_t Offset) {
assert(SlotMap.find(Offset) == SlotMap.end() &&
"GC Aggregate slot already allocated");
const GcSlotFlags UntrackedFlags =
(GcSlotFlags)(GC_SLOT_BASE | GC_SLOT_UNTRACKED);
GcSlotId SlotID = getSlot(Offset, UntrackedFlags);
return SlotID;
}
GcSlotId GcInfoEmitter::getPinnedSlot(int32_t Offset) {
// Only Object pointers can be pinned/
// Pinned slots are reported untracked, since they are frame-escaped
// and live throughout the function.
const GcSlotFlags PinnedFlags =
(GcSlotFlags)(GC_SLOT_BASE | GC_SLOT_PINNED | GC_SLOT_UNTRACKED);
return getSlot(Offset, PinnedFlags);
}

Просмотреть файл

@ -253,6 +253,7 @@ CorJitResult LLILCJit::compileMethod(ICorJitInfo *JitInfo,
DEBUG_METADATA_VERSION);
Context.MethodName = Context.CurrentModule->getModuleIdentifier();
Context.TheABIInfo = ABIInfo::get(*Context.CurrentModule);
Context.GcInfo = new GcInfo();
// Initialize per invocation JIT options. This should be done after the
// rest of the Context is filled out as it has dependencies on JitInfo,
@ -347,12 +348,6 @@ CorJitResult LLILCJit::compileMethod(ICorJitInfo *JitInfo,
// and lowering passes before generating code.
legacy::PassManager Passes;
Passes.add(createPlaceSafepointsPass());
PassManagerBuilder PMBuilder;
PMBuilder.OptLevel = 0; // Set optimization level to -O0
PMBuilder.SizeLevel = 0; // so that no additional phases are run.
PMBuilder.populateModulePassManager(Passes);
Passes.add(createRewriteStatepointsForGCPass());
Passes.run(*M);
}
@ -380,9 +375,10 @@ CorJitResult LLILCJit::compileMethod(ICorJitInfo *JitInfo,
assert(*NativeEntry >= MM.getHotCodeBlock());
GcInfoAllocator GcInfoAllocator;
GCInfo GcInfo(&Context, MM.getStackMapSection(), &GcInfoAllocator,
*NativeEntry - MM.getHotCodeBlock());
GcInfo.emitGCInfo();
GcInfoEmitter GcInfoEmitter(&Context, MM.getStackMapSection(),
&GcInfoAllocator,
*NativeEntry - MM.getHotCodeBlock());
GcInfoEmitter.emitGCInfo();
// Dump out any enabled timing info.
TimerGroup::printAll(errs());
@ -407,7 +403,9 @@ CorJitResult LLILCJit::compileMethod(ICorJitInfo *JitInfo,
// Clean up a bit more
delete Context.TheABIInfo;
delete Context.GcInfo;
Context.TheABIInfo = nullptr;
Context.GcInfo = nullptr;
return Result;
}

Просмотреть файл

@ -600,6 +600,7 @@ int ReaderBase::appendClassName(char16_t **Buffer, int32_t *BufferLen,
return Return;
}
// Construct The GC Layout from CoreCLR Type
GCLayout *ReaderBase::getClassGCLayout(CORINFO_CLASS_HANDLE Class) {
// The actual size of the byte array the runtime is expecting (gcLayoutSize)
// is one byte for every sizeof(void*) slot in the valueclass.

Просмотреть файл

@ -15,7 +15,6 @@
#include "earlyincludes.h"
#include "readerir.h"
#include "GcInfo.h"
#include "imeta.h"
#include "newvstate.h"
#include "llvm/ADT/Triple.h"
@ -292,6 +291,7 @@ void GenIR::readerPrePass(uint8_t *Buffer, uint32_t NumBytes) {
new (&ABIMethodSig)
ABIMethodSignature(MethodSignature, *this, *JitContext->TheABIInfo);
Function = ABIMethodSig.createFunction(*this, *JitContext->CurrentModule);
GcFuncInfo = JitContext->GcInfo->newGcInfo(Function);
llvm::LLVMContext &LLVMContext = *JitContext->LLVMContext;
EntryBlock = BasicBlock::Create(LLVMContext, "entry", Function);
@ -470,37 +470,20 @@ void GenIR::readerPostVisit() {
}
void GenIR::readerPostPass(bool IsImportOnly) {
if (JitContext->Options->DoInsertStatepoints) {
// Precise GC using statepoints cannot handle aggregates that contain
// managed pointers yet. So, check if this function deals with such values
// and fail early. (Issue #33)
SmallVector<Value *, 4> EscapingLocs;
GcFuncInfo->getEscapingLocations(EscapingLocs);
for (const Argument &Arg : Function->args()) {
if (isManagedAggregateType(Arg.getType())) {
throw NotYetImplementedException(
"NYI: Precice GC for Managed-Aggregate values");
}
}
if (EscapingLocs.size() > 0) {
Value *FrameEscape = Intrinsic::getDeclaration(JitContext->CurrentModule,
Intrinsic::localescape);
for (const BasicBlock &BB : *Function) {
for (const Instruction &Instr : BB) {
if (isManagedAggregateType(Instr.getType())) {
throw NotYetImplementedException(
"NYI: Precice GC for Managed-Aggregate values");
}
if (isa<AllocaInst>(Instr)) {
AllocaInst *Alloca = (AllocaInst *)(&Instr);
Type *AllocatedType = Alloca->getAllocatedType();
if (isManagedAggregateType(AllocatedType)) {
throw NotYetImplementedException(
"NYI: Precice GC for Managed-Aggregate values");
}
}
}
}
// Insert the LocalEscape Intrinsic at the end of the
// Prolog block, after all local allocations,
// before the basic-block Terminator
Instruction *InsertionPoint = Function->begin()->getTerminator();
LLVMBuilder->SetInsertPoint(InsertionPoint);
LLVMBuilder->CreateCall(FrameEscape, EscapingLocs);
}
// Crossgen in ReadyToRun mode records structs that method compilation
@ -539,38 +522,31 @@ void GenIR::insertIRToKeepGenericContextAlive() {
InsertPoint = makeStoreNonNull(This, ScratchLocalAlloca, false);
// The scratch local's address is the saved context address.
ContextLocalAddress = ScratchLocalAlloca;
GcFuncInfo->GenericsContextParamType = GENERIC_CONTEXTPARAM_THIS;
} else {
// We know the type arg is unmodified so we can use its initial
// spilled value location for reporting.
ASSERT(Options & (CORINFO_GENERICS_CTXT_FROM_METHODDESC |
CORINFO_GENERICS_CTXT_FROM_METHODTABLE));
if (Options & CORINFO_GENERICS_CTXT_FROM_METHODDESC) {
GcFuncInfo->GenericsContextParamType = GENERIC_CONTEXTPARAM_MD;
} else if (Options & CORINFO_GENERICS_CTXT_FROM_METHODTABLE) {
GcFuncInfo->GenericsContextParamType = GENERIC_CONTEXTPARAM_MT;
} else {
assert(false && "Unexpected Option");
}
ASSERT(MethodSignature.hasTypeArg());
ContextLocalAddress = Arguments[MethodSignature.getTypeArgIndex()];
}
// Indicate that the context location's address escapes by inserting a call
// to llvm.frameescape. Put that call just after the last alloca or the
// store to the scratch local.
LLVMBuilder->SetInsertPoint(InsertPoint->getNextNode());
Value *FrameEscape = Intrinsic::getDeclaration(JitContext->CurrentModule,
Intrinsic::localescape);
Value *Args[] = {ContextLocalAddress};
const bool MayThrow = false;
makeCall(FrameEscape, MayThrow, Args);
// Don't move TempInsertionPoint up since what we added was not an alloca
LLVMBuilder->restoreIP(SavedInsertPoint);
assert(isa<AllocaInst>(ContextLocalAddress) && "Stack Allocation expected");
GcFuncInfo->GenericsContext = cast<AllocaInst>(ContextLocalAddress);
// This method now requires a frame pointer.
// TargetMachine *TM = JitContext->TM;
// TM->Options.NoFramePointerElim = true;
// TODO: we must convey the offset of this local to the runtime
// via the GC encoding.
// https://github.com/dotnet/llilc/issues/766
if (JitContext->Options->DoInsertStatepoints) {
throw NotYetImplementedException("NYI: Generic Context reporting");
}
// Don't move TempInsertionPoint up since what we added was not an alloca
LLVMBuilder->restoreIP(SavedInsertPoint);
}
void GenIR::insertIRForSecurityObject() {
@ -602,16 +578,7 @@ void GenIR::insertIRForSecurityObject() {
LLVMBuilder->restoreIP(SavedInsertPoint);
// TODO: if passing the security object's address to the helper is not enough
// to keep it live throughout the method, find another way to ensure this.
// TODO: we must convey the offset of the security object to the runtime
// via the GC encoding.
// https://github.com/dotnet/llilc/issues/767
if (JitContext->Options->DoInsertStatepoints) {
throw NotYetImplementedException("NYI: Security Object Reporting");
}
GcFuncInfo->SecurityObject = cast<AllocaInst>(SecurityObjectAddress);
}
void GenIR::callMonitorHelper(bool IsEnter) {
@ -646,7 +613,7 @@ void GenIR::insertIRForUnmanagedCallFrame() {
// Mark this function as requiring a frame pointer and as using GC.
Function->addFnAttr("no-frame-pointer-elim", "true");
assert(GCInfo::isGCFunction(*Function));
assert(GcInfo::isGcFunction(Function));
// The call frame data structure is modeled as an opaque blob of bytes.
Type *CallFrameTy = ArrayType::get(Int8Ty, CallFrameInfo.size);
@ -733,6 +700,16 @@ void GenIR::insertIRForUnmanagedCallFrame() {
//
//===----------------------------------------------------------------------===//
AllocaInst *GenIR::createAlloca(Type *T, Value *ArraySize, const Twine &Name) {
AllocaInst *AllocaInst = LLVMBuilder->CreateAlloca(T, ArraySize, Name);
if (GcInfo::isGcAggregate(T)) {
GcFuncInfo->recordGcAggregate(AllocaInst);
}
return AllocaInst;
}
void GenIR::createSym(uint32_t Num, bool IsAuto, CorInfoType CorType,
CORINFO_CLASS_HANDLE Class, bool IsPinned,
ReaderSpecialSymbolType SymType) {
@ -774,10 +751,6 @@ void GenIR::createSym(uint32_t Num, bool IsAuto, CorInfoType CorType,
break;
}
if (IsPinned && JitContext->Options->DoInsertStatepoints) {
throw NotYetImplementedException("NYI: Pinning with Precise GC");
}
Type *LLVMType = this->getType(CorType, Class);
if (!IsAuto) {
const ABIArgInfo &Info = ABIMethodSig.getArgumentInfo(Num);
@ -787,9 +760,13 @@ void GenIR::createSym(uint32_t Num, bool IsAuto, CorInfoType CorType,
}
}
AllocaInst *AllocaInst = LLVMBuilder->CreateAlloca(
LLVMType, nullptr,
UseNumber ? Twine(SymName) + Twine(Number) : Twine(SymName));
AllocaInst *AllocaInst =
createAlloca(LLVMType, nullptr,
UseNumber ? Twine(SymName) + Twine(Number) : Twine(SymName));
if (IsPinned) {
GcFuncInfo->recordPinnedSlot(AllocaInst);
}
DIFile *Unit = DBuilder->createFile(LLILCDebugInfo.TheCU->getFilename(),
LLILCDebugInfo.TheCU->getDirectory());
@ -828,7 +805,7 @@ void GenIR::zeroInitLocals() {
bool InitAllLocals = isZeroInitLocals();
for (const auto &LocalVar : LocalVars) {
Type *LocalTy = LocalVar->getType()->getPointerElementType();
if (InitAllLocals || isManagedType(LocalTy)) {
if (InitAllLocals || GcInfo::isGcType(LocalTy)) {
// TODO: if InitAllLocals is false we only have to zero initialize
// GC pointers and GC pointer fields on structs. For now we are zero
// initalizing all fields in structs that have gc fields.
@ -934,7 +911,7 @@ void GenIR::copyStruct(CORINFO_CLASS_HANDLE Class, IRNode *Dst, IRNode *Src,
ElementStructLayout->getElementOffset(ElementIndex);
ElementType = ElementStructTy->getElementType(ElementIndex);
OffsetInElementType = OffsetInElementType - ElementOffset;
} while (!isManagedPointerType(ElementType));
} while (!GcInfo::isGcPointer(ElementType));
Type *ElementTypePointer = PointerType::get(
ElementType, GCSrcAddr->getType()->getPointerAddressSpace());
@ -1010,7 +987,7 @@ Instruction *GenIR::createTemporary(Type *Ty, const Twine &Name) {
LLVMBuilder->SetInsertPoint(InsertBefore);
}
AllocaInst *AllocaInst = LLVMBuilder->CreateAlloca(Ty, nullptr, Name);
AllocaInst *AllocaInst = createAlloca(Ty, nullptr, Name);
// Update the end of the alloca range.
TempInsertionPoint = AllocaInst;
LLVMBuilder->restoreIP(IP);
@ -2048,75 +2025,38 @@ Type *GenIR::getClassTypeWorker(
// Since padding is explicit, this is an LLVM packed struct.
StructTy->setBody(Fields, true /* isPacked */);
// For value classes we can do further checking and validate
// against the runtime's view of the class.
//
// Note the runtime only gives us size and gc info for value classes so
// we can't do this more generally.
// Verify overall size matches up.
if (HaveClassSize) {
ASSERT(EEClassSize == DataLayout->getTypeSizeInBits(StructTy) / 8);
// Verify that the LLVM type contains the same information
// as the GC field info from the runtime.
const StructLayout *MainStructLayout =
DataLayout->getStructLayout(StructTy);
const uint32_t PointerSize = DataLayout->getPointerSize();
// Walk through the type in pointer-sized jumps.
for (uint32_t GCOffset = 0; GCOffset < EEClassSize;
GCOffset += PointerSize) {
const uint32_t FieldIndex =
MainStructLayout->getElementContainingOffset(GCOffset);
Type *FieldTy = StructTy->getStructElementType(FieldIndex);
// If the field is a value class we need to dive in
// to its fields and so on, until we reach a primitive type.
if (FieldTy->isStructTy()) {
// Prepare to loop through the nesting.
const StructLayout *OuterStructLayout = MainStructLayout;
uint32_t OuterOffset = GCOffset;
uint32_t OuterIndex = FieldIndex;
while (FieldTy->isStructTy()) {
// Offset of the Inner class within the outer class
const uint32_t InnerBaseOffset =
OuterStructLayout->getElementOffset(OuterIndex);
// Inner class should start at or before the outer offset
ASSERT(InnerBaseOffset <= OuterOffset);
// Determine target offset relative to this inner class.
const uint32_t InnerOffset = OuterOffset - InnerBaseOffset;
// Get the inner class layout
StructType *InnerStructTy = cast<StructType>(FieldTy);
const StructLayout *InnerStructLayout =
DataLayout->getStructLayout(InnerStructTy);
// Find the field at that target offset.
const uint32_t InnerIndex =
InnerStructLayout->getElementContainingOffset(InnerOffset);
// Update for next iteration.
FieldTy = InnerStructTy->getStructElementType(InnerIndex);
OuterStructLayout = InnerStructLayout;
OuterOffset = InnerOffset;
OuterIndex = InnerIndex;
}
}
// For value classes we can do further checking and validate
// against the runtime's view of the class.
//
// Note the runtime only gives us size and gc info for value classes so
// we can't do this more generally.
#ifndef NDEBUG
// LLVM's type and the runtime must agree here.
GCLayout *RuntimeGCInfo = getClassGCLayout(ClassHandle);
const bool ExpectGCPointer =
(RuntimeGCInfo != nullptr) &&
(RuntimeGCInfo->GCPointers[GCOffset / PointerSize] !=
CorInfoGCType::TYPE_GC_NONE);
const bool IsGCPointer = isManagedPointerType(FieldTy);
assert((ExpectGCPointer == IsGCPointer) &&
"llvm type incorrectly describes location of gc references");
if (HaveClassSize) {
const uint32_t PointerSize = DataLayout->getPointerSize();
llvm::SmallVector<uint32_t, 4> GcPtrOffsets;
GCLayout *RuntimeGCInfo = getClassGCLayout(ClassHandle);
GcInfo::getGcPointers(StructTy, *DataLayout, GcPtrOffsets);
if (RuntimeGCInfo != nullptr) {
assert(RuntimeGCInfo->NumGCPointers == GcPtrOffsets.size() &&
"Runtime and LLVM Types differ in #GC-pointers");
for (uint32_t GcOffset : GcPtrOffsets) {
assert((RuntimeGCInfo->GCPointers[GcOffset / PointerSize] !=
CorInfoGCType::TYPE_GC_NONE) &&
"Runtime and LLVM Types differ in GC-pointer Locations");
}
free(RuntimeGCInfo);
#endif
} else {
assert(GcPtrOffsets.size() == 0 &&
"Runtime and LLVM Types differ in GC-ness");
}
}
#endif
// Return the struct or a pointer to it as requested.
return ResultTy;
@ -2156,7 +2096,7 @@ void GenIR::createOverlapFields(
uint32_t Offset = OverlapField.first;
uint32_t Size = DataLayout->getTypeSizeInBits(OverlapField.second) / 8;
OverlapEndOffset = std::max(OverlapEndOffset, Offset + Size);
if (isManagedPointerType(OverlapField.second)) {
if (GcInfo::isGcPointer(OverlapField.second)) {
assert(((Offset % getPointerByteSize()) == 0) &&
"expect aligned gc pointers");
if (GcOffsets.empty()) {
@ -2616,51 +2556,11 @@ IRNode *GenIR::convertFromStackType(IRNode *Node, CorInfoType CorType,
}
PointerType *GenIR::getManagedPointerType(Type *ElementType) {
return PointerType::get(ElementType, ManagedAddressSpace);
return PointerType::get(ElementType, GcInfo::ManagedAddressSpace);
}
PointerType *GenIR::getUnmanagedPointerType(Type *ElementType) {
return PointerType::get(ElementType, UnmanagedAddressSpace);
}
bool GenIR::isManagedPointerType(Type *Type) {
const PointerType *PtrType = dyn_cast<llvm::PointerType>(Type);
if (PtrType != nullptr) {
return PtrType->getAddressSpace() == ManagedAddressSpace;
}
return false;
}
bool GenIR::isManagedAggregateType(Type *AggType) {
VectorType *VecType = dyn_cast<VectorType>(AggType);
if (VecType != nullptr) {
return isManagedPointerType(VecType->getScalarType());
}
ArrayType *ArrType = dyn_cast<ArrayType>(AggType);
if (ArrType != nullptr) {
return isManagedType(ArrType->getElementType());
}
StructType *StType = dyn_cast<StructType>(AggType);
if (StType != nullptr) {
for (Type *SubType : StType->subtypes()) {
if (isManagedType(SubType)) {
return true;
}
}
}
return false;
}
bool GenIR::isManagedType(Type *Type) {
return isManagedPointerType(Type) || isManagedAggregateType(Type);
}
bool GenIR::isUnmanagedPointerType(llvm::Type *Type) {
return Type->isPointerTy() && !isManagedPointerType(Type);
return PointerType::get(ElementType, GcInfo::UnmanagedAddressSpace);
}
uint32_t GenIR::addArrayFields(std::vector<llvm::Type *> &Fields, bool IsVector,
@ -2703,7 +2603,7 @@ bool GenIR::isArrayType(llvm::Type *ArrayTy, llvm::Type *ElementTy) {
// Do some basic sanity checks that this type is one we created to model
// a CLR array. Note we can't be 100% sure without keeping a whitelist
// when we create these types.
assert(isManagedPointerType(ArrayTy) && "expected managed pointer");
assert(GcInfo::isGcPointer(ArrayTy) && "expected managed pointer");
Type *Type = cast<PointerType>(ArrayTy)->getPointerElementType();
if (!Type->isStructTy()) {
return false;
@ -3360,7 +3260,7 @@ IRNode *GenIR::loadLen(IRNode *Array, bool ArrayMayBeNull) {
IRNode *GenIR::loadStringLen(IRNode *Address) {
// Address should be a managed pointer type.
Type *AddressTy = Address->getType();
ASSERT(isManagedPointerType(AddressTy));
ASSERT(GcInfo::isGcPointer(AddressTy));
// Optionally do an explicit null check.
bool NullCheckBeforeLoad = UseExplicitNullChecks;
@ -3391,7 +3291,7 @@ IRNode *GenIR::loadStringLen(IRNode *Address) {
IRNode *GenIR::stringGetChar(IRNode *Address, IRNode *Index) {
// Address should be a managed pointer type.
Type *AddressTy = Address->getType();
ASSERT(isManagedPointerType(AddressTy));
ASSERT(GcInfo::isGcPointer(AddressTy));
// Optionally do an explicit null check.
bool NullCheckBeforeLoad = UseExplicitNullChecks;
@ -3702,8 +3602,8 @@ Type *GenIR::binaryOpType(ReaderBaseNS::BinaryOpcode Opcode, Type *Type1,
return Type2;
}
} else {
const bool Type1IsUnmanagedPointer = isUnmanagedPointerType(Type1);
const bool Type2IsUnmanagedPointer = isUnmanagedPointerType(Type2);
const bool Type1IsUnmanagedPointer = GcInfo::isUnmanagedPointer(Type1);
const bool Type2IsUnmanagedPointer = GcInfo::isUnmanagedPointer(Type2);
const bool IsStrictlyAdd = (Opcode == ReaderBaseNS::Add);
const bool IsAdd = IsStrictlyAdd || (Opcode == ReaderBaseNS::AddOvf) ||
(Opcode == ReaderBaseNS::AddOvfUn);
@ -3733,8 +3633,8 @@ Type *GenIR::binaryOpType(ReaderBaseNS::BinaryOpcode Opcode, Type *Type1,
return Type::getIntNTy(*JitContext->LLVMContext,
TargetPointerSizeInBits);
}
} else if (isManagedPointerType(Type1)) {
if (IsSub && isManagedPointerType(Type2)) {
} else if (GcInfo::isGcPointer(Type1)) {
if (IsSub && GcInfo::isGcPointer(Type2)) {
// The difference of two managed pointers is a native int.
return Type::getIntNTy(*JitContext->LLVMContext,
TargetPointerSizeInBits);
@ -4002,7 +3902,7 @@ IRNode *GenIR::getFieldAddress(CORINFO_RESOLVED_TOKEN *ResolvedToken,
}
ASSERT(AddressTy->isPointerTy());
const bool IsGcPointer = isManagedPointerType(AddressTy);
const bool IsGcPointer = GcInfo::isGcPointer(AddressTy);
Value *RawAddress = rdrGetFieldAddress(ResolvedToken, FieldInfo, Obj,
IsGcPointer, MustNullCheck);
@ -4053,8 +3953,7 @@ IRNode *GenIR::loadField(CORINFO_RESOLVED_TOKEN *ResolvedToken, IRNode *Obj,
// We haven't information abouth this field. Try to return struct
// type.
assert(VectorTypeToStructType.count(ObjType));
IRNode *Pointer =
(IRNode *)LLVMBuilder->CreateAlloca(Obj->getType(), nullptr);
IRNode *Pointer = (IRNode *)createAlloca(Obj->getType(), nullptr);
LLVMBuilder->CreateStore(Obj, Pointer);
Obj = (IRNode *)LLVMBuilder->CreateBitCast(
Pointer,
@ -4099,7 +3998,7 @@ IRNode *GenIR::loadField(CORINFO_RESOLVED_TOKEN *ResolvedToken, IRNode *Obj,
// Fields typed as GC pointers are always aligned,
// so ignore any smaller alignment prefix
if (FieldTy->isPointerTy() &&
isManagedPointerType(cast<PointerType>(FieldTy))) {
GcInfo::isGcPointer(cast<PointerType>(FieldTy))) {
AlignmentPrefix = Reader_AlignNatural;
}
@ -4108,11 +4007,11 @@ IRNode *GenIR::loadField(CORINFO_RESOLVED_TOKEN *ResolvedToken, IRNode *Obj,
// and do a load-indirect off it.
if (FieldInfo.fieldAccessor == CORINFO_FIELD_INSTANCE_HELPER) {
handleMemberAccess(FieldInfo.accessAllowed, FieldInfo.accessCalloutHelper);
IRNode *Destination;
const bool IsLoad = true;
IRNode *ValueToStore = nullptr;
if (FieldInfo.helper == CORINFO_HELP_GETFIELDSTRUCT) {
Destination = (IRNode *)createTemporary(FieldTy);
setValueRepresentsStruct(Destination);
@ -4272,8 +4171,8 @@ void GenIR::storeField(CORINFO_RESOLVED_TOKEN *FieldToken, IRNode *ValueToStore,
const bool IsLoad = false;
IRNode *Destination = nullptr;
rdrCallFieldHelper(FieldToken, FieldInfo.helper, IsLoad, Destination, Object,
ValueToStore, Alignment, IsVolatile);
rdrCallFieldHelper(FieldToken, FieldInfo.helper, IsLoad, Destination,
Object, ValueToStore, Alignment, IsVolatile);
return;
}
@ -4338,7 +4237,7 @@ void GenIR::storeNonPrimitiveType(IRNode *Value, IRNode *Addr,
void GenIR::storeIndirectArg(const CallArgType &ValueArgType,
llvm::Value *ValueToStore, llvm::Value *Address,
bool IsVolatile) {
assert(isManagedPointerType(cast<PointerType>(Address->getType())));
assert(GcInfo::isGcPointer(cast<PointerType>(Address->getType())));
Type *ValueToStoreType = ValueToStore->getType();
if (ValueToStoreType->isVectorTy()) {
@ -4797,7 +4696,7 @@ bool GenIR::arraySet(CORINFO_SIG_INFO *Sig) {
const bool IsVolatile = false;
// Store the value
if (isManagedPointerType(ElementTy)) {
if (GcInfo::isGcPointer(ElementTy)) {
// Since arrays are always on the heap, writing a GC pointer into an array
// always requires a write barrier.
CORINFO_RESOLVED_TOKEN *const ResolvedToken = nullptr;
@ -6392,7 +6291,8 @@ bool GenIR::interlockedCmpXchg(IRNode *Destination, IRNode *Exchange,
IRNode *Comparand, IRNode **Result,
CorInfoIntrinsics IntrinsicID) {
if (Exchange->getType()->isPointerTy()) {
Exchange = (IRNode *)LLVMBuilder->CreatePtrToInt(Exchange, Comparand->getType());
Exchange =
(IRNode *)LLVMBuilder->CreatePtrToInt(Exchange, Comparand->getType());
}
ASSERT(Exchange->getType() == Comparand->getType());
@ -6415,7 +6315,7 @@ bool GenIR::interlockedCmpXchg(IRNode *Destination, IRNode *Exchange,
Destination = (IRNode *)LLVMBuilder->CreateIntToPtr(Destination, CastTy);
} else {
ASSERT(DestinationTy->isPointerTy());
Type *CastTy = isManagedPointerType(DestinationTy)
Type *CastTy = GcInfo::isGcPointer(DestinationTy)
? getManagedPointerType(ComparandTy)
: getUnmanagedPointerType(ComparandTy);
Destination = (IRNode *)LLVMBuilder->CreatePointerCast(Destination, CastTy);
@ -6451,9 +6351,9 @@ bool GenIR::interlockedIntrinsicBinOp(IRNode *Arg1, IRNode *Arg2,
if (Op != AtomicRMWInst::BinOp::BAD_BINOP) {
assert(Arg1->getType()->isPointerTy());
Type *CastTy = isManagedPointerType(Arg1->getType())
? getManagedPointerType(Arg2->getType())
: getUnmanagedPointerType(Arg2->getType());
Type *CastTy = GcInfo::isGcPointer(Arg1->getType())
? getManagedPointerType(Arg2->getType())
: getUnmanagedPointerType(Arg2->getType());
Arg1 = (IRNode *)LLVMBuilder->CreatePointerCast(Arg1, CastTy);
Value *Result = LLVMBuilder->CreateAtomicRMW(
@ -6991,7 +6891,7 @@ IRNode *GenIR::derefAddress(IRNode *Address, bool DstIsGCPtr, bool IsConst,
Address = (IRNode *)LLVMBuilder->CreateIntToPtr(Address, CastTy);
} else if (AddressPointerTy->getElementType() != ReferentTy) {
// Cast to the appropriate referent type
Type *CastTy = isManagedPointerType(AddressPointerTy)
Type *CastTy = GcInfo::isGcPointer(AddressPointerTy)
? getManagedPointerType(ReferentTy)
: getUnmanagedPointerType(ReferentTy);
Address = (IRNode *)LLVMBuilder->CreatePointerCast(Address, CastTy);
@ -7268,7 +7168,7 @@ IRNode *GenIR::getTypedAddress(IRNode *Addr, CorInfoType CorInfoType,
throw NotYetImplementedException(
"unexpected type in load/store primitive");
}
assert(isManagedPointerType(ReferentTy));
assert(GcInfo::isGcPointer(ReferentTy));
assert(cast<PointerType>(ReferentTy)
->getPointerElementType()
->isStructTy());
@ -7292,7 +7192,7 @@ IRNode *GenIR::getTypedAddress(IRNode *Addr, CorInfoType CorInfoType,
if (PointerTy != nullptr) {
Type *ReferentTy = PointerTy->getPointerElementType();
if (ReferentTy != ExpectedTy) {
Type *PtrToExpectedTy = isManagedPointerType(PointerTy)
Type *PtrToExpectedTy = GcInfo::isGcPointer(PointerTy)
? getManagedPointerType(ExpectedTy)
: getUnmanagedPointerType(ExpectedTy);
TypedAddr =
@ -7323,7 +7223,7 @@ IRNode *GenIR::loadPrimitiveType(IRNode *Addr, CorInfoType CorInfoType,
PointerType *PointerTy = dyn_cast<PointerType>(AddressTy);
if (PointerTy != nullptr) {
Type *ReferentTy = PointerTy->getPointerElementType();
NeedsCoercion = !isManagedPointerType(ReferentTy);
NeedsCoercion = !GcInfo::isGcPointer(ReferentTy);
} else {
NeedsCoercion = true;
}
@ -7806,7 +7706,7 @@ IRNode *GenIR::localAlloc(IRNode *Arg, bool ZeroInit) {
const unsigned int Alignment = TargetPointerSizeInBits / 8;
LLVMContext &Context = *JitContext->LLVMContext;
Type *Ty = Type::getInt8Ty(Context);
AllocaInst *LocAlloc = LLVMBuilder->CreateAlloca(Ty, Arg, "LocAlloc");
AllocaInst *LocAlloc = createAlloca(Ty, Arg, "LocAlloc");
LocAlloc->setAlignment(Alignment);
// Zero the allocated region if so requested.
@ -7865,12 +7765,12 @@ IRNode *GenIR::makeRefAny(CORINFO_RESOLVED_TOKEN *ResolvedToken,
// pointer type (or native int in unverifiable code). But traditionally .Net
// jits have allowed arbitrary integers here too. So, tolerate this.
Type *ExpectedObjectTy = RefAnyStructTy->getContainedType(ValueIndex);
assert(isManagedPointerType(ExpectedObjectTy));
assert(GcInfo::isGcPointer(ExpectedObjectTy));
Type *ActualObjectTy = Object->getType();
Value *CastObject;
if (isManagedPointerType(ActualObjectTy)) {
if (GcInfo::isGcPointer(ActualObjectTy)) {
CastObject = LLVMBuilder->CreatePointerCast(Object, ExpectedObjectTy);
} else if (isUnmanagedPointerType(ActualObjectTy)) {
} else if (GcInfo::isUnmanagedPointer(ActualObjectTy)) {
CastObject = LLVMBuilder->CreateAddrSpaceCast(Object, ExpectedObjectTy);
} else {
assert(Object->getType()->isIntegerTy());
@ -8101,7 +8001,7 @@ Value *GenIR::changePHIOperandType(Value *Operand, BasicBlock *OperandBlock,
bool IsSigned = true;
return LLVMBuilder->CreateIntCast(Operand, NewTy, IsSigned);
} else {
assert(isUnmanagedPointerType(OperandTy));
assert(GcInfo::isUnmanagedPointer(OperandTy));
return LLVMBuilder->CreatePtrToInt(Operand, NewTy);
}
} else if (NewTy->isFloatingPointTy()) {
@ -8138,8 +8038,8 @@ Type *GenIR::getStackMergeType(Type *Ty1, Type *Ty2, bool IsStruct1,
}
// If we have unmanaged pointer and nativeint the result is nativeint.
if ((isUnmanagedPointerType(Ty1) && (Ty2 == NativeIntTy)) ||
(isUnmanagedPointerType(Ty2) && (Ty1 == NativeIntTy))) {
if ((GcInfo::isUnmanagedPointer(Ty1) && (Ty2 == NativeIntTy)) ||
(GcInfo::isUnmanagedPointer(Ty2) && (Ty1 == NativeIntTy))) {
return NativeIntTy;
}
@ -8147,8 +8047,7 @@ Type *GenIR::getStackMergeType(Type *Ty1, Type *Ty2, bool IsStruct1,
PointerType *PointerTy1 = dyn_cast<PointerType>(Ty1);
PointerType *PointerTy2 = dyn_cast<PointerType>(Ty2);
if ((PointerTy1 != nullptr) && (PointerTy2 != nullptr) &&
(isManagedPointerType(PointerTy1)) &&
(isManagedPointerType(PointerTy2))) {
(GcInfo::isGcPointer(PointerTy1)) && (GcInfo::isGcPointer(PointerTy2))) {
CORINFO_CLASS_HANDLE Class1 = nullptr;
auto MapElement1 = ReverseClassTypeMap->find(PointerTy1);