Support WSTOPPED and WCONTINUED

This commit is contained in:
jiangjianfeng
2025-06-12 03:11:16 +00:00
committed by Ruihan Li
parent bcf1e69878
commit 95faea0fb0
14 changed files with 644 additions and 163 deletions

View File

@ -543,11 +543,12 @@ fn create_child_process(
}
fn set_parent_and_group(parent: &Process, child: &Arc<Process>) {
// Lock order: process table -> children -> group of process
// Lock order: children of process -> process table -> group of process
// -> group inner -> session inner
let mut children_mut = parent.children().lock();
let mut process_table_mut = process_table::process_table_mut();
let mut children_mut = parent.children().lock();
let process_group_mut = parent.process_group.lock();
let process_group = process_group_mut.upgrade().unwrap();

View File

@ -33,7 +33,7 @@ pub use process_vm::{
pub use program_loader::{check_executable_file, ProgramToLoad};
pub use rlimit::ResourceType;
pub use term_status::TermStatus;
pub use wait::{wait_child_exit, WaitOptions};
pub use wait::{do_wait, WaitOptions};
pub(super) fn init() {
process::init();

View File

@ -199,6 +199,13 @@ impl PosixThread {
*self.signalled_waker.lock() = None;
}
/// Wakes up the signalled waker.
pub fn wake_signalled_waker(&self) {
if let Some(waker) = &*self.signalled_waker.lock() {
waker.wake_up();
}
}
/// Enqueues a thread-directed signal.
///
/// This method does not perform permission checks on user signals. Therefore, unless the
@ -233,9 +240,7 @@ impl PosixThread {
_sig_dispositions: MutexGuard<SigDispositions>,
) {
self.sig_queues.enqueue(signal);
if let Some(waker) = &*self.signalled_waker.lock() {
waker.wake_up();
}
self.wake_signalled_waker();
}
/// Returns a reference to the profiling clock of the current thread.

View File

@ -18,6 +18,7 @@ use super::{
};
use crate::{
prelude::*,
process::{status::StopWaitStatus, WaitOptions},
sched::{AtomicNice, Nice},
thread::{AsThread, Thread},
time::clocks::ProfClock,
@ -672,6 +673,47 @@ impl Process {
&self.status
}
/// Stops the process.
//
// FIXME: `ptrace` is another reason that can cause a process to stop.
// Consider extending the method signature to support `ptrace` if necessary.
pub fn stop(&self, sig_num: SigNum) {
if self.status.stop_status().stop(sig_num) {
self.wake_up_parent();
}
}
/// Resumes the stopped process.
pub fn resume(&self) {
if self.status.stop_status().resume() {
self.wake_up_parent();
// Note that the resume function is called by the thread which deals with SIGCONT,
// since SIGCONT is handled by any thread in this process, we need to wake
// up other stopped threads in the same process.
for task in self.tasks.lock().as_slice() {
let posix_thread = task.as_posix_thread().unwrap();
posix_thread.wake_signalled_waker();
}
}
}
/// Returns whether the process is stopped.
pub fn is_stopped(&self) -> bool {
self.status.stop_status().is_stopped()
}
/// Gets and clears the stop status changes for the `wait` syscall.
pub(super) fn wait_stopped_or_continued(&self, options: WaitOptions) -> Option<StopWaitStatus> {
self.status.stop_status().wait(options)
}
fn wake_up_parent(&self) {
let parent_guard = self.parent.lock();
let parent = parent_guard.process().upgrade().unwrap();
parent.children_wait_queue.wake_all();
}
// ******************* Subreaper ********************
/// Sets the child subreaper attribute of the current process.

View File

@ -140,12 +140,8 @@ pub fn handle_pending_signal(
do_exit_group(TermStatus::Killed(sig_num));
}
SigDefaultAction::Ign => {}
SigDefaultAction::Stop => {
let _ = ctx.thread.stop();
}
SigDefaultAction::Cont => {
let _ = ctx.thread.resume();
}
SigDefaultAction::Stop => ctx.process.stop(sig_num),
SigDefaultAction::Cont => ctx.process.resume(),
}
}
}

View File

@ -4,7 +4,10 @@
use core::sync::atomic::{AtomicBool, AtomicU32, Ordering};
use ostd::sync::SpinLock;
use super::ExitCode;
use crate::process::{signal::sig_num::SigNum, WaitOptions};
/// The status of a process.
///
@ -12,12 +15,14 @@ use super::ExitCode;
/// 1. Whether the process is a zombie (i.e., all its threads have exited);
/// 2. Whether the process is the vfork child, which shares the user-space virtual memory
/// with its parent process;
/// 3. The exit code of the process.
/// 3. The exit code of the process;
/// 4. Whether the process is stopped (by a signal or ptrace).
#[derive(Debug)]
pub struct ProcessStatus {
is_zombie: AtomicBool,
is_vfork_child: AtomicBool,
exit_code: AtomicU32,
stop_status: StopStatus,
}
impl Default for ProcessStatus {
@ -26,6 +31,7 @@ impl Default for ProcessStatus {
is_zombie: AtomicBool::new(false),
is_vfork_child: AtomicBool::new(false),
exit_code: AtomicU32::new(0),
stop_status: StopStatus::new(),
}
}
}
@ -71,3 +77,96 @@ impl ProcessStatus {
self.exit_code.store(exit_code, Ordering::Relaxed);
}
}
impl ProcessStatus {
pub(super) fn stop_status(&self) -> &StopStatus {
&self.stop_status
}
}
#[derive(Debug)]
pub(super) struct StopStatus {
/// Indicates whether the process is stopped.
is_stopped: AtomicBool,
/// Indicates whether the process's status has changed and has not yet been waited on.
///
/// User programs may use the wait* syscalls to check for changes in
/// the process's status. This field will be set to `Some(_)` once the
/// process's status changes and will be set to `None` if the process
/// has already been waited on.
wait_status: SpinLock<Option<StopWaitStatus>>,
}
impl StopStatus {
pub(self) const fn new() -> Self {
Self {
is_stopped: AtomicBool::new(false),
wait_status: SpinLock::new(None),
}
}
/// Stops the process by some signal.
///
/// The return value indicates whether the stop status has changed.
pub(super) fn stop(&self, signum: SigNum) -> bool {
// Hold the lock first to avoid race conditions
let mut wait_status = self.wait_status.lock();
if self.is_stopped.load(Ordering::Relaxed) {
false
} else {
self.is_stopped.store(true, Ordering::Relaxed);
*wait_status = Some(StopWaitStatus::Stopped(signum));
true
}
}
/// Resumes the process.
///
/// The return value indicates whether the stop status has changed.
pub(super) fn resume(&self) -> bool {
// Hold the lock first to avoid race conditions
let mut wait_status = self.wait_status.lock();
if self.is_stopped.load(Ordering::Relaxed) {
self.is_stopped.store(false, Ordering::Relaxed);
*wait_status = Some(StopWaitStatus::Continue);
true
} else {
false
}
}
/// Returns whether the process is stopped.
pub(super) fn is_stopped(&self) -> bool {
self.is_stopped.load(Ordering::Relaxed)
}
/// Gets and clears the stop status changes for the `wait` syscall.
pub(super) fn wait(&self, options: WaitOptions) -> Option<StopWaitStatus> {
let mut wait_status = self.wait_status.lock();
if options.contains(WaitOptions::WSTOPPED) {
if let Some(StopWaitStatus::Stopped(_)) = wait_status.as_ref() {
return wait_status.take();
}
}
if options.contains(WaitOptions::WCONTINUED) {
if let Some(StopWaitStatus::Continue) = wait_status.as_ref() {
return wait_status.take();
}
}
None
}
}
#[derive(Debug)]
pub(super) enum StopWaitStatus {
// FIXME: A process can also be stopped by ptrace.
// Extend this enum to support ptrace.
Stopped(SigNum),
Continue,
}

