Kernel: Various context switch fixes

These changes solve a number of problems with the software
context swithcing:

* The scheduler lock really should be held throughout context switches
* Transitioning from the initial (idle) thread to another needs to
  hold the scheduler lock
* Transitioning from a dying thread to another also needs to hold
  the scheduler lock
* Dying threads cannot necessarily be finalized if they haven't
  switched out of it yet, so flag them as active while a processor
  is running it (the Running state may be switched to Dying while
  it still is actually running)
This commit is contained in:
Tom 2020-07-05 14:32:07 -06:00 committed by Andreas Kling
parent 49f5069b76
commit 2a82a25fec
9 changed files with 235 additions and 89 deletions

View file

@ -1035,10 +1035,10 @@ extern "C" void enter_thread_context(Thread* from_thread, Thread* to_thread)
#define ENTER_THREAD_CONTEXT_ARGS_SIZE (2 * 4) // to_thread, from_thread
void Processor::switch_context(Thread* from_thread, Thread* to_thread)
void Processor::switch_context(Thread*& from_thread, Thread*& to_thread)
{
ASSERT(!in_irq());
ASSERT(!m_in_critical);
ASSERT(m_in_critical == 1);
ASSERT(is_kernel_mode());
#ifdef CONTEXT_SWITCH_DEBUG
dbg() << "switch_context --> switching out of: " << VirtualAddress(from_thread) << " " << *from_thread;
@ -1096,13 +1096,23 @@ extern "C" void context_first_init(Thread* from_thread, Thread* to_thread, TrapF
(void)to_thread;
(void)trap;
ASSERT(to_thread == Thread::current());
#ifdef CONTEXT_SWITCH_DEBUG
dbg() << "switch_context <-- from " << VirtualAddress(from_thread) << " " << *from_thread << " to " << VirtualAddress(to_thread) << " " << *to_thread << " (context_first_init)";
#endif
ASSERT(to_thread == Thread::current());
Scheduler::enter_current(*from_thread);
if (to_thread->process().wait_for_tracer_at_next_execve()) {
to_thread->send_urgent_signal_to_self(SIGSTOP);
}
// Since we got here and don't have Scheduler::context_switch in the
// call stack (because this is the first time we switched into this
// context), we need to unlock the scheduler lock manually. We're
// using the flags initially set up by init_context
g_scheduler_lock.unlock(trap->regs->eflags);
}
extern "C" void thread_context_first_enter(void);
@ -1124,10 +1134,12 @@ asm(
u32 Processor::init_context(Thread& thread, bool leave_crit)
{
ASSERT(is_kernel_mode());
ASSERT(g_scheduler_lock.is_locked());
if (leave_crit) {
ASSERT(in_critical());
m_in_critical--; // leave it without triggering anything
ASSERT(!in_critical());
// Leave the critical section we set up in in Process::exec,
// but because we still have the scheduler lock we should end up with 1
m_in_critical--; // leave it without triggering anything or restoring flags
ASSERT(in_critical() == 1);
}
const u32 kernel_stack_top = thread.kernel_stack_top();
@ -1213,7 +1225,6 @@ u32 Processor::init_context(Thread& thread, bool leave_crit)
extern "C" u32 do_init_context(Thread* thread, u32 flags)
{
ASSERT_INTERRUPTS_DISABLED();
ASSERT(Processor::current().in_critical());
thread->tss().eflags = flags;
return Processor::current().init_context(*thread, true);
}
@ -1246,6 +1257,9 @@ void Processor::assume_context(Thread& thread, u32 flags)
dbg() << "Assume context for thread " << VirtualAddress(&thread) << " " << thread;
#endif
ASSERT_INTERRUPTS_DISABLED();
// in_critical() should be 2 here. The critical section in Process::exec
// and then the scheduler lock
ASSERT(Processor::current().in_critical() == 2);
do_assume_context(&thread, flags);
ASSERT_NOT_REACHED();
}

View file

@ -751,6 +751,30 @@ public:
}
if (prev_flags & 0x200)
sti();
else
cli();
}
ALWAYS_INLINE u32 clear_critical(u32& prev_flags, bool enable_interrupts)
{
u32 prev_crit = m_in_critical;
m_in_critical = 0;
prev_flags = cpu_flags();
if (!m_in_irq)
check_invoke_scheduler();
if (enable_interrupts)
sti();
return prev_crit;
}
ALWAYS_INLINE void restore_critical(u32 prev_crit, u32 prev_flags)
{
ASSERT(m_in_critical == 0);
m_in_critical = prev_crit;
if (prev_flags & 0x200)
sti();
else
cli();
}
ALWAYS_INLINE u32& in_critical() { return m_in_critical; }
@ -772,7 +796,7 @@ public:
void exit_trap(TrapFrame& trap);
[[noreturn]] void initialize_context_switching(Thread& initial_thread);
void switch_context(Thread* from_thread, Thread* to_thread);
void switch_context(Thread*& from_thread, Thread*& to_thread);
[[noreturn]] static void assume_context(Thread& thread, u32 flags);
u32 init_context(Thread& thread, bool leave_crit);
static bool get_context_frame_ptr(Thread& thread, u32& frame_ptr, u32& eip);

