[ORC] Rewrite the VSO symbol table yet again. Update related utilities.

VSOs now track dependencies for materializing symbols. Each symbol must have its
dependencies registered with the VSO prior to finalization. Usually this will
involve registering the dependencies returned in
AsynchronousSymbolQuery::ResolutionResults for queries made while linking the
symbols being materialized.

Queries against symbols are notified that a symbol is ready once it and all of
its transitive dependencies are finalized, allowing compilation work to be
broken up and moved between threads without queries returning until their
symbols fully safe to access / execute.

Related utilities (VSO, MaterializationUnit, MaterializationResponsibility) are
updated to support dependence tracking and more explicitly track responsibility
for symbols from the point of definition until they are finalized.

llvm-svn: 332541
This commit is contained in:
Lang Hames 2018-05-16 22:24:30 +00:00
parent 2dc00a64a2
commit d261e1258c
19 changed files with 1549 additions and 960 deletions

View File

@ -48,6 +48,7 @@ private:
public:
KaleidoscopeJIT()
: Resolver(createLegacyLookupResolver(
ES,
[this](const std::string &Name) -> JITSymbol {
if (auto Sym = CompileLayer.findSymbol(Name, false))
return Sym;

View File

@ -58,6 +58,7 @@ private:
public:
KaleidoscopeJIT()
: Resolver(createLegacyLookupResolver(
ES,
[this](const std::string &Name) -> JITSymbol {
if (auto Sym = OptimizeLayer.findSymbol(Name, false))
return Sym;

View File

@ -98,6 +98,7 @@ public:
// Build a resolver and associate it with the new key.
Resolvers[K] = createLegacyLookupResolver(
ES,
[this](const std::string &Name) -> JITSymbol {
if (auto Sym = CompileLayer.findSymbol(Name, false))
return Sym;

View File

@ -91,6 +91,7 @@ private:
public:
KaleidoscopeJIT()
: Resolver(createLegacyLookupResolver(
ES,
[this](const std::string &Name) -> JITSymbol {
if (auto Sym = IndirectStubsMgr->findStub(Name, false))
return Sym;

View File

@ -97,6 +97,7 @@ private:
public:
KaleidoscopeJIT(MyRemote &Remote)
: Resolver(createLegacyLookupResolver(
ES,
[this](const std::string &Name) -> JITSymbol {
if (auto Sym = IndirectStubsMgr->findStub(Name, false))
return Sym;

View File

@ -45,6 +45,7 @@ public:
KaleidoscopeJIT()
: Resolver(createLegacyLookupResolver(
ES,
[this](const std::string &Name) {
return ObjectLayer.findSymbol(Name, true);
},

View File

@ -174,6 +174,9 @@ public:
/// Return the flags for this symbol.
JITSymbolFlags getFlags() const { return Flags; }
/// Set the flags for this symbol.
void setFlags(JITSymbolFlags Flags) { this->Flags = std::move(Flags); }
private:
JITTargetAddress Address = 0;
JITSymbolFlags Flags;

View File

@ -490,10 +490,11 @@ private:
return LD.BackingResolver->lookupFlags(SymbolFlags,
*NotFoundViaLegacyLookup);
},
[&LD, LegacyLookup](std::shared_ptr<AsynchronousSymbolQuery> Query,
SymbolNameSet Symbols) {
[this, &LD,
LegacyLookup](std::shared_ptr<AsynchronousSymbolQuery> Query,
SymbolNameSet Symbols) {
auto NotFoundViaLegacyLookup =
lookupWithLegacyFn(*Query, Symbols, LegacyLookup);
lookupWithLegacyFn(ES, *Query, Symbols, LegacyLookup);
return LD.BackingResolver->lookup(Query, NotFoundViaLegacyLookup);
});
@ -647,10 +648,10 @@ private:
return LD.BackingResolver->lookupFlags(SymbolFlags,
*NotFoundViaLegacyLookup);
},
[&LD, LegacyLookup](std::shared_ptr<AsynchronousSymbolQuery> Q,
SymbolNameSet Symbols) {
[this, &LD, LegacyLookup](std::shared_ptr<AsynchronousSymbolQuery> Q,
SymbolNameSet Symbols) {
auto NotFoundViaLegacyLookup =
lookupWithLegacyFn(*Q, Symbols, LegacyLookup);
lookupWithLegacyFn(ES, *Q, Symbols, LegacyLookup);
return LD.BackingResolver->lookup(Q,
std::move(NotFoundViaLegacyLookup));
});

View File

@ -27,6 +27,10 @@ namespace llvm {
namespace orc {
// Forward declare some classes.
class AsynchronousSymbolQuery;
class ExecutionSession;
class MaterializationUnit;
class MaterializationResponsibility;
class VSO;
/// VModuleKey provides a unique identifier (allocated and managed by
@ -55,6 +59,13 @@ raw_ostream &operator<<(raw_ostream &OS, const SymbolFlagsMap &Symbols);
/// A base class for materialization failures that allows the failing
/// symbols to be obtained for logging.
using SymbolDependenceMap = std::map<VSO *, SymbolNameSet>;
/// Render a SymbolDependendeMap.
raw_ostream &operator<<(raw_ostream &OS, const SymbolDependenceMap &Deps);
/// A base class for materialization failures that allows the failing
/// symbols to be obtained for logging.
class FailedToMaterialize : public ErrorInfo<FailedToMaterialize> {
public:
static char ID;
@ -91,14 +102,252 @@ private:
SymbolNameSet Symbols;
};
/// Tracks responsibility for materialization, and mediates interactions between
/// MaterializationUnits and VSOs.
///
/// An instance of this class is passed to MaterializationUnits when their
/// materialize method is called. It allows MaterializationUnits to resolve and
/// finalize symbols, or abandon materialization by notifying any unmaterialized
/// symbols of an error.
class MaterializationResponsibility {
public:
/// Create a MaterializationResponsibility for the given VSO and
/// initial symbols.
MaterializationResponsibility(VSO &V, SymbolFlagsMap SymbolFlags);
MaterializationResponsibility(MaterializationResponsibility &&) = default;
MaterializationResponsibility &
operator=(MaterializationResponsibility &&) = default;
/// Destruct a MaterializationResponsibility instance. In debug mode
/// this asserts that all symbols being tracked have been either
/// finalized or notified of an error.
~MaterializationResponsibility();
/// Returns the target VSO that these symbols are being materialized
/// into.
const VSO &getTargetVSO() const { return V; }
/// Resolves the given symbols. Individual calls to this method may
/// resolve a subset of the symbols, but all symbols must have been
/// resolved prior to calling finalize.
void resolve(const SymbolMap &Symbols);
/// Finalizes all symbols tracked by this instance.
void finalize();
/// Adds new symbols to the VSO and this responsibility instance.
/// VSO entries start out in the materializing state.
///
/// This method can be used by materialization units that want to add
/// additional symbols at materialization time (e.g. stubs, compile
/// callbacks, metadata).
Error defineMaterializing(const SymbolFlagsMap &SymbolFlags);
/// Notify all unfinalized symbols that an error has occurred.
/// This will remove all symbols covered by this MaterializationResponsibilty
/// from V, and send an error to any queries waiting on these symbols.
void failMaterialization(std::function<Error()> GenerateError);
/// Transfers responsibility to the given MaterializationUnit for all
/// symbols defined by that MaterializationUnit. This allows
/// materializers to break up work based on run-time information (e.g.
/// by introspecting which symbols have actually been looked up and
/// materializing only those).
void delegate(std::unique_ptr<MaterializationUnit> MU);
/// Add dependencies for the symbols in this dylib.
void addDependencies(const SymbolDependenceMap &Dependencies);
private:
VSO &V;
SymbolFlagsMap SymbolFlags;
};
/// A MaterializationUnit represents a set of symbol definitions that can
/// be materialized as a group, or individually discarded (when
/// overriding definitions are encountered).
///
/// MaterializationUnits are used when providing lazy definitions of symbols to
/// VSOs. The VSO will call materialize when the address of a symbol is
/// requested via the lookup method. The VSO will call discard if a stronger
/// definition is added or already present.
class MaterializationUnit {
public:
MaterializationUnit(SymbolFlagsMap SymbolFlags)
: SymbolFlags(std::move(SymbolFlags)) {}
virtual ~MaterializationUnit() {}
/// Return the set of symbols that this source provides.
const SymbolFlagsMap &getSymbols() const { return SymbolFlags; }
/// Called by materialization dispatchers (see
/// ExecutionSession::DispatchMaterializationFunction) to trigger
/// materialization of this MaterializationUnit.
void doMaterialize(VSO &V) {
materialize(MaterializationResponsibility(V, std::move(SymbolFlags)));
}
/// Called by VSOs to notify MaterializationUnits that the given symbol has
/// been overridden.
void doDiscard(const VSO &V, SymbolStringPtr Name) {
SymbolFlags.erase(Name);
discard(V, std::move(Name));
}
private:
virtual void anchor();
/// Implementations of this method should materialize all symbols
/// in the materialzation unit, except for those that have been
/// previously discarded.
virtual void materialize(MaterializationResponsibility R) = 0;
/// Implementations of this method should discard the given symbol
/// from the source (e.g. if the source is an LLVM IR Module and the
/// symbol is a function, delete the function body or mark it available
/// externally).
virtual void discard(const VSO &V, SymbolStringPtr Name) = 0;
SymbolFlagsMap SymbolFlags;
};
/// A MaterializationUnit implementation for pre-existing absolute symbols.
///
/// All symbols will be resolved and marked ready as soon as the unit is
/// materialized.
class AbsoluteSymbolsMaterializationUnit : public MaterializationUnit {
public:
AbsoluteSymbolsMaterializationUnit(SymbolMap Symbols);
private:
void materialize(MaterializationResponsibility R) override;
void discard(const VSO &V, SymbolStringPtr Name) override;
static SymbolFlagsMap extractFlags(const SymbolMap &Symbols);
SymbolMap Symbols;
};
/// Create an AbsoluteSymbolsMaterializationUnit with the given symbols.
/// Useful for inserting absolute symbols into a VSO. E.g.:
/// \code{.cpp}
/// VSO &V = ...;
/// SymbolStringPtr Foo = ...;
/// JITEvaluatedSymbol FooSym = ...;
/// if (auto Err = V.define(absoluteSymbols({{Foo, FooSym}})))
/// return Err;
/// \endcode
///
inline std::unique_ptr<AbsoluteSymbolsMaterializationUnit>
absoluteSymbols(SymbolMap Symbols) {
return llvm::make_unique<AbsoluteSymbolsMaterializationUnit>(
std::move(Symbols));
}
/// Base utilities for ExecutionSession.
class ExecutionSessionBase {
public:
/// For reporting errors.
using ErrorReporter = std::function<void(Error)>;
/// For dispatching MaterializationUnit::materialize calls.
using DispatchMaterializationFunction =
std::function<void(VSO &V, std::unique_ptr<MaterializationUnit> MU)>;
/// Construct an ExecutionSessionBase.
///
/// SymbolStringPools may be shared between ExecutionSessions.
ExecutionSessionBase(std::shared_ptr<SymbolStringPool> SSP = nullptr)
: SSP(SSP ? std::move(SSP) : std::make_shared<SymbolStringPool>()) {}
/// Returns the SymbolStringPool for this ExecutionSession.
SymbolStringPool &getSymbolStringPool() const { return *SSP; }
/// Run the given lambda with the session mutex locked.
template <typename Func> auto runSessionLocked(Func &&F) -> decltype(F()) {
std::lock_guard<std::recursive_mutex> Lock(SessionMutex);
return F();
}
/// Set the error reporter function.
ExecutionSessionBase &setErrorReporter(ErrorReporter ReportError) {
this->ReportError = std::move(ReportError);
return *this;
}
/// Set the materialization dispatch function.
ExecutionSessionBase &setDispatchMaterialization(
DispatchMaterializationFunction DispatchMaterialization) {
this->DispatchMaterialization = std::move(DispatchMaterialization);
return *this;
}
/// Report a error for this execution session.
///
/// Unhandled errors can be sent here to log them.
void reportError(Error Err) { ReportError(std::move(Err)); }
/// Allocate a module key for a new module to add to the JIT.
VModuleKey allocateVModule() { return ++LastKey; }
/// Return a module key to the ExecutionSession so that it can be
/// re-used. This should only be done once all resources associated
/// with the original key have been released.
void releaseVModule(VModuleKey Key) { /* FIXME: Recycle keys */
}
/// Cause the given query to fail with the given Error.
///
/// This should only be used by legacy APIs and will be deprecated in the
/// future.
void failQuery(AsynchronousSymbolQuery &Q, Error Err);
/// Materialize the given unit.
void dispatchMaterialization(VSO &V,
std::unique_ptr<MaterializationUnit> MU) {
DispatchMaterialization(V, std::move(MU));
}
private:
static void logErrorsToStdErr(Error Err) {
logAllUnhandledErrors(std::move(Err), errs(), "JIT session error: ");
}
static void
materializeOnCurrentThread(VSO &V, std::unique_ptr<MaterializationUnit> MU) {
MU->doMaterialize(V);
}
mutable std::recursive_mutex SessionMutex;
std::shared_ptr<SymbolStringPool> SSP;
VModuleKey LastKey = 0;
ErrorReporter ReportError = logErrorsToStdErr;
DispatchMaterializationFunction DispatchMaterialization =
materializeOnCurrentThread;
};
/// A symbol query that returns results via a callback when results are
/// ready.
///
/// makes a callback when all symbols are available.
class AsynchronousSymbolQuery {
friend class ExecutionSessionBase;
friend class VSO;
public:
class ResolutionResult {
public:
ResolutionResult(SymbolMap Symbols, const SymbolDependenceMap &Dependencies)
: Symbols(std::move(Symbols)), Dependencies(Dependencies) {}
SymbolMap Symbols;
const SymbolDependenceMap &Dependencies;
};
/// Callback to notify client that symbols have been resolved.
using SymbolsResolvedCallback = std::function<void(Expected<SymbolMap>)>;
using SymbolsResolvedCallback =
std::function<void(Expected<ResolutionResult>)>;
/// Callback to notify client that symbols are ready for execution.
using SymbolsReadyCallback = std::function<void(Error)>;
@ -109,36 +358,45 @@ public:
SymbolsResolvedCallback NotifySymbolsResolved,
SymbolsReadyCallback NotifySymbolsReady);
/// Notify client that the query failed.
///
/// If the notify-resolved callback has not been made yet, then it is called
/// with the given error, and the notify-finalized callback is never made.
///
/// If the notify-resolved callback has already been made then then the
/// notify-finalized callback is called with the given error.
///
/// It is illegal to call setFailed after both callbacks have been made.
void notifyMaterializationFailed(Error Err);
/// Set the resolved symbol information for the given symbol name.
void resolve(const SymbolStringPtr &Name, JITEvaluatedSymbol Sym);
/// Returns true if all symbols covered by this query have been
/// resolved.
bool isFullyResolved() const { return NotYetResolvedCount == 0; }
/// Call the NotifySymbolsResolved callback.
///
/// If this symbol was the last one not resolved, this will trigger a call to
/// the notify-finalized callback passing the completed sybol map.
void resolve(SymbolStringPtr Name, JITEvaluatedSymbol Sym);
/// This should only be called if all symbols covered by the query have been
/// resolved.
void handleFullyResolved();
/// Notify the query that a requested symbol is ready for execution.
void notifySymbolReady();
/// Returns true if all symbols covered by this query are ready.
bool isFullyReady() const { return NotYetReadyCount == 0; }
/// Calls the NotifySymbolsReady callback.
///
/// This decrements the query's internal count of not-yet-ready symbols. If
/// this call to notifySymbolFinalized sets the counter to zero, it will call
/// the notify-finalized callback with Error::success as the value.
void finalizeSymbol();
/// This should only be called if all symbols covered by this query are ready.
void handleFullyReady();
private:
SymbolMap Symbols;
size_t OutstandingResolutions = 0;
size_t OutstandingFinalizations = 0;
void addQueryDependence(VSO &V, SymbolStringPtr Name);
void removeQueryDependence(VSO &V, const SymbolStringPtr &Name);
void handleFailed(Error Err);
void detach();
SymbolsResolvedCallback NotifySymbolsResolved;
SymbolsReadyCallback NotifySymbolsReady;
SymbolDependenceMap QueryRegistrations;
SymbolMap ResolvedSymbols;
size_t NotYetResolvedCount;
size_t NotYetReadyCount;
};
/// SymbolResolver is a composable interface for looking up symbol flags
@ -206,311 +464,191 @@ createSymbolResolver(LookupFlagsFn &&LookupFlags, LookupFn &&Lookup) {
std::forward<LookupFlagsFn>(LookupFlags), std::forward<LookupFn>(Lookup));
}
/// Tracks responsibility for materialization.
/// A symbol table that supports asynchoronous symbol queries.
///
/// An instance of this class is passed to MaterializationUnits when their
/// materialize method is called. It allows MaterializationUnits to resolve and
/// finalize symbols, or abandon materialization by notifying any unmaterialized
/// symbols of an error.
class MaterializationResponsibility {
public:
/// Create a MaterializationResponsibility for the given VSO and
/// initial symbols.
MaterializationResponsibility(VSO &V, SymbolFlagsMap SymbolFlags);
MaterializationResponsibility(MaterializationResponsibility &&) = default;
MaterializationResponsibility &
operator=(MaterializationResponsibility &&) = default;
/// Destruct a MaterializationResponsibility instance. In debug mode
/// this asserts that all symbols being tracked have been either
/// finalized or notified of an error.
~MaterializationResponsibility();
/// Returns the target VSO that these symbols are being materialized
/// into.
const VSO &getTargetVSO() const { return V; }
/// Resolves the given symbols. Individual calls to this method may
/// resolve a subset of the symbols, but all symbols must have been
/// resolved prior to calling finalize.
void resolve(const SymbolMap &Symbols);
/// Finalizes all symbols tracked by this instance.
void finalize();
/// Notify all unfinalized symbols that an error has occurred.
/// This method should be called if materialization of any symbol is
/// abandoned.
void notifyMaterializationFailed();
/// Transfers responsibility for the given symbols to a new
/// MaterializationResponsibility class. This is useful if a
/// MaterializationUnit wants to transfer responsibility for a subset
/// of symbols to another MaterializationUnit or utility.
MaterializationResponsibility delegate(SymbolNameSet Symbols);
private:
VSO &V;
SymbolFlagsMap SymbolFlags;
};
/// A MaterializationUnit represents a set of symbol definitions that can
/// be materialized as a group, or individually discarded (when
/// overriding definitions are encountered).
///
/// MaterializationUnits are used when providing lazy definitions of symbols to
/// VSOs. The VSO will call materialize when the address of a symbol is
/// requested via the lookup method. The VSO will call discard if a stronger
/// definition is added or already present.
class MaterializationUnit {
public:
virtual ~MaterializationUnit() {}
/// Return the set of symbols that this source provides.
virtual SymbolFlagsMap getSymbols() = 0;
/// Implementations of this method should materialize all symbols
/// in the materialzation unit, except for those that have been
/// previously discarded.
virtual void materialize(MaterializationResponsibility R) = 0;
/// Implementations of this method should discard the given symbol
/// from the source (e.g. if the source is an LLVM IR Module and the
/// symbol is a function, delete the function body or mark it available
/// externally).
virtual void discard(const VSO &V, SymbolStringPtr Name) = 0;
private:
virtual void anchor();
};
/// Represents a dynamic linkage unit in a JIT process.
///
/// VSO acts as a symbol table (symbol definitions can be set and the dylib
/// queried to find symbol addresses) and as a key for tracking resources
/// (since a VSO's address is fixed).
/// Represents a virtual shared object. Instances can not be copied or moved, so
/// their addresses may be used as keys for resource management.
/// VSO state changes must be made via an ExecutionSession to guarantee that
/// they are synchronized with respect to other VSO operations.
class VSO {
friend class AsynchronousSymbolQuery;
friend class ExecutionSession;
friend class MaterializationResponsibility;
public:
enum RelativeLinkageStrength {
NewDefinitionIsStronger,
DuplicateDefinition,
ExistingDefinitionIsStronger
};
using AsynchronousSymbolQuerySet =
std::set<std::shared_ptr<AsynchronousSymbolQuery>>;
using SetDefinitionsResult =
std::map<SymbolStringPtr, RelativeLinkageStrength>;
using MaterializationUnitList =
std::vector<std::unique_ptr<MaterializationUnit>>;
struct Materializer {
public:
Materializer(std::unique_ptr<MaterializationUnit> MU,
MaterializationResponsibility R);
void operator()();
private:
std::unique_ptr<MaterializationUnit> MU;
MaterializationResponsibility R;
};
using MaterializerList = std::vector<Materializer>;
struct LookupResult {
MaterializerList Materializers;
SymbolNameSet UnresolvedSymbols;
};
VSO() = default;
using VSOList = std::vector<VSO *>;
VSO(const VSO &) = delete;
VSO &operator=(const VSO &) = delete;
VSO(VSO &&) = delete;
VSO &operator=(VSO &&) = delete;
/// Compare new linkage with existing linkage.
static RelativeLinkageStrength
compareLinkage(Optional<JITSymbolFlags> OldFlags, JITSymbolFlags NewFlags);
/// Get the name for this VSO.
const std::string &getName() const { return VSOName; }
/// Compare new linkage with an existing symbol's linkage.
RelativeLinkageStrength compareLinkage(SymbolStringPtr Name,
JITSymbolFlags NewFlags) const;
/// Get a reference to the ExecutionSession for this VSO.
ExecutionSessionBase &getExecutionSession() const { return ES; }
/// Adds the given symbols to the mapping as resolved, finalized
/// symbols.
/// Define all symbols provided by the materialization unit to be part
/// of the given VSO.
template <typename UniquePtrToMaterializationUnit>
typename std::enable_if<
std::is_convertible<
typename std::decay<UniquePtrToMaterializationUnit>::type,
std::unique_ptr<MaterializationUnit>>::value,
Error>::type
define(UniquePtrToMaterializationUnit &&MU) {
return ES.runSessionLocked([&, this]() -> Error {
assert(MU && "Can't define with a null MU");
if (auto Err = defineImpl(*MU))
return Err;
/// defineImpl succeeded.
auto UMI = std::make_shared<UnmaterializedInfo>(std::move(MU));
for (auto &KV : UMI->MU->getSymbols())
UnmaterializedInfos[KV.first] = UMI;
return Error::success();
});
}
/// Define a set of symbols already in the materializing state.
Error defineMaterializing(const SymbolFlagsMap &SymbolFlags);
/// Replace the definition of a set of materializing symbols with a new
/// MaterializationUnit.
///
/// FIXME: We can take this by const-ref once symbol-based laziness is
/// removed.
Error define(SymbolMap NewSymbols);
/// All symbols being replaced must be in the materializing state. If any
/// symbol being replaced has pending queries then the MU will be returned
/// for materialization. Otherwise it will be stored in the VSO and all
/// symbols covered by MU moved back to the lazy state.
void replace(std::unique_ptr<MaterializationUnit> MU);
/// Adds the given symbols to the mapping as lazy symbols.
Error defineLazy(std::unique_ptr<MaterializationUnit> Source);
/// Record dependencies between symbols in this VSO and symbols in
/// other VSOs.
void addDependencies(const SymbolFlagsMap &Dependents,
const SymbolDependenceMap &Dependencies);
/// Look up the flags for the given symbols.
/// Resolve the given symbols.
///
/// Returns the flags for the give symbols, together with the set of symbols
/// not found.
SymbolNameSet lookupFlags(SymbolFlagsMap &Flags, SymbolNameSet Symbols);
/// Apply the given query to the given symbols in this VSO.
///
/// For symbols in this VSO that have already been materialized, their address
/// will be set in the query immediately.
///
/// For symbols in this VSO that have not been materialized, the query will be
/// recorded and the source for those symbols (plus the set of symbols to be
/// materialized by that source) will be returned as the MaterializationWork
/// field of the LookupResult.
///
/// Any symbols not found in this VSO will be returned in the
/// UnresolvedSymbols field of the LookupResult.
LookupResult lookup(std::shared_ptr<AsynchronousSymbolQuery> Query,
SymbolNameSet Symbols);
private:
/// Add the given symbol/address mappings to the dylib, but do not
/// mark the symbols as finalized yet.
void resolve(const SymbolMap &SymbolValues);
/// Returns the list of queries that become fully resolved as a consequence of
/// this operation.
void resolve(const SymbolMap &Resolved);
/// Finalize the given symbols.
void finalize(const SymbolNameSet &SymbolsToFinalize);
///
/// Returns the list of queries that become fully ready as a consequence of
/// this operation.
void finalize(const SymbolFlagsMap &Finalized);
/// Notify the VSO that the given symbols failed to materialized.
void notifyMaterializationFailed(const SymbolNameSet &Names);
/// Fail to materialize the given symbols.
///
/// Returns the list of queries that fail as a consequence.
void notifyFailed(const SymbolFlagsMap &Failed,
std::function<Error()> GenerateError);
class UnmaterializedInfo {
public:
UnmaterializedInfo(std::unique_ptr<MaterializationUnit> MU);
void discard(VSO &V, SymbolStringPtr Name);
/// Search the given VSO for the symbols in Symbols. If found, store
/// the flags for each symbol in Flags. Returns any unresolved symbols.
SymbolNameSet lookupFlags(SymbolFlagsMap &Flags, const SymbolNameSet &Names);
/// Search the given VSOs in order for the symbols in Symbols. Results
/// (once they become available) will be returned via the given Query.
///
/// If any symbol is not found then the unresolved symbols will be returned,
/// and the query will not be applied. The Query is not failed and can be
/// re-used in a subsequent lookup once the symbols have been added, or
/// manually failed.
SymbolNameSet lookup(std::shared_ptr<AsynchronousSymbolQuery> Q,
SymbolNameSet Names);
/// Dump current VSO state to OS.
void dump(raw_ostream &OS);
private:
using AsynchronousSymbolQueryList =
std::vector<std::shared_ptr<AsynchronousSymbolQuery>>;
struct UnmaterializedInfo {
UnmaterializedInfo(std::unique_ptr<MaterializationUnit> MU)
: MU(std::move(MU)) {}
std::unique_ptr<MaterializationUnit> MU;
SymbolFlagsMap Symbols;
};
using UnmaterializedInfoList = std::list<UnmaterializedInfo>;
using UnmaterializedInfosMap =
std::map<SymbolStringPtr, std::shared_ptr<UnmaterializedInfo>>;
using UnmaterializedInfoIterator = UnmaterializedInfoList::iterator;
class MaterializingInfo {
public:
using QueryList = std::vector<std::shared_ptr<AsynchronousSymbolQuery>>;
QueryList PendingResolution;
QueryList PendingFinalization;
struct MaterializingInfo {
AsynchronousSymbolQueryList PendingQueries;
SymbolDependenceMap Dependants;
SymbolDependenceMap UnfinalizedDependencies;
bool IsFinalized = false;
};
using MaterializingInfoMap = std::map<SymbolStringPtr, MaterializingInfo>;
using MaterializingInfosMap = std::map<SymbolStringPtr, MaterializingInfo>;
using MaterializingInfoIterator = MaterializingInfoMap::iterator;
VSO(ExecutionSessionBase &ES, std::string Name)
: ES(ES), VSOName(std::move(Name)) {}
class SymbolTableEntry {
public:
SymbolTableEntry(JITSymbolFlags SymbolFlags,
UnmaterializedInfoIterator UnmaterializedInfoItr);
SymbolTableEntry(JITSymbolFlags SymbolFlags);
SymbolTableEntry(JITEvaluatedSymbol Sym);
SymbolTableEntry(SymbolTableEntry &&Other);
SymbolTableEntry &operator=(SymbolTableEntry &&Other);
~SymbolTableEntry();
ExecutionSessionBase &ES;
std::string VSOName;
SymbolMap Symbols;
UnmaterializedInfosMap UnmaterializedInfos;
MaterializingInfosMap MaterializingInfos;
// Change definition due to override. Only usable prior to materialization.
void replaceWith(VSO &V, SymbolStringPtr Name, JITEvaluatedSymbol Sym);
Error defineImpl(MaterializationUnit &MU);
// Change definition due to override. Only usable prior to materialization.
void replaceWith(VSO &V, SymbolStringPtr Name, JITSymbolFlags Flags,
UnmaterializedInfoIterator NewUMII);
void detachQueryHelper(AsynchronousSymbolQuery &Q,
const SymbolNameSet &QuerySymbols);
// Abandon old definition and move to materializing state.
// There is no need to call notifyMaterializing after this.
void replaceMaterializing(VSO &V, SymbolStringPtr Name,
JITSymbolFlags NewFlags);
// Notify this entry that it is being materialized.
void notifyMaterializing();
// Move entry to resolved state.
void resolve(VSO &V, JITEvaluatedSymbol Sym);
// Move entry to finalized state.
void finalize();
JITSymbolFlags Flags;
union {
JITTargetAddress Address;
UnmaterializedInfoIterator UMII;
};
private:
void destroy();
};
std::map<SymbolStringPtr, SymbolTableEntry> Symbols;
UnmaterializedInfoList UnmaterializedInfos;
MaterializingInfoMap MaterializingInfos;
void transferFinalizedNodeDependencies(MaterializingInfo &DependantMI,
const SymbolStringPtr &DependantName,
MaterializingInfo &FinalizedMI);
};
/// An ExecutionSession represents a running JIT program.
class ExecutionSession {
class ExecutionSession : public ExecutionSessionBase {
public:
using ErrorReporter = std::function<void(Error)>;
using DispatchMaterializationFunction =
std::function<void(VSO &V, std::unique_ptr<MaterializationUnit> MU)>;
/// Construct an ExecutionEngine.
///
/// SymbolStringPools may be shared between ExecutionSessions.
ExecutionSession(std::shared_ptr<SymbolStringPool> SSP = nullptr)
: SSP(SSP ? std::move(SSP) : std::make_shared<SymbolStringPool>()) {}
: ExecutionSessionBase(std::move(SSP)) {}
/// Returns the SymbolStringPool for this ExecutionSession.
SymbolStringPool &getSymbolStringPool() const { return *SSP; }
/// Add a new VSO to this ExecutionSession.
VSO &createVSO(std::string Name);
/// Set the error reporter function.
void setErrorReporter(ErrorReporter ReportError) {
this->ReportError = std::move(ReportError);
}
/// Report a error for this execution session.
///
/// Unhandled errors can be sent here to log them.
void reportError(Error Err) { ReportError(std::move(Err)); }
/// Allocate a module key for a new module to add to the JIT.
VModuleKey allocateVModule() { return ++LastKey; }
/// Return a module key to the ExecutionSession so that it can be
/// re-used. This should only be done once all resources associated
//// with the original key have been released.
void releaseVModule(VModuleKey Key) { /* FIXME: Recycle keys */ }
public:
static void logErrorsToStdErr(Error Err);
std::shared_ptr<SymbolStringPool> SSP;
VModuleKey LastKey = 0;
ErrorReporter ReportError = logErrorsToStdErr;
private:
std::vector<std::unique_ptr<VSO>> VSOs;
};
/// Runs Materializers on the current thread and reports errors to the given
/// ExecutionSession.
class MaterializeOnCurrentThread {
public:
void operator()(VSO::Materializer M) { M(); }
};
/// Materialization function object wrapper for the lookup method.
using MaterializationDispatcher = std::function<void(VSO::Materializer M)>;
/// Look up a set of symbols by searching a list of VSOs.
/// Look up the given names in the given VSOs.
/// VSOs will be searched in order and no VSO pointer may be null.
/// All symbols must be found within the given VSOs or an error
/// will be returned.
///
/// All VSOs in the list should be non-null.
/// If this lookup is being performed on behalf of a
/// MaterializationResponsibility then it must be passed in as R
/// (in order to record the symbol dependencies).
/// If this lookup is not being performed on behalf of a
/// MaterializationResponsibility then R should be left null.
Expected<SymbolMap> lookup(const std::vector<VSO *> &VSOs, SymbolNameSet Names,
MaterializationDispatcher DispatchMaterialization);
MaterializationResponsibility *R);
/// Look up a symbol by searching a list of VSOs.
Expected<JITEvaluatedSymbol>
lookup(const std::vector<VSO *> VSOs, SymbolStringPtr Name,
MaterializationDispatcher DispatchMaterialization);
Expected<JITEvaluatedSymbol> lookup(const std::vector<VSO *> VSOs,
SymbolStringPtr Name,
MaterializationResponsibility *R);
} // End namespace orc
} // End namespace llvm

View File

@ -22,7 +22,8 @@ namespace orc {
class JITSymbolResolverAdapter : public JITSymbolResolver {
public:
JITSymbolResolverAdapter(ExecutionSession &ES, SymbolResolver &R);
JITSymbolResolverAdapter(ExecutionSession &ES, SymbolResolver &R,
MaterializationResponsibility *MR);
Expected<LookupFlagsResult> lookupFlags(const LookupSet &Symbols) override;
Expected<LookupResult> lookup(const LookupSet &Symbols) override;
@ -30,6 +31,7 @@ private:
ExecutionSession &ES;
std::set<SymbolStringPtr> ResolvedStrings;
SymbolResolver &R;
MaterializationResponsibility *MR;
};
/// Use the given legacy-style FindSymbol function (i.e. a function that
@ -68,27 +70,35 @@ Expected<SymbolNameSet> lookupFlagsWithLegacyFn(SymbolFlagsMap &SymbolFlags,
///
/// Useful for implementing lookup bodies that query legacy resolvers.
template <typename FindSymbolFn>
SymbolNameSet lookupWithLegacyFn(AsynchronousSymbolQuery &Query,
const SymbolNameSet &Symbols,
FindSymbolFn FindSymbol) {
SymbolNameSet
lookupWithLegacyFn(ExecutionSession &ES, AsynchronousSymbolQuery &Query,
const SymbolNameSet &Symbols, FindSymbolFn FindSymbol) {
SymbolNameSet SymbolsNotFound;
bool NewSymbolsResolved = false;
for (auto &S : Symbols) {
if (JITSymbol Sym = FindSymbol(*S)) {
if (auto Addr = Sym.getAddress()) {
Query.resolve(S, JITEvaluatedSymbol(*Addr, Sym.getFlags()));
Query.finalizeSymbol();
Query.notifySymbolReady();
NewSymbolsResolved = true;
} else {
Query.notifyMaterializationFailed(Addr.takeError());
ES.failQuery(Query, Addr.takeError());
return SymbolNameSet();
}
} else if (auto Err = Sym.takeError()) {
Query.notifyMaterializationFailed(std::move(Err));
ES.failQuery(Query, std::move(Err));
return SymbolNameSet();
} else
SymbolsNotFound.insert(S);
}
if (NewSymbolsResolved && Query.isFullyResolved())
Query.handleFullyResolved();
if (NewSymbolsResolved && Query.isFullyReady())
Query.handleFullyReady();
return SymbolsNotFound;
}
@ -99,8 +109,9 @@ class LegacyLookupFnResolver final : public SymbolResolver {
public:
using ErrorReporter = std::function<void(Error)>;
LegacyLookupFnResolver(LegacyLookupFn LegacyLookup, ErrorReporter ReportError)
: LegacyLookup(std::move(LegacyLookup)),
LegacyLookupFnResolver(ExecutionSession &ES, LegacyLookupFn LegacyLookup,
ErrorReporter ReportError)
: ES(ES), LegacyLookup(std::move(LegacyLookup)),
ReportError(std::move(ReportError)) {}
SymbolNameSet lookupFlags(SymbolFlagsMap &Flags,
@ -116,20 +127,21 @@ public:
SymbolNameSet lookup(std::shared_ptr<AsynchronousSymbolQuery> Query,
SymbolNameSet Symbols) final {
return lookupWithLegacyFn(*Query, Symbols, LegacyLookup);
return lookupWithLegacyFn(ES, *Query, Symbols, LegacyLookup);
}
private:
ExecutionSession &ES;
LegacyLookupFn LegacyLookup;
ErrorReporter ReportError;
};
template <typename LegacyLookupFn>
std::shared_ptr<LegacyLookupFnResolver<LegacyLookupFn>>
createLegacyLookupResolver(LegacyLookupFn LegacyLookup,
createLegacyLookupResolver(ExecutionSession &ES, LegacyLookupFn LegacyLookup,
std::function<void(Error)> ErrorReporter) {
return std::make_shared<LegacyLookupFnResolver<LegacyLookupFn>>(
std::move(LegacyLookup), std::move(ErrorReporter));
ES, std::move(LegacyLookup), std::move(ErrorReporter));
}
} // End namespace orc

View File

@ -124,7 +124,8 @@ private:
Error finalize() override {
assert(PFC && "mapSectionAddress called on finalized LinkedObject");
JITSymbolResolverAdapter ResolverAdapter(PFC->Parent.ES, *PFC->Resolver);
JITSymbolResolverAdapter ResolverAdapter(PFC->Parent.ES, *PFC->Resolver,
nullptr);
PFC->RTDyld = llvm::make_unique<RuntimeDyld>(*MemMgr, ResolverAdapter);
PFC->RTDyld->setProcessAllSections(PFC->ProcessAllSections);

File diff suppressed because it is too large Load Diff

View File

@ -12,9 +12,9 @@
namespace llvm {
namespace orc {
JITSymbolResolverAdapter::JITSymbolResolverAdapter(ExecutionSession &ES,
SymbolResolver &R)
: ES(ES), R(R) {}
JITSymbolResolverAdapter::JITSymbolResolverAdapter(
ExecutionSession &ES, SymbolResolver &R, MaterializationResponsibility *MR)
: ES(ES), R(R), MR(MR) {}
Expected<JITSymbolResolverAdapter::LookupResult>
JITSymbolResolverAdapter::lookup(const LookupSet &Symbols) {
@ -25,26 +25,28 @@ JITSymbolResolverAdapter::lookup(const LookupSet &Symbols) {
for (auto &S : Symbols)
InternedSymbols.insert(ES.getSymbolStringPool().intern(S));
auto OnResolve = [&](Expected<SymbolMap> R) {
if (R) {
for (auto &KV : *R) {
ResolvedStrings.insert(KV.first);
Result[*KV.first] = KV.second;
}
} else
Err = joinErrors(std::move(Err), R.takeError());
};
auto OnResolve =
[&, this](Expected<AsynchronousSymbolQuery::ResolutionResult> RR) {
if (RR) {
// If this lookup was attached to a MaterializationResponsibility then
// record the dependencies.
if (MR)
MR->addDependencies(RR->Dependencies);
auto OnReady = [](Error Err) {
// FIXME: Report error to ExecutionSession.
logAllUnhandledErrors(std::move(Err), errs(),
"legacy resolver received on-ready error:\n");
};
for (auto &KV : RR->Symbols) {
ResolvedStrings.insert(KV.first);
Result[*KV.first] = KV.second;
}
} else
Err = joinErrors(std::move(Err), RR.takeError());
};
auto OnReady = [this](Error Err) { ES.reportError(std::move(Err)); };
auto Query = std::make_shared<AsynchronousSymbolQuery>(InternedSymbols,
OnResolve, OnReady);
auto UnresolvedSymbols = R.lookup(std::move(Query), InternedSymbols);
auto UnresolvedSymbols = R.lookup(Query, InternedSymbols);
if (!UnresolvedSymbols.empty()) {
std::string ErrorMsg = "Unresolved symbols: ";

View File

@ -155,16 +155,19 @@ private:
if (auto Addr = Sym.getAddress())
Query->resolve(S, JITEvaluatedSymbol(*Addr, Sym.getFlags()));
else {
Query->notifyMaterializationFailed(Addr.takeError());
Stack.ES.failQuery(*Query, Addr.takeError());
return orc::SymbolNameSet();
}
} else if (auto Err = Sym.takeError()) {
Query->notifyMaterializationFailed(std::move(Err));
Stack.ES.failQuery(*Query, std::move(Err));
return orc::SymbolNameSet();
} else
UnresolvedSymbols.insert(S);
}
if (Query->isFullyResolved())
Query->handleFullyResolved();
return UnresolvedSymbols;
}

View File

@ -171,34 +171,40 @@ class OrcMCJITReplacement : public ExecutionEngine {
SymbolNameSet lookup(std::shared_ptr<AsynchronousSymbolQuery> Query,
SymbolNameSet Symbols) override {
SymbolNameSet UnresolvedSymbols;
bool NewSymbolsResolved = false;
for (auto &S : Symbols) {
if (auto Sym = M.findMangledSymbol(*S)) {
if (auto Addr = Sym.getAddress())
if (auto Addr = Sym.getAddress()) {
Query->resolve(S, JITEvaluatedSymbol(*Addr, Sym.getFlags()));
else {
Query->notifyMaterializationFailed(Addr.takeError());
NewSymbolsResolved = true;
} else {
M.ES.failQuery(*Query, Addr.takeError());
return SymbolNameSet();
}
} else if (auto Err = Sym.takeError()) {
Query->notifyMaterializationFailed(std::move(Err));
M.ES.failQuery(*Query, std::move(Err));
return SymbolNameSet();
} else {
if (auto Sym2 = M.ClientResolver->findSymbol(*S)) {
if (auto Addr = Sym2.getAddress())
if (auto Addr = Sym2.getAddress()) {
Query->resolve(S, JITEvaluatedSymbol(*Addr, Sym2.getFlags()));
else {
Query->notifyMaterializationFailed(Addr.takeError());
NewSymbolsResolved = true;
} else {
M.ES.failQuery(*Query, Addr.takeError());
return SymbolNameSet();
}
} else if (auto Err = Sym2.takeError()) {
Query->notifyMaterializationFailed(std::move(Err));
M.ES.failQuery(*Query, std::move(Err));
return SymbolNameSet();
} else
UnresolvedSymbols.insert(S);
}
}
if (NewSymbolsResolved && Query->isFullyResolved())
Query->handleFullyResolved();
return UnresolvedSymbols;
}

View File

@ -174,9 +174,10 @@ public:
}
return std::move(*NotFoundViaLegacyLookup);
},
[LegacyLookup](std::shared_ptr<orc::AsynchronousSymbolQuery> Query,
[this,
LegacyLookup](std::shared_ptr<orc::AsynchronousSymbolQuery> Query,
orc::SymbolNameSet Symbols) {
return lookupWithLegacyFn(*Query, Symbols, LegacyLookup);
return lookupWithLegacyFn(ES, *Query, Symbols, LegacyLookup);
});
// Add the module to the JIT.

View File

@ -22,36 +22,36 @@ namespace {
class SimpleMaterializationUnit : public MaterializationUnit {
public:
using GetSymbolsFunction = std::function<SymbolFlagsMap()>;
using MaterializeFunction =
std::function<void(MaterializationResponsibility)>;
using DiscardFunction = std::function<void(const VSO &, SymbolStringPtr)>;
using DestructorFunction = std::function<void()>;
SimpleMaterializationUnit(
GetSymbolsFunction GetSymbols, MaterializeFunction Materialize,
DiscardFunction Discard,
SymbolFlagsMap SymbolFlags, MaterializeFunction Materialize,
DiscardFunction Discard = DiscardFunction(),
DestructorFunction Destructor = DestructorFunction())
: GetSymbols(std::move(GetSymbols)), Materialize(std::move(Materialize)),
Discard(std::move(Discard)), Destructor(std::move(Destructor)) {}
: MaterializationUnit(std::move(SymbolFlags)),
Materialize(std::move(Materialize)), Discard(std::move(Discard)),
Destructor(std::move(Destructor)) {}
~SimpleMaterializationUnit() override {
if (Destructor)
Destructor();
}
SymbolFlagsMap getSymbols() override { return GetSymbols(); }
void materialize(MaterializationResponsibility R) override {
Materialize(std::move(R));
}
void discard(const VSO &V, SymbolStringPtr Name) override {
Discard(V, std::move(Name));
if (Discard)
Discard(V, std::move(Name));
else
llvm_unreachable("Discard not supported");
}
private:
GetSymbolsFunction GetSymbols;
MaterializeFunction Materialize;
DiscardFunction Discard;
DestructorFunction Destructor;
@ -65,14 +65,16 @@ TEST(CoreAPIsTest, AsynchronousSymbolQuerySuccessfulResolutionOnly) {
bool OnResolutionRun = false;
bool OnReadyRun = false;
auto OnResolution = [&](Expected<SymbolMap> Result) {
EXPECT_TRUE(!!Result) << "Resolution unexpectedly returned error";
auto I = Result->find(Foo);
EXPECT_NE(I, Result->end()) << "Could not find symbol definition";
EXPECT_EQ(I->second.getAddress(), FakeAddr)
<< "Resolution returned incorrect result";
OnResolutionRun = true;
};
auto OnResolution =
[&](Expected<AsynchronousSymbolQuery::ResolutionResult> Result) {
EXPECT_TRUE(!!Result) << "Resolution unexpectedly returned error";
auto &Resolved = Result->Symbols;
auto I = Resolved.find(Foo);
EXPECT_NE(I, Resolved.end()) << "Could not find symbol definition";
EXPECT_EQ(I->second.getAddress(), FakeAddr)
<< "Resolution returned incorrect result";
OnResolutionRun = true;
};
auto OnReady = [&](Error Err) {
cantFail(std::move(Err));
OnReadyRun = true;
@ -82,24 +84,32 @@ TEST(CoreAPIsTest, AsynchronousSymbolQuerySuccessfulResolutionOnly) {
Q.resolve(Foo, JITEvaluatedSymbol(FakeAddr, JITSymbolFlags::Exported));
EXPECT_TRUE(Q.isFullyResolved()) << "Expected query to be fully resolved";
if (!Q.isFullyResolved())
return;
Q.handleFullyResolved();
EXPECT_TRUE(OnResolutionRun) << "OnResolutionCallback was not run";
EXPECT_FALSE(OnReadyRun) << "OnReady unexpectedly run";
}
TEST(CoreAPIsTest, AsynchronousSymbolQueryResolutionErrorOnly) {
SymbolStringPool SP;
auto Foo = SP.intern("foo");
TEST(CoreAPIsTest, ExecutionSessionFailQuery) {
ExecutionSession ES;
auto Foo = ES.getSymbolStringPool().intern("foo");
SymbolNameSet Names({Foo});
bool OnResolutionRun = false;
bool OnReadyRun = false;
auto OnResolution = [&](Expected<SymbolMap> Result) {
EXPECT_FALSE(!!Result) << "Resolution unexpectedly returned success";
auto Msg = toString(Result.takeError());
EXPECT_EQ(Msg, "xyz") << "Resolution returned incorrect result";
OnResolutionRun = true;
};
auto OnResolution =
[&](Expected<AsynchronousSymbolQuery::ResolutionResult> Result) {
EXPECT_FALSE(!!Result) << "Resolution unexpectedly returned success";
auto Msg = toString(Result.takeError());
EXPECT_EQ(Msg, "xyz") << "Resolution returned incorrect result";
OnResolutionRun = true;
};
auto OnReady = [&](Error Err) {
cantFail(std::move(Err));
OnReadyRun = true;
@ -107,30 +117,31 @@ TEST(CoreAPIsTest, AsynchronousSymbolQueryResolutionErrorOnly) {
AsynchronousSymbolQuery Q(Names, OnResolution, OnReady);
Q.notifyMaterializationFailed(
make_error<StringError>("xyz", inconvertibleErrorCode()));
ES.failQuery(Q, make_error<StringError>("xyz", inconvertibleErrorCode()));
EXPECT_TRUE(OnResolutionRun) << "OnResolutionCallback was not run";
EXPECT_FALSE(OnReadyRun) << "OnReady unexpectedly run";
}
TEST(CoreAPIsTest, SimpleAsynchronousSymbolQueryAgainstVSO) {
SymbolStringPool SP;
auto Foo = SP.intern("foo");
ExecutionSession ES;
auto Foo = ES.getSymbolStringPool().intern("foo");
constexpr JITTargetAddress FakeAddr = 0xdeadbeef;
SymbolNameSet Names({Foo});
bool OnResolutionRun = false;
bool OnReadyRun = false;
auto OnResolution = [&](Expected<SymbolMap> Result) {
EXPECT_TRUE(!!Result) << "Query unexpectedly returned error";
auto I = Result->find(Foo);
EXPECT_NE(I, Result->end()) << "Could not find symbol definition";
EXPECT_EQ(I->second.getAddress(), FakeAddr)
<< "Resolution returned incorrect result";
OnResolutionRun = true;
};
auto OnResolution =
[&](Expected<AsynchronousSymbolQuery::ResolutionResult> Result) {
EXPECT_TRUE(!!Result) << "Query unexpectedly returned error";
auto &Resolved = Result->Symbols;
auto I = Resolved.find(Foo);
EXPECT_NE(I, Resolved.end()) << "Could not find symbol definition";
EXPECT_EQ(I->second.getAddress(), FakeAddr)
<< "Resolution returned incorrect result";
OnResolutionRun = true;
};
auto OnReady = [&](Error Err) {
cantFail(std::move(Err));
@ -139,11 +150,12 @@ TEST(CoreAPIsTest, SimpleAsynchronousSymbolQueryAgainstVSO) {
auto Q =
std::make_shared<AsynchronousSymbolQuery>(Names, OnResolution, OnReady);
VSO V;
auto &V = ES.createVSO("V");
SymbolMap Defs;
Defs[Foo] = JITEvaluatedSymbol(FakeAddr, JITSymbolFlags::Exported);
cantFail(V.define(std::move(Defs)));
auto Defs = absoluteSymbols(
{{Foo, JITEvaluatedSymbol(FakeAddr, JITSymbolFlags::Exported)}});
cantFail(V.define(Defs));
assert(Defs == nullptr && "Defs should have been accepted");
V.lookup(Q, Names);
EXPECT_TRUE(OnResolutionRun) << "OnResolutionCallback was not run";
@ -155,33 +167,26 @@ TEST(CoreAPIsTest, LookupFlagsTest) {
// Test that lookupFlags works on a predefined symbol, and does not trigger
// materialization of a lazy symbol.
SymbolStringPool SP;
auto Foo = SP.intern("foo");
auto Bar = SP.intern("bar");
auto Baz = SP.intern("baz");
ExecutionSession ES;
auto Foo = ES.getSymbolStringPool().intern("foo");
auto Bar = ES.getSymbolStringPool().intern("bar");
auto Baz = ES.getSymbolStringPool().intern("baz");
JITSymbolFlags FooFlags = JITSymbolFlags::Exported;
JITSymbolFlags BarFlags = static_cast<JITSymbolFlags::FlagNames>(
JITSymbolFlags::Exported | JITSymbolFlags::Weak);
VSO V;
VSO &V = ES.createVSO("V");
auto MU = llvm::make_unique<SimpleMaterializationUnit>(
[=]() {
return SymbolFlagsMap({{Bar, BarFlags}});
},
SymbolFlagsMap({{Bar, BarFlags}}),
[](MaterializationResponsibility R) {
llvm_unreachable("Symbol materialized on flags lookup");
},
[](const VSO &V, SymbolStringPtr Name) {
llvm_unreachable("Symbol finalized on flags lookup");
});
SymbolMap InitialDefs;
InitialDefs[Foo] = JITEvaluatedSymbol(0xdeadbeef, FooFlags);
cantFail(V.define(std::move(InitialDefs)));
cantFail(V.defineLazy(std::move(MU)));
cantFail(V.define(
absoluteSymbols({{Foo, JITEvaluatedSymbol(0xdeadbeef, FooFlags)}})));
cantFail(V.define(std::move(MU)));
SymbolNameSet Names({Foo, Bar, Baz});
@ -199,18 +204,150 @@ TEST(CoreAPIsTest, LookupFlagsTest) {
EXPECT_EQ(SymbolFlags[Bar], BarFlags) << "Incorrect flags returned for Bar";
}
TEST(CoreAPIsTest, TestCircularDependenceInOneVSO) {
ExecutionSession ES;
auto &V = ES.createVSO("V");
// Create three symbols: Foo, Bar and Baz.
auto Foo = ES.getSymbolStringPool().intern("foo");
auto FooFlags = JITSymbolFlags::Exported;
auto FooSym = JITEvaluatedSymbol(1U, FooFlags);
auto Bar = ES.getSymbolStringPool().intern("bar");
auto BarFlags = JITSymbolFlags::Exported;
auto BarSym = JITEvaluatedSymbol(2U, BarFlags);
auto Baz = ES.getSymbolStringPool().intern("baz");
auto BazFlags = JITSymbolFlags::Exported;
auto BazSym = JITEvaluatedSymbol(3U, BazFlags);
// Create three MaterializationResponsibility objects: one for each symbol
// (these are optional because MaterializationResponsibility does not have
// a default constructor).
Optional<MaterializationResponsibility> FooR;
Optional<MaterializationResponsibility> BarR;
Optional<MaterializationResponsibility> BazR;
// Create a MaterializationUnit for each symbol that moves the
// MaterializationResponsibility into one of the locals above.
auto FooMU = llvm::make_unique<SimpleMaterializationUnit>(
SymbolFlagsMap({{Foo, FooFlags}}),
[&](MaterializationResponsibility R) { FooR.emplace(std::move(R)); });
auto BarMU = llvm::make_unique<SimpleMaterializationUnit>(
SymbolFlagsMap({{Bar, BarFlags}}),
[&](MaterializationResponsibility R) { BarR.emplace(std::move(R)); });
auto BazMU = llvm::make_unique<SimpleMaterializationUnit>(
SymbolFlagsMap({{Baz, BazFlags}}),
[&](MaterializationResponsibility R) { BazR.emplace(std::move(R)); });
// Define the symbols.
cantFail(V.define(FooMU));
cantFail(V.define(BarMU));
cantFail(V.define(BazMU));
// Query each of the symbols to trigger materialization.
bool FooResolved = false;
bool FooReady = false;
auto FooQ = std::make_shared<AsynchronousSymbolQuery>(
SymbolNameSet({Foo}),
[&](Expected<AsynchronousSymbolQuery::ResolutionResult> RR) {
cantFail(std::move(RR));
FooResolved = true;
},
[&](Error Err) {
cantFail(std::move(Err));
FooReady = true;
});
{
auto Unresolved = V.lookup(FooQ, {Foo});
EXPECT_TRUE(Unresolved.empty()) << "Failed to resolve \"Foo\"";
}
bool BarResolved = false;
bool BarReady = false;
auto BarQ = std::make_shared<AsynchronousSymbolQuery>(
SymbolNameSet({Bar}),
[&](Expected<AsynchronousSymbolQuery::ResolutionResult> RR) {
cantFail(std::move(RR));
BarResolved = true;
},
[&](Error Err) {
cantFail(std::move(Err));
BarReady = true;
});
{
auto Unresolved = V.lookup(BarQ, {Bar});
EXPECT_TRUE(Unresolved.empty()) << "Failed to resolve \"Bar\"";
}
bool BazResolved = false;
bool BazReady = false;
auto BazQ = std::make_shared<AsynchronousSymbolQuery>(
SymbolNameSet({Baz}),
[&](Expected<AsynchronousSymbolQuery::ResolutionResult> RR) {
cantFail(std::move(RR));
BazResolved = true;
},
[&](Error Err) {
cantFail(std::move(Err));
BazReady = true;
});
{
auto Unresolved = V.lookup(BazQ, {Baz});
EXPECT_TRUE(Unresolved.empty()) << "Failed to resolve \"Baz\"";
}
FooR->addDependencies({{&V, SymbolNameSet({Bar})}});
BarR->addDependencies({{&V, SymbolNameSet({Baz})}});
BazR->addDependencies({{&V, SymbolNameSet({Foo})}});
EXPECT_FALSE(FooResolved) << "\"Foo\" should not be resolved yet";
EXPECT_FALSE(BarResolved) << "\"Bar\" should not be resolved yet";
EXPECT_FALSE(BazResolved) << "\"Baz\" should not be resolved yet";
FooR->resolve({{Foo, FooSym}});
BarR->resolve({{Bar, BarSym}});
BazR->resolve({{Baz, BazSym}});
EXPECT_TRUE(FooResolved) << "\"Foo\" should be resolved now";
EXPECT_TRUE(BarResolved) << "\"Bar\" should be resolved now";
EXPECT_TRUE(BazResolved) << "\"Baz\" should be resolved now";
EXPECT_FALSE(FooReady) << "\"Foo\" should not be ready yet";
EXPECT_FALSE(BarReady) << "\"Bar\" should not be ready yet";
EXPECT_FALSE(BazReady) << "\"Baz\" should not be ready yet";
FooR->finalize();
BarR->finalize();
// Verify that nothing is ready until the circular dependence is resolved.
EXPECT_FALSE(FooReady) << "\"Foo\" still should not be ready";
EXPECT_FALSE(BarReady) << "\"Bar\" still should not be ready";
EXPECT_FALSE(BazReady) << "\"Baz\" still should not be ready";
BazR->finalize();
// Verify that everything becomes ready once the circular dependence resolved.
EXPECT_TRUE(FooReady) << "\"Foo\" should be ready now";
EXPECT_TRUE(BarReady) << "\"Bar\" should be ready now";
EXPECT_TRUE(BazReady) << "\"Baz\" should be ready now";
}
TEST(CoreAPIsTest, DropMaterializerWhenEmpty) {
SymbolStringPool SP;
auto Foo = SP.intern("foo");
auto Bar = SP.intern("bar");
ExecutionSession ES;
auto Foo = ES.getSymbolStringPool().intern("foo");
auto Bar = ES.getSymbolStringPool().intern("bar");
bool DestructorRun = false;
auto MU = llvm::make_unique<SimpleMaterializationUnit>(
[=]() {
return SymbolFlagsMap(
{{Foo, JITSymbolFlags::Weak}, {Bar, JITSymbolFlags::Weak}});
},
SymbolFlagsMap(
{{Foo, JITSymbolFlags::Weak}, {Bar, JITSymbolFlags::Weak}}),
[](MaterializationResponsibility R) {
llvm_unreachable("Unexpected call to materialize");
},
@ -220,18 +357,18 @@ TEST(CoreAPIsTest, DropMaterializerWhenEmpty) {
},
[&]() { DestructorRun = true; });
VSO V;
auto &V = ES.createVSO("V");
cantFail(V.defineLazy(std::move(MU)));
cantFail(V.define(MU));
auto FooSym = JITEvaluatedSymbol(1, JITSymbolFlags::Exported);
auto BarSym = JITEvaluatedSymbol(2, JITSymbolFlags::Exported);
cantFail(V.define(SymbolMap({{Foo, FooSym}})));
cantFail(V.define(absoluteSymbols({{Foo, FooSym}})));
EXPECT_FALSE(DestructorRun)
<< "MaterializationUnit should not have been destroyed yet";
cantFail(V.define(SymbolMap({{Bar, BarSym}})));
cantFail(V.define(absoluteSymbols({{Bar, BarSym}})));
EXPECT_TRUE(DestructorRun)
<< "MaterializationUnit should have been destroyed";
@ -242,22 +379,20 @@ TEST(CoreAPIsTest, AddAndMaterializeLazySymbol) {
constexpr JITTargetAddress FakeFooAddr = 0xdeadbeef;
constexpr JITTargetAddress FakeBarAddr = 0xcafef00d;
SymbolStringPool SP;
auto Foo = SP.intern("foo");
auto Bar = SP.intern("bar");
ExecutionSession ES;
auto Foo = ES.getSymbolStringPool().intern("foo");
auto Bar = ES.getSymbolStringPool().intern("bar");
bool FooMaterialized = false;
bool BarDiscarded = false;
VSO V;
auto &V = ES.createVSO("V");
auto MU = llvm::make_unique<SimpleMaterializationUnit>(
[=]() {
return SymbolFlagsMap(
{{Foo, JITSymbolFlags::Exported},
{Bar, static_cast<JITSymbolFlags::FlagNames>(
JITSymbolFlags::Exported | JITSymbolFlags::Weak)}});
},
SymbolFlagsMap(
{{Foo, JITSymbolFlags::Exported},
{Bar, static_cast<JITSymbolFlags::FlagNames>(
JITSymbolFlags::Exported | JITSymbolFlags::Weak)}}),
[&](MaterializationResponsibility R) {
assert(BarDiscarded && "Bar should have been discarded by this point");
SymbolMap SymbolsToResolve;
@ -272,25 +407,27 @@ TEST(CoreAPIsTest, AddAndMaterializeLazySymbol) {
BarDiscarded = true;
});
cantFail(V.defineLazy(std::move(MU)));
cantFail(V.define(MU));
SymbolMap BarOverride;
BarOverride[Bar] = JITEvaluatedSymbol(FakeBarAddr, JITSymbolFlags::Exported);
cantFail(V.define(std::move(BarOverride)));
;
cantFail(V.define(absoluteSymbols(
{{Bar, JITEvaluatedSymbol(FakeBarAddr, JITSymbolFlags::Exported)}})));
SymbolNameSet Names({Foo});
bool OnResolutionRun = false;
bool OnReadyRun = false;
auto OnResolution = [&](Expected<SymbolMap> Result) {
EXPECT_TRUE(!!Result) << "Resolution unexpectedly returned error";
auto I = Result->find(Foo);
EXPECT_NE(I, Result->end()) << "Could not find symbol definition";
EXPECT_EQ(I->second.getAddress(), FakeFooAddr)
<< "Resolution returned incorrect result";
OnResolutionRun = true;
};
auto OnResolution =
[&](Expected<AsynchronousSymbolQuery::ResolutionResult> Result) {
EXPECT_TRUE(!!Result) << "Resolution unexpectedly returned error";
auto I = Result->Symbols.find(Foo);
EXPECT_NE(I, Result->Symbols.end())
<< "Could not find symbol definition";
EXPECT_EQ(I->second.getAddress(), FakeFooAddr)
<< "Resolution returned incorrect result";
OnResolutionRun = true;
};
auto OnReady = [&](Error Err) {
cantFail(std::move(Err));
@ -300,40 +437,98 @@ TEST(CoreAPIsTest, AddAndMaterializeLazySymbol) {
auto Q =
std::make_shared<AsynchronousSymbolQuery>(Names, OnResolution, OnReady);
auto LR = V.lookup(std::move(Q), Names);
auto Unresolved = V.lookup(std::move(Q), Names);
for (auto &M : LR.Materializers)
M();
EXPECT_TRUE(LR.UnresolvedSymbols.empty()) << "Could not find Foo in dylib";
EXPECT_TRUE(Unresolved.empty()) << "Could not find Foo in dylib";
EXPECT_TRUE(FooMaterialized) << "Foo was not materialized";
EXPECT_TRUE(BarDiscarded) << "Bar was not discarded";
EXPECT_TRUE(OnResolutionRun) << "OnResolutionCallback was not run";
EXPECT_TRUE(OnReadyRun) << "OnReady was not run";
}
TEST(CoreAPIsTest, DefineMaterializingSymbol) {
ExecutionSession ES;
auto Foo = ES.getSymbolStringPool().intern("foo");
auto Bar = ES.getSymbolStringPool().intern("bar");
auto FooSym = JITEvaluatedSymbol(1, JITSymbolFlags::Exported);
auto BarSym = JITEvaluatedSymbol(2, JITSymbolFlags::Exported);
bool ExpectNoMoreMaterialization = false;
ES.setDispatchMaterialization(
[&](VSO &V, std::unique_ptr<MaterializationUnit> MU) {
if (ExpectNoMoreMaterialization)
ADD_FAILURE() << "Unexpected materialization";
MU->doMaterialize(V);
});
auto MU = llvm::make_unique<SimpleMaterializationUnit>(
SymbolFlagsMap({{Foo, FooSym.getFlags()}}),
[&](MaterializationResponsibility R) {
cantFail(
R.defineMaterializing(SymbolFlagsMap({{Bar, BarSym.getFlags()}})));
R.resolve(SymbolMap({{Foo, FooSym}, {Bar, BarSym}}));
R.finalize();
});
auto &V = ES.createVSO("V");
cantFail(V.define(MU));
auto OnResolution1 =
[&](Expected<AsynchronousSymbolQuery::ResolutionResult> Result) {
cantFail(std::move(Result));
};
auto OnReady1 = [](Error Err) { cantFail(std::move(Err)); };
auto Q1 = std::make_shared<AsynchronousSymbolQuery>(SymbolNameSet({Foo}),
OnResolution1, OnReady1);
V.lookup(std::move(Q1), {Foo});
bool BarResolved = false;
auto OnResolution2 =
[&](Expected<AsynchronousSymbolQuery::ResolutionResult> Result) {
auto R = cantFail(std::move(Result));
EXPECT_EQ(R.Symbols.size(), 1U) << "Expected to resolve one symbol";
EXPECT_EQ(R.Symbols.count(Bar), 1U) << "Expected to resolve 'Bar'";
EXPECT_EQ(R.Symbols[Bar].getAddress(), BarSym.getAddress())
<< "Expected Bar == BarSym";
BarResolved = true;
};
auto OnReady2 = [](Error Err) { cantFail(std::move(Err)); };
auto Q2 = std::make_shared<AsynchronousSymbolQuery>(SymbolNameSet({Bar}),
OnResolution2, OnReady2);
ExpectNoMoreMaterialization = true;
V.lookup(std::move(Q2), {Bar});
EXPECT_TRUE(BarResolved) << "Bar should have been resolved";
}
TEST(CoreAPIsTest, FailResolution) {
SymbolStringPool SP;
auto Foo = SP.intern("foo");
auto Bar = SP.intern("bar");
ExecutionSession ES;
auto Foo = ES.getSymbolStringPool().intern("foo");
auto Bar = ES.getSymbolStringPool().intern("bar");
SymbolNameSet Names({Foo, Bar});
auto MU = llvm::make_unique<SimpleMaterializationUnit>(
[=]() {
return SymbolFlagsMap(
{{Foo, JITSymbolFlags::Weak}, {Bar, JITSymbolFlags::Weak}});
},
[&](MaterializationResponsibility R) { R.notifyMaterializationFailed(); },
[&](const VSO &V, SymbolStringPtr Name) {
llvm_unreachable("Unexpected call to discard");
SymbolFlagsMap(
{{Foo, JITSymbolFlags::Weak}, {Bar, JITSymbolFlags::Weak}}),
[&](MaterializationResponsibility R) {
R.failMaterialization(
[&]() { return make_error<FailedToResolve>(Names); });
});
VSO V;
auto &V = ES.createVSO("V");
cantFail(V.defineLazy(std::move(MU)));
cantFail(V.define(MU));
auto OnResolution = [&](Expected<SymbolMap> Result) {
auto OnResolution = [&](Expected<AsynchronousSymbolQuery::ResolutionResult>
Result) {
handleAllErrors(Result.takeError(),
[&](FailedToResolve &F) {
EXPECT_EQ(F.getSymbols(), Names)
@ -359,23 +554,19 @@ TEST(CoreAPIsTest, FailResolution) {
auto Q =
std::make_shared<AsynchronousSymbolQuery>(Names, OnResolution, OnReady);
auto LR = V.lookup(std::move(Q), Names);
for (auto &M : LR.Materializers)
M();
V.lookup(std::move(Q), Names);
}
TEST(CoreAPIsTest, FailFinalization) {
SymbolStringPool SP;
auto Foo = SP.intern("foo");
auto Bar = SP.intern("bar");
ExecutionSession ES;
auto Foo = ES.getSymbolStringPool().intern("foo");
auto Bar = ES.getSymbolStringPool().intern("bar");
SymbolNameSet Names({Foo, Bar});
auto MU = llvm::make_unique<SimpleMaterializationUnit>(
[=]() {
return SymbolFlagsMap(
{{Foo, JITSymbolFlags::Exported}, {Bar, JITSymbolFlags::Exported}});
},
SymbolFlagsMap(
{{Foo, JITSymbolFlags::Exported}, {Bar, JITSymbolFlags::Exported}}),
[&](MaterializationResponsibility R) {
constexpr JITTargetAddress FakeFooAddr = 0xdeadbeef;
constexpr JITTargetAddress FakeBarAddr = 0xcafef00d;
@ -383,19 +574,18 @@ TEST(CoreAPIsTest, FailFinalization) {
auto FooSym = JITEvaluatedSymbol(FakeFooAddr, JITSymbolFlags::Exported);
auto BarSym = JITEvaluatedSymbol(FakeBarAddr, JITSymbolFlags::Exported);
R.resolve(SymbolMap({{Foo, FooSym}, {Bar, BarSym}}));
R.notifyMaterializationFailed();
},
[&](const VSO &V, SymbolStringPtr Name) {
llvm_unreachable("Unexpected call to discard");
R.failMaterialization(
[&]() { return make_error<FailedToFinalize>(Names); });
});
VSO V;
auto &V = ES.createVSO("V");
cantFail(V.defineLazy(std::move(MU)));
cantFail(V.define(MU));
auto OnResolution = [](Expected<SymbolMap> Result) {
cantFail(std::move(Result));
};
auto OnResolution =
[](Expected<AsynchronousSymbolQuery::ResolutionResult> Result) {
cantFail(std::move(Result));
};
auto OnReady = [&](Error Err) {
handleAllErrors(std::move(Err),
@ -418,32 +608,28 @@ TEST(CoreAPIsTest, FailFinalization) {
auto Q =
std::make_shared<AsynchronousSymbolQuery>(Names, OnResolution, OnReady);
auto LR = V.lookup(std::move(Q), Names);
for (auto &M : LR.Materializers)
M();
V.lookup(std::move(Q), Names);
}
TEST(CoreAPIsTest, TestLambdaSymbolResolver) {
JITEvaluatedSymbol FooSym(0xdeadbeef, JITSymbolFlags::Exported);
JITEvaluatedSymbol BarSym(0xcafef00d, JITSymbolFlags::Exported);
SymbolStringPool SP;
auto Foo = SP.intern("foo");
auto Bar = SP.intern("bar");
auto Baz = SP.intern("baz");
ExecutionSession ES;
VSO V;
cantFail(V.define({{Foo, FooSym}, {Bar, BarSym}}));
auto Foo = ES.getSymbolStringPool().intern("foo");
auto Bar = ES.getSymbolStringPool().intern("bar");
auto Baz = ES.getSymbolStringPool().intern("baz");
auto &V = ES.createVSO("V");
cantFail(V.define(absoluteSymbols({{Foo, FooSym}, {Bar, BarSym}})));
auto Resolver = createSymbolResolver(
[&](SymbolFlagsMap &SymbolFlags, const SymbolNameSet &Symbols) {
return V.lookupFlags(SymbolFlags, Symbols);
},
[&](std::shared_ptr<AsynchronousSymbolQuery> Q, SymbolNameSet Symbols) {
auto LR = V.lookup(std::move(Q), Symbols);
assert(LR.Materializers.empty() &&
"Test generated unexpected materialization work?");
return std::move(LR.UnresolvedSymbols);
return V.lookup(std::move(Q), Symbols);
});
SymbolNameSet Symbols({Foo, Bar, Baz});
@ -466,17 +652,21 @@ TEST(CoreAPIsTest, TestLambdaSymbolResolver) {
bool OnResolvedRun = false;
auto OnResolved = [&](Expected<SymbolMap> Result) {
OnResolvedRun = true;
EXPECT_TRUE(!!Result) << "Unexpected error";
EXPECT_EQ(Result->size(), 2U) << "Unexpected number of resolved symbols";
EXPECT_EQ(Result->count(Foo), 1U) << "Missing lookup result for foo";
EXPECT_EQ(Result->count(Bar), 1U) << "Missing lookup result for bar";
EXPECT_EQ((*Result)[Foo].getAddress(), FooSym.getAddress())
<< "Incorrect address for foo";
EXPECT_EQ((*Result)[Bar].getAddress(), BarSym.getAddress())
<< "Incorrect address for bar";
};
auto OnResolved =
[&](Expected<AsynchronousSymbolQuery::ResolutionResult> Result) {
OnResolvedRun = true;
EXPECT_TRUE(!!Result) << "Unexpected error";
EXPECT_EQ(Result->Symbols.size(), 2U)
<< "Unexpected number of resolved symbols";
EXPECT_EQ(Result->Symbols.count(Foo), 1U)
<< "Missing lookup result for foo";
EXPECT_EQ(Result->Symbols.count(Bar), 1U)
<< "Missing lookup result for bar";
EXPECT_EQ(Result->Symbols[Foo].getAddress(), FooSym.getAddress())
<< "Incorrect address for foo";
EXPECT_EQ(Result->Symbols[Bar].getAddress(), BarSym.getAddress())
<< "Incorrect address for bar";
};
auto OnReady = [&](Error Err) {
EXPECT_FALSE(!!Err) << "Finalization should never fail in this test";
};
@ -498,23 +688,17 @@ TEST(CoreAPIsTest, TestLookupWithUnthreadedMaterialization) {
auto Foo = ES.getSymbolStringPool().intern("foo");
auto MU = llvm::make_unique<SimpleMaterializationUnit>(
[=]() {
return SymbolFlagsMap({{Foo, JITSymbolFlags::Exported}});
},
SymbolFlagsMap({{Foo, JITSymbolFlags::Exported}}),
[&](MaterializationResponsibility R) {
R.resolve({{Foo, FooSym}});
R.finalize();
},
[](const VSO &V, SymbolStringPtr Name) {
llvm_unreachable("Not expecting finalization");
});
VSO V;
auto &V = ES.createVSO("V");
cantFail(V.defineLazy(std::move(MU)));
cantFail(V.define(MU));
auto FooLookupResult =
cantFail(lookup({&V}, Foo, MaterializeOnCurrentThread()));
auto FooLookupResult = cantFail(lookup({&V}, Foo, nullptr));
EXPECT_EQ(FooLookupResult.getAddress(), FooSym.getAddress())
<< "lookup returned an incorrect address";
@ -528,33 +712,20 @@ TEST(CoreAPIsTest, TestLookupWithThreadedMaterialization) {
JITEvaluatedSymbol FooSym(FakeFooAddr, JITSymbolFlags::Exported);
ExecutionSession ES(std::make_shared<SymbolStringPool>());
auto Foo = ES.getSymbolStringPool().intern("foo");
auto MU = llvm::make_unique<SimpleMaterializationUnit>(
[=]() {
return SymbolFlagsMap({{Foo, JITSymbolFlags::Exported}});
},
[&](MaterializationResponsibility R) {
R.resolve({{Foo, FooSym}});
R.finalize();
},
[](const VSO &V, SymbolStringPtr Name) {
llvm_unreachable("Not expecting finalization");
});
VSO V;
cantFail(V.defineLazy(std::move(MU)));
std::thread MaterializationThread;
auto MaterializeOnNewThread = [&](VSO::Materializer M) {
// FIXME: Use move capture once we move to C++14.
auto SharedM = std::make_shared<VSO::Materializer>(std::move(M));
MaterializationThread = std::thread([SharedM]() { (*SharedM)(); });
};
ES.setDispatchMaterialization(
[&](VSO &V, std::unique_ptr<MaterializationUnit> MU) {
auto SharedMU = std::shared_ptr<MaterializationUnit>(std::move(MU));
MaterializationThread =
std::thread([SharedMU, &V]() { SharedMU->doMaterialize(V); });
});
auto Foo = ES.getSymbolStringPool().intern("foo");
auto FooLookupResult =
cantFail(lookup({&V}, Foo, MaterializeOnNewThread));
auto &V = ES.createVSO("V");
cantFail(V.define(absoluteSymbols({{Foo, FooSym}})));
auto FooLookupResult = cantFail(lookup({&V}, Foo, nullptr));
EXPECT_EQ(FooLookupResult.getAddress(), FooSym.getAddress())
<< "lookup returned an incorrect address";

View File

@ -21,11 +21,9 @@ TEST(LegacyAPIInteropTest, QueryAgainstVSO) {
ExecutionSession ES(std::make_shared<SymbolStringPool>());
auto Foo = ES.getSymbolStringPool().intern("foo");
VSO V;
SymbolMap Defs;
auto &V = ES.createVSO("V");
JITEvaluatedSymbol FooSym(0xdeadbeef, JITSymbolFlags::Exported);
Defs[Foo] = FooSym;
cantFail(V.define(std::move(Defs)));
cantFail(V.define(absoluteSymbols({{Foo, FooSym}})));
auto LookupFlags = [&](SymbolFlagsMap &SymbolFlags,
const SymbolNameSet &Names) {
@ -34,15 +32,12 @@ TEST(LegacyAPIInteropTest, QueryAgainstVSO) {
auto Lookup = [&](std::shared_ptr<AsynchronousSymbolQuery> Query,
SymbolNameSet Symbols) {
auto R = V.lookup(std::move(Query), Symbols);
EXPECT_TRUE(R.Materializers.empty())
<< "Query resulted in unexpected materialization work";
return std::move(R.UnresolvedSymbols);
return V.lookup(std::move(Query), Symbols);
};
auto UnderlyingResolver =
createSymbolResolver(std::move(LookupFlags), std::move(Lookup));
JITSymbolResolverAdapter Resolver(ES, *UnderlyingResolver);
JITSymbolResolverAdapter Resolver(ES, *UnderlyingResolver, nullptr);
JITSymbolResolver::LookupSet Names{StringRef("foo")};
@ -90,10 +85,10 @@ TEST(LegacyAPIInteropTset, LegacyLookupHelpersFn) {
return nullptr;
};
SymbolStringPool SP;
auto Foo = SP.intern("foo");
auto Bar = SP.intern("bar");
auto Baz = SP.intern("baz");
ExecutionSession ES;
auto Foo = ES.getSymbolStringPool().intern("foo");
auto Bar = ES.getSymbolStringPool().intern("bar");
auto Baz = ES.getSymbolStringPool().intern("baz");
SymbolNameSet Symbols({Foo, Bar, Baz});
@ -115,24 +110,29 @@ TEST(LegacyAPIInteropTset, LegacyLookupHelpersFn) {
bool OnResolvedRun = false;
bool OnReadyRun = false;
auto OnResolved = [&](Expected<SymbolMap> Result) {
OnResolvedRun = true;
EXPECT_TRUE(!!Result) << "lookuWithLegacy failed to resolve";
EXPECT_EQ(Result->size(), 2U) << "Wrong number of symbols resolved";
EXPECT_EQ(Result->count(Foo), 1U) << "Result for foo missing";
EXPECT_EQ(Result->count(Bar), 1U) << "Result for bar missing";
EXPECT_EQ((*Result)[Foo].getAddress(), FooAddr) << "Wrong address for foo";
EXPECT_EQ((*Result)[Foo].getFlags(), FooFlags) << "Wrong flags for foo";
EXPECT_EQ((*Result)[Bar].getAddress(), BarAddr) << "Wrong address for bar";
EXPECT_EQ((*Result)[Bar].getFlags(), BarFlags) << "Wrong flags for bar";
};
auto OnResolved =
[&](Expected<AsynchronousSymbolQuery::ResolutionResult> Result) {
OnResolvedRun = true;
EXPECT_TRUE(!!Result) << "lookuWithLegacy failed to resolve";
auto &Resolved = Result->Symbols;
EXPECT_EQ(Resolved.size(), 2U) << "Wrong number of symbols resolved";
EXPECT_EQ(Resolved.count(Foo), 1U) << "Result for foo missing";
EXPECT_EQ(Resolved.count(Bar), 1U) << "Result for bar missing";
EXPECT_EQ(Resolved[Foo].getAddress(), FooAddr)
<< "Wrong address for foo";
EXPECT_EQ(Resolved[Foo].getFlags(), FooFlags) << "Wrong flags for foo";
EXPECT_EQ(Resolved[Bar].getAddress(), BarAddr)
<< "Wrong address for bar";
EXPECT_EQ(Resolved[Bar].getFlags(), BarFlags) << "Wrong flags for bar";
};
auto OnReady = [&](Error Err) {
EXPECT_FALSE(!!Err) << "Finalization unexpectedly failed";
OnReadyRun = true;
};
AsynchronousSymbolQuery Q({Foo, Bar}, OnResolved, OnReady);
auto Unresolved = lookupWithLegacyFn(Q, Symbols, LegacyLookup);
auto Unresolved = lookupWithLegacyFn(ES, Q, Symbols, LegacyLookup);
EXPECT_TRUE(OnResolvedRun) << "OnResolved was not run";
EXPECT_TRUE(OnReadyRun) << "OnReady was not run";

View File

@ -190,7 +190,7 @@ TEST_F(RTDyldObjectLinkingLayerExecutionTest, NoDuplicateFinalization) {
},
[&](std::shared_ptr<AsynchronousSymbolQuery> Query,
const SymbolNameSet &Symbols) {
return lookupWithLegacyFn(*Query, Symbols, LegacyLookup);
return lookupWithLegacyFn(ES, *Query, Symbols, LegacyLookup);
});
cantFail(ObjLayer.addObject(K2, std::move(Obj2)));