[ORC][JITLink] Merge JITLink AllocActionCall and ORC WrapperFunctionCall.

These types performed identical roles. Merging them simplifies interoperability
between JITLink and ORC APIs (allowing us to address a few FIXMEs).
This commit is contained in:
Lang Hames 2022-01-08 12:08:06 +11:00
parent 9d74582810
commit 089acf2522
29 changed files with 329 additions and 388 deletions

View File

@ -13,6 +13,8 @@
#include "adt.h"
#include "c_api.h"
#include "common.h"
#include "executor_address.h"
#include "wrapper_function_utils.h"
using namespace __orc_rt;
@ -49,20 +51,24 @@ void walkEHFrameSection(span<const char> EHFrameSection,
ORC_RT_INTERFACE __orc_rt_CWrapperFunctionResult
__orc_rt_macho_register_ehframe_section(char *ArgData, size_t ArgSize) {
// NOTE: Does not use SPS to deserialize arg buffer, instead the arg buffer
// is taken to be the range of the eh-frame section.
bool HasError = false;
walkEHFrameSection(span<const char>(ArgData, ArgSize), __register_frame);
return __orc_rt_CreateCWrapperFunctionResultFromRange((char*)&HasError,
sizeof(HasError));
return WrapperFunction<SPSError(SPSExecutorAddrRange)>::handle(
ArgData, ArgSize,
[](ExecutorAddrRange FrameSection) -> Error {
walkEHFrameSection(FrameSection.toSpan<const char>(),
__register_frame);
return Error::success();
})
.release();
}
ORC_RT_INTERFACE __orc_rt_CWrapperFunctionResult
__orc_rt_macho_deregister_ehframe_section(char *ArgData, size_t ArgSize) {
// NOTE: Does not use SPS to deserialize arg buffer, instead the arg buffer
// is taken to be the range of the eh-frame section.
bool HasError = false;
walkEHFrameSection(span<const char>(ArgData, ArgSize), __deregister_frame);
return __orc_rt_CreateCWrapperFunctionResultFromRange((char*)&HasError,
sizeof(HasError));
return WrapperFunction<SPSError(SPSExecutorAddrRange)>::handle(
ArgData, ArgSize,
[](ExecutorAddrRange FrameSection) -> Error {
walkEHFrameSection(FrameSection.toSpan<const char>(),
__deregister_frame);
return Error::success();
})
.release();
}

View File

@ -595,12 +595,11 @@ ORC_RT_INTERFACE __orc_rt_CWrapperFunctionResult
__orc_rt_macho_register_thread_data_section(char *ArgData, size_t ArgSize) {
// NOTE: Does not use SPS to deserialize arg buffer, instead the arg buffer
// is taken to be the range of the thread data section.
return WrapperFunction<SPSError()>::handle(
nullptr, 0,
[&]() {
return WrapperFunction<SPSError(SPSExecutorAddrRange)>::handle(
ArgData, ArgSize,
[](ExecutorAddrRange R) {
return MachOPlatformRuntimeState::get()
.registerThreadDataSection(
span<const char>(ArgData, ArgSize));
.registerThreadDataSection(R.toSpan<const char>());
})
.release();
}
@ -609,12 +608,11 @@ ORC_RT_INTERFACE __orc_rt_CWrapperFunctionResult
__orc_rt_macho_deregister_thread_data_section(char *ArgData, size_t ArgSize) {
// NOTE: Does not use SPS to deserialize arg buffer, instead the arg buffer
// is taken to be the range of the thread data section.
return WrapperFunction<SPSError()>::handle(
nullptr, 0,
[&]() {
return WrapperFunction<SPSError(SPSExecutorAddrRange)>::handle(
ArgData, ArgSize,
[](ExecutorAddrRange R) {
return MachOPlatformRuntimeState::get()
.deregisterThreadDataSection(
span<const char>(ArgData, ArgSize));
.deregisterThreadDataSection(R.toSpan<const char>());
})
.release();
}

View File

@ -25,20 +25,16 @@ namespace jitlink {
class EHFrameRegistrar {
public:
virtual ~EHFrameRegistrar();
virtual Error registerEHFrames(orc::ExecutorAddr EHFrameSectionAddr,
size_t EHFrameSectionSize) = 0;
virtual Error deregisterEHFrames(orc::ExecutorAddr EHFrameSectionAddr,
size_t EHFrameSectionSize) = 0;
virtual Error registerEHFrames(orc::ExecutorAddrRange EHFrameSection) = 0;
virtual Error deregisterEHFrames(orc::ExecutorAddrRange EHFrameSection) = 0;
};
/// Registers / Deregisters EH-frames in the current process.
class InProcessEHFrameRegistrar final : public EHFrameRegistrar {
public:
Error registerEHFrames(orc::ExecutorAddr EHFrameSectionAddr,
size_t EHFrameSectionSize) override;
Error registerEHFrames(orc::ExecutorAddrRange EHFrameSection) override;
Error deregisterEHFrames(orc::ExecutorAddr EHFrameSectionAddr,
size_t EHFrameSectionSize) override;
Error deregisterEHFrames(orc::ExecutorAddrRange EHFrameSection) override;
};
using StoreFrameRangeFunction = std::function<void(

View File

@ -797,6 +797,10 @@ public:
}
orc::ExecutorAddrDiff getSize() const { return getEnd() - getStart(); }
orc::ExecutorAddrRange getRange() const {
return orc::ExecutorAddrRange(getStart(), getEnd());
}
private:
Block *First = nullptr;
Block *Last = nullptr;
@ -1388,7 +1392,7 @@ public:
///
/// Accessing this object after finalization will result in undefined
/// behavior.
AllocActions &allocActions() { return AAs; }
orc::shared::AllocActions &allocActions() { return AAs; }
/// Dump the graph.
void dump(raw_ostream &OS);
@ -1406,7 +1410,7 @@ private:
SectionList Sections;
ExternalSymbolSet ExternalSymbols;
ExternalSymbolSet AbsoluteSymbols;
AllocActions AAs;
orc::shared::AllocActions AAs;
};
inline MutableArrayRef<char> Block::getMutableContent(LinkGraph &G) {

View File

@ -14,8 +14,10 @@
#define LLVM_EXECUTIONENGINE_JITLINK_JITLINKMEMORYMANAGER_H
#include "llvm/ADT/FunctionExtras.h"
#include "llvm/ADT/SmallVector.h"
#include "llvm/ExecutionEngine/JITLink/JITLinkDylib.h"
#include "llvm/ExecutionEngine/JITLink/MemoryFlags.h"
#include "llvm/ExecutionEngine/Orc/Shared/AllocationActions.h"
#include "llvm/ExecutionEngine/Orc/Shared/ExecutorAddress.h"
#include "llvm/Support/Allocator.h"
#include "llvm/Support/Error.h"
@ -34,47 +36,6 @@ class Block;
class LinkGraph;
class Section;
/// Represents a call to a graph-memory-management support function in the
/// executor.
///
/// Support functions are called as:
///
/// auto *Result =
/// ((char*(*)(const void*, size_t))FnAddr)(
/// (const void*)CtxAddr, (size_t)CtxSize)
///
/// A null result is interpreted as success.
///
/// A non-null result is interpreted as a heap-allocated string containing
/// an error message to report to the allocator (the allocator's
/// executor-side implementation code is responsible for freeing the error
/// string).
struct AllocActionCall {
orc::ExecutorAddr FnAddr;
orc::ExecutorAddr CtxAddr;
orc::ExecutorAddrDiff CtxSize;
};
/// A pair of AllocActionCalls, one to be run at finalization time, one to be
/// run at deallocation time.
///
/// AllocActionCallPairs should be constructed for paired operations (e.g.
/// __register_ehframe and __deregister_ehframe for eh-frame registration).
/// See comments for AllocActions for execution ordering.
///
/// For unpaired operations one or the other member can be left unused, as
/// AllocationActionCalls with an FnAddr of zero will be skipped.
struct AllocActionCallPair {
AllocActionCall Finalize;
AllocActionCall Dealloc;
};
/// A vector of allocation actions to be run for this allocation.
///
/// Finalize allocations will be run in order at finalize time. Dealloc
/// actions will be run in reverse order at deallocation time.
using AllocActions = std::vector<AllocActionCallPair>;
/// Manages allocations of JIT memory.
///
/// Instances of this class may be accessed concurrently from multiple threads
@ -315,7 +276,7 @@ public:
/// Returns a reference to the AllocActions in the graph.
/// This convenience function saves callers from having to #include
/// LinkGraph.h if all they need are allocation actions.
AllocActions &graphAllocActions();
orc::shared::AllocActions &graphAllocActions();
private:
LinkGraph &G;
@ -416,12 +377,12 @@ private:
// There shouldn't need to be a heap alloc for this.
struct FinalizedAllocInfo {
sys::MemoryBlock StandardSegments;
std::vector<AllocActionCall> DeallocActions;
std::vector<orc::shared::WrapperFunctionCall> DeallocActions;
};
FinalizedAlloc
createFinalizedAlloc(sys::MemoryBlock StandardSegments,
std::vector<AllocActionCall> DeallocActions);
FinalizedAlloc createFinalizedAlloc(
sys::MemoryBlock StandardSegments,
std::vector<orc::shared::WrapperFunctionCall> DeallocActions);
uint64_t PageSize;
std::mutex FinalizedAllocsMutex;

View File

@ -39,10 +39,8 @@ public:
: ES(ES), RegisterEHFrameWrapperFnAddr(RegisterEHFrameWrapperFnAddr),
DeregisterEHFrameWrapperFnAddr(DeregisterEHFRameWrapperFnAddr) {}
Error registerEHFrames(ExecutorAddr EHFrameSectionAddr,
size_t EHFrameSectionSize) override;
Error deregisterEHFrames(ExecutorAddr EHFrameSectionAddr,
size_t EHFrameSectionSize) override;
Error registerEHFrames(ExecutorAddrRange EHFrameSection) override;
Error deregisterEHFrames(ExecutorAddrRange EHFrameSection) override;
private:
ExecutionSession &ES;

View File

@ -89,11 +89,6 @@ private:
ExecutorAddr RemoteAddr;
};
struct EHFrame {
ExecutorAddr Addr;
uint64_t Size;
};
// Group of section allocations to be allocated together in the executor. The
// RemoteCodeAddr will stand in as the id of the group for deallocation
// purposes.
@ -107,7 +102,7 @@ private:
ExecutorAddrRange RemoteCode;
ExecutorAddrRange RemoteROData;
ExecutorAddrRange RemoteRWData;
std::vector<EHFrame> UnfinalizedEHFrames;
std::vector<ExecutorAddrRange> UnfinalizedEHFrames;
std::vector<Alloc> CodeAllocs, RODataAllocs, RWDataAllocs;
};

View File

@ -216,17 +216,11 @@ public:
ResourceKey SrcKey) override;
private:
struct EHFrameRange {
orc::ExecutorAddr Addr;
size_t Size;
};
std::mutex EHFramePluginMutex;
ExecutionSession &ES;
std::unique_ptr<jitlink::EHFrameRegistrar> Registrar;
DenseMap<MaterializationResponsibility *, EHFrameRange> InProcessLinks;
DenseMap<ResourceKey, std::vector<EHFrameRange>> EHFrameRanges;
DenseMap<MaterializationResponsibility *, ExecutorAddrRange> InProcessLinks;
DenseMap<ResourceKey, std::vector<ExecutorAddrRange>> EHFrameRanges;
};
} // end namespace orc

View File

@ -37,8 +37,8 @@ extern const char *MemoryWriteUInt32sWrapperName;
extern const char *MemoryWriteUInt64sWrapperName;
extern const char *MemoryWriteBuffersWrapperName;
extern const char *RegisterEHFrameSectionCustomDirectWrapperName;
extern const char *DeregisterEHFrameSectionCustomDirectWrapperName;
extern const char *RegisterEHFrameSectionWrapperName;
extern const char *DeregisterEHFrameSectionWrapperName;
extern const char *RunAsMainWrapperName;

View File

@ -33,6 +33,7 @@
#define LLVM_EXECUTIONENGINE_ORC_SHARED_SIMPLEPACKEDSERIALIZATION_H
#include "llvm/ADT/STLExtras.h"
#include "llvm/ADT/SmallVector.h"
#include "llvm/ADT/StringMap.h"
#include "llvm/ADT/StringRef.h"
#include "llvm/Support/Error.h"
@ -112,12 +113,22 @@ public:
static bool serialize(SPSOutputBuffer &OB) { return true; }
static bool deserialize(SPSInputBuffer &IB) { return true; }
static bool serializeToSmallVector(SmallVectorImpl<char> &V) { return true; }
static bool deserializeFromSmallVector(const SmallVectorImpl<char> &V) {
return true;
}
};
// Non-empty list specialization for SPSArgList.
template <typename SPSTagT, typename... SPSTagTs>
class SPSArgList<SPSTagT, SPSTagTs...> {
public:
// FIXME: This typedef is here to enable SPS arg serialization from
// JITLink. It can be removed once JITLink can access SPS directly.
using OutputBuffer = SPSOutputBuffer;
template <typename ArgT, typename... ArgTs>
static size_t size(const ArgT &Arg, const ArgTs &...Args) {
return SPSSerializationTraits<SPSTagT, ArgT>::size(Arg) +
@ -284,6 +295,40 @@ public:
}
};
/// Trivial SmallVectorImpl<T> -> SPSSequence<char> serialization.
template <typename SPSElementTagT, typename T>
class TrivialSPSSequenceSerialization<SPSElementTagT, SmallVectorImpl<T>> {
public:
static constexpr bool available = true;
};
/// Trivial SPSSequence<SPSElementTagT> -> SmallVectorImpl<T> deserialization.
template <typename SPSElementTagT, typename T>
class TrivialSPSSequenceDeserialization<SPSElementTagT, SmallVectorImpl<T>> {
public:
static constexpr bool available = true;
using element_type = typename SmallVectorImpl<T>::value_type;
static void reserve(SmallVectorImpl<T> &V, uint64_t Size) { V.reserve(Size); }
static bool append(SmallVectorImpl<T> &V, T E) {
V.push_back(std::move(E));
return true;
}
};
/// Trivial SmallVectorImpl<T> -> SPSSequence<char> serialization.
template <typename SPSElementTagT, typename T, unsigned N>
class TrivialSPSSequenceSerialization<SPSElementTagT, SmallVector<T, N>>
: public TrivialSPSSequenceSerialization<SPSElementTagT,
SmallVectorImpl<T>> {};
/// Trivial SPSSequence<SPSElementTagT> -> SmallVectorImpl<T> deserialization.
template <typename SPSElementTagT, typename T, unsigned N>
class TrivialSPSSequenceDeserialization<SPSElementTagT, SmallVector<T, N>>
: public TrivialSPSSequenceDeserialization<SPSElementTagT,
SmallVectorImpl<T>> {};
/// Trivial ArrayRef<T> -> SPSSequence<SPSElementTagT> serialization.
template <typename SPSElementTagT, typename T>
class TrivialSPSSequenceSerialization<SPSElementTagT, ArrayRef<T>> {

View File

@ -17,6 +17,7 @@
#include "llvm/ADT/ArrayRef.h"
#include "llvm/ADT/StringRef.h"
#include "llvm/ExecutionEngine/JITSymbol.h"
#include "llvm/ExecutionEngine/Orc/Shared/AllocationActions.h"
#include "llvm/ExecutionEngine/Orc/Shared/ExecutorAddress.h"
#include "llvm/ExecutionEngine/Orc/Shared/SimplePackedSerialization.h"
#include "llvm/ExecutionEngine/Orc/Shared/WrapperFunctionUtils.h"
@ -69,50 +70,6 @@ inline std::string getWireProtectionFlagsStr(WireProtectionFlags WPF) {
return Result;
}
struct WrapperFunctionCall {
ExecutorAddr Func;
ExecutorAddrRange ArgData;
WrapperFunctionCall() = default;
WrapperFunctionCall(ExecutorAddr Func, ExecutorAddr ArgData,
ExecutorAddrDiff ArgSize)
: Func(Func), ArgData(ArgData, ArgSize) {}
WrapperFunctionCall(ExecutorAddr Func, ExecutorAddrRange ArgData)
: Func(Func), ArgData(ArgData) {}
shared::WrapperFunctionResult run() {
using FnTy =
shared::CWrapperFunctionResult(const char *ArgData, size_t ArgSize);
return shared::WrapperFunctionResult(
Func.toPtr<FnTy *>()(ArgData.Start.toPtr<const char *>(),
static_cast<size_t>(ArgData.size())));
}
/// Run call and deserialize result using SPS.
template <typename SPSRetT, typename RetT> Error runWithSPSRet(RetT &RetVal) {
auto WFR = run();
if (const char *ErrMsg = WFR.getOutOfBandError())
return make_error<StringError>(ErrMsg, inconvertibleErrorCode());
shared::SPSInputBuffer IB(WFR.data(), WFR.size());
if (!shared::SPSSerializationTraits<SPSRetT, RetT>::deserialize(IB, RetVal))
return make_error<StringError>("Could not deserialize result from "
"serialized wrapper function call",
inconvertibleErrorCode());
return Error::success();
}
/// Overload for SPS functions returning void.
Error runWithSPSRet() {
shared::SPSEmpty E;
return runWithSPSRet<shared::SPSEmpty>(E);
}
};
struct AllocationActionsPair {
WrapperFunctionCall Finalize;
WrapperFunctionCall Deallocate;
};
struct SegFinalizeRequest {
WireProtectionFlags Prot;
ExecutorAddr Addr;
@ -122,7 +79,7 @@ struct SegFinalizeRequest {
struct FinalizeRequest {
std::vector<SegFinalizeRequest> Segments;
std::vector<AllocationActionsPair> Actions;
shared::AllocActions Actions;
};
template <typename T> struct UIntWrite {
@ -167,17 +124,12 @@ namespace shared {
class SPSMemoryProtectionFlags {};
using SPSWrapperFunctionCall = SPSTuple<SPSExecutorAddr, SPSExecutorAddrRange>;
using SPSSegFinalizeRequest =
SPSTuple<SPSMemoryProtectionFlags, SPSExecutorAddr, uint64_t,
SPSSequence<char>>;
using SPSAllocationActionsPair =
SPSTuple<SPSWrapperFunctionCall, SPSWrapperFunctionCall>;
using SPSFinalizeRequest = SPSTuple<SPSSequence<SPSSegFinalizeRequest>,
SPSSequence<SPSAllocationActionsPair>>;
SPSSequence<SPSAllocActionCallPair>>;
template <typename T>
using SPSMemoryAccessUIntWrite = SPSTuple<SPSExecutorAddr, T>;
@ -212,48 +164,6 @@ public:
}
};
template <>
class SPSSerializationTraits<SPSWrapperFunctionCall,
tpctypes::WrapperFunctionCall> {
using AL = SPSWrapperFunctionCall::AsArgList;
public:
static size_t size(const tpctypes::WrapperFunctionCall &WFC) {
return AL::size(WFC.Func, WFC.ArgData);
}
static bool serialize(SPSOutputBuffer &OB,
const tpctypes::WrapperFunctionCall &WFC) {
return AL::serialize(OB, WFC.Func, WFC.ArgData);
}
static bool deserialize(SPSInputBuffer &IB,
tpctypes::WrapperFunctionCall &WFC) {
return AL::deserialize(IB, WFC.Func, WFC.ArgData);
}
};
template <>
class SPSSerializationTraits<SPSAllocationActionsPair,
tpctypes::AllocationActionsPair> {
using AL = SPSAllocationActionsPair::AsArgList;
public:
static size_t size(const tpctypes::AllocationActionsPair &AAP) {
return AL::size(AAP.Finalize, AAP.Deallocate);
}
static bool serialize(SPSOutputBuffer &OB,
const tpctypes::AllocationActionsPair &AAP) {
return AL::serialize(OB, AAP.Finalize, AAP.Deallocate);
}
static bool deserialize(SPSInputBuffer &IB,
tpctypes::AllocationActionsPair &AAP) {
return AL::deserialize(IB, AAP.Finalize, AAP.Deallocate);
}
};
template <>
class SPSSerializationTraits<SPSSegFinalizeRequest,
tpctypes::SegFinalizeRequest> {

View File

@ -356,6 +356,15 @@ public:
}
};
template <typename SPSRetTagT>
class ResultSerializer<SPSRetTagT, ErrorSuccess> {
public:
static WrapperFunctionResult serialize(ErrorSuccess Err) {
return serializeViaSPSToWrapperFunctionResult<SPSArgList<SPSRetTagT>>(
toSPSSerializable(std::move(Err)));
}
};
template <typename SPSRetTagT, typename T>
class ResultSerializer<SPSRetTagT, Expected<T>> {
public:
@ -609,6 +618,116 @@ makeMethodWrapperHandler(RetT (ClassT::*Method)(ArgTs...)) {
return MethodWrapperHandler<RetT, ClassT, ArgTs...>(Method);
}
/// Represents a serialized wrapper function call.
/// Serializing calls themselves allows us to batch them: We can make one
/// "run-wrapper-functions" utility and send it a list of calls to run.
///
/// The motivating use-case for this API is JITLink allocation actions, where
/// we want to run multiple functions to finalize linked memory without having
/// to make separate IPC calls for each one.
class WrapperFunctionCall {
public:
using ArgDataBufferType = SmallVector<char, 24>;
/// Create a WrapperFunctionCall using the given SPS serializer to serialize
/// the arguments.
template <typename SPSSerializer, typename... ArgTs>
static Expected<WrapperFunctionCall> Create(ExecutorAddr FnAddr,
const ArgTs &...Args) {
ArgDataBufferType ArgData;
ArgData.resize(SPSSerializer::size(Args...));
SPSOutputBuffer OB(&ArgData[0], ArgData.size());
if (SPSSerializer::serialize(OB, Args...))
return WrapperFunctionCall(FnAddr, std::move(ArgData));
return make_error<StringError>("Cannot serialize arguments for "
"AllocActionCall",
inconvertibleErrorCode());
}
WrapperFunctionCall() = default;
/// Create a WrapperFunctionCall from a target function and arg buffer.
WrapperFunctionCall(ExecutorAddr FnAddr, ArgDataBufferType ArgData)
: FnAddr(FnAddr), ArgData(std::move(ArgData)) {}
/// Returns the address to be called.
const ExecutorAddr &getCallee() const { return FnAddr; }
/// Returns the argument data.
const ArgDataBufferType &getArgData() const { return ArgData; }
/// WrapperFunctionCalls convert to true if the callee is non-null.
explicit operator bool() const { return !!FnAddr; }
/// Run call returning raw WrapperFunctionResult.
shared::WrapperFunctionResult run() {
using FnTy =
shared::CWrapperFunctionResult(const char *ArgData, size_t ArgSize);
return shared::WrapperFunctionResult(
FnAddr.toPtr<FnTy *>()(ArgData.data(), ArgData.size()));
}
/// Run call and deserialize result using SPS.
template <typename SPSRetT, typename RetT>
std::enable_if_t<!std::is_same<SPSRetT, void>::value, Error>
runWithSPSRet(RetT &RetVal) {
auto WFR = run();
if (const char *ErrMsg = WFR.getOutOfBandError())
return make_error<StringError>(ErrMsg, inconvertibleErrorCode());
shared::SPSInputBuffer IB(WFR.data(), WFR.size());
if (!shared::SPSSerializationTraits<SPSRetT, RetT>::deserialize(IB, RetVal))
return make_error<StringError>("Could not deserialize result from "
"serialized wrapper function call",
inconvertibleErrorCode());
return Error::success();
}
/// Overload for SPS functions returning void.
template <typename SPSRetT>
std::enable_if_t<std::is_same<SPSRetT, void>::value, Error> runWithSPSRet() {
shared::SPSEmpty E;
return runWithSPSRet<shared::SPSEmpty>(E);
}
/// Run call and deserialize an SPSError result. SPSError returns and
/// deserialization failures are merged into the returned error.
Error runWithSPSRetErrorMerged() {
detail::SPSSerializableError RetErr;
if (auto Err = runWithSPSRet<SPSError>(RetErr))
return Err;
return detail::fromSPSSerializable(std::move(RetErr));
}
private:
orc::ExecutorAddr FnAddr;
ArgDataBufferType ArgData;
};
using SPSWrapperFunctionCall = SPSTuple<SPSExecutorAddr, SPSSequence<char>>;
template <>
class SPSSerializationTraits<SPSWrapperFunctionCall, WrapperFunctionCall> {
public:
static size_t size(const WrapperFunctionCall &WFC) {
return SPSWrapperFunctionCall::AsArgList::size(WFC.getCallee(),
WFC.getArgData());
}
static bool serialize(SPSOutputBuffer &OB, const WrapperFunctionCall &WFC) {
return SPSWrapperFunctionCall::AsArgList::serialize(OB, WFC.getCallee(),
WFC.getArgData());
}
static bool deserialize(SPSInputBuffer &IB, WrapperFunctionCall &WFC) {
ExecutorAddr FnAddr;
WrapperFunctionCall::ArgDataBufferType ArgData;
if (!SPSWrapperFunctionCall::AsArgList::deserialize(IB, FnAddr, ArgData))
return false;
WFC = WrapperFunctionCall(FnAddr, std::move(ArgData));
return true;
}
};
} // end namespace shared
} // end namespace orc
} // end namespace llvm

