aboutsummaryrefslogtreecommitdiff
path: root/Ryujinx.HLE/HOS/Kernel/Threading
diff options
context:
space:
mode:
authorgdkchan <gab.dark.100@gmail.com>2020-12-09 19:20:05 -0300
committerGitHub <noreply@github.com>2020-12-09 19:20:05 -0300
commit48278905d1470f89be31668c738397f569af156a (patch)
tree2e35b0695b33c8eb723f5948e3f6f040d84cfe76 /Ryujinx.HLE/HOS/Kernel/Threading
parent3484265d37732b32951709e5abfa52a260db349d (diff)
Rewrite scheduler context switch code (#1786)
* Rewrite scheduler context switch code * Fix race in UnmapIpcRestorePermission * Fix thread exit issue that could leave the scheduler in a invalid state * Change context switch method to not wait on guest thread, remove spin wait, use SignalAndWait to pass control * Remove multi-core setting (it is always on now) * Re-enable assert * Remove multicore from default config and schema * Fix race in KTimeManager
Diffstat (limited to 'Ryujinx.HLE/HOS/Kernel/Threading')
-rw-r--r--Ryujinx.HLE/HOS/Kernel/Threading/HleCoreManager.cs66
-rw-r--r--Ryujinx.HLE/HOS/Kernel/Threading/HleScheduler.cs150
-rw-r--r--Ryujinx.HLE/HOS/Kernel/Threading/KAddressArbiter.cs20
-rw-r--r--Ryujinx.HLE/HOS/Kernel/Threading/KConditionVariable.cs2
-rw-r--r--Ryujinx.HLE/HOS/Kernel/Threading/KCoreContext.cs79
-rw-r--r--Ryujinx.HLE/HOS/Kernel/Threading/KCriticalSection.cs70
-rw-r--r--Ryujinx.HLE/HOS/Kernel/Threading/KPriorityQueue.cs (renamed from Ryujinx.HLE/HOS/Kernel/Threading/KSchedulingData.cs)46
-rw-r--r--Ryujinx.HLE/HOS/Kernel/Threading/KScheduler.cs606
-rw-r--r--Ryujinx.HLE/HOS/Kernel/Threading/KSynchronization.cs2
-rw-r--r--Ryujinx.HLE/HOS/Kernel/Threading/KThread.cs413
-rw-r--r--Ryujinx.HLE/HOS/Kernel/Threading/KThreadContext.cs19
11 files changed, 694 insertions, 779 deletions
diff --git a/Ryujinx.HLE/HOS/Kernel/Threading/HleCoreManager.cs b/Ryujinx.HLE/HOS/Kernel/Threading/HleCoreManager.cs
deleted file mode 100644
index c2597990..00000000
--- a/Ryujinx.HLE/HOS/Kernel/Threading/HleCoreManager.cs
+++ /dev/null
@@ -1,66 +0,0 @@
-using System.Collections.Concurrent;
-using System.Threading;
-
-namespace Ryujinx.HLE.HOS.Kernel.Threading
-{
- class HleCoreManager
- {
- private class PausableThread
- {
- public ManualResetEvent Event { get; private set; }
-
- public bool IsExiting { get; set; }
-
- public PausableThread()
- {
- Event = new ManualResetEvent(false);
- }
- }
-
- private ConcurrentDictionary<Thread, PausableThread> _threads;
-
- public HleCoreManager()
- {
- _threads = new ConcurrentDictionary<Thread, PausableThread>();
- }
-
- public void Set(Thread thread)
- {
- GetThread(thread).Event.Set();
- }
-
- public void Reset(Thread thread)
- {
- GetThread(thread).Event.Reset();
- }
-
- public void Wait(Thread thread)
- {
- PausableThread pausableThread = GetThread(thread);
-
- if (!pausableThread.IsExiting)
- {
- pausableThread.Event.WaitOne();
- }
- }
-
- public void Exit(Thread thread)
- {
- GetThread(thread).IsExiting = true;
- }
-
- private PausableThread GetThread(Thread thread)
- {
- return _threads.GetOrAdd(thread, (key) => new PausableThread());
- }
-
- public void RemoveThread(Thread thread)
- {
- if (_threads.TryRemove(thread, out PausableThread pausableThread))
- {
- pausableThread.Event.Set();
- pausableThread.Event.Dispose();
- }
- }
- }
-} \ No newline at end of file
diff --git a/Ryujinx.HLE/HOS/Kernel/Threading/HleScheduler.cs b/Ryujinx.HLE/HOS/Kernel/Threading/HleScheduler.cs
deleted file mode 100644
index c4161d54..00000000
--- a/Ryujinx.HLE/HOS/Kernel/Threading/HleScheduler.cs
+++ /dev/null
@@ -1,150 +0,0 @@
-using System;
-using System.Threading;
-
-namespace Ryujinx.HLE.HOS.Kernel.Threading
-{
- partial class KScheduler
- {
- private const int RoundRobinTimeQuantumMs = 10;
-
- private int _currentCore;
-
- public bool MultiCoreScheduling { get; set; }
-
- public HleCoreManager CoreManager { get; private set; }
-
- private bool _keepPreempting;
-
- public void StartAutoPreemptionThread()
- {
- Thread preemptionThread = new Thread(PreemptCurrentThread)
- {
- Name = "HLE.PreemptionThread"
- };
-
- _keepPreempting = true;
-
- preemptionThread.Start();
- }
-
- public void ContextSwitch()
- {
- lock (CoreContexts)
- {
- if (MultiCoreScheduling)
- {
- int selectedCount = 0;
-
- for (int core = 0; core < CpuCoresCount; core++)
- {
- KCoreContext coreContext = CoreContexts[core];
-
- if (coreContext.ContextSwitchNeeded && (coreContext.CurrentThread?.IsCurrentHostThread() ?? false))
- {
- coreContext.ContextSwitch();
- }
-
- if (coreContext.CurrentThread?.IsCurrentHostThread() ?? false)
- {
- selectedCount++;
- }
- }
-
- if (selectedCount == 0)
- {
- CoreManager.Reset(Thread.CurrentThread);
- }
- else if (selectedCount == 1)
- {
- CoreManager.Set(Thread.CurrentThread);
- }
- else
- {
- throw new InvalidOperationException("Thread scheduled in more than one core!");
- }
- }
- else
- {
- KThread currentThread = CoreContexts[_currentCore].CurrentThread;
-
- bool hasThreadExecuting = currentThread != null;
-
- if (hasThreadExecuting)
- {
- // If this is not the thread that is currently executing, we need
- // to request an interrupt to allow safely starting another thread.
- if (!currentThread.IsCurrentHostThread())
- {
- currentThread.Context.RequestInterrupt();
-
- return;
- }
-
- CoreManager.Reset(currentThread.HostThread);
- }
-
- // Advance current core and try picking a thread,
- // keep advancing if it is null.
- for (int core = 0; core < 4; core++)
- {
- _currentCore = (_currentCore + 1) % CpuCoresCount;
-
- KCoreContext coreContext = CoreContexts[_currentCore];
-
- coreContext.UpdateCurrentThread();
-
- if (coreContext.CurrentThread != null)
- {
- CoreManager.Set(coreContext.CurrentThread.HostThread);
-
- coreContext.CurrentThread.Execute();
-
- break;
- }
- }
-
- // If nothing was running before, then we are on a "external"
- // HLE thread, we don't need to wait.
- if (!hasThreadExecuting)
- {
- return;
- }
- }
- }
-
- CoreManager.Wait(Thread.CurrentThread);
- }
-
- private void PreemptCurrentThread()
- {
- // Preempts current thread every 10 milliseconds on a round-robin fashion,
- // when multi core scheduling is disabled, to try ensuring that all threads
- // gets a chance to run.
- while (_keepPreempting)
- {
- lock (CoreContexts)
- {
- KThread currentThread = CoreContexts[_currentCore].CurrentThread;
-
- currentThread?.Context.RequestInterrupt();
- }
-
- PreemptThreads();
-
- Thread.Sleep(RoundRobinTimeQuantumMs);
- }
- }
-
- public void ExitThread(KThread thread)
- {
- thread.Context.StopRunning();
-
- CoreManager.Exit(thread.HostThread);
- }
-
- public void RemoveThread(KThread thread)
- {
- CoreManager.RemoveThread(thread.HostThread);
- }
- }
-} \ No newline at end of file
diff --git a/Ryujinx.HLE/HOS/Kernel/Threading/KAddressArbiter.cs b/Ryujinx.HLE/HOS/Kernel/Threading/KAddressArbiter.cs
index 2922ee1a..3ddcffc1 100644
--- a/Ryujinx.HLE/HOS/Kernel/Threading/KAddressArbiter.cs
+++ b/Ryujinx.HLE/HOS/Kernel/Threading/KAddressArbiter.cs
@@ -25,14 +25,14 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
public KernelResult ArbitrateLock(int ownerHandle, ulong mutexAddress, int requesterHandle)
{
- KThread currentThread = _context.Scheduler.GetCurrentThread();
+ KThread currentThread = KernelStatic.GetCurrentThread();
_context.CriticalSection.Enter();
currentThread.SignaledObj = null;
currentThread.ObjSyncResult = KernelResult.Success;
- KProcess currentProcess = _context.Scheduler.GetCurrentProcess();
+ KProcess currentProcess = KernelStatic.GetCurrentProcess();
if (!KernelTransfer.UserToKernelInt32(_context, mutexAddress, out int mutexValue))
{
@@ -81,7 +81,7 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
{
_context.CriticalSection.Enter();
- KThread currentThread = _context.Scheduler.GetCurrentThread();
+ KThread currentThread = KernelStatic.GetCurrentThread();
(KernelResult result, KThread newOwnerThread) = MutexUnlock(currentThread, mutexAddress);
@@ -104,7 +104,7 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
{
_context.CriticalSection.Enter();
- KThread currentThread = _context.Scheduler.GetCurrentThread();
+ KThread currentThread = KernelStatic.GetCurrentThread();
currentThread.SignaledObj = null;
currentThread.ObjSyncResult = KernelResult.TimedOut;
@@ -227,7 +227,7 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
{
ulong address = requester.MutexAddress;
- KProcess currentProcess = _context.Scheduler.GetCurrentProcess();
+ KProcess currentProcess = KernelStatic.GetCurrentProcess();
if (!currentProcess.CpuMemory.IsMapped(address))
{
@@ -293,7 +293,7 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
public KernelResult WaitForAddressIfEqual(ulong address, int value, long timeout)
{
- KThread currentThread = _context.Scheduler.GetCurrentThread();
+ KThread currentThread = KernelStatic.GetCurrentThread();
_context.CriticalSection.Enter();
@@ -368,7 +368,7 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
bool shouldDecrement,
long timeout)
{
- KThread currentThread = _context.Scheduler.GetCurrentThread();
+ KThread currentThread = KernelStatic.GetCurrentThread();
_context.CriticalSection.Enter();
@@ -383,7 +383,7 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
currentThread.SignaledObj = null;
currentThread.ObjSyncResult = KernelResult.TimedOut;
- KProcess currentProcess = _context.Scheduler.GetCurrentProcess();
+ KProcess currentProcess = KernelStatic.GetCurrentProcess();
if (!KernelTransfer.UserToKernelInt32(_context, address, out int currentValue))
{
@@ -483,7 +483,7 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
{
_context.CriticalSection.Enter();
- KProcess currentProcess = _context.Scheduler.GetCurrentProcess();
+ KProcess currentProcess = KernelStatic.GetCurrentProcess();
if (!currentProcess.CpuMemory.IsMapped(address))
{
@@ -544,7 +544,7 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
offset = 1;
}
- KProcess currentProcess = _context.Scheduler.GetCurrentProcess();
+ KProcess currentProcess = KernelStatic.GetCurrentProcess();
if (!currentProcess.CpuMemory.IsMapped(address))
{
diff --git a/Ryujinx.HLE/HOS/Kernel/Threading/KConditionVariable.cs b/Ryujinx.HLE/HOS/Kernel/Threading/KConditionVariable.cs
index dd7e67ae..d146bff0 100644
--- a/Ryujinx.HLE/HOS/Kernel/Threading/KConditionVariable.cs
+++ b/Ryujinx.HLE/HOS/Kernel/Threading/KConditionVariable.cs
@@ -7,7 +7,7 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
{
public static void Wait(KernelContext context, LinkedList<KThread> threadList, object mutex, long timeout)
{
- KThread currentThread = context.Scheduler.GetCurrentThread();
+ KThread currentThread = KernelStatic.GetCurrentThread();
context.CriticalSection.Enter();
diff --git a/Ryujinx.HLE/HOS/Kernel/Threading/KCoreContext.cs b/Ryujinx.HLE/HOS/Kernel/Threading/KCoreContext.cs
deleted file mode 100644
index 0aa12b0d..00000000
--- a/Ryujinx.HLE/HOS/Kernel/Threading/KCoreContext.cs
+++ /dev/null
@@ -1,79 +0,0 @@
-using Ryujinx.Common;
-
-namespace Ryujinx.HLE.HOS.Kernel.Threading
-{
- class KCoreContext
- {
- private KScheduler _scheduler;
-
- private HleCoreManager _coreManager;
-
- public bool ContextSwitchNeeded { get; private set; }
-
- public long LastContextSwitchTime { get; private set; }
-
- public long TotalIdleTimeTicks { get; private set; } //TODO
-
- public KThread CurrentThread { get; private set; }
- public KThread SelectedThread { get; private set; }
-
- public KCoreContext(KScheduler scheduler, HleCoreManager coreManager)
- {
- _scheduler = scheduler;
- _coreManager = coreManager;
- }
-
- public void SelectThread(KThread thread)
- {
- SelectedThread = thread;
-
- if (SelectedThread != CurrentThread)
- {
- ContextSwitchNeeded = true;
- }
- }
-
- public void UpdateCurrentThread()
- {
- ContextSwitchNeeded = false;
-
- LastContextSwitchTime = PerformanceCounter.ElapsedMilliseconds;
-
- CurrentThread = SelectedThread;
-
- if (CurrentThread != null)
- {
- long currentTime = PerformanceCounter.ElapsedMilliseconds;
-
- CurrentThread.TotalTimeRunning += currentTime - CurrentThread.LastScheduledTime;
- CurrentThread.LastScheduledTime = currentTime;
- }
- }
-
- public void ContextSwitch()
- {
- ContextSwitchNeeded = false;
-
- LastContextSwitchTime = PerformanceCounter.ElapsedMilliseconds;
-
- if (CurrentThread != null)
- {
- _coreManager.Reset(CurrentThread.HostThread);
- }
-
- CurrentThread = SelectedThread;
-
- if (CurrentThread != null)
- {
- long currentTime = PerformanceCounter.ElapsedMilliseconds;
-
- CurrentThread.TotalTimeRunning += currentTime - CurrentThread.LastScheduledTime;
- CurrentThread.LastScheduledTime = currentTime;
-
- _coreManager.Set(CurrentThread.HostThread);
-
- CurrentThread.Execute();
- }
- }
- }
-} \ No newline at end of file
diff --git a/Ryujinx.HLE/HOS/Kernel/Threading/KCriticalSection.cs b/Ryujinx.HLE/HOS/Kernel/Threading/KCriticalSection.cs
index b778c2a4..1d61f2f0 100644
--- a/Ryujinx.HLE/HOS/Kernel/Threading/KCriticalSection.cs
+++ b/Ryujinx.HLE/HOS/Kernel/Threading/KCriticalSection.cs
@@ -5,21 +5,20 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
class KCriticalSection
{
private readonly KernelContext _context;
-
- public object LockObj { get; private set; }
-
+ private readonly object _lock;
private int _recursionCount;
+ public object Lock => _lock;
+
public KCriticalSection(KernelContext context)
{
_context = context;
-
- LockObj = new object();
+ _lock = new object();
}
public void Enter()
{
- Monitor.Enter(LockObj);
+ Monitor.Enter(_lock);
_recursionCount++;
}
@@ -31,61 +30,34 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
return;
}
- bool doContextSwitch = false;
-
if (--_recursionCount == 0)
{
- if (_context.Scheduler.ThreadReselectionRequested)
- {
- _context.Scheduler.SelectThreads();
- }
+ ulong scheduledCoresMask = KScheduler.SelectThreads(_context);
- Monitor.Exit(LockObj);
+ Monitor.Exit(_lock);
- if (_context.Scheduler.MultiCoreScheduling)
+ KThread currentThread = KernelStatic.GetCurrentThread();
+ bool isCurrentThreadSchedulable = currentThread != null && currentThread.IsSchedulable;
+ if (isCurrentThreadSchedulable)
{
- lock (_context.Scheduler.CoreContexts)
- {
- for (int core = 0; core < KScheduler.CpuCoresCount; core++)
- {
- KCoreContext coreContext = _context.Scheduler.CoreContexts[core];
-
- if (coreContext.ContextSwitchNeeded)
- {
- KThread currentThread = coreContext.CurrentThread;
-
- if (currentThread == null)
- {
- // Nothing is running, we can perform the context switch immediately.
- coreContext.ContextSwitch();
- }
- else if (currentThread.IsCurrentHostThread())
- {
- // Thread running on the current core, context switch will block.
- doContextSwitch = true;
- }
- else
- {
- // Thread running on another core, request a interrupt.
- currentThread.Context.RequestInterrupt();
- }
- }
- }
- }
+ KScheduler.EnableScheduling(_context, scheduledCoresMask);
}
else
{
- doContextSwitch = true;
+ KScheduler.EnableSchedulingFromForeignThread(_context, scheduledCoresMask);
+
+ // If the thread exists but is not schedulable, we still want to suspend
+ // it if it's not runnable. That allows the kernel to still block HLE threads
+ // even if they are not scheduled on guest cores.
+ if (currentThread != null && !currentThread.IsSchedulable && currentThread.Context.Running)
+ {
+ currentThread.SchedulerWaitEvent.WaitOne();
+ }
}
}
else
{
- Monitor.Exit(LockObj);
- }
-
- if (doContextSwitch)
- {
- _context.Scheduler.ContextSwitch();
+ Monitor.Exit(_lock);
}
}
}
diff --git a/Ryujinx.HLE/HOS/Kernel/Threading/KSchedulingData.cs b/Ryujinx.HLE/HOS/Kernel/Threading/KPriorityQueue.cs
index 83c4a079..2c9d7574 100644
--- a/Ryujinx.HLE/HOS/Kernel/Threading/KSchedulingData.cs
+++ b/Ryujinx.HLE/HOS/Kernel/Threading/KPriorityQueue.cs
@@ -1,8 +1,9 @@
using System.Collections.Generic;
+using System.Numerics;
namespace Ryujinx.HLE.HOS.Kernel.Threading
{
- class KSchedulingData
+ class KPriorityQueue
{
private LinkedList<KThread>[][] _scheduledThreadsPerPrioPerCore;
private LinkedList<KThread>[][] _suggestedThreadsPerPrioPerCore;
@@ -10,7 +11,7 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
private long[] _scheduledPrioritiesPerCore;
private long[] _suggestedPrioritiesPerCore;
- public KSchedulingData()
+ public KPriorityQueue()
{
_suggestedThreadsPerPrioPerCore = new LinkedList<KThread>[KScheduler.PrioritiesCount][];
_scheduledThreadsPerPrioPerCore = new LinkedList<KThread>[KScheduler.PrioritiesCount][];
@@ -45,7 +46,7 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
{
long prioMask = prios[core];
- int prio = CountTrailingZeros(prioMask);
+ int prio = BitOperations.TrailingZeroCount(prioMask);
prioMask &= ~(1L << prio);
@@ -62,42 +63,22 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
node = node.Next;
}
- prio = CountTrailingZeros(prioMask);
+ prio = BitOperations.TrailingZeroCount(prioMask);
prioMask &= ~(1L << prio);
}
}
- private int CountTrailingZeros(long value)
- {
- int count = 0;
-
- while (((value >> count) & 0xf) == 0 && count < 64)
- {
- count += 4;
- }
-
- while (((value >> count) & 1) == 0 && count < 64)
- {
- count++;
- }
-
- return count;
- }
-
public void TransferToCore(int prio, int dstCore, KThread thread)
{
- bool schedulable = thread.DynamicPriority < KScheduler.PrioritiesCount;
-
- int srcCore = thread.CurrentCore;
-
- thread.CurrentCore = dstCore;
-
- if (srcCore == dstCore || !schedulable)
+ int srcCore = thread.ActiveCore;
+ if (srcCore == dstCore)
{
return;
}
+ thread.ActiveCore = dstCore;
+
if (srcCore >= 0)
{
Unschedule(prio, srcCore, thread);
@@ -168,13 +149,20 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
_scheduledPrioritiesPerCore[core] |= 1L << prio;
}
- public void Reschedule(int prio, int core, KThread thread)
+ public KThread Reschedule(int prio, int core, KThread thread)
{
+ if (prio >= KScheduler.PrioritiesCount)
+ {
+ return null;
+ }
+
LinkedList<KThread> queue = ScheduledQueue(prio, core);
queue.Remove(thread.SiblingsPerCore[core]);
thread.SiblingsPerCore[core] = queue.AddLast(thread);
+
+ return queue.First.Value;
}
public void Unschedule(int prio, int core, KThread thread)
diff --git a/Ryujinx.HLE/HOS/Kernel/Threading/KScheduler.cs b/Ryujinx.HLE/HOS/Kernel/Threading/KScheduler.cs
index c6da361d..e427f24d 100644
--- a/Ryujinx.HLE/HOS/Kernel/Threading/KScheduler.cs
+++ b/Ryujinx.HLE/HOS/Kernel/Threading/KScheduler.cs
@@ -1,7 +1,10 @@
+using Ryujinx.Common;
using Ryujinx.HLE.HOS.Kernel.Process;
using System;
using System.Collections.Generic;
using System.Linq;
+using System.Numerics;
+using System.Threading;
namespace Ryujinx.HLE.HOS.Kernel.Threading
{
@@ -10,130 +13,88 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
public const int PrioritiesCount = 64;
public const int CpuCoresCount = 4;
- private const int PreemptionPriorityCores012 = 59;
- private const int PreemptionPriorityCore3 = 63;
+ private const int RoundRobinTimeQuantumMs = 10;
+
+ private static readonly int[] PreemptionPriorities = new int[] { 59, 59, 59, 63 };
private readonly KernelContext _context;
+ private readonly int _coreId;
+
+ private struct SchedulingState
+ {
+ public bool NeedsScheduling;
+ public KThread SelectedThread;
+ }
+
+ private SchedulingState _state;
- public KSchedulingData SchedulingData { get; private set; }
+ private AutoResetEvent _idleInterruptEvent;
+ private readonly object _idleInterruptEventLock;
- public KCoreContext[] CoreContexts { get; private set; }
+ private KThread _previousThread;
+ private KThread _currentThread;
+ private readonly KThread _idleThread;
- public bool ThreadReselectionRequested { get; set; }
+ public KThread PreviousThread => _previousThread;
+ public long LastContextSwitchTime { get; private set; }
+ public long TotalIdleTimeTicks => _idleThread.TotalTimeRunning;
- public KScheduler(KernelContext context)
+ public KScheduler(KernelContext context, int coreId)
{
_context = context;
+ _coreId = coreId;
- SchedulingData = new KSchedulingData();
+ _idleInterruptEvent = new AutoResetEvent(false);
+ _idleInterruptEventLock = new object();
- CoreManager = new HleCoreManager();
+ KThread idleThread = CreateIdleThread(context, coreId);
- CoreContexts = new KCoreContext[CpuCoresCount];
+ _currentThread = idleThread;
+ _idleThread = idleThread;
- for (int core = 0; core < CpuCoresCount; core++)
- {
- CoreContexts[core] = new KCoreContext(this, CoreManager);
- }
+ idleThread.StartHostThread();
+ idleThread.SchedulerWaitEvent.Set();
}
- private void PreemptThreads()
+ private KThread CreateIdleThread(KernelContext context, int cpuCore)
{
- _context.CriticalSection.Enter();
+ KThread idleThread = new KThread(context);
- PreemptThread(PreemptionPriorityCores012, 0);
- PreemptThread(PreemptionPriorityCores012, 1);
- PreemptThread(PreemptionPriorityCores012, 2);
- PreemptThread(PreemptionPriorityCore3, 3);
+ idleThread.Initialize(0UL, 0UL, 0UL, PrioritiesCount, cpuCore, null, ThreadType.Dummy, IdleThreadLoop);
- _context.CriticalSection.Leave();
+ return idleThread;
}
- private void PreemptThread(int prio, int core)
+ public static ulong SelectThreads(KernelContext context)
{
- IEnumerable<KThread> scheduledThreads = SchedulingData.ScheduledThreads(core);
-
- KThread selectedThread = scheduledThreads.FirstOrDefault(x => x.DynamicPriority == prio);
-
- // Yield priority queue.
- if (selectedThread != null)
- {
- SchedulingData.Reschedule(prio, core, selectedThread);
- }
-
- IEnumerable<KThread> SuitableCandidates()
- {
- foreach (KThread thread in SchedulingData.SuggestedThreads(core))
- {
- int srcCore = thread.CurrentCore;
-
- if (srcCore >= 0)
- {
- KThread highestPrioSrcCore = SchedulingData.ScheduledThreads(srcCore).FirstOrDefault();
-
- if (highestPrioSrcCore != null && highestPrioSrcCore.DynamicPriority < 2)
- {
- break;
- }
-
- if (highestPrioSrcCore == thread)
- {
- continue;
- }
- }
-
- // If the candidate was scheduled after the current thread, then it's not worth it.
- if (selectedThread == null || selectedThread.LastScheduledTime >= thread.LastScheduledTime)
- {
- yield return thread;
- }
- }
- }
-
- // Select candidate threads that could run on this core.
- // Only take into account threads that are not yet selected.
- KThread dst = SuitableCandidates().FirstOrDefault(x => x.DynamicPriority == prio);
-
- if (dst != null)
+ if (context.ThreadReselectionRequested)
{
- SchedulingData.TransferToCore(prio, core, dst);
-
- selectedThread = dst;
+ return SelectThreadsImpl(context);
}
-
- // If the priority of the currently selected thread is lower than preemption priority,
- // then allow threads with lower priorities to be selected aswell.
- if (selectedThread != null && selectedThread.DynamicPriority > prio)
+ else
{
- Func<KThread, bool> predicate = x => x.DynamicPriority >= selectedThread.DynamicPriority;
-
- dst = SuitableCandidates().FirstOrDefault(predicate);
-
- if (dst != null)
- {
- SchedulingData.TransferToCore(dst.DynamicPriority, core, dst);
- }
+ return 0UL;
}
-
- ThreadReselectionRequested = true;
}
- public void SelectThreads()
+ private static ulong SelectThreadsImpl(KernelContext context)
{
- ThreadReselectionRequested = false;
+ context.ThreadReselectionRequested = false;
+
+ ulong scheduledCoresMask = 0UL;
for (int core = 0; core < CpuCoresCount; core++)
{
- KThread thread = SchedulingData.ScheduledThreads(core).FirstOrDefault();
+ KThread thread = context.PriorityQueue.ScheduledThreads(core).FirstOrDefault();
- CoreContexts[core].SelectThread(thread);
+ scheduledCoresMask |= context.Schedulers[core].SelectThread(thread);
}
for (int core = 0; core < CpuCoresCount; core++)
{
// If the core is not idle (there's already a thread running on it),
// then we don't need to attempt load balancing.
- if (SchedulingData.ScheduledThreads(core).Any())
+ if (context.PriorityQueue.ScheduledThreads(core).Any())
{
continue;
}
@@ -146,16 +107,15 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
// Select candidate threads that could run on this core.
// Give preference to threads that are not yet selected.
- foreach (KThread thread in SchedulingData.SuggestedThreads(core))
+ foreach (KThread suggested in context.PriorityQueue.SuggestedThreads(core))
{
- if (thread.CurrentCore < 0 || thread != CoreContexts[thread.CurrentCore].SelectedThread)
+ if (suggested.ActiveCore < 0 || suggested != context.Schedulers[suggested.ActiveCore]._state.SelectedThread)
{
- dst = thread;
-
+ dst = suggested;
break;
}
- srcCoresHighestPrioThreads[srcCoresHighestPrioThreadsCount++] = thread.CurrentCore;
+ srcCoresHighestPrioThreads[srcCoresHighestPrioThreadsCount++] = suggested.ActiveCore;
}
// Not yet selected candidate found.
@@ -165,9 +125,9 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
// threads, we should skip load balancing entirely.
if (dst.DynamicPriority >= 2)
{
- SchedulingData.TransferToCore(dst.DynamicPriority, core, dst);
+ context.PriorityQueue.TransferToCore(dst.DynamicPriority, core, dst);
- CoreContexts[core].SelectThread(dst);
+ scheduledCoresMask |= context.Schedulers[core].SelectThread(dst);
}
continue;
@@ -179,80 +139,480 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
{
int srcCore = srcCoresHighestPrioThreads[index];
- KThread src = SchedulingData.ScheduledThreads(srcCore).ElementAtOrDefault(1);
+ KThread src = context.PriorityQueue.ScheduledThreads(srcCore).ElementAtOrDefault(1);
if (src != null)
{
// Run the second thread on the queue on the source core,
// move the first one to the current core.
- KThread origSelectedCoreSrc = CoreContexts[srcCore].SelectedThread;
+ KThread origSelectedCoreSrc = context.Schedulers[srcCore]._state.SelectedThread;
- CoreContexts[srcCore].SelectThread(src);
+ scheduledCoresMask |= context.Schedulers[srcCore].SelectThread(src);
- SchedulingData.TransferToCore(origSelectedCoreSrc.DynamicPriority, core, origSelectedCoreSrc);
+ context.PriorityQueue.TransferToCore(origSelectedCoreSrc.DynamicPriority, core, origSelectedCoreSrc);
- CoreContexts[core].SelectThread(origSelectedCoreSrc);
+ scheduledCoresMask |= context.Schedulers[core].SelectThread(origSelectedCoreSrc);
}
}
}
+
+ return scheduledCoresMask;
+ }
+
+ private ulong SelectThread(KThread nextThread)
+ {
+ KThread previousThread = _state.SelectedThread;
+
+ if (previousThread != nextThread)
+ {
+ if (previousThread != null)
+ {
+ previousThread.LastScheduledTime = PerformanceCounter.ElapsedTicks;
+ }
+
+ _state.SelectedThread = nextThread;
+ _state.NeedsScheduling = true;
+ return 1UL << _coreId;
+ }
+ else
+ {
+ return 0UL;
+ }
+ }
+
+ public static void EnableScheduling(KernelContext context, ulong scheduledCoresMask)
+ {
+ KScheduler currentScheduler = context.Schedulers[KernelStatic.GetCurrentThread().CurrentCore];
+
+ // Note that "RescheduleCurrentCore" will block, so "RescheduleOtherCores" must be done first.
+ currentScheduler.RescheduleOtherCores(scheduledCoresMask);
+ currentScheduler.RescheduleCurrentCore();
+ }
+
+ public static void EnableSchedulingFromForeignThread(KernelContext context, ulong scheduledCoresMask)
+ {
+ RescheduleOtherCores(context, scheduledCoresMask);
+ }
+
+ private void RescheduleCurrentCore()
+ {
+ if (_state.NeedsScheduling)
+ {
+ Schedule();
+ }
+ }
+
+ private void RescheduleOtherCores(ulong scheduledCoresMask)
+ {
+ RescheduleOtherCores(_context, scheduledCoresMask & ~(1UL << _coreId));
+ }
+
+ private static void RescheduleOtherCores(KernelContext context, ulong scheduledCoresMask)
+ {
+ while (scheduledCoresMask != 0)
+ {
+ int coreToSignal = BitOperations.TrailingZeroCount(scheduledCoresMask);
+
+ KThread threadToSignal = context.Schedulers[coreToSignal]._currentThread;
+
+ // Request the thread running on that core to stop and reschedule, if we have one.
+ if (threadToSignal != context.Schedulers[coreToSignal]._idleThread)
+ {
+ threadToSignal.Context.RequestInterrupt();
+ }
+
+ // If the core is idle, ensure that the idle thread is awaken.
+ context.Schedulers[coreToSignal]._idleInterruptEvent.Set();
+
+ scheduledCoresMask &= ~(1UL << coreToSignal);
+ }
+ }
+
+ private void IdleThreadLoop()
+ {
+ while (_context.Running)
+ {
+ _state.NeedsScheduling = false;
+ Thread.MemoryBarrier();
+ KThread nextThread = PickNextThread(_state.SelectedThread);
+
+ if (_idleThread != nextThread)
+ {
+ _idleThread.SchedulerWaitEvent.Reset();
+ WaitHandle.SignalAndWait(nextThread.SchedulerWaitEvent, _idleThread.SchedulerWaitEvent);
+ }
+
+ _idleInterruptEvent.WaitOne();
+ }
+
+ lock (_idleInterruptEventLock)
+ {
+ _idleInterruptEvent.Dispose();
+ _idleInterruptEvent = null;
+ }
}
- public KThread GetCurrentThread()
+ public void Schedule()
{
- return GetCurrentThreadOrNull() ?? GetDummyThread();
+ _state.NeedsScheduling = false;
+ Thread.MemoryBarrier();
+ KThread currentThread = KernelStatic.GetCurrentThread();
+ KThread selectedThread = _state.SelectedThread;
+
+ // If the thread is already scheduled and running on the core, we have nothing to do.
+ if (currentThread == selectedThread)
+ {
+ return;
+ }
+
+ currentThread.SchedulerWaitEvent.Reset();
+ currentThread.ThreadContext.Unlock();
+
+ // Wake all the threads that might be waiting until this thread context is unlocked.
+ for (int core = 0; core < CpuCoresCount; core++)
+ {
+ _context.Schedulers[core]._idleInterruptEvent.Set();
+ }
+
+ KThread nextThread = PickNextThread(selectedThread);
+
+ if (currentThread.Context.Running)
+ {
+ // Wait until this thread is scheduled again, and allow the next thread to run.
+ WaitHandle.SignalAndWait(nextThread.SchedulerWaitEvent, currentThread.SchedulerWaitEvent);
+ }
+ else
+ {
+ // Allow the next thread to run.
+ nextThread.SchedulerWaitEvent.Set();
+
+ // We don't need to wait since the thread is exiting, however we need to
+ // make sure this thread will never call the scheduler again, since it is
+ // no longer assigned to a core.
+ currentThread.MakeUnschedulable();
+
+ // Just to be sure, set the core to a invalid value.
+ // This will trigger a exception if it attempts to call schedule again,
+ // rather than leaving the scheduler in a invalid state.
+ currentThread.CurrentCore = -1;
+ }
}
- public KThread GetCurrentThreadOrNull()
+ private KThread PickNextThread(KThread selectedThread)
{
- lock (CoreContexts)
+ while (true)
{
- for (int core = 0; core < CpuCoresCount; core++)
+ if (selectedThread != null)
{
- if (CoreContexts[core].CurrentThread?.IsCurrentHostThread() ?? false)
+ // Try to run the selected thread.
+ // We need to acquire the context lock to be sure the thread is not
+ // already running on another core. If it is, then we return here
+ // and the caller should try again once there is something available for scheduling.
+ // The thread currently running on the core should have been requested to
+ // interrupt so this is not expected to take long.
+ // The idle thread must also be paused if we are scheduling a thread
+ // on the core, as the scheduled thread will handle the next switch.
+ if (selectedThread.ThreadContext.Lock())
{
- return CoreContexts[core].CurrentThread;
+ SwitchTo(selectedThread);
+
+ if (!_state.NeedsScheduling)
+ {
+ return selectedThread;
+ }
+
+ selectedThread.ThreadContext.Unlock();
}
+ else
+ {
+ return _idleThread;
+ }
+ }
+ else
+ {
+ // The core is idle now, make sure that the idle thread can run
+ // and switch the core when a thread is available.
+ SwitchTo(null);
+ return _idleThread;
}
+
+ _state.NeedsScheduling = false;
+ Thread.MemoryBarrier();
+ selectedThread = _state.SelectedThread;
+ }
+ }
+
+ private void SwitchTo(KThread nextThread)
+ {
+ KProcess currentProcess = KernelStatic.GetCurrentProcess();
+ KThread currentThread = KernelStatic.GetCurrentThread();
+
+ nextThread ??= _idleThread;
+
+ if (currentThread == nextThread)
+ {
+ return;
}
- return null;
+ long previousTicks = LastContextSwitchTime;
+ long currentTicks = PerformanceCounter.ElapsedTicks;
+ long ticksDelta = currentTicks - previousTicks;
+
+ currentThread.AddCpuTime(ticksDelta);
+
+ if (currentProcess != null)
+ {
+ currentProcess.AddCpuTime(ticksDelta);
+ }
+
+ LastContextSwitchTime = currentTicks;
+
+ if (currentProcess != null)
+ {
+ _previousThread = !currentThread.TerminationRequested && currentThread.ActiveCore == _coreId ? currentThread : null;
+ }
+ else if (currentThread == _idleThread)
+ {
+ _previousThread = null;
+ }
+
+ if (nextThread.CurrentCore != _coreId)
+ {
+ nextThread.CurrentCore = _coreId;
+ }
+
+ _currentThread = nextThread;
}
- private KThread _dummyThread;
+ public static void PreemptionThreadLoop(KernelContext context)
+ {
+ while (context.Running)
+ {
+ context.CriticalSection.Enter();
- private KThread GetDummyThread()
+ for (int core = 0; core < CpuCoresCount; core++)
+ {
+ RotateScheduledQueue(context, core, PreemptionPriorities[core]);
+ }
+
+ context.CriticalSection.Leave();
+
+ Thread.Sleep(RoundRobinTimeQuantumMs);
+ }
+ }
+
+ private static void RotateScheduledQueue(KernelContext context, int core, int prio)
{
- if (_dummyThread != null)
+ IEnumerable<KThread> scheduledThreads = context.PriorityQueue.ScheduledThreads(core);
+
+ KThread selectedThread = scheduledThreads.FirstOrDefault(x => x.DynamicPriority == prio);
+ KThread nextThread = null;
+
+ // Yield priority queue.
+ if (selectedThread != null)
+ {
+ nextThread = context.PriorityQueue.Reschedule(prio, core, selectedThread);
+ }
+
+ IEnumerable<KThread> SuitableCandidates()
{
- return _dummyThread;
+ foreach (KThread suggested in context.PriorityQueue.SuggestedThreads(core))
+ {
+ int suggestedCore = suggested.ActiveCore;
+ if (suggestedCore >= 0)
+ {
+ KThread selectedSuggestedCore = context.PriorityQueue.ScheduledThreads(suggestedCore).FirstOrDefault();
+
+ if (selectedSuggestedCore == suggested || (selectedSuggestedCore != null && selectedSuggestedCore.DynamicPriority < 2))
+ {
+ continue;
+ }
+ }
+
+ // If the candidate was scheduled after the current thread, then it's not worth it.
+ if (nextThread == selectedThread ||
+ nextThread == null ||
+ nextThread.LastScheduledTime >= suggested.LastScheduledTime)
+ {
+ yield return suggested;
+ }
+ }
}
- KProcess dummyProcess = new KProcess(_context);
+ // Select candidate threads that could run on this core.
+ // Only take into account threads that are not yet selected.
+ KThread dst = SuitableCandidates().FirstOrDefault(x => x.DynamicPriority == prio);
- dummyProcess.HandleTable.Initialize(1024);
+ if (dst != null)
+ {
+ context.PriorityQueue.TransferToCore(prio, core, dst);
+ }
+
+ // If the priority of the currently selected thread is lower or same as the preemption priority,
+ // then try to migrate a thread with lower priority.
+ KThread bestCandidate = context.PriorityQueue.ScheduledThreads(core).FirstOrDefault();
+
+ if (bestCandidate != null && bestCandidate.DynamicPriority >= prio)
+ {
+ dst = SuitableCandidates().FirstOrDefault(x => x.DynamicPriority < bestCandidate.DynamicPriority);
+
+ if (dst != null)
+ {
+ context.PriorityQueue.TransferToCore(dst.DynamicPriority, core, dst);
+ }
+ }
+
+ context.ThreadReselectionRequested = true;
+ }
+
+ public static void Yield(KernelContext context)
+ {
+ KThread currentThread = KernelStatic.GetCurrentThread();
+
+ context.CriticalSection.Enter();
+
+ if (currentThread.SchedFlags != ThreadSchedState.Running)
+ {
+ context.CriticalSection.Leave();
+ return;
+ }
- KThread dummyThread = new KThread(_context);
+ KThread nextThread = context.PriorityQueue.Reschedule(currentThread.DynamicPriority, currentThread.ActiveCore, currentThread);
- dummyThread.Initialize(0, 0, 0, 44, 0, dummyProcess, ThreadType.Dummy);
+ if (nextThread != currentThread)
+ {
+ context.ThreadReselectionRequested = true;
+ }
- return _dummyThread = dummyThread;
+ context.CriticalSection.Leave();
}
- public KProcess GetCurrentProcess()
+ public static void YieldWithLoadBalancing(KernelContext context)
{
- return GetCurrentThread().Owner;
+ KThread currentThread = KernelStatic.GetCurrentThread();
+
+ context.CriticalSection.Enter();
+
+ if (currentThread.SchedFlags != ThreadSchedState.Running)
+ {
+ context.CriticalSection.Leave();
+ return;
+ }
+
+ int prio = currentThread.DynamicPriority;
+ int core = currentThread.ActiveCore;
+
+ // Move current thread to the end of the queue.
+ KThread nextThread = context.PriorityQueue.Reschedule(prio, core, currentThread);
+
+ IEnumerable<KThread> SuitableCandidates()
+ {
+ foreach (KThread suggested in context.PriorityQueue.SuggestedThreads(core))
+ {
+ int suggestedCore = suggested.ActiveCore;
+ if (suggestedCore >= 0)
+ {
+ KThread selectedSuggestedCore = context.Schedulers[suggestedCore]._state.SelectedThread;
+
+ if (selectedSuggestedCore == suggested || (selectedSuggestedCore != null && selectedSuggestedCore.DynamicPriority < 2))
+ {
+ continue;
+ }
+ }
+
+ // If the candidate was scheduled after the current thread, then it's not worth it,
+ // unless the priority is higher than the current one.
+ if (suggested.LastScheduledTime <= nextThread.LastScheduledTime ||
+ suggested.DynamicPriority < nextThread.DynamicPriority)
+ {
+ yield return suggested;
+ }
+ }
+ }
+
+ KThread dst = SuitableCandidates().FirstOrDefault(x => x.DynamicPriority <= prio);
+
+ if (dst != null)
+ {
+ context.PriorityQueue.TransferToCore(dst.DynamicPriority, core, dst);
+
+ context.ThreadReselectionRequested = true;
+ }
+ else if (currentThread != nextThread)
+ {
+ context.ThreadReselectionRequested = true;
+ }
+
+ context.CriticalSection.Leave();
}
- public void Dispose()
+ public static void YieldToAnyThread(KernelContext context)
{
- Dispose(true);
+ KThread currentThread = KernelStatic.GetCurrentThread();
+
+ context.CriticalSection.Enter();
+
+ if (currentThread.SchedFlags != ThreadSchedState.Running)
+ {
+ context.CriticalSection.Leave();
+ return;
+ }
+
+ int core = currentThread.ActiveCore;
+
+ context.PriorityQueue.TransferToCore(currentThread.DynamicPriority, -1, currentThread);
+
+ if (!context.PriorityQueue.ScheduledThreads(core).Any())
+ {
+ KThread selectedThread = null;
+
+ foreach (KThread suggested in context.PriorityQueue.SuggestedThreads(core))
+ {
+ int suggestedCore = suggested.ActiveCore;
+
+ if (suggestedCore < 0)
+ {
+ continue;
+ }
+
+ KThread firstCandidate = context.PriorityQueue.ScheduledThreads(suggestedCore).FirstOrDefault();
+
+ if (firstCandidate == suggested)
+ {
+ continue;
+ }
+
+ if (firstCandidate == null || firstCandidate.DynamicPriority >= 2)
+ {
+ context.PriorityQueue.TransferToCore(suggested.DynamicPriority, core, suggested);
+ }
+
+ selectedThread = suggested;
+ break;
+ }
+
+ if (currentThread != selectedThread)
+ {
+ context.ThreadReselectionRequested = true;
+ }
+ }
+ else
+ {
+ context.ThreadReselectionRequested = true;
+ }
+
+ context.CriticalSection.Leave();
}
- protected virtual void Dispose(bool disposing)
+ public void Dispose()
{
- if (disposing)
+ // Ensure that the idle thread is not blocked and can exit.
+ lock (_idleInterruptEventLock)
{
- _keepPreempting = false;
+ if (_idleInterruptEvent != null)
+ {
+ _idleInterruptEvent.Set();
+ }
}
}
}
diff --git a/Ryujinx.HLE/HOS/Kernel/Threading/KSynchronization.cs b/Ryujinx.HLE/HOS/Kernel/Threading/KSynchronization.cs
index 22610b22..419f1536 100644
--- a/Ryujinx.HLE/HOS/Kernel/Threading/KSynchronization.cs
+++ b/Ryujinx.HLE/HOS/Kernel/Threading/KSynchronization.cs
@@ -43,7 +43,7 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
return result;
}
- KThread currentThread = _context.Scheduler.GetCurrentThread();
+ KThread currentThread = KernelStatic.GetCurrentThread();
if (currentThread.ShallBeTerminated ||
currentThread.SchedFlags == ThreadSchedState.TerminationPending)
diff --git a/Ryujinx.HLE/HOS/Kernel/Threading/KThread.cs b/Ryujinx.HLE/HOS/Kernel/Threading/KThread.cs
index f523cb9c..b95b1e8e 100644
--- a/Ryujinx.HLE/HOS/Kernel/Threading/KThread.cs
+++ b/Ryujinx.HLE/HOS/Kernel/Threading/KThread.cs
@@ -4,8 +4,7 @@ using Ryujinx.HLE.HOS.Kernel.Common;
using Ryujinx.HLE.HOS.Kernel.Process;
using System;
using System.Collections.Generic;
-using System.Linq;
-using System.Text;
+using System.Numerics;
using System.Threading;
namespace Ryujinx.HLE.HOS.Kernel.Threading
@@ -14,17 +13,24 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
{
public const int MaxWaitSyncObjects = 64;
- private int _hostThreadRunning;
+ private ManualResetEvent _schedulerWaitEvent;
+
+ public ManualResetEvent SchedulerWaitEvent => _schedulerWaitEvent;
public Thread HostThread { get; private set; }
public ARMeilleure.State.ExecutionContext Context { get; private set; }
+ public KThreadContext ThreadContext { get; private set; }
+
+ public int DynamicPriority { get; set; }
public long AffinityMask { get; set; }
public long ThreadUid { get; private set; }
- public long TotalTimeRunning { get; set; }
+ private long _totalTimeRunning;
+
+ public long TotalTimeRunning => _totalTimeRunning;
public KSynchronizationObject SignaledObj { get; set; }
@@ -32,6 +38,9 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
private ulong _entrypoint;
private ThreadStart _customThreadStart;
+ private bool _forcedUnschedulable;
+
+ public bool IsSchedulable => _customThreadStart == null && !_forcedUnschedulable;
public ulong MutexAddress { get; set; }
@@ -65,11 +74,12 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
public KernelResult ObjSyncResult { get; set; }
- public int DynamicPriority { get; set; }
- public int CurrentCore { get; set; }
public int BasePriority { get; set; }
public int PreferredCore { get; set; }
+ public int CurrentCore { get; set; }
+ public int ActiveCore { get; set; }
+
private long _affinityMaskOverride;
private int _preferredCoreOverride;
#pragma warning disable CS0649
@@ -86,26 +96,21 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
set => _shallBeTerminated = value ? 1 : 0;
}
+ public bool TerminationRequested => ShallBeTerminated || SchedFlags == ThreadSchedState.TerminationPending;
+
public bool SyncCancelled { get; set; }
public bool WaitingSync { get; set; }
- private bool _hasExited;
+ private int _hasExited;
private bool _hasBeenInitialized;
private bool _hasBeenReleased;
public bool WaitingInArbitration { get; set; }
- private KScheduler _scheduler;
-
- private KSchedulingData _schedulingData;
-
public long LastPc { get; set; }
public KThread(KernelContext context) : base(context)
{
- _scheduler = KernelContext.Scheduler;
- _schedulingData = KernelContext.Scheduler.SchedulingData;
-
WaitSyncObjects = new KSynchronizationObject[MaxWaitSyncObjects];
WaitSyncHandles = new int[MaxWaitSyncObjects];
@@ -119,7 +124,7 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
ulong argsPtr,
ulong stackTop,
int priority,
- int defaultCpuCore,
+ int cpuCore,
KProcess owner,
ThreadType type,
ThreadStart customThreadStart = null)
@@ -129,20 +134,20 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
throw new ArgumentException($"Invalid thread type \"{type}\".");
}
- PreferredCore = defaultCpuCore;
+ ThreadContext = new KThreadContext();
- AffinityMask |= 1L << defaultCpuCore;
+ PreferredCore = cpuCore;
+ AffinityMask |= 1L << cpuCore;
SchedFlags = type == ThreadType.Dummy
? ThreadSchedState.Running
: ThreadSchedState.None;
- CurrentCore = PreferredCore;
-
+ ActiveCore = cpuCore;
+ ObjSyncResult = KernelResult.ThreadNotStarted;
DynamicPriority = priority;
BasePriority = priority;
-
- ObjSyncResult = KernelResult.ThreadNotStarted;
+ CurrentCore = cpuCore;
_entrypoint = entrypoint;
_customThreadStart = customThreadStart;
@@ -179,41 +184,38 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
Context = CpuContext.CreateExecutionContext();
- bool isAarch32 = !Owner.Flags.HasFlag(ProcessCreationFlags.Is64Bit);
-
- Context.IsAarch32 = isAarch32;
+ Context.IsAarch32 = !is64Bits;
Context.SetX(0, argsPtr);
- if (isAarch32)
+ if (is64Bits)
{
- Context.SetX(13, (uint)stackTop);
+ Context.SetX(31, stackTop);
}
else
{
- Context.SetX(31, stackTop);
+ Context.SetX(13, (uint)stackTop);
}
Context.CntfrqEl0 = 19200000;
Context.Tpidr = (long)_tlsAddress;
- owner.SubscribeThreadEventHandlers(Context);
-
ThreadUid = KernelContext.NewThreadUid();
- HostThread.Name = $"HLE.HostThread.{ThreadUid}";
+ HostThread.Name = customThreadStart != null ? $"HLE.OsThread.{ThreadUid}" : $"HLE.GuestThread.{ThreadUid}";
_hasBeenInitialized = true;
if (owner != null)
{
+ owner.SubscribeThreadEventHandlers(Context);
owner.AddThread(this);
if (owner.IsPaused)
{
KernelContext.CriticalSection.Enter();
- if (ShallBeTerminated || SchedFlags == ThreadSchedState.TerminationPending)
+ if (TerminationRequested)
{
KernelContext.CriticalSection.Leave();
@@ -237,7 +239,7 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
{
KernelContext.CriticalSection.Enter();
- if (!ShallBeTerminated && SchedFlags != ThreadSchedState.TerminationPending)
+ if (!TerminationRequested)
{
_forcePauseFlags |= ThreadSchedState.KernelInitPauseFlag;
@@ -253,20 +255,17 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
if (!ShallBeTerminated)
{
- KThread currentThread = KernelContext.Scheduler.GetCurrentThread();
+ KThread currentThread = KernelStatic.GetCurrentThread();
- while (SchedFlags != ThreadSchedState.TerminationPending &&
- currentThread.SchedFlags != ThreadSchedState.TerminationPending &&
- !currentThread.ShallBeTerminated)
+ while (SchedFlags != ThreadSchedState.TerminationPending && (currentThread == null || !currentThread.TerminationRequested))
{
if ((SchedFlags & ThreadSchedState.LowMask) != ThreadSchedState.None)
{
result = KernelResult.InvalidState;
-
break;
}
- if (currentThread._forcePauseFlags == ThreadSchedState.None)
+ if (currentThread == null || currentThread._forcePauseFlags == ThreadSchedState.None)
{
if (Owner != null && _forcePauseFlags != ThreadSchedState.None)
{
@@ -275,8 +274,9 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
SetNewSchedFlags(ThreadSchedState.Running);
- result = KernelResult.Success;
+ StartHostThread();
+ result = KernelResult.Success;
break;
}
else
@@ -299,28 +299,6 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
return result;
}
- public void Exit()
- {
- // TODO: Debug event.
-
- if (Owner != null)
- {
- Owner.ResourceLimit?.Release(LimitableResource.Thread, 0, 1);
-
- _hasBeenReleased = true;
- }
-
- KernelContext.CriticalSection.Enter();
-
- _forcePauseFlags &= ~ThreadSchedState.ForcePauseMask;
-
- ExitImpl();
-
- KernelContext.CriticalSection.Leave();
-
- DecrementReferenceCount();
- }
-
public ThreadSchedState PrepareForTermination()
{
KernelContext.CriticalSection.Enter();
@@ -387,9 +365,8 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
do
{
- if (ShallBeTerminated || SchedFlags == ThreadSchedState.TerminationPending)
+ if (TerminationRequested)
{
- KernelContext.Scheduler.ExitThread(this);
Exit();
// As the death of the thread is handled by the CPU emulator, we differ from the official kernel and return here.
@@ -398,7 +375,7 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
KernelContext.CriticalSection.Enter();
- if (ShallBeTerminated || SchedFlags == ThreadSchedState.TerminationPending)
+ if (TerminationRequested)
{
state = ThreadSchedState.TerminationPending;
}
@@ -416,200 +393,74 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
} while (state == ThreadSchedState.TerminationPending);
}
- private void ExitImpl()
+ public void Exit()
{
- KernelContext.CriticalSection.Enter();
-
- SetNewSchedFlags(ThreadSchedState.TerminationPending);
-
- _hasExited = true;
+ // TODO: Debug event.
- Signal();
+ if (Owner != null)
+ {
+ Owner.ResourceLimit?.Release(LimitableResource.Thread, 0, 1);
- KernelContext.CriticalSection.Leave();
- }
+ _hasBeenReleased = true;
+ }
- public KernelResult Sleep(long timeout)
- {
KernelContext.CriticalSection.Enter();
- if (ShallBeTerminated || SchedFlags == ThreadSchedState.TerminationPending)
- {
- KernelContext.CriticalSection.Leave();
+ _forcePauseFlags &= ~ThreadSchedState.ForcePauseMask;
- return KernelResult.ThreadTerminating;
- }
+ bool decRef = ExitImpl();
- SetNewSchedFlags(ThreadSchedState.Paused);
-
- if (timeout > 0)
- {
- KernelContext.TimeManager.ScheduleFutureInvocation(this, timeout);
- }
+ Context.StopRunning();
KernelContext.CriticalSection.Leave();
- if (timeout > 0)
+ if (decRef)
{
- KernelContext.TimeManager.UnscheduleFutureInvocation(this);
+ DecrementReferenceCount();
}
-
- return 0;
}
- public void Yield()
+ private bool ExitImpl()
{
KernelContext.CriticalSection.Enter();
- if (SchedFlags != ThreadSchedState.Running)
- {
- KernelContext.CriticalSection.Leave();
-
- KernelContext.Scheduler.ContextSwitch();
-
- return;
- }
+ SetNewSchedFlags(ThreadSchedState.TerminationPending);
- if (DynamicPriority < KScheduler.PrioritiesCount)
- {
- // Move current thread to the end of the queue.
- _schedulingData.Reschedule(DynamicPriority, CurrentCore, this);
- }
+ bool decRef = Interlocked.Exchange(ref _hasExited, 1) == 0;
- _scheduler.ThreadReselectionRequested = true;
+ Signal();
KernelContext.CriticalSection.Leave();
- KernelContext.Scheduler.ContextSwitch();
+ return decRef;
}
- public void YieldWithLoadBalancing()
+ public KernelResult Sleep(long timeout)
{
KernelContext.CriticalSection.Enter();
- if (SchedFlags != ThreadSchedState.Running)
+ if (ShallBeTerminated || SchedFlags == ThreadSchedState.TerminationPending)
{
KernelContext.CriticalSection.Leave();
- KernelContext.Scheduler.ContextSwitch();
-
- return;
- }
-
- int prio = DynamicPriority;
- int core = CurrentCore;
-
- KThread nextThreadOnCurrentQueue = null;
-
- if (DynamicPriority < KScheduler.PrioritiesCount)
- {
- // Move current thread to the end of the queue.
- _schedulingData.Reschedule(prio, core, this);
-
- Func<KThread, bool> predicate = x => x.DynamicPriority == prio;
-
- nextThreadOnCurrentQueue = _schedulingData.ScheduledThreads(core).FirstOrDefault(predicate);
- }
-
- IEnumerable<KThread> SuitableCandidates()
- {
- foreach (KThread thread in _schedulingData.SuggestedThreads(core))
- {
- int srcCore = thread.CurrentCore;
-
- if (srcCore >= 0)
- {
- KThread selectedSrcCore = _scheduler.CoreContexts[srcCore].SelectedThread;
-
- if (selectedSrcCore == thread || ((selectedSrcCore?.DynamicPriority ?? 2) < 2))
- {
- continue;
- }
- }
-
- // If the candidate was scheduled after the current thread, then it's not worth it,
- // unless the priority is higher than the current one.
- if (nextThreadOnCurrentQueue.LastScheduledTime >= thread.LastScheduledTime ||
- nextThreadOnCurrentQueue.DynamicPriority < thread.DynamicPriority)
- {
- yield return thread;
- }
- }
+ return KernelResult.ThreadTerminating;
}
- KThread dst = SuitableCandidates().FirstOrDefault(x => x.DynamicPriority <= prio);
-
- if (dst != null)
- {
- _schedulingData.TransferToCore(dst.DynamicPriority, core, dst);
-
- _scheduler.ThreadReselectionRequested = true;
- }
+ SetNewSchedFlags(ThreadSchedState.Paused);
- if (this != nextThreadOnCurrentQueue)
+ if (timeout > 0)
{
- _scheduler.ThreadReselectionRequested = true;
+ KernelContext.TimeManager.ScheduleFutureInvocation(this, timeout);
}
KernelContext.CriticalSection.Leave();
- KernelContext.Scheduler.ContextSwitch();
- }
-
- public void YieldAndWaitForLoadBalancing()
- {
- KernelContext.CriticalSection.Enter();
-
- if (SchedFlags != ThreadSchedState.Running)
- {
- KernelContext.CriticalSection.Leave();
-
- KernelContext.Scheduler.ContextSwitch();
-
- return;
- }
-
- int core = CurrentCore;
-
- _schedulingData.TransferToCore(DynamicPriority, -1, this);
-
- KThread selectedThread = null;
-
- if (!_schedulingData.ScheduledThreads(core).Any())
- {
- foreach (KThread thread in _schedulingData.SuggestedThreads(core))
- {
- if (thread.CurrentCore < 0)
- {
- continue;
- }
-
- KThread firstCandidate = _schedulingData.ScheduledThreads(thread.CurrentCore).FirstOrDefault();
-
- if (firstCandidate == thread)
- {
- continue;
- }
-
- if (firstCandidate == null || firstCandidate.DynamicPriority >= 2)
- {
- _schedulingData.TransferToCore(thread.DynamicPriority, core, thread);
-
- selectedThread = thread;
- }
-
- break;
- }
- }
-
- if (selectedThread != this)
+ if (timeout > 0)
{
- _scheduler.ThreadReselectionRequested = true;
+ KernelContext.TimeManager.UnscheduleFutureInvocation(this);
}
- KernelContext.CriticalSection.Leave();
-
- KernelContext.Scheduler.ContextSwitch();
+ return 0;
}
public void SetPriority(int priority)
@@ -751,17 +602,17 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
if (oldAffinityMask != newAffinityMask)
{
- int oldCore = CurrentCore;
+ int oldCore = ActiveCore;
- if (CurrentCore >= 0 && ((AffinityMask >> CurrentCore) & 1) == 0)
+ if (oldCore >= 0 && ((AffinityMask >> oldCore) & 1) == 0)
{
if (PreferredCore < 0)
{
- CurrentCore = HighestSetCore(AffinityMask);
+ ActiveCore = sizeof(ulong) * 8 - 1 - BitOperations.LeadingZeroCount((ulong)AffinityMask);
}
else
{
- CurrentCore = PreferredCore;
+ ActiveCore = PreferredCore;
}
}
@@ -774,19 +625,6 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
return KernelResult.Success;
}
- private static int HighestSetCore(long mask)
- {
- for (int core = KScheduler.CpuCoresCount - 1; core >= 0; core--)
- {
- if (((mask >> core) & 1) != 0)
- {
- return core;
- }
- }
-
- return -1;
- }
-
private void CombineForcePauseFlags()
{
ThreadSchedState oldFlags = SchedFlags;
@@ -995,92 +833,112 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
return;
}
+ if (!IsSchedulable)
+ {
+ // Ensure our thread is running and we have an event.
+ StartHostThread();
+
+ // If the thread is not schedulable, we want to just run or pause
+ // it directly as we don't care about priority or the core it is
+ // running on in this case.
+ if (SchedFlags == ThreadSchedState.Running)
+ {
+ _schedulerWaitEvent.Set();
+ }
+ else
+ {
+ _schedulerWaitEvent.Reset();
+ }
+
+ return;
+ }
+
if (oldFlags == ThreadSchedState.Running)
{
// Was running, now it's stopped.
- if (CurrentCore >= 0)
+ if (ActiveCore >= 0)
{
- _schedulingData.Unschedule(DynamicPriority, CurrentCore, this);
+ KernelContext.PriorityQueue.Unschedule(DynamicPriority, ActiveCore, this);
}
for (int core = 0; core < KScheduler.CpuCoresCount; core++)
{
- if (core != CurrentCore && ((AffinityMask >> core) & 1) != 0)
+ if (core != ActiveCore && ((AffinityMask >> core) & 1) != 0)
{
- _schedulingData.Unsuggest(DynamicPriority, core, this);
+ KernelContext.PriorityQueue.Unsuggest(DynamicPriority, core, this);
}
}
}
else if (SchedFlags == ThreadSchedState.Running)
{
// Was stopped, now it's running.
- if (CurrentCore >= 0)
+ if (ActiveCore >= 0)
{
- _schedulingData.Schedule(DynamicPriority, CurrentCore, this);
+ KernelContext.PriorityQueue.Schedule(DynamicPriority, ActiveCore, this);
}
for (int core = 0; core < KScheduler.CpuCoresCount; core++)
{
- if (core != CurrentCore && ((AffinityMask >> core) & 1) != 0)
+ if (core != ActiveCore && ((AffinityMask >> core) & 1) != 0)
{
- _schedulingData.Suggest(DynamicPriority, core, this);
+ KernelContext.PriorityQueue.Suggest(DynamicPriority, core, this);
}
}
}
- _scheduler.ThreadReselectionRequested = true;
+ KernelContext.ThreadReselectionRequested = true;
}
private void AdjustSchedulingForNewPriority(int oldPriority)
{
- if (SchedFlags != ThreadSchedState.Running)
+ if (SchedFlags != ThreadSchedState.Running || !IsSchedulable)
{
return;
}
// Remove thread from the old priority queues.
- if (CurrentCore >= 0)
+ if (ActiveCore >= 0)
{
- _schedulingData.Unschedule(oldPriority, CurrentCore, this);
+ KernelContext.PriorityQueue.Unschedule(oldPriority, ActiveCore, this);
}
for (int core = 0; core < KScheduler.CpuCoresCount; core++)
{
- if (core != CurrentCore && ((AffinityMask >> core) & 1) != 0)
+ if (core != ActiveCore && ((AffinityMask >> core) & 1) != 0)
{
- _schedulingData.Unsuggest(oldPriority, core, this);
+ KernelContext.PriorityQueue.Unsuggest(oldPriority, core, this);
}
}
// Add thread to the new priority queues.
- KThread currentThread = _scheduler.GetCurrentThread();
+ KThread currentThread = KernelStatic.GetCurrentThread();
- if (CurrentCore >= 0)
+ if (ActiveCore >= 0)
{
if (currentThread == this)
{
- _schedulingData.SchedulePrepend(DynamicPriority, CurrentCore, this);
+ KernelContext.PriorityQueue.SchedulePrepend(DynamicPriority, ActiveCore, this);
}
else
{
- _schedulingData.Schedule(DynamicPriority, CurrentCore, this);
+ KernelContext.PriorityQueue.Schedule(DynamicPriority, ActiveCore, this);
}
}
for (int core = 0; core < KScheduler.CpuCoresCount; core++)
{
- if (core != CurrentCore && ((AffinityMask >> core) & 1) != 0)
+ if (core != ActiveCore && ((AffinityMask >> core) & 1) != 0)
{
- _schedulingData.Suggest(DynamicPriority, core, this);
+ KernelContext.PriorityQueue.Suggest(DynamicPriority, core, this);
}
}
- _scheduler.ThreadReselectionRequested = true;
+ KernelContext.ThreadReselectionRequested = true;
}
private void AdjustSchedulingForNewAffinity(long oldAffinityMask, int oldCore)
{
- if (SchedFlags != ThreadSchedState.Running || DynamicPriority >= KScheduler.PrioritiesCount)
+ if (SchedFlags != ThreadSchedState.Running || DynamicPriority >= KScheduler.PrioritiesCount || !IsSchedulable)
{
return;
}
@@ -1092,11 +950,11 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
{
if (core == oldCore)
{
- _schedulingData.Unschedule(DynamicPriority, core, this);
+ KernelContext.PriorityQueue.Unschedule(DynamicPriority, core, this);
}
else
{
- _schedulingData.Unsuggest(DynamicPriority, core, this);
+ KernelContext.PriorityQueue.Unsuggest(DynamicPriority, core, this);
}
}
}
@@ -1106,18 +964,18 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
{
if (((AffinityMask >> core) & 1) != 0)
{
- if (core == CurrentCore)
+ if (core == ActiveCore)
{
- _schedulingData.Schedule(DynamicPriority, core, this);
+ KernelContext.PriorityQueue.Schedule(DynamicPriority, core, this);
}
else
{
- _schedulingData.Suggest(DynamicPriority, core, this);
+ KernelContext.PriorityQueue.Suggest(DynamicPriority, core, this);
}
}
}
- _scheduler.ThreadReselectionRequested = true;
+ KernelContext.ThreadReselectionRequested = true;
}
public void SetEntryArguments(long argsPtr, int threadHandle)
@@ -1141,17 +999,32 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
Logger.Info?.Print(LogClass.Cpu, $"Guest stack trace:\n{GetGuestStackTrace()}\n");
}
- public void Execute()
+ public void AddCpuTime(long ticks)
+ {
+ Interlocked.Add(ref _totalTimeRunning, ticks);
+ }
+
+ public void StartHostThread()
{
- if (Interlocked.CompareExchange(ref _hostThreadRunning, 1, 0) == 0)
+ if (_schedulerWaitEvent == null)
{
- HostThread.Start();
+ var schedulerWaitEvent = new ManualResetEvent(false);
+
+ if (Interlocked.Exchange(ref _schedulerWaitEvent, schedulerWaitEvent) == null)
+ {
+ HostThread.Start();
+ }
+ else
+ {
+ schedulerWaitEvent.Dispose();
+ }
}
}
private void ThreadStart()
{
- KernelStatic.SetKernelContext(KernelContext);
+ _schedulerWaitEvent.WaitOne();
+ KernelStatic.SetKernelContext(KernelContext, this);
if (_customThreadStart != null)
{
@@ -1162,20 +1035,18 @@ namespace Ryujinx.HLE.HOS.Kernel.Threading
Owner.Context.Execute(Context, _entrypoint);
}
- KernelContext.Scheduler.ExitThread(this);
- KernelContext.Scheduler.RemoveThread(this);
-
Context.Dispose();
+ _schedulerWaitEvent.Dispose();
}
- public bool IsCurrentHostThread()
+ public void MakeUnschedulable()
{
- return Thread.CurrentThread == HostThread;
+ _forcedUnschedulable = true;
}
public override bool IsSignaled()
{
- return _hasExited;
+ return _hasExited != 0;
}
protected override void Destroy()
diff --git a/Ryujinx.HLE/HOS/Kernel/Threading/KThreadContext.cs b/Ryujinx.HLE/HOS/Kernel/Threading/KThreadContext.cs
new file mode 100644
index 00000000..a7e9c4b3
--- /dev/null
+++ b/Ryujinx.HLE/HOS/Kernel/Threading/KThreadContext.cs
@@ -0,0 +1,19 @@
+using System.Threading;
+
+namespace Ryujinx.HLE.HOS.Kernel.Threading
+{
+ class KThreadContext
+ {
+ private int _locked;
+
+ public bool Lock()
+ {
+ return Interlocked.Exchange(ref _locked, 1) == 0;
+ }
+
+ public void Unlock()
+ {
+ Interlocked.Exchange(ref _locked, 0);
+ }
+ }
+}