View file

@ -73,8 +73,13 @@ void Lock::lock(Mode mode)
// switch will happen, so yield.
// The assumption is that if we call this from a critical section
// that we DO want to temporarily leave it
TemporaryChange change(Processor::current().in_critical(), 0u);
u32 prev_flags;
u32 prev_crit = Processor::current().clear_critical(prev_flags, !Processor::current().in_irq());
Scheduler::yield();
// Note, we may now be on a different CPU!
Processor::current().restore_critical(prev_crit, prev_flags);
}
}
}
@ -105,8 +110,13 @@ void Lock::unlock()
// I don't know *who* is using "m_lock", so just yield.
// The assumption is that if we call this from a critical section
// that we DO want to temporarily leave it
TemporaryChange change(Processor::current().in_critical(), 0u);
u32 prev_flags;
u32 prev_crit = Processor::current().clear_critical(prev_flags, false);
Scheduler::yield();
// Note, we may now be on a different CPU!
Processor::current().restore_critical(prev_crit, prev_flags);
}
}

View file

@ -1046,7 +1046,7 @@ int Process::do_exec(NonnullRefPtr<FileDescription> main_program_description, Ve
// and we don't want to deal with faults after this point.
u32 new_userspace_esp = new_main_thread->make_userspace_stack_for_main_thread(move(arguments), move(environment));
// We cli() manually here because we don't want to get interrupted between do_exec()
// We enter a critical section here because we don't want to get interrupted between do_exec()
// and Processor::assume_context() or the next context switch.
// If we used an InterruptDisabler that sti()'d on exit, we might timer tick'd too soon in exec().
Processor::current().enter_critical(prev_flags);
@ -1268,6 +1268,12 @@ int Process::exec(String path, Vector<String> arguments, Vector<String> environm
auto current_thread = Thread::current();
if (current_thread == new_main_thread) {
// We need to enter the scheduler lock before changing the state
// and it will be released after the context switch into that
// thread. We should also still be in our critical section
ASSERT(!g_scheduler_lock.is_locked());
ASSERT(Processor::current().in_critical() == 1);
g_scheduler_lock.lock();
current_thread->set_state(Thread::State::Running);
Processor::assume_context(*current_thread, prev_flags);
ASSERT_NOT_REACHED();

View file

@ -319,6 +319,11 @@ void Thread::consider_unblock(time_t now_sec, long now_usec)
void Scheduler::start()
{
ASSERT_INTERRUPTS_DISABLED();
// We need to acquire our scheduler lock, which will be released
// by the idle thread once control transferred there
g_scheduler_lock.lock();
auto& processor = Processor::current();
ASSERT(processor.is_initialized());
auto& idle_thread = *processor.idle_thread();
@ -402,7 +407,10 @@ bool Scheduler::pick_next()
#ifdef SCHEDULER_RUNNABLE_DEBUG
dbg() << "Non-runnables:";
Scheduler::for_each_nonrunnable([](Thread& thread) -> IterationDecision {
dbg() << " " << String::format("%-12s", thread.state_string()) << " " << thread << " @ " << String::format("%w", thread.tss().cs) << ":" << String::format("%x", thread.tss().eip) << " Reason: " << (thread.wait_reason() ? thread.wait_reason() : "none");
if (thread.state() == Thread::Queued)
dbg() << " " << String::format("%-12s", thread.state_string()) << " " << thread << " @ " << String::format("%w", thread.tss().cs) << ":" << String::format("%x", thread.tss().eip) << " Reason: " << (thread.wait_reason() ? thread.wait_reason() : "none");
else if (thread.state() == Thread::Dying)
dbg() << " " << String::format("%-12s", thread.state_string()) << " " << thread << " @ " << String::format("%w", thread.tss().cs) << ":" << String::format("%x", thread.tss().eip) << " Finalizable: " << thread.is_finalizable();
return IterationDecision::Continue;
});
@ -447,8 +455,7 @@ bool Scheduler::pick_next()
dbg() << "Scheduler[" << Processor::current().id() << "]: Switch to " << *thread_to_schedule << " @ " << String::format("%04x:%08x", thread_to_schedule->tss().cs, thread_to_schedule->tss().eip);
#endif
lock.unlock();
return context_switch(*thread_to_schedule);
return context_switch(thread_to_schedule);
}
bool Scheduler::yield()
@ -476,7 +483,7 @@ bool Scheduler::yield()
bool Scheduler::donate_to(Thread* beneficiary, const char* reason)
{
InterruptDisabler disabler;
ScopedSpinLock lock(g_scheduler_lock);
auto& proc = Processor::current();
ASSERT(!proc.in_irq());
if (!Thread::is_thread(beneficiary))
@ -497,41 +504,66 @@ bool Scheduler::donate_to(Thread* beneficiary, const char* reason)
dbg() << "Scheduler[" << proc.id() << "]: Donating " << ticks_to_donate << " ticks to " << *beneficiary << ", reason=" << reason;
#endif
beneficiary->set_ticks_left(ticks_to_donate);
Scheduler::context_switch(*beneficiary);
Scheduler::context_switch(beneficiary);
return false;
}
bool Scheduler::context_switch(Thread& thread)
bool Scheduler::context_switch(Thread* thread)
{
thread.set_ticks_left(time_slice_for(thread));
thread.did_schedule();
thread->set_ticks_left(time_slice_for(*thread));
thread->did_schedule();
auto current_thread = Thread::current();
if (current_thread == &thread)
auto from_thread = Thread::current();
if (from_thread == thread)
return false;
if (current_thread) {
if (from_thread) {
// If the last process hasn't blocked (still marked as running),
// mark it as runnable for the next round.
if (current_thread->state() == Thread::Running)
current_thread->set_state(Thread::Runnable);
if (from_thread->state() == Thread::Running)
from_thread->set_state(Thread::Runnable);
#ifdef LOG_EVERY_CONTEXT_SWITCH
dbg() << "Scheduler[" << Processor::current().id() << "]: " << *current_thread << " -> " << thread << " [" << thread.priority() << "] " << String::format("%w", thread.tss().cs) << ":" << String::format("%x", thread.tss().eip);
dbg() << "Scheduler[" << Processor::current().id() << "]: " << *from_thread << " -> " << *thread << " [" << thread->priority() << "] " << String::format("%w", thread->tss().cs) << ":" << String::format("%x", thread->tss().eip);
#endif
}
auto& proc = Processor::current();
if (!thread.is_initialized()) {
proc.init_context(thread, false);
thread.set_initialized(true);
if (!thread->is_initialized()) {
proc.init_context(*thread, false);
thread->set_initialized(true);
}
thread.set_state(Thread::Running);
thread->set_state(Thread::Running);
// Mark it as active because we are using this thread. This is similar
// to comparing it with Processor::current_thread, but when there are
// multiple processors there's no easy way to check whether the thread
// is actually still needed. This prevents accidental finalization when
// a thread is no longer in Running state, but running on another core.
thread->set_active(true);
proc.switch_context(from_thread, thread);
// NOTE: from_thread at this point reflects the thread we were
// switched from, and thread reflects Thread::current()
enter_current(*from_thread);
ASSERT(thread == Thread::current());
proc.switch_context(current_thread, &thread);
return true;
}
void Scheduler::enter_current(Thread& prev_thread)
{
ASSERT(g_scheduler_lock.is_locked());
prev_thread.set_active(false);
if (prev_thread.state() == Thread::Dying) {
// If the thread we switched from is marked as dying, then notify
// the finalizer. Note that as soon as we leave the scheduler lock
// the finalizer may free from_thread!
notify_finalizer();
}
}
Process* Scheduler::colonel()
{
return s_colonel_process;
@ -622,6 +654,12 @@ void Scheduler::invoke_async()
pick_next();
}
void Scheduler::notify_finalizer()
{
if (g_finalizer_has_work.exchange(true, AK::MemoryOrder::memory_order_acq_rel) == false)
g_finalizer_wait_queue->wake_all();
}
void Scheduler::idle_loop()
{
dbg() << "Scheduler[" << Processor::current().id() << "]: idle loop running";

View file

@ -58,11 +58,13 @@ public:
static timeval time_since_boot();
static bool yield();
static bool donate_to(Thread*, const char* reason);
static bool context_switch(Thread&);
static bool context_switch(Thread*);
static void enter_current(Thread& prev_thread);
static Process* colonel();
static void beep();
static void idle_loop();
static void invoke_async();
static void notify_finalizer();
template<typename Callback>
static inline IterationDecision for_each_runnable(Callback);

View file

@ -42,6 +42,8 @@ public:
SpinLock() = default;
SpinLock(const SpinLock&) = delete;
SpinLock(SpinLock&&) = delete;
SpinLock& operator=(const SpinLock&) = delete;
SpinLock& operator=(SpinLock&&) = delete;
ALWAYS_INLINE u32 lock()
{
@ -82,6 +84,8 @@ public:
RecursiveSpinLock() = default;
RecursiveSpinLock(const RecursiveSpinLock&) = delete;
RecursiveSpinLock(RecursiveSpinLock&&) = delete;
RecursiveSpinLock& operator=(const RecursiveSpinLock&) = delete;
RecursiveSpinLock& operator=(RecursiveSpinLock&&) = delete;
ALWAYS_INLINE u32 lock()
{
@ -128,6 +132,9 @@ class ScopedSpinLock
public:
ScopedSpinLock() = delete;
ScopedSpinLock(const ScopedSpinLock&) = delete;
ScopedSpinLock& operator=(const ScopedSpinLock&) = delete;
ScopedSpinLock& operator=(ScopedSpinLock&&) = delete;
ScopedSpinLock(LockType& lock):
m_lock(&lock)
@ -147,8 +154,6 @@ public:
from.m_have_lock = false;
}
ScopedSpinLock(const ScopedSpinLock&) = delete;
~ScopedSpinLock()
{
if (m_lock && m_have_lock) {

View file

@ -153,7 +153,7 @@ void Thread::set_should_die()
#endif
return;
}
InterruptDisabler disabler;
ScopedCritical critical;
// Remember that we should die instead of returning to
// the userspace.
@ -181,36 +181,43 @@ void Thread::die_if_needed()
if (!m_should_die)
return;
u32 prev_crit;
unlock_process_if_locked(prev_crit);
unlock_process_if_locked();
InterruptDisabler disabler;
ScopedCritical critical;
set_state(Thread::State::Dying);
// Flag a context switch. Because we're in a critical section,
// Scheduler::yield will actually only mark a pending scontext switch
// Simply leaving the critical section would not necessarily trigger
// a switch.
Scheduler::yield();
// Now leave the critical section so that we can also trigger the
// actual context switch
u32 prev_flags;
Processor::current().clear_critical(prev_flags, false);
// We should never get here, but the scoped scheduler lock
// will be released by Scheduler::context_switch again
ASSERT_NOT_REACHED();
}
void Thread::yield_without_holding_big_lock()
{
u32 prev_crit;
bool did_unlock = unlock_process_if_locked(prev_crit);
bool did_unlock = unlock_process_if_locked();
Scheduler::yield();
relock_process(did_unlock, prev_crit);
relock_process(did_unlock);
}
bool Thread::unlock_process_if_locked(u32& prev_crit)
bool Thread::unlock_process_if_locked()
{
auto& in_critical = Processor::current().in_critical();
prev_crit = in_critical;
in_critical = 0;
return process().big_lock().force_unlock_if_locked();
}
void Thread::relock_process(bool did_unlock, u32 prev_crit)
void Thread::relock_process(bool did_unlock)
{
if (did_unlock)
process().big_lock().lock();
ASSERT(!Processor::current().in_critical());
Processor::current().in_critical() = prev_crit;
}
u64 Thread::sleep(u32 ticks)
@ -266,6 +273,7 @@ const char* Thread::state_string() const
void Thread::finalize()
{
ASSERT(Thread::current() == g_finalizer);
ASSERT(Thread::current() != this);
#ifdef THREAD_DEBUG
dbg() << "Finalizing thread " << *this;
@ -290,9 +298,10 @@ void Thread::finalize_dying_threads()
ASSERT(Thread::current() == g_finalizer);
Vector<Thread*, 32> dying_threads;
{
InterruptDisabler disabler;
ScopedSpinLock lock(g_scheduler_lock);
for_each_in_state(Thread::State::Dying, [&](Thread& thread) {
dying_threads.append(&thread);
if (thread.is_finalizable())
dying_threads.append(&thread);
return IterationDecision::Continue;
});
}
@ -723,20 +732,18 @@ void Thread::set_state(State new_state)
m_state = new_state;
#ifdef THREAD_DEBUG
dbg() << "Set Thread " << VirtualAddress(this) << " " << *this << " state to " << state_string();
dbg() << "Set Thread " << *this << " state to " << state_string();
#endif
if (m_process.pid() != 0) {
Scheduler::update_state_for_thread(*this);
}
if (new_state == Dying)
notify_finalizer();
}
void Thread::notify_finalizer()
{
g_finalizer_has_work.store(true, AK::MemoryOrder::memory_order_release);
g_finalizer_wait_queue->wake_all();
if (m_state == Dying && this != Thread::current() && is_finalizable()) {
// Some other thread set this thread to Dying, notify the
// finalizer right away as it can be cleaned up now
Scheduler::notify_finalizer();
}
}
String Thread::backtrace(ProcessInspectionHandle&)
@ -855,44 +862,72 @@ const LogStream& operator<<(const LogStream& stream, const Thread& value)
Thread::BlockResult Thread::wait_on(WaitQueue& queue, const char* reason, timeval* timeout, Atomic<bool>* lock, Thread* beneficiary)
{
TimerId timer_id {};
u32 prev_crit;
bool did_unlock;
{
InterruptDisabler disable;
did_unlock = unlock_process_if_locked(prev_crit);
if (lock)
*lock = false;
set_state(State::Queued);
m_wait_reason = reason;
queue.enqueue(*Thread::current());
ScopedCritical critical;
// We need to be in a critical section *and* then also acquire the
// scheduler lock. The only way acquiring the scheduler lock could
// block us is if another core were to be holding it, in which case
// we need to wait until the scheduler lock is released again
{
ScopedSpinLock sched_lock(g_scheduler_lock);
did_unlock = unlock_process_if_locked();
if (lock)
*lock = false;
set_state(State::Queued);
m_wait_reason = reason;
queue.enqueue(*Thread::current());
if (timeout) {
timer_id = TimerQueue::the().add_timer(*timeout, [&]() {
wake_from_queue();
});
if (timeout) {
timer_id = TimerQueue::the().add_timer(*timeout, [&]() {
ScopedSpinLock sched_lock(g_scheduler_lock);
wake_from_queue();
});
}
// Yield and wait for the queue to wake us up again.
if (beneficiary)
Scheduler::donate_to(beneficiary, reason);
else
Scheduler::yield();
}
// Yield and wait for the queue to wake us up again.
if (beneficiary)
Scheduler::donate_to(beneficiary, reason);
else
Scheduler::yield();
// Clearing the critical section may trigger the context switch
// flagged by calling Scheduler::donate_to or Scheduler::yield
// above. We have to do it this way because we intentionally
// leave the critical section here to be able to switch contexts.
u32 prev_flags;
u32 prev_crit = Processor::current().clear_critical(prev_flags, true);
// We've unblocked, relock the process if needed and carry on.
relock_process(did_unlock);
// NOTE: We may be on a differenct CPU now!
Processor::current().restore_critical(prev_crit, prev_flags);
// This looks counter productive, but we may not actually leave
// the critical section we just restored. It depends on whether
// we were in one while being called.
}
if (!are_interrupts_enabled())
sti();
// We've unblocked, relock the process if needed and carry on.
relock_process(did_unlock, prev_crit);
BlockResult result = m_wait_queue_node.is_in_list() ? BlockResult::InterruptedByTimeout : BlockResult::WokeNormally;
// Make sure we cancel the timer if woke normally.
if (timeout && result == BlockResult::WokeNormally)
TimerQueue::the().cancel_timer(timer_id);
BlockResult result;
{
// To be able to look at m_wait_queue_node we once again need the
// scheduler lock, which is held when we insert into the queue
ScopedSpinLock sched_lock(g_scheduler_lock);
result = m_wait_queue_node.is_in_list() ? BlockResult::InterruptedByTimeout : BlockResult::WokeNormally;
// Make sure we cancel the timer if woke normally.
if (timeout && result == BlockResult::WokeNormally)
TimerQueue::the().cancel_timer(timer_id);
}
// The API contract guarantees we return with interrupts enabled,
// regardless of how we got called
sti();
return result;
}

View file

@ -438,6 +438,18 @@ public:
return m_wait_reason;
}
void set_active(bool active)
{
ASSERT(g_scheduler_lock.is_locked());
m_is_active = active;
}
bool is_finalizable() const
{
ASSERT(g_scheduler_lock.is_locked());
return !m_is_active;
}
Thread* clone(Process&);
template<typename Callback>
@ -467,8 +479,8 @@ private:
private:
friend class SchedulerData;
friend class WaitQueue;
bool unlock_process_if_locked(u32& prev_crit);
void relock_process(bool did_unlock, u32 prev_crit);
bool unlock_process_if_locked();
void relock_process(bool did_unlock);
String backtrace_impl();
void reset_fpu_state();
@ -491,6 +503,7 @@ private:
Blocker* m_blocker { nullptr };
const char* m_wait_reason { nullptr };
bool m_is_active { false };
bool m_is_joinable { true };
Thread* m_joiner { nullptr };
Thread* m_joinee { nullptr };
@ -526,7 +539,6 @@ private:
OwnPtr<ThreadTracer> m_tracer;
void notify_finalizer();
void yield_without_holding_big_lock();
};