View File

@ -1,7 +1,5 @@
// SPDX-License-Identifier: MPL-2.0
#![expect(dead_code)]
use super::{
process_filter::ProcessFilter,
signal::{constants::SIGCHLD, with_sigmask_changed},
@ -12,14 +10,16 @@ use crate::{
process::{
posix_thread::{thread_table, AsPosixThread},
process_table,
signal::sig_num::SigNum,
status::StopWaitStatus,
},
time::clocks::ProfClock,
};
// The definition of WaitOptions is from Occlum
bitflags! {
pub struct WaitOptions: u32 {
const WNOHANG = 0x1;
//Note: Below flags are not supported yet
const WSTOPPED = 0x2; // Same as WUNTRACED
const WEXITED = 0x4;
const WCONTINUED = 0x8;
@ -31,34 +31,56 @@ bitflags! {
}
impl WaitOptions {
pub fn supported(&self) -> bool {
let unsupported_flags = WaitOptions::all() - WaitOptions::WNOHANG;
!self.intersects(unsupported_flags)
pub fn check(&self) -> Result<()> {
if self.intersects(WaitOptions::WSTOPPED | WaitOptions::WCONTINUED)
&& self.contains(WaitOptions::WNOWAIT)
{
return_errno_with_message!(
Errno::EINVAL,
"WNOWAIT cannot be used toghther with WSTOPPED or WCONTINUED"
);
}
let supported_args = WaitOptions::WNOHANG
| WaitOptions::WSTOPPED
| WaitOptions::WCONTINUED
| WaitOptions::WNOWAIT;
if !supported_args.contains(*self) {
warn!(
"unsupported wait options are found: {:?}",
*self - supported_args
);
}
Ok(())
}
}
pub fn wait_child_exit(
pub fn do_wait(
child_filter: ProcessFilter,
wait_options: WaitOptions,
ctx: &Context,
) -> Result<Option<Arc<Process>>> {
let current = ctx.process;
) -> Result<Option<WaitStatus>> {
wait_options.check()?;
let zombie_child = with_sigmask_changed(
ctx,
|sigmask| sigmask + SIGCHLD,
|| {
current.children_wait_queue().pause_until(|| {
let unwaited_children = current
.children()
.lock()
ctx.process.children_wait_queue().pause_until(|| {
// Acquire the children lock at first to prevent race conditions.
// We want to ensure that multiple waiting threads
// do not return the same waited process status.
let mut children_lock = ctx.process.children().lock();
let unwaited_children = children_lock
.values()
.filter(|child| match child_filter {
ProcessFilter::Any => true,
ProcessFilter::WithPid(pid) => child.pid() == pid,
ProcessFilter::WithPgid(pgid) => child.pgid() == pgid,
})
.cloned()
.collect::<Vec<_>>();
.collect::<Box<_>>();
if unwaited_children.is_empty() {
return Some(Err(Error::with_message(
@ -67,20 +89,15 @@ pub fn wait_child_exit(
)));
}
// return immediately if we find a zombie child
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();
if wait_options.contains(WaitOptions::WNOWAIT) {
// does not reap child, directly return
return Some(Ok(Some(zombie_child.clone())));
} else {
reap_zombie_child(current, zombie_pid);
return Some(Ok(Some(zombie_child.clone())));
if let Some(status) = wait_zombie(&unwaited_children) {
if !wait_options.contains(WaitOptions::WNOWAIT) {
reap_zombie_child(status.pid(), &mut children_lock);
}
return Some(Ok(Some(status)));
}
if let Some(status) = wait_stopped_or_continued(&unwaited_children, wait_options) {
return Some(Ok(Some(status)));
}
if wait_options.contains(WaitOptions::WNOHANG) {
@ -96,17 +113,80 @@ pub fn wait_child_exit(
Ok(zombie_child)
}
pub enum WaitStatus {
Zombie(Arc<Process>),
Stop(Arc<Process>, SigNum),
Continue(Arc<Process>),
}
impl WaitStatus {
pub fn pid(&self) -> u32 {
self.process().pid()
}
pub fn status_code(&self) -> u32 {
match self {
Self::Zombie(process) => process.status().exit_code(),
Self::Stop(_, sig_num) => ((sig_num.as_u8() as u32) << 8) | 0x7f,
Self::Continue(_) => 0xffff,
}
}
pub fn prof_clock(&self) -> &Arc<ProfClock> {
self.process().prof_clock()
}
fn process(&self) -> &Arc<Process> {
match self {
WaitStatus::Zombie(process)
| WaitStatus::Stop(process, _)
| WaitStatus::Continue(process) => process,
}
}
}
fn wait_zombie(unwaited_children: &[&Arc<Process>]) -> Option<WaitStatus> {
unwaited_children
.iter()
.find(|child| child.status().is_zombie())
.map(|child| WaitStatus::Zombie((*child).clone()))
}
fn wait_stopped_or_continued(
unwaited_children: &[&Arc<Process>],
wait_options: WaitOptions,
) -> Option<WaitStatus> {
if !wait_options.intersects(WaitOptions::WSTOPPED | WaitOptions::WCONTINUED) {
return None;
}
// Lock order: children of process -> tasks of process
for process in unwaited_children.iter() {
let Some(stop_wait_status) = process.wait_stopped_or_continued(wait_options) else {
continue;
};
let wait_status = match stop_wait_status {
StopWaitStatus::Stopped(sig_num) => WaitStatus::Stop((*process).clone(), sig_num),
StopWaitStatus::Continue => WaitStatus::Continue((*process).clone()),
};
return Some(wait_status);
}
None
}
/// 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();
fn reap_zombie_child(pid: Pid, children_lock: &mut BTreeMap<Pid, Arc<Process>>) -> ExitCode {
let child_process = children_lock.remove(&pid).unwrap();
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());
}
// Lock order: session table -> group table -> process table -> group of process
// -> group inner -> session inner
// Lock order: children of process -> session table -> group table
// -> process table -> group of process -> group inner -> session inner
let mut session_table_mut = process_table::session_table_mut();
let mut group_table_mut = process_table::group_table_mut();

View File

@ -3,12 +3,12 @@
use super::{getrusage::rusage_t, SyscallReturn};
use crate::{
prelude::*,
process::{wait_child_exit, ProcessFilter, WaitOptions},
process::{do_wait, ProcessFilter, WaitOptions},
};
pub fn sys_wait4(
wait_pid: u64,
exit_status_ptr: u64,
status_ptr: u64,
wait_options: u32,
rusage_addr: Vaddr,
ctx: &Context,
@ -16,31 +16,30 @@ pub fn sys_wait4(
let wait_options = WaitOptions::from_bits(wait_options)
.ok_or_else(|| Error::with_message(Errno::EINVAL, "unknown wait option"))?;
debug!(
"pid = {}, exit_status_ptr = {}, wait_options: {:?}",
wait_pid as i32, exit_status_ptr, wait_options
"pid = {}, status_ptr = {}, wait_options: {:?}",
wait_pid as i32, status_ptr, wait_options
);
debug!("wait4 current pid = {}", ctx.process.pid());
let process_filter = ProcessFilter::from_id(wait_pid as _);
let waited_process =
wait_child_exit(process_filter, wait_options, ctx).map_err(|err| match err.error() {
let wait_status =
do_wait(process_filter, wait_options, ctx).map_err(|err| match err.error() {
Errno::EINTR => Error::new(Errno::ERESTARTSYS),
_ => err,
})?;
let Some(process) = waited_process else {
let Some(wait_status) = wait_status else {
return Ok(SyscallReturn::Return(0 as _));
};
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)?;
let (return_pid, status_code) = (wait_status.pid(), wait_status.status_code());
if status_ptr != 0 {
ctx.user_space().write_val(status_ptr as _, &status_code)?;
}
if rusage_addr != 0 {
let rusage = rusage_t {
ru_utime: process.prof_clock().user_clock().read_time().into(),
ru_stime: process.prof_clock().kernel_clock().read_time().into(),
ru_utime: wait_status.prof_clock().user_clock().read_time().into(),
ru_stime: wait_status.prof_clock().kernel_clock().read_time().into(),
..Default::default()
};

View File

@ -3,7 +3,7 @@
use super::SyscallReturn;
use crate::{
prelude::*,
process::{wait_child_exit, ProcessFilter, WaitOptions},
process::{do_wait, ProcessFilter, WaitOptions},
};
pub fn sys_waitid(
@ -19,12 +19,12 @@ pub fn sys_waitid(
let wait_options = WaitOptions::from_bits(options as u32)
.ok_or(Error::with_message(Errno::EINVAL, "invalid options"))?;
let waited_process =
wait_child_exit(process_filter, wait_options, ctx).map_err(|err| match err.error() {
let wait_status =
do_wait(process_filter, wait_options, ctx).map_err(|err| match err.error() {
Errno::EINTR => Error::new(Errno::ERESTARTSYS),
_ => err,
})?;
let pid = waited_process.map_or(0, |process| process.pid());
let pid = wait_status.map_or(0, |wait_status| wait_status.pid());
Ok(SyscallReturn::Return(pid as _))
}

View File

@ -2,14 +2,13 @@
//! Posix thread implementation
use core::sync::atomic::Ordering;
use core::sync::atomic::{AtomicBool, Ordering};
use ostd::{
cpu::{AtomicCpuSet, CpuSet},
task::Task,
};
use self::status::{AtomicThreadStatus, ThreadStatus};
use crate::{
prelude::*,
sched::{SchedAttr, SchedPolicy},
@ -18,7 +17,6 @@ use crate::{
pub mod exception;
pub mod kernel_thread;
pub mod oops;
pub mod status;
pub mod task;
pub mod work_queue;
@ -52,7 +50,7 @@ pub struct Thread {
// mutable part
/// Thread status
status: AtomicThreadStatus,
is_exited: AtomicBool,
/// Thread CPU affinity
cpu_affinity: AtomicCpuSet,
sched_attr: SchedAttr,
@ -69,7 +67,7 @@ impl Thread {
Thread {
task,
data: Box::new(data),
status: AtomicThreadStatus::new(ThreadStatus::Init),
is_exited: AtomicBool::new(false),
cpu_affinity: AtomicCpuSet::new(cpu_affinity),
sched_attr: SchedAttr::new(sched_policy),
}
@ -91,59 +89,16 @@ impl Thread {
/// Runs this thread at once.
#[track_caller]
pub fn run(&self) {
self.status.store(ThreadStatus::Running, Ordering::Release);
self.task.upgrade().unwrap().run();
}
/// Returns whether the thread is exited.
pub fn is_exited(&self) -> bool {
self.status.load(Ordering::Acquire).is_exited()
}
/// Returns whether the thread is stopped.
pub fn is_stopped(&self) -> bool {
self.status.load(Ordering::Acquire).is_stopped()
}
/// Stops the thread if it is running.
///
/// If the previous status is not [`ThreadStatus::Running`], this function
/// returns [`Err`] with the previous state. Otherwise, it sets the status
/// to [`ThreadStatus::Stopped`] and returns [`Ok`] with the previous state.
///
/// This function only sets the status to [`ThreadStatus::Stopped`],
/// without initiating a reschedule.
pub fn stop(&self) -> core::result::Result<ThreadStatus, ThreadStatus> {
self.status.compare_exchange(
ThreadStatus::Running,
ThreadStatus::Stopped,
Ordering::AcqRel,
Ordering::Acquire,
)
}
/// Resumes running the thread if it is stopped.
///
/// If the previous status is not [`ThreadStatus::Stopped`], this function
/// returns [`None`]. Otherwise, it sets the status to
/// [`ThreadStatus::Running`] and returns [`Some(())`].
///
/// This function only sets the status to [`ThreadStatus::Running`],
/// without initiating a reschedule.
pub fn resume(&self) -> Option<()> {
self.status
.compare_exchange(
ThreadStatus::Stopped,
ThreadStatus::Running,
Ordering::AcqRel,
Ordering::Acquire,
)
.ok()
.map(|_| ())
self.is_exited.load(Ordering::Acquire)
}
pub(super) fn exit(&self) {
self.status.store(ThreadStatus::Exited, Ordering::Release);
self.is_exited.store(true, Ordering::Release);
}
/// Returns the reference to the atomic CPU affinity.

View File

@ -1,40 +0,0 @@
// SPDX-License-Identifier: MPL-2.0
use core::sync::atomic::AtomicU8;
use atomic_integer_wrapper::define_atomic_version_of_integer_like_type;
use int_to_c_enum::TryFromInt;
define_atomic_version_of_integer_like_type!(ThreadStatus, try_from = true, {
#[derive(Debug)]
pub struct AtomicThreadStatus(AtomicU8);
});
#[derive(Clone, Copy, PartialEq, Eq, Debug, TryFromInt)]
#[repr(u8)]
pub enum ThreadStatus {
Init = 0,
Running = 1,
Exited = 2,
Stopped = 3,
}
impl ThreadStatus {
pub fn is_running(&self) -> bool {
*self == ThreadStatus::Running
}
pub fn is_exited(&self) -> bool {
*self == ThreadStatus::Exited
}
pub fn is_stopped(&self) -> bool {
*self == ThreadStatus::Stopped
}
}
impl From<ThreadStatus> for u8 {
fn from(value: ThreadStatus) -> Self {
value as u8
}
}

View File

@ -2,6 +2,7 @@
use ostd::{
cpu::context::UserContext,
sync::Waiter,
task::{Task, TaskOptions},
user::{ReturnReason, UserContextApi, UserMode},
};
@ -32,6 +33,7 @@ pub fn create_new_user_task(
let current_posix_thread = current_thread.as_posix_thread().unwrap();
let current_thread_local = current_task.as_thread_local().unwrap();
let current_process = current_posix_thread.process();
let (stop_waiter, _) = Waiter::new_pair();
let user_ctx = current_task
.user_ctx()
@ -71,11 +73,13 @@ pub fn create_new_user_task(
task: &current_task,
};
loop {
while !current_thread.is_exited() {
// Execute the user code
let return_reason = user_mode.execute(has_kernel_event_fn);
// Handle user events
let user_ctx = user_mode.context_mut();
let mut syscall_number = None;
// handle user event:
match return_reason {
ReturnReason::UserException => handle_exception(&ctx, user_ctx),
ReturnReason::UserSyscall => {
@ -85,22 +89,26 @@ pub fn create_new_user_task(
ReturnReason::KernelEvent => {}
};
// Exit if the thread terminates
if current_thread.is_exited() {
break;
}
// Handle signals
handle_pending_signal(user_ctx, &ctx, syscall_number);
// If current is suspended, wait for a signal to wake up self
while current_thread.is_stopped() {
Thread::yield_now();
debug!("{} is suspended.", current_posix_thread.tid());
// Handle signals while the thread is stopped
// FIXME: Currently, we handle all signals when the process is stopped.
// However, when the process is stopped, at least signals with user-provided handlers
// should not be handled; these signals should only be handled when the process is continued.
// Certain signals, such as SIGKILL, should be handled even if the process is stopped.
// We need to further investigate Linux behavior regarding which signals should be handled
// when the thread is stopped.
while !current_thread.is_exited() && current_process.is_stopped() {
let _ = stop_waiter.pause_until(|| (!current_process.is_stopped()).then_some(()));
handle_pending_signal(user_ctx, &ctx, None);
}
if current_thread.is_exited() {
debug!("exit due to signal");
break;
}
}
debug!("exit user loop");
}
TaskOptions::new(|| {

335
test/apps/process/wait4.c Normal file
View File

@ -0,0 +1,335 @@
// SPDX-License-Identifier: MPL-2.0
#include "../network/test.h"
#include <stdio.h>
#include <stdlib.h>
#include <sys/types.h>
#include <sys/wait.h>
#include <unistd.h>
#include <signal.h>
#include <linux/wait.h>
#include <pthread.h>
#include <sys/mman.h>
#include <fcntl.h>
static pid_t pid;
static int status;
FN_SETUP(fork_child)
{
pid = CHECK(fork());
if (pid == 0) {
// Child entering an infinite loop until killed by parent.
while (1) {
usleep(100);
}
exit(EXIT_SUCCESS);
}
// Parent process
sleep(1); // Ensure the child process is running
}
END_SETUP()
FN_TEST(stop_child)
{
// Stop the child process
TEST_SUCC(kill(pid, SIGSTOP));
TEST_RES(wait4(pid, &status, WSTOPPED, NULL),
_ret == pid && WIFSTOPPED(status) &&
WSTOPSIG(status) == SIGSTOP);
status = 0;
TEST_RES(wait4(pid, &status, WSTOPPED | WNOHANG, NULL),
_ret == 0 && status == 0);
}
END_TEST()
FN_TEST(continue_child)
{
TEST_SUCC(kill(pid, SIGCONT));
status = 0;
TEST_RES(wait4(pid, &status, WCONTINUED, NULL),
_ret == pid && WIFCONTINUED(status));
status = 0;
TEST_RES(wait4(pid, &status, WCONTINUED | WNOHANG, NULL),
_ret == 0 && status == 0);
}
END_TEST()
FN_TEST(wait_nowait)
{
TEST_SUCC(kill(pid, SIGSTOP));
status = 0;
TEST_ERRNO(wait4(pid, &status, WSTOPPED | WNOWAIT, NULL), EINVAL);
status = 0;
TEST_RES(wait4(pid, &status, WSTOPPED, NULL),
_ret == pid && WIFSTOPPED(status) &&
WSTOPSIG(status) == SIGSTOP);
status = 0;
TEST_RES(wait4(pid, &status, WSTOPPED | WNOHANG, NULL),
_ret == 0 && status == 0);
TEST_SUCC(kill(pid, SIGCONT));
status = 0;
TEST_ERRNO(wait4(pid, &status, WCONTINUED | WNOWAIT, NULL), EINVAL);
status = 0;
TEST_RES(wait4(pid, &status, WCONTINUED, NULL),
_ret == pid && WIFCONTINUED(status));
status = 0;
TEST_RES(wait4(pid, &status, WCONTINUED | WNOHANG, NULL),
_ret == 0 && status == 0);
}
END_TEST()
FN_TEST(wait_stopped_and_continued)
{
TEST_SUCC(kill(pid, SIGSTOP));
sleep(1);
TEST_SUCC(kill(pid, SIGCONT));
sleep(1);
status = 0;
TEST_RES(wait4(pid, &status, WSTOPPED | WCONTINUED, NULL),
_ret == pid && WIFCONTINUED(status));
status = 0;
TEST_RES(wait4(pid, &status, WSTOPPED | WCONTINUED | WNOHANG, NULL),
_ret == 0 && status == 0);
}
END_TEST()
FN_TEST(continue_not_stopped)
{
TEST_SUCC(kill(pid, SIGCONT));
status = 0;
TEST_RES(wait4(pid, &status, WCONTINUED | WNOHANG, NULL),
_ret == 0 && status == 0);
}
END_TEST()
FN_TEST(stop_continue_continue)
{
TEST_SUCC(kill(pid, SIGSTOP));
sleep(1);
TEST_SUCC(kill(pid, SIGCONT));
sleep(1);
TEST_SUCC(kill(pid, SIGCONT));
sleep(1);
status = 0;
TEST_RES(wait4(pid, &status, WSTOPPED | WNOHANG, NULL),
status == 0 && _ret == 0);
status = 0;
TEST_RES(wait4(pid, &status, WSTOPPED | WCONTINUED, NULL),
_ret == pid && WIFCONTINUED(status));
status = 0;
TEST_RES(wait4(pid, &status, WSTOPPED | WCONTINUED | WNOHANG, NULL),
_ret == 0 && status == 0);
}
END_TEST()
FN_TEST(stop_continue_stop)
{
TEST_SUCC(kill(pid, SIGSTOP));
sleep(1);
TEST_SUCC(kill(pid, SIGCONT));
sleep(1);
TEST_SUCC(kill(pid, SIGSTOP));
sleep(1);
status = 0;
TEST_RES(wait4(pid, &status, WCONTINUED | WNOHANG, NULL), status == 0);
status = 0;
TEST_RES(wait4(pid, &status, WSTOPPED, NULL),
_ret == pid && WIFSTOPPED(status) &&
WSTOPSIG(status) == SIGSTOP);
status = 0;
TEST_RES(wait4(pid, &status, WSTOPPED | WNOHANG, NULL),
_ret == 0 && status == 0);
// Restore the state
TEST_SUCC(kill(pid, SIGCONT));
status = 0;
TEST_RES(wait4(pid, &status, WCONTINUED, NULL),
_ret == pid && WIFCONTINUED(status));
status = 0;
TEST_RES(wait4(pid, &status, WCONTINUED | WSTOPPED | WNOHANG, NULL),
_ret == 0 && status == 0);
}
END_TEST()
FN_TEST(stop_stopped)
{
TEST_SUCC(kill(pid, SIGSTOP));
status = 0;
TEST_RES(wait4(pid, &status, WSTOPPED, NULL),
_ret == pid && WIFSTOPPED(status) &&
WSTOPSIG(status) == SIGSTOP);
TEST_SUCC(kill(pid, SIGSTOP));
sleep(1);
status = 0;
TEST_RES(wait4(pid, &status, WSTOPPED | WNOHANG, NULL),
_ret == 0 && status == 0);
}
END_TEST()
void *thread_func(void *arg)
{
while (1) {
sleep(0.1);
}
}
void child_process()
{
pthread_t t1, t2;
CHECK(pthread_create(&t1, NULL, thread_func, NULL));
CHECK(pthread_create(&t2, NULL, thread_func, NULL));
pthread_join(t1, NULL);
pthread_join(t2, NULL);
exit(EXIT_SUCCESS);
}
FN_TEST(multithread)
{
pid_t child_pid = TEST_SUCC(fork());
if (child_pid == 0) {
child_process();
}
sleep(1);
TEST_SUCC(kill(child_pid, SIGSTOP));
int status = 0;
TEST_RES(wait4(child_pid, &status, WSTOPPED, NULL),
_ret == child_pid && WIFSTOPPED(status) &&
WSTOPSIG(status) == SIGSTOP);
TEST_RES(wait4(child_pid, &status, WSTOPPED | WNOHANG, NULL),
_ret == 0);
TEST_SUCC(kill(child_pid, SIGKILL));
TEST_RES(wait4(child_pid, NULL, 0, NULL), _ret == child_pid);
}
END_TEST()
volatile int sigint_counter = 0;
volatile int sigtrap_counter = 0;
void handle_sigint(int signum, siginfo_t *_info, void *_context)
{
sigint_counter += 1;
}
void handle_sigtrap(int signum, siginfo_t *_info, void *_context)
{
sigtrap_counter += 1;
}
void child_process2(int *pipe_fds)
{
CHECK(close(pipe_fds[0]));
struct sigaction new_action = {};
new_action.sa_sigaction = handle_sigint;
CHECK(sigaction(SIGINT, &new_action, NULL));
new_action.sa_sigaction = handle_sigtrap;
CHECK(sigaction(SIGTRAP, &new_action, NULL));
while (1) {
usleep(100);
if (sigint_counter == 1) {
sigint_counter = 0;
CHECK(write(pipe_fds[1], "a", 1));
}
if (sigtrap_counter == 1) {
sigtrap_counter = 0;
CHECK(write(pipe_fds[1], "b", 1));
}
}
exit(EXIT_SUCCESS);
}
FN_TEST(nested_signals)
{
int pipe_fds[2];
TEST_SUCC(pipe(pipe_fds));
for (int i = 0; i < 2; i++) {
int fd = pipe_fds[i];
int flags = TEST_SUCC(fcntl(fd, F_GETFL, 0));
TEST_SUCC(fcntl(fd, F_SETFL, flags | O_NONBLOCK));
}
int child_pid = TEST_SUCC(fork());
if (child_pid == 0) {
child_process2(pipe_fds);
}
TEST_SUCC(close(pipe_fds[1]));
sleep(1);
char buf[1] = { 0 };
// SIGINT -> SIGTRAP
TEST_SUCC(kill(child_pid, SIGINT));
sleep(1);
TEST_RES(read(pipe_fds[0], buf, 1), _ret == 1 && buf[0] == 'a');
TEST_SUCC(kill(child_pid, SIGTRAP));
sleep(1);
TEST_RES(read(pipe_fds[0], buf, 1), _ret == 1 && buf[0] == 'b');
// SIGSTOP -> SIGINT -> SIGTRAP -> SIGCONT
TEST_SUCC(kill(child_pid, SIGSTOP));
sleep(1);
// FIXME: The following two read pipe checks are commented out because
// Asterinas currently handles signals with user-provided handlers
// when the thread is stopped, while Linux does not.
TEST_SUCC(kill(child_pid, SIGINT));
sleep(1);
// TEST_ERRNO(read(pipe_fds[0], buf, 1), EAGAIN);
TEST_SUCC(kill(child_pid, SIGTRAP));
sleep(1);
// TEST_ERRNO(read(pipe_fds[0], buf, 1), EAGAIN);
TEST_RES(wait4(child_pid, &status, WSTOPPED, NULL),
_ret == child_pid && WIFSTOPPED(status) &&
WSTOPSIG(status) == SIGSTOP);
TEST_SUCC(kill(child_pid, SIGCONT));
sleep(1);
TEST_RES(read(pipe_fds[0], buf, 1), _ret == 1 && buf[0] == 'a');
TEST_RES(read(pipe_fds[0], buf, 1), _ret == 1 && buf[0] == 'b');
TEST_RES(wait4(child_pid, &status, WCONTINUED, NULL),
_ret == child_pid && WIFCONTINUED(status));
// SIGKILL
TEST_SUCC(kill(child_pid, SIGKILL));
TEST_RES(wait4(child_pid, NULL, 0, NULL), _ret == child_pid);
}
END_TEST()
FN_SETUP(kill_stopped)
{
CHECK(kill(pid, SIGKILL));
sleep(1);
CHECK_WITH(wait4(pid, &status, WSTOPPED, NULL),
_ret == pid && WIFSIGNALED(status) &&
WTERMSIG(status) == SIGKILL);
}
END_SETUP()

View File

@ -33,6 +33,7 @@ mmap/mmap_readahead
mmap/mmap_vmrss
process/group_session
process/job_control
process/wait4
pthread/pthread_test
pty/open_pty
pty/pty_blocking