Re-implement the alarm syscall

This commit is contained in:
Chen Chengjun 2024-04-30 11:23:19 +08:00 committed by Tate, Hongliang Tian
parent b226928349
commit 2d17177cb5
6 changed files with 82 additions and 176 deletions

View File

@ -2,15 +2,12 @@
use aster_frame::user::UserSpace; use aster_frame::user::UserSpace;
use super::{PosixThread, PosixThreadExt, RealTimer}; use super::PosixThread;
use crate::{ use crate::{
prelude::*, prelude::*,
process::{ process::{
posix_thread::name::ThreadName, posix_thread::name::ThreadName,
signal::{ signal::{sig_mask::SigMask, sig_queues::SigQueues},
constants::SIGALRM, sig_mask::SigMask, sig_queues::SigQueues,
signals::kernel::KernelSignal,
},
Credentials, Process, Credentials, Process,
}, },
thread::{status::ThreadStatus, task, thread_table, Thread, Tid}, thread::{status::ThreadStatus, task, thread_table, Thread, Tid},
@ -94,20 +91,6 @@ impl PosixThreadBuilder {
is_main_thread, is_main_thread,
} = self; } = self;
let real_timer = RealTimer::new(move || {
let process = {
let Some(current_thread) = thread_table::get_thread(tid) else {
return;
};
let posix_thread = current_thread.as_posix_thread().unwrap();
posix_thread.process()
};
let signal = KernelSignal::new(SIGALRM);
process.enqueue_signal(signal);
})
.unwrap();
let thread = Arc::new_cyclic(|thread_ref| { let thread = Arc::new_cyclic(|thread_ref| {
let task = task::create_new_user_task(user_space, thread_ref.clone()); let task = task::create_new_user_task(user_space, thread_ref.clone());
let status = ThreadStatus::Init; let status = ThreadStatus::Init;
@ -118,7 +101,6 @@ impl PosixThreadBuilder {
set_child_tid: Mutex::new(set_child_tid), set_child_tid: Mutex::new(set_child_tid),
clear_child_tid: Mutex::new(clear_child_tid), clear_child_tid: Mutex::new(clear_child_tid),
credentials, credentials,
real_timer: Mutex::new(real_timer),
sig_mask: Mutex::new(sig_mask), sig_mask: Mutex::new(sig_mask),
sig_queues, sig_queues,
sig_context: Mutex::new(None), sig_context: Mutex::new(None),

View File

@ -26,13 +26,11 @@ pub mod futex;
mod name; mod name;
mod posix_thread_ext; mod posix_thread_ext;
mod robust_list; mod robust_list;
mod timer;
pub use builder::PosixThreadBuilder; pub use builder::PosixThreadBuilder;
pub use name::{ThreadName, MAX_THREAD_NAME_LEN}; pub use name::{ThreadName, MAX_THREAD_NAME_LEN};
pub use posix_thread_ext::PosixThreadExt; pub use posix_thread_ext::PosixThreadExt;
pub use robust_list::RobustListHead; pub use robust_list::RobustListHead;
pub use timer::RealTimer;
pub struct PosixThread { pub struct PosixThread {
// Immutable part // Immutable part
@ -52,9 +50,6 @@ pub struct PosixThread {
/// Process credentials. At the kernel level, credentials are a per-thread attribute. /// Process credentials. At the kernel level, credentials are a per-thread attribute.
credentials: Credentials, credentials: Credentials,
/// The timer counts down in real (i.e., wall clock) time
real_timer: Mutex<RealTimer>,
// Signal // Signal
/// Blocked signals /// Blocked signals
sig_mask: Mutex<SigMask>, sig_mask: Mutex<SigMask>,
@ -146,10 +141,6 @@ impl PosixThread {
return_errno_with_message!(Errno::EPERM, "sending signal to the thread is not allowed."); return_errno_with_message!(Errno::EPERM, "sending signal to the thread is not allowed.");
} }
pub fn real_timer(&self) -> &Mutex<RealTimer> {
&self.real_timer
}
pub(in crate::process) fn enqueue_signal(&self, signal: Box<dyn Signal>) { pub(in crate::process) fn enqueue_signal(&self, signal: Box<dyn Signal>) {
self.sig_queues.enqueue(signal); self.sig_queues.enqueue(signal);
} }
@ -242,8 +233,6 @@ impl PosixThread {
thread_table::remove_thread(tid); thread_table::remove_thread(tid);
} }
self.real_timer.lock().clear();
if self.is_main_thread() || self.is_last_thread() { if self.is_main_thread() || self.is_last_thread() {
// exit current process. // exit current process.
debug!("self is main thread or last thread"); debug!("self is main thread or last thread");

View File

@ -1,111 +0,0 @@
// SPDX-License-Identifier: MPL-2.0
use core::time::Duration;
use crate::{
prelude::*,
thread::work_queue::{submit_work_item, work_item::WorkItem, WorkPriority},
time::SystemTime,
};
/// A timer that counts down in real (wall clock) time to run delayed callbacks in process context.
///
/// Unlike the `Timer` in `aster-frame`, the callbacks of this `RealTimer` will be executed in process
/// context instead of interrupt context. This leads to two differences:
///
/// 1. The callbacks of this `RealTimer` can sleep, whereas the callbacks of `Timer` in `aster-frame` cannot.
/// 2. The callbacks of this `RealTimer` may be delayed by an arbitrary amount of time due to scheduler strategy.
///
/// Note that the callbacks may not be executed in the process context of the timer's creator, so macros such
/// as `current` and `current_thread` should **NOT** be used in the callback.
///
/// # Example
/// ```rust
/// let current_tid = current_thread!().tid();
/// let timer = RealTimer::new(move || {
/// let current_thread = thread_table::get_thread(current_tid);
/// let posix_thread = current_thread.as_posix_thread().unwrap();
/// let process = posix_thread.process();
/// println!("Task executed for PID: {}", process.pid());
/// }, Duration::from_secs(1));
pub struct RealTimer {
timer: Arc<aster_frame::timer::Timer>,
expired_time: Option<SystemTime>,
}
impl RealTimer {
/// Creates a new `RealTimer`. The `callback` parameter will be called once the timeout is reached.
pub fn new(callback: impl Fn() + Send + Sync + Copy + 'static) -> Result<Self> {
let timer = {
aster_frame::timer::Timer::new(move |timer| {
let work_func = Box::new(callback);
let work_item = { Arc::new(WorkItem::new(work_func)) };
// FIXME: set a higher priority like `WorkPriority::Alarm`.
submit_work_item(work_item, WorkPriority::High);
})?
};
Ok(Self {
timer,
expired_time: None,
})
}
/// Sets a new timeout value. If the old timeout is already set, the timeout will be refreshed.
pub fn set(&mut self, timeout: Duration) -> Result<()> {
assert_ne!(timeout, ZERO_DURATION);
let new_expired_time = {
let now = SystemTime::now();
now.checked_add(timeout)
.ok_or_else(|| Error::with_message(Errno::EINVAL, "Invalid duration"))?
};
self.expired_time = Some(new_expired_time);
self.timer.set(timeout);
Ok(())
}
/// Returns the remaining time until the task is executed. If the `timer` is expired or cleared,
/// this method will return zero.
pub fn remain(&self) -> Duration {
let Some(expired_time) = &self.expired_time else {
return ZERO_DURATION;
};
let now = SystemTime::now();
match expired_time.duration_since(&now) {
Ok(duration) => duration,
Err(_) => ZERO_DURATION,
}
}
/// Returns whether the timer has expired.
pub fn is_expired(&self) -> bool {
self.remain() == ZERO_DURATION
}
/// Clears the timer.
pub fn clear(&mut self) {
self.timer.clear();
self.expired_time = None;
}
}
impl Drop for RealTimer {
fn drop(&mut self) {
self.timer.clear();
}
}
impl PartialEq for RealTimer {
fn eq(&self, other: &Self) -> bool {
self.expired_time == other.expired_time
}
}
impl Eq for RealTimer {}
const ZERO_DURATION: Duration = Duration::new(0, 0);

View File

@ -173,7 +173,7 @@ impl<'a> ProcessBuilder<'a> {
let process = { let process = {
let threads = Vec::new(); let threads = Vec::new();
Arc::new(Process::new( Process::new(
pid, pid,
parent, parent,
threads, threads,
@ -185,7 +185,7 @@ impl<'a> ProcessBuilder<'a> {
sig_dispositions, sig_dispositions,
resource_limits, resource_limits,
nice, nice,
)) )
}; };
let thread = if let Some(thread_builder) = main_thread_builder { let thread = if let Some(thread_builder) = main_thread_builder {

View File

@ -6,7 +6,10 @@ use super::{
process_vm::{Heap, InitStackReader, ProcessVm}, process_vm::{Heap, InitStackReader, ProcessVm},
rlimit::ResourceLimits, rlimit::ResourceLimits,
signal::{ signal::{
constants::SIGCHLD, sig_disposition::SigDispositions, sig_mask::SigMask, signals::Signal, constants::{SIGALRM, SIGCHLD},
sig_disposition::SigDispositions,
sig_mask::SigMask,
signals::{kernel::KernelSignal, Signal},
Pauser, Pauser,
}, },
status::ProcessStatus, status::ProcessStatus,
@ -17,7 +20,12 @@ use crate::{
fs::{file_table::FileTable, fs_resolver::FsResolver, utils::FileCreationMask}, fs::{file_table::FileTable, fs_resolver::FsResolver, utils::FileCreationMask},
prelude::*, prelude::*,
sched::nice::Nice, sched::nice::Nice,
thread::{allocate_tid, Thread}, thread::{
allocate_tid,
work_queue::{submit_work_item, work_item::WorkItem},
Thread,
},
time::{clocks::RealTimeClock, Timer},
vm::vmar::Vmar, vm::vmar::Vmar,
}; };
@ -52,7 +60,8 @@ pub struct Process {
process_vm: ProcessVm, process_vm: ProcessVm,
/// Wait for child status changed /// Wait for child status changed
children_pauser: Arc<Pauser>, children_pauser: Arc<Pauser>,
/// The timer counts down in real (i.e., wall clock) time
alarm_timer: Arc<Timer>,
// Mutable Part // Mutable Part
/// The executable path. /// The executable path.
executable_path: RwLock<String>, executable_path: RwLock<String>,
@ -84,6 +93,26 @@ pub struct Process {
sig_dispositions: Arc<Mutex<SigDispositions>>, sig_dispositions: Arc<Mutex<SigDispositions>>,
} }
fn create_process_timer_callback(process_ref: &Weak<Process>) -> impl Fn() {
let current_process = process_ref.clone();
let sent_signal = move || {
let signal = KernelSignal::new(SIGALRM);
if let Some(process) = current_process.upgrade() {
process.enqueue_signal(signal);
}
};
let work_func = Box::new(sent_signal);
let work_item = Arc::new(WorkItem::new(work_func));
move || {
submit_work_item(
work_item.clone(),
crate::thread::work_queue::WorkPriority::High,
);
}
}
impl Process { impl Process {
#[allow(clippy::too_many_arguments)] #[allow(clippy::too_many_arguments)]
fn new( fn new(
@ -98,7 +127,7 @@ impl Process {
sig_dispositions: Arc<Mutex<SigDispositions>>, sig_dispositions: Arc<Mutex<SigDispositions>>,
resource_limits: ResourceLimits, resource_limits: ResourceLimits,
nice: Nice, nice: Nice,
) -> Self { ) -> Arc<Self> {
let children_pauser = { let children_pauser = {
// SIGCHID does not interrupt pauser. Child process will // SIGCHID does not interrupt pauser. Child process will
// resume paused parent when doing exit. // resume paused parent when doing exit.
@ -106,23 +135,29 @@ impl Process {
Pauser::new_with_mask(sigmask) Pauser::new_with_mask(sigmask)
}; };
Self { Arc::new_cyclic(|process_ref: &Weak<Process>| {
pid, let callback = create_process_timer_callback(process_ref);
threads: Mutex::new(threads), let alarm_timer = RealTimeClock::timer_manager().create_timer(callback);
executable_path: RwLock::new(executable_path),
process_vm, Self {
children_pauser, pid,
status: Mutex::new(ProcessStatus::Uninit), threads: Mutex::new(threads),
parent: Mutex::new(parent), executable_path: RwLock::new(executable_path),
children: Mutex::new(BTreeMap::new()), process_vm,
process_group: Mutex::new(Weak::new()), children_pauser,
file_table, alarm_timer,
fs, status: Mutex::new(ProcessStatus::Uninit),
umask, parent: Mutex::new(parent),
sig_dispositions, children: Mutex::new(BTreeMap::new()),
resource_limits: Mutex::new(resource_limits), process_group: Mutex::new(Weak::new()),
nice: Atomic::new(nice), file_table,
} fs,
umask,
sig_dispositions,
resource_limits: Mutex::new(resource_limits),
nice: Atomic::new(nice),
}
})
} }
/// init a user process and run the process /// init a user process and run the process
@ -198,6 +233,10 @@ impl Process {
self.pid self.pid
} }
pub fn alarm_timer(&self) -> &Arc<Timer> {
&self.alarm_timer
}
pub fn threads(&self) -> &Mutex<Vec<Arc<Thread>>> { pub fn threads(&self) -> &Mutex<Vec<Arc<Thread>>> {
&self.threads &self.threads
} }
@ -597,6 +636,9 @@ pub fn current() -> Arc<Process> {
#[cfg(ktest)] #[cfg(ktest)]
mod test { mod test {
use spin::Once;
use super::*; use super::*;
fn new_process(parent: Option<Arc<Process>>) -> Arc<Process> { fn new_process(parent: Option<Arc<Process>>) -> Arc<Process> {
@ -608,7 +650,7 @@ mod test {
} else { } else {
Weak::new() Weak::new()
}; };
Arc::new(Process::new( Process::new(
pid, pid,
parent, parent,
vec![], vec![],
@ -620,7 +662,7 @@ mod test {
Arc::new(Mutex::new(SigDispositions::default())), Arc::new(Mutex::new(SigDispositions::default())),
ResourceLimits::default(), ResourceLimits::default(),
Nice::default(), Nice::default(),
)) )
} }
fn new_process_in_session(parent: Option<Arc<Process>>) -> Arc<Process> { fn new_process_in_session(parent: Option<Arc<Process>>) -> Arc<Process> {
@ -660,6 +702,7 @@ mod test {
#[ktest] #[ktest]
fn init_process() { fn init_process() {
crate::time::clocks::init_for_ktest();
let process = new_process(None); let process = new_process(None);
assert!(process.process_group().is_none()); assert!(process.process_group().is_none());
assert!(process.session().is_none()); assert!(process.session().is_none());
@ -667,6 +710,7 @@ mod test {
#[ktest] #[ktest]
fn init_process_in_session() { fn init_process_in_session() {
crate::time::clocks::init_for_ktest();
let process = new_process_in_session(None); let process = new_process_in_session(None);
assert!(process.is_group_leader()); assert!(process.is_group_leader());
assert!(process.is_session_leader()); assert!(process.is_session_leader());
@ -675,6 +719,7 @@ mod test {
#[ktest] #[ktest]
fn to_new_session() { fn to_new_session() {
crate::time::clocks::init_for_ktest();
let process = new_process_in_session(None); let process = new_process_in_session(None);
let sess = process.session().unwrap(); let sess = process.session().unwrap();
sess.inner.lock().leader = None; sess.inner.lock().leader = None;

View File

@ -3,26 +3,27 @@
use core::time::Duration; use core::time::Duration;
use super::SyscallReturn; use super::SyscallReturn;
use crate::{prelude::*, process::posix_thread::PosixThreadExt}; use crate::prelude::*;
pub fn sys_alarm(seconds: u32) -> Result<SyscallReturn> { pub fn sys_alarm(seconds: u32) -> Result<SyscallReturn> {
debug!("seconds = {}", seconds); debug!("seconds = {}", seconds);
let current_thread = current_thread!(); let current = current!();
let mut real_timer = { let alarm_timer = current.alarm_timer();
let posix_thread = current_thread.as_posix_thread().unwrap();
posix_thread.real_timer().lock()
};
let remaining_secs = real_timer.remain().as_secs(); let remaining = alarm_timer.remain();
let mut remaining_secs = remaining.as_secs();
if remaining.subsec_nanos() > 0 {
remaining_secs += 1;
}
if seconds == 0 { if seconds == 0 {
// Clear previous timer // Clear previous timer
real_timer.clear(); alarm_timer.cancel();
return Ok(SyscallReturn::Return(remaining_secs as _)); return Ok(SyscallReturn::Return(remaining_secs as _));
} }
real_timer.set(Duration::from_secs(seconds as u64))?; alarm_timer.set_timeout(Duration::from_secs(seconds as u64));
Ok(SyscallReturn::Return(remaining_secs as _)) Ok(SyscallReturn::Return(remaining_secs as _))
} }