shadow_rs/host/
process.rs

1//! An emulated Linux process.
2
3use std::cell::{Cell, Ref, RefCell, RefMut};
4use std::collections::BTreeMap;
5use std::ffi::{CStr, CString, c_char, c_void};
6use std::fmt::Write;
7use std::num::TryFromIntError;
8use std::ops::{Deref, DerefMut};
9use std::os::fd::AsRawFd;
10use std::path::{Path, PathBuf};
11use std::sync::Arc;
12use std::sync::atomic::Ordering;
13#[cfg(feature = "perf_timers")]
14use std::time::Duration;
15
16use linux_api::errno::Errno;
17use linux_api::fcntl::OFlag;
18use linux_api::posix_types::Pid;
19use linux_api::sched::{CloneFlags, SuidDump};
20use linux_api::signal::{
21    LinuxDefaultAction, SigActionFlags, Signal, SignalFromI32Error, defaultaction, siginfo_t,
22    sigset_t,
23};
24use log::{debug, trace, warn};
25use rustix::process::{WaitOptions, WaitStatus};
26use shadow_shim_helper_rs::HostId;
27use shadow_shim_helper_rs::explicit_drop::{ExplicitDrop, ExplicitDropper};
28use shadow_shim_helper_rs::rootedcell::Root;
29use shadow_shim_helper_rs::rootedcell::rc::RootedRc;
30use shadow_shim_helper_rs::rootedcell::refcell::RootedRefCell;
31use shadow_shim_helper_rs::shim_shmem::ProcessShmem;
32use shadow_shim_helper_rs::simulation_time::SimulationTime;
33use shadow_shim_helper_rs::syscall_types::{ForeignPtr, ManagedPhysicalMemoryAddr};
34use shadow_shmem::allocator::ShMemBlock;
35
36use super::descriptor::descriptor_table::{DescriptorHandle, DescriptorTable};
37use super::descriptor::listener::StateEventSource;
38use super::descriptor::{FileSignals, FileState};
39use super::host::Host;
40use super::memory_manager::{MemoryManager, ProcessMemoryRef, ProcessMemoryRefMut};
41use super::syscall::formatter::StraceFmtMode;
42use super::syscall::types::ForeignArrayPtr;
43use super::thread::{Thread, ThreadId};
44use super::timer::Timer;
45use crate::core::configuration::{ProcessFinalState, RunningVal};
46use crate::core::work::task::TaskRef;
47use crate::core::worker::Worker;
48use crate::cshadow;
49use crate::host::context::ProcessContext;
50use crate::host::descriptor::Descriptor;
51use crate::host::managed_thread::ManagedThread;
52use crate::host::syscall::formatter::FmtOptions;
53use crate::utility::callback_queue::CallbackQueue;
54#[cfg(feature = "perf_timers")]
55use crate::utility::perf_timer::PerfTimer;
56use crate::utility::{self, debug_assert_cloexec};
57
58/// Virtual pid of a shadow process
59#[derive(Debug, PartialEq, Eq, Hash, Copy, Clone, Ord, PartialOrd)]
60pub struct ProcessId(u32);
61
62impl ProcessId {
63    // The first Process to run after boot is the "init" process, and has pid=1.
64    // In Shadow simulations, this roughly corresponds to Shadow itself. e.g.
65    // processes spawned by Shadow itself have a parent pid of 1.
66    pub const INIT: Self = ProcessId(1);
67
68    /// Returns what the `ProcessId` would be of a `Process` whose thread
69    /// group leader has id `thread_group_leader_tid`.
70    pub fn from_thread_group_leader_tid(thread_group_leader_tid: ThreadId) -> Self {
71        ProcessId::try_from(libc::pid_t::from(thread_group_leader_tid)).unwrap()
72    }
73}
74
75impl std::fmt::Display for ProcessId {
76    fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
77        write!(f, "{}", self.0)
78    }
79}
80
81impl TryFrom<u32> for ProcessId {
82    type Error = TryFromIntError;
83
84    fn try_from(val: u32) -> Result<Self, Self::Error> {
85        // we don't actually want the value as a `pid_t`, we just want to make sure it can be
86        // converted successfully
87        let _ = libc::pid_t::try_from(val)?;
88        Ok(ProcessId(val))
89    }
90}
91
92impl TryFrom<libc::pid_t> for ProcessId {
93    type Error = TryFromIntError;
94
95    fn try_from(value: libc::pid_t) -> Result<Self, Self::Error> {
96        Ok(ProcessId(value.try_into()?))
97    }
98}
99
100impl From<ProcessId> for u32 {
101    fn from(val: ProcessId) -> Self {
102        val.0
103    }
104}
105
106impl From<ProcessId> for libc::pid_t {
107    fn from(val: ProcessId) -> Self {
108        val.0.try_into().unwrap()
109    }
110}
111
112impl From<ThreadId> for ProcessId {
113    fn from(value: ThreadId) -> Self {
114        ProcessId::try_from(libc::pid_t::from(value)).unwrap()
115    }
116}
117
118#[derive(Debug, Copy, Clone, Eq, PartialEq)]
119pub enum ExitStatus {
120    Normal(i32),
121    Signaled(Signal),
122    /// The process was killed by Shadow rather than exiting "naturally" as part
123    /// of the simulation. Currently this only happens when the process is still
124    /// running when the simulation stop_time is reached.
125    ///
126    /// A signal delivered via `shutdown_signal` does not result in this status;
127    /// e.g. if the process is killed directly by the signal the ExitStatus will
128    /// be `Signaled`; if the process handles the signal and exits by calling
129    /// `exit`, the status will be `Normal`.
130    StoppedByShadow,
131}
132
133#[derive(Debug)]
134struct StraceLogging {
135    file: RootedRefCell<std::fs::File>,
136    options: FmtOptions,
137}
138
139/// Parts of the process that are present in all states.
140struct Common {
141    id: ProcessId,
142    host_id: HostId,
143
144    // Parent pid (aka `ppid`), as returned e.g. by `getppid`.  This can change
145    // at runtime if the original parent exits and is reaped.
146    parent_pid: Cell<ProcessId>,
147
148    // Process group id (aka `pgid`), as returned e.g. by `getpgid`.
149    group_id: Cell<ProcessId>,
150
151    // Session id, as returned e.g. by `getsid`.
152    session_id: Cell<ProcessId>,
153
154    // Signal to send to parent on death.
155    exit_signal: Option<Signal>,
156
157    // unique id of the program that this process should run
158    name: CString,
159
160    // the name of the executable as provided in shadow's config, for logging purposes
161    plugin_name: CString,
162
163    // absolute path to the process's working directory.
164    // This must remain in sync with the actual working dir of the native process.
165    // See https://github.com/shadow/shadow/issues/2960
166    working_dir: CString,
167
168    // (emulated) Process-wide resource limits. We don't enforce these, but track
169    // what they are so that we can return the expected value for e.g. `getrlimit`.
170    rlimits: [linux_api::resource::rlimit64; linux_api::resource::RLIM_NLIMITS as usize],
171}
172
173impl Common {
174    fn id(&self) -> ProcessId {
175        self.id
176    }
177
178    fn physical_address(&self, vptr: ForeignPtr<()>) -> ManagedPhysicalMemoryAddr {
179        // We currently don't keep a true system-wide virtual <-> physical address
180        // mapping. Instead we simply assume that no shadow processes map the same
181        // underlying physical memory, and that therefore (pid, virtual address)
182        // uniquely defines a physical address.
183        //
184        // If we ever want to support futexes in memory shared between processes,
185        // we'll need to change this.  The most foolproof way to do so is probably
186        // to change ManagedPhysicalMemoryAddr to be a bigger struct that identifies where
187        // the mapped region came from (e.g. what file), and the offset into that
188        // region. Such "fat" physical pointers might make memory management a
189        // little more cumbersome though, e.g. when using them as keys in the futex
190        // table.
191        //
192        // Alternatively we could hash the region+offset to a 64-bit value, but
193        // then we'd need to deal with potential collisions. On average we'd expect
194        // a collision after 2**32 physical addresses; i.e. they *probably*
195        // wouldn't happen in practice for realistic simulations.
196
197        // Linux uses the bottom 48-bits for user-space virtual addresses, giving
198        // us 16 bits for the pid.
199        const PADDR_BITS: i32 = 64;
200        const VADDR_BITS: i32 = 48;
201        const PID_BITS: i32 = 16;
202        assert_eq!(PADDR_BITS, PID_BITS + VADDR_BITS);
203
204        let high_part: u64 = u64::from(u32::from(self.id())) << VADDR_BITS;
205        assert_eq!(
206            ProcessId::try_from((high_part >> VADDR_BITS) as u32),
207            Ok(self.id())
208        );
209
210        let low_part = u64::from(vptr);
211        assert_eq!(low_part >> VADDR_BITS, 0);
212
213        ManagedPhysicalMemoryAddr::from(high_part | low_part)
214    }
215
216    fn name(&self) -> &str {
217        self.name.to_str().unwrap()
218    }
219
220    pub fn thread_group_leader_id(&self) -> ThreadId {
221        // tid of the thread group leader is equal to the pid.
222        ThreadId::from(self.id())
223    }
224}
225
226/// A process that is currently runnable.
227pub struct RunnableProcess {
228    common: Common,
229
230    // Expected end state, if any. We'll report an error if this is present and
231    // doesn't match the actual exit status.
232    //
233    // This will be None e.g. for processes created via `fork` instead of
234    // spawned directly from Shadow's config file. In those cases it's the
235    // parent's responsibility to reap and interpret the exit status.
236    expected_final_state: Option<ProcessFinalState>,
237
238    // Shared memory allocation for shared state with shim.
239    shim_shared_mem_block: ShMemBlock<'static, ProcessShmem>,
240
241    // Shared with forked Processes
242    strace_logging: Option<Arc<StraceLogging>>,
243
244    // The shim's log file. This gets dup'd into the ManagedProcess
245    // where the shim can write to it directly. We persist it to handle the case
246    // where we need to recreatea a ManagedProcess and have it continue writing
247    // to the same file.
248    //
249    // Shared with forked Processes
250    shimlog_file: Arc<std::fs::File>,
251
252    // "dumpable" state, as manipulated via the prctl operations PR_SET_DUMPABLE
253    // and PR_GET_DUMPABLE.
254    dumpable: Cell<SuidDump>,
255
256    native_pid: Pid,
257
258    // timer that tracks the amount of CPU time we spend on plugin execution and processing
259    #[cfg(feature = "perf_timers")]
260    cpu_delay_timer: RefCell<PerfTimer>,
261    #[cfg(feature = "perf_timers")]
262    total_run_time: Cell<Duration>,
263
264    itimer_real: RefCell<Timer>,
265
266    // The `RootedRc` lets us hold a reference to a thread without holding a
267    // reference to the thread list. e.g. this lets us implement the `clone`
268    // syscall, which adds a thread to the list while we have a reference to the
269    // parent thread.
270    threads: RefCell<BTreeMap<ThreadId, RootedRc<RootedRefCell<Thread>>>>,
271
272    // References to `Self::memory_manager` cached on behalf of C code using legacy
273    // C memory access APIs.
274    // TODO: Remove these when we've migrated Shadow off of the APIs that need
275    // them (probably by migrating all the calling code to Rust).
276    //
277    // SAFETY: Must be before memory_manager for drop order.
278    unsafe_borrow_mut: RefCell<Option<UnsafeBorrowMut>>,
279    unsafe_borrows: RefCell<Vec<UnsafeBorrow>>,
280
281    // `clone(2)` documents that if `CLONE_THREAD` is set, then `CLONE_VM` must
282    // also be set. Hence all threads in a process always share the same virtual
283    // address space, and hence we have a `MemoryManager` at the `Process` level
284    // rather than the `Thread` level.
285    // SAFETY: Must come after `unsafe_borrows` and `unsafe_borrow_mut`.
286    // Boxed to avoid invalidating those if Self is moved.
287    memory_manager: Box<RefCell<MemoryManager>>,
288
289    // Listeners for child-events.
290    // e.g. these listeners are notified when a child of this process exits.
291    child_process_event_listeners: RefCell<StateEventSource>,
292}
293
294impl RunnableProcess {
295    /// Spawn a `ManagedThread` corresponding to the given `exec` syscall
296    /// parameters.  Intended for use by the `exec` syscall handlers. Whether it
297    /// succeeds or fails, does *not* mutate `self`, though `self`'s strace and
298    /// shim log files will be passed into the new `ManagedThread`.
299    ///
300    /// In case the native `exec` syscall fails, the corresponding error is returned.
301    pub fn spawn_mthread_for_exec(
302        &self,
303        host: &Host,
304        plugin_path: &CStr,
305        argv: Vec<CString>,
306        envv: Vec<CString>,
307    ) -> Result<ManagedThread, Errno> {
308        ManagedThread::spawn(
309            plugin_path,
310            argv,
311            envv,
312            self.strace_logging
313                .as_ref()
314                .map(|s| s.file.borrow(host.root()))
315                .as_deref(),
316            &self.shimlog_file,
317            host.preload_paths(),
318        )
319    }
320
321    /// Call after a thread has exited. Removes the thread and does corresponding cleanup and notifications.
322    fn reap_thread(&self, host: &Host, threadrc: RootedRc<RootedRefCell<Thread>>) {
323        let threadrc = ExplicitDropper::new(threadrc, |t| {
324            t.explicit_drop_recursive(host.root(), host);
325        });
326        let thread = threadrc.borrow(host.root());
327
328        assert!(!thread.is_running());
329
330        // If the `clear_child_tid` attribute on the thread is set, and there are
331        // any other threads left alive in the process, perform a futex wake on
332        // that address. This mechanism is typically used in `pthread_join` etc.
333        // See `set_tid_address(2)`.
334        let clear_child_tid_pvp = thread.get_tid_address();
335        if !clear_child_tid_pvp.is_null() && !self.threads.borrow().is_empty() {
336            self.memory_manager
337                .borrow_mut()
338                .write(clear_child_tid_pvp, &0)
339                .unwrap();
340
341            // Wake the corresponding futex.
342            let futexes = host.futextable_borrow();
343            let addr = self
344                .common
345                .physical_address(clear_child_tid_pvp.cast::<()>());
346
347            if let Some(futex) = futexes.get(addr) {
348                futex.wake(1);
349            }
350        }
351    }
352
353    /// This cleans up memory references left over from legacy C code; usually
354    /// a syscall handler.
355    ///
356    /// Writes the leftover mutable ref to memory (if any), and frees
357    /// all memory refs.
358    pub fn free_unsafe_borrows_flush(&self) -> Result<(), Errno> {
359        self.unsafe_borrows.borrow_mut().clear();
360
361        let unsafe_borrow_mut = self.unsafe_borrow_mut.borrow_mut().take();
362        if let Some(borrow) = unsafe_borrow_mut {
363            borrow.flush()
364        } else {
365            Ok(())
366        }
367    }
368
369    /// This cleans up memory references left over from legacy C code; usually
370    /// a syscall handler.
371    ///
372    /// Frees all memory refs without writing back to memory.
373    pub fn free_unsafe_borrows_noflush(&self) {
374        self.unsafe_borrows.borrow_mut().clear();
375
376        let unsafe_borrow_mut = self.unsafe_borrow_mut.borrow_mut().take();
377        if let Some(borrow) = unsafe_borrow_mut {
378            borrow.noflush();
379        }
380    }
381
382    #[track_caller]
383    pub fn memory_borrow(&self) -> impl Deref<Target = MemoryManager> + '_ {
384        self.memory_manager.borrow()
385    }
386
387    #[track_caller]
388    pub fn memory_borrow_mut(&self) -> impl DerefMut<Target = MemoryManager> + '_ {
389        self.memory_manager.borrow_mut()
390    }
391
392    pub fn strace_logging_options(&self) -> Option<FmtOptions> {
393        self.strace_logging.as_ref().map(|x| x.options)
394    }
395
396    /// If strace logging is disabled, this function will do nothing and return `None`.
397    pub fn with_strace_file<T>(&self, f: impl FnOnce(&mut std::fs::File) -> T) -> Option<T> {
398        // TODO: get Host from caller. Would need t update syscall-logger.
399        Worker::with_active_host(|host| {
400            let strace_logging = self.strace_logging.as_ref()?;
401            let mut file = strace_logging.file.borrow_mut(host.root());
402            Some(f(&mut file))
403        })
404        .unwrap()
405    }
406
407    pub fn native_pid(&self) -> Pid {
408        self.native_pid
409    }
410
411    #[track_caller]
412    fn first_live_thread(&self, root: &Root) -> Option<Ref<'_, RootedRc<RootedRefCell<Thread>>>> {
413        Ref::filter_map(self.threads.borrow(), |threads| {
414            threads.values().next().inspect(|thread| {
415                // There shouldn't be any non-running threads in the table.
416                assert!(thread.borrow(root).is_running());
417            })
418        })
419        .ok()
420    }
421
422    /// Returns a dynamically borrowed reference to the first live thread.
423    /// This is meant primarily for the MemoryManager.
424    #[track_caller]
425    pub fn first_live_thread_borrow(
426        &self,
427        root: &Root,
428    ) -> Option<impl Deref<Target = RootedRc<RootedRefCell<Thread>>> + '_> {
429        self.first_live_thread(root)
430    }
431
432    #[track_caller]
433    fn thread(&self, virtual_tid: ThreadId) -> Option<Ref<'_, RootedRc<RootedRefCell<Thread>>>> {
434        Ref::filter_map(self.threads.borrow(), |threads| threads.get(&virtual_tid)).ok()
435    }
436
437    #[track_caller]
438    pub fn thread_borrow(
439        &self,
440        virtual_tid: ThreadId,
441    ) -> Option<impl Deref<Target = RootedRc<RootedRefCell<Thread>>> + '_> {
442        self.thread(virtual_tid)
443    }
444
445    // Disposes of `self`, returning the internal `Common` for reuse.
446    // Used internally when changing states.
447    fn into_common(self) -> Common {
448        // There shouldn't be any outstanding unsafe borrows when changing
449        // states, since that would indicate C code might still have a pointer
450        // to memory.
451        assert!(self.unsafe_borrow_mut.take().is_none());
452        assert!(self.unsafe_borrows.take().is_empty());
453
454        self.common
455    }
456
457    /// Starts the CPU delay timer.
458    /// Panics if the timer is already running.
459    #[cfg(feature = "perf_timers")]
460    pub fn start_cpu_delay_timer(&self) {
461        self.cpu_delay_timer.borrow_mut().start()
462    }
463
464    /// Stop the timer and return the most recent (not cumulative) duration.
465    /// Panics if the timer was not already running.
466    #[cfg(feature = "perf_timers")]
467    pub fn stop_cpu_delay_timer(&self, host: &Host) -> Duration {
468        let mut timer = self.cpu_delay_timer.borrow_mut();
469        timer.stop();
470        let total_elapsed = timer.elapsed();
471        let prev_total = self.total_run_time.replace(total_elapsed);
472        let delta = total_elapsed - prev_total;
473
474        host.cpu_borrow_mut().add_delay(delta);
475
476        delta
477    }
478
479    fn interrupt_with_signal(&self, host: &Host, signal: Signal) {
480        let threads = self.threads.borrow();
481        for thread in threads.values() {
482            let thread = thread.borrow(host.root());
483            {
484                let thread_shmem = thread.shmem();
485                let host_lock = host.shim_shmem_lock_borrow().unwrap();
486                let thread_shmem_protected = thread_shmem.protected.borrow(&host_lock.root);
487                let blocked_signals = thread_shmem_protected.blocked_signals;
488                if blocked_signals.has(signal) {
489                    continue;
490                }
491            }
492            let Some(mut cond) = thread.syscall_condition_mut() else {
493                // Defensively handle this gracefully, but it probably shouldn't happen.
494                // The only thread in the process not blocked on a syscall should be
495                // the current-running thread (if any), but the caller should have
496                // delivered the signal synchronously instead of using this function
497                // in that case.
498                warn!("thread {:?} has no syscall_condition. How?", thread.id());
499                continue;
500            };
501            cond.wakeup_for_signal(host, signal);
502            break;
503        }
504    }
505
506    /// Send the signal described in `siginfo` to `process`. `current_thread`
507    /// should be set if there is one (e.g. if this is being called from a syscall
508    /// handler), and `None` otherwise (e.g. when called from a timer expiration event).
509    ///
510    /// An event will be scheduled to deliver the signal unless `current_thread`
511    /// is set, and belongs to the process `self`, and doesn't have the signal
512    /// blocked.  In that the signal will be processed synchronously when
513    /// returning from the current syscall.
514    pub fn signal(&self, host: &Host, current_thread: Option<&Thread>, siginfo_t: &siginfo_t) {
515        let signal = match siginfo_t.signal() {
516            Ok(s) => s,
517            Err(SignalFromI32Error(0)) => return,
518            Err(SignalFromI32Error(n)) => panic!("Bad signo {n}"),
519        };
520
521        // Scope for `process_shmem_protected`
522        {
523            let host_shmem = host.shim_shmem_lock_borrow().unwrap();
524            let mut process_shmem_protected = self
525                .shim_shared_mem_block
526                .protected
527                .borrow_mut(&host_shmem.root);
528            // SAFETY: We don't try to call any of the function pointers.
529            let action = unsafe { process_shmem_protected.signal_action(signal) };
530            match unsafe { action.handler() } {
531                linux_api::signal::SignalHandler::Handler(_) => (),
532                linux_api::signal::SignalHandler::Action(_) => (),
533                linux_api::signal::SignalHandler::SigIgn => return,
534                linux_api::signal::SignalHandler::SigDfl => {
535                    if defaultaction(signal) == LinuxDefaultAction::IGN {
536                        return;
537                    }
538                }
539            }
540
541            if process_shmem_protected.pending_signals.has(signal) {
542                // Signal is already pending. From signal(7):In the case where a
543                // standard signal is already pending, the siginfo_t structure (see
544                // sigaction(2)) associated with that signal is not overwritten on
545                // arrival of subsequent instances of the same signal.
546                return;
547            }
548            process_shmem_protected.pending_signals.add(signal);
549            process_shmem_protected.set_pending_standard_siginfo(signal, siginfo_t);
550        }
551
552        if let Some(thread) = current_thread
553            && thread.process_id() == self.common.id()
554        {
555            let host_shmem = host.shim_shmem_lock_borrow().unwrap();
556            let threadmem = thread.shmem();
557            let threadprotmem = threadmem.protected.borrow(&host_shmem.root);
558            if !threadprotmem.blocked_signals.has(signal) {
559                // Target process is this process, and current thread hasn't blocked
560                // the signal.  It will be delivered to this thread when it resumes.
561                return;
562            }
563        }
564
565        self.interrupt_with_signal(host, signal);
566    }
567
568    /// Adds a new thread to the process and schedules it to run.
569    /// Intended for use by `clone`.
570    pub fn add_thread(&self, host: &Host, thread: RootedRc<RootedRefCell<Thread>>) {
571        let pid = self.common.id();
572        let tid = thread.borrow(host.root()).id();
573        self.threads.borrow_mut().insert(tid, thread);
574
575        // Schedule thread to start. We're giving the caller's reference to thread
576        // to the TaskRef here, which is why we don't increment its ref count to
577        // create the TaskRef, but do decrement it on cleanup.
578        let task = TaskRef::new(move |host| {
579            host.resume(pid, tid);
580        });
581        host.schedule_task_with_delay(task, SimulationTime::ZERO);
582    }
583
584    /// Create a new `Process`, forked from `self`, with the thread `new_thread_group_leader`.
585    pub fn new_forked_process(
586        &self,
587        host: &Host,
588        flags: CloneFlags,
589        exit_signal: Option<Signal>,
590        new_thread_group_leader: RootedRc<RootedRefCell<Thread>>,
591    ) -> RootedRc<RootedRefCell<Process>> {
592        let new_tgl_tid;
593        let native_pid;
594        {
595            let new_tgl = new_thread_group_leader.borrow(host.root());
596            new_tgl_tid = new_tgl.id();
597            native_pid = new_tgl.native_pid();
598        }
599        let pid = ProcessId::from_thread_group_leader_tid(new_tgl_tid);
600        assert_eq!(
601            pid,
602            new_thread_group_leader.borrow(host.root()).process_id()
603        );
604        let plugin_name = self.common.plugin_name.clone();
605        let name = make_name(host, plugin_name.to_str().unwrap(), pid);
606
607        let parent_pid = if flags.contains(CloneFlags::CLONE_PARENT) {
608            self.common.parent_pid.get()
609        } else {
610            self.common.id
611        };
612
613        // Process group is always inherited from the parent process.
614        let process_group_id = self.common.group_id.get();
615
616        // Session is always inherited from the parent process.
617        let session_id = self.common.session_id.get();
618
619        let common = Common {
620            id: pid,
621            host_id: host.id(),
622            name,
623            plugin_name,
624            working_dir: self.common.working_dir.clone(),
625            parent_pid: Cell::new(parent_pid),
626            group_id: Cell::new(process_group_id),
627            session_id: Cell::new(session_id),
628            exit_signal,
629            rlimits: self.common.rlimits,
630        };
631
632        // The child will log to the same strace log file. Entries contain thread IDs,
633        // though it might be tricky to map those back to processes.
634        let strace_logging = self.strace_logging.as_ref().cloned();
635
636        // `fork(2)`:
637        //  > The child does not inherit timers from its parent
638        //  > (setitimer(2), alarm(2), timer_create(2)).
639        let itimer_real = RefCell::new(Timer::new(move |host| itimer_real_expiration(host, pid)));
640
641        let threads = RefCell::new(BTreeMap::from([(new_tgl_tid, new_thread_group_leader)]));
642
643        let shim_shared_mem = ProcessShmem::new(
644            &host.shim_shmem_lock_borrow().unwrap().root,
645            host.shim_shmem().serialize(),
646            host.id(),
647            strace_logging
648                .as_ref()
649                .map(|x| x.file.borrow(host.root()).as_raw_fd()),
650        );
651        let shim_shared_mem_block = shadow_shmem::allocator::shmalloc(shim_shared_mem);
652
653        let runnable_process = RunnableProcess {
654            common,
655            expected_final_state: None,
656            shim_shared_mem_block,
657            strace_logging,
658            dumpable: self.dumpable.clone(),
659            native_pid,
660            #[cfg(feature = "perf_timers")]
661            cpu_delay_timer: RefCell::new(PerfTimer::new_stopped()),
662            #[cfg(feature = "perf_timers")]
663            total_run_time: Cell::new(Duration::ZERO),
664            itimer_real,
665            threads,
666            unsafe_borrow_mut: RefCell::new(None),
667            unsafe_borrows: RefCell::new(Vec::new()),
668            memory_manager: Box::new(RefCell::new(unsafe { MemoryManager::new(native_pid) })),
669            child_process_event_listeners: Default::default(),
670            shimlog_file: self.shimlog_file.clone(),
671        };
672        let child_process = Process {
673            state: RefCell::new(Some(ProcessState::Runnable(runnable_process))),
674        };
675        RootedRc::new(host.root(), RootedRefCell::new(host.root(), child_process))
676    }
677
678    /// Shared memory for this process.
679    pub fn shmem(&self) -> impl Deref<Target = ShMemBlock<'static, ProcessShmem>> + '_ {
680        &self.shim_shared_mem_block
681    }
682}
683
684impl ExplicitDrop for RunnableProcess {
685    type ExplicitDropParam = Host;
686    type ExplicitDropResult = ();
687
688    fn explicit_drop(mut self, host: &Self::ExplicitDropParam) -> Self::ExplicitDropResult {
689        let threads = std::mem::take(self.threads.get_mut());
690        for thread in threads.into_values() {
691            thread.explicit_drop_recursive(host.root(), host);
692        }
693    }
694}
695
696/// A process that has exited.
697pub struct ZombieProcess {
698    common: Common,
699
700    exit_status: ExitStatus,
701}
702
703impl ZombieProcess {
704    pub fn exit_status(&self) -> ExitStatus {
705        self.exit_status
706    }
707
708    /// Process that can reap this zombie process, if any.
709    pub fn reaper<'host>(
710        &self,
711        host: &'host Host,
712    ) -> Option<impl Deref<Target = RootedRc<RootedRefCell<Process>>> + 'host> {
713        let parent_pid = self.common.parent_pid.get();
714        if parent_pid == ProcessId::INIT {
715            return None;
716        }
717        let parentrc = host.process_borrow(parent_pid)?;
718
719        // If the parent has *explicitly* ignored the exit signal, then it
720        // doesn't reap.
721        //
722        // `waitpid(2)`:
723        // > POSIX.1-2001 specifies that if the disposition of SIGCHLD is set to SIG_IGN or the SA_NOCLDWAIT flag is set for SIGCHLD  (see
724        // > sigaction(2)),  then  children  that  terminate  do not become zombies and a call to wait() or waitpid() will block until all
725        // > children have terminated, and then fail with errno set to ECHILD.  (The original POSIX standard left the behavior of  setting
726        // > SIGCHLD to SIG_IGN unspecified.  Note that even though the default disposition of SIGCHLD is "ignore", explicitly setting the
727        // > disposition to SIG_IGN results in different treatment of zombie process children.)
728        //
729        // TODO: validate that this applies to whatever signal is configured as the exit
730        // signal, even if it's not SIGCHLD.
731        if let Some(exit_signal) = self.common.exit_signal {
732            let parent = parentrc.borrow(host.root());
733            let parent_shmem = parent.shmem();
734            let host_shmem_lock = host.shim_shmem_lock_borrow().unwrap();
735            let parent_shmem_protected = parent_shmem.protected.borrow(&host_shmem_lock.root);
736            // SAFETY: We don't dereference function pointers.
737            let action = unsafe { parent_shmem_protected.signal_action(exit_signal) };
738            if action.is_ignore() {
739                return None;
740            }
741        }
742
743        Some(parentrc)
744    }
745
746    fn notify_parent_of_exit(&self, host: &Host) {
747        let Some(exit_signal) = self.common.exit_signal else {
748            trace!("Not notifying parent of exit: no signal specified");
749            return;
750        };
751        let parent_pid = self.common.parent_pid.get();
752        if parent_pid == ProcessId::INIT {
753            trace!("Not notifying parent of exit: parent is 'init'");
754            return;
755        }
756        let Some(parent_rc) = host.process_borrow(parent_pid) else {
757            trace!("Not notifying parent of exit: parent {parent_pid:?} not found");
758            return;
759        };
760        let parent = parent_rc.borrow(host.root());
761        let siginfo = self.exit_siginfo(exit_signal);
762
763        let Some(parent_runnable) = parent.as_runnable() else {
764            trace!("Not notifying parent of exit: {parent_pid:?} not running");
765            debug_panic!("Non-running parent process shouldn't be possible.");
766            #[allow(unreachable_code)]
767            {
768                return;
769            }
770        };
771        parent_runnable.signal(host, None, &siginfo);
772        CallbackQueue::queue_and_run_with_legacy(|q| {
773            let mut parent_child_listeners =
774                parent_runnable.child_process_event_listeners.borrow_mut();
775            parent_child_listeners.notify_listeners(
776                FileState::CHILD_EVENT,
777                FileState::CHILD_EVENT,
778                FileSignals::empty(),
779                q,
780            );
781        });
782    }
783
784    /// Construct a siginfo containing information about how the process exited.
785    /// Used internally to send a signal to the parent process, and by the
786    /// `waitid` syscall handler.
787    ///
788    /// `exit_signal` is the signal to set in the `siginfo_t`.
789    pub fn exit_siginfo(&self, exit_signal: Signal) -> siginfo_t {
790        match self.exit_status {
791            ExitStatus::Normal(exit_code) => siginfo_t::new_for_sigchld_exited(
792                exit_signal,
793                self.common.id.into(),
794                0,
795                exit_code,
796                0,
797                0,
798            ),
799            ExitStatus::Signaled(fatal_signal) => {
800                // This ought to be `siginfo_t::new_for_sigchld_dumped` if
801                // the child dumped core, but that depends on various other
802                // system variables outside of our control. We always report
803                // that no core was dropped for determinism.
804                siginfo_t::new_for_sigchld_killed(
805                    exit_signal,
806                    self.common.id.into(),
807                    0,
808                    fatal_signal,
809                    0,
810                    0,
811                )
812            }
813
814            ExitStatus::StoppedByShadow => unreachable!(),
815        }
816    }
817}
818
819/// Inner implementation of a simulated process.
820// We could box the variants, but it's unclear whether it's really worth the extra code and extra
821// allocations. Most of the values of this type will be in the larger `Runnable` variant rather than
822// the smaller `Zombie` variant anyways.
823#[allow(clippy::large_enum_variant)]
824enum ProcessState {
825    Runnable(RunnableProcess),
826    Zombie(ZombieProcess),
827}
828
829impl ProcessState {
830    fn common(&self) -> &Common {
831        match self {
832            ProcessState::Runnable(r) => &r.common,
833            ProcessState::Zombie(z) => &z.common,
834        }
835    }
836
837    fn common_mut(&mut self) -> &mut Common {
838        match self {
839            ProcessState::Runnable(r) => &mut r.common,
840            ProcessState::Zombie(z) => &mut z.common,
841        }
842    }
843
844    fn as_runnable(&self) -> Option<&RunnableProcess> {
845        match self {
846            ProcessState::Runnable(r) => Some(r),
847            ProcessState::Zombie(_) => None,
848        }
849    }
850
851    fn as_runnable_mut(&mut self) -> Option<&mut RunnableProcess> {
852        match self {
853            ProcessState::Runnable(r) => Some(r),
854            ProcessState::Zombie(_) => None,
855        }
856    }
857
858    fn as_zombie(&self) -> Option<&ZombieProcess> {
859        match self {
860            ProcessState::Runnable(_) => None,
861            ProcessState::Zombie(z) => Some(z),
862        }
863    }
864}
865
866impl ExplicitDrop for ProcessState {
867    type ExplicitDropParam = Host;
868    type ExplicitDropResult = ();
869
870    fn explicit_drop(self, host: &Self::ExplicitDropParam) -> Self::ExplicitDropResult {
871        match self {
872            ProcessState::Runnable(r) => r.explicit_drop(host),
873            ProcessState::Zombie(_) => (),
874        }
875    }
876}
877
878/// A simulated process.
879pub struct Process {
880    // Most of the implementation should be in [`ProcessState`].
881    // This wrapper allows us to change the state.
882    state: RefCell<Option<ProcessState>>,
883}
884
885fn itimer_real_expiration(host: &Host, pid: ProcessId) {
886    let Some(process) = host.process_borrow(pid) else {
887        debug!("Process {pid:?} no longer exists");
888        return;
889    };
890    let process = process.borrow(host.root());
891    let Some(runnable) = process.as_runnable() else {
892        debug!("Process {:?} no longer running", &*process.name());
893        return;
894    };
895    let timer = runnable.itimer_real.borrow();
896    // The siginfo_t structure only has an i32. Presumably we want to just truncate in
897    // case of overflow.
898    let expiration_count = timer.expiration_count() as i32;
899    let siginfo_t = siginfo_t::new_for_timer(Signal::SIGALRM, 0, expiration_count);
900    process.signal(host, None, &siginfo_t);
901}
902
903impl Process {
904    fn common(&self) -> Ref<'_, Common> {
905        Ref::map(self.state.borrow(), |state| {
906            state.as_ref().unwrap().common()
907        })
908    }
909
910    fn common_mut(&self) -> RefMut<'_, Common> {
911        RefMut::map(self.state.borrow_mut(), |state| {
912            state.as_mut().unwrap().common_mut()
913        })
914    }
915
916    fn as_runnable(&self) -> Option<Ref<'_, RunnableProcess>> {
917        Ref::filter_map(self.state.borrow(), |state| {
918            state.as_ref().unwrap().as_runnable()
919        })
920        .ok()
921    }
922
923    fn as_runnable_mut(&self) -> Option<RefMut<'_, RunnableProcess>> {
924        RefMut::filter_map(self.state.borrow_mut(), |state| {
925            state.as_mut().unwrap().as_runnable_mut()
926        })
927        .ok()
928    }
929
930    /// Borrows a reference to the internal [`RunnableProcess`] if `self` is runnable.
931    pub fn borrow_as_runnable(&self) -> Option<impl Deref<Target = RunnableProcess> + '_> {
932        self.as_runnable()
933    }
934
935    fn as_zombie(&self) -> Option<Ref<'_, ZombieProcess>> {
936        Ref::filter_map(self.state.borrow(), |state| {
937            state.as_ref().unwrap().as_zombie()
938        })
939        .ok()
940    }
941
942    /// Borrows a reference to the internal [`ZombieProcess`] if `self` is a zombie.
943    pub fn borrow_as_zombie(&self) -> Option<impl Deref<Target = ZombieProcess> + '_> {
944        self.as_zombie()
945    }
946
947    /// Spawn a new process. The process will be runnable via [`Self::resume`]
948    /// once it has been added to the `Host`'s process list.
949    pub fn spawn(
950        host: &Host,
951        plugin_name: CString,
952        plugin_path: &CStr,
953        argv: Vec<CString>,
954        envv: Vec<CString>,
955        pause_for_debugging: bool,
956        strace_logging_options: Option<FmtOptions>,
957        expected_final_state: ProcessFinalState,
958    ) -> Result<RootedRc<RootedRefCell<Process>>, Errno> {
959        debug!("starting process '{plugin_name:?}'");
960
961        let main_thread_id = host.get_new_thread_id();
962        let process_id = ProcessId::from(main_thread_id);
963
964        let desc_table = RootedRc::new(
965            host.root(),
966            RootedRefCell::new(host.root(), DescriptorTable::new()),
967        );
968        let itimer_real = RefCell::new(Timer::new(move |host| {
969            itimer_real_expiration(host, process_id)
970        }));
971
972        let name = make_name(host, plugin_name.to_str().unwrap(), process_id);
973
974        let mut file_basename = PathBuf::new();
975        file_basename.push(host.data_dir_path());
976        file_basename.push(format!(
977            "{exe_name}.{id}",
978            exe_name = plugin_name.to_str().unwrap(),
979            id = u32::from(process_id)
980        ));
981
982        let strace_logging = strace_logging_options.map(|options| {
983            let file =
984                std::fs::File::create(Self::static_output_file_name(&file_basename, "strace"))
985                    .unwrap();
986            debug_assert_cloexec(&file);
987            Arc::new(StraceLogging {
988                file: RootedRefCell::new(host.root(), file),
989                options,
990            })
991        });
992
993        let shim_shared_mem = ProcessShmem::new(
994            &host.shim_shmem_lock_borrow().unwrap().root,
995            host.shim_shmem().serialize(),
996            host.id(),
997            strace_logging
998                .as_ref()
999                .map(|x| x.file.borrow(host.root()).as_raw_fd()),
1000        );
1001        let shim_shared_mem_block = shadow_shmem::allocator::shmalloc(shim_shared_mem);
1002
1003        let working_dir = utility::pathbuf_to_nul_term_cstring(
1004            std::fs::canonicalize(host.data_dir_path()).unwrap(),
1005        );
1006
1007        {
1008            let mut descriptor_table = desc_table.borrow_mut(host.root());
1009            Self::open_stdio_file_helper(
1010                &mut descriptor_table,
1011                libc::STDIN_FILENO.try_into().unwrap(),
1012                "/dev/null".into(),
1013                OFlag::O_RDONLY,
1014            );
1015
1016            let name = Self::static_output_file_name(&file_basename, "stdout");
1017            Self::open_stdio_file_helper(
1018                &mut descriptor_table,
1019                libc::STDOUT_FILENO.try_into().unwrap(),
1020                name,
1021                OFlag::O_WRONLY,
1022            );
1023
1024            let name = Self::static_output_file_name(&file_basename, "stderr");
1025            Self::open_stdio_file_helper(
1026                &mut descriptor_table,
1027                libc::STDERR_FILENO.try_into().unwrap(),
1028                name,
1029                OFlag::O_WRONLY,
1030            );
1031        }
1032
1033        let shimlog_file = Arc::new(
1034            std::fs::File::create(Self::static_output_file_name(&file_basename, "shimlog"))
1035                .unwrap(),
1036        );
1037        debug_assert_cloexec(&shimlog_file);
1038
1039        let mthread = ManagedThread::spawn(
1040            plugin_path,
1041            argv,
1042            envv,
1043            strace_logging
1044                .as_ref()
1045                .map(|s| s.file.borrow(host.root()))
1046                .as_deref(),
1047            &shimlog_file,
1048            host.preload_paths(),
1049        )?;
1050        let native_pid = mthread.native_pid();
1051        let main_thread =
1052            Thread::wrap_mthread(host, mthread, desc_table, process_id, main_thread_id).unwrap();
1053
1054        debug!("process '{plugin_name:?}' started");
1055
1056        if pause_for_debugging {
1057            // will block until logger output has been flushed
1058            // there is a race condition where other threads may log between the
1059            // `eprintln` and `raise` below, but it should be rare
1060            log::logger().flush();
1061
1062            // Use a single `eprintln` to ensure we hold the lock for the whole message.
1063            // Defensively pre-construct a single string so that `eprintln` is
1064            // more likely to use a single `write` call, to minimize the chance
1065            // of more lines being written to stdout in the meantime, and in
1066            // case of C code writing to `STDERR` directly without taking Rust's
1067            // lock.
1068            let msg = format!(
1069                "\
1070              \n** Pausing with SIGTSTP to enable debugger attachment to managed process\
1071              \n** '{plugin_name:?}' (pid {native_pid:?}).\
1072              \n** If running Shadow under Bash, resume Shadow by pressing Ctrl-Z to background\
1073              \n** this task, and then typing \"fg\".\
1074              \n** If running GDB, resume Shadow by typing \"signal SIGCONT\"."
1075            );
1076            eprintln!("{msg}");
1077
1078            rustix::process::kill_process(rustix::process::getpid(), rustix::process::Signal::Tstp)
1079                .unwrap();
1080        }
1081
1082        // Initialize emulated rlimits to their native values.
1083        // TODO: For determinism, we may want to use fixed limits for some or all of these.
1084        // Some applications like Tor will change behavior depending on these limits.
1085        // Ultimately they'd need to be compatible with the native limits though.
1086        let mut rlimits: [linux_api::resource::rlimit64; _] =
1087            [shadow_pod::zeroed(); linux_api::resource::RLIM_NLIMITS as usize];
1088        for r in 0..linux_api::resource::RLIM_NLIMITS {
1089            let r = linux_api::resource::Resource::try_from(r).unwrap();
1090            // SAFETY: target process isn't our own, and we're only retrieving
1091            // limits, not changing them.
1092            unsafe {
1093                linux_api::resource::prlimit64(
1094                    native_pid,
1095                    r,
1096                    None,
1097                    Some(&mut rlimits[usize::try_from(u32::from(r)).unwrap()]),
1098                )
1099            }
1100            .unwrap();
1101        }
1102
1103        let memory_manager = unsafe { MemoryManager::new(native_pid) };
1104        let threads = RefCell::new(BTreeMap::from([(
1105            main_thread_id,
1106            RootedRc::new(host.root(), RootedRefCell::new(host.root(), main_thread)),
1107        )]));
1108
1109        let common = Common {
1110            id: process_id,
1111            host_id: host.id(),
1112            working_dir,
1113            name,
1114            plugin_name,
1115            parent_pid: Cell::new(ProcessId::INIT),
1116            group_id: Cell::new(ProcessId::INIT),
1117            session_id: Cell::new(ProcessId::INIT),
1118            // Exit signal is moot; since parent is INIT there will never
1119            // be a valid target for it.
1120            exit_signal: None,
1121            rlimits,
1122        };
1123        Ok(RootedRc::new(
1124            host.root(),
1125            RootedRefCell::new(
1126                host.root(),
1127                Self {
1128                    state: RefCell::new(Some(ProcessState::Runnable(RunnableProcess {
1129                        common,
1130                        expected_final_state: Some(expected_final_state),
1131                        shim_shared_mem_block,
1132                        memory_manager: Box::new(RefCell::new(memory_manager)),
1133                        itimer_real,
1134                        strace_logging,
1135                        dumpable: Cell::new(SuidDump::SUID_DUMP_USER),
1136                        native_pid,
1137                        unsafe_borrow_mut: RefCell::new(None),
1138                        unsafe_borrows: RefCell::new(Vec::new()),
1139                        threads,
1140                        #[cfg(feature = "perf_timers")]
1141                        cpu_delay_timer: RefCell::new(PerfTimer::new_stopped()),
1142                        #[cfg(feature = "perf_timers")]
1143                        total_run_time: Cell::new(Duration::ZERO),
1144                        child_process_event_listeners: Default::default(),
1145                        shimlog_file,
1146                    }))),
1147                },
1148            ),
1149        ))
1150    }
1151
1152    pub fn id(&self) -> ProcessId {
1153        self.common().id
1154    }
1155
1156    pub fn parent_id(&self) -> ProcessId {
1157        self.common().parent_pid.get()
1158    }
1159
1160    pub fn set_parent_id(&self, pid: ProcessId) {
1161        self.common().parent_pid.set(pid)
1162    }
1163
1164    pub fn group_id(&self) -> ProcessId {
1165        self.common().group_id.get()
1166    }
1167
1168    pub fn set_group_id(&self, id: ProcessId) {
1169        self.common().group_id.set(id)
1170    }
1171
1172    pub fn session_id(&self) -> ProcessId {
1173        self.common().session_id.get()
1174    }
1175
1176    pub fn set_session_id(&self, id: ProcessId) {
1177        self.common().session_id.set(id)
1178    }
1179
1180    pub fn host_id(&self) -> HostId {
1181        self.common().host_id
1182    }
1183
1184    /// Get process's "dumpable" state, as manipulated by the prctl operations `PR_SET_DUMPABLE` and
1185    /// `PR_GET_DUMPABLE`.
1186    pub fn dumpable(&self) -> SuidDump {
1187        self.as_runnable().unwrap().dumpable.get()
1188    }
1189
1190    /// Set process's "dumpable" state, as manipulated by the prctl operations `PR_SET_DUMPABLE` and
1191    /// `PR_GET_DUMPABLE`.
1192    pub fn set_dumpable(&self, val: SuidDump) {
1193        assert!(val == SuidDump::SUID_DUMP_DISABLE || val == SuidDump::SUID_DUMP_USER);
1194        self.as_runnable().unwrap().dumpable.set(val)
1195    }
1196
1197    /// Deprecated wrapper for `RunnableProcess::start_cpu_delay_timer`
1198    #[cfg(feature = "perf_timers")]
1199    pub fn start_cpu_delay_timer(&self) {
1200        self.as_runnable().unwrap().start_cpu_delay_timer()
1201    }
1202
1203    /// Deprecated wrapper for `RunnableProcess::stop_cpu_delay_timer`
1204    #[cfg(feature = "perf_timers")]
1205    pub fn stop_cpu_delay_timer(&self, host: &Host) -> Duration {
1206        self.as_runnable().unwrap().stop_cpu_delay_timer(host)
1207    }
1208
1209    pub fn thread_group_leader_id(&self) -> ThreadId {
1210        self.common().thread_group_leader_id()
1211    }
1212
1213    /// Resume execution of `tid` (if it exists).
1214    /// Should only be called from `Host::resume`.
1215    pub fn resume(&self, host: &Host, tid: ThreadId) {
1216        trace!("Continuing thread {} in process {}", tid, self.id());
1217
1218        let threadrc = {
1219            let Some(runnable) = self.as_runnable() else {
1220                debug!("Process {} is no longer running", &*self.name());
1221                return;
1222            };
1223            let threads = runnable.threads.borrow();
1224            let Some(thread) = threads.get(&tid) else {
1225                debug!("Thread {tid} no longer exists");
1226                return;
1227            };
1228            // Clone the thread reference, so that we don't hold a dynamically
1229            // borrowed reference to the thread list while running the thread.
1230            thread.clone(host.root())
1231        };
1232        let threadrc = ExplicitDropper::new(threadrc, |t| {
1233            t.explicit_drop_recursive(host.root(), host);
1234        });
1235        let thread = threadrc.borrow(host.root());
1236
1237        Worker::set_active_thread(&threadrc);
1238
1239        #[cfg(feature = "perf_timers")]
1240        self.start_cpu_delay_timer();
1241
1242        Process::set_shared_time(host);
1243
1244        // Discard any unapplied latency.
1245        // We currently only want this mechanism to force a yield if the thread itself
1246        // never yields; we don't want unapplied latency to accumulate and force a yield
1247        // under normal circumstances.
1248        host.shim_shmem_lock_borrow_mut()
1249            .unwrap()
1250            .unapplied_cpu_latency = SimulationTime::ZERO;
1251
1252        let ctx = ProcessContext::new(host, self);
1253        let res = thread.resume(&ctx);
1254
1255        #[cfg(feature = "perf_timers")]
1256        {
1257            let delay = self.stop_cpu_delay_timer(host);
1258            debug!("process '{}' ran for {:?}", &*self.name(), delay);
1259        }
1260        #[cfg(not(feature = "perf_timers"))]
1261        debug!("process '{}' done continuing", &*self.name());
1262
1263        match res {
1264            crate::host::thread::ResumeResult::Blocked => {
1265                debug!(
1266                    "thread {tid} in process '{}' still running, but blocked",
1267                    &*self.name()
1268                );
1269            }
1270            crate::host::thread::ResumeResult::ExitedThread(return_code) => {
1271                debug!(
1272                    "thread {tid} in process '{}' exited with code {return_code}",
1273                    &*self.name(),
1274                );
1275                let (threadrc, last_thread) = {
1276                    let runnable = self.as_runnable().unwrap();
1277                    let mut threads = runnable.threads.borrow_mut();
1278                    let threadrc = threads.remove(&tid).unwrap();
1279                    (threadrc, threads.is_empty())
1280                };
1281                self.as_runnable().unwrap().reap_thread(host, threadrc);
1282                if last_thread {
1283                    self.handle_process_exit(host, false);
1284                }
1285            }
1286            crate::host::thread::ResumeResult::ExitedProcess => {
1287                debug!(
1288                    "Process {} exited while running thread {tid}",
1289                    &*self.name(),
1290                );
1291                self.handle_process_exit(host, false);
1292            }
1293        };
1294
1295        Worker::clear_active_thread();
1296    }
1297
1298    /// Terminate the Process.
1299    ///
1300    /// Should only be called from [`Host::free_all_applications`].
1301    pub fn stop(&self, host: &Host) {
1302        // Scope for `runnable`
1303        {
1304            let Some(runnable) = self.as_runnable() else {
1305                debug!("process {} has already stopped", &*self.name());
1306                return;
1307            };
1308            debug!("terminating process {}", &*self.name());
1309
1310            #[cfg(feature = "perf_timers")]
1311            runnable.start_cpu_delay_timer();
1312
1313            if let Err(err) = rustix::process::kill_process(
1314                runnable.native_pid().into(),
1315                rustix::process::Signal::Kill,
1316            ) {
1317                warn!("kill: {err:?}");
1318            }
1319
1320            #[cfg(feature = "perf_timers")]
1321            {
1322                let delay = runnable.stop_cpu_delay_timer(host);
1323                debug!("process '{}' stopped in {:?}", &*self.name(), delay);
1324            }
1325            #[cfg(not(feature = "perf_timers"))]
1326            debug!("process '{}' stopped", &*self.name());
1327        }
1328
1329        // Mutates `self.state`, so we need to have dropped `runnable`.
1330        self.handle_process_exit(host, true);
1331    }
1332
1333    /// See `RunnableProcess::signal`.
1334    ///
1335    /// No-op if the `self` is a `ZombieProcess`.
1336    pub fn signal(&self, host: &Host, current_thread: Option<&Thread>, siginfo_t: &siginfo_t) {
1337        // Using full-match here to force update if we add more states later.
1338        match self.state.borrow().as_ref().unwrap() {
1339            ProcessState::Runnable(r) => r.signal(host, current_thread, siginfo_t),
1340            ProcessState::Zombie(_) => {
1341                // Sending a signal to a zombie process is a no-op.
1342                debug!("Process {} no longer running", &*self.name());
1343            }
1344        }
1345    }
1346
1347    fn open_stdio_file_helper(
1348        descriptor_table: &mut DescriptorTable,
1349        fd: DescriptorHandle,
1350        path: PathBuf,
1351        access_mode: OFlag,
1352    ) {
1353        let stdfile = unsafe { cshadow::regularfile_new() };
1354        let cwd = rustix::process::getcwd(Vec::new()).unwrap();
1355        let path = utility::pathbuf_to_nul_term_cstring(path);
1356        // "Convert" to libc int, assuming here that the kernel's `OFlag` values
1357        // are compatible with libc's values.
1358        // XXX: We're assuming here that the kernel and libc flags are ABI
1359        // compatible, which isn't guaranteed, but is mostly true in practice.
1360        // TODO: We probably ought to change `regularfile_open` and friends to
1361        // use a direct syscall instead of libc's wrappers, and explicitly take
1362        // the kernel version of flags, mode, etc.
1363        let access_mode = access_mode.bits();
1364        let errorcode = unsafe {
1365            cshadow::regularfile_open(
1366                stdfile,
1367                path.as_ptr(),
1368                access_mode | libc::O_CREAT | libc::O_TRUNC,
1369                libc::S_IRUSR | libc::S_IWUSR | libc::S_IRGRP | libc::S_IROTH,
1370                cwd.as_ptr(),
1371            )
1372        };
1373        if errorcode != 0 {
1374            panic!(
1375                "Opening {}: {:?}",
1376                path.to_str().unwrap(),
1377                linux_api::errno::Errno::try_from(-errorcode).unwrap()
1378            );
1379        }
1380        let desc = unsafe {
1381            Descriptor::from_legacy_file(
1382                stdfile as *mut cshadow::LegacyFile,
1383                linux_api::fcntl::OFlag::empty(),
1384            )
1385        };
1386        let prev = descriptor_table.register_descriptor_with_fd(desc, fd);
1387        assert!(prev.is_none());
1388        trace!(
1389            "Successfully opened fd {} at {}",
1390            fd,
1391            path.to_str().unwrap()
1392        );
1393    }
1394
1395    // Needed during early init, before `Self` is created.
1396    fn static_output_file_name(file_basename: &Path, extension: &str) -> PathBuf {
1397        let mut path = file_basename.to_owned().into_os_string();
1398        path.push(".");
1399        path.push(extension);
1400        path.into()
1401    }
1402
1403    pub fn name(&self) -> impl Deref<Target = str> + '_ {
1404        Ref::map(self.common(), |c| c.name.to_str().unwrap())
1405    }
1406
1407    pub fn plugin_name(&self) -> impl Deref<Target = str> + '_ {
1408        Ref::map(self.common(), |c| c.plugin_name.to_str().unwrap())
1409    }
1410
1411    /// Deprecated wrapper for `RunnableProcess::memory_borrow_mut`
1412    #[track_caller]
1413    pub fn memory_borrow_mut(&self) -> impl DerefMut<Target = MemoryManager> + '_ {
1414        std_util::nested_ref::NestedRefMut::map(self.as_runnable().unwrap(), |runnable| {
1415            runnable.memory_manager.borrow_mut()
1416        })
1417    }
1418
1419    /// Deprecated wrapper for `RunnableProcess::memory_borrow`
1420    #[track_caller]
1421    pub fn memory_borrow(&self) -> impl Deref<Target = MemoryManager> + '_ {
1422        std_util::nested_ref::NestedRef::map(self.as_runnable().unwrap(), |runnable| {
1423            runnable.memory_manager.borrow()
1424        })
1425    }
1426
1427    /// Deprecated wrapper for `RunnableProcess::strace_logging_options`
1428    pub fn strace_logging_options(&self) -> Option<FmtOptions> {
1429        self.as_runnable().unwrap().strace_logging_options()
1430    }
1431
1432    /// Deprecated wrapper for `RunnableProcess::with_strace_file`
1433    pub fn with_strace_file<T>(&self, f: impl FnOnce(&mut std::fs::File) -> T) -> Option<T> {
1434        self.as_runnable().unwrap().with_strace_file(f)
1435    }
1436
1437    /// Deprecated wrapper for `RunnableProcess::native_pid`
1438    pub fn native_pid(&self) -> Pid {
1439        self.as_runnable().unwrap().native_pid()
1440    }
1441
1442    /// Deprecated wrapper for `RunnableProcess::realtime_timer_borrow`
1443    #[track_caller]
1444    pub fn realtime_timer_borrow(&self) -> impl Deref<Target = Timer> + '_ {
1445        std_util::nested_ref::NestedRef::map(self.as_runnable().unwrap(), |runnable| {
1446            runnable.itimer_real.borrow()
1447        })
1448    }
1449
1450    /// Deprecated wrapper for `RunnableProcess::realtime_timer_borrow_mut`
1451    #[track_caller]
1452    pub fn realtime_timer_borrow_mut(&self) -> impl DerefMut<Target = Timer> + '_ {
1453        std_util::nested_ref::NestedRefMut::map(self.as_runnable().unwrap(), |runnable| {
1454            runnable.itimer_real.borrow_mut()
1455        })
1456    }
1457
1458    /// Deprecated wrapper for `RunnableProcess::first_live_thread_borrow`
1459    #[track_caller]
1460    pub fn first_live_thread_borrow(
1461        &self,
1462        root: &Root,
1463    ) -> Option<impl Deref<Target = RootedRc<RootedRefCell<Thread>>> + '_> {
1464        std_util::nested_ref::NestedRef::filter_map(self.as_runnable()?, |runnable| {
1465            runnable.first_live_thread(root)
1466        })
1467    }
1468
1469    /// Deprecated wrapper for `RunnableProcess::thread_borrow`
1470    pub fn thread_borrow(
1471        &self,
1472        virtual_tid: ThreadId,
1473    ) -> Option<impl Deref<Target = RootedRc<RootedRefCell<Thread>>> + '_> {
1474        std_util::nested_ref::NestedRef::filter_map(self.as_runnable()?, |runnable| {
1475            runnable.thread(virtual_tid)
1476        })
1477    }
1478
1479    /// Deprecated wrapper for [`RunnableProcess::free_unsafe_borrows_flush`].
1480    pub fn free_unsafe_borrows_flush(&self) -> Result<(), Errno> {
1481        self.as_runnable().unwrap().free_unsafe_borrows_flush()
1482    }
1483
1484    /// Deprecated wrapper for [`RunnableProcess::free_unsafe_borrows_noflush`].
1485    pub fn free_unsafe_borrows_noflush(&self) {
1486        self.as_runnable().unwrap().free_unsafe_borrows_noflush()
1487    }
1488
1489    pub fn physical_address(&self, vptr: ForeignPtr<()>) -> ManagedPhysicalMemoryAddr {
1490        self.common().physical_address(vptr)
1491    }
1492
1493    pub fn is_running(&self) -> bool {
1494        self.as_runnable().is_some()
1495    }
1496
1497    /// Transitions `self` from a `RunnableProcess` to a `ZombieProcess`.
1498    fn handle_process_exit(&self, host: &Host, killed_by_shadow: bool) {
1499        debug!(
1500            "process '{}' has completed or is otherwise no longer running",
1501            &*self.name()
1502        );
1503
1504        // Take and dispose of all of the threads.
1505        // TODO: consider doing this while the `self.state` mutable reference is held
1506        // as with the other cleanup below. Right now this breaks some C code that expects
1507        // to be able to lookup the thread's process name.
1508        {
1509            let runnable = self.as_runnable().unwrap();
1510            let threads = std::mem::take(&mut *runnable.threads.borrow_mut());
1511            for (_tid, threadrc) in threads.into_iter() {
1512                threadrc.borrow(host.root()).handle_process_exit();
1513                runnable.reap_thread(host, threadrc);
1514            }
1515        }
1516
1517        // Intentionally hold the borrow on self.state to ensure the state
1518        // transition is "atomic".
1519        let mut opt_state = self.state.borrow_mut();
1520
1521        let state = opt_state.take().unwrap();
1522        let ProcessState::Runnable(runnable) = state else {
1523            unreachable!("Tried to handle process exit of non-running process");
1524        };
1525
1526        #[cfg(feature = "perf_timers")]
1527        debug!(
1528            "total runtime for process '{}' was {:?}",
1529            runnable.common.name(),
1530            runnable.total_run_time.get()
1531        );
1532
1533        let wait_res: Option<WaitStatus> =
1534            rustix::process::waitpid(Some(runnable.native_pid().into()), WaitOptions::empty())
1535                .unwrap_or_else(|e| {
1536                    panic!("Error waiting for {:?}: {:?}", runnable.native_pid(), e)
1537                });
1538        let wait_status = wait_res.unwrap();
1539        let exit_status = if killed_by_shadow {
1540            if wait_status.terminating_signal()
1541                != Some(Signal::SIGKILL.as_i32().try_into().unwrap())
1542            {
1543                warn!("Unexpected waitstatus after killed by shadow: {wait_status:?}");
1544            }
1545            ExitStatus::StoppedByShadow
1546        } else if let Some(code) = wait_status.exit_status() {
1547            ExitStatus::Normal(code.try_into().unwrap())
1548        } else if let Some(signal) = wait_status.terminating_signal() {
1549            ExitStatus::Signaled(Signal::try_from(i32::try_from(signal).unwrap()).unwrap())
1550        } else {
1551            panic!(
1552                "Unexpected status: {wait_status:?} for pid {:?}",
1553                runnable.native_pid()
1554            );
1555        };
1556
1557        let (main_result_string, log_level) = {
1558            let mut s = format!(
1559                "process '{name}' exited with status {exit_status:?}",
1560                name = runnable.common.name()
1561            );
1562            if let Some(expected_final_state) = runnable.expected_final_state {
1563                let actual_final_state = match exit_status {
1564                    ExitStatus::Normal(i) => ProcessFinalState::Exited { exited: i },
1565                    ExitStatus::Signaled(s) => ProcessFinalState::Signaled {
1566                        // This conversion will fail on realtime signals, but that
1567                        // should currently be impossible since we don't support
1568                        // sending realtime signals.
1569                        signaled: s.try_into().unwrap(),
1570                    },
1571                    ExitStatus::StoppedByShadow => ProcessFinalState::Running(RunningVal::Running),
1572                };
1573                if expected_final_state == actual_final_state {
1574                    (s, log::Level::Debug)
1575                } else {
1576                    Worker::increment_plugin_error_count();
1577                    write!(s, "; expected end state was {expected_final_state} but was {actual_final_state}").unwrap();
1578                    (s, log::Level::Error)
1579                }
1580            } else {
1581                (s, log::Level::Debug)
1582            }
1583        };
1584        log::log!(log_level, "{main_result_string}");
1585
1586        let zombie = ZombieProcess {
1587            common: runnable.into_common(),
1588            exit_status,
1589        };
1590        zombie.notify_parent_of_exit(host);
1591
1592        *opt_state = Some(ProcessState::Zombie(zombie));
1593    }
1594
1595    /// Deprecated wrapper for `RunnableProcess::add_thread`
1596    pub fn add_thread(&self, host: &Host, thread: RootedRc<RootedRefCell<Thread>>) {
1597        self.as_runnable().unwrap().add_thread(host, thread)
1598    }
1599
1600    /// Emulate the `prlimit64` syscall for this process, operating on potentially both
1601    /// native and emulated resource limits.
1602    pub fn prlimit64(
1603        &self,
1604        resource: linux_api::resource::Resource,
1605        new_rlim: Option<&linux_api::resource::rlimit64>,
1606        old_rlim: Option<&mut linux_api::resource::rlimit64>,
1607    ) -> Result<(), linux_api::errno::Errno> {
1608        let idx = usize::try_from(u32::from(resource)).unwrap();
1609        let cur = self.common().rlimits[idx];
1610        if let Some(old_rlim) = old_rlim {
1611            *old_rlim = cur;
1612        }
1613        let Some(new_rlim) = new_rlim else {
1614            // Nothing else to do.
1615            return Ok(());
1616        };
1617        if new_rlim.rlim_cur > new_rlim.rlim_max {
1618            return Err(linux_api::errno::Errno::EINVAL);
1619        }
1620        // For now don't allow increasing rlim_max. We'd only be able to actually do this
1621        // natively if shadow is running with CAP_SYS_RESOURCE. We could pretend to do it
1622        // without changing the native limit, but that might just lead to confusion if and
1623        // when the native limit is exceeded without exceeding the emulated limit.
1624        if new_rlim.rlim_max > cur.rlim_max {
1625            return Err(linux_api::errno::Errno::EPERM);
1626        }
1627
1628        // Update our emulated limits to what was requested.
1629        self.common_mut().rlimits[idx] = *new_rlim;
1630
1631        let native_pid = if let Some(runnable) = self.as_runnable() {
1632            runnable.native_pid()
1633        } else {
1634            // The process is a zombie. No need to update the native limits, and
1635            // we can't since we already reaped the native process when
1636            // converting to the zombie state.
1637            return Ok(());
1638        };
1639
1640        // Get the current native limit.
1641        // Theoretically we could cache this, but doesn't seem worth the
1642        // complexity and fragility.
1643        let mut native_rlim = shadow_pod::zeroed();
1644        // SAFETY: we're only getting, not setting.
1645        unsafe {
1646            linux_api::resource::prlimit64(native_pid, resource, None, Some(&mut native_rlim))
1647        }
1648        .unwrap();
1649
1650        let mut new_rlim = *new_rlim;
1651        if new_rlim.rlim_cur < native_rlim.rlim_cur {
1652            // We don't permit lowering limits past their initial values,
1653            // since the shadow shim may use resources beyond what the managed process
1654            // itself needs (see <https://github.com/shadow/shadow/issues/3681>).
1655            log::warn!(
1656                "Only pretending to lower native {resource:?} rlim_cur from {} to {}",
1657                native_rlim.rlim_cur,
1658                new_rlim.rlim_cur
1659            );
1660            new_rlim.rlim_cur = native_rlim.rlim_cur;
1661        }
1662        if new_rlim.rlim_max < native_rlim.rlim_cur {
1663            // We can allow lowering the native max, since we currently never try to increase
1664            // the limit beyond its initial value. But the kernel won't let us lower beyond rlim_cur.
1665            log::warn!(
1666                "Only pretending to lower native {resource:?} rlim_max from {} to {}",
1667                native_rlim.rlim_max,
1668                new_rlim.rlim_max
1669            );
1670            new_rlim.rlim_max = native_rlim.rlim_cur;
1671        }
1672
1673        // Update the native limits. This should always succeed with the validations we already did above.
1674        // SAFETY: Not our process, and the checks we did above should ensure we don't lower
1675        // the limits to something the shim can't handle.
1676        unsafe { linux_api::resource::prlimit64(native_pid, resource, Some(&new_rlim), None) }
1677            .unwrap();
1678
1679        Ok(())
1680    }
1681
1682    /// FIXME: still needed? Time is now updated more granularly in the Thread code
1683    /// when xferring control to/from shim.
1684    fn set_shared_time(host: &Host) {
1685        let mut host_shmem = host.shim_shmem_lock_borrow_mut().unwrap();
1686        host_shmem.max_runahead_time = Worker::max_event_runahead_time(host);
1687        host.shim_shmem()
1688            .sim_time
1689            .store(Worker::current_time().unwrap(), Ordering::Relaxed);
1690    }
1691
1692    /// Deprecated wrapper for `RunnableProcess::shmem`
1693    pub fn shmem(&self) -> impl Deref<Target = ShMemBlock<'static, ProcessShmem>> + '_ {
1694        Ref::map(self.as_runnable().unwrap(), |r| &r.shim_shared_mem_block)
1695    }
1696
1697    /// Resource usage, as returned e.g. by the `getrusage` syscall.
1698    pub fn rusage(&self) -> linux_api::resource::rusage {
1699        warn_once_then_debug!(
1700            "resource usage (rusage) tracking unimplemented; Returning bogus zeroed values"
1701        );
1702        // TODO: Actually track some of these.
1703        // Assuming we want to support `RUSAGE_THREAD` in the `getrusage`
1704        // syscall, we'll actually want to track at the thread level, and either
1705        // increment at both thread and process level at the points where we do
1706        // the tracking, or dynamically iterate over the threads here and sum
1707        // the results.
1708        linux_api::resource::rusage {
1709            ru_utime: linux_api::time::kernel_old_timeval {
1710                tv_sec: 0,
1711                tv_usec: 0,
1712            },
1713            ru_stime: linux_api::time::kernel_old_timeval {
1714                tv_sec: 0,
1715                tv_usec: 0,
1716            },
1717            ru_maxrss: 0,
1718            ru_ixrss: 0,
1719            ru_idrss: 0,
1720            ru_isrss: 0,
1721            ru_minflt: 0,
1722            ru_majflt: 0,
1723            ru_nswap: 0,
1724            ru_inblock: 0,
1725            ru_oublock: 0,
1726            ru_msgsnd: 0,
1727            ru_msgrcv: 0,
1728            ru_nsignals: 0,
1729            ru_nvcsw: 0,
1730            ru_nivcsw: 0,
1731        }
1732    }
1733
1734    /// Signal that will be sent to parent process on exit. Typically `Some(SIGCHLD)`.
1735    pub fn exit_signal(&self) -> Option<Signal> {
1736        self.common().exit_signal
1737    }
1738
1739    pub fn current_working_dir(&self) -> impl Deref<Target = CString> + '_ {
1740        Ref::map(self.common(), |common| &common.working_dir)
1741    }
1742
1743    /// Set the process's working directory.
1744    /// This must be kept in sync with the actual working dir of the native process.
1745    /// See <https://github.com/shadow/shadow/issues/2960>
1746    // TODO: This ought to be at the thread level, to support `CLONE_FS`.
1747    pub fn set_current_working_dir(&self, path: CString) {
1748        self.common_mut().working_dir = path;
1749    }
1750
1751    /// Update `self` to complete an `exec` syscall from thread `tid`, replacing
1752    /// the running managed process with `mthread`.
1753    pub fn update_for_exec(&mut self, host: &Host, tid: ThreadId, mthread: ManagedThread) {
1754        let Some(mut runnable) = self.as_runnable_mut() else {
1755            // This could happen if another event runs before the "execve completion" event
1756            // and kills the process. e.g. another thread in the process could run and
1757            // execute the `exit_group` syscall.
1758            log::debug!(
1759                "Process {:?} exited before it could complete execve",
1760                self.id()
1761            );
1762            mthread.kill_and_drop();
1763            return;
1764        };
1765        let old_native_pid = std::mem::replace(&mut runnable.native_pid, mthread.native_pid());
1766
1767        // Kill the previous native process
1768        rustix::process::kill_process(old_native_pid.into(), rustix::process::Signal::Kill)
1769            .expect("Unable to send kill signal to managed process {old_native_pid:?}");
1770        let wait_res = rustix::process::waitpid(Some(old_native_pid.into()), WaitOptions::empty())
1771            .unwrap()
1772            .unwrap();
1773        assert_eq!(
1774            wait_res.terminating_signal(),
1775            Some(Signal::SIGKILL.as_i32().try_into().unwrap())
1776        );
1777
1778        let execing_thread = runnable.threads.borrow_mut().remove(&tid).unwrap();
1779
1780        // Dispose of all threads other than the thread that's running `exec`.
1781        for (_tid, thread) in runnable.threads.replace(BTreeMap::new()) {
1782            // Notify the ManagedThread that the native process has exited.
1783            thread.borrow(host.root()).mthread().handle_process_exit();
1784
1785            thread.explicit_drop_recursive(host.root(), host);
1786        }
1787
1788        // Recreate the `MemoryManager`
1789        {
1790            // We can't safely replace the memory manager if there are outstanding
1791            // unsafe references in C code. There shouldn't be any, though, since
1792            // this is only called from the `execve` and `execveat` syscall handlers,
1793            // which are in Rust.
1794            let unsafe_borrow_mut = runnable.unsafe_borrow_mut.borrow();
1795            let unsafe_borrows = runnable.unsafe_borrows.borrow();
1796            assert!(unsafe_borrow_mut.is_none());
1797            assert!(unsafe_borrows.is_empty());
1798            // Replace the MM, while still holding the references to the unsafe borrows
1799            // to ensure none exist.
1800            runnable
1801                .memory_manager
1802                .replace(unsafe { MemoryManager::new(mthread.native_pid()) });
1803        }
1804
1805        let new_tid = runnable.common.thread_group_leader_id();
1806        log::trace!(
1807            "updating for exec; pid:{pid}, tid:{tid:?}, new_tid:{new_tid:?}",
1808            pid = runnable.common.id
1809        );
1810        execing_thread
1811            .borrow_mut(host.root())
1812            .update_for_exec(host, mthread, new_tid);
1813
1814        runnable
1815            .threads
1816            .borrow_mut()
1817            .insert(new_tid, execing_thread);
1818
1819        // Exit signal is reset to SIGCHLD.
1820        runnable.common.exit_signal = Some(Signal::SIGCHLD);
1821
1822        // Reset signal actions to default.
1823        // `execve(2)`:
1824        // POSIX.1 specifies that the dispositions of any signals that
1825        // are ignored or set to the default are left unchanged.  POSIX.1
1826        // specifies one exception: if SIGCHLD is being ignored, then an
1827        // implementation may leave the disposition unchanged or reset it
1828        // to the default; Linux does the former.
1829        let host_shmem_prot = host.shim_shmem_lock_borrow_mut().unwrap();
1830        let mut shmem_prot = runnable
1831            .shim_shared_mem_block
1832            .protected
1833            .borrow_mut(&host_shmem_prot.root);
1834        for signal in Signal::standard_signals() {
1835            let current_action = unsafe { shmem_prot.signal_action(signal) };
1836            if !(current_action.is_default()
1837                || current_action.is_ignore()
1838                || signal == Signal::SIGCHLD && current_action.is_ignore())
1839            {
1840                unsafe {
1841                    *shmem_prot.signal_action_mut(signal) = linux_api::signal::sigaction::new_raw(
1842                        linux_api::signal::SignalHandler::SigDfl,
1843                        SigActionFlags::empty(),
1844                        sigset_t::EMPTY,
1845                        None,
1846                    )
1847                };
1848            }
1849        }
1850    }
1851}
1852
1853impl Drop for Process {
1854    fn drop(&mut self) {
1855        // Should have been explicitly dropped.
1856        debug_assert!(self.state.borrow().is_none());
1857    }
1858}
1859
1860impl ExplicitDrop for Process {
1861    type ExplicitDropParam = Host;
1862    type ExplicitDropResult = ();
1863
1864    fn explicit_drop(mut self, host: &Self::ExplicitDropParam) -> Self::ExplicitDropResult {
1865        // Should normally only be dropped in the zombie state.
1866        debug_assert!(self.as_zombie().is_some() || std::thread::panicking());
1867
1868        let state = self.state.get_mut().take().unwrap();
1869        state.explicit_drop(host);
1870    }
1871}
1872
1873/// Tracks a memory reference made by a legacy C memory-read API.
1874struct UnsafeBorrow {
1875    // Must come before `manager`, so that it's dropped first, since it's
1876    // borrowed from it.
1877    _memory: ProcessMemoryRef<'static, u8>,
1878    _manager: Ref<'static, MemoryManager>,
1879}
1880
1881impl UnsafeBorrow {
1882    /// Creates a raw readable pointer, and saves an instance of `Self` into
1883    /// `process` for later clean-up.
1884    ///
1885    /// # Safety
1886    ///
1887    /// The pointer is invalidated when one of the Process memory flush methods is called.
1888    unsafe fn readable_ptr(
1889        process: &Process,
1890        ptr: ForeignArrayPtr<u8>,
1891    ) -> Result<*const c_void, Errno> {
1892        let runnable = process.as_runnable().unwrap();
1893        let manager = runnable.memory_manager.borrow();
1894        // SAFETY: We ensure that the `memory` is dropped before the `manager`,
1895        // and `Process` ensures that this whole object is dropped before
1896        // `MemoryManager` can be moved, freed, etc.
1897        let manager = unsafe {
1898            std::mem::transmute::<Ref<'_, MemoryManager>, Ref<'static, MemoryManager>>(manager)
1899        };
1900        let memory = manager.memory_ref(ptr)?;
1901        let memory = unsafe {
1902            std::mem::transmute::<ProcessMemoryRef<'_, u8>, ProcessMemoryRef<'static, u8>>(memory)
1903        };
1904        let vptr = memory.as_ptr() as *mut c_void;
1905        runnable.unsafe_borrows.borrow_mut().push(Self {
1906            _manager: manager,
1907            _memory: memory,
1908        });
1909        Ok(vptr)
1910    }
1911
1912    /// Creates a raw readable string, and saves an instance of `Self` into
1913    /// `process` for later clean-up.
1914    ///
1915    /// # Safety
1916    ///
1917    /// The pointer is invalidated when one of the Process memory flush methods is called.
1918    unsafe fn readable_string(
1919        process: &Process,
1920        ptr: ForeignArrayPtr<c_char>,
1921    ) -> Result<(*const c_char, libc::size_t), Errno> {
1922        let runnable = process.as_runnable().unwrap();
1923        let manager = runnable.memory_manager.borrow();
1924        // SAFETY: We ensure that the `memory` is dropped before the `manager`,
1925        // and `Process` ensures that this whole object is dropped before
1926        // `MemoryManager` can be moved, freed, etc.
1927        let manager = unsafe {
1928            std::mem::transmute::<Ref<'_, MemoryManager>, Ref<'static, MemoryManager>>(manager)
1929        };
1930        let ptr = ptr.cast_u8();
1931        let memory = manager.memory_ref_prefix(ptr)?;
1932        let memory = unsafe {
1933            std::mem::transmute::<ProcessMemoryRef<'_, u8>, ProcessMemoryRef<'static, u8>>(memory)
1934        };
1935        if !memory.contains(&0) {
1936            return Err(Errno::ENAMETOOLONG);
1937        }
1938        assert_eq!(std::mem::size_of::<c_char>(), std::mem::size_of::<u8>());
1939        let ptr = memory.as_ptr() as *const c_char;
1940        let len = memory.len();
1941        runnable.unsafe_borrows.borrow_mut().push(Self {
1942            _manager: manager,
1943            _memory: memory,
1944        });
1945        Ok((ptr, len))
1946    }
1947}
1948
1949// Safety: Normally the Ref would make this non-Send, since it could end then
1950// end up trying to manipulate the source RefCell (which is !Sync) from multiple
1951// threads.  We ensure that these objects never escape Process, which itself is
1952// non-Sync, ensuring this doesn't happen.
1953//
1954// This is admittedly hand-wavy and making some assumptions about the
1955// implementation of RefCell, but this whole type is temporary scaffolding to
1956// support legacy C code.
1957unsafe impl Send for UnsafeBorrow {}
1958
1959/// Tracks a memory reference made by a legacy C memory-write API.
1960struct UnsafeBorrowMut {
1961    // Must come before `manager`, so that it's dropped first, since it's
1962    // borrowed from it.
1963    memory: Option<ProcessMemoryRefMut<'static, u8>>,
1964    _manager: RefMut<'static, MemoryManager>,
1965}
1966
1967impl UnsafeBorrowMut {
1968    /// Creates a raw writable pointer, and saves an instance of `Self` into
1969    /// `process` for later clean-up. The initial contents of the pointer is unspecified.
1970    ///
1971    /// # Safety
1972    ///
1973    /// The pointer is invalidated when one of the Process memory flush methods is called.
1974    unsafe fn writable_ptr(
1975        process: &Process,
1976        ptr: ForeignArrayPtr<u8>,
1977    ) -> Result<*mut c_void, Errno> {
1978        let runnable = process.as_runnable().unwrap();
1979        let manager = runnable.memory_manager.borrow_mut();
1980        // SAFETY: We ensure that the `memory` is dropped before the `manager`,
1981        // and `Process` ensures that this whole object is dropped before
1982        // `MemoryManager` can be moved, freed, etc.
1983        let mut manager = unsafe {
1984            std::mem::transmute::<RefMut<'_, MemoryManager>, RefMut<'static, MemoryManager>>(
1985                manager,
1986            )
1987        };
1988        let memory = manager.memory_ref_mut_uninit(ptr)?;
1989        let mut memory = unsafe {
1990            std::mem::transmute::<ProcessMemoryRefMut<'_, u8>, ProcessMemoryRefMut<'static, u8>>(
1991                memory,
1992            )
1993        };
1994        let vptr = memory.as_mut_ptr() as *mut c_void;
1995        let prev = runnable.unsafe_borrow_mut.borrow_mut().replace(Self {
1996            _manager: manager,
1997            memory: Some(memory),
1998        });
1999        assert!(prev.is_none());
2000        Ok(vptr)
2001    }
2002
2003    /// Creates a raw mutable pointer, and saves an instance of `Self` into
2004    /// `process` for later clean-up.
2005    ///
2006    /// # Safety
2007    ///
2008    /// The pointer is invalidated when one of the Process memory flush methods is called.
2009    unsafe fn mutable_ptr(
2010        process: &Process,
2011        ptr: ForeignArrayPtr<u8>,
2012    ) -> Result<*mut c_void, Errno> {
2013        let runnable = process.as_runnable().unwrap();
2014        let manager = runnable.memory_manager.borrow_mut();
2015        // SAFETY: We ensure that the `memory` is dropped before the `manager`,
2016        // and `Process` ensures that this whole object is dropped before
2017        // `MemoryManager` can be moved, freed, etc.
2018        let mut manager = unsafe {
2019            std::mem::transmute::<RefMut<'_, MemoryManager>, RefMut<'static, MemoryManager>>(
2020                manager,
2021            )
2022        };
2023        let memory = manager.memory_ref_mut(ptr)?;
2024        let mut memory = unsafe {
2025            std::mem::transmute::<ProcessMemoryRefMut<'_, u8>, ProcessMemoryRefMut<'static, u8>>(
2026                memory,
2027            )
2028        };
2029        let vptr = memory.as_mut_ptr() as *mut c_void;
2030        let prev = runnable.unsafe_borrow_mut.borrow_mut().replace(Self {
2031            _manager: manager,
2032            memory: Some(memory),
2033        });
2034        assert!(prev.is_none());
2035        Ok(vptr)
2036    }
2037
2038    /// Free this reference, writing back to process memory.
2039    fn flush(mut self) -> Result<(), Errno> {
2040        self.memory.take().unwrap().flush()
2041    }
2042
2043    /// Free this reference without writing back to process memory.
2044    fn noflush(mut self) {
2045        self.memory.take().unwrap().noflush()
2046    }
2047}
2048
2049// Safety: Normally the RefMut would make this non-Send, since it could end then
2050// end up trying to manipulate the source RefCell (which is !Sync) from multiple
2051// threads.  We ensure that these objects never escape Process, which itself is
2052// non-Sync, ensuring this doesn't happen.
2053//
2054// This is admittedly hand-wavy and making some assumptions about the implementation of
2055// RefCell, but this whole type is temporary scaffolding to support legacy C code.
2056unsafe impl Send for UnsafeBorrowMut {}
2057
2058fn make_name(host: &Host, exe_name: &str, id: ProcessId) -> CString {
2059    CString::new(format!(
2060        "{host_name}.{exe_name}.{id}",
2061        host_name = host.name(),
2062        exe_name = exe_name,
2063        id = u32::from(id)
2064    ))
2065    .unwrap()
2066}
2067
2068mod export {
2069    use std::os::raw::c_void;
2070
2071    use libc::size_t;
2072    use log::trace;
2073    use shadow_shim_helper_rs::notnull::*;
2074    use shadow_shim_helper_rs::shim_shmem::export::ShimShmemProcess;
2075    use shadow_shim_helper_rs::syscall_types::UntypedForeignPtr;
2076
2077    use super::*;
2078    use crate::utility::HostTreePointer;
2079
2080    /// Copy `n` bytes from `src` to `dst`. Returns 0 on success or -EFAULT if any of
2081    /// the specified range couldn't be accessed. Always succeeds with n==0.
2082    #[unsafe(no_mangle)]
2083    pub extern "C-unwind" fn process_readPtr(
2084        proc: *const Process,
2085        dst: *mut c_void,
2086        src: UntypedForeignPtr,
2087        n: usize,
2088    ) -> i32 {
2089        let proc = unsafe { proc.as_ref().unwrap() };
2090        let src = ForeignArrayPtr::new(src.cast::<u8>(), n);
2091        let dst = unsafe { std::slice::from_raw_parts_mut(notnull_mut_debug(dst) as *mut u8, n) };
2092
2093        match proc.memory_borrow().copy_from_ptr(dst, src) {
2094            Ok(_) => 0,
2095            Err(e) => {
2096                trace!("Couldn't read {src:?} into {dst:?}: {e:?}");
2097                e.to_negated_i32()
2098            }
2099        }
2100    }
2101
2102    /// Copy `n` bytes from `src` to `dst`. Returns 0 on success or -EFAULT if any of
2103    /// the specified range couldn't be accessed. The write is flushed immediately.
2104    #[unsafe(no_mangle)]
2105    pub unsafe extern "C-unwind" fn process_writePtr(
2106        proc: *const Process,
2107        dst: UntypedForeignPtr,
2108        src: *const c_void,
2109        n: usize,
2110    ) -> i32 {
2111        let proc = unsafe { proc.as_ref().unwrap() };
2112        let dst = ForeignArrayPtr::new(dst.cast::<u8>(), n);
2113        let src = unsafe { std::slice::from_raw_parts(notnull_debug(src) as *const u8, n) };
2114        match proc.memory_borrow_mut().copy_to_ptr(dst, src) {
2115            Ok(_) => 0,
2116            Err(e) => {
2117                trace!("Couldn't write {src:?} into {dst:?}: {e:?}");
2118                e.to_negated_i32()
2119            }
2120        }
2121    }
2122
2123    /// Make the data at plugin_src available in shadow's address space.
2124    ///
2125    /// The returned pointer is invalidated when one of the process memory flush
2126    /// methods is called; typically after a syscall has completed.
2127    #[unsafe(no_mangle)]
2128    pub unsafe extern "C-unwind" fn process_getReadablePtr(
2129        proc: *const Process,
2130        plugin_src: UntypedForeignPtr,
2131        n: usize,
2132    ) -> *const c_void {
2133        let proc = unsafe { proc.as_ref().unwrap() };
2134        let plugin_src = ForeignArrayPtr::new(plugin_src.cast::<u8>(), n);
2135        unsafe { UnsafeBorrow::readable_ptr(proc, plugin_src).unwrap_or(std::ptr::null()) }
2136    }
2137
2138    /// Returns a writable pointer corresponding to the named region. The
2139    /// initial contents of the returned memory are unspecified.
2140    ///
2141    /// The returned pointer is invalidated when one of the process memory flush
2142    /// methods is called; typically after a syscall has completed.
2143    ///
2144    /// CAUTION: if the unspecified contents aren't overwritten, and the pointer
2145    /// isn't explicitly freed via `process_freePtrsWithoutFlushing`, those
2146    /// unspecified contents may be written back into process memory.
2147    #[unsafe(no_mangle)]
2148    pub unsafe extern "C-unwind" fn process_getWriteablePtr(
2149        proc: *const Process,
2150        plugin_src: UntypedForeignPtr,
2151        n: usize,
2152    ) -> *mut c_void {
2153        let proc = unsafe { proc.as_ref().unwrap() };
2154        let plugin_src = ForeignArrayPtr::new(plugin_src.cast::<u8>(), n);
2155        unsafe { UnsafeBorrowMut::writable_ptr(proc, plugin_src).unwrap_or(std::ptr::null_mut()) }
2156    }
2157
2158    /// Returns a writeable pointer corresponding to the specified src. Use when
2159    /// the data at the given address needs to be both read and written.
2160    ///
2161    /// The returned pointer is invalidated when one of the process memory flush
2162    /// methods is called; typically after a syscall has completed.
2163    #[unsafe(no_mangle)]
2164    pub unsafe extern "C-unwind" fn process_getMutablePtr(
2165        proc: *const Process,
2166        plugin_src: UntypedForeignPtr,
2167        n: usize,
2168    ) -> *mut c_void {
2169        let proc = unsafe { proc.as_ref().unwrap() };
2170        let plugin_src = ForeignArrayPtr::new(plugin_src.cast::<u8>(), n);
2171        unsafe { UnsafeBorrowMut::mutable_ptr(proc, plugin_src).unwrap_or(std::ptr::null_mut()) }
2172    }
2173
2174    /// Reads up to `n` bytes into `str`.
2175    ///
2176    /// Returns:
2177    /// strlen(str) on success.
2178    /// -ENAMETOOLONG if there was no NULL byte in the first `n` characters.
2179    /// -EFAULT if the string extends beyond the accessible address space.
2180    #[unsafe(no_mangle)]
2181    pub unsafe extern "C-unwind" fn process_readString(
2182        proc: *const Process,
2183        strbuf: *mut libc::c_char,
2184        ptr: UntypedForeignPtr,
2185        maxlen: libc::size_t,
2186    ) -> libc::ssize_t {
2187        let proc = unsafe { proc.as_ref().unwrap() };
2188        let memory_manager = proc.memory_borrow();
2189        let buf =
2190            unsafe { std::slice::from_raw_parts_mut(notnull_mut_debug(strbuf) as *mut u8, maxlen) };
2191        let cstr = match memory_manager
2192            .copy_str_from_ptr(buf, ForeignArrayPtr::new(ptr.cast::<u8>(), maxlen))
2193        {
2194            Ok(cstr) => cstr,
2195            Err(e) => return e.to_negated_i32() as isize,
2196        };
2197        cstr.to_bytes().len().try_into().unwrap()
2198    }
2199
2200    /// Reads up to `n` bytes into `str`.
2201    ///
2202    /// Returns:
2203    /// strlen(str) on success.
2204    /// -ENAMETOOLONG if there was no NULL byte in the first `n` characters.
2205    /// -EFAULT if the string extends beyond the accessible address space.
2206    #[unsafe(no_mangle)]
2207    pub unsafe extern "C-unwind" fn process_getReadableString(
2208        proc: *const Process,
2209        plugin_src: UntypedForeignPtr,
2210        n: usize,
2211        out_str: *mut *const c_char,
2212        out_strlen: *mut size_t,
2213    ) -> i32 {
2214        let proc = unsafe { proc.as_ref().unwrap() };
2215        let ptr = ForeignArrayPtr::new(plugin_src.cast::<c_char>(), n);
2216        match unsafe { UnsafeBorrow::readable_string(proc, ptr) } {
2217            Ok((str, strlen)) => {
2218                assert!(!out_str.is_null());
2219                unsafe { out_str.write(str) };
2220                if !out_strlen.is_null() {
2221                    unsafe { out_strlen.write(strlen) };
2222                }
2223                0
2224            }
2225            Err(e) => e.to_negated_i32(),
2226        }
2227    }
2228
2229    /// Returns the processID that was assigned to us in process_new
2230    #[unsafe(no_mangle)]
2231    pub unsafe extern "C-unwind" fn process_getProcessID(proc: *const Process) -> libc::pid_t {
2232        let proc = unsafe { proc.as_ref().unwrap() };
2233        proc.id().into()
2234    }
2235
2236    #[unsafe(no_mangle)]
2237    pub unsafe extern "C-unwind" fn process_getName(proc: *const Process) -> *const c_char {
2238        let proc = unsafe { proc.as_ref().unwrap() };
2239        proc.common().name.as_ptr()
2240    }
2241
2242    /// Safety:
2243    ///
2244    /// The returned pointer is invalidated when the host shmem lock is released, e.g. via
2245    /// Host::unlock_shmem.
2246    #[unsafe(no_mangle)]
2247    pub unsafe extern "C-unwind" fn process_getSharedMem(
2248        proc: *const Process,
2249    ) -> *const ShimShmemProcess {
2250        let proc = unsafe { proc.as_ref().unwrap() };
2251        std::ptr::from_ref(proc.as_runnable().unwrap().shim_shared_mem_block.deref())
2252    }
2253
2254    #[unsafe(no_mangle)]
2255    pub unsafe extern "C-unwind" fn process_getWorkingDir(proc: *const Process) -> *const c_char {
2256        let proc = unsafe { proc.as_ref().unwrap() };
2257        proc.common().working_dir.as_ptr()
2258    }
2259
2260    #[unsafe(no_mangle)]
2261    pub unsafe extern "C-unwind" fn process_straceLoggingMode(
2262        proc: *const Process,
2263    ) -> StraceFmtMode {
2264        let proc = unsafe { proc.as_ref().unwrap() };
2265        proc.strace_logging_options().into()
2266    }
2267
2268    #[unsafe(no_mangle)]
2269    pub unsafe extern "C-unwind" fn process_getNativePid(proc: *const Process) -> libc::pid_t {
2270        let proc = unsafe { proc.as_ref().unwrap() };
2271        proc.native_pid().as_raw_nonzero().get()
2272    }
2273
2274    /// Flushes and invalidates all previously returned readable/writable plugin
2275    /// pointers, as if returning control to the plugin. This can be useful in
2276    /// conjunction with `thread_nativeSyscall` operations that touch memory, or
2277    /// to gracefully handle failed writes.
2278    ///
2279    /// Returns 0 on success or a negative errno on failure.
2280    #[unsafe(no_mangle)]
2281    pub unsafe extern "C-unwind" fn process_flushPtrs(proc: *const Process) -> i32 {
2282        let proc = unsafe { proc.as_ref().unwrap() };
2283        match proc.free_unsafe_borrows_flush() {
2284            Ok(_) => 0,
2285            Err(e) => e.to_negated_i32(),
2286        }
2287    }
2288
2289    /// Frees all readable/writable foreign pointers. Unlike process_flushPtrs, any
2290    /// previously returned writable pointer is *not* written back. Useful
2291    /// if an uninitialized writable pointer was obtained via `process_getWriteablePtr`,
2292    /// and we end up not wanting to write anything after all (in particular, don't
2293    /// write back whatever garbage data was in the uninialized bueffer).
2294    #[unsafe(no_mangle)]
2295    pub unsafe extern "C-unwind" fn process_freePtrsWithoutFlushing(proc: *const Process) {
2296        let proc = unsafe { proc.as_ref().unwrap() };
2297        proc.free_unsafe_borrows_noflush();
2298    }
2299
2300    #[unsafe(no_mangle)]
2301    pub unsafe extern "C-unwind" fn process_getThread(
2302        proc: *const Process,
2303        tid: libc::pid_t,
2304    ) -> *const Thread {
2305        let proc = unsafe { proc.as_ref().unwrap() };
2306        Worker::with_active_host(|host| {
2307            let tid = ThreadId::try_from(tid).unwrap();
2308            let Some(thread) = proc.thread_borrow(tid) else {
2309                return std::ptr::null();
2310            };
2311            let thread = thread.borrow(host.root());
2312            &*thread
2313        })
2314        .unwrap()
2315    }
2316
2317    /// Returns a pointer to an arbitrary live thread in the process.
2318    #[unsafe(no_mangle)]
2319    pub unsafe extern "C-unwind" fn process_firstLiveThread(proc: *const Process) -> *const Thread {
2320        let proc = unsafe { proc.as_ref().unwrap() };
2321        Worker::with_active_host(|host| {
2322            let Some(thread) = proc.first_live_thread_borrow(host.root()) else {
2323                return std::ptr::null();
2324            };
2325            let thread = thread.borrow(host.root());
2326            &*thread
2327        })
2328        .unwrap()
2329    }
2330
2331    #[unsafe(no_mangle)]
2332    pub unsafe extern "C-unwind" fn process_isRunning(proc: *const Process) -> bool {
2333        let proc = unsafe { proc.as_ref().unwrap() };
2334        proc.is_running()
2335    }
2336
2337    // FIXME: still needed? Time is now updated more granularly in the Thread code
2338    // when xferring control to/from shim.
2339    #[unsafe(no_mangle)]
2340    pub unsafe extern "C-unwind" fn process_setSharedTime() {
2341        Worker::with_active_host(Process::set_shared_time).unwrap();
2342    }
2343
2344    #[unsafe(no_mangle)]
2345    pub unsafe extern "C-unwind" fn process_getPhysicalAddress(
2346        proc: *const Process,
2347        vptr: UntypedForeignPtr,
2348    ) -> ManagedPhysicalMemoryAddr {
2349        let proc = unsafe { proc.as_ref().unwrap() };
2350        proc.physical_address(vptr)
2351    }
2352
2353    #[unsafe(no_mangle)]
2354    pub unsafe extern "C-unwind" fn process_addChildEventListener(
2355        host: *const Host,
2356        process: *const Process,
2357        listener: *mut cshadow::StatusListener,
2358    ) {
2359        let host = unsafe { host.as_ref().unwrap() };
2360        let process = unsafe { process.as_ref().unwrap() };
2361        let listener = HostTreePointer::new_for_host(host.id(), listener);
2362        process
2363            .borrow_as_runnable()
2364            .unwrap()
2365            .child_process_event_listeners
2366            .borrow_mut()
2367            .add_legacy_listener(listener)
2368    }
2369
2370    #[unsafe(no_mangle)]
2371    pub unsafe extern "C-unwind" fn process_removeChildEventListener(
2372        _host: *const Host,
2373        process: *const Process,
2374        listener: *mut cshadow::StatusListener,
2375    ) {
2376        let process = unsafe { process.as_ref().unwrap() };
2377        process
2378            .borrow_as_runnable()
2379            .unwrap()
2380            .child_process_event_listeners
2381            .borrow_mut()
2382            .remove_legacy_listener(listener)
2383    }
2384}