View File

@ -33,22 +33,6 @@ Error deregisterEHFrameSection(const void *EHFrameSectionAddr,
} // end namespace orc
} // end namespace llvm
/// An eh-frame registration utility suitable for use as a support function
/// call. This function expects the direct address and size of the eh-frame
/// section to register as its arguments (it does not treat its arguments as
/// pointers to an SPS-serialized arg buffer).
extern "C" llvm::orc::shared::CWrapperFunctionResult
llvm_orc_registerEHFrameSectionCustomDirectWrapper(
const char *EHFrameSectionAddr, uint64_t Size);
/// An eh-frame deregistration utility suitable for use as a support function
/// call. This function expects the direct address and size of the eh-frame
/// section to register as its arguments (it does not treat its arguments as
/// pointers to an SPS-serialized arg buffer).
extern "C" llvm::orc::shared::CWrapperFunctionResult
llvm_orc_deregisterEHFrameSectionCustomDirectWrapper(
const char *EHFrameSectionAddr, uint64_t Size);
extern "C" llvm::orc::shared::CWrapperFunctionResult
llvm_orc_registerEHFrameSectionWrapper(const char *Data, uint64_t Size);

View File

@ -43,7 +43,7 @@ public:
private:
struct Allocation {
size_t Size = 0;
std::vector<tpctypes::WrapperFunctionCall> DeallocationActions;
std::vector<shared::WrapperFunctionCall> DeallocationActions;
};
using AllocationsMap = DenseMap<void *, Allocation>;

