1
Fork 0
mirror of https://github.com/RGBCube/serenity synced 2025-07-25 23:47:45 +00:00

Kernel: Turn Thread::current and Process::current into functions

This allows us to query the current thread and process on a
per processor basis
This commit is contained in:
Tom 2020-06-28 15:34:31 -06:00 committed by Andreas Kling
parent cdc78515b6
commit 16783bd14d
39 changed files with 518 additions and 369 deletions

View file

@ -84,10 +84,10 @@
#include <LibELF/Validation.h>
#include <LibKeyboard/CharacterMapData.h>
//#define PROCESS_DEBUG
#define PROCESS_DEBUG
//#define DEBUG_POLL_SELECT
//#define DEBUG_IO
//#define TASK_DEBUG
#define TASK_DEBUG
//#define FORK_DEBUG
//#define EXEC_DEBUG
//#define SIGNAL_DEBUG
@ -97,9 +97,8 @@ namespace Kernel {
static void create_signal_trampolines();
Process* Process::current;
static pid_t next_pid;
RecursiveSpinLock g_processes_lock;
static Atomic<pid_t> next_pid;
InlineLinkedList<Process>* g_processes;
static String* s_hostname;
static Lock* s_hostname_lock;
@ -108,15 +107,14 @@ HashMap<String, OwnPtr<Module>>* g_modules;
pid_t Process::allocate_pid()
{
InterruptDisabler disabler;
return next_pid++;
return next_pid.fetch_add(1, AK::MemoryOrder::memory_order_acq_rel);
}
void Process::initialize()
{
g_modules = new HashMap<String, OwnPtr<Module>>;
next_pid = 0;
next_pid.store(0, AK::MemoryOrder::memory_order_release);
g_processes = new InlineLinkedList<Process>;
s_hostname = new String("courage");
s_hostname_lock = new Lock;
@ -127,7 +125,7 @@ void Process::initialize()
Vector<pid_t> Process::all_pids()
{
Vector<pid_t> pids;
InterruptDisabler disabler;
ScopedSpinLock lock(g_processes_lock);
pids.ensure_capacity((int)g_processes->size_slow());
for (auto& process : *g_processes)
pids.append(process.pid());
@ -137,7 +135,7 @@ Vector<pid_t> Process::all_pids()
Vector<Process*> Process::all_processes()
{
Vector<Process*> processes;
InterruptDisabler disabler;
ScopedSpinLock lock(g_processes_lock);
processes.ensure_capacity((int)g_processes->size_slow());
for (auto& process : *g_processes)
processes.append(&process);
@ -234,7 +232,7 @@ Region* Process::allocate_region_with_vmobject(VirtualAddress vaddr, size_t size
bool Process::deallocate_region(Region& region)
{
InterruptDisabler disabler;
ScopedSpinLock lock(m_lock);
if (m_region_lookup_cache.region == &region)
m_region_lookup_cache.region = nullptr;
for (size_t i = 0; i < m_regions.size(); ++i) {
@ -248,6 +246,7 @@ bool Process::deallocate_region(Region& region)
Region* Process::region_from_range(const Range& range)
{
ScopedSpinLock lock(m_lock);
if (m_region_lookup_cache.range == range && m_region_lookup_cache.region)
return m_region_lookup_cache.region;
@ -264,6 +263,7 @@ Region* Process::region_from_range(const Range& range)
Region* Process::region_containing(const Range& range)
{
ScopedSpinLock lock(m_lock);
for (auto& region : m_regions) {
if (region.contains(range))
return &region;
@ -738,17 +738,6 @@ pid_t Process::sys$fork(RegisterState& regs)
dbg() << "fork: child=" << child;
#endif
for (auto& region : m_regions) {
#ifdef FORK_DEBUG
dbg() << "fork: cloning Region{" << &region << "} '" << region.name() << "' @ " << region.vaddr();
#endif
auto& child_region = child->add_region(region.clone());
child_region.map(child->page_directory());
if (&region == m_master_tls_region)
child->m_master_tls_region = child_region.make_weak_ptr();
}
child->m_extra_gids = m_extra_gids;
auto& child_tss = child_first_thread->m_tss;
@ -773,8 +762,20 @@ pid_t Process::sys$fork(RegisterState& regs)
dbg() << "fork: child will begin executing at " << String::format("%w", child_tss.cs) << ":" << String::format("%x", child_tss.eip) << " with stack " << String::format("%w", child_tss.ss) << ":" << String::format("%x", child_tss.esp) << ", kstack " << String::format("%w", child_tss.ss0) << ":" << String::format("%x", child_tss.esp0);
#endif
ScopedSpinLock lock(m_lock);
for (auto& region : m_regions) {
#ifdef FORK_DEBUG
dbg() << "fork: cloning Region{" << &region << "} '" << region.name() << "' @ " << region.vaddr();
#endif
auto& child_region = child->add_region(region.clone());
child_region.map(child->page_directory());
if (&region == m_master_tls_region)
child->m_master_tls_region = child_region.make_weak_ptr();
}
{
InterruptDisabler disabler;
ScopedSpinLock lock(g_processes_lock);
g_processes->prepend(child);
}
#ifdef TASK_DEBUG
@ -789,11 +790,12 @@ void Process::kill_threads_except_self()
{
InterruptDisabler disabler;
if (m_thread_count <= 1)
if (thread_count() <= 1)
return;
auto current_thread = Thread::current();
for_each_thread([&](Thread& thread) {
if (&thread == Thread::current
if (&thread == current_thread
|| thread.state() == Thread::State::Dead
|| thread.state() == Thread::State::Dying)
return IterationDecision::Continue;
@ -857,7 +859,8 @@ int Process::do_exec(NonnullRefPtr<FileDescription> main_program_description, Ve
// Mark this thread as the current thread that does exec
// No other thread from this process will be scheduled to run
m_exec_tid = Thread::current->tid();
auto current_thread = Thread::current();
m_exec_tid = current_thread->tid();
RefPtr<PageDirectory> old_page_directory;
NonnullOwnPtrVector<Region> old_regions;
@ -905,7 +908,7 @@ int Process::do_exec(NonnullRefPtr<FileDescription> main_program_description, Ve
RefPtr<ELF::Loader> loader;
{
ArmedScopeGuard rollback_regions_guard([&]() {
ASSERT(Process::current == this);
ASSERT(Process::current() == this);
// Need to make sure we don't swap contexts in the middle
InterruptDisabler disabler;
m_page_directory = move(old_page_directory);
@ -1005,9 +1008,9 @@ int Process::do_exec(NonnullRefPtr<FileDescription> main_program_description, Ve
m_egid = m_sgid = main_program_metadata.gid;
}
Thread::current->set_default_signal_dispositions();
Thread::current->m_signal_mask = 0;
Thread::current->m_pending_signals = 0;
current_thread->set_default_signal_dispositions();
current_thread->m_signal_mask = 0;
current_thread->m_pending_signals = 0;
m_futex_queues.clear();
@ -1024,8 +1027,8 @@ int Process::do_exec(NonnullRefPtr<FileDescription> main_program_description, Ve
}
Thread* new_main_thread = nullptr;
if (Process::current == this) {
new_main_thread = Thread::current;
if (&current_thread->process() == this) {
new_main_thread = current_thread;
} else {
for_each_thread([&](auto& thread) {
new_main_thread = &thread;
@ -1041,7 +1044,7 @@ int Process::do_exec(NonnullRefPtr<FileDescription> main_program_description, Ve
// We cli() manually here because we don't want to get interrupted between do_exec() and Processor::assume_context().
// The reason is that the task redirection we've set up above will be clobbered by the timer IRQ.
// If we used an InterruptDisabler that sti()'d on exit, we might timer tick'd too soon in exec().
if (Process::current == this)
if (&current_thread->process() == this)
cli();
// NOTE: Be careful to not trigger any page faults below!
@ -1253,16 +1256,17 @@ int Process::exec(String path, Vector<String> arguments, Vector<String> environm
if (rc < 0)
return rc;
auto current_thread = Thread::current();
if (m_wait_for_tracer_at_next_execve) {
ASSERT(Thread::current->state() == Thread::State::Skip1SchedulerPass);
ASSERT(current_thread->state() == Thread::State::Skip1SchedulerPass);
// State::Skip1SchedulerPass is irrelevant since we block the thread
Thread::current->set_state(Thread::State::Running);
Thread::current->send_urgent_signal_to_self(SIGSTOP);
current_thread->set_state(Thread::State::Running);
current_thread->send_urgent_signal_to_self(SIGSTOP);
}
if (Process::current == this) {
Thread::current->set_state(Thread::State::Running);
Processor::assume_context(*Thread::current);
if (&current_thread->process() == this) {
current_thread->set_state(Thread::State::Running);
Processor::assume_context(*current_thread);
ASSERT_NOT_REACHED();
}
return 0;
@ -1282,7 +1286,7 @@ int Process::sys$execve(const Syscall::SC_execve_params* user_params)
return -E2BIG;
if (m_wait_for_tracer_at_next_execve)
Thread::current->send_urgent_signal_to_self(SIGSTOP);
Thread::current()->send_urgent_signal_to_self(SIGSTOP);
String path;
{
@ -1331,7 +1335,7 @@ Process* Process::create_user_process(Thread*& first_thread, const String& path,
RefPtr<Custody> cwd;
RefPtr<Custody> root;
{
InterruptDisabler disabler;
ScopedSpinLock lock(g_processes_lock);
if (auto* parent = Process::from_pid(parent_pid)) {
cwd = parent->m_cwd;
root = parent->m_root_directory;
@ -1361,7 +1365,7 @@ Process* Process::create_user_process(Thread*& first_thread, const String& path,
}
{
InterruptDisabler disabler;
ScopedSpinLock lock(g_processes_lock);
g_processes->prepend(process);
}
#ifdef TASK_DEBUG
@ -1371,19 +1375,20 @@ Process* Process::create_user_process(Thread*& first_thread, const String& path,
return process;
}
Process* Process::create_kernel_process(Thread*& first_thread, String&& name, void (*e)())
Process* Process::create_kernel_process(Thread*& first_thread, String&& name, void (*e)(), u32 affinity)
{
auto* process = new Process(first_thread, move(name), (uid_t)0, (gid_t)0, (pid_t)0, Ring0);
first_thread->tss().eip = (FlatPtr)e;
if (process->pid() != 0) {
InterruptDisabler disabler;
ScopedSpinLock lock(g_processes_lock);
g_processes->prepend(process);
#ifdef TASK_DEBUG
klog() << "Kernel process " << process->pid() << " (" << process->name().characters() << ") spawned @ " << String::format("%p", first_thread->tss().eip);
#endif
}
first_thread->set_affinity(affinity);
first_thread->set_state(Thread::State::Runnable);
return process;
}
@ -1414,7 +1419,7 @@ Process::Process(Thread*& first_thread, const String& name, uid_t uid, gid_t gid
if (fork_parent) {
// NOTE: fork() doesn't clone all threads; the thread that called fork() becomes the only thread in the new process.
first_thread = Thread::current->clone(*this);
first_thread = Thread::current()->clone(*this);
} else {
// NOTE: This non-forked code path is only taken when the kernel creates a process "manually" (at boot.)
first_thread = new Thread(*this);
@ -1449,7 +1454,7 @@ void Process::sys$exit(int status)
m_termination_status = status;
m_termination_signal = 0;
die();
Thread::current->die_if_needed();
Thread::current()->die_if_needed();
ASSERT_NOT_REACHED();
}
@ -1513,7 +1518,7 @@ int Process::sys$sigreturn(RegisterState& registers)
//pop the stored eax, ebp, return address, handler and signal code
stack_ptr += 5;
Thread::current->m_signal_mask = *stack_ptr;
Thread::current()->m_signal_mask = *stack_ptr;
stack_ptr++;
//pop edi, esi, ebp, esp, ebx, edx, ecx and eax
@ -1534,7 +1539,7 @@ void Process::crash(int signal, u32 eip, bool out_of_memory)
{
ASSERT_INTERRUPTS_DISABLED();
ASSERT(!is_dead());
ASSERT(Process::current == this);
ASSERT(Process::current() == this);
if (out_of_memory) {
dbg() << "\033[31;1mOut of memory\033[m, killing: " << *this;
@ -1555,13 +1560,14 @@ void Process::crash(int signal, u32 eip, bool out_of_memory)
die();
// We can not return from here, as there is nowhere
// to unwind to, so die right away.
Thread::current->die_if_needed();
Thread::current()->die_if_needed();
ASSERT_NOT_REACHED();
}
Process* Process::from_pid(pid_t pid)
{
ASSERT_INTERRUPTS_DISABLED();
ScopedSpinLock lock(g_processes_lock);
for (auto& process : *g_processes) {
if (process.pid() == pid)
return &process;
@ -1713,7 +1719,7 @@ ssize_t Process::do_write(FileDescription& description, const u8* data, int data
#ifdef IO_DEBUG
dbg() << "block write on " << description.absolute_path();
#endif
if (Thread::current->block<Thread::WriteBlocker>(description) != Thread::BlockResult::WokeNormally) {
if (Thread::current()->block<Thread::WriteBlocker>(description) != Thread::BlockResult::WokeNormally) {
if (nwritten == 0)
return -EINTR;
}
@ -1776,7 +1782,7 @@ ssize_t Process::sys$read(int fd, u8* buffer, ssize_t size)
return -EISDIR;
if (description->is_blocking()) {
if (!description->can_read()) {
if (Thread::current->block<Thread::ReadBlocker>(*description) != Thread::BlockResult::WokeNormally)
if (Thread::current()->block<Thread::ReadBlocker>(*description) != Thread::BlockResult::WokeNormally)
return -EINTR;
if (!description->can_read())
return -EAGAIN;
@ -2323,6 +2329,7 @@ KResult Process::do_killall(int signal)
KResult error = KSuccess;
// Send the signal to all processes we have access to for.
ScopedSpinLock lock(g_processes_lock);
for (auto& process : *g_processes) {
KResult res = KSuccess;
if (process.pid() == m_pid)
@ -2346,9 +2353,10 @@ KResult Process::do_killself(int signal)
if (signal == 0)
return KSuccess;
if (!Thread::current->should_ignore_signal(signal)) {
Thread::current->send_signal(signal, this);
(void)Thread::current->block<Thread::SemiPermanentBlocker>(Thread::SemiPermanentBlocker::Reason::Signal);
auto current_thread = Thread::current();
if (!current_thread->should_ignore_signal(signal)) {
current_thread->send_signal(signal, this);
(void)current_thread->block<Thread::SemiPermanentBlocker>(Thread::SemiPermanentBlocker::Reason::Signal);
}
return KSuccess;
@ -2373,7 +2381,7 @@ int Process::sys$kill(pid_t pid, int signal)
if (pid == m_pid) {
return do_killself(signal);
}
InterruptDisabler disabler;
ScopedSpinLock lock(g_processes_lock);
auto* peer = Process::from_pid(pid);
if (!peer)
return -ESRCH;
@ -2385,7 +2393,7 @@ int Process::sys$usleep(useconds_t usec)
REQUIRE_PROMISE(stdio);
if (!usec)
return 0;
u64 wakeup_time = Thread::current->sleep(usec / 1000);
u64 wakeup_time = Thread::current()->sleep(usec / 1000);
if (wakeup_time > g_uptime)
return -EINTR;
return 0;
@ -2396,7 +2404,7 @@ int Process::sys$sleep(unsigned seconds)
REQUIRE_PROMISE(stdio);
if (!seconds)
return 0;
u64 wakeup_time = Thread::current->sleep(seconds * TimeManagement::the().ticks_per_second());
u64 wakeup_time = Thread::current()->sleep(seconds * TimeManagement::the().ticks_per_second());
if (wakeup_time > g_uptime) {
u32 ticks_left_until_original_wakeup_time = wakeup_time - g_uptime;
return ticks_left_until_original_wakeup_time / TimeManagement::the().ticks_per_second();
@ -2519,23 +2527,22 @@ siginfo_t Process::reap(Process& process)
siginfo.si_code = CLD_EXITED;
}
{
InterruptDisabler disabler;
ASSERT(g_processes_lock.is_locked());
if (process.ppid()) {
auto* parent = Process::from_pid(process.ppid());
if (parent) {
parent->m_ticks_in_user_for_dead_children += process.m_ticks_in_user + process.m_ticks_in_user_for_dead_children;
parent->m_ticks_in_kernel_for_dead_children += process.m_ticks_in_kernel + process.m_ticks_in_kernel_for_dead_children;
}
if (process.ppid()) {
auto* parent = Process::from_pid(process.ppid());
if (parent) {
parent->m_ticks_in_user_for_dead_children += process.m_ticks_in_user + process.m_ticks_in_user_for_dead_children;
parent->m_ticks_in_kernel_for_dead_children += process.m_ticks_in_kernel + process.m_ticks_in_kernel_for_dead_children;
}
}
#ifdef PROCESS_DEBUG
dbg() << "Reaping process " << process;
dbg() << "Reaping process " << process;
#endif
ASSERT(process.is_dead());
g_processes->remove(&process);
}
ASSERT(process.is_dead());
g_processes->remove(&process);
delete &process;
return siginfo;
}
@ -2543,7 +2550,7 @@ siginfo_t Process::reap(Process& process)
KResultOr<siginfo_t> Process::do_waitid(idtype_t idtype, int id, int options)
{
if (idtype == P_PID) {
InterruptDisabler disabler;
ScopedSpinLock lock(g_processes_lock);
if (idtype == P_PID && !Process::from_pid(id))
return KResult(-ECHILD);
}
@ -2560,10 +2567,10 @@ KResultOr<siginfo_t> Process::do_waitid(idtype_t idtype, int id, int options)
return KResult(-EINVAL);
}
if (Thread::current->block<Thread::WaitBlocker>(options, waitee_pid) != Thread::BlockResult::WokeNormally)
if (Thread::current()->block<Thread::WaitBlocker>(options, waitee_pid) != Thread::BlockResult::WokeNormally)
return KResult(-EINTR);
InterruptDisabler disabler;
ScopedSpinLock lock(g_processes_lock);
// NOTE: If waitee was -1, m_waitee_pid will have been filled in by the scheduler.
Process* waitee_process = Process::from_pid(waitee_pid);
@ -2660,7 +2667,7 @@ pid_t Process::sys$getsid(pid_t pid)
REQUIRE_PROMISE(proc);
if (pid == 0)
return m_sid;
InterruptDisabler disabler;
ScopedSpinLock lock(g_processes_lock);
auto* process = Process::from_pid(pid);
if (!process)
return -ESRCH;
@ -2691,7 +2698,7 @@ pid_t Process::sys$getpgid(pid_t pid)
REQUIRE_PROMISE(proc);
if (pid == 0)
return m_pgid;
InterruptDisabler disabler; // FIXME: Use a ProcessHandle
ScopedSpinLock lock(g_processes_lock); // FIXME: Use a ProcessHandle
auto* process = Process::from_pid(pid);
if (!process)
return -ESRCH;
@ -2706,7 +2713,7 @@ pid_t Process::sys$getpgrp()
static pid_t get_sid_from_pgid(pid_t pgid)
{
InterruptDisabler disabler;
ScopedSpinLock lock(g_processes_lock);
auto* group_leader = Process::from_pid(pgid);
if (!group_leader)
return -1;
@ -2716,7 +2723,7 @@ static pid_t get_sid_from_pgid(pid_t pgid)
int Process::sys$setpgid(pid_t specified_pid, pid_t specified_pgid)
{
REQUIRE_PROMISE(proc);
InterruptDisabler disabler; // FIXME: Use a ProcessHandle
ScopedSpinLock lock(g_processes_lock); // FIXME: Use a ProcessHandle
pid_t pid = specified_pid ? specified_pid : m_pid;
if (specified_pgid < 0) {
// The value of the pgid argument is less than 0, or is not a value supported by the implementation.
@ -2790,10 +2797,11 @@ int Process::sys$dup2(int old_fd, int new_fd)
int Process::sys$sigprocmask(int how, const sigset_t* set, sigset_t* old_set)
{
REQUIRE_PROMISE(sigaction);
auto current_thread = Thread::current();
if (old_set) {
if (!validate_write_typed(old_set))
return -EFAULT;
copy_to_user(old_set, &Thread::current->m_signal_mask);
copy_to_user(old_set, &current_thread->m_signal_mask);
}
if (set) {
if (!validate_read_typed(set))
@ -2802,13 +2810,13 @@ int Process::sys$sigprocmask(int how, const sigset_t* set, sigset_t* old_set)
copy_from_user(&set_value, set);
switch (how) {
case SIG_BLOCK:
Thread::current->m_signal_mask &= ~set_value;
current_thread->m_signal_mask &= ~set_value;
break;
case SIG_UNBLOCK:
Thread::current->m_signal_mask |= set_value;
current_thread->m_signal_mask |= set_value;
break;
case SIG_SETMASK:
Thread::current->m_signal_mask = set_value;
current_thread->m_signal_mask = set_value;
break;
default:
return -EINVAL;
@ -2822,7 +2830,7 @@ int Process::sys$sigpending(sigset_t* set)
REQUIRE_PROMISE(stdio);
if (!validate_write_typed(set))
return -EFAULT;
copy_to_user(set, &Thread::current->m_pending_signals);
copy_to_user(set, &Thread::current()->m_pending_signals);
return 0;
}
@ -2834,7 +2842,7 @@ int Process::sys$sigaction(int signum, const sigaction* act, sigaction* old_act)
if (!validate_read_typed(act))
return -EFAULT;
InterruptDisabler disabler; // FIXME: This should use a narrower lock. Maybe a way to ignore signals temporarily?
auto& action = Thread::current->m_signal_action_data[signum];
auto& action = Thread::current()->m_signal_action_data[signum];
if (old_act) {
if (!validate_write_typed(old_act))
return -EFAULT;
@ -2988,9 +2996,10 @@ int Process::sys$select(const Syscall::SC_select_params* params)
select_has_timeout = true;
}
ScopedValueRollback scoped_sigmask(Thread::current->m_signal_mask);
auto current_thread = Thread::current();
ScopedValueRollback scoped_sigmask(current_thread->m_signal_mask);
if (sigmask)
Thread::current->m_signal_mask = *sigmask;
current_thread->m_signal_mask = *sigmask;
Thread::SelectBlocker::FDVector rfds;
Thread::SelectBlocker::FDVector wfds;
@ -3023,7 +3032,7 @@ int Process::sys$select(const Syscall::SC_select_params* params)
#endif
if (!timeout || select_has_timeout) {
if (Thread::current->block<Thread::SelectBlocker>(computed_timeout, select_has_timeout, rfds, wfds, efds) != Thread::BlockResult::WokeNormally)
if (current_thread->block<Thread::SelectBlocker>(computed_timeout, select_has_timeout, rfds, wfds, efds) != Thread::BlockResult::WokeNormally)
return -EINTR;
// While we blocked, the process lock was dropped. This gave other threads
// the opportunity to mess with the memory. For example, it could free the
@ -3099,16 +3108,17 @@ int Process::sys$poll(const Syscall::SC_poll_params* params)
has_timeout = true;
}
ScopedValueRollback scoped_sigmask(Thread::current->m_signal_mask);
auto current_thread = Thread::current();
ScopedValueRollback scoped_sigmask(current_thread->m_signal_mask);
if (sigmask)
Thread::current->m_signal_mask = *sigmask;
current_thread->m_signal_mask = *sigmask;
#if defined(DEBUG_IO) || defined(DEBUG_POLL_SELECT)
dbg() << "polling on (read:" << rfds.size() << ", write:" << wfds.size() << "), timeout=" << timeout;
#endif
if (!timeout || has_timeout) {
if (Thread::current->block<Thread::SelectBlocker>(actual_timeout, has_timeout, rfds, wfds, Thread::SelectBlocker::FDVector()) != Thread::BlockResult::WokeNormally)
if (current_thread->block<Thread::SelectBlocker>(actual_timeout, has_timeout, rfds, wfds, Thread::SelectBlocker::FDVector()) != Thread::BlockResult::WokeNormally)
return -EINTR;
}
@ -3245,7 +3255,7 @@ int Process::sys$chown(const Syscall::SC_chown_params* user_params)
void Process::finalize()
{
ASSERT(Thread::current == g_finalizer);
ASSERT(Thread::current() == g_finalizer);
#ifdef PROCESS_DEBUG
dbg() << "Finalizing process " << *this;
#endif
@ -3460,7 +3470,7 @@ int Process::sys$accept(int accepting_socket_fd, sockaddr* user_address, socklen
if (!socket.can_accept()) {
if (accepting_socket_description->is_blocking()) {
if (Thread::current->block<Thread::AcceptBlocker>(*accepting_socket_description) != Thread::BlockResult::WokeNormally)
if (Thread::current()->block<Thread::AcceptBlocker>(*accepting_socket_description) != Thread::BlockResult::WokeNormally)
return -EINTR;
} else {
return -EAGAIN;
@ -3663,7 +3673,7 @@ int Process::sys$sched_setparam(int tid, const struct sched_param* param)
copy_from_user(&desired_priority, &param->sched_priority);
InterruptDisabler disabler;
auto* peer = Thread::current;
auto* peer = Thread::current();
if (tid != 0)
peer = Thread::from_tid(tid);
@ -3687,7 +3697,7 @@ int Process::sys$sched_getparam(pid_t pid, struct sched_param* param)
return -EFAULT;
InterruptDisabler disabler;
auto* peer = Thread::current;
auto* peer = Thread::current();
if (pid != 0)
peer = Thread::from_tid(pid);
@ -3808,7 +3818,7 @@ int Process::sys$shbuf_allow_pid(int shbuf_id, pid_t peer_pid)
if (!shared_buffer.is_shared_with(m_pid))
return -EPERM;
{
InterruptDisabler disabler;
ScopedSpinLock lock(g_processes_lock);
auto* peer = Process::from_pid(peer_pid);
if (!peer)
return -ESRCH;
@ -3987,14 +3997,35 @@ int Process::sys$create_thread(void* (*entry)(void*), const Syscall::SC_create_t
return thread->tid();
}
Thread* Process::create_kernel_thread(void (*entry)(), u32 priority, const String& name, u32 affinity, bool joinable)
{
ASSERT((priority >= THREAD_PRIORITY_MIN) && (priority <= THREAD_PRIORITY_MAX));
// FIXME: Do something with guard pages?
auto* thread = new Thread(*this);
thread->set_name(name);
thread->set_affinity(affinity);
thread->set_priority(priority);
thread->set_joinable(joinable);
auto& tss = thread->tss();
tss.eip = (FlatPtr)entry;
thread->set_state(Thread::State::Runnable);
return thread;
}
void Process::sys$exit_thread(void* exit_value)
{
REQUIRE_PROMISE(thread);
cli();
Thread::current->m_exit_value = exit_value;
Thread::current->set_should_die();
auto current_thread = Thread::current();
current_thread->m_exit_value = exit_value;
current_thread->set_should_die();
big_lock().force_unlock_if_locked();
Thread::current->die_if_needed();
current_thread->die_if_needed();
ASSERT_NOT_REACHED();
}
@ -4024,13 +4055,14 @@ int Process::sys$join_thread(int tid, void** exit_value)
if (!thread || thread->pid() != pid())
return -ESRCH;
if (thread == Thread::current)
auto current_thread = Thread::current();
if (thread == current_thread)
return -EDEADLK;
if (thread->m_joinee == Thread::current)
if (thread->m_joinee == current_thread)
return -EDEADLK;
ASSERT(thread->m_joiner != Thread::current);
ASSERT(thread->m_joiner != current_thread);
if (thread->m_joiner)
return -EINVAL;
@ -4041,15 +4073,15 @@ int Process::sys$join_thread(int tid, void** exit_value)
// NOTE: pthread_join() cannot be interrupted by signals. Only by death.
for (;;) {
auto result = Thread::current->block<Thread::JoinBlocker>(*thread, joinee_exit_value);
auto result = current_thread->block<Thread::JoinBlocker>(*thread, joinee_exit_value);
if (result == Thread::BlockResult::InterruptedByDeath) {
// NOTE: This cleans things up so that Thread::finalize() won't
// get confused about a missing joiner when finalizing the joinee.
InterruptDisabler disabler_t;
if (Thread::current->m_joinee) {
Thread::current->m_joinee->m_joiner = nullptr;
Thread::current->m_joinee = nullptr;
if (current_thread->m_joinee) {
current_thread->m_joinee->m_joiner = nullptr;
current_thread->m_joinee = nullptr;
}
break;
@ -4107,7 +4139,7 @@ int Process::sys$get_thread_name(int tid, char* buffer, size_t buffer_size)
int Process::sys$gettid()
{
REQUIRE_PROMISE(stdio);
return Thread::current->tid();
return Thread::current()->tid();
}
int Process::sys$donate(int tid)
@ -4528,12 +4560,12 @@ int Process::sys$clock_nanosleep(const Syscall::SC_clock_nanosleep_params* user_
u64 wakeup_time;
if (is_absolute) {
u64 time_to_wake = (requested_sleep.tv_sec * 1000 + requested_sleep.tv_nsec / 1000000);
wakeup_time = Thread::current->sleep_until(time_to_wake);
wakeup_time = Thread::current()->sleep_until(time_to_wake);
} else {
u32 ticks_to_sleep = (requested_sleep.tv_sec * 1000 + requested_sleep.tv_nsec / 1000000);
if (!ticks_to_sleep)
return 0;
wakeup_time = Thread::current->sleep(ticks_to_sleep);
wakeup_time = Thread::current()->sleep(ticks_to_sleep);
}
if (wakeup_time > g_uptime) {
u32 ticks_left = wakeup_time - g_uptime;
@ -4571,14 +4603,14 @@ int Process::sys$sync()
int Process::sys$yield()
{
REQUIRE_PROMISE(stdio);
Thread::current->yield_without_holding_big_lock();
Thread::current()->yield_without_holding_big_lock();
return 0;
}
int Process::sys$beep()
{
PCSpeaker::tone_on(440);
u64 wakeup_time = Thread::current->sleep(100);
u64 wakeup_time = Thread::current()->sleep(100);
PCSpeaker::tone_off();
if (wakeup_time > g_uptime)
return -EINTR;
@ -4738,7 +4770,7 @@ int Process::sys$module_unload(const char* user_name, size_t name_length)
int Process::sys$profiling_enable(pid_t pid)
{
REQUIRE_NO_PROMISES;
InterruptDisabler disabler;
ScopedSpinLock lock(g_processes_lock);
auto* process = Process::from_pid(pid);
if (!process)
return -ESRCH;
@ -4753,7 +4785,7 @@ int Process::sys$profiling_enable(pid_t pid)
int Process::sys$profiling_disable(pid_t pid)
{
InterruptDisabler disabler;
ScopedSpinLock lock(g_processes_lock);
auto* process = Process::from_pid(pid);
if (!process)
return -ESRCH;
@ -4811,7 +4843,7 @@ int Process::sys$futex(const Syscall::SC_futex_params* user_params)
}
// FIXME: This is supposed to be interruptible by a signal, but right now WaitQueue cannot be interrupted.
Thread::BlockResult result = Thread::current->wait_on(wait_queue, optional_timeout);
Thread::BlockResult result = Thread::current()->wait_on(wait_queue, optional_timeout);
if (result == Thread::BlockResult::InterruptedByTimeout) {
return -ETIMEDOUT;
}
@ -4854,7 +4886,7 @@ int Process::sys$set_process_boost(pid_t pid, int amount)
REQUIRE_PROMISE(proc);
if (amount < 0 || amount > 20)
return -EINVAL;
InterruptDisabler disabler;
ScopedSpinLock lock(g_processes_lock);
auto* process = Process::from_pid(pid);
if (!process || process->is_dead())
return -ESRCH;
@ -4976,6 +5008,7 @@ int Process::sys$pledge(const Syscall::SC_pledge_params* user_params)
Region& Process::add_region(NonnullOwnPtr<Region> region)
{
auto* ptr = region.ptr();
ScopedSpinLock lock(m_lock);
m_regions.append(move(region));
return *ptr;
}
@ -5091,7 +5124,7 @@ int Process::sys$get_stack_bounds(FlatPtr* user_stack_base, size_t* user_stack_s
if (!validate_write_typed(user_stack_size))
return -EFAULT;
FlatPtr stack_pointer = Thread::current->get_register_dump_from_stack().userspace_esp;
FlatPtr stack_pointer = Thread::current()->get_register_dump_from_stack().userspace_esp;
auto* stack_region = MM.region_from_vaddr(*this, VirtualAddress(stack_pointer));
if (!stack_region) {
ASSERT_NOT_REACHED();