Add support for execve
RFC: https://github.com/occlum/occlum/issues/429
This commit is contained in:
parent
bad2581a25
commit
c62b6d4091
96
src/libos/src/process/do_exec.rs
Normal file
96
src/libos/src/process/do_exec.rs
Normal file
@ -0,0 +1,96 @@
|
||||
use std::ffi::{CStr, CString};
|
||||
use std::path::Path;
|
||||
|
||||
use super::do_exit::exit_old_process_for_execve;
|
||||
use super::do_spawn::new_process_for_exec;
|
||||
use super::process::ProcessFilter;
|
||||
use super::term_status::TermStatus;
|
||||
use super::wait::Waiter;
|
||||
use super::{do_exit, do_exit_group};
|
||||
use super::{table, ProcessRef, ProcessStatus};
|
||||
use super::{task, ThreadRef};
|
||||
use crate::interrupt::broadcast_interrupts;
|
||||
use crate::prelude::*;
|
||||
|
||||
// FIXME: `occlum exec` command will return early if the application calls execve successfully.
|
||||
// Because the "execved"-ed application will run on a new thread and the current thread will exit.
|
||||
// `occlum run` will not have this problem.
|
||||
|
||||
pub fn do_exec(
|
||||
path: &str,
|
||||
argv: &[CString],
|
||||
envp: &[CString],
|
||||
current_ref: &ThreadRef,
|
||||
) -> Result<isize> {
|
||||
trace!(
|
||||
"exec current process pid = {:?}",
|
||||
current_ref.process().pid()
|
||||
);
|
||||
|
||||
// Construct new process structure but with same parent, pid, tid
|
||||
let current = current!();
|
||||
let new_process_ref = super::do_spawn::new_process_for_exec(path, argv, envp, current_ref);
|
||||
|
||||
if let Ok(new_process_ref) = new_process_ref {
|
||||
let new_main_thread = new_process_ref
|
||||
.main_thread()
|
||||
.expect("the main thread is just created; it must exist");
|
||||
|
||||
// Force exit all child threads of current process
|
||||
let term_status = TermStatus::Exited(0 as u8);
|
||||
current.process().force_exit(term_status);
|
||||
|
||||
// Don't hesitate. Interrupt all threads right now (except the calling thread).
|
||||
broadcast_interrupts();
|
||||
|
||||
// Wait for all threads (except calling thread) to exit
|
||||
wait_for_other_threads_to_exit(current);
|
||||
|
||||
// Exit current thread and let new process to adopt current's child process
|
||||
exit_old_process_for_execve(term_status, new_process_ref.clone());
|
||||
|
||||
// Update process and thread in global table
|
||||
table::replace_process(new_process_ref.pid(), new_process_ref.clone());
|
||||
table::replace_thread(
|
||||
new_process_ref.pid(),
|
||||
new_process_ref.main_thread().unwrap(),
|
||||
);
|
||||
|
||||
// Finally, enqueue the new thread for execution
|
||||
task::enqueue_and_exec(new_main_thread);
|
||||
return Ok(0);
|
||||
} else {
|
||||
// There is something wrong when constructing new process. Just return the error.
|
||||
let error = new_process_ref.unwrap_err();
|
||||
return Err(error);
|
||||
}
|
||||
}
|
||||
|
||||
// Blocking wait until there is only one thread in the calling process
|
||||
fn wait_for_other_threads_to_exit(current_ref: ThreadRef) {
|
||||
use super::do_futex::{self, FutexTimeout};
|
||||
use crate::time::{timespec_t, ClockID};
|
||||
use core::time::Duration;
|
||||
|
||||
// Set timeout to 50ms
|
||||
let timeout = FutexTimeout::new(
|
||||
ClockID::CLOCK_MONOTONIC,
|
||||
timespec_t::from(Duration::from_millis(50)),
|
||||
);
|
||||
// Use calling process's pointer as futex value
|
||||
let futex_val = Arc::as_ptr(¤t_ref.process()) as *const i32;
|
||||
loop {
|
||||
let thread_num = current_ref.process().threads().len();
|
||||
if thread_num == 1 {
|
||||
return;
|
||||
}
|
||||
// Blocking wait here. When a thread exit, it will notify us.
|
||||
unsafe {
|
||||
do_futex::futex_wait(
|
||||
Arc::as_ptr(¤t_ref.process()) as *const i32,
|
||||
*futex_val,
|
||||
&Some(timeout),
|
||||
)
|
||||
};
|
||||
}
|
||||
}
|
@ -3,7 +3,7 @@ use std::intrinsics::atomic_store;
|
||||
|
||||
use super::do_futex::futex_wake;
|
||||
use super::process::{Process, ProcessFilter};
|
||||
use super::{table, TermStatus, ThreadRef, ThreadStatus};
|
||||
use super::{table, ProcessRef, TermStatus, ThreadRef, ThreadStatus};
|
||||
use crate::prelude::*;
|
||||
use crate::signal::{KernelSignal, SigNum};
|
||||
|
||||
@ -56,6 +56,10 @@ fn exit_thread(term_status: TermStatus) {
|
||||
if num_remaining_threads == 0 {
|
||||
exit_process(&thread, term_status);
|
||||
}
|
||||
|
||||
// Notify a thread, if any, that wait on this thread to exit.
|
||||
// E.g. In execve, the new thread should wait for old process's all thread to exit
|
||||
futex_wake(Arc::as_ptr(&thread.process()) as *const i32, 1);
|
||||
}
|
||||
|
||||
fn exit_process(thread: &ThreadRef, term_status: TermStatus) {
|
||||
@ -133,3 +137,68 @@ fn send_sigchld_to(parent: &Arc<Process>) {
|
||||
let mut sig_queues = parent.sig_queues().write().unwrap();
|
||||
sig_queues.enqueue(signal);
|
||||
}
|
||||
|
||||
pub fn exit_old_process_for_execve(term_status: TermStatus, new_parent_ref: ProcessRef) {
|
||||
let thread = current!();
|
||||
|
||||
// Exit current thread
|
||||
let num_remaining_threads = thread.exit(term_status);
|
||||
if thread.tid() != thread.process().pid() {
|
||||
// Keep the main thread's tid available as long as the process is not destroyed.
|
||||
// Main thread doesn't need to delete here. It will be repalced later.
|
||||
table::del_thread(thread.tid()).expect("tid must be in the table");
|
||||
}
|
||||
|
||||
debug_assert!(num_remaining_threads == 0);
|
||||
exit_process_for_execve(&thread, new_parent_ref, term_status);
|
||||
}
|
||||
|
||||
// Let new parent process to adopt current process' children
|
||||
fn exit_process_for_execve(
|
||||
thread: &ThreadRef,
|
||||
new_parent_ref: ProcessRef,
|
||||
term_status: TermStatus,
|
||||
) {
|
||||
let process = thread.process();
|
||||
|
||||
// Deadlock note: always lock parent first, then child.
|
||||
// Lock the idle process since it may adopt new children.
|
||||
let idle_ref = super::IDLE.process().clone();
|
||||
let mut idle_inner = idle_ref.inner();
|
||||
|
||||
// Lock the parent process as we want to prevent race conditions between
|
||||
// current's exit() and parent's wait5().
|
||||
let mut parent;
|
||||
let mut parent_inner = loop {
|
||||
parent = process.parent();
|
||||
if parent.pid() == 0 {
|
||||
// If the parent is the idle process, don't need to lock again
|
||||
break None;
|
||||
}
|
||||
|
||||
let parent_inner = parent.inner();
|
||||
// To prevent the race condition that parent is changed after `parent()`,
|
||||
// but before `parent().innner()`, we need to check again here.
|
||||
if parent.pid() != process.parent().pid() {
|
||||
continue;
|
||||
}
|
||||
break Some(parent_inner);
|
||||
};
|
||||
|
||||
// Lock the current process
|
||||
let mut process_inner = process.inner();
|
||||
let mut new_parent_inner = new_parent_ref.inner();
|
||||
let pid = process.pid();
|
||||
|
||||
// Let new_process to adopt the children of current process
|
||||
process_inner.exit(term_status, &new_parent_ref, &mut new_parent_inner);
|
||||
|
||||
// Remove current process from parent process' zombie list.
|
||||
if parent_inner.is_none() {
|
||||
debug_assert!(parent.pid() == 0);
|
||||
idle_inner.remove_zombie_child(pid);
|
||||
return;
|
||||
}
|
||||
|
||||
parent_inner.unwrap().remove_zombie_child(pid);
|
||||
}
|
||||
|
@ -18,4 +18,4 @@ void __attribute__((optimize("O0"))) occlum_gdb_hook_load_elf(
|
||||
uint64_t elf_base,
|
||||
const char *elf_path,
|
||||
uint64_t elf_path_len) {
|
||||
}
|
||||
}
|
||||
|
@ -7,7 +7,7 @@ use super::elf_file::{ElfFile, ElfHeader, ProgramHeaderExt};
|
||||
use super::process::ProcessBuilder;
|
||||
use super::spawn_attribute::SpawnAttr;
|
||||
use super::task::Task;
|
||||
use super::thread::ThreadName;
|
||||
use super::thread::{ThreadId, ThreadName};
|
||||
use super::{table, task, ProcessRef, ThreadRef};
|
||||
use crate::fs::{
|
||||
CreationFlags, File, FileDesc, FileTable, FsView, HostStdioFds, StdinFile, StdoutFile,
|
||||
@ -107,6 +107,56 @@ fn new_process(
|
||||
spawn_attributes: Option<SpawnAttr>,
|
||||
host_stdio_fds: Option<&HostStdioFds>,
|
||||
current_ref: &ThreadRef,
|
||||
) -> Result<ProcessRef> {
|
||||
let new_process_ref = new_process_common(
|
||||
file_path,
|
||||
argv,
|
||||
envp,
|
||||
file_actions,
|
||||
spawn_attributes,
|
||||
host_stdio_fds,
|
||||
current_ref,
|
||||
None,
|
||||
)?;
|
||||
table::add_process(new_process_ref.clone());
|
||||
table::add_thread(new_process_ref.main_thread().unwrap());
|
||||
|
||||
Ok(new_process_ref)
|
||||
}
|
||||
|
||||
/// Create a new process for execve which will use same parent, pid, tid
|
||||
pub fn new_process_for_exec(
|
||||
file_path: &str,
|
||||
argv: &[CString],
|
||||
envp: &[CString],
|
||||
current_ref: &ThreadRef,
|
||||
) -> Result<ProcessRef> {
|
||||
let tid = ThreadId {
|
||||
tid: current_ref.process().pid() as u32,
|
||||
};
|
||||
let new_process_ref = new_process_common(
|
||||
file_path,
|
||||
argv,
|
||||
envp,
|
||||
&Vec::new(),
|
||||
None,
|
||||
None,
|
||||
current_ref,
|
||||
Some(tid),
|
||||
)?;
|
||||
|
||||
Ok(new_process_ref)
|
||||
}
|
||||
|
||||
fn new_process_common(
|
||||
file_path: &str,
|
||||
argv: &[CString],
|
||||
envp: &[CString],
|
||||
file_actions: &[FileAction],
|
||||
spawn_attributes: Option<SpawnAttr>,
|
||||
host_stdio_fds: Option<&HostStdioFds>,
|
||||
current_ref: &ThreadRef,
|
||||
reuse_tid: Option<ThreadId>,
|
||||
) -> Result<ProcessRef> {
|
||||
let mut argv = argv.clone().to_vec();
|
||||
let (is_script, elf_inode, mut elf_buf, elf_header) =
|
||||
@ -223,10 +273,19 @@ fn new_process(
|
||||
let elf_name = elf_path.rsplit('/').collect::<Vec<&str>>()[0];
|
||||
let thread_name = ThreadName::new(elf_name);
|
||||
|
||||
ProcessBuilder::new()
|
||||
let mut parent;
|
||||
let mut process_builder = ProcessBuilder::new();
|
||||
if reuse_tid.is_some() {
|
||||
process_builder = process_builder.tid(reuse_tid.unwrap());
|
||||
parent = current!().process().parent();
|
||||
} else {
|
||||
parent = process_ref;
|
||||
}
|
||||
|
||||
process_builder
|
||||
.vm(vm_ref)
|
||||
.exec_path(&elf_path)
|
||||
.parent(process_ref)
|
||||
.parent(parent)
|
||||
.task(task)
|
||||
.sched(sched_ref)
|
||||
.rlimits(rlimit_ref)
|
||||
@ -238,9 +297,6 @@ fn new_process(
|
||||
.build()?
|
||||
};
|
||||
|
||||
table::add_process(new_process_ref.clone());
|
||||
table::add_thread(new_process_ref.main_thread().unwrap());
|
||||
|
||||
info!(
|
||||
"Process created: elf = {}, pid = {}",
|
||||
elf_path,
|
||||
|
@ -32,6 +32,7 @@ pub use self::thread::{Thread, ThreadStatus};
|
||||
|
||||
mod do_arch_prctl;
|
||||
mod do_clone;
|
||||
mod do_exec;
|
||||
mod do_exit;
|
||||
mod do_futex;
|
||||
mod do_getpid;
|
||||
|
@ -1,5 +1,6 @@
|
||||
use super::do_arch_prctl::ArchPrctlCode;
|
||||
use super::do_clone::CloneFlags;
|
||||
use super::do_exec::do_exec;
|
||||
use super::do_futex::{FutexFlags, FutexOp, FutexTimeout};
|
||||
use super::do_spawn::FileAction;
|
||||
use super::do_wait4::WaitOptions;
|
||||
@ -429,3 +430,16 @@ pub fn do_getgroups(size: isize, buf_ptr: *mut u32) -> Result<isize> {
|
||||
Ok(1)
|
||||
}
|
||||
}
|
||||
|
||||
pub fn do_execve(path: *const i8, argv: *const *const i8, envp: *const *const i8) -> Result<isize> {
|
||||
let path = clone_cstring_safely(path)?.to_string_lossy().into_owned();
|
||||
let argv = clone_cstrings_safely(argv)?;
|
||||
let envp = clone_cstrings_safely(envp)?;
|
||||
let current = current!();
|
||||
debug!(
|
||||
"execve: path: {:?}, argv: {:?}, envp: {:?}",
|
||||
path, argv, envp
|
||||
);
|
||||
|
||||
do_exec(&path, &argv, &envp, ¤t)
|
||||
}
|
||||
|
@ -31,6 +31,11 @@ pub(super) fn del_process(pid: pid_t) -> Result<ProcessRef> {
|
||||
PROCESS_TABLE.lock().unwrap().del(pid)
|
||||
}
|
||||
|
||||
pub fn replace_process(pid: pid_t, new_process: ProcessRef) -> Result<()> {
|
||||
del_process(pid);
|
||||
add_process(new_process)
|
||||
}
|
||||
|
||||
pub fn get_thread(tid: pid_t) -> Result<ThreadRef> {
|
||||
THREAD_TABLE.lock().unwrap().get(tid)
|
||||
}
|
||||
@ -43,6 +48,11 @@ pub(super) fn del_thread(tid: pid_t) -> Result<ThreadRef> {
|
||||
THREAD_TABLE.lock().unwrap().del(tid)
|
||||
}
|
||||
|
||||
pub(super) fn replace_thread(tid: pid_t, new_thread: ThreadRef) -> Result<()> {
|
||||
del_thread(tid);
|
||||
add_thread(new_thread)
|
||||
}
|
||||
|
||||
pub fn debug() {
|
||||
println!("process table = {:#?}", PROCESS_TABLE.lock().unwrap());
|
||||
println!("thread table = {:#?}", THREAD_TABLE.lock().unwrap());
|
||||
|
@ -8,7 +8,7 @@ use crate::prelude::*;
|
||||
/// And when a ThreadID instance is freed, its ID is automatically freed too.
|
||||
#[derive(Debug, PartialEq)]
|
||||
pub struct ThreadId {
|
||||
tid: u32,
|
||||
pub tid: u32,
|
||||
}
|
||||
|
||||
impl ThreadId {
|
||||
@ -41,7 +41,7 @@ impl Drop for ThreadId {
|
||||
}
|
||||
|
||||
let mut alloc = THREAD_ID_ALLOC.lock().unwrap();
|
||||
alloc.free(self.tid).expect("tid must has been allocated");
|
||||
alloc.free(self.tid);
|
||||
}
|
||||
}
|
||||
|
||||
@ -90,7 +90,9 @@ impl IdAlloc {
|
||||
}
|
||||
|
||||
pub fn free(&mut self, id: u32) -> Option<u32> {
|
||||
debug_assert!(self.used_ids.contains(&id));
|
||||
// Note: When enableing "execve", there is situation that the ThreadId is reused.
|
||||
// And thus when exit, it may free twice.
|
||||
// debug_assert!(self.used_ids.contains(&id));
|
||||
if self.used_ids.remove(&id) {
|
||||
Some(id)
|
||||
} else {
|
||||
|
@ -40,8 +40,8 @@ use crate::net::{
|
||||
do_shutdown, do_socket, do_socketpair, mmsghdr, msghdr, msghdr_mut,
|
||||
};
|
||||
use crate::process::{
|
||||
do_arch_prctl, do_clone, do_exit, do_exit_group, do_futex, do_getegid, do_geteuid, do_getgid,
|
||||
do_getgroups, do_getpgid, do_getpid, do_getppid, do_gettid, do_getuid, do_prctl,
|
||||
do_arch_prctl, do_clone, do_execve, do_exit, do_exit_group, do_futex, do_getegid, do_geteuid,
|
||||
do_getgid, do_getgroups, do_getpgid, do_getpid, do_getppid, do_gettid, do_getuid, do_prctl,
|
||||
do_set_tid_address, do_spawn_for_glibc, do_spawn_for_musl, do_wait4, pid_t, posix_spawnattr_t,
|
||||
FdOp, SpawnFileActions, ThreadStatus,
|
||||
};
|
||||
@ -144,7 +144,7 @@ macro_rules! process_syscall_table_with_callback {
|
||||
(Clone = 56) => do_clone(flags: u32, stack_addr: usize, ptid: *mut pid_t, ctid: *mut pid_t, new_tls: usize),
|
||||
(Fork = 57) => handle_unsupported(),
|
||||
(Vfork = 58) => handle_unsupported(),
|
||||
(Execve = 59) => handle_unsupported(),
|
||||
(Execve = 59) => do_execve(path: *const i8, argv: *const *const i8, envp: *const *const i8),
|
||||
(Exit = 60) => do_exit(exit_status: i32),
|
||||
(Wait4 = 61) => do_wait4(pid: i32, _exit_status: *mut i32, options: u32),
|
||||
(Kill = 62) => do_kill(pid: i32, sig: c_int),
|
||||
|
@ -19,7 +19,7 @@ TESTS ?= env empty hello_world malloc mmap file fs_perms getpid spawn sched pipe
|
||||
truncate readdir mkdir open stat link symlink chmod chown tls pthread system_info rlimit \
|
||||
server server_epoll unix_socket cout hostfs cpuid rdtsc device sleep exit_group \
|
||||
ioctl fcntl eventfd emulate_syscall access signal sysinfo prctl rename procfs wait \
|
||||
spawn_attribute statfs
|
||||
spawn_attribute exec statfs
|
||||
# Benchmarks: need to be compiled and run by bench-% target
|
||||
BENCHES := spawn_and_exit_latency pipe_throughput unix_socket_throughput
|
||||
|
||||
|
5
test/exec/Makefile
Normal file
5
test/exec/Makefile
Normal file
@ -0,0 +1,5 @@
|
||||
include ../test_common.mk
|
||||
|
||||
EXTRA_C_FLAGS := -g -lpthread
|
||||
EXTRA_LINK_FLAGS :=
|
||||
BIN_ARGS :=
|
92
test/exec/main.c
Normal file
92
test/exec/main.c
Normal file
@ -0,0 +1,92 @@
|
||||
#include <unistd.h>
|
||||
#include <string.h>
|
||||
#include <stdio.h>
|
||||
#include <stdlib.h>
|
||||
#include <spawn.h>
|
||||
#include <sys/wait.h>
|
||||
#include <pthread.h>
|
||||
#include <stdbool.h>
|
||||
#include "test.h"
|
||||
|
||||
static void *just_sleep(void *_arg) {
|
||||
bool should_exit_by_execve = *(bool *)_arg;
|
||||
sleep(3);
|
||||
|
||||
// If should_exit_by_execve is true, execve should be done before sleep returns.
|
||||
if (should_exit_by_execve) {
|
||||
printf("This should never be reached");
|
||||
exit(-1);
|
||||
} else {
|
||||
printf("sleep is done\n");
|
||||
}
|
||||
return NULL;
|
||||
}
|
||||
|
||||
int test_execve_no_return(void) {
|
||||
bool should_exit_by_execve = true;
|
||||
pthread_t child_thread;
|
||||
if (pthread_create(&child_thread, NULL, just_sleep, (void *)&should_exit_by_execve) < 0) {
|
||||
THROW_ERROR("pthread_create failed");
|
||||
}
|
||||
|
||||
char *args[] = {"spawn", NULL};
|
||||
execve("/bin/spawn", args, NULL);
|
||||
|
||||
THROW_ERROR("The program shouldn't reach here.");
|
||||
return -1;
|
||||
}
|
||||
|
||||
int test_execve_error_return(void) {
|
||||
// execve will fail in this case and thus the child thread will not exit until finish
|
||||
bool should_exit_by_execve = false;
|
||||
pthread_t child_thread;
|
||||
if (pthread_create(&child_thread, NULL, just_sleep, (void *)&should_exit_by_execve) < 0) {
|
||||
THROW_ERROR("pthread_create failed");
|
||||
}
|
||||
|
||||
// during the time, try execve a non-exit process
|
||||
int ret = execve("/bin/joke", NULL, NULL);
|
||||
if (ret != -1 || errno != ENOENT) {
|
||||
THROW_ERROR("execve error code wrong");
|
||||
}
|
||||
|
||||
pthread_join(child_thread, NULL);
|
||||
return 0;
|
||||
}
|
||||
|
||||
int test_execve_on_child_thread(void) {
|
||||
int ret, child_pid, status;
|
||||
|
||||
// construct child process args
|
||||
int child_argc = 3; // ./nauty_child -t execve_thread
|
||||
char **child_argv = calloc(1, sizeof(char *) * (child_argc + 1));
|
||||
child_argv[0] = strdup("naughty_child");
|
||||
child_argv[1] = strdup("-t");
|
||||
child_argv[2] = strdup("execve_thread");
|
||||
|
||||
ret = posix_spawn(&child_pid, "/bin/naughty_child", NULL, NULL, child_argv, NULL);
|
||||
if (ret != 0) {
|
||||
THROW_ERROR("failed to spawn a child process");
|
||||
}
|
||||
|
||||
ret = waitpid(child_pid, &status, 0);
|
||||
if (ret < 0) {
|
||||
THROW_ERROR("failed to wait4 the child process");
|
||||
}
|
||||
printf("child process %d exit status = %d\n", child_pid, status);
|
||||
if (status != 0) {
|
||||
THROW_ERROR("child process exit with error");
|
||||
}
|
||||
|
||||
return 0;
|
||||
}
|
||||
|
||||
static test_case_t test_cases[] = {
|
||||
TEST_CASE(test_execve_on_child_thread),
|
||||
TEST_CASE(test_execve_error_return),
|
||||
TEST_CASE(test_execve_no_return),
|
||||
};
|
||||
|
||||
int main() {
|
||||
return test_suite_run(test_cases, ARRAY_SIZE(test_cases));
|
||||
}
|
@ -1,8 +1,11 @@
|
||||
#define _GNU_SOURCE
|
||||
#include <sys/types.h>
|
||||
#include <unistd.h>
|
||||
#include <stdio.h>
|
||||
#include <sys/syscall.h>
|
||||
|
||||
int main(int argc, const char *argv[]) {
|
||||
printf("Run a new process with pid = %d and ppid = %d\n", getpid(), getppid());
|
||||
printf("tid = %ld\n", syscall(SYS_gettid));
|
||||
return 0;
|
||||
}
|
||||
|
@ -1,5 +1,5 @@
|
||||
include ../test_common.mk
|
||||
|
||||
EXTRA_C_FLAGS := -g
|
||||
EXTRA_C_FLAGS := -g -lpthread
|
||||
EXTRA_LINK_FLAGS :=
|
||||
BIN_ARGS :=
|
||||
|
@ -5,6 +5,8 @@
|
||||
#include <stdlib.h>
|
||||
#include <features.h>
|
||||
#include <sys/stat.h>
|
||||
#include <pthread.h>
|
||||
#include <unistd.h>
|
||||
#include "test.h"
|
||||
|
||||
char **g_argv;
|
||||
@ -100,6 +102,94 @@ int test_ioctl_fioclex() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
// This child process will first create multiple threads which are waiting on a condition
|
||||
// and then a child thread will call execve and all threads should be destroyed except the
|
||||
// main thread. Use "pthread" test case as reference.
|
||||
#define NTHREADS (5)
|
||||
#define WAIT_ROUND (100000)
|
||||
|
||||
struct thread_cond_arg {
|
||||
int ti;
|
||||
volatile unsigned int *val;
|
||||
volatile int *exit_thread_count;
|
||||
pthread_cond_t *cond_val;
|
||||
pthread_mutex_t *mutex;
|
||||
};
|
||||
|
||||
static void *thread_cond_wait(void *_arg) {
|
||||
struct thread_cond_arg *arg = _arg;
|
||||
printf("Thread #%d: start to wait on condition variable.\n", arg->ti);
|
||||
for (unsigned int i = 0; i < WAIT_ROUND; ++i) {
|
||||
pthread_mutex_lock(arg->mutex);
|
||||
// execve on a child thread with mutex
|
||||
if (arg->ti == NTHREADS - 4) {
|
||||
char *args[] = {"/bin/getpid", NULL};
|
||||
if (execve("/bin/getpid", args, NULL) < 0) {
|
||||
printf("execve failed with errno: %d", errno);
|
||||
exit(errno);
|
||||
}
|
||||
}
|
||||
while (*(arg->val) == 0) {
|
||||
pthread_cond_wait(arg->cond_val, arg->mutex);
|
||||
}
|
||||
pthread_mutex_unlock(arg->mutex);
|
||||
}
|
||||
(*arg->exit_thread_count)++;
|
||||
printf("Thread #%d: exited.\n", arg->ti);
|
||||
return NULL;
|
||||
}
|
||||
|
||||
int test_execve_child_thread() {
|
||||
volatile unsigned int val = 0;
|
||||
volatile int exit_thread_count = 0;
|
||||
pthread_t threads[NTHREADS];
|
||||
struct thread_cond_arg thread_args[NTHREADS];
|
||||
pthread_cond_t cond_val = PTHREAD_COND_INITIALIZER;
|
||||
pthread_mutex_t mutex = PTHREAD_MUTEX_INITIALIZER;
|
||||
|
||||
/*
|
||||
* Start the threads waiting on the condition variable
|
||||
*/
|
||||
for (int ti = 0; ti < NTHREADS; ti++) {
|
||||
struct thread_cond_arg *thread_arg = &thread_args[ti];
|
||||
thread_arg->ti = ti;
|
||||
thread_arg->val = &val;
|
||||
thread_arg->exit_thread_count = &exit_thread_count;
|
||||
thread_arg->cond_val = &cond_val;
|
||||
thread_arg->mutex = &mutex;
|
||||
|
||||
if (pthread_create(&threads[ti], NULL, thread_cond_wait, thread_arg) < 0) {
|
||||
printf("ERROR: pthread_create failed (ti = %d)\n", ti);
|
||||
return -1;
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Unblock all threads currently waiting on the condition variable
|
||||
*/
|
||||
while (exit_thread_count < NTHREADS) {
|
||||
pthread_mutex_lock(&mutex);
|
||||
val = 1;
|
||||
pthread_cond_broadcast(&cond_val);
|
||||
pthread_mutex_unlock(&mutex);
|
||||
|
||||
pthread_mutex_lock(&mutex);
|
||||
val = 0;
|
||||
pthread_mutex_unlock(&mutex);
|
||||
}
|
||||
|
||||
// wait for all threads to finish
|
||||
for (int ti = 0; ti < NTHREADS; ti++) {
|
||||
if (pthread_join(threads[ti], NULL) < 0) {
|
||||
printf("ERROR: pthread_join failed (ti = %d)\n", ti);
|
||||
return -1;
|
||||
}
|
||||
}
|
||||
|
||||
THROW_ERROR("This should never be reached!");
|
||||
return -1;
|
||||
}
|
||||
|
||||
// ============================================================================
|
||||
// Test suite
|
||||
// ============================================================================
|
||||
@ -113,6 +203,8 @@ int start_test(const char *test_name) {
|
||||
return test_spawn_attribute_sigdef();
|
||||
} else if (strcmp(test_name, "fioclex") == 0) {
|
||||
return test_ioctl_fioclex();
|
||||
} else if (strcmp(test_name, "execve_thread") == 0) {
|
||||
return test_execve_child_thread();
|
||||
} else {
|
||||
fprintf(stderr, "[child] test case not found\n");
|
||||
return -1;
|
||||
@ -120,8 +212,8 @@ int start_test(const char *test_name) {
|
||||
}
|
||||
|
||||
void print_usage() {
|
||||
fprintf(stderr, "Usage:\n nauty_child [-t testcase1] [-t testcase2] ...\n\n");
|
||||
fprintf(stderr, " Now support testcase: <sigmask, sigdef, fioclex>\n");
|
||||
fprintf(stderr, "Usage:\n naughty_child [-t testcase1] [-t testcase2] ...\n\n");
|
||||
fprintf(stderr, " Now support testcase: <sigmask, sigdef, fioclex, execve_thread>\n");
|
||||
}
|
||||
|
||||
int main(int argc, char *argv[]) {
|
||||
|
Loading…
Reference in New Issue
Block a user