View File

@ -759,15 +759,15 @@ Error EHFrameNullTerminator::operator()(LinkGraph &G) {
EHFrameRegistrar::~EHFrameRegistrar() {}
Error InProcessEHFrameRegistrar::registerEHFrames(
orc::ExecutorAddr EHFrameSectionAddr, size_t EHFrameSectionSize) {
return orc::registerEHFrameSection(EHFrameSectionAddr.toPtr<void *>(),
EHFrameSectionSize);
orc::ExecutorAddrRange EHFrameSection) {
return orc::registerEHFrameSection(EHFrameSection.Start.toPtr<void *>(),
EHFrameSection.size());
}
Error InProcessEHFrameRegistrar::deregisterEHFrames(
orc::ExecutorAddr EHFrameSectionAddr, size_t EHFrameSectionSize) {
return orc::deregisterEHFrameSection(EHFrameSectionAddr.toPtr<void *>(),
EHFrameSectionSize);
orc::ExecutorAddrRange EHFrameSection) {
return orc::deregisterEHFrameSection(EHFrameSection.Start.toPtr<void *>(),
EHFrameSection.size());
}
LinkGraphPassFunction

View File

@ -15,63 +15,12 @@
using namespace llvm;
namespace {
// FIXME: Remove this copy of CWrapperFunctionResult as soon as JITLink can
// depend on shared utils from Orc.
// Must be kept in-sync with compiler-rt/lib/orc/c-api.h.
union CWrapperFunctionResultDataUnion {
char *ValuePtr;
char Value[sizeof(ValuePtr)];
};
// Must be kept in-sync with compiler-rt/lib/orc/c-api.h.
typedef struct {
CWrapperFunctionResultDataUnion Data;
size_t Size;
} CWrapperFunctionResult;
Error toError(CWrapperFunctionResult R) {
bool HasError = false;
std::string ErrMsg;
if (R.Size) {
bool Large = R.Size > sizeof(CWrapperFunctionResultDataUnion);
char *Content = Large ? R.Data.ValuePtr : R.Data.Value;
if (Content[0]) {
HasError = true;
constexpr unsigned StrStart = 1 + sizeof(uint64_t);
ErrMsg.resize(R.Size - StrStart);
memcpy(&ErrMsg[0], Content + StrStart, R.Size - StrStart);
}
if (Large)
free(R.Data.ValuePtr);
} else if (R.Data.ValuePtr) {
HasError = true;
ErrMsg = R.Data.ValuePtr;
free(R.Data.ValuePtr);
}
if (HasError)
return make_error<StringError>(std::move(ErrMsg), inconvertibleErrorCode());
return Error::success();
}
} // namespace
namespace llvm {
namespace jitlink {
JITLinkMemoryManager::~JITLinkMemoryManager() = default;
JITLinkMemoryManager::InFlightAlloc::~InFlightAlloc() = default;
static Error runAllocAction(AllocActionCall &C) {
using WrapperFnTy = CWrapperFunctionResult (*)(const void *, size_t);
auto *Fn = C.FnAddr.toPtr<WrapperFnTy>();
return toError(
Fn(C.CtxAddr.toPtr<const void *>(), static_cast<size_t>(C.CtxSize)));
}
BasicLayout::BasicLayout(LinkGraph &G) : G(G) {
for (auto &Sec : G.sections()) {
@ -189,7 +138,9 @@ Error BasicLayout::apply() {
return Error::success();
}
AllocActions &BasicLayout::graphAllocActions() { return G.allocActions(); }
orc::shared::AllocActions &BasicLayout::graphAllocActions() {
return G.allocActions();
}
void SimpleSegmentAlloc::Create(JITLinkMemoryManager &MemMgr,
const JITLinkDylib *JD, SegmentMap Segments,
@ -297,15 +248,15 @@ public:
// Run finalization actions.
// FIXME: Roll back previous successful actions on failure.
std::vector<AllocActionCall> DeallocActions;
std::vector<orc::shared::WrapperFunctionCall> DeallocActions;
DeallocActions.reserve(G.allocActions().size());
for (auto &ActPair : G.allocActions()) {
if (ActPair.Finalize.FnAddr)
if (auto Err = runAllocAction(ActPair.Finalize)) {
if (ActPair.Finalize)
if (auto Err = ActPair.Finalize.runWithSPSRetErrorMerged()) {
OnFinalized(std::move(Err));
return;
}
if (ActPair.Dealloc.FnAddr)
if (ActPair.Dealloc)
DeallocActions.push_back(ActPair.Dealloc);
}
G.allocActions().clear();
@ -474,7 +425,7 @@ void InProcessMemoryManager::allocate(const JITLinkDylib *JD, LinkGraph &G,
void InProcessMemoryManager::deallocate(std::vector<FinalizedAlloc> Allocs,
OnDeallocatedFunction OnDeallocated) {
std::vector<sys::MemoryBlock> StandardSegmentsList;
std::vector<std::vector<AllocActionCall>> DeallocActionsList;
std::vector<std::vector<orc::shared::WrapperFunctionCall>> DeallocActionsList;
{
std::lock_guard<std::mutex> Lock(FinalizedAllocsMutex);
@ -496,7 +447,7 @@ void InProcessMemoryManager::deallocate(std::vector<FinalizedAlloc> Allocs,
/// Run any deallocate calls.
while (!DeallocActions.empty()) {
if (auto Err = runAllocAction(DeallocActions.back()))
if (auto Err = DeallocActions.back().runWithSPSRetErrorMerged())
DeallocErr = joinErrors(std::move(DeallocErr), std::move(Err));
DeallocActions.pop_back();
}
@ -515,7 +466,7 @@ void InProcessMemoryManager::deallocate(std::vector<FinalizedAlloc> Allocs,
JITLinkMemoryManager::FinalizedAlloc
InProcessMemoryManager::createFinalizedAlloc(
sys::MemoryBlock StandardSegments,
std::vector<AllocActionCall> DeallocActions) {
std::vector<orc::shared::WrapperFunctionCall> DeallocActions) {
std::lock_guard<std::mutex> Lock(FinalizedAllocsMutex);
auto *FA = FinalizedAllocInfos.Allocate<FinalizedAllocInfo>();
new (FA) FinalizedAllocInfo(

View File

@ -349,8 +349,10 @@ public:
}
SectionRange R(MachOContainerBlock->getSection());
G.allocActions().push_back(
{{RegisterActionAddr, R.getStart(), R.getSize()}, {}});
G.allocActions().push_back({cantFail(shared::WrapperFunctionCall::Create<
SPSArgList<SPSExecutorAddrRange>>(
RegisterActionAddr, R.getRange())),
{}});
return Error::success();
}

View File

@ -56,18 +56,15 @@ EPCEHFrameRegistrar::Create(ExecutionSession &ES) {
ExecutorAddr(DeregisterEHFrameWrapperFnAddr));
}
Error EPCEHFrameRegistrar::registerEHFrames(ExecutorAddr EHFrameSectionAddr,
size_t EHFrameSectionSize) {
return ES.callSPSWrapper<void(SPSExecutorAddr, uint64_t)>(
RegisterEHFrameWrapperFnAddr, EHFrameSectionAddr,
static_cast<uint64_t>(EHFrameSectionSize));
Error EPCEHFrameRegistrar::registerEHFrames(ExecutorAddrRange EHFrameSection) {
return ES.callSPSWrapper<void(SPSExecutorAddrRange)>(
RegisterEHFrameWrapperFnAddr, EHFrameSection);
}
Error EPCEHFrameRegistrar::deregisterEHFrames(ExecutorAddr EHFrameSectionAddr,
size_t EHFrameSectionSize) {
return ES.callSPSWrapper<void(SPSExecutorAddr, uint64_t)>(
DeregisterEHFrameWrapperFnAddr, EHFrameSectionAddr,
static_cast<uint64_t>(EHFrameSectionSize));
Error EPCEHFrameRegistrar::deregisterEHFrames(
ExecutorAddrRange EHFrameSection) {
return ES.callSPSWrapper<void(SPSExecutorAddrRange)>(
DeregisterEHFrameWrapperFnAddr, EHFrameSection);
}
} // end namespace orc

View File

@ -56,17 +56,7 @@ public:
}
// Transfer allocation actions.
// FIXME: Merge JITLink and ORC SupportFunctionCall and Action list types,
// turn this into a std::swap.
FR.Actions.reserve(G.allocActions().size());
for (auto &ActPair : G.allocActions())
FR.Actions.push_back({{ExecutorAddr(ActPair.Finalize.FnAddr),
{ExecutorAddr(ActPair.Finalize.CtxAddr),
ExecutorAddrDiff(ActPair.Finalize.CtxSize)}},
{ExecutorAddr(ActPair.Dealloc.FnAddr),
{ExecutorAddr(ActPair.Dealloc.CtxAddr),
ExecutorAddrDiff(ActPair.Dealloc.CtxSize)}}});
G.allocActions().clear();
std::swap(FR.Actions, G.allocActions());
Parent.EPC.callSPSWrapperAsync<
rt::SPSSimpleExecutorMemoryManagerFinalizeSignature>(

View File

@ -14,6 +14,8 @@
#define DEBUG_TYPE "orc"
using namespace llvm::orc::shared;
namespace llvm {
namespace orc {
@ -27,10 +29,8 @@ EPCGenericRTDyldMemoryManager::CreateWithDefaultBootstrapSymbols(
{SAs.Finalize, rt::SimpleExecutorMemoryManagerFinalizeWrapperName},
{SAs.Deallocate,
rt::SimpleExecutorMemoryManagerDeallocateWrapperName},
{SAs.RegisterEHFrame,
rt::RegisterEHFrameSectionCustomDirectWrapperName},
{SAs.DeregisterEHFrame,
rt::DeregisterEHFrameSectionCustomDirectWrapperName}}))
{SAs.RegisterEHFrame, rt::RegisterEHFrameSectionWrapperName},
{SAs.DeregisterEHFrame, rt::DeregisterEHFrameSectionWrapperName}}))
return std::move(Err);
return std::make_unique<EPCGenericRTDyldMemoryManager>(EPC, std::move(SAs));
}
@ -263,10 +263,12 @@ bool EPCGenericRTDyldMemoryManager::finalizeMemory(std::string *ErrMsg) {
for (auto &Frame : ObjAllocs.UnfinalizedEHFrames)
FR.Actions.push_back(
{{SAs.RegisterEHFrame,
{ExecutorAddr(Frame.Addr), ExecutorAddrDiff(Frame.Size)}},
{SAs.DeregisterEHFrame,
{ExecutorAddr(Frame.Addr), ExecutorAddrDiff(Frame.Size)}}});
{cantFail(
WrapperFunctionCall::Create<SPSArgList<SPSExecutorAddrRange>>(
SAs.RegisterEHFrame, Frame)),
cantFail(
WrapperFunctionCall::Create<SPSArgList<SPSExecutorAddrRange>>(
SAs.DeregisterEHFrame, Frame))});
// We'll also need to make an extra allocation for the eh-frame wrapper call
// arguments.

View File

@ -106,8 +106,8 @@ private:
auto HeaderContent = G.allocateString(
StringRef(reinterpret_cast<const char *>(&Hdr), sizeof(Hdr)));
return G.createContentBlock(HeaderSection, HeaderContent,
orc::ExecutorAddr(), 8, 0);
return G.createContentBlock(HeaderSection, HeaderContent, ExecutorAddr(), 8,
0);
}
static MaterializationUnit::Interface
@ -880,10 +880,13 @@ Error MachOPlatform::MachOPlatformPlugin::registerEHAndTLVSections(
if (auto *EHFrameSection = G.findSectionByName(EHFrameSectionName)) {
jitlink::SectionRange R(*EHFrameSection);
if (!R.empty())
G.allocActions().push_back({{MP.orc_rt_macho_register_ehframe_section,
R.getStart(), R.getSize()},
{MP.orc_rt_macho_deregister_ehframe_section,
R.getStart(), R.getSize()}});
G.allocActions().push_back(
{cantFail(
WrapperFunctionCall::Create<SPSArgList<SPSExecutorAddrRange>>(
MP.orc_rt_macho_register_ehframe_section, R.getRange())),
cantFail(
WrapperFunctionCall::Create<SPSArgList<SPSExecutorAddrRange>>(
MP.orc_rt_macho_deregister_ehframe_section, R.getRange()))});
}
// Get a pointer to the thread data section if there is one. It will be used
@ -913,10 +916,13 @@ Error MachOPlatform::MachOPlatformPlugin::registerEHAndTLVSections(
inconvertibleErrorCode());
G.allocActions().push_back(
{{MP.orc_rt_macho_register_thread_data_section, R.getStart(),
R.getSize()},
{MP.orc_rt_macho_deregister_thread_data_section, R.getStart(),
R.getSize()}});
{cantFail(
WrapperFunctionCall::Create<SPSArgList<SPSExecutorAddrRange>>(
MP.orc_rt_macho_register_thread_data_section, R.getRange())),
cantFail(
WrapperFunctionCall::Create<SPSArgList<SPSExecutorAddrRange>>(
MP.orc_rt_macho_deregister_thread_data_section,
R.getRange()))});
}
}
return Error::success();
@ -963,8 +969,10 @@ Error MachOPlatform::MachOPlatformPlugin::registerEHSectionsPhase1(
// Otherwise, add allocation actions to the graph to register eh-frames for
// this object.
G.allocActions().push_back(
{{orc_rt_macho_register_ehframe_section, R.getStart(), R.getSize()},
{orc_rt_macho_deregister_ehframe_section, R.getStart(), R.getSize()}});
{cantFail(WrapperFunctionCall::Create<SPSArgList<SPSExecutorAddrRange>>(
orc_rt_macho_register_ehframe_section, R.getRange())),
cantFail(WrapperFunctionCall::Create<SPSArgList<SPSExecutorAddrRange>>(
orc_rt_macho_deregister_ehframe_section, R.getRange()))});
return Error::success();
}

View File

@ -756,7 +756,7 @@ void EHFrameRegistrationPlugin::modifyPassConfig(
Error EHFrameRegistrationPlugin::notifyEmitted(
MaterializationResponsibility &MR) {
EHFrameRange EmittedRange;
ExecutorAddrRange EmittedRange;
{
std::lock_guard<std::mutex> Lock(EHFramePluginMutex);
@ -765,7 +765,7 @@ Error EHFrameRegistrationPlugin::notifyEmitted(
return Error::success();
EmittedRange = EHFrameRangeItr->second;
assert(EmittedRange.Addr && "eh-frame addr to register can not be null");
assert(EmittedRange.Start && "eh-frame addr to register can not be null");
InProcessLinks.erase(EHFrameRangeItr);
}
@ -773,7 +773,7 @@ Error EHFrameRegistrationPlugin::notifyEmitted(
[&](ResourceKey K) { EHFrameRanges[K].push_back(EmittedRange); }))
return Err;
return Registrar->registerEHFrames(EmittedRange.Addr, EmittedRange.Size);
return Registrar->registerEHFrames(EmittedRange);
}
Error EHFrameRegistrationPlugin::notifyFailed(
@ -784,7 +784,7 @@ Error EHFrameRegistrationPlugin::notifyFailed(
}
Error EHFrameRegistrationPlugin::notifyRemovingResources(ResourceKey K) {
std::vector<EHFrameRange> RangesToRemove;
std::vector<ExecutorAddrRange> RangesToRemove;
ES.runSessionLocked([&] {
auto I = EHFrameRanges.find(K);
@ -798,10 +798,9 @@ Error EHFrameRegistrationPlugin::notifyRemovingResources(ResourceKey K) {
while (!RangesToRemove.empty()) {
auto RangeToRemove = RangesToRemove.back();
RangesToRemove.pop_back();
assert(RangeToRemove.Addr && "Untracked eh-frame range must not be null");
Err = joinErrors(
std::move(Err),
Registrar->deregisterEHFrames(RangeToRemove.Addr, RangeToRemove.Size));
assert(RangeToRemove.Start && "Untracked eh-frame range must not be null");
Err = joinErrors(std::move(Err),
Registrar->deregisterEHFrames(RangeToRemove));
}
return Err;

View File

@ -36,10 +36,10 @@ const char *MemoryWriteUInt64sWrapperName =
"__llvm_orc_bootstrap_mem_write_uint64s_wrapper";
const char *MemoryWriteBuffersWrapperName =
"__llvm_orc_bootstrap_mem_write_buffers_wrapper";
const char *RegisterEHFrameSectionCustomDirectWrapperName =
"__llvm_orc_bootstrap_register_ehframe_section_custom_direct_wrapper";
const char *DeregisterEHFrameSectionCustomDirectWrapperName =
"__llvm_orc_bootstrap_deregister_ehframe_section_custom_direct_wrapper";
const char *RegisterEHFrameSectionWrapperName =
"__llvm_orc_bootstrap_register_ehframe_section_wrapper";
const char *DeregisterEHFrameSectionWrapperName =
"__llvm_orc_bootstrap_deregister_ehframe_section_wrapper";
const char *RunAsMainWrapperName = "__llvm_orc_bootstrap_run_as_main_wrapper";
} // end namespace rt

View File

@ -105,22 +105,25 @@ static void registerJITLoaderGDBImpl(const char *ObjAddr, size_t Size) {
extern "C" orc::shared::CWrapperFunctionResult
llvm_orc_registerJITLoaderGDBAllocAction(const char *Data, size_t Size) {
using namespace orc::shared;
return WrapperFunction<SPSError()>::handle(nullptr, 0,
[=]() -> Error {
registerJITLoaderGDBImpl(Data,
Size);
return Error::success();
})
return WrapperFunction<SPSError(SPSExecutorAddrRange)>::handle(
Data, Size,
[](ExecutorAddrRange R) {
registerJITLoaderGDBImpl(R.Start.toPtr<const char *>(),
R.size());
return Error::success();
})
.release();
}
extern "C" orc::shared::CWrapperFunctionResult
llvm_orc_registerJITLoaderGDBWrapper(const char *Data, uint64_t Size) {
using namespace orc::shared;
return WrapperFunction<void(SPSExecutorAddrRange)>::handle(
return WrapperFunction<SPSError(SPSExecutorAddrRange)>::handle(
Data, Size,
[](ExecutorAddrRange R) {
registerJITLoaderGDBImpl(R.Start.toPtr<char *>(), R.size());
registerJITLoaderGDBImpl(R.Start.toPtr<const char *>(),
R.size());
return Error::success();
})
.release();
}

View File

@ -71,11 +71,10 @@ void addTo(StringMap<ExecutorAddr> &M) {
shared::SPSMemoryAccessUInt64Write>);
M[rt::MemoryWriteBuffersWrapperName] =
ExecutorAddr::fromPtr(&writeBuffersWrapper);
M[rt::RegisterEHFrameSectionCustomDirectWrapperName] = ExecutorAddr::fromPtr(
&llvm_orc_registerEHFrameSectionCustomDirectWrapper);
M[rt::DeregisterEHFrameSectionCustomDirectWrapperName] =
ExecutorAddr::fromPtr(
&llvm_orc_deregisterEHFrameSectionCustomDirectWrapper);
M[rt::RegisterEHFrameSectionWrapperName] =
ExecutorAddr::fromPtr(&llvm_orc_registerEHFrameSectionWrapper);
M[rt::DeregisterEHFrameSectionWrapperName] =
ExecutorAddr::fromPtr(&llvm_orc_deregisterEHFrameSectionWrapper);
M[rt::RunAsMainWrapperName] = ExecutorAddr::fromPtr(&runAsMainWrapper);
}

View File

@ -158,42 +158,26 @@ Error deregisterEHFrameSection(const void *EHFrameSectionAddr,
} // end namespace orc
} // end namespace llvm
extern "C" llvm::orc::shared::CWrapperFunctionResult
llvm_orc_registerEHFrameSectionCustomDirectWrapper(
const char *EHFrameSectionAddr, uint64_t Size) {
if (auto Err = registerEHFrameSection(EHFrameSectionAddr, Size))
return WrapperFunctionResult::createOutOfBandError(toString(std::move(Err)))
.release();
return llvm::orc::shared::CWrapperFunctionResult();
static Error registerEHFrameWrapper(ExecutorAddrRange EHFrame) {
return llvm::orc::registerEHFrameSection(EHFrame.Start.toPtr<const void *>(),
EHFrame.size());
}
extern "C" llvm::orc::shared::CWrapperFunctionResult
llvm_orc_deregisterEHFrameSectionCustomDirectWrapper(
const char *EHFrameSectionAddr, uint64_t Size) {
if (auto Err = deregisterEHFrameSection(EHFrameSectionAddr, Size))
return WrapperFunctionResult::createOutOfBandError(toString(std::move(Err)))
.release();
return llvm::orc::shared::CWrapperFunctionResult();
}
static Error registerEHFrameWrapper(ExecutorAddr Addr, uint64_t Size) {
return llvm::orc::registerEHFrameSection(Addr.toPtr<const void *>(), Size);
}
static Error deregisterEHFrameWrapper(ExecutorAddr Addr, uint64_t Size) {
return llvm::orc::deregisterEHFrameSection(Addr.toPtr<const void *>(), Size);
static Error deregisterEHFrameWrapper(ExecutorAddrRange EHFrame) {
return llvm::orc::deregisterEHFrameSection(
EHFrame.Start.toPtr<const void *>(), EHFrame.size());
}
extern "C" orc::shared::CWrapperFunctionResult
llvm_orc_registerEHFrameSectionWrapper(const char *Data, uint64_t Size) {
return WrapperFunction<SPSError(SPSExecutorAddr, uint64_t)>::handle(
return WrapperFunction<SPSError(SPSExecutorAddrRange)>::handle(
Data, Size, registerEHFrameWrapper)
.release();
}
extern "C" orc::shared::CWrapperFunctionResult
llvm_orc_deregisterEHFrameSectionWrapper(const char *Data, uint64_t Size) {
return WrapperFunction<SPSError(SPSExecutorAddr, uint64_t)>::handle(
return WrapperFunction<SPSError(SPSExecutorAddrRange)>::handle(
Data, Size, deregisterEHFrameWrapper)
.release();
}

View File

@ -35,7 +35,7 @@ Expected<ExecutorAddr> SimpleExecutorMemoryManager::allocate(uint64_t Size) {
Error SimpleExecutorMemoryManager::finalize(tpctypes::FinalizeRequest &FR) {
ExecutorAddr Base(~0ULL);
std::vector<tpctypes::WrapperFunctionCall> DeallocationActions;
std::vector<shared::WrapperFunctionCall> DeallocationActions;
size_t SuccessfulFinalizationActions = 0;
if (FR.Segments.empty()) {
@ -52,8 +52,8 @@ Error SimpleExecutorMemoryManager::finalize(tpctypes::FinalizeRequest &FR) {
Base = std::min(Base, Seg.Addr);
for (auto &ActPair : FR.Actions)
if (ActPair.Deallocate.Func)
DeallocationActions.push_back(ActPair.Deallocate);
if (ActPair.Dealloc)
DeallocationActions.push_back(ActPair.Dealloc);
// Get the Allocation for this finalization.
size_t AllocSize = 0;
@ -96,7 +96,7 @@ Error SimpleExecutorMemoryManager::finalize(tpctypes::FinalizeRequest &FR) {
while (SuccessfulFinalizationActions)
Err =
joinErrors(std::move(Err), FR.Actions[--SuccessfulFinalizationActions]
.Deallocate.runWithSPSRet());
.Dealloc.runWithSPSRetErrorMerged());
// Deallocate memory.
sys::MemoryBlock MB(AllocToDestroy.first, AllocToDestroy.second.Size);
@ -139,7 +139,7 @@ Error SimpleExecutorMemoryManager::finalize(tpctypes::FinalizeRequest &FR) {
// Run finalization actions.
for (auto &ActPair : FR.Actions) {
if (auto Err = ActPair.Finalize.runWithSPSRet())
if (auto Err = ActPair.Finalize.runWithSPSRetErrorMerged())
return BailOut(std::move(Err));
++SuccessfulFinalizationActions;
}
@ -212,7 +212,7 @@ Error SimpleExecutorMemoryManager::deallocateImpl(void *Base, Allocation &A) {
while (!A.DeallocationActions.empty()) {
Err = joinErrors(std::move(Err),
A.DeallocationActions.back().runWithSPSRet());
A.DeallocationActions.back().runWithSPSRetErrorMerged());
A.DeallocationActions.pop_back();
}

View File

@ -398,7 +398,7 @@ class JITLinkSlabAllocator final : public JITLinkMemoryManager {
private:
struct FinalizedAllocInfo {
sys::MemoryBlock Mem;
std::vector<AllocActionCall> DeallocActions;
std::vector<shared::WrapperFunctionCall> DeallocActions;
};
public:

View File

@ -22,8 +22,12 @@ namespace {
orc::shared::CWrapperFunctionResult incrementWrapper(const char *ArgData,
size_t ArgSize) {
return WrapperFunction<void(SPSExecutorAddr)>::handle(
ArgData, ArgSize, [](ExecutorAddr A) { *A.toPtr<int *>() += 1; })
return WrapperFunction<SPSError(SPSExecutorAddr)>::handle(
ArgData, ArgSize,
[](ExecutorAddr A) -> Error {
*A.toPtr<int *>() += 1;
return Error::success();
})
.release();
}
@ -37,15 +41,7 @@ TEST(SimpleExecutorMemoryManagerTest, AllocFinalizeFree) {
std::string HW = "Hello, world!";
int FinalizeCounter = 0;
auto FinalizeCounterAddrArgBuffer =
orc::shared::detail::serializeViaSPSToWrapperFunctionResult<
SPSArgList<SPSExecutorAddr>>(ExecutorAddr::fromPtr(&FinalizeCounter));
int DeallocateCounter = 0;
auto DeallocateCounterAddrArgBuffer =
orc::shared::detail::serializeViaSPSToWrapperFunctionResult<
SPSArgList<SPSExecutorAddr>>(
ExecutorAddr::fromPtr(&DeallocateCounter));
tpctypes::FinalizeRequest FR;
FR.Segments.push_back(
@ -55,13 +51,13 @@ TEST(SimpleExecutorMemoryManagerTest, AllocFinalizeFree) {
{HW.data(), HW.size() + 1}});
FR.Actions.push_back(
{/* Finalize: */
{ExecutorAddr::fromPtr(incrementWrapper),
{ExecutorAddr::fromPtr(FinalizeCounterAddrArgBuffer.data()),
ExecutorAddrDiff(FinalizeCounterAddrArgBuffer.size())}},
cantFail(WrapperFunctionCall::Create<SPSArgList<SPSExecutorAddr>>(
ExecutorAddr::fromPtr(incrementWrapper),
ExecutorAddr::fromPtr(&FinalizeCounter))),
/* Deallocate: */
{ExecutorAddr::fromPtr(incrementWrapper),
{ExecutorAddr::fromPtr(DeallocateCounterAddrArgBuffer.data()),
ExecutorAddrDiff(DeallocateCounterAddrArgBuffer.size())}}});
cantFail(WrapperFunctionCall::Create<SPSArgList<SPSExecutorAddr>>(
ExecutorAddr::fromPtr(incrementWrapper),
ExecutorAddr::fromPtr(&DeallocateCounter)))});
EXPECT_EQ(FinalizeCounter, 0);
EXPECT_EQ(DeallocateCounter, 0);