mirror of
https://github.com/asterinas/asterinas.git
synced 2025-06-08 21:06:48 +00:00
Rewrite exit()
and exit_group()
This commit is contained in:
parent
5efc32b6ea
commit
35c20620bc
@ -23,7 +23,7 @@ impl CmdlineFileOps {
|
||||
|
||||
impl FileOps for CmdlineFileOps {
|
||||
fn data(&self) -> Result<Vec<u8>> {
|
||||
let cmdline_output = if self.0.is_zombie() {
|
||||
let cmdline_output = if self.0.status().is_zombie() {
|
||||
// Returns 0 characters for zombie process.
|
||||
Vec::new()
|
||||
} else {
|
||||
|
@ -24,7 +24,7 @@ impl FdDirOps {
|
||||
.parent(parent)
|
||||
.build()
|
||||
.unwrap();
|
||||
let main_thread = process_ref.main_thread().unwrap();
|
||||
let main_thread = process_ref.main_thread();
|
||||
let file_table = main_thread.as_posix_thread().unwrap().file_table().lock();
|
||||
let weak_ptr = Arc::downgrade(&fd_inode);
|
||||
file_table.register_observer(weak_ptr);
|
||||
@ -51,7 +51,7 @@ impl DirOps for FdDirOps {
|
||||
let fd = name
|
||||
.parse::<FileDesc>()
|
||||
.map_err(|_| Error::new(Errno::ENOENT))?;
|
||||
let main_thread = self.0.main_thread().unwrap();
|
||||
let main_thread = self.0.main_thread();
|
||||
let file_table = main_thread.as_posix_thread().unwrap().file_table().lock();
|
||||
file_table
|
||||
.get_file(fd)
|
||||
@ -67,7 +67,7 @@ impl DirOps for FdDirOps {
|
||||
this.downcast_ref::<ProcDir<FdDirOps>>().unwrap().this()
|
||||
};
|
||||
let mut cached_children = this.cached_children().write();
|
||||
let main_thread = self.0.main_thread().unwrap();
|
||||
let main_thread = self.0.main_thread();
|
||||
let file_table = main_thread.as_posix_thread().unwrap().file_table().lock();
|
||||
for (fd, file) in file_table.fds_and_files() {
|
||||
cached_children.put_entry_if_not_found(&fd.to_string(), || {
|
||||
|
@ -33,7 +33,7 @@ impl PidDirOps {
|
||||
.volatile()
|
||||
.build()
|
||||
.unwrap();
|
||||
let main_thread = process_ref.main_thread().unwrap();
|
||||
let main_thread = process_ref.main_thread();
|
||||
let file_table = main_thread.as_posix_thread().unwrap().file_table().lock();
|
||||
let weak_ptr = Arc::downgrade(&pid_inode);
|
||||
file_table.register_observer(weak_ptr);
|
||||
|
@ -30,7 +30,7 @@ impl StatFileOps {
|
||||
impl FileOps for StatFileOps {
|
||||
fn data(&self) -> Result<Vec<u8>> {
|
||||
let process = &self.0;
|
||||
let main_thread = process.main_thread().unwrap();
|
||||
let main_thread = process.main_thread();
|
||||
let file_table = main_thread.as_posix_thread().unwrap().file_table();
|
||||
|
||||
let mut stat_output = String::new();
|
||||
@ -43,7 +43,7 @@ impl FileOps for StatFileOps {
|
||||
process.parent().pid(),
|
||||
process.parent().pid(),
|
||||
file_table.lock().len(),
|
||||
process.tasks().lock().len()
|
||||
process.tasks().lock().as_slice().len(),
|
||||
)
|
||||
.unwrap();
|
||||
Ok(stat_output.into_bytes())
|
||||
|
@ -72,7 +72,7 @@ impl StatusFileOps {
|
||||
impl FileOps for StatusFileOps {
|
||||
fn data(&self) -> Result<Vec<u8>> {
|
||||
let process = &self.0;
|
||||
let main_thread = process.main_thread().unwrap();
|
||||
let main_thread = process.main_thread();
|
||||
let file_table = main_thread.as_posix_thread().unwrap().file_table();
|
||||
|
||||
let mut status_output = String::new();
|
||||
@ -82,7 +82,12 @@ impl FileOps for StatusFileOps {
|
||||
writeln!(status_output, "PPid:\t{}", process.parent().pid()).unwrap();
|
||||
writeln!(status_output, "TracerPid:\t{}", process.parent().pid()).unwrap(); // Assuming TracerPid is the same as PPid
|
||||
writeln!(status_output, "FDSize:\t{}", file_table.lock().len()).unwrap();
|
||||
writeln!(status_output, "Threads:\t{}", process.tasks().lock().len()).unwrap();
|
||||
writeln!(
|
||||
status_output,
|
||||
"Threads:\t{}",
|
||||
process.tasks().lock().as_slice().len()
|
||||
)
|
||||
.unwrap();
|
||||
Ok(status_output.into_bytes())
|
||||
}
|
||||
}
|
||||
|
@ -63,7 +63,7 @@ impl DirOps for ThreadDirOps {
|
||||
|
||||
impl DirOps for TaskDirOps {
|
||||
fn lookup_child(&self, this_ptr: Weak<dyn Inode>, name: &str) -> Result<Arc<dyn Inode>> {
|
||||
for task in self.0.tasks().lock().iter() {
|
||||
for task in self.0.tasks().lock().as_slice() {
|
||||
if task.as_posix_thread().unwrap().tid() != name.parse::<u32>().unwrap() {
|
||||
continue;
|
||||
}
|
||||
@ -78,7 +78,7 @@ impl DirOps for TaskDirOps {
|
||||
this.downcast_ref::<ProcDir<TaskDirOps>>().unwrap().this()
|
||||
};
|
||||
let mut cached_children = this.cached_children().write();
|
||||
for task in self.0.tasks().lock().iter() {
|
||||
for task in self.0.tasks().lock().as_slice() {
|
||||
cached_children.put_entry_if_not_found(
|
||||
&format!("{}", task.as_posix_thread().unwrap().tid()),
|
||||
|| ThreadDirOps::new_inode(self.0.clone(), this_ptr.clone()),
|
||||
|
@ -150,14 +150,14 @@ fn init_thread() {
|
||||
)
|
||||
.expect("Run init process failed.");
|
||||
// Wait till initproc become zombie.
|
||||
while !initproc.is_zombie() {
|
||||
while !initproc.status().is_zombie() {
|
||||
// We don't have preemptive scheduler now.
|
||||
// The long running init thread should yield its own execution to allow other tasks to go on.
|
||||
Thread::yield_now();
|
||||
}
|
||||
|
||||
// TODO: exit via qemu isa debug device should not be the only way.
|
||||
let exit_code = if initproc.exit_code() == 0 {
|
||||
let exit_code = if initproc.status().exit_code() == 0 {
|
||||
QemuExitCode::Success
|
||||
} else {
|
||||
QemuExitCode::Failed
|
||||
|
@ -260,7 +260,11 @@ fn clone_child_task(
|
||||
thread_builder.build()
|
||||
};
|
||||
|
||||
process.tasks().lock().push(child_task.clone());
|
||||
process
|
||||
.tasks()
|
||||
.lock()
|
||||
.insert(child_task.clone())
|
||||
.map_err(|_| Error::with_message(Errno::EINTR, "the process has exited"))?;
|
||||
|
||||
let child_posix_thread = child_task.as_posix_thread().unwrap();
|
||||
clone_parent_settid(child_tid, clone_args.parent_tid, clone_flags)?;
|
||||
|
@ -1,75 +1,74 @@
|
||||
// SPDX-License-Identifier: MPL-2.0
|
||||
|
||||
use super::{process_table, Pid, Process, TermStatus};
|
||||
use crate::{
|
||||
prelude::*,
|
||||
process::{
|
||||
posix_thread::{do_exit, AsPosixThread},
|
||||
signal::signals::kernel::KernelSignal,
|
||||
},
|
||||
thread::AsThread,
|
||||
};
|
||||
use super::{posix_thread::PosixThread, process_table, Pid, Process};
|
||||
use crate::{prelude::*, process::signal::signals::kernel::KernelSignal};
|
||||
|
||||
pub fn do_exit_group(term_status: TermStatus) {
|
||||
let current = current!();
|
||||
debug!("exit group was called");
|
||||
if current.is_zombie() {
|
||||
return;
|
||||
}
|
||||
current.set_zombie(term_status);
|
||||
/// Exits the current POSIX process.
|
||||
///
|
||||
/// This is for internal use. Do NOT call this directly. When the last thread in the process exits,
|
||||
/// [`do_exit`] or [`do_exit_group`] will invoke this method automatically.
|
||||
///
|
||||
/// [`do_exit`]: crate::process::posix_thread::do_exit
|
||||
/// [`do_exit_group`]: crate::process::posix_thread::do_exit_group
|
||||
pub(super) fn exit_process(current_thread: &PosixThread, current_process: &Process) {
|
||||
current_process.status().set_zombie();
|
||||
|
||||
// Exit all threads
|
||||
let tasks = current.tasks().lock().clone();
|
||||
for task in tasks {
|
||||
let thread = task.as_thread().unwrap();
|
||||
let posix_thread = thread.as_posix_thread().unwrap();
|
||||
if let Err(e) = do_exit(thread, posix_thread, term_status) {
|
||||
debug!("Ignore error when call exit: {:?}", e);
|
||||
}
|
||||
}
|
||||
// FIXME: This is obviously wrong in a number of ways, since different threads can have
|
||||
// different file tables, and different processes can share the same file table.
|
||||
current_thread.file_table().lock().close_all();
|
||||
|
||||
// Sends parent-death signal
|
||||
// FIXME: according to linux spec, the signal should be sent when a posix thread which
|
||||
// creates child process exits, not when the whole process exits group.
|
||||
for (_, child) in current.children().lock().iter() {
|
||||
send_parent_death_signal(current_process);
|
||||
|
||||
move_children_to_init(current_process);
|
||||
|
||||
send_child_death_signal(current_process);
|
||||
}
|
||||
|
||||
/// Sends parent-death signals to the children.
|
||||
//
|
||||
// FIXME: According to the Linux implementation, the signal should be sent when the POSIX thread
|
||||
// that created the child exits, not when the whole process exits. For more details, see the
|
||||
// "CAVEATS" section in <https://man7.org/linux/man-pages/man2/pr_set_pdeathsig.2const.html>.
|
||||
fn send_parent_death_signal(current_process: &Process) {
|
||||
for (_, child) in current_process.children().lock().iter() {
|
||||
let Some(signum) = child.parent_death_signal() else {
|
||||
continue;
|
||||
};
|
||||
|
||||
// FIXME: set pid of the signal
|
||||
// FIXME: Set `si_pid` in the `siginfo_t` argument.
|
||||
let signal = KernelSignal::new(signum);
|
||||
child.enqueue_signal(signal);
|
||||
}
|
||||
}
|
||||
|
||||
// Close all files then exit the process.
|
||||
//
|
||||
// FIXME: This is obviously wrong in a number of ways, since different threads can have
|
||||
// different file tables, and different processes can share the same file table.
|
||||
let main_thread = current.main_thread().unwrap();
|
||||
let mut files = main_thread.as_posix_thread().unwrap().file_table().lock();
|
||||
files.close_all();
|
||||
drop(files);
|
||||
|
||||
// Move children to the init process
|
||||
if !is_init_process(¤t) {
|
||||
if let Some(init_process) = get_init_process() {
|
||||
let mut init_children = init_process.children().lock();
|
||||
for (_, child_process) in current.children().lock().extract_if(|_, _| true) {
|
||||
let mut parent = child_process.parent.lock();
|
||||
init_children.insert(child_process.pid(), child_process.clone());
|
||||
parent.set_process(&init_process);
|
||||
}
|
||||
}
|
||||
/// Moves the children to the init process.
|
||||
fn move_children_to_init(current_process: &Process) {
|
||||
if is_init_process(current_process) {
|
||||
return;
|
||||
}
|
||||
|
||||
let parent = current.parent().lock().process();
|
||||
if let Some(parent) = parent.upgrade() {
|
||||
// Notify parent
|
||||
if let Some(signal) = current.exit_signal().map(KernelSignal::new) {
|
||||
parent.enqueue_signal(signal);
|
||||
};
|
||||
parent.children_wait_queue().wake_all();
|
||||
let Some(init_process) = get_init_process() else {
|
||||
return;
|
||||
};
|
||||
|
||||
let mut init_children = init_process.children().lock();
|
||||
for (_, child_process) in current_process.children().lock().extract_if(|_, _| true) {
|
||||
let mut parent = child_process.parent.lock();
|
||||
init_children.insert(child_process.pid(), child_process.clone());
|
||||
parent.set_process(&init_process);
|
||||
}
|
||||
}
|
||||
|
||||
/// Sends a child-death signal to the parent.
|
||||
fn send_child_death_signal(current_process: &Process) {
|
||||
let Some(parent) = current_process.parent().lock().process().upgrade() else {
|
||||
return;
|
||||
};
|
||||
|
||||
if let Some(signal) = current_process.exit_signal().map(KernelSignal::new) {
|
||||
parent.enqueue_signal(signal);
|
||||
};
|
||||
parent.children_wait_queue().wake_all();
|
||||
}
|
||||
|
||||
const INIT_PROCESS_PID: Pid = 1;
|
||||
|
@ -123,7 +123,7 @@ fn kill_process(process: &Process, signal: Option<UserSignal>, ctx: &Context) ->
|
||||
let sender_ids = current_thread_sender_ids(signum.as_ref(), ctx);
|
||||
|
||||
let mut permitted_thread = None;
|
||||
for task in tasks.iter() {
|
||||
for task in tasks.as_slice() {
|
||||
let posix_thread = task.as_posix_thread().unwrap();
|
||||
|
||||
// First check permission
|
||||
|
@ -15,12 +15,12 @@ pub mod rlimit;
|
||||
pub mod signal;
|
||||
mod status;
|
||||
pub mod sync;
|
||||
mod task_set;
|
||||
mod term_status;
|
||||
mod wait;
|
||||
|
||||
pub use clone::{clone_child, CloneArgs, CloneFlags};
|
||||
pub use credentials::{Credentials, Gid, Uid};
|
||||
pub use exit::do_exit_group;
|
||||
pub use kill::{kill, kill_all, kill_group, tgkill};
|
||||
pub use process::{
|
||||
ExitCode, JobControl, Pgid, Pid, Process, ProcessBuilder, ProcessGroup, Session, Sid, Terminal,
|
||||
|
@ -1,66 +1,134 @@
|
||||
// SPDX-License-Identifier: MPL-2.0
|
||||
|
||||
use super::{futex::futex_wake, robust_list::wake_robust_futex, thread_table, PosixThread};
|
||||
use ostd::task::{CurrentTask, Task};
|
||||
|
||||
use super::{
|
||||
futex::futex_wake, robust_list::wake_robust_futex, thread_table, AsPosixThread, PosixThread,
|
||||
};
|
||||
use crate::{
|
||||
current_userspace,
|
||||
prelude::*,
|
||||
process::{do_exit_group, TermStatus},
|
||||
thread::{Thread, Tid},
|
||||
process::{
|
||||
exit::exit_process,
|
||||
signal::{constants::SIGKILL, signals::kernel::KernelSignal},
|
||||
task_set::TaskSet,
|
||||
TermStatus,
|
||||
},
|
||||
thread::AsThread,
|
||||
};
|
||||
|
||||
/// Exits the thread if the thread is a POSIX thread.
|
||||
/// Exits the current POSIX thread.
|
||||
///
|
||||
/// # Panics
|
||||
///
|
||||
/// If the thread is not a POSIX thread, this method will panic.
|
||||
pub fn do_exit(thread: &Thread, posix_thread: &PosixThread, term_status: TermStatus) -> Result<()> {
|
||||
if thread.is_exited() {
|
||||
return Ok(());
|
||||
}
|
||||
thread.exit();
|
||||
|
||||
let tid = posix_thread.tid;
|
||||
|
||||
let mut clear_ctid = posix_thread.clear_child_tid().lock();
|
||||
// If clear_ctid !=0 ,do a futex wake and write zero to the clear_ctid addr.
|
||||
if *clear_ctid != 0 {
|
||||
// FIXME: the correct write length?
|
||||
if let Err(e) = current_userspace!().write_val(*clear_ctid, &0u32) {
|
||||
debug!("Ignore error during exit process: {:?}", e);
|
||||
}
|
||||
futex_wake(*clear_ctid, 1, None)?;
|
||||
*clear_ctid = 0;
|
||||
}
|
||||
drop(clear_ctid);
|
||||
// exit the robust list: walk the robust list; mark futex words as dead and do futex wake
|
||||
wake_robust_list(posix_thread, tid);
|
||||
|
||||
if tid != posix_thread.process().pid() {
|
||||
// We don't remove main thread.
|
||||
// The main thread is removed when the process is reaped.
|
||||
thread_table::remove_thread(tid);
|
||||
}
|
||||
|
||||
if posix_thread.is_main_thread(tid) || posix_thread.is_last_thread() {
|
||||
// exit current process.
|
||||
do_exit_group(term_status);
|
||||
}
|
||||
|
||||
futex_wake(Arc::as_ptr(&posix_thread.process()) as Vaddr, 1, None)?;
|
||||
Ok(())
|
||||
/// If the current thread is not a POSIX thread, this method will panic.
|
||||
pub fn do_exit(term_status: TermStatus) {
|
||||
exit_internal(term_status, false);
|
||||
}
|
||||
|
||||
/// Walks the robust futex list, marking futex dead and wake waiters.
|
||||
/// It corresponds to Linux's exit_robust_list(), errors are silently ignored.
|
||||
fn wake_robust_list(thread: &PosixThread, tid: Tid) {
|
||||
let mut robust_list = thread.robust_list.lock();
|
||||
/// Kills all threads and exits the current POSIX process.
|
||||
///
|
||||
/// # Panics
|
||||
///
|
||||
/// If the current thread is not a POSIX thread, this method will panic.
|
||||
pub fn do_exit_group(term_status: TermStatus) {
|
||||
exit_internal(term_status, true);
|
||||
}
|
||||
|
||||
/// Exits the current POSIX thread or process.
|
||||
fn exit_internal(term_status: TermStatus, is_exiting_group: bool) {
|
||||
let current_task = Task::current().unwrap();
|
||||
let current_thread = current_task.as_thread().unwrap();
|
||||
let posix_thread = current_thread.as_posix_thread().unwrap();
|
||||
let posix_process = posix_thread.process();
|
||||
|
||||
let is_last_thread = {
|
||||
let mut tasks = posix_process.tasks().lock();
|
||||
let has_exited_group = tasks.has_exited_group();
|
||||
|
||||
if is_exiting_group && !has_exited_group {
|
||||
sigkill_other_threads(¤t_task, &tasks);
|
||||
tasks.set_exited_group();
|
||||
}
|
||||
|
||||
// According to Linux's behavior, the last thread's exit code will become the process's
|
||||
// exit code, so here we should just overwrite the old value (if any).
|
||||
if !has_exited_group {
|
||||
posix_process.status().set_exit_code(term_status.as_u32());
|
||||
}
|
||||
|
||||
// We should only change the thread status when running as the thread, so no race
|
||||
// conditions can occur in between.
|
||||
if current_thread.is_exited() {
|
||||
return;
|
||||
}
|
||||
current_thread.exit();
|
||||
|
||||
tasks.remove_exited(¤t_task)
|
||||
};
|
||||
|
||||
wake_clear_ctid(posix_thread);
|
||||
|
||||
wake_robust_list(posix_thread);
|
||||
|
||||
// According to Linux behavior, the main thread shouldn't be removed from the table until the
|
||||
// process is reaped by its parent.
|
||||
if posix_thread.tid() != posix_process.pid() {
|
||||
thread_table::remove_thread(posix_thread.tid());
|
||||
}
|
||||
|
||||
if is_last_thread {
|
||||
exit_process(posix_thread, &posix_process);
|
||||
}
|
||||
}
|
||||
|
||||
/// Sends `SIGKILL` to all other threads in the current process.
|
||||
///
|
||||
/// This is only needed when initiating an `exit_group` for the first time.
|
||||
fn sigkill_other_threads(current_task: &CurrentTask, task_set: &TaskSet) {
|
||||
for task in task_set.as_slice() {
|
||||
if core::ptr::eq(current_task.as_ref(), task.as_ref()) {
|
||||
continue;
|
||||
}
|
||||
task.as_posix_thread()
|
||||
.unwrap()
|
||||
.enqueue_signal(Box::new(KernelSignal::new(SIGKILL)));
|
||||
}
|
||||
}
|
||||
|
||||
/// Writes zero to `clear_child_tid` and performs a futex wake.
|
||||
fn wake_clear_ctid(current_thread: &PosixThread) {
|
||||
let mut clear_ctid = current_thread.clear_child_tid().lock();
|
||||
|
||||
if *clear_ctid == 0 {
|
||||
return;
|
||||
}
|
||||
|
||||
let _ = current_userspace!()
|
||||
.write_val(*clear_ctid, &0u32)
|
||||
.inspect_err(|err| debug!("exit: cannot clear the child TID: {:?}", err));
|
||||
let _ = futex_wake(*clear_ctid, 1, None)
|
||||
.inspect_err(|err| debug!("exit: cannot wake the futex on the child TID: {:?}", err));
|
||||
|
||||
*clear_ctid = 0;
|
||||
}
|
||||
|
||||
/// Walks the robust futex list, marking futex dead and waking waiters.
|
||||
///
|
||||
/// This corresponds to Linux's `exit_robust_list`. Errors are silently ignored.
|
||||
fn wake_robust_list(current_thread: &PosixThread) {
|
||||
let mut robust_list = current_thread.robust_list.lock();
|
||||
|
||||
let list_head = match *robust_list {
|
||||
Some(robust_list_head) => robust_list_head,
|
||||
None => return,
|
||||
};
|
||||
trace!("wake the rubust_list: {:?}", list_head);
|
||||
|
||||
trace!("exit: wake up the rubust list: {:?}", list_head);
|
||||
for futex_addr in list_head.futexes() {
|
||||
wake_robust_futex(futex_addr, tid).unwrap();
|
||||
let _ = wake_robust_futex(futex_addr, current_thread.tid)
|
||||
.inspect_err(|err| debug!("exit: cannot wake up the robust futex: {:?}", err));
|
||||
}
|
||||
|
||||
*robust_list = None;
|
||||
}
|
||||
|
@ -24,7 +24,7 @@ use crate::{
|
||||
fs::{file_table::FileTable, thread_info::ThreadFsInfo},
|
||||
prelude::*,
|
||||
process::signal::constants::SIGCONT,
|
||||
thread::{AsThread, Thread, Tid},
|
||||
thread::{Thread, Tid},
|
||||
time::{clocks::ProfClock, Timer, TimerManager},
|
||||
};
|
||||
|
||||
@ -37,7 +37,7 @@ mod robust_list;
|
||||
pub mod thread_table;
|
||||
|
||||
pub use builder::PosixThreadBuilder;
|
||||
pub use exit::do_exit;
|
||||
pub use exit::{do_exit, do_exit_group};
|
||||
pub use name::{ThreadName, MAX_THREAD_NAME_LEN};
|
||||
pub use posix_thread_ext::{create_posix_task_from_executable, AsPosixThread};
|
||||
pub use robust_list::RobustListHead;
|
||||
@ -285,20 +285,6 @@ impl PosixThread {
|
||||
&self.robust_list
|
||||
}
|
||||
|
||||
fn is_main_thread(&self, tid: Tid) -> bool {
|
||||
let process = self.process();
|
||||
let pid = process.pid();
|
||||
tid == pid
|
||||
}
|
||||
|
||||
fn is_last_thread(&self) -> bool {
|
||||
let process = self.process.upgrade().unwrap();
|
||||
let tasks = process.tasks().lock();
|
||||
tasks
|
||||
.iter()
|
||||
.all(|task| task.as_thread().unwrap().is_exited())
|
||||
}
|
||||
|
||||
/// Gets the read-only credentials of the thread.
|
||||
pub fn credentials(&self) -> Credentials<ReadOp> {
|
||||
self.credentials.dup().restrict()
|
||||
|
@ -135,19 +135,15 @@ impl<'a> ProcessBuilder<'a> {
|
||||
|
||||
let nice = nice.or_else(|| Some(Nice::default())).unwrap();
|
||||
|
||||
let process = {
|
||||
let threads = Vec::new();
|
||||
Process::new(
|
||||
pid,
|
||||
parent,
|
||||
threads,
|
||||
executable_path.to_string(),
|
||||
process_vm,
|
||||
resource_limits,
|
||||
nice,
|
||||
sig_dispositions,
|
||||
)
|
||||
};
|
||||
let process = Process::new(
|
||||
pid,
|
||||
parent,
|
||||
executable_path.to_string(),
|
||||
process_vm,
|
||||
resource_limits,
|
||||
nice,
|
||||
sig_dispositions,
|
||||
);
|
||||
|
||||
let task = if let Some(thread_builder) = main_thread_builder {
|
||||
let builder = thread_builder.process(Arc::downgrade(&process));
|
||||
@ -164,9 +160,7 @@ impl<'a> ProcessBuilder<'a> {
|
||||
)?
|
||||
};
|
||||
|
||||
process.tasks().lock().push(task);
|
||||
|
||||
process.set_runnable();
|
||||
process.tasks().lock().insert(task).unwrap();
|
||||
|
||||
Ok(process)
|
||||
}
|
||||
|
@ -14,7 +14,8 @@ use super::{
|
||||
signals::Signal,
|
||||
},
|
||||
status::ProcessStatus,
|
||||
Credentials, TermStatus,
|
||||
task_set::TaskSet,
|
||||
Credentials,
|
||||
};
|
||||
use crate::{
|
||||
device::tty::open_ntty_as_controlling_terminal,
|
||||
@ -71,7 +72,7 @@ pub struct Process {
|
||||
/// The executable path.
|
||||
executable_path: RwLock<String>,
|
||||
/// The threads
|
||||
tasks: Mutex<Vec<Arc<Task>>>,
|
||||
tasks: Mutex<TaskSet>,
|
||||
/// Process status
|
||||
status: ProcessStatus,
|
||||
/// Parent process
|
||||
@ -174,7 +175,6 @@ impl Process {
|
||||
fn new(
|
||||
pid: Pid,
|
||||
parent: Weak<Process>,
|
||||
tasks: Vec<Arc<Task>>,
|
||||
executable_path: String,
|
||||
process_vm: ProcessVm,
|
||||
|
||||
@ -190,11 +190,11 @@ impl Process {
|
||||
|
||||
Arc::new_cyclic(|process_ref: &Weak<Process>| Self {
|
||||
pid,
|
||||
tasks: Mutex::new(tasks),
|
||||
tasks: Mutex::new(TaskSet::new()),
|
||||
executable_path: RwLock::new(executable_path),
|
||||
process_vm,
|
||||
children_wait_queue,
|
||||
status: ProcessStatus::new_uninit(),
|
||||
status: ProcessStatus::default(),
|
||||
parent: ParentProcess::new(parent),
|
||||
children: Mutex::new(BTreeMap::new()),
|
||||
process_group: Mutex::new(Weak::new()),
|
||||
@ -267,9 +267,9 @@ impl Process {
|
||||
pub fn run(&self) {
|
||||
let tasks = self.tasks.lock();
|
||||
// when run the process, the process should has only one thread
|
||||
debug_assert!(tasks.len() == 1);
|
||||
debug_assert!(self.is_runnable());
|
||||
let task = tasks[0].clone();
|
||||
debug_assert!(tasks.as_slice().len() == 1);
|
||||
debug_assert!(!self.status().is_zombie());
|
||||
let task = tasks.main().clone();
|
||||
// should not hold the lock when run thread
|
||||
drop(tasks);
|
||||
let thread = task.as_thread().unwrap();
|
||||
@ -292,7 +292,7 @@ impl Process {
|
||||
&self.timer_manager
|
||||
}
|
||||
|
||||
pub fn tasks(&self) -> &Mutex<Vec<Arc<Task>>> {
|
||||
pub fn tasks(&self) -> &Mutex<TaskSet> {
|
||||
&self.tasks
|
||||
}
|
||||
|
||||
@ -312,15 +312,8 @@ impl Process {
|
||||
&self.nice
|
||||
}
|
||||
|
||||
pub fn main_thread(&self) -> Option<Arc<Thread>> {
|
||||
self.tasks
|
||||
.lock()
|
||||
.iter()
|
||||
.find_map(|task| {
|
||||
let thread = task.as_thread().unwrap();
|
||||
(thread.as_posix_thread().unwrap().tid() == self.pid).then_some(thread)
|
||||
})
|
||||
.cloned()
|
||||
pub fn main_thread(&self) -> Arc<Thread> {
|
||||
self.tasks.lock().main().as_thread().unwrap().clone()
|
||||
}
|
||||
|
||||
// *********** Parent and child ***********
|
||||
@ -621,7 +614,7 @@ impl Process {
|
||||
///
|
||||
/// TODO: restrict these method with access control tool.
|
||||
pub fn enqueue_signal(&self, signal: impl Signal + Clone + 'static) {
|
||||
if self.is_zombie() {
|
||||
if self.status.is_zombie() {
|
||||
return;
|
||||
}
|
||||
|
||||
@ -629,7 +622,7 @@ impl Process {
|
||||
|
||||
// Enqueue signal to the first thread that does not block the signal
|
||||
let threads = self.tasks.lock();
|
||||
for thread in threads.iter() {
|
||||
for thread in threads.as_slice() {
|
||||
let posix_thread = thread.as_posix_thread().unwrap();
|
||||
if !posix_thread.has_signal_blocked(signal.num()) {
|
||||
posix_thread.enqueue_signal(Box::new(signal));
|
||||
@ -637,8 +630,8 @@ impl Process {
|
||||
}
|
||||
}
|
||||
|
||||
// If all threads block the signal, enqueue signal to the first thread
|
||||
let thread = threads.iter().next().unwrap();
|
||||
// If all threads block the signal, enqueue signal to the main thread
|
||||
let thread = threads.main();
|
||||
let posix_thread = thread.as_posix_thread().unwrap();
|
||||
posix_thread.enqueue_signal(Box::new(signal));
|
||||
}
|
||||
@ -671,24 +664,9 @@ impl Process {
|
||||
|
||||
// ******************* Status ********************
|
||||
|
||||
fn set_runnable(&self) {
|
||||
self.status.set_runnable();
|
||||
}
|
||||
|
||||
fn is_runnable(&self) -> bool {
|
||||
self.status.is_runnable()
|
||||
}
|
||||
|
||||
pub fn is_zombie(&self) -> bool {
|
||||
self.status.is_zombie()
|
||||
}
|
||||
|
||||
pub fn set_zombie(&self, term_status: TermStatus) {
|
||||
self.status.set_zombie(term_status);
|
||||
}
|
||||
|
||||
pub fn exit_code(&self) -> ExitCode {
|
||||
self.status.exit_code()
|
||||
/// Returns a reference to the process status.
|
||||
pub fn status(&self) -> &ProcessStatus {
|
||||
&self.status
|
||||
}
|
||||
}
|
||||
|
||||
@ -711,7 +689,6 @@ mod test {
|
||||
Process::new(
|
||||
pid,
|
||||
parent,
|
||||
vec![],
|
||||
String::new(),
|
||||
ProcessVm::alloc(),
|
||||
ResourceLimits::default(),
|
||||
|
@ -19,7 +19,7 @@ use crate::{
|
||||
},
|
||||
prelude::*,
|
||||
process::{
|
||||
do_exit_group,
|
||||
posix_thread::do_exit_group,
|
||||
process_vm::{AuxKey, AuxVec, ProcessVm},
|
||||
TermStatus,
|
||||
},
|
||||
|
@ -31,7 +31,7 @@ use crate::{
|
||||
cpu::LinuxAbi,
|
||||
current_userspace,
|
||||
prelude::*,
|
||||
process::{do_exit_group, TermStatus},
|
||||
process::{posix_thread::do_exit_group, TermStatus},
|
||||
};
|
||||
|
||||
pub trait SignalContext {
|
||||
|
@ -1,59 +1,57 @@
|
||||
// SPDX-License-Identifier: MPL-2.0
|
||||
|
||||
#![allow(dead_code)]
|
||||
//! The process status.
|
||||
|
||||
//! The process status
|
||||
use core::sync::atomic::{AtomicBool, AtomicU32, Ordering};
|
||||
|
||||
use core::sync::atomic::{AtomicU64, Ordering};
|
||||
use super::ExitCode;
|
||||
|
||||
use super::{ExitCode, TermStatus};
|
||||
|
||||
/// The status of process.
|
||||
/// The status of a process.
|
||||
///
|
||||
/// The `ProcessStatus` can be viewed as two parts,
|
||||
/// the highest 32 bits is the value of `TermStatus`, if any,
|
||||
/// the lowest 32 bits is the value of status.
|
||||
/// This maintains:
|
||||
/// 1. Whether the process is a zombie (i.e., all its threads have exited);
|
||||
/// 2. The exit code of the process.
|
||||
#[derive(Debug)]
|
||||
pub struct ProcessStatus(AtomicU64);
|
||||
pub struct ProcessStatus {
|
||||
is_zombie: AtomicBool,
|
||||
exit_code: AtomicU32,
|
||||
}
|
||||
|
||||
#[repr(u8)]
|
||||
enum Status {
|
||||
Uninit = 0,
|
||||
Runnable = 1,
|
||||
Zombie = 2,
|
||||
impl Default for ProcessStatus {
|
||||
fn default() -> Self {
|
||||
Self {
|
||||
is_zombie: AtomicBool::new(false),
|
||||
exit_code: AtomicU32::new(0),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl ProcessStatus {
|
||||
const LOW_MASK: u64 = 0xffff_ffff;
|
||||
|
||||
pub fn new_uninit() -> Self {
|
||||
Self(AtomicU64::new(Status::Uninit as u64))
|
||||
}
|
||||
|
||||
pub fn set_zombie(&self, term_status: TermStatus) {
|
||||
let new_val = (term_status.as_u32() as u64) << 32 | Status::Zombie as u64;
|
||||
self.0.store(new_val, Ordering::Relaxed);
|
||||
}
|
||||
|
||||
/// Returns whether the process is a zombie process.
|
||||
pub fn is_zombie(&self) -> bool {
|
||||
self.0.load(Ordering::Relaxed) & Self::LOW_MASK == Status::Zombie as u64
|
||||
// Use the `Acquire` memory order to make the exit code visible.
|
||||
self.is_zombie.load(Ordering::Acquire)
|
||||
}
|
||||
|
||||
pub fn set_runnable(&self) {
|
||||
let new_val = Status::Runnable as u64;
|
||||
self.0.store(new_val, Ordering::Relaxed);
|
||||
}
|
||||
|
||||
pub fn is_runnable(&self) -> bool {
|
||||
self.0.load(Ordering::Relaxed) & Self::LOW_MASK == Status::Runnable as u64
|
||||
}
|
||||
|
||||
/// Returns the exit code.
|
||||
/// Sets the process to be a zombie process.
|
||||
///
|
||||
/// If the process is not exited, the exit code is zero.
|
||||
/// But if exit code is zero, the process may or may not exit.
|
||||
pub fn exit_code(&self) -> ExitCode {
|
||||
let val = self.0.load(Ordering::Relaxed);
|
||||
(val >> 32 & Self::LOW_MASK) as ExitCode
|
||||
/// This method should be called when the process completes its exit. The current thread must
|
||||
/// be the last thread in the process, so that no threads belonging to the process can run
|
||||
/// after it.
|
||||
pub(super) fn set_zombie(&self) {
|
||||
// Use the `Release` memory order to make the exit code visible.
|
||||
self.is_zombie.store(true, Ordering::Release);
|
||||
}
|
||||
}
|
||||
|
||||
impl ProcessStatus {
|
||||
/// Returns the exit code.
|
||||
pub fn exit_code(&self) -> ExitCode {
|
||||
self.exit_code.load(Ordering::Relaxed)
|
||||
}
|
||||
|
||||
/// Sets the exit code.
|
||||
pub(super) fn set_exit_code(&self, exit_code: ExitCode) {
|
||||
self.exit_code.store(exit_code, Ordering::Relaxed);
|
||||
}
|
||||
}
|
||||
|
86
kernel/src/process/task_set.rs
Normal file
86
kernel/src/process/task_set.rs
Normal file
@ -0,0 +1,86 @@
|
||||
// SPDX-License-Identifier: MPL-2.0
|
||||
|
||||
//! Task sets.
|
||||
|
||||
use ostd::task::{CurrentTask, Task};
|
||||
|
||||
use crate::prelude::*;
|
||||
|
||||
/// A task set that maintains all tasks in a POSIX process.
|
||||
pub struct TaskSet {
|
||||
tasks: Vec<Arc<Task>>,
|
||||
has_exited_main: bool,
|
||||
has_exited_group: bool,
|
||||
}
|
||||
|
||||
impl TaskSet {
|
||||
/// Creates a new task set.
|
||||
pub(super) fn new() -> Self {
|
||||
Self {
|
||||
tasks: Vec::new(),
|
||||
has_exited_main: false,
|
||||
has_exited_group: false,
|
||||
}
|
||||
}
|
||||
|
||||
/// Inserts a new task to the task set.
|
||||
///
|
||||
/// This method will fail if [`Self::set_exited_group`] has been called before.
|
||||
pub(super) fn insert(&mut self, task: Arc<Task>) -> core::result::Result<(), Arc<Task>> {
|
||||
if self.has_exited_group {
|
||||
return Err(task);
|
||||
}
|
||||
|
||||
self.tasks.push(task);
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Removes the exited task from the task set if necessary.
|
||||
///
|
||||
/// The task will be removed from the task set if the corresponding thread is not the main
|
||||
/// thread.
|
||||
///
|
||||
/// This method will return true if there are no more alive tasks in the task set.
|
||||
///
|
||||
/// # Panics
|
||||
///
|
||||
/// This method will panic if the task is not in the task set.
|
||||
pub(super) fn remove_exited(&mut self, task: &CurrentTask) -> bool {
|
||||
let position = self
|
||||
.tasks
|
||||
.iter()
|
||||
.position(|some_task| core::ptr::eq(some_task.as_ref(), task.as_ref()))
|
||||
.unwrap();
|
||||
|
||||
if position == 0 {
|
||||
assert!(!self.has_exited_main);
|
||||
self.has_exited_main = true;
|
||||
} else {
|
||||
self.tasks.swap_remove(position);
|
||||
}
|
||||
|
||||
self.has_exited_main && self.tasks.len() == 1
|
||||
}
|
||||
|
||||
/// Sets a flag that denotes that an `exit_group` has been initiated.
|
||||
pub(super) fn set_exited_group(&mut self) {
|
||||
self.has_exited_group = true;
|
||||
}
|
||||
|
||||
/// Returns whether an `exit_group` has been initiated.
|
||||
pub(super) fn has_exited_group(&self) -> bool {
|
||||
self.has_exited_group
|
||||
}
|
||||
}
|
||||
|
||||
impl TaskSet {
|
||||
/// Returns a slice of the tasks in the task set.
|
||||
pub fn as_slice(&self) -> &[Arc<Task>] {
|
||||
self.tasks.as_slice()
|
||||
}
|
||||
|
||||
/// Returns the main task/thread.
|
||||
pub fn main(&self) -> &Arc<Task> {
|
||||
&self.tasks[0]
|
||||
}
|
||||
}
|
@ -62,7 +62,9 @@ pub fn wait_child_exit(
|
||||
}
|
||||
|
||||
// return immediately if we find a zombie child
|
||||
let zombie_child = unwaited_children.iter().find(|child| child.is_zombie());
|
||||
let zombie_child = unwaited_children
|
||||
.iter()
|
||||
.find(|child| child.status().is_zombie());
|
||||
|
||||
if let Some(zombie_child) = zombie_child {
|
||||
let zombie_pid = zombie_child.pid();
|
||||
@ -90,8 +92,8 @@ pub fn wait_child_exit(
|
||||
/// Free zombie child with pid, returns the exit code of child process.
|
||||
fn reap_zombie_child(process: &Process, pid: Pid) -> ExitCode {
|
||||
let child_process = process.children().lock().remove(&pid).unwrap();
|
||||
assert!(child_process.is_zombie());
|
||||
for task in &*child_process.tasks().lock() {
|
||||
assert!(child_process.status().is_zombie());
|
||||
for task in child_process.tasks().lock().as_slice() {
|
||||
thread_table::remove_thread(task.as_posix_thread().unwrap().tid());
|
||||
}
|
||||
|
||||
@ -122,5 +124,5 @@ fn reap_zombie_child(process: &Process, pid: Pid) -> ExitCode {
|
||||
}
|
||||
|
||||
process_table_mut.remove(child_process.pid());
|
||||
child_process.exit_code()
|
||||
child_process.status().exit_code()
|
||||
}
|
||||
|
@ -6,11 +6,11 @@ use crate::{
|
||||
syscall::SyscallReturn,
|
||||
};
|
||||
|
||||
pub fn sys_exit(exit_code: i32, ctx: &Context) -> Result<SyscallReturn> {
|
||||
pub fn sys_exit(exit_code: i32, _ctx: &Context) -> Result<SyscallReturn> {
|
||||
debug!("exid code = {}", exit_code);
|
||||
|
||||
let term_status = TermStatus::Exited(exit_code as _);
|
||||
do_exit(ctx.thread, ctx.posix_thread, term_status)?;
|
||||
do_exit(term_status);
|
||||
|
||||
Ok(SyscallReturn::Return(0))
|
||||
}
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
use crate::{
|
||||
prelude::*,
|
||||
process::{do_exit_group, TermStatus},
|
||||
process::{posix_thread::do_exit_group, TermStatus},
|
||||
syscall::SyscallReturn,
|
||||
};
|
||||
|
||||
|
@ -72,12 +72,8 @@ fn get_processes(prio_target: PriorityTarget) -> Result<Vec<Arc<Process>>> {
|
||||
let processes: Vec<Arc<Process>> = process_table::process_table_mut()
|
||||
.iter()
|
||||
.filter(|process| {
|
||||
let Some(main_thread) = process.main_thread() else {
|
||||
return false;
|
||||
};
|
||||
let Some(posix_thread) = main_thread.as_posix_thread() else {
|
||||
return false;
|
||||
};
|
||||
let main_thread = process.main_thread();
|
||||
let posix_thread = main_thread.as_posix_thread().unwrap();
|
||||
uid == posix_thread.credentials().ruid()
|
||||
})
|
||||
.cloned()
|
||||
|
@ -31,7 +31,7 @@ pub fn sys_wait4(
|
||||
return Ok(SyscallReturn::Return(0 as _));
|
||||
};
|
||||
|
||||
let (return_pid, exit_code) = (process.pid(), process.exit_code());
|
||||
let (return_pid, exit_code) = (process.pid(), process.status().exit_code());
|
||||
if exit_status_ptr != 0 {
|
||||
ctx.user_space()
|
||||
.write_val(exit_status_ptr as _, &exit_code)?;
|
||||
|
Loading…
x
Reference in New Issue
Block a user