mirror of
https://github.com/FEX-Emu/FEX.git
synced 2025-02-12 10:29:24 +00:00
FEXLoader: Moves thread management to the frontend
Lots going on here. This moves OS thread object lifetime management and internal thread state lifetime management to the frontend. This causes a bunch of thread handling to move from the FEXCore Context to the frontend. Looking at `FEXCore/include/FEXCore/Core/Context.h` really shows how much of the API has moved to the frontend that FEXCore no longer needs to manage. Primarily this makes FEXCore itself no longer need to care about most of the management of the emulation state. A large amount of the behaviour moved wholesale from Core.cpp to LinuxEmulation's ThreadManager.cpp. Which this manages the lifetimes of both the OS threads and the FEXCore thread state objects. One feature lost was the instruction capability, but this was already buggy and is going to be rewritten/fixed when gdbserver work continues. Now that all of this management is moved to the frontend, the gdbserver can start improving since it can start managing all thread state directly.
This commit is contained in:
parent
b4b8e81f24
commit
58f2693954
@ -34,10 +34,6 @@ namespace FEXCore::Context {
|
||||
return CustomExitHandler;
|
||||
}
|
||||
|
||||
void FEXCore::Context::ContextImpl::Stop() {
|
||||
Stop(false);
|
||||
}
|
||||
|
||||
void FEXCore::Context::ContextImpl::CompileRIP(FEXCore::Core::InternalThreadState *Thread, uint64_t GuestRIP) {
|
||||
CompileBlock(Thread->CurrentFrame, GuestRIP);
|
||||
}
|
||||
@ -46,10 +42,6 @@ namespace FEXCore::Context {
|
||||
CompileBlock(Thread->CurrentFrame, GuestRIP, MaxInst);
|
||||
}
|
||||
|
||||
bool FEXCore::Context::ContextImpl::IsDone() const {
|
||||
return IsPaused();
|
||||
}
|
||||
|
||||
void FEXCore::Context::ContextImpl::SetCustomCPUBackendFactory(CustomCPUFactoryType Factory) {
|
||||
CustomCPUFactory = std::move(Factory);
|
||||
}
|
||||
|
@ -76,11 +76,6 @@ namespace FEXCore::Context {
|
||||
void SetExitHandler(ExitHandler handler) override;
|
||||
ExitHandler GetExitHandler() const override;
|
||||
|
||||
void Pause() override;
|
||||
void Run() override;
|
||||
void Stop() override;
|
||||
void Step() override;
|
||||
|
||||
ExitReason RunUntilExit(FEXCore::Core::InternalThreadState *Thread) override;
|
||||
|
||||
void ExecuteThread(FEXCore::Core::InternalThreadState *Thread) override;
|
||||
@ -88,8 +83,6 @@ namespace FEXCore::Context {
|
||||
void CompileRIP(FEXCore::Core::InternalThreadState *Thread, uint64_t GuestRIP) override;
|
||||
void CompileRIPCount(FEXCore::Core::InternalThreadState *Thread, uint64_t GuestRIP, uint64_t MaxInst) override;
|
||||
|
||||
bool IsDone() const override;
|
||||
|
||||
void SetCustomCPUBackendFactory(CustomCPUFactoryType Factory) override;
|
||||
|
||||
HostFeatures GetHostFeatures() const override;
|
||||
@ -128,17 +121,11 @@ namespace FEXCore::Context {
|
||||
* - HandleCallback(Thread, RIP);
|
||||
*/
|
||||
|
||||
FEXCore::Core::InternalThreadState* CreateThread(uint64_t InitialRIP, uint64_t StackPointer, ManagedBy WhoManages, FEXCore::Core::CPUState *NewThreadState, uint64_t ParentTID) override;
|
||||
FEXCore::Core::InternalThreadState* CreateThread(uint64_t InitialRIP, uint64_t StackPointer, FEXCore::Core::CPUState *NewThreadState, uint64_t ParentTID) override;
|
||||
|
||||
// Public for threading
|
||||
void ExecutionThread(FEXCore::Core::InternalThreadState *Thread) override;
|
||||
/**
|
||||
* @brief Starts the OS thread object to start executing guest code
|
||||
*
|
||||
* @param Thread The internal FEX thread state object
|
||||
*/
|
||||
void RunThread(FEXCore::Core::InternalThreadState *Thread) override;
|
||||
void StopThread(FEXCore::Core::InternalThreadState *Thread) override;
|
||||
|
||||
/**
|
||||
* @brief Destroys this FEX thread object and stops tracking it internally
|
||||
*
|
||||
@ -176,6 +163,8 @@ namespace FEXCore::Context {
|
||||
void WriteFilesWithCode(AOTIRCodeFileWriterFn Writer) override {
|
||||
IRCaptureCache.WriteFilesWithCode(Writer);
|
||||
}
|
||||
|
||||
void ClearCodeCache(FEXCore::Core::InternalThreadState *Thread) override;
|
||||
void InvalidateGuestCodeRange(FEXCore::Core::InternalThreadState *Thread, uint64_t Start, uint64_t Length) override;
|
||||
void InvalidateGuestCodeRange(FEXCore::Core::InternalThreadState *Thread, uint64_t Start, uint64_t Length, CodeRangeInvalidationFn callback) override;
|
||||
void MarkMemoryShared(FEXCore::Core::InternalThreadState *Thread) override;
|
||||
@ -233,18 +222,9 @@ namespace FEXCore::Context {
|
||||
FEX_CONFIG_OPT(DisableVixlIndirectCalls, DISABLE_VIXL_INDIRECT_RUNTIME_CALLS);
|
||||
} Config;
|
||||
|
||||
std::mutex ThreadCreationMutex;
|
||||
FEXCore::Core::InternalThreadState* ParentThread{};
|
||||
fextl::vector<FEXCore::Core::InternalThreadState*> Threads;
|
||||
|
||||
std::atomic_bool CoreShuttingDown{false};
|
||||
|
||||
std::mutex IdleWaitMutex;
|
||||
std::condition_variable IdleWaitCV;
|
||||
std::atomic<uint32_t> IdleWaitRefCount{};
|
||||
|
||||
Event PauseWait;
|
||||
bool Running{};
|
||||
|
||||
FEXCore::ForkableSharedMutex CodeInvalidationMutex;
|
||||
|
||||
FEXCore::HostFeatures HostFeatures;
|
||||
@ -268,12 +248,6 @@ namespace FEXCore::Context {
|
||||
ContextImpl();
|
||||
~ContextImpl();
|
||||
|
||||
bool IsPaused() const { return !Running; }
|
||||
void WaitForThreadsToRun() override;
|
||||
void Stop(bool IgnoreCurrentThread);
|
||||
void WaitForIdle() override;
|
||||
void SignalThread(FEXCore::Core::InternalThreadState *Thread, FEXCore::Core::SignalEvent Event);
|
||||
|
||||
static void ThreadRemoveCodeEntry(FEXCore::Core::InternalThreadState *Thread, uint64_t GuestRIP);
|
||||
static void ThreadAddBlockLink(FEXCore::Core::InternalThreadState *Thread, uint64_t GuestDestination, uintptr_t HostLink, const std::function<void()> &delinker);
|
||||
|
||||
@ -349,10 +323,6 @@ namespace FEXCore::Context {
|
||||
}
|
||||
}
|
||||
|
||||
void IncrementIdleRefCount() override {
|
||||
++IdleWaitRefCount;
|
||||
}
|
||||
|
||||
FEXCore::Utils::PooledAllocatorVirtual OpDispatcherAllocator;
|
||||
FEXCore::Utils::PooledAllocatorVirtual FrontendAllocator;
|
||||
|
||||
@ -382,18 +352,9 @@ namespace FEXCore::Context {
|
||||
|
||||
bool ExitOnHLTEnabled() const { return ExitOnHLT; }
|
||||
|
||||
ThreadsState GetThreads() override {
|
||||
return ThreadsState {
|
||||
.ParentThread = ParentThread,
|
||||
.Threads = &Threads,
|
||||
};
|
||||
}
|
||||
|
||||
FEXCore::CPU::CPUBackendFeatures BackendFeatures;
|
||||
|
||||
protected:
|
||||
void ClearCodeCache(FEXCore::Core::InternalThreadState *Thread);
|
||||
|
||||
void UpdateAtomicTSOEmulationConfig() {
|
||||
if (SupportsHardwareTSO) {
|
||||
// If the hardware supports TSO then we don't need to emulate it through atomics.
|
||||
@ -415,15 +376,8 @@ namespace FEXCore::Context {
|
||||
*/
|
||||
void InitializeCompiler(FEXCore::Core::InternalThreadState* Thread);
|
||||
|
||||
void WaitForIdleWithTimeout();
|
||||
|
||||
void NotifyPause();
|
||||
|
||||
void AddBlockMapping(FEXCore::Core::InternalThreadState *Thread, uint64_t Address, void *Ptr);
|
||||
|
||||
// Entry Cache
|
||||
std::mutex ExitMutex;
|
||||
|
||||
IR::AOTIRCaptureCache IRCaptureCache;
|
||||
fextl::unique_ptr<FEXCore::CodeSerialize::CodeObjectSerializeService> CodeObjectCacheService;
|
||||
|
||||
|
@ -107,17 +107,6 @@ namespace FEXCore::Context {
|
||||
if (CodeObjectCacheService) {
|
||||
CodeObjectCacheService->Shutdown();
|
||||
}
|
||||
|
||||
for (auto &Thread : Threads) {
|
||||
if (Thread->ExecutionThread->joinable()) {
|
||||
Thread->ExecutionThread->join(nullptr);
|
||||
}
|
||||
}
|
||||
|
||||
for (auto &Thread : Threads) {
|
||||
delete Thread;
|
||||
}
|
||||
Threads.clear();
|
||||
}
|
||||
}
|
||||
|
||||
@ -333,161 +322,9 @@ namespace FEXCore::Context {
|
||||
static_cast<ContextImpl*>(Thread->CTX)->Dispatcher->ExecuteJITCallback(Thread->CurrentFrame, RIP);
|
||||
}
|
||||
|
||||
void ContextImpl::WaitForIdle() {
|
||||
std::unique_lock<std::mutex> lk(IdleWaitMutex);
|
||||
IdleWaitCV.wait(lk, [this] {
|
||||
return IdleWaitRefCount.load() == 0;
|
||||
});
|
||||
|
||||
Running = false;
|
||||
}
|
||||
|
||||
void ContextImpl::WaitForIdleWithTimeout() {
|
||||
std::unique_lock<std::mutex> lk(IdleWaitMutex);
|
||||
bool WaitResult = IdleWaitCV.wait_for(lk, std::chrono::milliseconds(1500),
|
||||
[this] {
|
||||
return IdleWaitRefCount.load() == 0;
|
||||
});
|
||||
|
||||
if (!WaitResult) {
|
||||
// The wait failed, this will occur if we stepped in to a syscall
|
||||
// That's okay, we just need to pause the threads manually
|
||||
NotifyPause();
|
||||
}
|
||||
|
||||
// We have sent every thread a pause signal
|
||||
// Now wait again because they /will/ be going to sleep
|
||||
WaitForIdle();
|
||||
}
|
||||
|
||||
void ContextImpl::NotifyPause() {
|
||||
|
||||
// Tell all the threads that they should pause
|
||||
std::lock_guard<std::mutex> lk(ThreadCreationMutex);
|
||||
for (auto &Thread : Threads) {
|
||||
SignalDelegation->SignalThread(Thread, FEXCore::Core::SignalEvent::Pause);
|
||||
}
|
||||
}
|
||||
|
||||
void ContextImpl::Pause() {
|
||||
// If we aren't running, WaitForIdle will never compete.
|
||||
if (Running) {
|
||||
NotifyPause();
|
||||
|
||||
WaitForIdle();
|
||||
}
|
||||
}
|
||||
|
||||
void ContextImpl::Run() {
|
||||
// Spin up all the threads
|
||||
std::lock_guard<std::mutex> lk(ThreadCreationMutex);
|
||||
for (auto &Thread : Threads) {
|
||||
Thread->SignalReason.store(FEXCore::Core::SignalEvent::Return);
|
||||
}
|
||||
|
||||
for (auto &Thread : Threads) {
|
||||
Thread->StartRunning.NotifyAll();
|
||||
}
|
||||
}
|
||||
|
||||
void ContextImpl::WaitForThreadsToRun() {
|
||||
size_t NumThreads{};
|
||||
{
|
||||
std::lock_guard<std::mutex> lk(ThreadCreationMutex);
|
||||
NumThreads = Threads.size();
|
||||
}
|
||||
|
||||
// Spin while waiting for the threads to start up
|
||||
std::unique_lock<std::mutex> lk(IdleWaitMutex);
|
||||
IdleWaitCV.wait(lk, [this, NumThreads] {
|
||||
return IdleWaitRefCount.load() >= NumThreads;
|
||||
});
|
||||
|
||||
Running = true;
|
||||
}
|
||||
|
||||
void ContextImpl::Step() {
|
||||
{
|
||||
std::lock_guard<std::mutex> lk(ThreadCreationMutex);
|
||||
// Walk the threads and tell them to clear their caches
|
||||
// Useful when our block size is set to a large number and we need to step a single instruction
|
||||
for (auto &Thread : Threads) {
|
||||
ClearCodeCache(Thread);
|
||||
}
|
||||
}
|
||||
CoreRunningMode PreviousRunningMode = this->Config.RunningMode;
|
||||
int64_t PreviousMaxIntPerBlock = this->Config.MaxInstPerBlock;
|
||||
this->Config.RunningMode = FEXCore::Context::CoreRunningMode::MODE_SINGLESTEP;
|
||||
this->Config.MaxInstPerBlock = 1;
|
||||
Run();
|
||||
WaitForThreadsToRun();
|
||||
WaitForIdle();
|
||||
this->Config.RunningMode = PreviousRunningMode;
|
||||
this->Config.MaxInstPerBlock = PreviousMaxIntPerBlock;
|
||||
}
|
||||
|
||||
void ContextImpl::Stop(bool IgnoreCurrentThread) {
|
||||
pid_t tid = FHU::Syscalls::gettid();
|
||||
FEXCore::Core::InternalThreadState* CurrentThread{};
|
||||
|
||||
// Tell all the threads that they should stop
|
||||
{
|
||||
std::lock_guard<std::mutex> lk(ThreadCreationMutex);
|
||||
for (auto &Thread : Threads) {
|
||||
if (IgnoreCurrentThread &&
|
||||
Thread->ThreadManager.TID == tid) {
|
||||
// If we are callign stop from the current thread then we can ignore sending signals to this thread
|
||||
// This means that this thread is already gone
|
||||
continue;
|
||||
}
|
||||
else if (Thread->ThreadManager.TID == tid) {
|
||||
// We need to save the current thread for last to ensure all threads receive their stop signals
|
||||
CurrentThread = Thread;
|
||||
continue;
|
||||
}
|
||||
if (Thread->RunningEvents.Running.load()) {
|
||||
StopThread(Thread);
|
||||
}
|
||||
|
||||
// If the thread is waiting to start but immediately killed then there can be a hang
|
||||
// This occurs in the case of gdb attach with immediate kill
|
||||
if (Thread->RunningEvents.WaitingToStart.load()) {
|
||||
Thread->RunningEvents.EarlyExit = true;
|
||||
Thread->StartRunning.NotifyAll();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Stop the current thread now if we aren't ignoring it
|
||||
if (CurrentThread) {
|
||||
StopThread(CurrentThread);
|
||||
}
|
||||
}
|
||||
|
||||
void ContextImpl::StopThread(FEXCore::Core::InternalThreadState *Thread) {
|
||||
if (Thread->RunningEvents.Running.exchange(false)) {
|
||||
SignalDelegation->SignalThread(Thread, FEXCore::Core::SignalEvent::Stop);
|
||||
}
|
||||
}
|
||||
|
||||
void ContextImpl::SignalThread(FEXCore::Core::InternalThreadState *Thread, FEXCore::Core::SignalEvent Event) {
|
||||
if (Thread->RunningEvents.Running.load()) {
|
||||
SignalDelegation->SignalThread(Thread, Event);
|
||||
}
|
||||
}
|
||||
|
||||
FEXCore::Context::ExitReason ContextImpl::RunUntilExit(FEXCore::Core::InternalThreadState *Thread) {
|
||||
if(!StartPaused) {
|
||||
// We will only have one thread at this point, but just in case run notify everything
|
||||
std::lock_guard lk(ThreadCreationMutex);
|
||||
for (auto &Thread : Threads) {
|
||||
Thread->StartRunning.NotifyAll();
|
||||
}
|
||||
}
|
||||
|
||||
ExecutionThread(Thread);
|
||||
while(true) {
|
||||
this->WaitForIdle();
|
||||
auto reason = Thread->ExitReason;
|
||||
|
||||
// Don't return if a custom exit handling the exit
|
||||
@ -523,11 +360,6 @@ namespace FEXCore::Context {
|
||||
#endif
|
||||
}
|
||||
|
||||
void ContextImpl::RunThread(FEXCore::Core::InternalThreadState *Thread) {
|
||||
// Tell the thread to start executing
|
||||
Thread->StartRunning.NotifyAll();
|
||||
}
|
||||
|
||||
void ContextImpl::InitializeCompiler(FEXCore::Core::InternalThreadState* Thread) {
|
||||
Thread->OpDispatcher = fextl::make_unique<FEXCore::IR::OpDispatchBuilder>(this);
|
||||
Thread->OpDispatcher->SetMultiblock(Config.Multiblock);
|
||||
@ -566,7 +398,7 @@ namespace FEXCore::Context {
|
||||
Thread->PassManager->Finalize();
|
||||
}
|
||||
|
||||
FEXCore::Core::InternalThreadState* ContextImpl::CreateThread(uint64_t InitialRIP, uint64_t StackPointer, ManagedBy WhoManages, FEXCore::Core::CPUState *NewThreadState, uint64_t ParentTID) {
|
||||
FEXCore::Core::InternalThreadState* ContextImpl::CreateThread(uint64_t InitialRIP, uint64_t StackPointer, FEXCore::Core::CPUState *NewThreadState, uint64_t ParentTID) {
|
||||
FEXCore::Core::InternalThreadState *Thread = new FEXCore::Core::InternalThreadState{};
|
||||
|
||||
Thread->CurrentFrame->State.gregs[X86State::REG_RSP] = StackPointer;
|
||||
@ -585,39 +417,11 @@ namespace FEXCore::Context {
|
||||
|
||||
Thread->CurrentFrame->State.DeferredSignalRefCount.Store(0);
|
||||
Thread->CurrentFrame->State.DeferredSignalFaultAddress = reinterpret_cast<Core::NonAtomicRefCounter<uint64_t>*>(FEXCore::Allocator::VirtualAlloc(4096));
|
||||
Thread->DestroyedByParent = WhoManages == ManagedBy::FRONTEND;
|
||||
|
||||
// Insert after the Thread object has been fully initialized
|
||||
if (WhoManages == ManagedBy::CORE) {
|
||||
std::lock_guard lk(ThreadCreationMutex);
|
||||
Threads.push_back(Thread);
|
||||
}
|
||||
|
||||
return Thread;
|
||||
}
|
||||
|
||||
void ContextImpl::DestroyThread(FEXCore::Core::InternalThreadState *Thread, bool NeedsTLSUninstall) {
|
||||
// remove new thread object
|
||||
{
|
||||
std::lock_guard lk(ThreadCreationMutex);
|
||||
|
||||
auto It = std::find(Threads.begin(), Threads.end(), Thread);
|
||||
// TODO: Some threads aren't currently tracked in FEXCore.
|
||||
// Re-enable once tracking is in frontend.
|
||||
// LOGMAN_THROW_A_FMT(It != Threads.end(), "Thread wasn't in Threads");
|
||||
|
||||
if (It != Threads.end()) {
|
||||
Threads.erase(It);
|
||||
}
|
||||
}
|
||||
|
||||
if (Thread->ExecutionThread &&
|
||||
Thread->ExecutionThread->IsSelf()) {
|
||||
// To be able to delete a thread from itself, we need to detached the std::thread object
|
||||
Thread->ExecutionThread->detach();
|
||||
}
|
||||
|
||||
// TODO: This is temporary until the frontend has full ownership of threads.
|
||||
if (NeedsTLSUninstall) {
|
||||
#ifndef _WIN32
|
||||
Alloc::OSAllocator::UninstallTLSData(Thread);
|
||||
@ -640,43 +444,6 @@ namespace FEXCore::Context {
|
||||
CodeInvalidationMutex.unlock();
|
||||
return;
|
||||
}
|
||||
|
||||
// This function is called after fork
|
||||
// We need to cleanup some of the thread data that is dead
|
||||
for (auto &DeadThread : Threads) {
|
||||
if (DeadThread == LiveThread) {
|
||||
continue;
|
||||
}
|
||||
|
||||
// Setting running to false ensures that when they are shutdown we won't send signals to kill them
|
||||
DeadThread->RunningEvents.Running = false;
|
||||
|
||||
// Despite what google searches may susgest, glibc actually has special code to handle forks
|
||||
// with multiple active threads.
|
||||
// It cleans up the stacks of dead threads and marks them as terminated.
|
||||
// It also cleans up a bunch of internal mutexes.
|
||||
|
||||
// FIXME: TLS is probally still alive. Investigate
|
||||
|
||||
// Deconstructing the Interneal thread state should clean up most of the state.
|
||||
// But if anything on the now deleted stack is holding a refrence to the heap, it will be leaked
|
||||
delete DeadThread;
|
||||
|
||||
// FIXME: Make sure sure nothing gets leaked via the heap. Ideas:
|
||||
// * Make sure nothing is allocated on the heap without ref in InternalThreadState
|
||||
// * Surround any code that heap allocates with a per-thread mutex.
|
||||
// Before forking, the the forking thread can lock all thread mutexes.
|
||||
}
|
||||
|
||||
// Remove all threads but the live thread from Threads
|
||||
Threads.clear();
|
||||
Threads.push_back(LiveThread);
|
||||
|
||||
// We now only have one thread
|
||||
IdleWaitRefCount = 1;
|
||||
|
||||
// Clean up dead stacks
|
||||
FEXCore::Threads::Thread::CleanupAfterFork();
|
||||
}
|
||||
|
||||
void ContextImpl::LockBeforeFork(FEXCore::Core::InternalThreadState *Thread) {
|
||||
@ -1101,7 +868,6 @@ namespace FEXCore::Context {
|
||||
Thread->ExitReason = FEXCore::Context::ExitReason::EXIT_WAITING;
|
||||
|
||||
InitializeThreadTLSData(Thread);
|
||||
++IdleWaitRefCount;
|
||||
|
||||
// Now notify the thread that we are initialized
|
||||
Thread->ThreadWaiting.NotifyAll();
|
||||
@ -1141,18 +907,10 @@ namespace FEXCore::Context {
|
||||
}
|
||||
}
|
||||
|
||||
--IdleWaitRefCount;
|
||||
IdleWaitCV.notify_all();
|
||||
|
||||
#ifndef _WIN32
|
||||
Alloc::OSAllocator::UninstallTLSData(Thread);
|
||||
#endif
|
||||
SignalDelegation->UninstallTLSState(Thread);
|
||||
|
||||
// If the parent thread is waiting to join, then we can't destroy our thread object
|
||||
if (!Thread->DestroyedByParent) {
|
||||
Thread->CTX->DestroyThread(Thread);
|
||||
}
|
||||
}
|
||||
|
||||
static void InvalidateGuestThreadCodeRange(FEXCore::Core::InternalThreadState *Thread, uint64_t Start, uint64_t Length) {
|
||||
@ -1169,30 +927,13 @@ namespace FEXCore::Context {
|
||||
}
|
||||
}
|
||||
|
||||
static void InvalidateGuestCodeRangeInternal(FEXCore::Core::InternalThreadState *CallingThread, ContextImpl *CTX, uint64_t Start, uint64_t Length) {
|
||||
std::lock_guard lk(static_cast<ContextImpl*>(CTX)->ThreadCreationMutex);
|
||||
|
||||
for (auto &Thread : static_cast<ContextImpl*>(CTX)->Threads) {
|
||||
|
||||
// TODO: Skip calling thread.
|
||||
// Remove once frontend has thread ownership.
|
||||
if (CallingThread == Thread) continue;
|
||||
InvalidateGuestThreadCodeRange(Thread, Start, Length);
|
||||
}
|
||||
|
||||
// Now invalidate calling thread's code.
|
||||
if (CallingThread) {
|
||||
InvalidateGuestThreadCodeRange(CallingThread, Start, Length);
|
||||
}
|
||||
}
|
||||
|
||||
void ContextImpl::InvalidateGuestCodeRange(FEXCore::Core::InternalThreadState *Thread, uint64_t Start, uint64_t Length) {
|
||||
// Potential deferred since Thread might not be valid.
|
||||
// Thread object isn't valid very early in frontend's initialization.
|
||||
// To be more optimal the frontend should provide this code with a valid Thread object earlier.
|
||||
auto lk = GuardSignalDeferringSectionWithFallback(CodeInvalidationMutex, Thread);
|
||||
|
||||
InvalidateGuestCodeRangeInternal(Thread, this, Start, Length);
|
||||
InvalidateGuestThreadCodeRange(Thread, Start, Length);
|
||||
}
|
||||
|
||||
void ContextImpl::InvalidateGuestCodeRange(FEXCore::Core::InternalThreadState *Thread, uint64_t Start, uint64_t Length, CodeRangeInvalidationFn CallAfter) {
|
||||
@ -1201,7 +942,7 @@ namespace FEXCore::Context {
|
||||
// To be more optimal the frontend should provide this code with a valid Thread object earlier.
|
||||
auto lk = GuardSignalDeferringSectionWithFallback(CodeInvalidationMutex, Thread);
|
||||
|
||||
InvalidateGuestCodeRangeInternal(Thread, this, Start, Length);
|
||||
InvalidateGuestThreadCodeRange(Thread, Start, Length);
|
||||
CallAfter(Start, Length);
|
||||
}
|
||||
|
||||
@ -1211,8 +952,6 @@ namespace FEXCore::Context {
|
||||
UpdateAtomicTSOEmulationConfig();
|
||||
|
||||
if (Config.TSOAutoMigration) {
|
||||
std::lock_guard<std::mutex> lkThreads(ThreadCreationMutex);
|
||||
|
||||
// Only the lookup cache is cleared here, so that old code can keep running until next compilation
|
||||
std::lock_guard<std::recursive_mutex> lkLookupCache(Thread->LookupCache->WriteLock);
|
||||
Thread->LookupCache->ClearCache();
|
||||
|
@ -12,6 +12,7 @@
|
||||
#include <FEXCore/Core/SignalDelegator.h>
|
||||
#include <FEXCore/Core/X86Enums.h>
|
||||
#include <FEXCore/Debug/InternalThreadState.h>
|
||||
#include <FEXCore/HLE/SyscallHandler.h>
|
||||
#include <FEXCore/Utils/Event.h>
|
||||
#include <FEXCore/Utils/LogManager.h>
|
||||
#include <FEXCore/Utils/MathUtils.h>
|
||||
@ -24,22 +25,8 @@
|
||||
|
||||
namespace FEXCore::CPU {
|
||||
|
||||
void Dispatcher::SleepThread(FEXCore::Context::ContextImpl *ctx, FEXCore::Core::CpuStateFrame *Frame) {
|
||||
auto Thread = Frame->Thread;
|
||||
|
||||
--ctx->IdleWaitRefCount;
|
||||
ctx->IdleWaitCV.notify_all();
|
||||
|
||||
Thread->RunningEvents.ThreadSleeping = true;
|
||||
|
||||
// Go to sleep
|
||||
Thread->StartRunning.Wait();
|
||||
|
||||
Thread->RunningEvents.Running = true;
|
||||
++ctx->IdleWaitRefCount;
|
||||
Thread->RunningEvents.ThreadSleeping = false;
|
||||
|
||||
ctx->IdleWaitCV.notify_all();
|
||||
static void SleepThread(FEXCore::Context::ContextImpl *CTX, FEXCore::Core::CpuStateFrame *Frame) {
|
||||
CTX->SyscallHandler->SleepThread(CTX, Frame);
|
||||
}
|
||||
|
||||
constexpr size_t MAX_DISPATCHER_CODE_SIZE = 4096 * 2;
|
||||
|
@ -112,8 +112,6 @@ protected:
|
||||
FEXCore::Context::ContextImpl *CTX;
|
||||
DispatcherConfig config;
|
||||
|
||||
static void SleepThread(FEXCore::Context::ContextImpl *ctx, FEXCore::Core::CpuStateFrame *Frame);
|
||||
|
||||
using AsmDispatch = void(*)(FEXCore::Core::CpuStateFrame *Frame);
|
||||
using JITCallback = void(*)(FEXCore::Core::CpuStateFrame *Frame, uint64_t RIP);
|
||||
|
||||
|
@ -129,48 +129,6 @@ namespace FEXCore::Context {
|
||||
FEX_DEFAULT_VISIBILITY virtual void SetExitHandler(ExitHandler handler) = 0;
|
||||
FEX_DEFAULT_VISIBILITY virtual ExitHandler GetExitHandler() const = 0;
|
||||
|
||||
/**
|
||||
* @brief Pauses execution on the CPU core
|
||||
*
|
||||
* Blocks until all threads have paused.
|
||||
*/
|
||||
FEX_DEFAULT_VISIBILITY virtual void Pause() = 0;
|
||||
|
||||
/**
|
||||
* @brief Waits for all threads to be idle.
|
||||
*
|
||||
* Idling can happen when the process is shutting down or the debugger has asked for all threads to pause.
|
||||
*/
|
||||
FEX_DEFAULT_VISIBILITY virtual void WaitForIdle() = 0;
|
||||
|
||||
/**
|
||||
* @brief When resuming from a paused state, waits for all threads to start executing before returning.
|
||||
*/
|
||||
FEX_DEFAULT_VISIBILITY virtual void WaitForThreadsToRun() = 0;
|
||||
|
||||
/**
|
||||
* @brief Starts (or continues) the CPU core
|
||||
*
|
||||
* This function is async and returns immediately.
|
||||
* Use RunUntilExit() for synchonous executions
|
||||
*
|
||||
*/
|
||||
FEX_DEFAULT_VISIBILITY virtual void Run() = 0;
|
||||
|
||||
/**
|
||||
* @brief Tells the core to shutdown
|
||||
*
|
||||
* Blocks until shutdown
|
||||
*/
|
||||
FEX_DEFAULT_VISIBILITY virtual void Stop() = 0;
|
||||
|
||||
/**
|
||||
* @brief Executes one instruction
|
||||
*
|
||||
* Returns once execution is complete.
|
||||
*/
|
||||
FEX_DEFAULT_VISIBILITY virtual void Step() = 0;
|
||||
|
||||
/**
|
||||
* @brief Runs the CPU core until it exits
|
||||
*
|
||||
@ -191,17 +149,6 @@ namespace FEXCore::Context {
|
||||
FEX_DEFAULT_VISIBILITY virtual void CompileRIP(FEXCore::Core::InternalThreadState *Thread, uint64_t GuestRIP) = 0;
|
||||
FEX_DEFAULT_VISIBILITY virtual void CompileRIPCount(FEXCore::Core::InternalThreadState *Thread, uint64_t GuestRIP, uint64_t MaxInst) = 0;
|
||||
|
||||
/**
|
||||
* @brief [[theadsafe]] Checks if the Context is either done working or paused(in the case of single stepping)
|
||||
*
|
||||
* Use this when the context is async running to determine if it is done
|
||||
*
|
||||
* @param CTX the context that we created
|
||||
*
|
||||
* @return true if the core is done or paused
|
||||
*/
|
||||
FEX_DEFAULT_VISIBILITY virtual bool IsDone() const = 0;
|
||||
|
||||
/**
|
||||
* @brief Allows the frontend to pass in a custom CPUBackend creation factory
|
||||
*
|
||||
@ -248,23 +195,15 @@ namespace FEXCore::Context {
|
||||
*
|
||||
* @param InitialRIP The starting RIP of this thread
|
||||
* @param StackPointer The starting RSP of this thread
|
||||
* @param WhoManages The flag to determine what manages ownership of the InternalThreadState object
|
||||
* @param NewThreadState The thread state to inherit from if not nullptr.
|
||||
* @param ParentTID The thread ID that the parent is inheriting from
|
||||
*
|
||||
* @return A new InternalThreadState object for using with a new guest thread.
|
||||
*/
|
||||
|
||||
// TODO: This is a temporary construct and will be removed once the frontend has full ownership of InternalThreadState objects.
|
||||
enum class [[deprecated]] ManagedBy {
|
||||
CORE,
|
||||
FRONTEND,
|
||||
};
|
||||
FEX_DEFAULT_VISIBILITY virtual FEXCore::Core::InternalThreadState* CreateThread(uint64_t InitialRIP, uint64_t StackPointer, ManagedBy WhoManages, FEXCore::Core::CPUState *NewThreadState = nullptr, uint64_t ParentTID = 0) = 0;
|
||||
FEX_DEFAULT_VISIBILITY virtual FEXCore::Core::InternalThreadState* CreateThread(uint64_t InitialRIP, uint64_t StackPointer, FEXCore::Core::CPUState *NewThreadState = nullptr, uint64_t ParentTID = 0) = 0;
|
||||
|
||||
FEX_DEFAULT_VISIBILITY virtual void ExecutionThread(FEXCore::Core::InternalThreadState *Thread) = 0;
|
||||
FEX_DEFAULT_VISIBILITY virtual void RunThread(FEXCore::Core::InternalThreadState *Thread) = 0;
|
||||
FEX_DEFAULT_VISIBILITY virtual void StopThread(FEXCore::Core::InternalThreadState *Thread) = 0;
|
||||
FEX_DEFAULT_VISIBILITY virtual void DestroyThread(FEXCore::Core::InternalThreadState *Thread, bool NeedsTLSUninstall = false) = 0;
|
||||
#ifndef _WIN32
|
||||
FEX_DEFAULT_VISIBILITY virtual void LockBeforeFork(FEXCore::Core::InternalThreadState *Thread) {}
|
||||
@ -286,6 +225,8 @@ namespace FEXCore::Context {
|
||||
|
||||
FEX_DEFAULT_VISIBILITY virtual void FinalizeAOTIRCache() = 0;
|
||||
FEX_DEFAULT_VISIBILITY virtual void WriteFilesWithCode(AOTIRCodeFileWriterFn Writer) = 0;
|
||||
|
||||
FEX_DEFAULT_VISIBILITY virtual void ClearCodeCache(FEXCore::Core::InternalThreadState *Thread) = 0;
|
||||
FEX_DEFAULT_VISIBILITY virtual void InvalidateGuestCodeRange(FEXCore::Core::InternalThreadState *Thread, uint64_t Start, uint64_t Length) = 0;
|
||||
FEX_DEFAULT_VISIBILITY virtual void InvalidateGuestCodeRange(FEXCore::Core::InternalThreadState *Thread, uint64_t Start, uint64_t Length, CodeRangeInvalidationFn callback) = 0;
|
||||
FEX_DEFAULT_VISIBILITY virtual void MarkMemoryShared(FEXCore::Core::InternalThreadState *Thread) = 0;
|
||||
@ -303,8 +244,6 @@ namespace FEXCore::Context {
|
||||
|
||||
FEX_DEFAULT_VISIBILITY virtual void GetVDSOSigReturn(VDSOSigReturn *VDSOPointers) = 0;
|
||||
|
||||
FEX_DEFAULT_VISIBILITY virtual void IncrementIdleRefCount() = 0;
|
||||
|
||||
/**
|
||||
* @brief Informs the context if hardware TSO is supported.
|
||||
* Once hardware TSO is enabled, then TSO emulation through atomics is disabled and relies on the hardware.
|
||||
@ -321,13 +260,6 @@ namespace FEXCore::Context {
|
||||
*/
|
||||
FEX_DEFAULT_VISIBILITY virtual void EnableExitOnHLT() = 0;
|
||||
|
||||
/**
|
||||
* @brief Gets the thread data for FEX's internal tracked threads.
|
||||
*
|
||||
* @return struct containing all the thread information.
|
||||
*/
|
||||
FEX_DEFAULT_VISIBILITY virtual ThreadsState GetThreads() = 0;
|
||||
|
||||
private:
|
||||
};
|
||||
|
||||
|
@ -127,7 +127,6 @@ namespace FEXCore::Core {
|
||||
std::shared_ptr<FEXCore::CompileService> CompileService;
|
||||
|
||||
std::shared_mutex ObjectCacheRefCounter{};
|
||||
bool DestroyedByParent{false}; // Should the parent destroy this thread, or it destory itself
|
||||
|
||||
struct DeferredSignalState {
|
||||
#ifndef _WIN32
|
||||
|
@ -79,6 +79,9 @@ namespace FEXCore::HLE {
|
||||
virtual AOTIRCacheEntryLookupResult LookupAOTIRCacheEntry(FEXCore::Core::InternalThreadState *Thread, uint64_t GuestAddr) = 0;
|
||||
|
||||
virtual SourcecodeResolver *GetSourcecodeResolver() { return nullptr; }
|
||||
|
||||
virtual void SleepThread(FEXCore::Context::Context *CTX, FEXCore::Core::CpuStateFrame *Frame) {}
|
||||
|
||||
protected:
|
||||
SyscallOSABI OSABI;
|
||||
};
|
||||
|
@ -561,7 +561,7 @@ int main(int argc, char **argv, char **const envp) {
|
||||
if (!CTX->InitCore()) {
|
||||
return -1;
|
||||
}
|
||||
auto ParentThread = CTX->CreateThread(0, 0, FEXCore::Context::Context::ManagedBy::FRONTEND);
|
||||
auto ParentThread = CTX->CreateThread(0, 0);
|
||||
|
||||
// Calculate the base stats for instruction testing.
|
||||
CodeSize::Validation.CalculateBaseStats(CTX.get(), ParentThread);
|
||||
|
@ -106,7 +106,7 @@ void AOTGenSection(FEXCore::Context::Context *CTX, ELFCodeLoader::LoadedSection
|
||||
setpriority(PRIO_PROCESS, FHU::Syscalls::gettid(), 19);
|
||||
|
||||
// Setup thread - Each compilation thread uses its own backing FEX thread
|
||||
auto Thread = CTX->CreateThread(0, 0, FEXCore::Context::Context::ManagedBy::FRONTEND);
|
||||
auto Thread = CTX->CreateThread(0, 0);
|
||||
fextl::set<uint64_t> ExternalBranchesLocal;
|
||||
CTX->ConfigureAOTGen(Thread, &ExternalBranchesLocal, SectionMaxAddress);
|
||||
|
||||
|
@ -490,7 +490,7 @@ int main(int argc, char **argv, char **const envp) {
|
||||
return 1;
|
||||
}
|
||||
|
||||
auto ParentThread = CTX->CreateThread(Loader.DefaultRIP(), Loader.GetStackPointer(), FEXCore::Context::Context::ManagedBy::FRONTEND);
|
||||
auto ParentThread = SyscallHandler->TM.CreateThread(Loader.DefaultRIP(), Loader.GetStackPointer());
|
||||
|
||||
// Pass in our VDSO thunks
|
||||
CTX->AppendThunkDefinitions(FEX::VDSO::GetVDSOThunkDefinitions());
|
||||
@ -503,7 +503,7 @@ int main(int argc, char **argv, char **const envp) {
|
||||
CTX->SetExitHandler([&](uint64_t thread, FEXCore::Context::ExitReason reason) {
|
||||
if (reason != FEXCore::Context::ExitReason::EXIT_DEBUG) {
|
||||
ShutdownReason = reason;
|
||||
CTX->Stop();
|
||||
SyscallHandler->TM.Stop();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -181,7 +181,7 @@ int main(int argc, char **argv, char **const envp)
|
||||
if (!CTX->InitCore()) {
|
||||
return -1;
|
||||
}
|
||||
auto ParentThread = CTX->CreateThread(Loader.DefaultRIP(), Loader.GetStackPointer(), FEXCore::Context::Context::ManagedBy::FRONTEND);
|
||||
auto ParentThread = CTX->CreateThread(Loader.DefaultRIP(), Loader.GetStackPointer());
|
||||
|
||||
auto ShutdownReason = FEXCore::Context::ExitReason::EXIT_SHUTDOWN;
|
||||
|
||||
@ -192,7 +192,6 @@ int main(int argc, char **argv, char **const envp)
|
||||
if (reason != FEXCore::Context::ExitReason::EXIT_DEBUG)
|
||||
{
|
||||
ShutdownReason = reason;
|
||||
CTX->Stop();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -10,6 +10,7 @@ set (SRCS
|
||||
LinuxSyscalls/Syscalls.cpp
|
||||
LinuxSyscalls/SyscallsSMCTracking.cpp
|
||||
LinuxSyscalls/SyscallsVMATracking.cpp
|
||||
LinuxSyscalls/ThreadManager.cpp
|
||||
LinuxSyscalls/Utils/Threads.cpp
|
||||
LinuxSyscalls/x32/Syscalls.cpp
|
||||
LinuxSyscalls/x32/EPoll.cpp
|
||||
|
@ -135,7 +135,7 @@ GdbServer::~GdbServer() {
|
||||
}
|
||||
}
|
||||
|
||||
GdbServer::GdbServer(FEXCore::Context::Context *ctx, FEX::HLE::SignalDelegator *SignalDelegation, FEXCore::HLE::SyscallHandler *const SyscallHandler)
|
||||
GdbServer::GdbServer(FEXCore::Context::Context *ctx, FEX::HLE::SignalDelegator *SignalDelegation, FEX::HLE::SyscallHandler *const SyscallHandler)
|
||||
: CTX(ctx)
|
||||
, SyscallHandler {SyscallHandler} {
|
||||
// Pass all signals by default
|
||||
@ -339,11 +339,11 @@ fextl::string GdbServer::readRegs() {
|
||||
GDBContextDefinition GDB{};
|
||||
FEXCore::Core::CPUState state{};
|
||||
|
||||
auto Threads = CTX->GetThreads();
|
||||
FEXCore::Core::InternalThreadState *CurrentThread { Threads.ParentThread };
|
||||
auto Threads = SyscallHandler->TM.GetThreads();
|
||||
FEXCore::Core::InternalThreadState *CurrentThread { Threads->at(0) };
|
||||
bool Found = false;
|
||||
|
||||
for (auto &Thread : *Threads.Threads) {
|
||||
for (auto &Thread : *Threads) {
|
||||
if (Thread->ThreadManager.GetTID() != CurrentDebuggingThread) {
|
||||
continue;
|
||||
}
|
||||
@ -355,7 +355,7 @@ fextl::string GdbServer::readRegs() {
|
||||
|
||||
if (!Found) {
|
||||
// If set to an invalid thread then just get the parent thread ID
|
||||
memcpy(&state, Threads.ParentThread->CurrentFrame, sizeof(state));
|
||||
memcpy(&state, CurrentThread->CurrentFrame, sizeof(state));
|
||||
}
|
||||
|
||||
// Encode the GDB context definition
|
||||
@ -390,11 +390,11 @@ GdbServer::HandledPacketType GdbServer::readReg(const fextl::string& packet) {
|
||||
|
||||
FEXCore::Core::CPUState state{};
|
||||
|
||||
auto Threads = CTX->GetThreads();
|
||||
FEXCore::Core::InternalThreadState *CurrentThread { Threads.ParentThread };
|
||||
auto Threads = SyscallHandler->TM.GetThreads();
|
||||
FEXCore::Core::InternalThreadState *CurrentThread { Threads->at(0) };
|
||||
bool Found = false;
|
||||
|
||||
for (auto &Thread : *Threads.Threads) {
|
||||
for (auto &Thread : *Threads) {
|
||||
if (Thread->ThreadManager.GetTID() != CurrentDebuggingThread) {
|
||||
continue;
|
||||
}
|
||||
@ -406,7 +406,7 @@ GdbServer::HandledPacketType GdbServer::readReg(const fextl::string& packet) {
|
||||
|
||||
if (!Found) {
|
||||
// If set to an invalid thread then just get the parent thread ID
|
||||
memcpy(&state, Threads.ParentThread->CurrentFrame, sizeof(state));
|
||||
memcpy(&state, CurrentThread->CurrentFrame, sizeof(state));
|
||||
}
|
||||
|
||||
|
||||
@ -744,12 +744,12 @@ GdbServer::HandledPacketType GdbServer::handleXfer(const fextl::string &packet)
|
||||
|
||||
if (object == "threads") {
|
||||
if (offset == 0) {
|
||||
auto Threads = CTX->GetThreads();
|
||||
auto Threads = SyscallHandler->TM.GetThreads();
|
||||
|
||||
ThreadString.clear();
|
||||
fextl::ostringstream ss;
|
||||
ss << "<threads>\n";
|
||||
for (auto &Thread : *Threads.Threads) {
|
||||
for (auto &Thread : *Threads) {
|
||||
// Thread id is in hex without 0x prefix
|
||||
const auto ThreadName = getThreadName(Thread->ThreadManager.GetTID());
|
||||
ss << "<thread id=\"" << std::hex << Thread->ThreadManager.GetTID() << "\"";
|
||||
@ -982,14 +982,14 @@ GdbServer::HandledPacketType GdbServer::handleQuery(const fextl::string &packet)
|
||||
return {"", HandledPacketType::TYPE_ACK};
|
||||
}
|
||||
if (match("qfThreadInfo")) {
|
||||
auto Threads = CTX->GetThreads();
|
||||
auto Threads = SyscallHandler->TM.GetThreads();
|
||||
|
||||
fextl::ostringstream ss;
|
||||
ss << "m";
|
||||
for (size_t i = 0; i < Threads.Threads->size(); ++i) {
|
||||
auto Thread = Threads.Threads->at(i);
|
||||
for (size_t i = 0; i < Threads->size(); ++i) {
|
||||
auto Thread = Threads->at(i);
|
||||
ss << std::hex << Thread->ThreadManager.TID;
|
||||
if (i != (Threads.Threads->size() - 1)) {
|
||||
if (i != (Threads->size() - 1)) {
|
||||
ss << ",";
|
||||
}
|
||||
}
|
||||
@ -1009,8 +1009,9 @@ GdbServer::HandledPacketType GdbServer::handleQuery(const fextl::string &packet)
|
||||
}
|
||||
if (match("qC")) {
|
||||
// Returns the current Thread ID
|
||||
auto Threads = SyscallHandler->TM.GetThreads();
|
||||
fextl::ostringstream ss;
|
||||
ss << "m" << std::hex << CTX->GetThreads().ParentThread->ThreadManager.TID;
|
||||
ss << "m" << std::hex << Threads->at(0)->ThreadManager.TID;
|
||||
return {ss.str(), HandledPacketType::TYPE_ACK};
|
||||
}
|
||||
if (match("QStartNoAckMode")) {
|
||||
@ -1114,13 +1115,13 @@ GdbServer::HandledPacketType GdbServer::handleQuery(const fextl::string &packet)
|
||||
GdbServer::HandledPacketType GdbServer::ThreadAction(char action, uint32_t tid) {
|
||||
switch (action) {
|
||||
case 'c': {
|
||||
CTX->Run();
|
||||
SyscallHandler->TM.Run();
|
||||
ThreadBreakEvent.NotifyAll();
|
||||
CTX->WaitForThreadsToRun();
|
||||
SyscallHandler->TM.WaitForThreadsToRun();
|
||||
return {"", HandledPacketType::TYPE_ONLYACK};
|
||||
}
|
||||
case 's': {
|
||||
CTX->Step();
|
||||
SyscallHandler->TM.Step();
|
||||
SendPacketPair({"OK", HandledPacketType::TYPE_ACK});
|
||||
fextl::string str = fextl::fmt::format("T05thread:{:02x};", getpid());
|
||||
if (LibraryMapChanged) {
|
||||
@ -1133,7 +1134,7 @@ GdbServer::HandledPacketType GdbServer::ThreadAction(char action, uint32_t tid)
|
||||
}
|
||||
case 't':
|
||||
// This thread isn't part of the thread pool
|
||||
CTX->Stop();
|
||||
SyscallHandler->TM.Stop();
|
||||
return {"OK", HandledPacketType::TYPE_ACK};
|
||||
default:
|
||||
return {"E00", HandledPacketType::TYPE_ACK};
|
||||
@ -1241,7 +1242,7 @@ GdbServer::HandledPacketType GdbServer::handleThreadOp(const fextl::string &pack
|
||||
ss.seekg(fextl::string("Hc").size());
|
||||
ss >> std::hex >> CurrentDebuggingThread;
|
||||
|
||||
CTX->Pause();
|
||||
SyscallHandler->TM.Pause();
|
||||
return {"OK", HandledPacketType::TYPE_ACK};
|
||||
}
|
||||
|
||||
@ -1252,7 +1253,7 @@ GdbServer::HandledPacketType GdbServer::handleThreadOp(const fextl::string &pack
|
||||
ss >> std::hex >> CurrentDebuggingThread;
|
||||
|
||||
// This must return quick otherwise IDA complains
|
||||
CTX->Pause();
|
||||
SyscallHandler->TM.Pause();
|
||||
return {"OK", HandledPacketType::TYPE_ACK};
|
||||
}
|
||||
|
||||
@ -1272,7 +1273,7 @@ GdbServer::HandledPacketType GdbServer::handleBreakpoint(const fextl::string &pa
|
||||
ss.get(); // discard comma
|
||||
ss >> std::hex >> Type;
|
||||
|
||||
CTX->Pause();
|
||||
SyscallHandler->TM.Pause();
|
||||
return {"OK", HandledPacketType::TYPE_ACK};
|
||||
}
|
||||
|
||||
@ -1291,13 +1292,13 @@ GdbServer::HandledPacketType GdbServer::ProcessPacket(const fextl::string &packe
|
||||
case 'D':
|
||||
// Detach
|
||||
// Ensure the threads are back in running state on detach
|
||||
CTX->Run();
|
||||
CTX->WaitForThreadsToRun();
|
||||
SyscallHandler->TM.Run();
|
||||
SyscallHandler->TM.WaitForThreadsToRun();
|
||||
return {"OK", HandledPacketType::TYPE_ACK};
|
||||
case 'g':
|
||||
// We might be running while we try reading
|
||||
// Pause up front
|
||||
CTX->Pause();
|
||||
SyscallHandler->TM.Pause();
|
||||
return {readRegs(), HandledPacketType::TYPE_ACK};
|
||||
case 'p':
|
||||
return readReg(packet);
|
||||
@ -1320,8 +1321,8 @@ GdbServer::HandledPacketType GdbServer::ProcessPacket(const fextl::string &packe
|
||||
case 'Z': // Inserts breakpoint or watchpoint
|
||||
return handleBreakpoint(packet);
|
||||
case 'k': // Kill the process
|
||||
CTX->Stop();
|
||||
CTX->WaitForIdle(); // Block until exit
|
||||
SyscallHandler->TM.Stop();
|
||||
SyscallHandler->TM.WaitForIdle(); // Block until exit
|
||||
return {"", HandledPacketType::TYPE_NONE};
|
||||
default:
|
||||
return {"", HandledPacketType::TYPE_UNKNOWN};
|
||||
@ -1416,7 +1417,7 @@ void GdbServer::GdbServerLoop() {
|
||||
}
|
||||
break;
|
||||
case '\x03': { // ASCII EOT
|
||||
CTX->Pause();
|
||||
SyscallHandler->TM.Pause();
|
||||
fextl::string str = fextl::fmt::format("T02thread:{:02x};", getpid());
|
||||
if (LibraryMapChanged) {
|
||||
// If libraries have changed then let gdb know
|
||||
|
@ -25,7 +25,7 @@ namespace FEX {
|
||||
|
||||
class GdbServer {
|
||||
public:
|
||||
GdbServer(FEXCore::Context::Context *ctx, FEX::HLE::SignalDelegator *SignalDelegation, FEXCore::HLE::SyscallHandler *const SyscallHandler);
|
||||
GdbServer(FEXCore::Context::Context *ctx, FEX::HLE::SignalDelegator *SignalDelegation, FEX::HLE::SyscallHandler *const SyscallHandler);
|
||||
~GdbServer();
|
||||
|
||||
// Public for threading
|
||||
@ -84,7 +84,7 @@ private:
|
||||
HandledPacketType readReg(const fextl::string& packet);
|
||||
|
||||
FEXCore::Context::Context *CTX;
|
||||
FEXCore::HLE::SyscallHandler *const SyscallHandler;
|
||||
FEX::HLE::SyscallHandler *const SyscallHandler;
|
||||
fextl::unique_ptr<FEXCore::Threads::Thread> gdbServerThread;
|
||||
fextl::unique_ptr<std::iostream> CommsStream;
|
||||
std::mutex sendMutex;
|
||||
|
@ -7,6 +7,7 @@ $end_info$
|
||||
*/
|
||||
|
||||
#include "LinuxSyscalls/SignalDelegator.h"
|
||||
#include "LinuxSyscalls/Syscalls.h"
|
||||
|
||||
#include <FEXCore/Core/Context.h>
|
||||
#include <FEXCore/Core/CoreState.h>
|
||||
@ -1364,7 +1365,7 @@ namespace FEX::HLE {
|
||||
if (Thread->RunningEvents.ThreadSleeping) {
|
||||
// If the thread was sleeping then its idle counter was decremented
|
||||
// Reincrement it here to not break logic
|
||||
CTX->IncrementIdleRefCount();
|
||||
FEX::HLE::_SyscallHandler->TM.IncrementIdleRefCount();
|
||||
}
|
||||
|
||||
Thread->SignalReason.store(FEXCore::Core::SignalEvent::Nothing);
|
||||
|
@ -606,10 +606,7 @@ uint64_t CloneHandler(FEXCore::Core::CpuStateFrame *Frame, FEX::HLE::clone3_args
|
||||
if (Result != 0) {
|
||||
// Parent
|
||||
// Unlock the mutexes on both sides of the fork
|
||||
FEX::HLE::_SyscallHandler->UnlockAfterFork(false);
|
||||
|
||||
// Clear all the other threads that are being tracked
|
||||
Thread->CTX->UnlockAfterFork(Frame->Thread, false);
|
||||
FEX::HLE::_SyscallHandler->UnlockAfterFork(Frame->Thread, false);
|
||||
|
||||
::syscall(SYS_rt_sigprocmask, SIG_SETMASK, &args->SignalMask, nullptr, sizeof(args->SignalMask));
|
||||
}
|
||||
@ -648,12 +645,8 @@ uint64_t CloneHandler(FEXCore::Core::CpuStateFrame *Frame, FEX::HLE::clone3_args
|
||||
// Return the new threads TID
|
||||
uint64_t Result = NewThread->ThreadManager.GetTID();
|
||||
|
||||
if (flags & CLONE_VFORK) {
|
||||
NewThread->DestroyedByParent = true;
|
||||
}
|
||||
|
||||
// Actually start the thread
|
||||
Thread->CTX->RunThread(NewThread);
|
||||
FEX::HLE::_SyscallHandler->TM.RunThread(NewThread);
|
||||
|
||||
if (flags & CLONE_VFORK) {
|
||||
// If VFORK is set then the calling process is suspended until the thread exits with execve or exit
|
||||
@ -744,7 +737,8 @@ void SyscallHandler::DefaultProgramBreak(uint64_t Base, uint64_t Size) {
|
||||
}
|
||||
|
||||
SyscallHandler::SyscallHandler(FEXCore::Context::Context *_CTX, FEX::HLE::SignalDelegator *_SignalDelegation)
|
||||
: FM {_CTX}
|
||||
: TM {_CTX, _SignalDelegation}
|
||||
, FM {_CTX}
|
||||
, CTX {_CTX}
|
||||
, SignalDelegation {_SignalDelegation} {
|
||||
FEX::HLE::_SyscallHandler = this;
|
||||
@ -840,13 +834,16 @@ void SyscallHandler::LockBeforeFork() {
|
||||
VMATracking.Mutex.lock();
|
||||
}
|
||||
|
||||
void SyscallHandler::UnlockAfterFork(bool Child) {
|
||||
void SyscallHandler::UnlockAfterFork(FEXCore::Core::InternalThreadState *LiveThread, bool Child) {
|
||||
if (Child) {
|
||||
VMATracking.Mutex.StealAndDropActiveLocks();
|
||||
}
|
||||
else {
|
||||
VMATracking.Mutex.unlock();
|
||||
}
|
||||
|
||||
// Clear all the other threads that are being tracked
|
||||
TM.UnlockAfterFork(LiveThread, Child);
|
||||
}
|
||||
|
||||
static bool isHEX(char c) {
|
||||
|
@ -96,8 +96,77 @@ struct ExecveAtArgs {
|
||||
|
||||
uint64_t ExecveHandler(const char *pathname, char* const* argv, char* const* envp, ExecveAtArgs Args);
|
||||
|
||||
class ThreadManager final {
|
||||
public:
|
||||
ThreadManager(FEXCore::Context::Context *CTX, FEX::HLE::SignalDelegator *SignalDelegation)
|
||||
: CTX {CTX}
|
||||
, SignalDelegation {SignalDelegation} {}
|
||||
|
||||
~ThreadManager();
|
||||
|
||||
FEXCore::Core::InternalThreadState *CreateThread(uint64_t InitialRIP, uint64_t StackPointer, FEXCore::Core::CPUState *NewThreadState = nullptr, uint64_t ParentTID = 0);
|
||||
|
||||
void DestroyThread(FEXCore::Core::InternalThreadState *Thread);
|
||||
void StopThread(FEXCore::Core::InternalThreadState *Thread);
|
||||
void RunThread(FEXCore::Core::InternalThreadState *Thread);
|
||||
|
||||
void Pause();
|
||||
void Run();
|
||||
void Step();
|
||||
void Stop(bool IgnoreCurrentThread = false);
|
||||
|
||||
void WaitForIdle();
|
||||
void WaitForIdleWithTimeout();
|
||||
void WaitForThreadsToRun();
|
||||
|
||||
void SleepThread(FEXCore::Context::Context *CTX, FEXCore::Core::CpuStateFrame *Frame);
|
||||
|
||||
void UnlockAfterFork(FEXCore::Core::InternalThreadState *Thread, bool Child);
|
||||
|
||||
void IncrementIdleRefCount() {
|
||||
++IdleWaitRefCount;
|
||||
}
|
||||
|
||||
void InvalidateGuestCodeRange(FEXCore::Core::InternalThreadState *CallingThread, uint64_t Start, uint64_t Length) {
|
||||
std::lock_guard lk(ThreadCreationMutex);
|
||||
|
||||
for (auto &Thread : Threads) {
|
||||
CTX->InvalidateGuestCodeRange(Thread, Start, Length);
|
||||
}
|
||||
}
|
||||
|
||||
void InvalidateGuestCodeRange(FEXCore::Core::InternalThreadState *CallingThread, uint64_t Start, uint64_t Length, FEXCore::Context::CodeRangeInvalidationFn callback) {
|
||||
std::lock_guard lk(ThreadCreationMutex);
|
||||
|
||||
for (auto &Thread : Threads) {
|
||||
CTX->InvalidateGuestCodeRange(Thread, Start, Length, callback);
|
||||
}
|
||||
}
|
||||
|
||||
fextl::vector<FEXCore::Core::InternalThreadState *> const *GetThreads() const {
|
||||
return &Threads;
|
||||
}
|
||||
|
||||
private:
|
||||
FEXCore::Context::Context *CTX;
|
||||
FEX::HLE::SignalDelegator *SignalDelegation;
|
||||
|
||||
std::mutex ThreadCreationMutex;
|
||||
fextl::vector<FEXCore::Core::InternalThreadState *> Threads;
|
||||
|
||||
// Thread idling support.
|
||||
bool Running{};
|
||||
std::mutex IdleWaitMutex;
|
||||
std::condition_variable IdleWaitCV;
|
||||
std::atomic<uint32_t> IdleWaitRefCount{};
|
||||
|
||||
void HandleThreadDeletion(FEXCore::Core::InternalThreadState *Thread);
|
||||
void NotifyPause();
|
||||
};
|
||||
|
||||
class SyscallHandler : public FEXCore::HLE::SyscallHandler, FEXCore::HLE::SourcecodeResolver, public FEXCore::Allocator::FEXAllocOperators {
|
||||
public:
|
||||
ThreadManager TM;
|
||||
virtual ~SyscallHandler();
|
||||
|
||||
// In the case that the syscall doesn't hit the optimized path then we still need to go here
|
||||
@ -221,10 +290,14 @@ public:
|
||||
|
||||
///// FORK tracking /////
|
||||
void LockBeforeFork();
|
||||
void UnlockAfterFork(bool Child);
|
||||
void UnlockAfterFork(FEXCore::Core::InternalThreadState *LiveThread, bool Child);
|
||||
|
||||
SourcecodeResolver *GetSourcecodeResolver() override { return this; }
|
||||
|
||||
void SleepThread(FEXCore::Context::Context *CTX, FEXCore::Core::CpuStateFrame *Frame) override {
|
||||
TM.SleepThread(CTX, Frame);
|
||||
}
|
||||
|
||||
bool NeedXIDCheck() const { return NeedToCheckXID; }
|
||||
void DisableXIDCheck() { NeedToCheckXID = false; }
|
||||
|
||||
|
@ -73,7 +73,7 @@ namespace FEX::HLE {
|
||||
NewThreadState.gregs[FEXCore::X86State::REG_RSP] = args->args.stack;
|
||||
}
|
||||
|
||||
auto NewThread = CTX->CreateThread(0, 0, FEXCore::Context::Context::ManagedBy::CORE, &NewThreadState, args->args.parent_tid);
|
||||
auto NewThread = FEX::HLE::_SyscallHandler->TM.CreateThread(0, 0, &NewThreadState, args->args.parent_tid);
|
||||
|
||||
if (FEX::HLE::_SyscallHandler->Is64BitMode()) {
|
||||
if (flags & CLONE_SETTLS) {
|
||||
@ -166,7 +166,7 @@ namespace FEX::HLE {
|
||||
}
|
||||
|
||||
// Overwrite thread
|
||||
NewThread = CTX->CreateThread(0, 0, FEXCore::Context::Context::ManagedBy::CORE, &NewThreadState, GuestArgs->parent_tid);
|
||||
NewThread = FEX::HLE::_SyscallHandler->TM.CreateThread(0, 0, &NewThreadState, GuestArgs->parent_tid);
|
||||
|
||||
// CLONE_PARENT_SETTID, CLONE_CHILD_SETTID, CLONE_CHILD_CLEARTID, CLONE_PIDFD will be handled by kernel
|
||||
// Call execution thread directly since we already are on the new thread
|
||||
@ -177,10 +177,7 @@ namespace FEX::HLE {
|
||||
// Clear all the other threads that are being tracked
|
||||
// Frame->Thread is /ONLY/ safe to access when CLONE_THREAD flag is not set
|
||||
// Unlock the mutexes on both sides of the fork
|
||||
FEX::HLE::_SyscallHandler->UnlockAfterFork(true);
|
||||
|
||||
// Clear all the other threads that are being tracked
|
||||
Thread->CTX->UnlockAfterFork(Frame->Thread, true);
|
||||
FEX::HLE::_SyscallHandler->UnlockAfterFork(Frame->Thread, true);
|
||||
|
||||
::syscall(SYS_rt_sigprocmask, SIG_SETMASK, &CloneArgs->SignalMask, nullptr, sizeof(CloneArgs->SignalMask));
|
||||
|
||||
@ -266,10 +263,7 @@ namespace FEX::HLE {
|
||||
|
||||
if (IsChild) {
|
||||
// Unlock the mutexes on both sides of the fork
|
||||
FEX::HLE::_SyscallHandler->UnlockAfterFork(IsChild);
|
||||
|
||||
// Clear all the other threads that are being tracked
|
||||
Thread->CTX->UnlockAfterFork(Frame->Thread, IsChild);
|
||||
FEX::HLE::_SyscallHandler->UnlockAfterFork(Frame->Thread, IsChild);
|
||||
|
||||
::syscall(SYS_rt_sigprocmask, SIG_SETMASK, &Mask, nullptr, sizeof(Mask));
|
||||
|
||||
@ -326,10 +320,7 @@ namespace FEX::HLE {
|
||||
}
|
||||
|
||||
// Unlock the mutexes on both sides of the fork
|
||||
FEX::HLE::_SyscallHandler->UnlockAfterFork(IsChild);
|
||||
|
||||
// Clear all the other threads that are being tracked
|
||||
Thread->CTX->UnlockAfterFork(Frame->Thread, IsChild);
|
||||
FEX::HLE::_SyscallHandler->UnlockAfterFork(Frame->Thread, IsChild);
|
||||
|
||||
::syscall(SYS_rt_sigprocmask, SIG_SETMASK, &Mask, nullptr, sizeof(Mask));
|
||||
|
||||
@ -404,7 +395,7 @@ namespace FEX::HLE {
|
||||
}
|
||||
|
||||
Thread->StatusCode = status;
|
||||
Thread->CTX->StopThread(Thread);
|
||||
FEX::HLE::_SyscallHandler->TM.StopThread(Thread);
|
||||
|
||||
return 0;
|
||||
});
|
||||
|
@ -48,8 +48,6 @@ auto SyscallHandler::VMAFlags::fromFlags(int Flags) -> VMAFlags {
|
||||
|
||||
// SMC interactions
|
||||
bool SyscallHandler::HandleSegfault(FEXCore::Core::InternalThreadState *Thread, int Signal, void *info, void *ucontext) {
|
||||
auto CTX = Thread->CTX;
|
||||
|
||||
const auto FaultAddress = (uintptr_t)((siginfo_t *)info)->si_addr;
|
||||
|
||||
{
|
||||
@ -82,17 +80,17 @@ bool SyscallHandler::HandleSegfault(FEXCore::Core::InternalThreadState *Thread,
|
||||
auto FaultBaseMirrored = Offset - VMA->Offset + VMA->Base;
|
||||
|
||||
if (VMA->Prot.Writable) {
|
||||
CTX->InvalidateGuestCodeRange(Thread, FaultBaseMirrored, FHU::FEX_PAGE_SIZE, [](uintptr_t Start, uintptr_t Length) {
|
||||
_SyscallHandler->TM.InvalidateGuestCodeRange(Thread, FaultBaseMirrored, FHU::FEX_PAGE_SIZE, [](uintptr_t Start, uintptr_t Length) {
|
||||
auto rv = mprotect((void *)Start, Length, PROT_READ | PROT_WRITE);
|
||||
LogMan::Throw::AAFmt(rv == 0, "mprotect({}, {}) failed", Start, Length);
|
||||
});
|
||||
} else {
|
||||
CTX->InvalidateGuestCodeRange(Thread, FaultBaseMirrored, FHU::FEX_PAGE_SIZE);
|
||||
_SyscallHandler->TM.InvalidateGuestCodeRange(Thread, FaultBaseMirrored, FHU::FEX_PAGE_SIZE);
|
||||
}
|
||||
}
|
||||
} while ((VMA = VMA->ResourceNextVMA));
|
||||
} else {
|
||||
CTX->InvalidateGuestCodeRange(Thread, FaultBase, FHU::FEX_PAGE_SIZE, [](uintptr_t Start, uintptr_t Length) {
|
||||
_SyscallHandler->TM.InvalidateGuestCodeRange(Thread, FaultBase, FHU::FEX_PAGE_SIZE, [](uintptr_t Start, uintptr_t Length) {
|
||||
auto rv = mprotect((void *)Start, Length, PROT_READ | PROT_WRITE);
|
||||
LogMan::Throw::AAFmt(rv == 0, "mprotect({}, {}) failed", Start, Length);
|
||||
});
|
||||
@ -234,7 +232,7 @@ void SyscallHandler::TrackMmap(FEXCore::Core::InternalThreadState *Thread, uintp
|
||||
|
||||
if (SMCChecks != FEXCore::Config::CONFIG_SMC_NONE) {
|
||||
// VMATracking.Mutex can't be held while executing this, otherwise it hangs if the JIT is in the process of looking up code in the AOT JIT.
|
||||
CTX->InvalidateGuestCodeRange(Thread, (uintptr_t)Base, Size);
|
||||
_SyscallHandler->TM.InvalidateGuestCodeRange(Thread, (uintptr_t)Base, Size);
|
||||
}
|
||||
}
|
||||
|
||||
@ -251,7 +249,7 @@ void SyscallHandler::TrackMunmap(FEXCore::Core::InternalThreadState *Thread, uin
|
||||
}
|
||||
|
||||
if (SMCChecks != FEXCore::Config::CONFIG_SMC_NONE) {
|
||||
CTX->InvalidateGuestCodeRange(Thread, (uintptr_t)Base, Size);
|
||||
_SyscallHandler->TM.InvalidateGuestCodeRange(Thread, (uintptr_t)Base, Size);
|
||||
}
|
||||
}
|
||||
|
||||
@ -265,7 +263,7 @@ void SyscallHandler::TrackMprotect(FEXCore::Core::InternalThreadState *Thread, u
|
||||
}
|
||||
|
||||
if (SMCChecks != FEXCore::Config::CONFIG_SMC_NONE) {
|
||||
CTX->InvalidateGuestCodeRange(Thread, Base, Size);
|
||||
_SyscallHandler->TM.InvalidateGuestCodeRange(Thread, Base, Size);
|
||||
}
|
||||
}
|
||||
|
||||
@ -310,12 +308,12 @@ void SyscallHandler::TrackMremap(FEXCore::Core::InternalThreadState *Thread, uin
|
||||
if (OldAddress != NewAddress) {
|
||||
if (OldSize != 0) {
|
||||
// This also handles the MREMAP_DONTUNMAP case
|
||||
CTX->InvalidateGuestCodeRange(Thread, OldAddress, OldSize);
|
||||
_SyscallHandler->TM.InvalidateGuestCodeRange(Thread, OldAddress, OldSize);
|
||||
}
|
||||
} else {
|
||||
// If mapping shrunk, flush the unmapped region
|
||||
if (OldSize > NewSize) {
|
||||
CTX->InvalidateGuestCodeRange(Thread, OldAddress + NewSize, OldSize - NewSize);
|
||||
_SyscallHandler->TM.InvalidateGuestCodeRange(Thread, OldAddress + NewSize, OldSize - NewSize);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -347,7 +345,7 @@ void SyscallHandler::TrackShmat(FEXCore::Core::InternalThreadState *Thread, int
|
||||
);
|
||||
}
|
||||
if (SMCChecks != FEXCore::Config::CONFIG_SMC_NONE) {
|
||||
CTX->InvalidateGuestCodeRange(Thread, Base, Length);
|
||||
_SyscallHandler->TM.InvalidateGuestCodeRange(Thread, Base, Length);
|
||||
}
|
||||
}
|
||||
|
||||
@ -361,7 +359,7 @@ void SyscallHandler::TrackShmdt(FEXCore::Core::InternalThreadState *Thread, uint
|
||||
|
||||
if (SMCChecks != FEXCore::Config::CONFIG_SMC_NONE) {
|
||||
// This might over flush if the shm has holes in it
|
||||
CTX->InvalidateGuestCodeRange(Thread, Base, Length);
|
||||
_SyscallHandler->TM.InvalidateGuestCodeRange(Thread, Base, Length);
|
||||
}
|
||||
}
|
||||
|
||||
|
250
Source/Tools/LinuxEmulation/LinuxSyscalls/ThreadManager.cpp
Normal file
250
Source/Tools/LinuxEmulation/LinuxSyscalls/ThreadManager.cpp
Normal file
@ -0,0 +1,250 @@
|
||||
// SPDX-License-Identifier: MIT
|
||||
|
||||
#include "LinuxSyscalls/Syscalls.h"
|
||||
#include "LinuxSyscalls/SignalDelegator.h"
|
||||
|
||||
#include <FEXHeaderUtils/Syscalls.h>
|
||||
|
||||
namespace FEX::HLE {
|
||||
FEXCore::Core::InternalThreadState *ThreadManager::CreateThread(uint64_t InitialRIP, uint64_t StackPointer, FEXCore::Core::CPUState *NewThreadState, uint64_t ParentTID) {
|
||||
auto Thread = CTX->CreateThread(InitialRIP, StackPointer, NewThreadState, ParentTID);
|
||||
|
||||
std::lock_guard lk(ThreadCreationMutex);
|
||||
Threads.emplace_back(Thread);
|
||||
|
||||
++IdleWaitRefCount;
|
||||
return Thread;
|
||||
}
|
||||
|
||||
void ThreadManager::DestroyThread(FEXCore::Core::InternalThreadState *Thread) {
|
||||
{
|
||||
std::lock_guard lk(ThreadCreationMutex);
|
||||
auto It = std::find(Threads.begin(), Threads.end(), Thread);
|
||||
LOGMAN_THROW_A_FMT(It != Threads.end(), "Thread wasn't in Threads");
|
||||
Threads.erase(It);
|
||||
}
|
||||
|
||||
HandleThreadDeletion(Thread);
|
||||
}
|
||||
|
||||
void ThreadManager::StopThread(FEXCore::Core::InternalThreadState *Thread) {
|
||||
if (Thread->RunningEvents.Running.exchange(false)) {
|
||||
SignalDelegation->SignalThread(Thread, FEXCore::Core::SignalEvent::Stop);
|
||||
}
|
||||
}
|
||||
|
||||
void ThreadManager::RunThread(FEXCore::Core::InternalThreadState *Thread) {
|
||||
// Tell the thread to start executing
|
||||
Thread->StartRunning.NotifyAll();
|
||||
}
|
||||
|
||||
void ThreadManager::HandleThreadDeletion(FEXCore::Core::InternalThreadState *Thread) {
|
||||
if (Thread->ExecutionThread) {
|
||||
if (Thread->ExecutionThread->joinable()) {
|
||||
Thread->ExecutionThread->join(nullptr);
|
||||
}
|
||||
|
||||
if (Thread->ExecutionThread->IsSelf()) {
|
||||
Thread->ExecutionThread->detach();
|
||||
}
|
||||
}
|
||||
|
||||
CTX->DestroyThread(Thread);
|
||||
--IdleWaitRefCount;
|
||||
IdleWaitCV.notify_all();
|
||||
}
|
||||
|
||||
void ThreadManager::NotifyPause() {
|
||||
// Tell all the threads that they should pause
|
||||
std::lock_guard lk(ThreadCreationMutex);
|
||||
for (auto &Thread : Threads) {
|
||||
SignalDelegation->SignalThread(Thread, FEXCore::Core::SignalEvent::Pause);
|
||||
}
|
||||
}
|
||||
|
||||
void ThreadManager::Pause() {
|
||||
NotifyPause();
|
||||
WaitForIdle();
|
||||
}
|
||||
|
||||
void ThreadManager::Run() {
|
||||
// Spin up all the threads
|
||||
std::lock_guard lk(ThreadCreationMutex);
|
||||
for (auto &Thread : Threads) {
|
||||
Thread->SignalReason.store(FEXCore::Core::SignalEvent::Return);
|
||||
}
|
||||
|
||||
for (auto &Thread : Threads) {
|
||||
Thread->StartRunning.NotifyAll();
|
||||
}
|
||||
}
|
||||
|
||||
void ThreadManager::WaitForIdleWithTimeout() {
|
||||
std::unique_lock<std::mutex> lk(IdleWaitMutex);
|
||||
bool WaitResult = IdleWaitCV.wait_for(lk, std::chrono::milliseconds(1500),
|
||||
[this] {
|
||||
return IdleWaitRefCount.load() == 0;
|
||||
});
|
||||
|
||||
if (!WaitResult) {
|
||||
// The wait failed, this will occur if we stepped in to a syscall
|
||||
// That's okay, we just need to pause the threads manually
|
||||
NotifyPause();
|
||||
}
|
||||
|
||||
// We have sent every thread a pause signal
|
||||
// Now wait again because they /will/ be going to sleep
|
||||
WaitForIdle();
|
||||
}
|
||||
|
||||
void ThreadManager::WaitForThreadsToRun() {
|
||||
size_t NumThreads{};
|
||||
{
|
||||
std::lock_guard<std::mutex> lk(ThreadCreationMutex);
|
||||
NumThreads = Threads.size();
|
||||
}
|
||||
|
||||
// Spin while waiting for the threads to start up
|
||||
std::unique_lock<std::mutex> lk(IdleWaitMutex);
|
||||
IdleWaitCV.wait(lk, [this, NumThreads] {
|
||||
return IdleWaitRefCount.load() >= NumThreads;
|
||||
});
|
||||
|
||||
Running = true;
|
||||
}
|
||||
|
||||
void ThreadManager::Step() {
|
||||
LogMan::Msg::AFmt("ThreadManager::Step currently not implemented");
|
||||
{
|
||||
std::lock_guard lk(ThreadCreationMutex);
|
||||
// Walk the threads and tell them to clear their caches
|
||||
// Useful when our block size is set to a large number and we need to step a single instruction
|
||||
for (auto &Thread : Threads) {
|
||||
CTX->ClearCodeCache(Thread);
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: Set to single step mode.
|
||||
Run();
|
||||
WaitForThreadsToRun();
|
||||
WaitForIdle();
|
||||
// TODO: Set back to full running mode.
|
||||
}
|
||||
|
||||
void ThreadManager::Stop(bool IgnoreCurrentThread) {
|
||||
pid_t tid = FHU::Syscalls::gettid();
|
||||
FEXCore::Core::InternalThreadState* CurrentThread{};
|
||||
|
||||
// Tell all the threads that they should stop
|
||||
{
|
||||
std::lock_guard<std::mutex> lk(ThreadCreationMutex);
|
||||
for (auto &Thread : Threads) {
|
||||
if (IgnoreCurrentThread &&
|
||||
Thread->ThreadManager.TID == tid) {
|
||||
// If we are callign stop from the current thread then we can ignore sending signals to this thread
|
||||
// This means that this thread is already gone
|
||||
}
|
||||
else if (Thread->ThreadManager.TID == tid) {
|
||||
// We need to save the current thread for last to ensure all threads receive their stop signals
|
||||
CurrentThread = Thread;
|
||||
continue;
|
||||
}
|
||||
|
||||
if (Thread->RunningEvents.Running.load()) {
|
||||
StopThread(Thread);
|
||||
}
|
||||
|
||||
// If the thread is waiting to start but immediately killed then there can be a hang
|
||||
// This occurs in the case of gdb attach with immediate kill
|
||||
if (Thread->RunningEvents.WaitingToStart.load()) {
|
||||
Thread->RunningEvents.EarlyExit = true;
|
||||
Thread->StartRunning.NotifyAll();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Stop the current thread now if we aren't ignoring it
|
||||
if (CurrentThread) {
|
||||
StopThread(CurrentThread);
|
||||
}
|
||||
}
|
||||
|
||||
void ThreadManager::SleepThread(FEXCore::Context::Context *CTX, FEXCore::Core::CpuStateFrame *Frame) {
|
||||
auto Thread = Frame->Thread;
|
||||
|
||||
--IdleWaitRefCount;
|
||||
IdleWaitCV.notify_all();
|
||||
|
||||
Thread->RunningEvents.ThreadSleeping = true;
|
||||
|
||||
// Go to sleep
|
||||
Thread->StartRunning.Wait();
|
||||
|
||||
Thread->RunningEvents.Running = true;
|
||||
++IdleWaitRefCount;
|
||||
Thread->RunningEvents.ThreadSleeping = false;
|
||||
|
||||
IdleWaitCV.notify_all();
|
||||
}
|
||||
|
||||
void ThreadManager::UnlockAfterFork(FEXCore::Core::InternalThreadState *LiveThread, bool Child) {
|
||||
CTX->UnlockAfterFork(LiveThread, Child);
|
||||
|
||||
if (!Child) return;
|
||||
|
||||
// This function is called after fork
|
||||
// We need to cleanup some of the thread data that is dead
|
||||
for (auto &DeadThread : Threads) {
|
||||
if (DeadThread == LiveThread) {
|
||||
continue;
|
||||
}
|
||||
|
||||
// Setting running to false ensures that when they are shutdown we won't send signals to kill them
|
||||
DeadThread->RunningEvents.Running = false;
|
||||
|
||||
// Despite what google searches may susgest, glibc actually has special code to handle forks
|
||||
// with multiple active threads.
|
||||
// It cleans up the stacks of dead threads and marks them as terminated.
|
||||
// It also cleans up a bunch of internal mutexes.
|
||||
|
||||
// FIXME: TLS is probally still alive. Investigate
|
||||
|
||||
// Deconstructing the Interneal thread state should clean up most of the state.
|
||||
// But if anything on the now deleted stack is holding a refrence to the heap, it will be leaked
|
||||
CTX->DestroyThread(DeadThread);
|
||||
|
||||
// FIXME: Make sure sure nothing gets leaked via the heap. Ideas:
|
||||
// * Make sure nothing is allocated on the heap without ref in InternalThreadState
|
||||
// * Surround any code that heap allocates with a per-thread mutex.
|
||||
// Before forking, the the forking thread can lock all thread mutexes.
|
||||
}
|
||||
|
||||
// Remove all threads but the live thread from Threads
|
||||
Threads.clear();
|
||||
Threads.push_back(LiveThread);
|
||||
|
||||
// Clean up dead stacks
|
||||
FEXCore::Threads::Thread::CleanupAfterFork();
|
||||
|
||||
// We now only have one thread.
|
||||
IdleWaitRefCount = 1;
|
||||
}
|
||||
|
||||
void ThreadManager::WaitForIdle() {
|
||||
std::unique_lock<std::mutex> lk(IdleWaitMutex);
|
||||
IdleWaitCV.wait(lk, [this] {
|
||||
return IdleWaitRefCount.load() == 0;
|
||||
});
|
||||
|
||||
Running = false;
|
||||
}
|
||||
|
||||
ThreadManager::~ThreadManager() {
|
||||
std::lock_guard lk(ThreadCreationMutex);
|
||||
|
||||
for (auto &Thread : Threads) {
|
||||
HandleThreadDeletion(Thread);
|
||||
}
|
||||
Threads.clear();
|
||||
}
|
||||
}
|
@ -304,7 +304,7 @@ int main(int argc, char **argv, char **const envp) {
|
||||
if (!CTX->InitCore()) {
|
||||
return 1;
|
||||
}
|
||||
auto ParentThread = CTX->CreateThread(Loader.DefaultRIP(), Loader.GetStackPointer(), FEXCore::Context::Context::ManagedBy::FRONTEND);
|
||||
auto ParentThread = CTX->CreateThread(Loader.DefaultRIP(), Loader.GetStackPointer());
|
||||
|
||||
if (!ParentThread) {
|
||||
return 1;
|
||||
|
@ -554,7 +554,7 @@ void BTCpuProcessInit() {
|
||||
}
|
||||
|
||||
NTSTATUS BTCpuThreadInit() {
|
||||
GetTLS().ThreadState() = CTX->CreateThread(0, 0, FEXCore::Context::Context::ManagedBy::FRONTEND);
|
||||
GetTLS().ThreadState() = CTX->CreateThread(0, 0);
|
||||
|
||||
std::scoped_lock Lock(ThreadSuspendLock);
|
||||
InitializedWOWThreads.emplace(GetCurrentThreadId());
|
||||
|
Loading…
x
Reference in New Issue
Block a user