From 598687343727f5109fdfb9a1d30f9027142b0b73 Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Fri, 24 Nov 2023 06:07:03 -0300 Subject: [PATCH 01/50] use clone instead of fork for linux in prepare worker --- .../node/core/pvf/prepare-worker/Cargo.toml | 2 +- .../node/core/pvf/prepare-worker/src/lib.rs | 150 ++++++++++++------ 2 files changed, 101 insertions(+), 51 deletions(-) diff --git a/polkadot/node/core/pvf/prepare-worker/Cargo.toml b/polkadot/node/core/pvf/prepare-worker/Cargo.toml index 1cd221533f48..e9c366261e8f 100644 --- a/polkadot/node/core/pvf/prepare-worker/Cargo.toml +++ b/polkadot/node/core/pvf/prepare-worker/Cargo.toml @@ -15,7 +15,7 @@ tracking-allocator = { package = "staging-tracking-allocator", path = "../../../ tikv-jemalloc-ctl = { version = "0.5.0", optional = true } tikv-jemallocator = { version = "0.5.0", optional = true } os_pipe = "1.1.4" -nix = { version = "0.27.1", features = ["resource", "process"]} +nix = { version = "0.27.1", features = ["resource", "process", "sched"] } parity-scale-codec = { version = "3.6.1", default-features = false, features = ["derive"] } diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index 151b54efc2d1..caf060f45dcb 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -37,6 +37,9 @@ use nix::{ }, unistd::{ForkResult, Pid}, }; + +#[cfg(target_os = "linux")] +use nix::sched::CloneFlags; use os_pipe::{self, PipeReader, PipeWriter}; use parity_scale_codec::{Decode, Encode}; use polkadot_node_core_pvf_common::{ @@ -218,7 +221,7 @@ pub fn worker_entrypoint( let prepare_job_kind = pvf.prep_kind(); let executor_params = pvf.executor_params(); - let (pipe_reader, pipe_writer) = os_pipe::pipe()?; + let (pipe_reader, mut pipe_writer) = os_pipe::pipe()?; let usage_before = match nix::sys::resource::getrusage(UsageWho::RUSAGE_CHILDREN) { Ok(usage) => usage, @@ -229,50 +232,97 @@ pub fn worker_entrypoint( }, }; - // SAFETY: new process is spawned within a single threaded process. This invariant - // is enforced by tests. - let result = match unsafe { nix::unistd::fork() } { - Err(errno) => Err(error_from_errno("fork", errno)), - Ok(ForkResult::Child) => { - // Dropping the stream closes the underlying socket. We want to make sure - // that the sandboxed child can't get any kind of information from the - // outside world. The only IPC it should be able to do is sending its - // response over the pipe. - drop(stream); - // Drop the read end so we don't have too many FDs open. - drop(pipe_reader); - - handle_child_process( - pvf, - pipe_writer, - preparation_timeout, - prepare_job_kind, - executor_params, - ) - }, - Ok(ForkResult::Parent { child }) => { - // the read end will wait until all write ends have been closed, - // this drop is necessary to avoid deadlock - drop(pipe_writer); - - handle_parent_process( - pipe_reader, - child, - temp_artifact_dest.clone(), - worker_pid, - usage_before, - preparation_timeout, - ) - }, - }; - - gum::trace!( - target: LOG_TARGET, - %worker_pid, - "worker: sending result to host: {:?}", - result - ); - send_response(&mut stream, result)?; + cfg_if::cfg_if! { + if #[cfg(any(target_os = "linux")) + ] { + let stack = 1024 * 1024; + let mut stack: Vec = vec![0u8; stack]; + let flags = CloneFlags::CLONE_NEWCGROUP + | CloneFlags::CLONE_NEWIPC + | CloneFlags::CLONE_NEWNET + | CloneFlags::CLONE_NEWNS + | CloneFlags::CLONE_NEWPID + | CloneFlags::CLONE_NEWUSER + | CloneFlags::CLONE_NEWUTS + | CloneFlags::from_bits_retain(libc::SIGCHLD); + + // SAFETY: new process is spawned within a single threaded process. This invariant + // is enforced by tests. Stack size being specified to ensure child doesn't overflow + let result = match unsafe { nix::sched::clone( + Box::new(|| { + handle_child_process( + pvf.clone(), + &mut pipe_writer, + preparation_timeout, + prepare_job_kind, + Arc::clone(&executor_params), + ); }), stack.as_mut_slice(), flags, None) + } { + Err(errno) => Err(error_from_errno("clone", errno)), + Ok(child) => { + // the read end will wait until all write ends have been closed, + // this drop is necessary to avoid deadlock + drop(pipe_writer); + handle_parent_process( + pipe_reader, + child, + temp_artifact_dest.clone(), + worker_pid, + usage_before, + preparation_timeout, + ) + }, + }; + send_response(&mut stream, result)?; + } else { + // SAFETY: new process is spawned within a single threaded process. This invariant + // is enforced by tests. + let result = match unsafe { nix::unistd::fork() } { + Err(errno) => Err(error_from_errno("fork", errno)), + Ok(ForkResult::Child) => { + // Dropping the stream closes the underlying socket. We want to make sure + // that the sandboxed child can't get any kind of information from the + // outside world. The only IPC it should be able to do is sending its + // response over the pipe. + drop(stream); + + // Drop the read end so we don't have too many FDs open. + drop(pipe_reader); + + handle_child_process( + pvf, + pipe_writer, + preparation_timeout, + prepare_job_kind, + executor_params, + ) + }, + Ok(ForkResult::Parent { child }) => { + // the read end will wait until all write ends have been closed, + // this drop is necessary to avoid deadlock + drop(pipe_writer); + + handle_parent_process( + pipe_reader, + child, + temp_artifact_dest.clone(), + worker_pid, + usage_before, + preparation_timeout, + ) + }, + }; + + + gum::trace!( + target: LOG_TARGET, + %worker_pid, + "worker: sending result to host: {:?}", + result + ); + send_response(&mut stream, result)?; + } + } } }, ); @@ -331,7 +381,7 @@ struct JobResponse { /// - If success, pipe back `JobResponse`. fn handle_child_process( pvf: PvfPrepData, - mut pipe_write: PipeWriter, + pipe_write: &mut PipeWriter, preparation_timeout: Duration, prepare_job_kind: PrepareJobKind, executor_params: Arc, @@ -380,7 +430,7 @@ fn handle_child_process( WaitOutcome::TimedOut, ) .unwrap_or_else(|err| { - send_child_response(&mut pipe_write, Err(PrepareError::IoErr(err.to_string()))) + send_child_response(pipe_write, Err(PrepareError::IoErr(err.to_string()))) }); let prepare_thread = spawn_worker_thread( @@ -410,7 +460,7 @@ fn handle_child_process( WaitOutcome::Finished, ) .unwrap_or_else(|err| { - send_child_response(&mut pipe_write, Err(PrepareError::IoErr(err.to_string()))) + send_child_response(pipe_write, Err(PrepareError::IoErr(err.to_string()))) }); let outcome = thread::wait_for_threads(condvar); @@ -432,7 +482,7 @@ fn handle_child_process( match prepare_thread.join().unwrap_or_else(|err| { send_child_response( - &mut pipe_write, + pipe_write, Err(PrepareError::JobError(stringify_panic_payload(err))), ) }) { @@ -477,7 +527,7 @@ fn handle_child_process( unreachable!("we run wait_while until the outcome is no longer pending; qed"), }; - send_child_response(&mut pipe_write, result); + send_child_response(pipe_write, result); } /// Waits for child process to finish and handle child response from pipe. From 510714a1a859253f46f02d5c6bcee3f741462e74 Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Fri, 24 Nov 2023 08:24:56 -0300 Subject: [PATCH 02/50] minor fix --- polkadot/node/core/pvf/prepare-worker/src/lib.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index 263ad5c52e78..df4e44fdfee6 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -291,7 +291,7 @@ pub fn worker_entrypoint( handle_child_process( pvf, - pipe_writer, + &mut pipe_writer, preparation_timeout, prepare_job_kind, executor_params, From 3f81a53de3f696416b703ee81dfab694d06dbf5b Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Fri, 24 Nov 2023 16:24:52 -0300 Subject: [PATCH 03/50] minor fix --- polkadot/node/core/pvf/prepare-worker/src/lib.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index df4e44fdfee6..c627c55c411b 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -244,7 +244,7 @@ pub fn worker_entrypoint( | CloneFlags::CLONE_NEWPID | CloneFlags::CLONE_NEWUSER | CloneFlags::CLONE_NEWUTS - | CloneFlags::fr&mut om_bits_retain(libc::SIGCHLD); + | CloneFlags::from_bits_retain(libc::SIGCHLD); // SAFETY: new process is spawned within a single threaded process. This invariant // is enforced by tests. Stack size being specified to ensure child doesn't overflow @@ -265,9 +265,9 @@ pub fn worker_entrypoint( drop(pipe_writer); handle_parent_process( pipe_reader, + worker_pid, child, temp_artifact_dest.clone(), - worker_pid, usage_before, preparation_timeout, ) From a777c6e681437fbfcd76aec6c39db4b2655eb126 Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Sat, 25 Nov 2023 18:37:49 -0300 Subject: [PATCH 04/50] remove unused import --- polkadot/node/core/pvf/prepare-worker/src/lib.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index c627c55c411b..074b2e8ad4b7 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -29,13 +29,15 @@ use crate::memory_stats::max_rss_stat::{extract_max_rss_stat, get_max_rss_thread #[cfg(any(target_os = "linux", feature = "jemalloc-allocator"))] use crate::memory_stats::memory_tracker::{get_memory_tracker_loop_stats, memory_tracker_loop}; use libc; +#[cfg(not(target_os = "linux"))] +use nix::unistd::ForkResult; use nix::{ errno::Errno, sys::{ resource::{Usage, UsageWho}, wait::WaitStatus, }, - unistd::{ForkResult, Pid}, + unistd::Pid, }; #[cfg(target_os = "linux")] @@ -233,8 +235,7 @@ pub fn worker_entrypoint( }; cfg_if::cfg_if! { - if #[cfg(any(target_os = "linux")) - ] { + if #[cfg(any(target_os = "linux"))] { let stack = 1024 * 1024; let mut stack: Vec = vec![0u8; stack]; let flags = CloneFlags::CLONE_NEWCGROUP From 8c4a0805e0d794e4e54295a93a93d7be3b1fd969 Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Wed, 29 Nov 2023 08:15:34 -0300 Subject: [PATCH 05/50] use raw fd instead of pipe on clone and fork call --- .../node/core/pvf/common/src/worker/mod.rs | 67 +++++++++++++ .../node/core/pvf/prepare-worker/src/lib.rs | 95 +++++++++++++------ polkadot/node/core/pvf/tests/it/adder.rs | 1 + 3 files changed, 133 insertions(+), 30 deletions(-) diff --git a/polkadot/node/core/pvf/common/src/worker/mod.rs b/polkadot/node/core/pvf/common/src/worker/mod.rs index d4f9bbc27ea6..ceb2c782f86e 100644 --- a/polkadot/node/core/pvf/common/src/worker/mod.rs +++ b/polkadot/node/core/pvf/common/src/worker/mod.rs @@ -21,6 +21,7 @@ pub mod security; use crate::{SecurityStatus, LOG_TARGET}; use cpu_time::ProcessTime; use futures::never::Never; +use polkadot_primitives::executor_params::DEFAULT_NATIVE_STACK_MAX; use std::{ any::Any, fmt, io, @@ -30,6 +31,42 @@ use std::{ time::Duration, }; +// Wasmtime powers the Substrate Executor. It compiles the wasm bytecode into native code. +// That native code does not create any stacks and just reuses the stack of the thread that +// wasmtime was invoked from. +// +// Also, we configure the executor to provide the deterministic stack and that requires +// supplying the amount of the native stack space that wasm is allowed to use. This is +// realized by supplying the limit into `wasmtime::Config::max_wasm_stack`. +// +// There are quirks to that configuration knob: +// +// 1. It only limits the amount of stack space consumed by wasm but does not ensure nor check that +// the stack space is actually available. +// +// That means, if the calling thread has 1 MiB of stack space left and the wasm code consumes +// more, then the wasmtime limit will **not** trigger. Instead, the wasm code will hit the +// guard page and the Rust stack overflow handler will be triggered. That leads to an +// **abort**. +// +// 2. It cannot and does not limit the stack space consumed by Rust code. +// +// Meaning that if the wasm code leaves no stack space for Rust code, then the Rust code +// will abort and that will abort the process as well. +// +// Typically on Linux the main thread gets the stack size specified by the `ulimit` and +// typically it's configured to 8 MiB. Rust's spawned threads are 2 MiB. OTOH, the +// DEFAULT_NATIVE_STACK_MAX is set to 256 MiB. Not nearly enough. +// +// Hence we need to increase it. The simplest way to fix that is to spawn a thread with the desired +// stack limit. +// +// The reasoning why we pick this particular size is: +// +// The default Rust thread stack limit 2 MiB + 256 MiB wasm stack. +/// The stack size for the execute thread. +pub const EXECUTE_THREAD_STACK_SIZE: usize = 2 * 1024 * 1024 + DEFAULT_NATIVE_STACK_MAX as usize; + /// Use this macro to declare a `fn main() {}` that will create an executable that can be used for /// spawning the desired worker. #[macro_export] @@ -184,6 +221,36 @@ macro_rules! decl_worker_main { }; } +// os_pipe lib +#[cfg(not(any(target_os = "ios", target_os = "macos", target_os = "haiku")))] +pub fn pipe2_cloexec() -> io::Result<(libc::c_int, libc::c_int)> { + let mut fds: [libc::c_int; 2] = [0; 2]; + let res = unsafe { libc::pipe2(fds.as_mut_ptr(), libc::O_CLOEXEC) }; + if res != 0 { + return Err(io::Error::last_os_error()); + } + Ok((fds[0], fds[1])) +} + +// os_pipe lib +#[cfg(any(target_os = "ios", target_os = "macos", target_os = "haiku"))] +pub fn pipe2_cloexec() -> io::Result<(libc::c_int, libc::c_int)> { + let mut fds: [libc::c_int; 2] = [0; 2]; + let res = unsafe { libc::pipe(fds.as_mut_ptr()) }; + if res != 0 { + return Err(io::Error::last_os_error()); + } + let res = unsafe { libc::fcntl(fds[0], libc::F_SETFD, libc::FD_CLOEXEC) }; + if res != 0 { + return Err(io::Error::last_os_error()); + } + let res = unsafe { libc::fcntl(fds[1], libc::F_SETFD, libc::FD_CLOEXEC) }; + if res != 0 { + return Err(io::Error::last_os_error()); + } + Ok((fds[0], fds[1])) +} + /// Some allowed overhead that we account for in the "CPU time monitor" thread's sleeps, on the /// child process. pub const JOB_TIMEOUT_OVERHEAD: Duration = Duration::from_millis(50); diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index 074b2e8ad4b7..4281512a5ff1 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -18,7 +18,13 @@ mod memory_stats; -use polkadot_node_core_pvf_common::executor_intf::{prepare, prevalidate}; +use polkadot_node_core_pvf_common::{ + executor_intf::{prepare, prevalidate}, + worker::pipe2_cloexec, +}; + +#[cfg(target_os = "linux")] +use polkadot_node_core_pvf_common::worker::EXECUTE_THREAD_STACK_SIZE; // NOTE: Initializing logging in e.g. tests will not have an effect in the workers, as they are // separate spawned processes. Run with e.g. `RUST_LOG=parachain::pvf-prepare-worker=trace`. @@ -62,7 +68,7 @@ use std::{ fs, io::{self, Read}, os::{ - fd::{AsRawFd, RawFd}, + fd::{AsRawFd, FromRawFd, IntoRawFd, RawFd}, unix::net::UnixStream, }, path::PathBuf, @@ -223,7 +229,7 @@ pub fn worker_entrypoint( let prepare_job_kind = pvf.prep_kind(); let executor_params = pvf.executor_params(); - let (pipe_reader, mut pipe_writer) = os_pipe::pipe()?; + let (pipe_reader, pipe_writer) = pipe2_cloexec()?; let usage_before = match nix::sys::resource::getrusage(UsageWho::RUSAGE_CHILDREN) { Ok(usage) => usage, @@ -234,10 +240,11 @@ pub fn worker_entrypoint( }, }; + let stream_fd = stream.as_raw_fd(); + cfg_if::cfg_if! { - if #[cfg(any(target_os = "linux"))] { - let stack = 1024 * 1024; - let mut stack: Vec = vec![0u8; stack]; + if #[cfg(target_os = "linux")] { + let mut stack: Vec = vec![0u8; EXECUTE_THREAD_STACK_SIZE]; let flags = CloneFlags::CLONE_NEWCGROUP | CloneFlags::CLONE_NEWIPC | CloneFlags::CLONE_NEWNET @@ -245,15 +252,30 @@ pub fn worker_entrypoint( | CloneFlags::CLONE_NEWPID | CloneFlags::CLONE_NEWUSER | CloneFlags::CLONE_NEWUTS + + // SIGCHLD flag is used to inform clone that the parent process is + // expecting a child termination signal, without this flag `waitpid` function + // return `ECHILD` error. | CloneFlags::from_bits_retain(libc::SIGCHLD); // SAFETY: new process is spawned within a single threaded process. This invariant // is enforced by tests. Stack size being specified to ensure child doesn't overflow let result = match unsafe { nix::sched::clone( Box::new(|| { + // SAFETY: pipe_writer is an open and owned file descriptor at this point. + let mut pipe_writer = PipeWriter::from_raw_fd(pipe_writer); + + if let Err(errno) = nix::unistd::close(pipe_reader) { + send_child_response(&mut pipe_writer, JobResult::Err(error_from_errno("closing pipe", errno))); + } + + if let Err(errno) = nix::unistd::close(stream_fd) { + send_child_response(&mut pipe_writer, JobResult::Err(error_from_errno("error closing stream", errno))); + } + handle_child_process( pvf.clone(), - &mut pipe_writer, + pipe_writer, preparation_timeout, prepare_job_kind, Arc::clone(&executor_params), @@ -263,7 +285,11 @@ pub fn worker_entrypoint( Ok(child) => { // the read end will wait until all write ends have been closed, // this drop is necessary to avoid deadlock - drop(pipe_writer); + nix::unistd::close(pipe_writer)?; + + // SAFETY: pipe_read is an open and owned file descriptor at this point. + let pipe_reader = unsafe { PipeReader::from_raw_fd(pipe_reader) }; + handle_parent_process( pipe_reader, worker_pid, @@ -281,18 +307,25 @@ pub fn worker_entrypoint( let result = match unsafe { nix::unistd::fork() } { Err(errno) => Err(error_from_errno("fork", errno)), Ok(ForkResult::Child) => { + // SAFETY: pipe_writer is an open and owned file descriptor at this point. + let mut pipe_writer = unsafe { PipeWriter::from_raw_fd(pipe_writer) }; + + // Drop the read end so we don't have too many FDs open. + if let Err(errno) = nix::unistd::close(pipe_reader) { + send_child_response(&mut pipe_writer, JobResult::Err(error_from_errno("closing pipe", errno))); + } + // Dropping the stream closes the underlying socket. We want to make sure // that the sandboxed child can't get any kind of information from the // outside world. The only IPC it should be able to do is sending its // response over the pipe. - drop(stream); - - // Drop the read end so we don't have too many FDs open. - drop(pipe_reader); + if let Err(errno) = nix::unistd::close(stream_fd) { + send_child_response(&mut pipe_writer, JobResult::Err(error_from_errno("error closing stream", errno))); + } handle_child_process( pvf, - &mut pipe_writer, + pipe_writer, preparation_timeout, prepare_job_kind, executor_params, @@ -301,18 +334,20 @@ pub fn worker_entrypoint( Ok(ForkResult::Parent { child }) => { // the read end will wait until all write ends have been closed, // this drop is necessary to avoid deadlock - drop(pipe_writer); - - handle_parent_process( - pipe_reader, - worker_pid, - child, - temp_artifact_dest.clone(), - usage_before, - preparation_timeout, - ) - }, - }; + nix::unistd::close(pipe_writer)?; + + // SAFETY: pipe_read is an open and owned file descriptor at this point. + let pipe_reader = unsafe { PipeReader::from_raw_fd(pipe_reader) }; + handle_parent_process( + pipe_reader, + worker_pid, + child, + temp_artifact_dest.clone(), + usage_before, + preparation_timeout, + ) + }, + }; gum::trace!( @@ -382,7 +417,7 @@ struct JobResponse { /// - If success, pipe back `JobResponse`. fn handle_child_process( pvf: PvfPrepData, - pipe_write: &mut PipeWriter, + mut pipe_write: PipeWriter, preparation_timeout: Duration, prepare_job_kind: PrepareJobKind, executor_params: Arc, @@ -431,7 +466,7 @@ fn handle_child_process( WaitOutcome::TimedOut, ) .unwrap_or_else(|err| { - send_child_response(pipe_write, Err(PrepareError::IoErr(err.to_string()))) + send_child_response(&mut pipe_write, Err(PrepareError::IoErr(err.to_string()))) }); let prepare_thread = spawn_worker_thread( @@ -461,7 +496,7 @@ fn handle_child_process( WaitOutcome::Finished, ) .unwrap_or_else(|err| { - send_child_response(pipe_write, Err(PrepareError::IoErr(err.to_string()))) + send_child_response(&mut pipe_write, Err(PrepareError::IoErr(err.to_string()))) }); let outcome = thread::wait_for_threads(condvar); @@ -483,7 +518,7 @@ fn handle_child_process( match prepare_thread.join().unwrap_or_else(|err| { send_child_response( - pipe_write, + &mut pipe_write, Err(PrepareError::JobError(stringify_panic_payload(err))), ) }) { @@ -528,7 +563,7 @@ fn handle_child_process( unreachable!("we run wait_while until the outcome is no longer pending; qed"), }; - send_child_response(pipe_write, result); + send_child_response(&mut pipe_write, result); } /// Waits for child process to finish and handle child response from pipe. diff --git a/polkadot/node/core/pvf/tests/it/adder.rs b/polkadot/node/core/pvf/tests/it/adder.rs index 9a7ddcb40890..08e20d114f19 100644 --- a/polkadot/node/core/pvf/tests/it/adder.rs +++ b/polkadot/node/core/pvf/tests/it/adder.rs @@ -19,6 +19,7 @@ use super::TestHost; use adder::{hash_state, BlockData, HeadData}; use parity_scale_codec::{Decode, Encode}; +use polkadot_node_core_pvf_common::sp_tracing::try_init_simple; use polkadot_parachain_primitives::primitives::{ BlockData as GenericBlockData, HeadData as GenericHeadData, RelayChainBlockNumber, ValidationParams, From 0bac33f0e67ee0e1191cab10f60a343286832c0a Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Wed, 29 Nov 2023 08:45:59 -0300 Subject: [PATCH 06/50] remove unused import --- polkadot/node/core/pvf/prepare-worker/src/lib.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index 71582a00ed3a..7176ecda0de8 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -68,7 +68,7 @@ use std::{ fs, io::{self, Read}, os::{ - fd::{AsRawFd, FromRawFd, IntoRawFd, RawFd}, + fd::RawFd, unix::net::UnixStream, }, path::PathBuf, From 8e313c84a808e83eeacaf54ac0b26518955c147f Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Wed, 29 Nov 2023 08:54:11 -0300 Subject: [PATCH 07/50] fix fmt --- polkadot/node/core/pvf/prepare-worker/src/lib.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index 7176ecda0de8..8011a6b439d6 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -68,7 +68,7 @@ use std::{ fs, io::{self, Read}, os::{ - fd::RawFd, + fd::{AsRawFd, FromRawFd, RawFd}, unix::net::UnixStream, }, path::PathBuf, From ec5b4af1a87c8dad36e81339a68d1f4bed8b9ec3 Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Wed, 29 Nov 2023 09:24:07 -0300 Subject: [PATCH 08/50] remove unused import --- polkadot/node/core/pvf/tests/it/adder.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/polkadot/node/core/pvf/tests/it/adder.rs b/polkadot/node/core/pvf/tests/it/adder.rs index 08e20d114f19..9a7ddcb40890 100644 --- a/polkadot/node/core/pvf/tests/it/adder.rs +++ b/polkadot/node/core/pvf/tests/it/adder.rs @@ -19,7 +19,6 @@ use super::TestHost; use adder::{hash_state, BlockData, HeadData}; use parity_scale_codec::{Decode, Encode}; -use polkadot_node_core_pvf_common::sp_tracing::try_init_simple; use polkadot_parachain_primitives::primitives::{ BlockData as GenericBlockData, HeadData as GenericHeadData, RelayChainBlockNumber, ValidationParams, From c512b43f0cbfcaf76f2c86f1746a61f702e18464 Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Sat, 2 Dec 2023 18:38:28 -0300 Subject: [PATCH 09/50] remove os_pipe --- Cargo.lock | 1 - polkadot/node/core/pvf/common/src/lib.rs | 2 +- .../node/core/pvf/common/src/worker/mod.rs | 45 +------ .../node/core/pvf/prepare-worker/Cargo.toml | 1 - .../node/core/pvf/prepare-worker/src/lib.rs | 118 +++++++++++------- 5 files changed, 76 insertions(+), 91 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index db5717a67ecc..097fa8d49a58 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -12295,7 +12295,6 @@ dependencies = [ "criterion 0.4.0", "libc", "nix 0.27.1", - "os_pipe", "parity-scale-codec", "polkadot-node-core-pvf-common", "polkadot-primitives", diff --git a/polkadot/node/core/pvf/common/src/lib.rs b/polkadot/node/core/pvf/common/src/lib.rs index dced43ef2134..f24febe7578e 100644 --- a/polkadot/node/core/pvf/common/src/lib.rs +++ b/polkadot/node/core/pvf/common/src/lib.rs @@ -46,7 +46,7 @@ pub mod tests { pub const TEST_PREPARATION_TIMEOUT: Duration = Duration::from_secs(30); } -/// Status of security features on the current system. +/// Status of security features on thfe current system. #[derive(Debug, Clone, Default, PartialEq, Eq)] pub struct SecurityStatus { /// Whether the landlock features we use are fully available on this system. diff --git a/polkadot/node/core/pvf/common/src/worker/mod.rs b/polkadot/node/core/pvf/common/src/worker/mod.rs index ceb2c782f86e..06c68ba97188 100644 --- a/polkadot/node/core/pvf/common/src/worker/mod.rs +++ b/polkadot/node/core/pvf/common/src/worker/mod.rs @@ -21,7 +21,6 @@ pub mod security; use crate::{SecurityStatus, LOG_TARGET}; use cpu_time::ProcessTime; use futures::never::Never; -use polkadot_primitives::executor_params::DEFAULT_NATIVE_STACK_MAX; use std::{ any::Any, fmt, io, @@ -31,42 +30,6 @@ use std::{ time::Duration, }; -// Wasmtime powers the Substrate Executor. It compiles the wasm bytecode into native code. -// That native code does not create any stacks and just reuses the stack of the thread that -// wasmtime was invoked from. -// -// Also, we configure the executor to provide the deterministic stack and that requires -// supplying the amount of the native stack space that wasm is allowed to use. This is -// realized by supplying the limit into `wasmtime::Config::max_wasm_stack`. -// -// There are quirks to that configuration knob: -// -// 1. It only limits the amount of stack space consumed by wasm but does not ensure nor check that -// the stack space is actually available. -// -// That means, if the calling thread has 1 MiB of stack space left and the wasm code consumes -// more, then the wasmtime limit will **not** trigger. Instead, the wasm code will hit the -// guard page and the Rust stack overflow handler will be triggered. That leads to an -// **abort**. -// -// 2. It cannot and does not limit the stack space consumed by Rust code. -// -// Meaning that if the wasm code leaves no stack space for Rust code, then the Rust code -// will abort and that will abort the process as well. -// -// Typically on Linux the main thread gets the stack size specified by the `ulimit` and -// typically it's configured to 8 MiB. Rust's spawned threads are 2 MiB. OTOH, the -// DEFAULT_NATIVE_STACK_MAX is set to 256 MiB. Not nearly enough. -// -// Hence we need to increase it. The simplest way to fix that is to spawn a thread with the desired -// stack limit. -// -// The reasoning why we pick this particular size is: -// -// The default Rust thread stack limit 2 MiB + 256 MiB wasm stack. -/// The stack size for the execute thread. -pub const EXECUTE_THREAD_STACK_SIZE: usize = 2 * 1024 * 1024 + DEFAULT_NATIVE_STACK_MAX as usize; - /// Use this macro to declare a `fn main() {}` that will create an executable that can be used for /// spawning the desired worker. #[macro_export] @@ -221,8 +184,9 @@ macro_rules! decl_worker_main { }; } -// os_pipe lib -#[cfg(not(any(target_os = "ios", target_os = "macos", target_os = "haiku")))] +//taken from the os_pipe crate. Copied here to reduce one dependency and +// because its type-safe abstractions do not play well with nix's clone +#[cfg(not(any(target_os = "macos")))] pub fn pipe2_cloexec() -> io::Result<(libc::c_int, libc::c_int)> { let mut fds: [libc::c_int; 2] = [0; 2]; let res = unsafe { libc::pipe2(fds.as_mut_ptr(), libc::O_CLOEXEC) }; @@ -232,8 +196,7 @@ pub fn pipe2_cloexec() -> io::Result<(libc::c_int, libc::c_int)> { Ok((fds[0], fds[1])) } -// os_pipe lib -#[cfg(any(target_os = "ios", target_os = "macos", target_os = "haiku"))] +#[cfg(any(target_os = "macos"))] pub fn pipe2_cloexec() -> io::Result<(libc::c_int, libc::c_int)> { let mut fds: [libc::c_int; 2] = [0; 2]; let res = unsafe { libc::pipe(fds.as_mut_ptr()) }; diff --git a/polkadot/node/core/pvf/prepare-worker/Cargo.toml b/polkadot/node/core/pvf/prepare-worker/Cargo.toml index ff33d119012a..b9ce958b1cdd 100644 --- a/polkadot/node/core/pvf/prepare-worker/Cargo.toml +++ b/polkadot/node/core/pvf/prepare-worker/Cargo.toml @@ -15,7 +15,6 @@ rayon = "1.5.1" tracking-allocator = { package = "staging-tracking-allocator", path = "../../../tracking-allocator" } tikv-jemalloc-ctl = { version = "0.5.0", optional = true } tikv-jemallocator = { version = "0.5.0", optional = true } -os_pipe = "1.1.4" nix = { version = "0.27.1", features = ["resource", "process", "sched"] } parity-scale-codec = { version = "3.6.1", default-features = false, features = ["derive"] } diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index 8011a6b439d6..a2534579eca5 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -48,7 +48,6 @@ use nix::{ #[cfg(target_os = "linux")] use nix::sched::CloneFlags; -use os_pipe::{self, PipeReader, PipeWriter}; use parity_scale_codec::{Decode, Encode}; use polkadot_node_core_pvf_common::{ error::{PrepareError, PrepareWorkerResult}, @@ -244,7 +243,8 @@ pub fn worker_entrypoint( cfg_if::cfg_if! { if #[cfg(target_os = "linux")] { - let mut stack: Vec = vec![0u8; EXECUTE_THREAD_STACK_SIZE]; + let stack_size = 2 * 1024 * 1024; // 2mb + let mut stack: Vec = vec![0u8; stack_size]; let flags = CloneFlags::CLONE_NEWCGROUP | CloneFlags::CLONE_NEWIPC | CloneFlags::CLONE_NEWNET @@ -262,20 +262,11 @@ pub fn worker_entrypoint( // is enforced by tests. Stack size being specified to ensure child doesn't overflow let result = match unsafe { nix::sched::clone( Box::new(|| { - // SAFETY: pipe_writer is an open and owned file descriptor at this point. - let mut pipe_writer = PipeWriter::from_raw_fd(pipe_writer); - - if let Err(errno) = nix::unistd::close(pipe_reader) { - send_child_response(&mut pipe_writer, JobResult::Err(error_from_errno("closing pipe", errno))); - } - - if let Err(errno) = nix::unistd::close(stream_fd) { - send_child_response(&mut pipe_writer, JobResult::Err(error_from_errno("error closing stream", errno))); - } - handle_child_process( pvf.clone(), pipe_writer, + pipe_reader, + stream_fd, preparation_timeout, prepare_job_kind, Arc::clone(&executor_params), @@ -292,6 +283,7 @@ pub fn worker_entrypoint( handle_parent_process( pipe_reader, + pipe_writer, worker_pid, child, temp_artifact_dest.clone(), @@ -307,39 +299,21 @@ pub fn worker_entrypoint( let result = match unsafe { nix::unistd::fork() } { Err(errno) => Err(error_from_errno("fork", errno)), Ok(ForkResult::Child) => { - // SAFETY: pipe_writer is an open and owned file descriptor at this point. - let mut pipe_writer = unsafe { PipeWriter::from_raw_fd(pipe_writer) }; - - // Drop the read end so we don't have too many FDs open. - if let Err(errno) = nix::unistd::close(pipe_reader) { - send_child_response(&mut pipe_writer, JobResult::Err(error_from_errno("closing pipe", errno))); - } - - // Dropping the stream closes the underlying socket. We want to make sure - // that the sandboxed child can't get any kind of information from the - // outside world. The only IPC it should be able to do is sending its - // response over the pipe. - if let Err(errno) = nix::unistd::close(stream_fd) { - send_child_response(&mut pipe_writer, JobResult::Err(error_from_errno("error closing stream", errno))); - } handle_child_process( pvf, pipe_writer, + pipe_reader, + stream_fd, preparation_timeout, prepare_job_kind, executor_params, ) }, Ok(ForkResult::Parent { child }) => { - // the read end will wait until all write ends have been closed, - // this drop is necessary to avoid deadlock - nix::unistd::close(pipe_writer)?; - - // SAFETY: pipe_read is an open and owned file descriptor at this point. - let pipe_reader = unsafe { PipeReader::from_raw_fd(pipe_reader) }; handle_parent_process( pipe_reader, + pipe_writer, worker_pid, child, temp_artifact_dest.clone(), @@ -402,7 +376,13 @@ struct JobResponse { /// /// - `pvf`: `PvfPrepData` structure, containing data to prepare the artifact /// -/// - `pipe_write`: A `PipeWriter` structure, the writing end of a pipe. +/// - `pipe_write_fd`: A `i32`, refers to pipe write end file descriptor. +/// +/// - `pipe_read_fd`: A `i32`, refers to pipe read end file descriptor, used to close the file +/// descriptor in the child process. +/// +/// - `stream_fd`: A `i32`, refers to UnixStream file descriptor, used to close the file descriptor +/// in the child process. /// /// - `preparation_timeout`: The timeout in `Duration`. /// @@ -417,11 +397,32 @@ struct JobResponse { /// - If success, pipe back `JobResponse`. fn handle_child_process( pvf: PvfPrepData, - mut pipe_write: PipeWriter, + pipe_write_fd: i32, + pipe_read_fd: i32, + stream_fd: i32, preparation_timeout: Duration, prepare_job_kind: PrepareJobKind, executor_params: Arc, ) -> ! { + // SAFETY: pipe_writer is an open and owned file descriptor at this point. + let pipe_write = unsafe { FromRawFd::from_raw_fd(pipe_write_fd) }; + + // Drop the read end so we don't have too many FDs open. + if let Err(errno) = nix::unistd::close(pipe_read_fd) { + send_child_response(pipe_write, JobResult::Err(error_from_errno("closing pipe", errno))); + } + + // Dropping the stream closes the underlying socket. We want to make sure + // that the sandboxed child can't get any kind of information from the + // outside world. The only IPC it should be able to do is sending its + // response over the pipe. + if let Err(errno) = nix::unistd::close(stream_fd) { + send_child_response( + pipe_write, + JobResult::Err(error_from_errno("error closing stream", errno)), + ); + } + let worker_job_pid = process::id(); gum::debug!( target: LOG_TARGET, @@ -466,7 +467,7 @@ fn handle_child_process( WaitOutcome::TimedOut, ) .unwrap_or_else(|err| { - send_child_response(&mut pipe_write, Err(PrepareError::IoErr(err.to_string()))) + send_child_response(pipe_write, Err(PrepareError::IoErr(err.to_string()))) }); let prepare_thread = spawn_worker_thread( @@ -496,7 +497,7 @@ fn handle_child_process( WaitOutcome::Finished, ) .unwrap_or_else(|err| { - send_child_response(&mut pipe_write, Err(PrepareError::IoErr(err.to_string()))) + send_child_response(pipe_write, Err(PrepareError::IoErr(err.to_string()))) }); let outcome = thread::wait_for_threads(condvar); @@ -518,7 +519,7 @@ fn handle_child_process( match prepare_thread.join().unwrap_or_else(|err| { send_child_response( - &mut pipe_write, + pipe_write, Err(PrepareError::JobError(stringify_panic_payload(err))), ) }) { @@ -563,14 +564,18 @@ fn handle_child_process( unreachable!("we run wait_while until the outcome is no longer pending; qed"), }; - send_child_response(&mut pipe_write, result); + send_child_response(pipe_write, result); } /// Waits for child process to finish and handle child response from pipe. /// /// # Arguments /// -/// - `pipe_read`: A `PipeReader` used to read data from the child process. +/// - `pipe_read_fd`: A `i32`, refers to pipe read end file descriptor, used to read data from the +/// child process. +/// +/// - `pipe_write_fd`: A `i32`, refers to pipe write end file descriptor, used to close file +/// descriptor in the parent process. /// /// - `child`: The child pid. /// @@ -591,17 +596,25 @@ fn handle_child_process( /// /// - If the child process timeout, it returns `PrepareError::TimedOut`. fn handle_parent_process( - mut pipe_read: PipeReader, + pipe_read_fd: i32, + pipe_write_fd: i32, worker_pid: u32, job_pid: Pid, temp_artifact_dest: PathBuf, usage_before: Usage, timeout: Duration, ) -> Result { + // the read end will wait until all write ends have been closed, + // this drop is necessary to avoid deadlock + if let Err(errno) = nix::unistd::close(pipe_write_fd) { + return Err(error_from_errno("closing pipe write fd", errno)); + }; + + //SAFETY: pipe_read is an open and owned file descriptor at this point. + let pipe_read: RawFd = unsafe { FromRawFd::from_raw_fd(pipe_read_fd) }; // Read from the child. Don't decode unless the process exited normally, which we check later. let mut received_data = Vec::new(); - pipe_read - .read_to_end(&mut received_data) + let readed = nix::unistd::read(pipe_read, &mut received_data) .map_err(|err| PrepareError::IoErr(err.to_string()))?; let status = nix::sys::wait::waitpid(job_pid, None); @@ -733,9 +746,20 @@ fn recv_child_response(received_data: &mut io::BufReader<&[u8]>) -> io::Result ! { - framed_send_blocking(pipe_write, response.encode().as_slice()) - .unwrap_or_else(|_| process::exit(libc::EXIT_FAILURE)); +fn send_child_response(pipe_write_fd: RawFd, response: JobResult) -> ! { + let binding = response.encode(); + let buffer = binding.as_slice(); + let half = buffer.len() / 2; + nix::unistd::write(pipe_write_fd, &buffer[..half]).unwrap_or_else(|err| { + let err_string = err.to_string(); + + process::exit(libc::EXIT_FAILURE); + }); + nix::unistd::write(pipe_write_fd, &buffer[half..]).unwrap_or_else(|err| { + let err_string = err.to_string(); + + process::exit(libc::EXIT_FAILURE); + }); if response.is_ok() { process::exit(libc::EXIT_SUCCESS) From eca157ceaf0bb2e6323526d0f2efedba58ac18ab Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Sun, 3 Dec 2023 11:43:08 -0300 Subject: [PATCH 10/50] use file instead of raw fd --- .../node/core/pvf/prepare-worker/src/lib.rs | 43 ++++++++----------- 1 file changed, 18 insertions(+), 25 deletions(-) diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index a2534579eca5..a5dacc2c37f7 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -64,8 +64,8 @@ use polkadot_node_core_pvf_common::{ }; use polkadot_primitives::ExecutorParams; use std::{ - fs, - io::{self, Read}, + fs::{self, File}, + io::{self, Read, Write}, os::{ fd::{AsRawFd, FromRawFd, RawFd}, unix::net::UnixStream, @@ -405,11 +405,14 @@ fn handle_child_process( executor_params: Arc, ) -> ! { // SAFETY: pipe_writer is an open and owned file descriptor at this point. - let pipe_write = unsafe { FromRawFd::from_raw_fd(pipe_write_fd) }; + let mut pipe_write = unsafe { File::from_raw_fd(pipe_write_fd) }; // Drop the read end so we don't have too many FDs open. if let Err(errno) = nix::unistd::close(pipe_read_fd) { - send_child_response(pipe_write, JobResult::Err(error_from_errno("closing pipe", errno))); + send_child_response( + &mut pipe_write, + JobResult::Err(error_from_errno("closing pipe", errno)), + ); } // Dropping the stream closes the underlying socket. We want to make sure @@ -418,7 +421,7 @@ fn handle_child_process( // response over the pipe. if let Err(errno) = nix::unistd::close(stream_fd) { send_child_response( - pipe_write, + &mut pipe_write, JobResult::Err(error_from_errno("error closing stream", errno)), ); } @@ -467,7 +470,7 @@ fn handle_child_process( WaitOutcome::TimedOut, ) .unwrap_or_else(|err| { - send_child_response(pipe_write, Err(PrepareError::IoErr(err.to_string()))) + send_child_response(&mut pipe_write, Err(PrepareError::IoErr(err.to_string()))) }); let prepare_thread = spawn_worker_thread( @@ -497,7 +500,7 @@ fn handle_child_process( WaitOutcome::Finished, ) .unwrap_or_else(|err| { - send_child_response(pipe_write, Err(PrepareError::IoErr(err.to_string()))) + send_child_response(&mut pipe_write, Err(PrepareError::IoErr(err.to_string()))) }); let outcome = thread::wait_for_threads(condvar); @@ -519,7 +522,7 @@ fn handle_child_process( match prepare_thread.join().unwrap_or_else(|err| { send_child_response( - pipe_write, + &mut pipe_write, Err(PrepareError::JobError(stringify_panic_payload(err))), ) }) { @@ -564,7 +567,7 @@ fn handle_child_process( unreachable!("we run wait_while until the outcome is no longer pending; qed"), }; - send_child_response(pipe_write, result); + send_child_response(&mut pipe_write, result); } /// Waits for child process to finish and handle child response from pipe. @@ -611,10 +614,11 @@ fn handle_parent_process( }; //SAFETY: pipe_read is an open and owned file descriptor at this point. - let pipe_read: RawFd = unsafe { FromRawFd::from_raw_fd(pipe_read_fd) }; + let mut pipe_read = unsafe { File::from_raw_fd(pipe_read_fd) }; // Read from the child. Don't decode unless the process exited normally, which we check later. let mut received_data = Vec::new(); - let readed = nix::unistd::read(pipe_read, &mut received_data) + pipe_read + .read_to_end(&mut received_data) .map_err(|err| PrepareError::IoErr(err.to_string()))?; let status = nix::sys::wait::waitpid(job_pid, None); @@ -746,20 +750,9 @@ fn recv_child_response(received_data: &mut io::BufReader<&[u8]>) -> io::Result ! { - let binding = response.encode(); - let buffer = binding.as_slice(); - let half = buffer.len() / 2; - nix::unistd::write(pipe_write_fd, &buffer[..half]).unwrap_or_else(|err| { - let err_string = err.to_string(); - - process::exit(libc::EXIT_FAILURE); - }); - nix::unistd::write(pipe_write_fd, &buffer[half..]).unwrap_or_else(|err| { - let err_string = err.to_string(); - - process::exit(libc::EXIT_FAILURE); - }); +fn send_child_response(pipe_write: &mut File, response: JobResult) -> ! { + framed_send_blocking(pipe_write, response.encode().as_slice()) + .unwrap_or_else(|_| process::exit(libc::EXIT_FAILURE)); if response.is_ok() { process::exit(libc::EXIT_SUCCESS) From 2fc41804d54950c254cec6b4b873a5e86fe75fcb Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Mon, 18 Dec 2023 08:35:51 -0300 Subject: [PATCH 11/50] minor fixes --- polkadot/node/core/pvf/common/src/lib.rs | 2 +- polkadot/node/core/pvf/prepare-worker/src/lib.rs | 12 +----------- 2 files changed, 2 insertions(+), 12 deletions(-) diff --git a/polkadot/node/core/pvf/common/src/lib.rs b/polkadot/node/core/pvf/common/src/lib.rs index abebd06f71a4..3a7d2161201b 100644 --- a/polkadot/node/core/pvf/common/src/lib.rs +++ b/polkadot/node/core/pvf/common/src/lib.rs @@ -85,4 +85,4 @@ pub fn framed_recv_blocking(r: &mut (impl Read + Unpin)) -> io::Result> let mut buf = vec![0; len]; r.read_exact(&mut buf)?; Ok(buf) -} +} \ No newline at end of file diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index 3d5584410174..e7309d545447 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -23,9 +23,6 @@ use polkadot_node_core_pvf_common::{ worker::pipe2_cloexec, }; -#[cfg(target_os = "linux")] -use polkadot_node_core_pvf_common::worker::EXECUTE_THREAD_STACK_SIZE; - // NOTE: Initializing logging in e.g. tests will not have an effect in the workers, as they are // separate spawned processes. Run with e.g. `RUST_LOG=parachain::pvf-prepare-worker=trace`. const LOG_TARGET: &str = "parachain::pvf-prepare-worker"; @@ -241,7 +238,7 @@ pub fn worker_entrypoint( cfg_if::cfg_if! { if #[cfg(target_os = "linux")] { - let stack_size = 2 * 1024 * 1024; // 2mb + let stack_size = 2 * 1024 * 1024; // 2MiB let mut stack: Vec = vec![0u8; stack_size]; let flags = CloneFlags::CLONE_NEWCGROUP | CloneFlags::CLONE_NEWIPC @@ -272,13 +269,6 @@ pub fn worker_entrypoint( } { Err(errno) => Err(error_from_errno("clone", errno)), Ok(child) => { - // the read end will wait until all write ends have been closed, - // this drop is necessary to avoid deadlock - nix::unistd::close(pipe_writer)?; - - // SAFETY: pipe_read is an open and owned file descriptor at this point. - let pipe_reader = unsafe { PipeReader::from_raw_fd(pipe_reader) }; - handle_parent_process( pipe_reader, pipe_writer, From 6c39804ec4cd1f972cc50f7cf054d62d031e5a25 Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Mon, 18 Dec 2023 14:30:42 +0100 Subject: [PATCH 12/50] fmt manually Some code was not formatted (because rustfmt skips macros) --- polkadot/node/core/pvf/common/src/lib.rs | 2 +- .../node/core/pvf/prepare-worker/src/lib.rs | 61 ++++++++++--------- 2 files changed, 34 insertions(+), 29 deletions(-) diff --git a/polkadot/node/core/pvf/common/src/lib.rs b/polkadot/node/core/pvf/common/src/lib.rs index 3a7d2161201b..abebd06f71a4 100644 --- a/polkadot/node/core/pvf/common/src/lib.rs +++ b/polkadot/node/core/pvf/common/src/lib.rs @@ -85,4 +85,4 @@ pub fn framed_recv_blocking(r: &mut (impl Read + Unpin)) -> io::Result> let mut buf = vec![0; len]; r.read_exact(&mut buf)?; Ok(buf) -} \ No newline at end of file +} diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index e7309d545447..71d66f1a70cf 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -240,32 +240,37 @@ pub fn worker_entrypoint( if #[cfg(target_os = "linux")] { let stack_size = 2 * 1024 * 1024; // 2MiB let mut stack: Vec = vec![0u8; stack_size]; - let flags = CloneFlags::CLONE_NEWCGROUP - | CloneFlags::CLONE_NEWIPC - | CloneFlags::CLONE_NEWNET - | CloneFlags::CLONE_NEWNS - | CloneFlags::CLONE_NEWPID - | CloneFlags::CLONE_NEWUSER - | CloneFlags::CLONE_NEWUTS - // SIGCHLD flag is used to inform clone that the parent process is // expecting a child termination signal, without this flag `waitpid` function - // return `ECHILD` error. - | CloneFlags::from_bits_retain(libc::SIGCHLD); + // return `ECHILD` error. + let flags = CloneFlags::CLONE_NEWCGROUP + | CloneFlags::CLONE_NEWIPC + | CloneFlags::CLONE_NEWNET + | CloneFlags::CLONE_NEWNS + | CloneFlags::CLONE_NEWPID + | CloneFlags::CLONE_NEWUSER + | CloneFlags::CLONE_NEWUTS + | CloneFlags::from_bits_retain(libc::SIGCHLD); // SAFETY: new process is spawned within a single threaded process. This invariant // is enforced by tests. Stack size being specified to ensure child doesn't overflow - let result = match unsafe { nix::sched::clone( - Box::new(|| { - handle_child_process( - pvf.clone(), - pipe_writer, - pipe_reader, - stream_fd, - preparation_timeout, - prepare_job_kind, - Arc::clone(&executor_params), - ); }), stack.as_mut_slice(), flags, None) + let result = match unsafe { + nix::sched::clone( + Box::new(|| { + handle_child_process( + pvf.clone(), + pipe_writer, + pipe_reader, + stream_fd, + preparation_timeout, + prepare_job_kind, + Arc::clone(&executor_params), + ) + }), + stack.as_mut_slice(), + flags, + None + ) } { Err(errno) => Err(error_from_errno("clone", errno)), Ok(child) => { @@ -300,13 +305,13 @@ pub fn worker_entrypoint( }, Ok(ForkResult::Parent { child }) => { handle_parent_process( - pipe_reader, - pipe_writer, - worker_pid, - child, - temp_artifact_dest.clone(), - usage_before, - preparation_timeout, + pipe_reader, + pipe_writer, + worker_pid, + child, + temp_artifact_dest.clone(), + usage_before, + preparation_timeout, ) }, }; From 5304a5945cd36b32adeabb767aacdffa86e37839 Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Tue, 19 Dec 2023 07:13:00 -0300 Subject: [PATCH 13/50] wrap File type with PipeFd --- .../node/core/pvf/common/src/worker/mod.rs | 38 +++++++++++++++++-- .../node/core/pvf/prepare-worker/src/lib.rs | 8 ++-- 2 files changed, 38 insertions(+), 8 deletions(-) diff --git a/polkadot/node/core/pvf/common/src/worker/mod.rs b/polkadot/node/core/pvf/common/src/worker/mod.rs index a2aea49b7242..8423058601a7 100644 --- a/polkadot/node/core/pvf/common/src/worker/mod.rs +++ b/polkadot/node/core/pvf/common/src/worker/mod.rs @@ -24,11 +24,11 @@ use futures::never::Never; use parity_scale_codec::Decode; use std::{ any::Any, - fmt, io, - os::unix::net::UnixStream, + fmt::{self}, + os::{unix::net::UnixStream, fd::{FromRawFd, AsRawFd}}, path::PathBuf, sync::mpsc::{Receiver, RecvTimeoutError}, - time::Duration, + time::Duration, fs::File, io::{Write, self}, }; /// Use this macro to declare a `fn main() {}` that will create an executable that can be used for @@ -173,7 +173,7 @@ macro_rules! decl_worker_main { //taken from the os_pipe crate. Copied here to reduce one dependency and // because its type-safe abstractions do not play well with nix's clone -#[cfg(not(any(target_os = "macos")))] +#[cfg(not(target_os = "macos"))] pub fn pipe2_cloexec() -> io::Result<(libc::c_int, libc::c_int)> { let mut fds: [libc::c_int; 2] = [0; 2]; let res = unsafe { libc::pipe2(fds.as_mut_ptr(), libc::O_CLOEXEC) }; @@ -201,6 +201,36 @@ pub fn pipe2_cloexec() -> io::Result<(libc::c_int, libc::c_int)> { Ok((fds[0], fds[1])) } +pub struct PipeFd { + file: File, +} + +impl PipeFd { + pub fn new(fd: i32) -> Self { + // SAFETY: pipe_writer is an open and owned file descriptor at this point. + let file = unsafe { File::from_raw_fd(fd) }; + PipeFd { file } + } + + pub fn as_raw_fd(&self) -> i32 { + self.file.as_raw_fd() + } +} + +impl Write for PipeFd { + fn write(&mut self, buf: &[u8]) -> io::Result { + self.file.write(buf) + } + + fn flush(&mut self) -> io::Result<()> { + self.file.flush() + } + + fn write_all(&mut self, buf: &[u8]) -> io::Result<()> { + self.file.write_all(buf) + } +} + /// Some allowed overhead that we account for in the "CPU time monitor" thread's sleeps, on the /// child process. pub const JOB_TIMEOUT_OVERHEAD: Duration = Duration::from_millis(50); diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index e7309d545447..ad503efc0494 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -20,7 +20,7 @@ mod memory_stats; use polkadot_node_core_pvf_common::{ executor_interface::{prepare, prevalidate}, - worker::pipe2_cloexec, + worker::{pipe2_cloexec, PipeFd}, }; // NOTE: Initializing logging in e.g. tests will not have an effect in the workers, as they are @@ -62,7 +62,7 @@ use polkadot_node_core_pvf_common::{ use polkadot_primitives::ExecutorParams; use std::{ fs::{self, File}, - io::{self, Read, Write}, + io::{self, Read}, os::{ fd::{AsRawFd, FromRawFd, RawFd}, unix::net::UnixStream, @@ -393,7 +393,7 @@ fn handle_child_process( executor_params: Arc, ) -> ! { // SAFETY: pipe_writer is an open and owned file descriptor at this point. - let mut pipe_write = unsafe { File::from_raw_fd(pipe_write_fd) }; + let mut pipe_write = unsafe { PipeFd::new(pipe_write_fd) }; // Drop the read end so we don't have too many FDs open. if let Err(errno) = nix::unistd::close(pipe_read_fd) { @@ -738,7 +738,7 @@ fn recv_child_response(received_data: &mut io::BufReader<&[u8]>) -> io::Result ! { +fn send_child_response(pipe_write: &mut PipeFd, response: JobResult) -> ! { framed_send_blocking(pipe_write, response.encode().as_slice()) .unwrap_or_else(|_| process::exit(libc::EXIT_FAILURE)); From 4d9e5ef0c2081ec2a881284589ace98103fec772 Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Tue, 19 Dec 2023 07:14:02 -0300 Subject: [PATCH 14/50] fmt --- polkadot/node/core/pvf/common/src/worker/mod.rs | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/polkadot/node/core/pvf/common/src/worker/mod.rs b/polkadot/node/core/pvf/common/src/worker/mod.rs index 8423058601a7..c588e71ed885 100644 --- a/polkadot/node/core/pvf/common/src/worker/mod.rs +++ b/polkadot/node/core/pvf/common/src/worker/mod.rs @@ -25,10 +25,15 @@ use parity_scale_codec::Decode; use std::{ any::Any, fmt::{self}, - os::{unix::net::UnixStream, fd::{FromRawFd, AsRawFd}}, + fs::File, + io::{self, Write}, + os::{ + fd::{AsRawFd, FromRawFd}, + unix::net::UnixStream, + }, path::PathBuf, sync::mpsc::{Receiver, RecvTimeoutError}, - time::Duration, fs::File, io::{Write, self}, + time::Duration, }; /// Use this macro to declare a `fn main() {}` that will create an executable that can be used for @@ -208,9 +213,9 @@ pub struct PipeFd { impl PipeFd { pub fn new(fd: i32) -> Self { // SAFETY: pipe_writer is an open and owned file descriptor at this point. - let file = unsafe { File::from_raw_fd(fd) }; - PipeFd { file } - } + let file = unsafe { File::from_raw_fd(fd) }; + PipeFd { file } + } pub fn as_raw_fd(&self) -> i32 { self.file.as_raw_fd() From 4cf8949b50eb1b8f922b8609e3d604e88402c68f Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Tue, 19 Dec 2023 08:06:29 -0300 Subject: [PATCH 15/50] remove unnecessary `unsafe` block --- polkadot/node/core/pvf/prepare-worker/src/lib.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index 8810bd3a6f87..1eea8c1a4dfe 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -397,8 +397,7 @@ fn handle_child_process( prepare_job_kind: PrepareJobKind, executor_params: Arc, ) -> ! { - // SAFETY: pipe_writer is an open and owned file descriptor at this point. - let mut pipe_write = unsafe { PipeFd::new(pipe_write_fd) }; + let mut pipe_write = PipeFd::new(pipe_write_fd); // Drop the read end so we don't have too many FDs open. if let Err(errno) = nix::unistd::close(pipe_read_fd) { From 1df58cd1016ac0c575156c57bc0ce90621c831b5 Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Tue, 19 Dec 2023 18:22:05 -0300 Subject: [PATCH 16/50] add clone to excute worker --- .../node/core/pvf/common/src/worker/mod.rs | 6 +- .../node/core/pvf/execute-worker/Cargo.toml | 2 +- .../node/core/pvf/execute-worker/src/lib.rs | 186 +++++++++++++----- .../node/core/pvf/prepare-worker/src/lib.rs | 23 +-- 4 files changed, 156 insertions(+), 61 deletions(-) diff --git a/polkadot/node/core/pvf/common/src/worker/mod.rs b/polkadot/node/core/pvf/common/src/worker/mod.rs index c588e71ed885..95128e9bc12d 100644 --- a/polkadot/node/core/pvf/common/src/worker/mod.rs +++ b/polkadot/node/core/pvf/common/src/worker/mod.rs @@ -26,7 +26,7 @@ use std::{ any::Any, fmt::{self}, fs::File, - io::{self, Write}, + io::{self, Write, Read}, os::{ fd::{AsRawFd, FromRawFd}, unix::net::UnixStream, @@ -220,6 +220,10 @@ impl PipeFd { pub fn as_raw_fd(&self) -> i32 { self.file.as_raw_fd() } + + pub fn read_to_end(&mut self, buf: &mut Vec) -> io::Result { + self.file.read_to_end(buf) + } } impl Write for PipeFd { diff --git a/polkadot/node/core/pvf/execute-worker/Cargo.toml b/polkadot/node/core/pvf/execute-worker/Cargo.toml index 97dde59ebc2e..297e26f6c85a 100644 --- a/polkadot/node/core/pvf/execute-worker/Cargo.toml +++ b/polkadot/node/core/pvf/execute-worker/Cargo.toml @@ -12,7 +12,7 @@ workspace = true [dependencies] cpu-time = "1.0.0" gum = { package = "tracing-gum", path = "../../../gum" } -os_pipe = "1.1.4" +cfg-if = "1.0" nix = { version = "0.27.1", features = ["process", "resource"] } libc = "0.2.139" diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index b33a9d5069df..185b4a6257e9 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -31,7 +31,6 @@ use nix::{ }, unistd::{ForkResult, Pid}, }; -use os_pipe::{self, PipeReader, PipeWriter}; use parity_scale_codec::{Decode, Encode}; use polkadot_node_core_pvf_common::{ error::InternalValidationError, @@ -40,14 +39,14 @@ use polkadot_node_core_pvf_common::{ worker::{ cpu_time_monitor_loop, run_worker, stringify_panic_payload, thread::{self, WaitOutcome}, - WorkerKind, + WorkerKind, pipe2_cloexec, PipeFd, }, }; use polkadot_parachain_primitives::primitives::ValidationResult; use polkadot_primitives::{executor_params::DEFAULT_NATIVE_STACK_MAX, ExecutorParams}; use std::{ io::{self, Read}, - os::unix::net::UnixStream, + os::{unix::net::UnixStream, fd::AsRawFd}, path::PathBuf, process, sync::{mpsc::channel, Arc}, @@ -172,7 +171,7 @@ pub fn worker_entrypoint( }, }; - let (pipe_reader, pipe_writer) = os_pipe::pipe()?; + let (pipe_reader, pipe_writer) = pipe2_cloexec()?; let usage_before = match nix::sys::resource::getrusage(UsageWho::RUSAGE_CHILDREN) { Ok(usage) => usage, @@ -182,44 +181,103 @@ pub fn worker_entrypoint( continue }, }; - - // SAFETY: new process is spawned within a single threaded process. This invariant - // is enforced by tests. - let response = match unsafe { nix::unistd::fork() } { - Err(errno) => internal_error_from_errno("fork", errno), - Ok(ForkResult::Child) => { - // Dropping the stream closes the underlying socket. We want to make sure - // that the sandboxed child can't get any kind of information from the - // outside world. The only IPC it should be able to do is sending its - // response over the pipe. - drop(stream); - // Drop the read end so we don't have too many FDs open. - drop(pipe_reader); - - handle_child_process( - pipe_writer, - compiled_artifact_blob, - executor_params, - params, - execution_timeout, - ) - }, - Ok(ForkResult::Parent { child }) => { - // the read end will wait until all write ends have been closed, - // this drop is necessary to avoid deadlock - drop(pipe_writer); - - handle_parent_process( - pipe_reader, - child, - worker_pid, - usage_before, - execution_timeout, - )? - }, - }; - - send_response(&mut stream, response)?; + let stream_fd = stream.as_raw_fd(); + + cfg_if::cfg_if! { + if #[cfg(target_os = "linux")] { + let stack_size = 2 * 1024 * 1024; // 2MiB + let mut stack: Vec = vec![0u8; stack_size]; + // SIGCHLD flag is used to inform clone that the parent process is + // expecting a child termination signal, without this flag `waitpid` function + // return `ECHILD` error. + let flags = CloneFlags::CLONE_NEWCGROUP + | CloneFlags::CLONE_NEWIPC + | CloneFlags::CLONE_NEWNET + | CloneFlags::CLONE_NEWNS + | CloneFlags::CLONE_NEWPID + | CloneFlags::CLONE_NEWUSER + | CloneFlags::CLONE_NEWUTS + | CloneFlags::from_bits_retain(libc::SIGCHLD); + + // SAFETY: new process is spawned within a single threaded process. This invariant + // is enforced by tests. Stack size being specified to ensure child doesn't overflow + let result = match unsafe { + nix::sched::clone( + Box::new(|| { + handle_child_process( + pipe_writer, + pipe_reader, + stream_fd, + compiled_artifact_blob, + executor_params, + params, + execution_timeout, + ) + }), + stack.as_mut_slice(), + flags, + None + ) + } { + Err(errno) => Err(error_from_errno("clone", errno)), + Ok(child) => { + handle_parent_process( + pipe_reader, + pipe_writer, + child, + worker_pid, + usage_before, + execution_timeout, + ) + }, + }; + + + gum::trace!( + target: LOG_TARGET, + %worker_pid, + "worker: sending result to host: {:?}", + result + ); + send_response(&mut stream, result)?; + } else { + // SAFETY: new process is spawned within a single threaded process. This invariant + // is enforced by tests. + let result = match unsafe { nix::unistd::fork() } { + Err(errno) => internal_error_from_errno("fork", errno), + Ok(ForkResult::Child) => { + handle_child_process( + pipe_writer, + pipe_reader, + stream_fd, + compiled_artifact_blob, + executor_params, + params, + execution_timeout, + ) + }, + Ok(ForkResult::Parent { child }) => { + handle_parent_process( + pipe_reader, + pipe_writer, + child, + worker_pid, + usage_before, + execution_timeout, + )? + }, + }; + + + gum::trace!( + target: LOG_TARGET, + %worker_pid, + "worker: sending result to host: {:?}", + result + ); + send_response(&mut stream, result)?; + } + } } }, ); @@ -271,12 +329,35 @@ fn validate_using_artifact( /// /// - pipe back `JobResponse` to the parent process. fn handle_child_process( - mut pipe_write: PipeWriter, + pipe_write_fd: i32, + pipe_read_fd: i32, + stream_fd: i32, compiled_artifact_blob: Vec, executor_params: ExecutorParams, params: Vec, execution_timeout: Duration, ) -> ! { + let mut pipe_write = PipeFd::new(pipe_write_fd); + + // Drop the read end so we don't have too many FDs open. + if let Err(errno) = nix::unistd::close(pipe_read_fd) { + send_child_response( + &mut pipe_write, + Err(JobError::Panic("closing pipe".to_string())), + ); + } + + // Dropping the stream closes the underlying socket. We want to make sure + // that the sandboxed child can't get any kind of information from the + // outside world. The only IPC it should be able to do is sending its + // response over the pipe. + if let Err(errno) = nix::unistd::close(stream_fd) { + send_child_response( + &mut pipe_write, + Err(JobError::Panic(format!("error closing stream {}", errno))), + ); + } + gum::debug!( target: LOG_TARGET, worker_job_pid = %process::id(), @@ -350,12 +431,21 @@ fn handle_child_process( /// /// - The response, either `Ok` or some error state. fn handle_parent_process( - mut pipe_read: PipeReader, + pipe_read_fd: i32, + pipe_write_fd: i32, job_pid: Pid, worker_pid: u32, usage_before: Usage, timeout: Duration, ) -> io::Result { + // the read end will wait until all write ends have been closed, + // this drop is necessary to avoid deadlock + if let Err(errno) = nix::unistd::close(pipe_write_fd) { + return Ok(internal_error_from_errno("closing pipe write fd", errno)); + }; + + let mut pipe_read = PipeFd::new(pipe_read_fd); + // Read from the child. Don't decode unless the process exited normally, which we check later. let mut received_data = Vec::new(); pipe_read @@ -481,14 +571,14 @@ fn recv_child_response(received_data: &mut io::BufReader<&[u8]>) -> io::Result ! { +/// - `response`: Child process response +fn send_child_response(pipe_write: &mut PipeFd, response: JobResult) -> ! { framed_send_blocking(pipe_write, response.encode().as_slice()) .unwrap_or_else(|_| process::exit(libc::EXIT_FAILURE)); diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index 1eea8c1a4dfe..f6a7f955bc9c 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -315,17 +315,17 @@ pub fn worker_entrypoint( ) }, }; - - - gum::trace!( - target: LOG_TARGET, - %worker_pid, - "worker: sending result to host: {:?}", - result - ); - send_response(&mut stream, result)?; } } + + + gum::trace!( + target: LOG_TARGET, + %worker_pid, + "worker: sending result to host: {:?}", + result + ); + send_response(&mut stream, result)?; } }, ); @@ -606,7 +606,8 @@ fn handle_parent_process( }; //SAFETY: pipe_read is an open and owned file descriptor at this point. - let mut pipe_read = unsafe { File::from_raw_fd(pipe_read_fd) }; + let mut pipe_read = unsafe { PipeFd::new(pipe_read_fd) }; + // Read from the child. Don't decode unless the process exited normally, which we check later. let mut received_data = Vec::new(); pipe_read @@ -739,7 +740,7 @@ fn recv_child_response(received_data: &mut io::BufReader<&[u8]>) -> io::Result ! { From c85f326ad7b6e7a1d1a1e3fe935b6e38a2aca5a3 Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Tue, 19 Dec 2023 18:22:45 -0300 Subject: [PATCH 17/50] add clone to excute worker --- Cargo.lock | 12 +----------- polkadot/node/core/pvf/common/src/worker/mod.rs | 2 +- polkadot/node/core/pvf/execute-worker/src/lib.rs | 11 ++++------- polkadot/node/core/pvf/prepare-worker/src/lib.rs | 1 - 4 files changed, 6 insertions(+), 20 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index c451e42cc684..870c42040c30 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8959,16 +8959,6 @@ dependencies = [ "num-traits", ] -[[package]] -name = "os_pipe" -version = "1.1.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ae859aa07428ca9a929b936690f8b12dc5f11dd8c6992a18ca93919f28bc177" -dependencies = [ - "libc", - "windows-sys 0.48.0", -] - [[package]] name = "os_str_bytes" version = "6.5.1" @@ -12512,10 +12502,10 @@ dependencies = [ name = "polkadot-node-core-pvf-execute-worker" version = "1.0.0" dependencies = [ + "cfg-if", "cpu-time", "libc", "nix 0.27.1", - "os_pipe", "parity-scale-codec", "polkadot-node-core-pvf-common", "polkadot-parachain-primitives", diff --git a/polkadot/node/core/pvf/common/src/worker/mod.rs b/polkadot/node/core/pvf/common/src/worker/mod.rs index 95128e9bc12d..dbf78311c66a 100644 --- a/polkadot/node/core/pvf/common/src/worker/mod.rs +++ b/polkadot/node/core/pvf/common/src/worker/mod.rs @@ -26,7 +26,7 @@ use std::{ any::Any, fmt::{self}, fs::File, - io::{self, Write, Read}, + io::{self, Read, Write}, os::{ fd::{AsRawFd, FromRawFd}, unix::net::UnixStream, diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index 185b4a6257e9..359a6738a987 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -37,16 +37,16 @@ use polkadot_node_core_pvf_common::{ execute::{Handshake, JobError, JobResponse, JobResult, WorkerResponse}, framed_recv_blocking, framed_send_blocking, worker::{ - cpu_time_monitor_loop, run_worker, stringify_panic_payload, + cpu_time_monitor_loop, pipe2_cloexec, run_worker, stringify_panic_payload, thread::{self, WaitOutcome}, - WorkerKind, pipe2_cloexec, PipeFd, + PipeFd, WorkerKind, }, }; use polkadot_parachain_primitives::primitives::ValidationResult; use polkadot_primitives::{executor_params::DEFAULT_NATIVE_STACK_MAX, ExecutorParams}; use std::{ io::{self, Read}, - os::{unix::net::UnixStream, fd::AsRawFd}, + os::{fd::AsRawFd, unix::net::UnixStream}, path::PathBuf, process, sync::{mpsc::channel, Arc}, @@ -341,10 +341,7 @@ fn handle_child_process( // Drop the read end so we don't have too many FDs open. if let Err(errno) = nix::unistd::close(pipe_read_fd) { - send_child_response( - &mut pipe_write, - Err(JobError::Panic("closing pipe".to_string())), - ); + send_child_response(&mut pipe_write, Err(JobError::Panic("closing pipe".to_string()))); } // Dropping the stream closes the underlying socket. We want to make sure diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index f6a7f955bc9c..52b7d48f28ad 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -318,7 +318,6 @@ pub fn worker_entrypoint( } } - gum::trace!( target: LOG_TARGET, %worker_pid, From e87f1457409d37fe26934ed3046182d11767e506 Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Wed, 20 Dec 2023 06:53:28 -0300 Subject: [PATCH 18/50] add documentation to PipeFd --- .../node/core/pvf/common/src/worker/mod.rs | 19 ++++++++++++++++--- .../node/core/pvf/execute-worker/src/lib.rs | 6 ++++-- .../node/core/pvf/prepare-worker/src/lib.rs | 3 ++- 3 files changed, 22 insertions(+), 6 deletions(-) diff --git a/polkadot/node/core/pvf/common/src/worker/mod.rs b/polkadot/node/core/pvf/common/src/worker/mod.rs index dbf78311c66a..f2d4375962b2 100644 --- a/polkadot/node/core/pvf/common/src/worker/mod.rs +++ b/polkadot/node/core/pvf/common/src/worker/mod.rs @@ -206,24 +206,37 @@ pub fn pipe2_cloexec() -> io::Result<(libc::c_int, libc::c_int)> { Ok((fds[0], fds[1])) } +/// A wrapper around a file descriptor used to encapsulate and restrict +/// functionality for pipe operations. pub struct PipeFd { file: File, } impl PipeFd { - pub fn new(fd: i32) -> Self { - // SAFETY: pipe_writer is an open and owned file descriptor at this point. + /// Creates a new `PipeFd` instance from a raw file descriptor. + /// + /// # Safety + /// + /// The fd passed in must be an owned file descriptor; in particular, it must be open. + pub unsafe fn new(fd: i32) -> Self { let file = unsafe { File::from_raw_fd(fd) }; PipeFd { file } } + /// Returns the raw file descriptor associated with this `PipeFd` pub fn as_raw_fd(&self) -> i32 { self.file.as_raw_fd() } +} - pub fn read_to_end(&mut self, buf: &mut Vec) -> io::Result { +impl Read for PipeFd { + fn read_to_end(&mut self, buf: &mut Vec) -> io::Result { self.file.read_to_end(buf) } + + fn read(&mut self, buf: &mut [u8]) -> io::Result { + self.file.read(buf) + } } impl Write for PipeFd { diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index 359a6738a987..a360ec754baa 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -337,7 +337,8 @@ fn handle_child_process( params: Vec, execution_timeout: Duration, ) -> ! { - let mut pipe_write = PipeFd::new(pipe_write_fd); + // SAFETY: pipe_writer is an open and owned file descriptor at this point. + let mut pipe_write = unsafe { PipeFd::new(pipe_write_fd) }; // Drop the read end so we don't have too many FDs open. if let Err(errno) = nix::unistd::close(pipe_read_fd) { @@ -441,7 +442,8 @@ fn handle_parent_process( return Ok(internal_error_from_errno("closing pipe write fd", errno)); }; - let mut pipe_read = PipeFd::new(pipe_read_fd); + // SAFETY: pipe_writer is an open and owned file descriptor at this point. + let mut pipe_read = unsafe { PipeFd::new(pipe_read_fd) }; // Read from the child. Don't decode unless the process exited normally, which we check later. let mut received_data = Vec::new(); diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index 52b7d48f28ad..d9b714698429 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -396,7 +396,8 @@ fn handle_child_process( prepare_job_kind: PrepareJobKind, executor_params: Arc, ) -> ! { - let mut pipe_write = PipeFd::new(pipe_write_fd); + // SAFETY: pipe_writer is an open and owned file descriptor at this point. + let mut pipe_write = unsafe { PipeFd::new(pipe_write_fd) }; // Drop the read end so we don't have too many FDs open. if let Err(errno) = nix::unistd::close(pipe_read_fd) { From f9e61cc7c1da05773e86457406c12346995b63b0 Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Wed, 20 Dec 2023 09:02:36 -0300 Subject: [PATCH 19/50] minor fixes --- polkadot/node/core/pvf/common/src/worker/mod.rs | 2 +- polkadot/node/core/pvf/execute-worker/Cargo.toml | 2 +- polkadot/node/core/pvf/execute-worker/src/lib.rs | 15 +++++++++------ 3 files changed, 11 insertions(+), 8 deletions(-) diff --git a/polkadot/node/core/pvf/common/src/worker/mod.rs b/polkadot/node/core/pvf/common/src/worker/mod.rs index f2d4375962b2..713722c0fa66 100644 --- a/polkadot/node/core/pvf/common/src/worker/mod.rs +++ b/polkadot/node/core/pvf/common/src/worker/mod.rs @@ -219,7 +219,7 @@ impl PipeFd { /// /// The fd passed in must be an owned file descriptor; in particular, it must be open. pub unsafe fn new(fd: i32) -> Self { - let file = unsafe { File::from_raw_fd(fd) }; + let file = File::from_raw_fd(fd); PipeFd { file } } diff --git a/polkadot/node/core/pvf/execute-worker/Cargo.toml b/polkadot/node/core/pvf/execute-worker/Cargo.toml index 297e26f6c85a..74a5aff401b7 100644 --- a/polkadot/node/core/pvf/execute-worker/Cargo.toml +++ b/polkadot/node/core/pvf/execute-worker/Cargo.toml @@ -13,7 +13,7 @@ workspace = true cpu-time = "1.0.0" gum = { package = "tracing-gum", path = "../../../gum" } cfg-if = "1.0" -nix = { version = "0.27.1", features = ["process", "resource"] } +nix = { version = "0.27.1", features = ["process", "resource", "sched"] } libc = "0.2.139" parity-scale-codec = { version = "3.6.1", default-features = false, features = ["derive"] } diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index a360ec754baa..739bed59864b 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -23,13 +23,17 @@ pub use polkadot_node_core_pvf_common::{executor_interface::execute_artifact, wo const LOG_TARGET: &str = "parachain::pvf-execute-worker"; use cpu_time::ProcessTime; +#[cfg(target_os = "linux")] +use nix::sched::CloneFlags; +#[cfg(not(target_os = "linux"))] +use nix::unistd::ForkResult; use nix::{ errno::Errno, sys::{ resource::{Usage, UsageWho}, wait::WaitStatus, }, - unistd::{ForkResult, Pid}, + unistd::Pid, }; use parity_scale_codec::{Decode, Encode}; use polkadot_node_core_pvf_common::{ @@ -185,8 +189,7 @@ pub fn worker_entrypoint( cfg_if::cfg_if! { if #[cfg(target_os = "linux")] { - let stack_size = 2 * 1024 * 1024; // 2MiB - let mut stack: Vec = vec![0u8; stack_size]; + let mut stack: Vec = vec![0u8; EXECUTE_THREAD_STACK_SIZE]; // SIGCHLD flag is used to inform clone that the parent process is // expecting a child termination signal, without this flag `waitpid` function // return `ECHILD` error. @@ -219,7 +222,7 @@ pub fn worker_entrypoint( None ) } { - Err(errno) => Err(error_from_errno("clone", errno)), + Err(errno) => internal_error_from_errno("clone", errno), Ok(child) => { handle_parent_process( pipe_reader, @@ -228,7 +231,7 @@ pub fn worker_entrypoint( worker_pid, usage_before, execution_timeout, - ) + )? }, }; @@ -442,7 +445,7 @@ fn handle_parent_process( return Ok(internal_error_from_errno("closing pipe write fd", errno)); }; - // SAFETY: pipe_writer is an open and owned file descriptor at this point. + // SAFETY: pipe_read is an open and owned file descriptor at this point. let mut pipe_read = unsafe { PipeFd::new(pipe_read_fd) }; // Read from the child. Don't decode unless the process exited normally, which we check later. From fcbe008b73c4ed681a064b2a5cc827a2e370b15d Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Fri, 22 Dec 2023 12:01:13 +0100 Subject: [PATCH 20/50] Fix some errors --- polkadot/node/core/pvf/common/src/execute.rs | 3 +++ polkadot/node/core/pvf/execute-worker/src/lib.rs | 13 +++++++------ polkadot/node/core/pvf/prepare-worker/src/lib.rs | 2 +- 3 files changed, 11 insertions(+), 7 deletions(-) diff --git a/polkadot/node/core/pvf/common/src/execute.rs b/polkadot/node/core/pvf/common/src/execute.rs index aa1c1c539682..6b3becf524d7 100644 --- a/polkadot/node/core/pvf/common/src/execute.rs +++ b/polkadot/node/core/pvf/common/src/execute.rs @@ -92,6 +92,9 @@ pub enum JobError { TimedOut, #[error("An unexpected panic has occurred in the execution job: {0}")] Panic(String), + /// Some error occurred when interfacing with the kernel. + #[error("Error interfacing with the kernel: {0}")] + Kernel(String), #[error("Could not spawn the requested thread: {0}")] CouldNotSpawnThread(String), #[error("An error occurred in the CPU time monitor thread: {0}")] diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index 739bed59864b..fcdc476d2ff4 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -345,7 +345,7 @@ fn handle_child_process( // Drop the read end so we don't have too many FDs open. if let Err(errno) = nix::unistd::close(pipe_read_fd) { - send_child_response(&mut pipe_write, Err(JobError::Panic("closing pipe".to_string()))); + send_child_response(&mut pipe_write, job_error_from_errno("closing pipe", errno)); } // Dropping the stream closes the underlying socket. We want to make sure @@ -353,10 +353,7 @@ fn handle_child_process( // outside world. The only IPC it should be able to do is sending its // response over the pipe. if let Err(errno) = nix::unistd::close(stream_fd) { - send_child_response( - &mut pipe_write, - Err(JobError::Panic(format!("error closing stream {}", errno))), - ); + send_child_response(&mut pipe_write, job_error_from_errno("closing stream", errno)); } gum::debug!( @@ -445,7 +442,7 @@ fn handle_parent_process( return Ok(internal_error_from_errno("closing pipe write fd", errno)); }; - // SAFETY: pipe_read is an open and owned file descriptor at this point. + // SAFETY: pipe_read_fd is an open and owned file descriptor at this point. let mut pipe_read = unsafe { PipeFd::new(pipe_read_fd) }; // Read from the child. Don't decode unless the process exited normally, which we check later. @@ -599,3 +596,7 @@ fn internal_error_from_errno(context: &'static str, errno: Errno) -> WorkerRespo io::Error::last_os_error() ))) } + +fn job_error_from_errno(context: &'static str, errno: Errno) -> JobResult { + Err(JobError::Kernel(format!("{}: {}: {}", context, errno, io::Error::last_os_error()))) +} diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index d9b714698429..7137af7b0880 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -605,7 +605,7 @@ fn handle_parent_process( return Err(error_from_errno("closing pipe write fd", errno)); }; - //SAFETY: pipe_read is an open and owned file descriptor at this point. + // SAFETY: pipe_read_fd is an open and owned file descriptor at this point. let mut pipe_read = unsafe { PipeFd::new(pipe_read_fd) }; // Read from the child. Don't decode unless the process exited normally, which we check later. From b90a9b8afef4fb9ede1e58568511e1a1fc01b225 Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Fri, 22 Dec 2023 14:37:07 +0100 Subject: [PATCH 21/50] Fix some more stuff - Move clone flags to common place - Fix some borrowch issues - Fix some duplicated code --- Cargo.lock | 1 + polkadot/node/core/pvf/common/Cargo.toml | 1 + .../pvf/common/src/worker/security/clone.rs | 31 ++++++++++ .../pvf/common/src/worker/security/mod.rs | 6 +- .../node/core/pvf/execute-worker/src/lib.rs | 59 ++++++++----------- .../node/core/pvf/prepare-worker/src/lib.rs | 15 ++--- 6 files changed, 64 insertions(+), 49 deletions(-) create mode 100644 polkadot/node/core/pvf/common/src/worker/security/clone.rs diff --git a/Cargo.lock b/Cargo.lock index b06b51a8a6ce..fc73a683b4ed 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -12494,6 +12494,7 @@ dependencies = [ "futures", "landlock", "libc", + "nix 0.27.1", "parity-scale-codec", "polkadot-parachain-primitives", "polkadot-primitives", diff --git a/polkadot/node/core/pvf/common/Cargo.toml b/polkadot/node/core/pvf/common/Cargo.toml index c5c09300e8af..3c7052725439 100644 --- a/polkadot/node/core/pvf/common/Cargo.toml +++ b/polkadot/node/core/pvf/common/Cargo.toml @@ -33,6 +33,7 @@ sp-tracing = { path = "../../../../../substrate/primitives/tracing" } [target.'cfg(target_os = "linux")'.dependencies] landlock = "0.3.0" +nix = { version = "0.27.1", features = ["sched"] } seccompiler = "0.4.0" [dev-dependencies] diff --git a/polkadot/node/core/pvf/common/src/worker/security/clone.rs b/polkadot/node/core/pvf/common/src/worker/security/clone.rs new file mode 100644 index 000000000000..aaa6ac3c58eb --- /dev/null +++ b/polkadot/node/core/pvf/common/src/worker/security/clone.rs @@ -0,0 +1,31 @@ +// Copyright (C) Parity Technologies (UK) Ltd. +// This file is part of Polkadot. + +// Polkadot is free software: you can redistribute it and/or modify +// it under the terms of the GNU General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. + +// Polkadot is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU General Public License for more details. + +// You should have received a copy of the GNU General Public License +// along with Polkadot. If not, see . + +//! Functionality for securing the job processes spawned by the workers using `clone`. If +//! unsupported, falls back to `fork`. + +use nix::sched::CloneFlags; + +/// Returns all the sandbox-related flags for `clone`. +pub fn clone_sandbox_flags() -> CloneFlags { + CloneFlags::CLONE_NEWCGROUP | + CloneFlags::CLONE_NEWIPC | + CloneFlags::CLONE_NEWNET | + CloneFlags::CLONE_NEWNS | + CloneFlags::CLONE_NEWPID | + CloneFlags::CLONE_NEWUSER | + CloneFlags::CLONE_NEWUTS +} diff --git a/polkadot/node/core/pvf/common/src/worker/security/mod.rs b/polkadot/node/core/pvf/common/src/worker/security/mod.rs index ff4c712f6bdc..72d47235d47a 100644 --- a/polkadot/node/core/pvf/common/src/worker/security/mod.rs +++ b/polkadot/node/core/pvf/common/src/worker/security/mod.rs @@ -27,15 +27,17 @@ //! - Restrict networking by blocking socket creation and io_uring. //! - Remove env vars -use crate::{worker::WorkerInfo, LOG_TARGET}; - #[cfg(target_os = "linux")] pub mod change_root; #[cfg(target_os = "linux")] +pub mod clone; +#[cfg(target_os = "linux")] pub mod landlock; #[cfg(all(target_os = "linux", target_arch = "x86_64"))] pub mod seccomp; +use crate::{worker::WorkerInfo, LOG_TARGET}; + /// Require env vars to have been removed when spawning the process, to prevent malicious code from /// accessing them. pub fn check_env_vars_were_cleared(worker_info: &WorkerInfo) -> bool { diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index fcdc476d2ff4..99dd140def98 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -41,7 +41,7 @@ use polkadot_node_core_pvf_common::{ execute::{Handshake, JobError, JobResponse, JobResult, WorkerResponse}, framed_recv_blocking, framed_send_blocking, worker::{ - cpu_time_monitor_loop, pipe2_cloexec, run_worker, stringify_panic_payload, + cpu_time_monitor_loop, pipe2_cloexec, run_worker, security, stringify_panic_payload, thread::{self, WaitOutcome}, PipeFd, WorkerKind, }, @@ -187,20 +187,18 @@ pub fn worker_entrypoint( }; let stream_fd = stream.as_raw_fd(); + let compiled_artifact_blob = Arc::new(compiled_artifact_blob); + let executor_params = Arc::new(executor_params.clone()); + let params = Arc::new(params); + cfg_if::cfg_if! { if #[cfg(target_os = "linux")] { let mut stack: Vec = vec![0u8; EXECUTE_THREAD_STACK_SIZE]; // SIGCHLD flag is used to inform clone that the parent process is // expecting a child termination signal, without this flag `waitpid` function // return `ECHILD` error. - let flags = CloneFlags::CLONE_NEWCGROUP - | CloneFlags::CLONE_NEWIPC - | CloneFlags::CLONE_NEWNET - | CloneFlags::CLONE_NEWNS - | CloneFlags::CLONE_NEWPID - | CloneFlags::CLONE_NEWUSER - | CloneFlags::CLONE_NEWUTS - | CloneFlags::from_bits_retain(libc::SIGCHLD); + let flags = + security::clone::clone_sandbox_flags() | CloneFlags::from_bits_retain(libc::SIGCHLD); // SAFETY: new process is spawned within a single threaded process. This invariant // is enforced by tests. Stack size being specified to ensure child doesn't overflow @@ -211,9 +209,9 @@ pub fn worker_entrypoint( pipe_writer, pipe_reader, stream_fd, - compiled_artifact_blob, - executor_params, - params, + Arc::clone(&compiled_artifact_blob), + Arc::clone(&executor_params), + Arc::clone(¶ms), execution_timeout, ) }), @@ -234,15 +232,6 @@ pub fn worker_entrypoint( )? }, }; - - - gum::trace!( - target: LOG_TARGET, - %worker_pid, - "worker: sending result to host: {:?}", - result - ); - send_response(&mut stream, result)?; } else { // SAFETY: new process is spawned within a single threaded process. This invariant // is enforced by tests. @@ -270,17 +259,16 @@ pub fn worker_entrypoint( )? }, }; - - - gum::trace!( - target: LOG_TARGET, - %worker_pid, - "worker: sending result to host: {:?}", - result - ); - send_response(&mut stream, result)?; } - } + }; + + gum::trace!( + target: LOG_TARGET, + %worker_pid, + "worker: sending result to host: {:?}", + result + ); + send_response(&mut stream, result)?; } }, ); @@ -335,9 +323,9 @@ fn handle_child_process( pipe_write_fd: i32, pipe_read_fd: i32, stream_fd: i32, - compiled_artifact_blob: Vec, - executor_params: ExecutorParams, - params: Vec, + compiled_artifact_blob: Arc>, + executor_params: Arc, + params: Arc>, execution_timeout: Duration, ) -> ! { // SAFETY: pipe_writer is an open and owned file descriptor at this point. @@ -378,10 +366,9 @@ fn handle_child_process( send_child_response(&mut pipe_write, Err(JobError::CouldNotSpawnThread(err.to_string()))) }); - let executor_params_2 = executor_params.clone(); let execute_thread = thread::spawn_worker_thread_with_stack_size( "execute thread", - move || validate_using_artifact(&compiled_artifact_blob, &executor_params_2, ¶ms), + move || validate_using_artifact(&compiled_artifact_blob, &executor_params, ¶ms), Arc::clone(&condvar), WaitOutcome::Finished, EXECUTE_THREAD_STACK_SIZE, diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index 7137af7b0880..89db55a4dbcf 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -20,7 +20,7 @@ mod memory_stats; use polkadot_node_core_pvf_common::{ executor_interface::{prepare, prevalidate}, - worker::{pipe2_cloexec, PipeFd}, + worker::{pipe2_cloexec, security, PipeFd}, }; // NOTE: Initializing logging in e.g. tests will not have an effect in the workers, as they are @@ -243,14 +243,8 @@ pub fn worker_entrypoint( // SIGCHLD flag is used to inform clone that the parent process is // expecting a child termination signal, without this flag `waitpid` function // return `ECHILD` error. - let flags = CloneFlags::CLONE_NEWCGROUP - | CloneFlags::CLONE_NEWIPC - | CloneFlags::CLONE_NEWNET - | CloneFlags::CLONE_NEWNS - | CloneFlags::CLONE_NEWPID - | CloneFlags::CLONE_NEWUSER - | CloneFlags::CLONE_NEWUTS - | CloneFlags::from_bits_retain(libc::SIGCHLD); + let flags = + security::clone::clone_sandbox_flags() | CloneFlags::from_bits_retain(libc::SIGCHLD); // SAFETY: new process is spawned within a single threaded process. This invariant // is enforced by tests. Stack size being specified to ensure child doesn't overflow @@ -285,7 +279,6 @@ pub fn worker_entrypoint( ) }, }; - send_response(&mut stream, result)?; } else { // SAFETY: new process is spawned within a single threaded process. This invariant // is enforced by tests. @@ -316,7 +309,7 @@ pub fn worker_entrypoint( }, }; } - } + }; gum::trace!( target: LOG_TARGET, From adc1213c75e308787a1f19b481b360a29bcd721c Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Fri, 22 Dec 2023 16:26:21 +0100 Subject: [PATCH 22/50] Some fixes --- polkadot/node/core/pvf/common/src/error.rs | 2 +- .../node/core/pvf/common/src/worker/security/clone.rs | 3 ++- polkadot/node/core/pvf/execute-worker/src/lib.rs | 4 +++- polkadot/node/core/pvf/prepare-worker/src/lib.rs | 8 +++++--- 4 files changed, 11 insertions(+), 6 deletions(-) diff --git a/polkadot/node/core/pvf/common/src/error.rs b/polkadot/node/core/pvf/common/src/error.rs index 7db7f9a59451..d62bbb6fbd86 100644 --- a/polkadot/node/core/pvf/common/src/error.rs +++ b/polkadot/node/core/pvf/common/src/error.rs @@ -77,8 +77,8 @@ pub enum PrepareError { #[codec(index = 9)] ClearWorkerDir(String), /// The preparation job process died, due to OOM, a seccomp violation, or some other factor. - JobDied { err: String, job_pid: i32 }, #[codec(index = 10)] + JobDied { err: String, job_pid: i32 }, /// Some error occurred when interfacing with the kernel. #[codec(index = 11)] Kernel(String), diff --git a/polkadot/node/core/pvf/common/src/worker/security/clone.rs b/polkadot/node/core/pvf/common/src/worker/security/clone.rs index aaa6ac3c58eb..52a513c3e8c7 100644 --- a/polkadot/node/core/pvf/common/src/worker/security/clone.rs +++ b/polkadot/node/core/pvf/common/src/worker/security/clone.rs @@ -21,11 +21,12 @@ use nix::sched::CloneFlags; /// Returns all the sandbox-related flags for `clone`. pub fn clone_sandbox_flags() -> CloneFlags { + // NOTE: CLONE_NEWUSER does not work when cloning job processes, but in Secure Validator Mode it + // is already set by the worker. CloneFlags::CLONE_NEWCGROUP | CloneFlags::CLONE_NEWIPC | CloneFlags::CLONE_NEWNET | CloneFlags::CLONE_NEWNS | CloneFlags::CLONE_NEWPID | - CloneFlags::CLONE_NEWUSER | CloneFlags::CLONE_NEWUTS } diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index 99dd140def98..be5c79178c3c 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -41,7 +41,7 @@ use polkadot_node_core_pvf_common::{ execute::{Handshake, JobError, JobResponse, JobResult, WorkerResponse}, framed_recv_blocking, framed_send_blocking, worker::{ - cpu_time_monitor_loop, pipe2_cloexec, run_worker, security, stringify_panic_payload, + cpu_time_monitor_loop, pipe2_cloexec, run_worker, stringify_panic_payload, thread::{self, WaitOutcome}, PipeFd, WorkerKind, }, @@ -193,6 +193,8 @@ pub fn worker_entrypoint( cfg_if::cfg_if! { if #[cfg(target_os = "linux")] { + use polkadot_node_core_pvf_common::worker::security; + let mut stack: Vec = vec![0u8; EXECUTE_THREAD_STACK_SIZE]; // SIGCHLD flag is used to inform clone that the parent process is // expecting a child termination signal, without this flag `waitpid` function diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index 89db55a4dbcf..1d61998c3401 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -20,7 +20,7 @@ mod memory_stats; use polkadot_node_core_pvf_common::{ executor_interface::{prepare, prevalidate}, - worker::{pipe2_cloexec, security, PipeFd}, + worker::{pipe2_cloexec, PipeFd}, }; // NOTE: Initializing logging in e.g. tests will not have an effect in the workers, as they are @@ -61,10 +61,10 @@ use polkadot_node_core_pvf_common::{ }; use polkadot_primitives::ExecutorParams; use std::{ - fs::{self, File}, + fs, io::{self, Read}, os::{ - fd::{AsRawFd, FromRawFd, RawFd}, + fd::{AsRawFd, RawFd}, unix::net::UnixStream, }, path::PathBuf, @@ -238,6 +238,8 @@ pub fn worker_entrypoint( cfg_if::cfg_if! { if #[cfg(target_os = "linux")] { + use polkadot_node_core_pvf_common::worker::security; + let stack_size = 2 * 1024 * 1024; // 2MiB let mut stack: Vec = vec![0u8; stack_size]; // SIGCHLD flag is used to inform clone that the parent process is From 96f44512567899e29e5d6ae7531a9eee08883922 Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Tue, 26 Dec 2023 16:59:16 +0100 Subject: [PATCH 23/50] Add check for clone capability; many fixes --- polkadot/node/core/pvf/common/src/lib.rs | 2 + .../node/core/pvf/common/src/worker/mod.rs | 35 ++++-- .../common/src/worker/security/change_root.rs | 10 +- .../pvf/common/src/worker/security/clone.rs | 68 +++++++++++- .../common/src/worker/security/landlock.rs | 8 +- .../pvf/common/src/worker/security/seccomp.rs | 4 +- .../node/core/pvf/execute-worker/src/lib.rs | 53 +++++---- .../node/core/pvf/prepare-worker/src/lib.rs | 56 ++++------ polkadot/node/core/pvf/src/host.rs | 2 +- polkadot/node/core/pvf/src/security.rs | 101 ++++++++++++++++-- polkadot/node/core/pvf/tests/it/main.rs | 3 + 11 files changed, 239 insertions(+), 103 deletions(-) diff --git a/polkadot/node/core/pvf/common/src/lib.rs b/polkadot/node/core/pvf/common/src/lib.rs index abebd06f71a4..90aebc1d6ef1 100644 --- a/polkadot/node/core/pvf/common/src/lib.rs +++ b/polkadot/node/core/pvf/common/src/lib.rs @@ -59,6 +59,8 @@ pub struct SecurityStatus { pub can_enable_seccomp: bool, /// Whether we are able to unshare the user namespace and change the filesystem root. pub can_unshare_user_namespace_and_change_root: bool, + /// Whether we are able to call `clone` with all sandboxing flags. + pub can_do_secure_clone: bool, } /// A handshake with information for the worker. diff --git a/polkadot/node/core/pvf/common/src/worker/mod.rs b/polkadot/node/core/pvf/common/src/worker/mod.rs index 713722c0fa66..dd2086bac620 100644 --- a/polkadot/node/core/pvf/common/src/worker/mod.rs +++ b/polkadot/node/core/pvf/common/src/worker/mod.rs @@ -83,7 +83,7 @@ macro_rules! decl_worker_main { "--check-can-enable-landlock" => { #[cfg(target_os = "linux")] - let status = if let Err(err) = security::landlock::check_is_fully_enabled() { + let status = if let Err(err) = security::landlock::check_can_fully_enable() { // Write the error to stderr, log it on the host-side. eprintln!("{}", err); -1 @@ -96,7 +96,7 @@ macro_rules! decl_worker_main { }, "--check-can-enable-seccomp" => { #[cfg(all(target_os = "linux", target_arch = "x86_64"))] - let status = if let Err(err) = security::seccomp::check_is_fully_enabled() { + let status = if let Err(err) = security::seccomp::check_can_fully_enable() { // Write the error to stderr, log it on the host-side. eprintln!("{}", err); -1 @@ -112,7 +112,7 @@ macro_rules! decl_worker_main { let cache_path_tempdir = std::path::Path::new(&args[2]); #[cfg(target_os = "linux")] let status = if let Err(err) = - security::change_root::check_is_fully_enabled(&cache_path_tempdir) + security::change_root::check_can_fully_enable(&cache_path_tempdir) { // Write the error to stderr, log it on the host-side. eprintln!("{}", err); @@ -124,6 +124,21 @@ macro_rules! decl_worker_main { let status = -1; std::process::exit(status) }, + "--check-can-do-secure-clone" => { + #[cfg(target_os = "linux")] + // SAFETY: new process is spawned within a single threaded process. This + // invariant is enforced by tests. + let status = if let Err(err) = unsafe { security::clone::check_can_fully_clone() } { + // Write the error to stderr, log it on the host-side. + eprintln!("{}", err); + -1 + } else { + 0 + }; + #[cfg(not(target_os = "linux"))] + let status = -1; + std::process::exit(status) + }, "test-sleep" => { std::thread::sleep(std::time::Duration::from_secs(5)); @@ -274,14 +289,12 @@ impl fmt::Display for WorkerKind { } } -// Some fields are only used for logging, and dead-code analysis ignores Debug. -#[allow(dead_code)] #[derive(Debug)] pub struct WorkerInfo { - pid: u32, - kind: WorkerKind, - version: Option, - worker_dir_path: PathBuf, + pub pid: u32, + pub kind: WorkerKind, + pub version: Option, + pub worker_dir_path: PathBuf, } // NOTE: The worker version must be passed in so that we accurately get the version of the worker, @@ -300,7 +313,7 @@ pub fn run_worker( worker_version: Option<&str>, mut event_loop: F, ) where - F: FnMut(UnixStream, PathBuf) -> io::Result, + F: FnMut(UnixStream, &WorkerInfo) -> io::Result, { #[cfg_attr(not(target_os = "linux"), allow(unused_mut))] let mut worker_info = WorkerInfo { @@ -435,7 +448,7 @@ pub fn run_worker( } // Run the main worker loop. - let err = event_loop(stream, worker_info.worker_dir_path.clone()) + let err = event_loop(stream, &worker_info) // It's never `Ok` because it's `Ok(Never)`. .unwrap_err(); diff --git a/polkadot/node/core/pvf/common/src/worker/security/change_root.rs b/polkadot/node/core/pvf/common/src/worker/security/change_root.rs index 375cc8ff6f28..9ec66906819f 100644 --- a/polkadot/node/core/pvf/common/src/worker/security/change_root.rs +++ b/polkadot/node/core/pvf/common/src/worker/security/change_root.rs @@ -54,8 +54,7 @@ pub fn enable_for_worker(worker_info: &WorkerInfo) -> Result<()> { /// /// NOTE: This should not be called in a multi-threaded context. `unshare(2)`: /// "CLONE_NEWUSER requires that the calling process is not threaded." -#[cfg(target_os = "linux")] -pub fn check_is_fully_enabled(tempdir: &Path) -> Result<()> { +pub fn check_can_fully_enable(tempdir: &Path) -> Result<()> { let worker_dir_path = tempdir.to_owned(); try_restrict(&WorkerInfo { pid: std::process::id(), @@ -69,7 +68,6 @@ pub fn check_is_fully_enabled(tempdir: &Path) -> Result<()> { /// /// NOTE: This should not be called in a multi-threaded context. `unshare(2)`: /// "CLONE_NEWUSER requires that the calling process is not threaded." -#[cfg(target_os = "linux")] fn try_restrict(worker_info: &WorkerInfo) -> Result<()> { // TODO: Remove this once this is stable: https://github.com/rust-lang/rust/issues/105723 macro_rules! cstr_ptr { @@ -78,12 +76,6 @@ fn try_restrict(worker_info: &WorkerInfo) -> Result<()> { }; } - gum::trace!( - target: LOG_TARGET, - ?worker_info, - "unsharing the user namespace and calling pivot_root", - ); - let worker_dir_path_c = CString::new(worker_info.worker_dir_path.as_os_str().as_bytes()) .expect("on unix; the path will never contain 0 bytes; qed"); diff --git a/polkadot/node/core/pvf/common/src/worker/security/clone.rs b/polkadot/node/core/pvf/common/src/worker/security/clone.rs index 52a513c3e8c7..003b07ab4464 100644 --- a/polkadot/node/core/pvf/common/src/worker/security/clone.rs +++ b/polkadot/node/core/pvf/common/src/worker/security/clone.rs @@ -17,16 +17,76 @@ //! Functionality for securing the job processes spawned by the workers using `clone`. If //! unsupported, falls back to `fork`. -use nix::sched::CloneFlags; +use crate::{worker::WorkerInfo, LOG_TARGET}; +use nix::{ + errno::Errno, + sched::{CloneCb, CloneFlags}, + unistd::Pid, +}; -/// Returns all the sandbox-related flags for `clone`. -pub fn clone_sandbox_flags() -> CloneFlags { +#[derive(thiserror::Error, Debug)] +pub enum Error { + #[error("could not clone, errno: {0}")] + Clone(Errno), +} + +pub type Result = std::result::Result; + +/// Try to run clone(2) on the current worker. +/// +/// SAFETY: new process should be either spawned within a single threaded process, or use only +/// async-signal-safe functions. +pub unsafe fn clone_on_worker( + worker_info: &WorkerInfo, + cb: CloneCb, + stack_size: usize, +) -> Result { + let flags = clone_flags(); + + gum::trace!( + target: LOG_TARGET, + ?worker_info, + %stack_size, + "calling clone with flags: {:?}", + flags + ); + + try_clone(cb, stack_size, flags) +} + +/// Runs a check for clone(2) with all sandboxing flags and returns an error indicating whether it +/// can be fully enabled on the current Linux environment. +/// +/// SAFETY: new process should be either spawned within a single threaded process, or use only +/// async-signal-safe functions. +pub unsafe fn check_can_fully_clone() -> Result<()> { + let stack_size = 2 * 1024 * 1024; // Use same as prepare worker for this check. + try_clone(Box::new(|| 0), stack_size, clone_flags()).map(|_pid| ()) +} + +/// Runs clone(2) with all sandboxing flags. +/// +/// SAFETY: new process should be either spawned within a single threaded process, or use only +/// async-signal-safe functions. +unsafe fn try_clone(cb: CloneCb, stack_size: usize, flags: CloneFlags) -> Result { + let mut stack: Vec = vec![0u8; stack_size]; + + nix::sched::clone(cb, stack.as_mut_slice(), flags, None).map_err(|errno| Error::Clone(errno)) +} + +/// Returns flags for `clone(2)`, including all the sandbox-related ones. +fn clone_flags() -> CloneFlags { // NOTE: CLONE_NEWUSER does not work when cloning job processes, but in Secure Validator Mode it // is already set by the worker. + // + // SIGCHLD flag is used to inform clone that the parent process is + // expecting a child termination signal, without this flag `waitpid` function + // return `ECHILD` error. CloneFlags::CLONE_NEWCGROUP | CloneFlags::CLONE_NEWIPC | CloneFlags::CLONE_NEWNET | CloneFlags::CLONE_NEWNS | CloneFlags::CLONE_NEWPID | - CloneFlags::CLONE_NEWUTS + CloneFlags::CLONE_NEWUTS | + CloneFlags::from_bits_retain(libc::SIGCHLD) } diff --git a/polkadot/node/core/pvf/common/src/worker/security/landlock.rs b/polkadot/node/core/pvf/common/src/worker/security/landlock.rs index 211d12c2e443..10d00a0e2c66 100644 --- a/polkadot/node/core/pvf/common/src/worker/security/landlock.rs +++ b/polkadot/node/core/pvf/common/src/worker/security/landlock.rs @@ -112,7 +112,7 @@ pub fn enable_for_worker(worker_info: &WorkerInfo) -> Result<()> { // TODO: /// Runs a check for landlock in its own thread, and returns an error indicating whether the given /// landlock ABI is fully enabled on the current Linux environment. -pub fn check_is_fully_enabled() -> Result<()> { +pub fn check_can_fully_enable() -> Result<()> { match std::thread::spawn(|| try_restrict(std::iter::empty::<(PathBuf, AccessFs)>())).join() { Ok(Ok(())) => Ok(()), Ok(Err(err)) => Err(err), @@ -165,7 +165,7 @@ mod tests { #[test] fn restricted_thread_cannot_read_file() { // TODO: This would be nice: . - if check_is_fully_enabled().is_err() { + if check_can_fully_enable().is_err() { return } @@ -230,7 +230,7 @@ mod tests { #[test] fn restricted_thread_cannot_write_file() { // TODO: This would be nice: . - if check_is_fully_enabled().is_err() { + if check_can_fully_enable().is_err() { return } @@ -289,7 +289,7 @@ mod tests { #[test] fn restricted_thread_can_truncate_file() { // TODO: This would be nice: . - if check_is_fully_enabled().is_err() { + if check_can_fully_enable().is_err() { return } diff --git a/polkadot/node/core/pvf/common/src/worker/security/seccomp.rs b/polkadot/node/core/pvf/common/src/worker/security/seccomp.rs index 4f270f75b345..f6100d236c8b 100644 --- a/polkadot/node/core/pvf/common/src/worker/security/seccomp.rs +++ b/polkadot/node/core/pvf/common/src/worker/security/seccomp.rs @@ -110,7 +110,7 @@ pub fn enable_for_worker(worker_info: &WorkerInfo) -> Result<()> { /// Runs a check for seccomp in its own thread, and returns an error indicating whether seccomp with /// our rules is fully enabled on the current Linux environment. -pub fn check_is_fully_enabled() -> Result<()> { +pub fn check_can_fully_enable() -> Result<()> { match std::thread::spawn(|| try_restrict()).join() { Ok(Ok(())) => Ok(()), Ok(Err(err)) => Err(err), @@ -161,7 +161,7 @@ mod tests { #[test] fn sandboxed_thread_cannot_use_sockets() { // TODO: This would be nice: . - if check_is_fully_enabled().is_err() { + if check_can_fully_enable().is_err() { return } diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index be5c79178c3c..6d38517c02f7 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -16,7 +16,9 @@ //! Contains the logic for executing PVFs. Used by the polkadot-execute-worker binary. -pub use polkadot_node_core_pvf_common::{executor_interface::execute_artifact, worker_dir}; +pub use polkadot_node_core_pvf_common::{ + executor_interface::execute_artifact, worker::WorkerInfo, worker_dir, +}; // NOTE: Initializing logging in e.g. tests will not have an effect in the workers, as they are // separate spawned processes. Run with e.g. `RUST_LOG=parachain::pvf-execute-worker=trace`. @@ -148,9 +150,8 @@ pub fn worker_entrypoint( worker_dir_path, node_version, worker_version, - |mut stream, worker_dir_path| { - let worker_pid = process::id(); - let artifact_path = worker_dir::execute_artifact(&worker_dir_path); + |mut stream, worker_info| { + let artifact_path = worker_dir::execute_artifact(&worker_info.worker_dir_path); let Handshake { executor_params } = recv_execute_handshake(&mut stream)?; @@ -158,7 +159,7 @@ pub fn worker_entrypoint( let (params, execution_timeout) = recv_request(&mut stream)?; gum::debug!( target: LOG_TARGET, - %worker_pid, + ?worker_info, "worker: validating artifact {}", artifact_path.display(), ); @@ -195,17 +196,13 @@ pub fn worker_entrypoint( if #[cfg(target_os = "linux")] { use polkadot_node_core_pvf_common::worker::security; - let mut stack: Vec = vec![0u8; EXECUTE_THREAD_STACK_SIZE]; - // SIGCHLD flag is used to inform clone that the parent process is - // expecting a child termination signal, without this flag `waitpid` function - // return `ECHILD` error. - let flags = - security::clone::clone_sandbox_flags() | CloneFlags::from_bits_retain(libc::SIGCHLD); + let stack_size = EXECUTE_THREAD_STACK_SIZE; // SAFETY: new process is spawned within a single threaded process. This invariant // is enforced by tests. Stack size being specified to ensure child doesn't overflow let result = match unsafe { - nix::sched::clone( + security::clone::clone_on_worker( + worker_info, Box::new(|| { handle_child_process( pipe_writer, @@ -217,22 +214,20 @@ pub fn worker_entrypoint( execution_timeout, ) }), - stack.as_mut_slice(), - flags, - None + stack_size, ) } { - Err(errno) => internal_error_from_errno("clone", errno), Ok(child) => { handle_parent_process( pipe_reader, pipe_writer, + worker_info, child, - worker_pid, usage_before, execution_timeout, )? }, + Err(security::clone::Error::Clone(errno)) => internal_error_from_errno("clone", errno), }; } else { // SAFETY: new process is spawned within a single threaded process. This invariant @@ -254,8 +249,8 @@ pub fn worker_entrypoint( handle_parent_process( pipe_reader, pipe_writer, + worker_info, child, - worker_pid, usage_before, execution_timeout, )? @@ -266,7 +261,7 @@ pub fn worker_entrypoint( gum::trace!( target: LOG_TARGET, - %worker_pid, + ?worker_info, "worker: sending result to host: {:?}", result ); @@ -404,15 +399,19 @@ fn handle_child_process( /// Waits for child process to finish and handle child response from pipe. /// -/// # Arguments +/// # Parameters +/// +/// - `pipe_read_fd`: Refers to pipe read end, used to read data from the child process. +/// +/// - `pipe_write_fd`: Refers to pipe write end, used to close write end in the parent process. /// -/// - `pipe_read`: A `PipeReader` used to read data from the child process. +/// - `worker_info`: Info about the worker. /// -/// - `child`: The child pid. +/// - `job_pid`: The child pid. /// /// - `usage_before`: Resource usage statistics before executing the child process. /// -/// - `timeout`: The maximum allowed time for the child process to finish, in `Duration`. +/// - `timeout`: The maximum allowed time for the child process to finish. /// /// # Returns /// @@ -420,8 +419,8 @@ fn handle_child_process( fn handle_parent_process( pipe_read_fd: i32, pipe_write_fd: i32, + worker_info: &WorkerInfo, job_pid: Pid, - worker_pid: u32, usage_before: Usage, timeout: Duration, ) -> io::Result { @@ -445,7 +444,7 @@ fn handle_parent_process( let status = nix::sys::wait::waitpid(job_pid, None); gum::trace!( target: LOG_TARGET, - %worker_pid, + ?worker_info, %job_pid, "execute worker received wait status from job: {:?}", status, @@ -465,7 +464,7 @@ fn handle_parent_process( if cpu_tv >= timeout { gum::warn!( target: LOG_TARGET, - %worker_pid, + ?worker_info, %job_pid, "execute job took {}ms cpu time, exceeded execute timeout {}ms", cpu_tv.as_millis(), @@ -498,7 +497,7 @@ fn handle_parent_process( Err(job_error) => { gum::warn!( target: LOG_TARGET, - %worker_pid, + ?worker_info, %job_pid, "execute job error: {}", job_error, diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index 1d61998c3401..7f03ece46d21 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -20,7 +20,7 @@ mod memory_stats; use polkadot_node_core_pvf_common::{ executor_interface::{prepare, prevalidate}, - worker::{pipe2_cloexec, PipeFd}, + worker::{pipe2_cloexec, PipeFd, WorkerInfo}, }; // NOTE: Initializing logging in e.g. tests will not have an effect in the workers, as they are @@ -43,8 +43,6 @@ use nix::{ unistd::Pid, }; -#[cfg(target_os = "linux")] -use nix::sched::CloneFlags; use parity_scale_codec::{Decode, Encode}; use polkadot_node_core_pvf_common::{ error::{PrepareError, PrepareWorkerResult}, @@ -207,15 +205,14 @@ pub fn worker_entrypoint( worker_dir_path, node_version, worker_version, - |mut stream, worker_dir_path| { - let worker_pid = process::id(); - let temp_artifact_dest = worker_dir::prepare_tmp_artifact(&worker_dir_path); + |mut stream, worker_info| { + let temp_artifact_dest = worker_dir::prepare_tmp_artifact(&worker_info.worker_dir_path); loop { let pvf = recv_request(&mut stream)?; gum::debug!( target: LOG_TARGET, - %worker_pid, + ?worker_info, "worker: preparing artifact", ); @@ -241,17 +238,12 @@ pub fn worker_entrypoint( use polkadot_node_core_pvf_common::worker::security; let stack_size = 2 * 1024 * 1024; // 2MiB - let mut stack: Vec = vec![0u8; stack_size]; - // SIGCHLD flag is used to inform clone that the parent process is - // expecting a child termination signal, without this flag `waitpid` function - // return `ECHILD` error. - let flags = - security::clone::clone_sandbox_flags() | CloneFlags::from_bits_retain(libc::SIGCHLD); // SAFETY: new process is spawned within a single threaded process. This invariant // is enforced by tests. Stack size being specified to ensure child doesn't overflow let result = match unsafe { - nix::sched::clone( + security::clone::clone_on_worker( + worker_info, Box::new(|| { handle_child_process( pvf.clone(), @@ -263,23 +255,21 @@ pub fn worker_entrypoint( Arc::clone(&executor_params), ) }), - stack.as_mut_slice(), - flags, - None + stack_size, ) } { - Err(errno) => Err(error_from_errno("clone", errno)), Ok(child) => { handle_parent_process( pipe_reader, pipe_writer, - worker_pid, + worker_info, child, temp_artifact_dest.clone(), usage_before, preparation_timeout, ) }, + Err(security::clone::Error::Clone(errno)) => Err(error_from_errno("clone", errno)), }; } else { // SAFETY: new process is spawned within a single threaded process. This invariant @@ -302,7 +292,7 @@ pub fn worker_entrypoint( handle_parent_process( pipe_reader, pipe_writer, - worker_pid, + worker_info, child, temp_artifact_dest.clone(), usage_before, @@ -315,7 +305,7 @@ pub fn worker_entrypoint( gum::trace!( target: LOG_TARGET, - %worker_pid, + ?worker_info, "worker: sending result to host: {:?}", result ); @@ -559,23 +549,21 @@ fn handle_child_process( /// Waits for child process to finish and handle child response from pipe. /// -/// # Arguments +/// # Parameters /// -/// - `pipe_read_fd`: A `i32`, refers to pipe read end file descriptor, used to read data from the -/// child process. +/// - `pipe_read_fd`: Refers to pipe read end, used to read data from the child process. /// -/// - `pipe_write_fd`: A `i32`, refers to pipe write end file descriptor, used to close file -/// descriptor in the parent process. +/// - `pipe_write_fd`: Refers to pipe write end, used to close write end in the parent process. /// -/// - `child`: The child pid. +/// - `worker_info`: Info about the worker. /// -/// - `temp_artifact_dest`: The destination `PathBuf` to write the temporary artifact file. +/// - `job_pid`: The child pid. /// -/// - `worker_pid`: The PID of the child process. +/// - `temp_artifact_dest`: The destination `PathBuf` to write the temporary artifact file. /// /// - `usage_before`: Resource usage statistics before executing the child process. /// -/// - `timeout`: The maximum allowed time for the child process to finish, in `Duration`. +/// - `timeout`: The maximum allowed time for the child process to finish. /// /// # Returns /// @@ -588,7 +576,7 @@ fn handle_child_process( fn handle_parent_process( pipe_read_fd: i32, pipe_write_fd: i32, - worker_pid: u32, + worker_info: &WorkerInfo, job_pid: Pid, temp_artifact_dest: PathBuf, usage_before: Usage, @@ -612,7 +600,7 @@ fn handle_parent_process( let status = nix::sys::wait::waitpid(job_pid, None); gum::trace!( target: LOG_TARGET, - %worker_pid, + ?worker_info, %job_pid, "prepare worker received wait status from job: {:?}", status, @@ -630,7 +618,7 @@ fn handle_parent_process( if cpu_tv >= timeout { gum::warn!( target: LOG_TARGET, - %worker_pid, + ?worker_info, %job_pid, "prepare job took {}ms cpu time, exceeded prepare timeout {}ms", cpu_tv.as_millis(), @@ -665,7 +653,7 @@ fn handle_parent_process( // success. gum::debug!( target: LOG_TARGET, - %worker_pid, + ?worker_info, %job_pid, "worker: writing artifact to {}", temp_artifact_dest.display(), diff --git a/polkadot/node/core/pvf/src/host.rs b/polkadot/node/core/pvf/src/host.rs index d17a4d918e00..7097653012dd 100644 --- a/polkadot/node/core/pvf/src/host.rs +++ b/polkadot/node/core/pvf/src/host.rs @@ -906,7 +906,7 @@ pub(crate) mod tests { let _ = pulse.next().await.unwrap(); let el = start.elapsed().as_millis(); - assert!(el > 50 && el < 150, "{}", el); + assert!(el > 50 && el < 150, "pulse duration: {}", el); } } diff --git a/polkadot/node/core/pvf/src/security.rs b/polkadot/node/core/pvf/src/security.rs index 9d0d4cf49afe..abdf8101203e 100644 --- a/polkadot/node/core/pvf/src/security.rs +++ b/polkadot/node/core/pvf/src/security.rs @@ -32,14 +32,20 @@ use std::{fmt, path::Path}; pub async fn check_security_status(config: &Config) -> Result { let Config { prepare_worker_program_path, secure_validator_mode, cache_path, .. } = config; - let (landlock, seccomp, change_root) = join!( + let (landlock, seccomp, change_root, secure_clone) = join!( check_landlock(prepare_worker_program_path), check_seccomp(prepare_worker_program_path), - check_can_unshare_user_namespace_and_change_root(prepare_worker_program_path, cache_path) + check_can_unshare_user_namespace_and_change_root(prepare_worker_program_path, cache_path), + check_can_do_secure_clone(prepare_worker_program_path), ); - let full_security_status = - FullSecurityStatus::new(*secure_validator_mode, landlock, seccomp, change_root); + let full_security_status = FullSecurityStatus::new( + *secure_validator_mode, + landlock, + seccomp, + change_root, + secure_clone, + ); let security_status = full_security_status.as_partial(); if full_security_status.err_occurred() { @@ -73,6 +79,7 @@ impl FullSecurityStatus { landlock: SecureModeResult, seccomp: SecureModeResult, change_root: SecureModeResult, + secure_clone: SecureModeResult, ) -> Self { Self { partial: SecurityStatus { @@ -80,8 +87,9 @@ impl FullSecurityStatus { can_enable_landlock: landlock.is_ok(), can_enable_seccomp: seccomp.is_ok(), can_unshare_user_namespace_and_change_root: change_root.is_ok(), + can_do_secure_clone: secure_clone.is_ok(), }, - errs: [landlock, seccomp, change_root] + errs: [landlock, seccomp, change_root, secure_clone] .into_iter() .filter_map(|result| result.err()) .collect(), @@ -123,6 +131,7 @@ enum SecureModeError { CannotEnableLandlock(String), CannotEnableSeccomp(String), CannotUnshareUserNamespaceAndChangeRoot(String), + CannotDoSecureClone(String), } impl SecureModeError { @@ -138,6 +147,9 @@ impl SecureModeError { // Should always be present on modern Linuxes. If not, Landlock also provides FS // sandboxing, so don't enforce this. CannotUnshareUserNamespaceAndChangeRoot(_) => security_status.can_enable_landlock, + // We have not determined the kernel requirements for this capability, and it's also not + // necessary for FS or networking restrictions. + CannotDoSecureClone(_) => true, } } } @@ -149,6 +161,7 @@ impl fmt::Display for SecureModeError { CannotEnableLandlock(err) => write!(f, "Cannot enable landlock, a Linux 5.13+ kernel security feature: {err}"), CannotEnableSeccomp(err) => write!(f, "Cannot enable seccomp, a Linux-specific kernel security feature: {err}"), CannotUnshareUserNamespaceAndChangeRoot(err) => write!(f, "Cannot unshare user namespace and change root, which are Linux-specific kernel security features: {err}"), + CannotDoSecureClone(err) => write!(f, "Cannot call clone with all sandboxing flags, a Linux-specific kernel security features: {err}"), } } } @@ -348,6 +361,50 @@ async fn check_seccomp( } } +/// Check if we can call `clone` with all sandboxing flags, and return an error if not. +/// +/// We do this check by spawning a new process and trying to sandbox it. To get as close as possible +/// to running the check in a worker, we try it... in a worker. The expected return status is 0 on +/// success and -1 on failure. +async fn check_can_do_secure_clone( + #[cfg_attr(not(target_os = "linux"), allow(unused_variables))] + prepare_worker_program_path: &Path, +) -> SecureModeResult { + cfg_if::cfg_if! { + if #[cfg(target_os = "linux")] { + match tokio::process::Command::new(prepare_worker_program_path) + .arg("--check-can-do-secure-clone") + .output() + .await + { + Ok(output) if output.status.success() => Ok(()), + Ok(output) => { + let stderr = std::str::from_utf8(&output.stderr) + .expect("child process writes a UTF-8 string to stderr; qed") + .trim(); + if stderr.is_empty() { + Err(SecureModeError::CannotDoSecureClone( + "not available".into() + )) + } else { + Err(SecureModeError::CannotDoSecureClone( + format!("not available: {}", stderr) + )) + } + }, + Err(err) => + Err(SecureModeError::CannotDoSecureClone( + format!("could not start child process: {}", err) + )), + } + } else { + Err(SecureModeError::CannotDoSecureClone( + "only available on Linux".into() + )) + } + } +} + #[cfg(test)] mod tests { use super::*; @@ -359,13 +416,15 @@ mod tests { secure_validator_mode: true, can_enable_landlock: false, can_enable_seccomp: false, - can_unshare_user_namespace_and_change_root: true + can_unshare_user_namespace_and_change_root: true, + can_do_secure_clone: true, })); assert!(!err.is_allowed_in_secure_mode(&SecurityStatus { secure_validator_mode: true, can_enable_landlock: false, can_enable_seccomp: true, - can_unshare_user_namespace_and_change_root: false + can_unshare_user_namespace_and_change_root: false, + can_do_secure_clone: false, })); let err = SecureModeError::CannotEnableSeccomp(String::new()); @@ -373,13 +432,15 @@ mod tests { secure_validator_mode: true, can_enable_landlock: false, can_enable_seccomp: false, - can_unshare_user_namespace_and_change_root: true + can_unshare_user_namespace_and_change_root: true, + can_do_secure_clone: true, })); assert!(!err.is_allowed_in_secure_mode(&SecurityStatus { secure_validator_mode: true, can_enable_landlock: false, can_enable_seccomp: true, - can_unshare_user_namespace_and_change_root: false + can_unshare_user_namespace_and_change_root: false, + can_do_secure_clone: false, })); let err = SecureModeError::CannotUnshareUserNamespaceAndChangeRoot(String::new()); @@ -387,13 +448,31 @@ mod tests { secure_validator_mode: true, can_enable_landlock: true, can_enable_seccomp: false, - can_unshare_user_namespace_and_change_root: false + can_unshare_user_namespace_and_change_root: false, + can_do_secure_clone: false, })); assert!(!err.is_allowed_in_secure_mode(&SecurityStatus { secure_validator_mode: true, can_enable_landlock: false, can_enable_seccomp: true, - can_unshare_user_namespace_and_change_root: false + can_unshare_user_namespace_and_change_root: false, + can_do_secure_clone: false, + })); + + let err = SecureModeError::CannotDoSecureClone(String::new()); + assert!(err.is_allowed_in_secure_mode(&SecurityStatus { + secure_validator_mode: true, + can_enable_landlock: true, + can_enable_seccomp: true, + can_unshare_user_namespace_and_change_root: true, + can_do_secure_clone: true, + })); + assert!(err.is_allowed_in_secure_mode(&SecurityStatus { + secure_validator_mode: false, + can_enable_landlock: false, + can_enable_seccomp: false, + can_unshare_user_namespace_and_change_root: false, + can_do_secure_clone: false, })); } } diff --git a/polkadot/node/core/pvf/tests/it/main.rs b/polkadot/node/core/pvf/tests/it/main.rs index 09f975b706d2..df900618b62b 100644 --- a/polkadot/node/core/pvf/tests/it/main.rs +++ b/polkadot/node/core/pvf/tests/it/main.rs @@ -438,10 +438,13 @@ async fn all_security_features_work() { assert_eq!( host.security_status().await, SecurityStatus { + // Disabled in tests to not enforce the presence of security features. This CI-only test + // is the only one that tests them. secure_validator_mode: false, can_enable_landlock, can_enable_seccomp: true, can_unshare_user_namespace_and_change_root: true, + can_do_secure_clone: true, } ); } From 678acac4a252de6c1a767db443c44da0490b6429 Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Tue, 26 Dec 2023 17:02:12 +0100 Subject: [PATCH 24/50] Update syscall lists --- polkadot/scripts/list-syscalls/execute-worker-syscalls | 2 -- polkadot/scripts/list-syscalls/prepare-worker-syscalls | 2 -- 2 files changed, 4 deletions(-) diff --git a/polkadot/scripts/list-syscalls/execute-worker-syscalls b/polkadot/scripts/list-syscalls/execute-worker-syscalls index 349af783cf1a..c864d5ee7259 100644 --- a/polkadot/scripts/list-syscalls/execute-worker-syscalls +++ b/polkadot/scripts/list-syscalls/execute-worker-syscalls @@ -26,7 +26,6 @@ 45 (recvfrom) 46 (sendmsg) 56 (clone) -57 (fork) 60 (exit) 61 (wait4) 62 (kill) @@ -51,7 +50,6 @@ 158 (arch_prctl) 165 (mount) 166 (umount2) -186 (gettid) 200 (tkill) 202 (futex) 204 (sched_getaffinity) diff --git a/polkadot/scripts/list-syscalls/prepare-worker-syscalls b/polkadot/scripts/list-syscalls/prepare-worker-syscalls index 05281b61591a..3ecf5b589e6a 100644 --- a/polkadot/scripts/list-syscalls/prepare-worker-syscalls +++ b/polkadot/scripts/list-syscalls/prepare-worker-syscalls @@ -26,7 +26,6 @@ 45 (recvfrom) 46 (sendmsg) 56 (clone) -57 (fork) 60 (exit) 61 (wait4) 62 (kill) @@ -51,7 +50,6 @@ 158 (arch_prctl) 165 (mount) 166 (umount2) -186 (gettid) 200 (tkill) 202 (futex) 203 (sched_setaffinity) From 4132b7ba6008f2da00724ec786ae65c4c659df6f Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Tue, 26 Dec 2023 17:12:58 +0100 Subject: [PATCH 25/50] =?UTF-8?q?Add=20prdoc!=20=F0=9F=99=82?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../node/core/pvf/execute-worker/src/lib.rs | 2 -- ...477-use-clone-instead-of-fork-on-pvf.prdoc | 20 +++++++++++++++++++ 2 files changed, 20 insertions(+), 2 deletions(-) create mode 100644 prdoc/pr_2477-use-clone-instead-of-fork-on-pvf.prdoc diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index 6d38517c02f7..b7e26d9c5101 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -25,8 +25,6 @@ pub use polkadot_node_core_pvf_common::{ const LOG_TARGET: &str = "parachain::pvf-execute-worker"; use cpu_time::ProcessTime; -#[cfg(target_os = "linux")] -use nix::sched::CloneFlags; #[cfg(not(target_os = "linux"))] use nix::unistd::ForkResult; use nix::{ diff --git a/prdoc/pr_2477-use-clone-instead-of-fork-on-pvf.prdoc b/prdoc/pr_2477-use-clone-instead-of-fork-on-pvf.prdoc new file mode 100644 index 000000000000..5d5ea9092bd4 --- /dev/null +++ b/prdoc/pr_2477-use-clone-instead-of-fork-on-pvf.prdoc @@ -0,0 +1,20 @@ +title: "Use clone instead of fork on pvf" + +doc: + - audience: Node Operator + description: | + For validators: Adds a new, optional security capability. + Most modern Linux machines should support it, otherwise you will get a warning like: + "- Optional: Cannot call clone with all sandboxing flags, a Linux-specific kernel security features: not available" + If you are already running in a secure environment such as a container, this may conflict with our security features; your only option may be to ignore the warning. + Otherwise, it is recommended to upgrade your Linux version! + +migrations: + db: [] + + runtime: [] + +crates: + - name: polkadot-node-core-pvf + +host_functions: [] From fd462509e4b8c978bdaeebfa8f0c9dd78a31cc44 Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Tue, 26 Dec 2023 17:18:08 +0100 Subject: [PATCH 26/50] Small update to prdoc --- prdoc/pr_2477-use-clone-instead-of-fork-on-pvf.prdoc | 2 ++ 1 file changed, 2 insertions(+) diff --git a/prdoc/pr_2477-use-clone-instead-of-fork-on-pvf.prdoc b/prdoc/pr_2477-use-clone-instead-of-fork-on-pvf.prdoc index 5d5ea9092bd4..ac9a0a501b6c 100644 --- a/prdoc/pr_2477-use-clone-instead-of-fork-on-pvf.prdoc +++ b/prdoc/pr_2477-use-clone-instead-of-fork-on-pvf.prdoc @@ -16,5 +16,7 @@ migrations: crates: - name: polkadot-node-core-pvf + - name: polkadot-node-core-pvf-prepare-worker + - name: polkadot-node-core-pvf-execute-worker host_functions: [] From 3a8434fe479abd5213e8f1f6f26fcdf3b767615c Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Tue, 26 Dec 2023 16:52:24 -0300 Subject: [PATCH 27/50] change execute worker stack size --- .../node/core/pvf/execute-worker/src/lib.rs | 64 +++++++------------ 1 file changed, 24 insertions(+), 40 deletions(-) diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index b7e26d9c5101..5697f2aa457e 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -39,6 +39,7 @@ use parity_scale_codec::{Decode, Encode}; use polkadot_node_core_pvf_common::{ error::InternalValidationError, execute::{Handshake, JobError, JobResponse, JobResult, WorkerResponse}, + executor_interface::params_to_wasmtime_semantics, framed_recv_blocking, framed_send_blocking, worker::{ cpu_time_monitor_loop, pipe2_cloexec, run_worker, stringify_panic_payload, @@ -47,7 +48,7 @@ use polkadot_node_core_pvf_common::{ }, }; use polkadot_parachain_primitives::primitives::ValidationResult; -use polkadot_primitives::{executor_params::DEFAULT_NATIVE_STACK_MAX, ExecutorParams}; +use polkadot_primitives::ExecutorParams; use std::{ io::{self, Read}, os::{fd::AsRawFd, unix::net::UnixStream}, @@ -57,42 +58,6 @@ use std::{ time::Duration, }; -// Wasmtime powers the Substrate Executor. It compiles the wasm bytecode into native code. -// That native code does not create any stacks and just reuses the stack of the thread that -// wasmtime was invoked from. -// -// Also, we configure the executor to provide the deterministic stack and that requires -// supplying the amount of the native stack space that wasm is allowed to use. This is -// realized by supplying the limit into `wasmtime::Config::max_wasm_stack`. -// -// There are quirks to that configuration knob: -// -// 1. It only limits the amount of stack space consumed by wasm but does not ensure nor check that -// the stack space is actually available. -// -// That means, if the calling thread has 1 MiB of stack space left and the wasm code consumes -// more, then the wasmtime limit will **not** trigger. Instead, the wasm code will hit the -// guard page and the Rust stack overflow handler will be triggered. That leads to an -// **abort**. -// -// 2. It cannot and does not limit the stack space consumed by Rust code. -// -// Meaning that if the wasm code leaves no stack space for Rust code, then the Rust code -// will abort and that will abort the process as well. -// -// Typically on Linux the main thread gets the stack size specified by the `ulimit` and -// typically it's configured to 8 MiB. Rust's spawned threads are 2 MiB. OTOH, the -// DEFAULT_NATIVE_STACK_MAX is set to 256 MiB. Not nearly enough. -// -// Hence we need to increase it. The simplest way to fix that is to spawn a thread with the desired -// stack limit. -// -// The reasoning why we pick this particular size is: -// -// The default Rust thread stack limit 2 MiB + 256 MiB wasm stack. -/// The stack size for the execute thread. -pub const EXECUTE_THREAD_STACK_SIZE: usize = 2 * 1024 * 1024 + DEFAULT_NATIVE_STACK_MAX as usize; - /// Receives a handshake with information specific to the execute worker. fn recv_execute_handshake(stream: &mut UnixStream) -> io::Result { let handshake_enc = framed_recv_blocking(stream)?; @@ -190,11 +155,17 @@ pub fn worker_entrypoint( let executor_params = Arc::new(executor_params.clone()); let params = Arc::new(params); + let execute_worker_stack_size = get_max_stack_size(&executor_params, 1); + cfg_if::cfg_if! { if #[cfg(target_os = "linux")] { use polkadot_node_core_pvf_common::worker::security; - let stack_size = EXECUTE_THREAD_STACK_SIZE; + // Number of threads here is 3: + // 1 - Main thread + // 2 - Cpu monitor thread + // 3 - Execute thread + let stack_size = get_max_stack_size(&executor_params, 3); // SAFETY: new process is spawned within a single threaded process. This invariant // is enforced by tests. Stack size being specified to ensure child doesn't overflow @@ -210,9 +181,10 @@ pub fn worker_entrypoint( Arc::clone(&executor_params), Arc::clone(¶ms), execution_timeout, + execute_worker_stack_size, ) }), - stack_size, + stack_size as usize, ) } { Ok(child) => { @@ -241,6 +213,7 @@ pub fn worker_entrypoint( executor_params, params, execution_timeout, + execute_worker_stack_size, ) }, Ok(ForkResult::Parent { child }) => { @@ -322,6 +295,7 @@ fn handle_child_process( executor_params: Arc, params: Arc>, execution_timeout: Duration, + execute_worker_stack: usize, ) -> ! { // SAFETY: pipe_writer is an open and owned file descriptor at this point. let mut pipe_write = unsafe { PipeFd::new(pipe_write_fd) }; @@ -366,7 +340,7 @@ fn handle_child_process( move || validate_using_artifact(&compiled_artifact_blob, &executor_params, ¶ms), Arc::clone(&condvar), WaitOutcome::Finished, - EXECUTE_THREAD_STACK_SIZE, + execute_worker_stack, ) .unwrap_or_else(|err| { send_child_response(&mut pipe_write, Err(JobError::CouldNotSpawnThread(err.to_string()))) @@ -395,6 +369,16 @@ fn handle_child_process( send_child_response(&mut pipe_write, response); } +/// Returns clone stack size +/// The stack size is represented by 2MiB * number of threads + native stack; +fn get_max_stack_size(executor_params: &ExecutorParams, number_of_threads: u32) -> usize { + let deterministc_stack_limit = params_to_wasmtime_semantics(&*executor_params) + .deterministic_stack_limit + .expect("the default value is always available."); + return (2 * 1024 * 1024 * number_of_threads + deterministc_stack_limit.native_stack_max) + as usize; +} + /// Waits for child process to finish and handle child response from pipe. /// /// # Parameters From a8c422969554092abe74fd040c7fddd699ca0730 Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Tue, 26 Dec 2023 17:21:15 -0300 Subject: [PATCH 28/50] minor fixes --- polkadot/node/core/pvf/execute-worker/src/lib.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index 5697f2aa457e..12d61b584b90 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -155,7 +155,7 @@ pub fn worker_entrypoint( let executor_params = Arc::new(executor_params.clone()); let params = Arc::new(params); - let execute_worker_stack_size = get_max_stack_size(&executor_params, 1); + let execute_worker_stack_size = get_max_stack_size(&*executor_params, 1); cfg_if::cfg_if! { if #[cfg(target_os = "linux")] { @@ -165,7 +165,7 @@ pub fn worker_entrypoint( // 1 - Main thread // 2 - Cpu monitor thread // 3 - Execute thread - let stack_size = get_max_stack_size(&executor_params, 3); + let stack_size = get_max_stack_size(&*executor_params, 3); // SAFETY: new process is spawned within a single threaded process. This invariant // is enforced by tests. Stack size being specified to ensure child doesn't overflow @@ -372,7 +372,7 @@ fn handle_child_process( /// Returns clone stack size /// The stack size is represented by 2MiB * number of threads + native stack; fn get_max_stack_size(executor_params: &ExecutorParams, number_of_threads: u32) -> usize { - let deterministc_stack_limit = params_to_wasmtime_semantics(&*executor_params) + let deterministc_stack_limit = params_to_wasmtime_semantics(executor_params) .deterministic_stack_limit .expect("the default value is always available."); return (2 * 1024 * 1024 * number_of_threads + deterministc_stack_limit.native_stack_max) From 77b276f7c2387d6180ecfd0b41d16ae7cc625ba9 Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Tue, 26 Dec 2023 18:07:53 -0300 Subject: [PATCH 29/50] minor fixes --- polkadot/node/core/pvf/execute-worker/src/lib.rs | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index 12d61b584b90..48ab8a8408c9 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -152,10 +152,9 @@ pub fn worker_entrypoint( let stream_fd = stream.as_raw_fd(); let compiled_artifact_blob = Arc::new(compiled_artifact_blob); - let executor_params = Arc::new(executor_params.clone()); + let arc_executor_params: Arc = Arc::new(executor_params.clone()); let params = Arc::new(params); - - let execute_worker_stack_size = get_max_stack_size(&*executor_params, 1); + let execute_worker_stack_size = get_max_stack_size(&executor_params, 1); cfg_if::cfg_if! { if #[cfg(target_os = "linux")] { @@ -165,7 +164,7 @@ pub fn worker_entrypoint( // 1 - Main thread // 2 - Cpu monitor thread // 3 - Execute thread - let stack_size = get_max_stack_size(&*executor_params, 3); + let stack_size = get_max_stack_size(&executor_params, 3); // SAFETY: new process is spawned within a single threaded process. This invariant // is enforced by tests. Stack size being specified to ensure child doesn't overflow @@ -178,7 +177,7 @@ pub fn worker_entrypoint( pipe_reader, stream_fd, Arc::clone(&compiled_artifact_blob), - Arc::clone(&executor_params), + Arc::clone(&arc_executor_params), Arc::clone(¶ms), execution_timeout, execute_worker_stack_size, @@ -210,7 +209,7 @@ pub fn worker_entrypoint( pipe_reader, stream_fd, compiled_artifact_blob, - executor_params, + arc_executor_params, params, execution_timeout, execute_worker_stack_size, From 4c00ae0fbef93b233273555e96c06ffea0627410 Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Thu, 28 Dec 2023 11:25:20 +0100 Subject: [PATCH 30/50] Move exec param operations out of loop --- .../node/core/pvf/execute-worker/src/lib.rs | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index 48ab8a8408c9..d8d4be2d4eb1 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -118,6 +118,15 @@ pub fn worker_entrypoint( let Handshake { executor_params } = recv_execute_handshake(&mut stream)?; + let executor_params: Arc = Arc::new(executor_params); + let execute_worker_stack_size = get_max_stack_size(&executor_params, 1); + // Number of threads here is 3: + // 1 - Main thread + // 2 - Cpu monitor thread + // 3 - Execute thread + #[cfg(target_os = "linux")] + let clone_stack_size = get_max_stack_size(&executor_params, 3); + loop { let (params, execution_timeout) = recv_request(&mut stream)?; gum::debug!( @@ -152,20 +161,12 @@ pub fn worker_entrypoint( let stream_fd = stream.as_raw_fd(); let compiled_artifact_blob = Arc::new(compiled_artifact_blob); - let arc_executor_params: Arc = Arc::new(executor_params.clone()); let params = Arc::new(params); - let execute_worker_stack_size = get_max_stack_size(&executor_params, 1); cfg_if::cfg_if! { if #[cfg(target_os = "linux")] { use polkadot_node_core_pvf_common::worker::security; - // Number of threads here is 3: - // 1 - Main thread - // 2 - Cpu monitor thread - // 3 - Execute thread - let stack_size = get_max_stack_size(&executor_params, 3); - // SAFETY: new process is spawned within a single threaded process. This invariant // is enforced by tests. Stack size being specified to ensure child doesn't overflow let result = match unsafe { @@ -177,13 +178,13 @@ pub fn worker_entrypoint( pipe_reader, stream_fd, Arc::clone(&compiled_artifact_blob), - Arc::clone(&arc_executor_params), + Arc::clone(&executor_params), Arc::clone(¶ms), execution_timeout, execute_worker_stack_size, ) }), - stack_size as usize, + clone_stack_size as usize, ) } { Ok(child) => { @@ -209,7 +210,7 @@ pub fn worker_entrypoint( pipe_reader, stream_fd, compiled_artifact_blob, - arc_executor_params, + executor_params, params, execution_timeout, execute_worker_stack_size, From 3366dc2142973e74c882fab1c207458b0945ebba Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Thu, 28 Dec 2023 11:38:55 +0100 Subject: [PATCH 31/50] Remove any --- polkadot/node/core/pvf/common/src/worker/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/polkadot/node/core/pvf/common/src/worker/mod.rs b/polkadot/node/core/pvf/common/src/worker/mod.rs index dd2086bac620..f939d5ef0542 100644 --- a/polkadot/node/core/pvf/common/src/worker/mod.rs +++ b/polkadot/node/core/pvf/common/src/worker/mod.rs @@ -203,7 +203,7 @@ pub fn pipe2_cloexec() -> io::Result<(libc::c_int, libc::c_int)> { Ok((fds[0], fds[1])) } -#[cfg(any(target_os = "macos"))] +#[cfg(target_os = "macos")] pub fn pipe2_cloexec() -> io::Result<(libc::c_int, libc::c_int)> { let mut fds: [libc::c_int; 2] = [0; 2]; let res = unsafe { libc::pipe(fds.as_mut_ptr()) }; From dc4dc1a26f5ca49659901ba644fec1ccedac0d48 Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Thu, 28 Dec 2023 08:47:37 -0300 Subject: [PATCH 32/50] add solid proof for `deterministic_stack_limit`, fix comment for get_max_stack_size and other fixes --- .../node/core/pvf/execute-worker/src/lib.rs | 51 ++++++++++++++++--- 1 file changed, 44 insertions(+), 7 deletions(-) diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index 48ab8a8408c9..c74cacb33e82 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -164,6 +164,8 @@ pub fn worker_entrypoint( // 1 - Main thread // 2 - Cpu monitor thread // 3 - Execute thread + // + // NOTE: if the number of threads inside the child process change in the future, this value must be changed as well. let stack_size = get_max_stack_size(&executor_params, 3); // SAFETY: new process is spawned within a single threaded process. This invariant @@ -183,7 +185,7 @@ pub fn worker_entrypoint( execute_worker_stack_size, ) }), - stack_size as usize, + stack_size, ) } { Ok(child) => { @@ -294,7 +296,7 @@ fn handle_child_process( executor_params: Arc, params: Arc>, execution_timeout: Duration, - execute_worker_stack: usize, + execute_worker_stack_size: usize, ) -> ! { // SAFETY: pipe_writer is an open and owned file descriptor at this point. let mut pipe_write = unsafe { PipeFd::new(pipe_write_fd) }; @@ -339,7 +341,7 @@ fn handle_child_process( move || validate_using_artifact(&compiled_artifact_blob, &executor_params, ¶ms), Arc::clone(&condvar), WaitOutcome::Finished, - execute_worker_stack, + execute_worker_stack_size, ) .unwrap_or_else(|err| { send_child_response(&mut pipe_write, Err(JobError::CouldNotSpawnThread(err.to_string()))) @@ -368,13 +370,48 @@ fn handle_child_process( send_child_response(&mut pipe_write, response); } -/// Returns clone stack size +/// Returns stack size based on the number of threads. /// The stack size is represented by 2MiB * number of threads + native stack; +/// +/// Wasmtime powers the Substrate Executor. It compiles the wasm bytecode into native code. +/// That native code does not create any stacks and just reuses the stack of the thread that +/// wasmtime was invoked from. +/// +/// Also, we configure the executor to provide the deterministic stack and that requires +/// supplying the amount of the native stack space that wasm is allowed to use. This is +/// realized by supplying the limit into `wasmtime::Config::max_wasm_stack`. +/// +/// There are quirks to that configuration knob: +/// +/// 1. It only limits the amount of stack space consumed by wasm but does not ensure nor check that +/// the stack space is actually available. +/// +/// That means, if the calling thread has 1 MiB of stack space left and the wasm code consumes +/// more, then the wasmtime limit will **not** trigger. Instead, the wasm code will hit the +/// guard page and the Rust stack overflow handler will be triggered. That leads to an +/// **abort**. +/// +/// 2. It cannot and does not limit the stack space consumed by Rust code. +/// +/// Meaning that if the wasm code leaves no stack space for Rust code, then the Rust code +/// will abort and that will abort the process as well. +/// +/// Typically on Linux the main thread gets the stack size specified by the `ulimit` and +/// typically it's configured to 8 MiB. Rust's spawned threads are 2 MiB. OTOH, the +/// DEFAULT_NATIVE_STACK_MAX is set to 256 MiB. Not nearly enough. +/// +/// Hence we need to increase it. The simplest way to fix that is to spawn a thread with the desired +/// stack limit. +/// +/// The reasoning why we pick this particular size is: +/// +/// The default Rust thread stack limit 2 MiB + 256 MiB wasm stack. +/// The stack size for the execute thread. fn get_max_stack_size(executor_params: &ExecutorParams, number_of_threads: u32) -> usize { - let deterministc_stack_limit = params_to_wasmtime_semantics(executor_params) + let deterministic_stack_limit = params_to_wasmtime_semantics(executor_params) .deterministic_stack_limit - .expect("the default value is always available."); - return (2 * 1024 * 1024 * number_of_threads + deterministc_stack_limit.native_stack_max) + .expect("`params_to_wasmtime_semantics` get this value from the `DEFAULT_CONFIG`; it should always be available; qed"); + return (2 * 1024 * 1024 * number_of_threads + deterministic_stack_limit.native_stack_max) as usize; } From 82ee6c467a98b165f1aaff2f10583b7702d70722 Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Thu, 28 Dec 2023 08:57:27 -0300 Subject: [PATCH 33/50] fmt --- polkadot/node/core/pvf/execute-worker/src/lib.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index e657952be391..a9208cd09a12 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -124,8 +124,9 @@ pub fn worker_entrypoint( // 1 - Main thread // 2 - Cpu monitor thread // 3 - Execute thread - // - // NOTE: if the number of threads inside the child process change in the future, this value must be changed as well. + // + // NOTE: if the number of threads inside the child process change in the future, this + // value must be changed as well. #[cfg(target_os = "linux")] let clone_stack_size = get_max_stack_size(&executor_params, 3); @@ -373,7 +374,7 @@ fn handle_child_process( /// Returns stack size based on the number of threads. /// The stack size is represented by 2MiB * number of threads + native stack; -/// +/// /// Wasmtime powers the Substrate Executor. It compiles the wasm bytecode into native code. /// That native code does not create any stacks and just reuses the stack of the thread that /// wasmtime was invoked from. From f72d8ef9f647ce69f03ca8d28c3f59e46e611800 Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Thu, 28 Dec 2023 15:01:46 -0300 Subject: [PATCH 34/50] move execute worker thread number to a constant --- .../node/core/pvf/execute-worker/src/lib.rs | 25 ++++++++----------- polkadot/node/core/pvf/tests/it/process.rs | 2 +- 2 files changed, 12 insertions(+), 15 deletions(-) diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index a9208cd09a12..5d815c582e8e 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -24,6 +24,15 @@ pub use polkadot_node_core_pvf_common::{ // separate spawned processes. Run with e.g. `RUST_LOG=parachain::pvf-execute-worker=trace`. const LOG_TARGET: &str = "parachain::pvf-execute-worker"; +/// The number of threads for the child process: +/// 1 - Main thread +/// 2 - Cpu monitor thread +/// 3 - Execute thread +/// +/// NOTE: The correctness of this value is enforced by a test. If the number of threads inside +/// the child process change in the future, this value must be changed as well. +pub const EXECUTE_WORKER_THREAD_NUMBER: u32 = 3; + use cpu_time::ProcessTime; #[cfg(not(target_os = "linux"))] use nix::unistd::ForkResult; @@ -120,15 +129,8 @@ pub fn worker_entrypoint( let executor_params: Arc = Arc::new(executor_params); let execute_worker_stack_size = get_max_stack_size(&executor_params, 1); - // Number of threads here is 3: - // 1 - Main thread - // 2 - Cpu monitor thread - // 3 - Execute thread - // - // NOTE: if the number of threads inside the child process change in the future, this - // value must be changed as well. #[cfg(target_os = "linux")] - let clone_stack_size = get_max_stack_size(&executor_params, 3); + let clone_stack_size = get_max_stack_size(&executor_params, EXECUTE_WORKER_THREAD_NUMBER); loop { let (params, execution_timeout) = recv_request(&mut stream)?; @@ -373,7 +375,7 @@ fn handle_child_process( } /// Returns stack size based on the number of threads. -/// The stack size is represented by 2MiB * number of threads + native stack; +/// The stack size is represented by 2MiB * EXECUTE_WORKER_THREAD_NUMBER + native stack; /// /// Wasmtime powers the Substrate Executor. It compiles the wasm bytecode into native code. /// That native code does not create any stacks and just reuses the stack of the thread that @@ -404,11 +406,6 @@ fn handle_child_process( /// /// Hence we need to increase it. The simplest way to fix that is to spawn a thread with the desired /// stack limit. -/// -/// The reasoning why we pick this particular size is: -/// -/// The default Rust thread stack limit 2 MiB + 256 MiB wasm stack. -/// The stack size for the execute thread. fn get_max_stack_size(executor_params: &ExecutorParams, number_of_threads: u32) -> usize { let deterministic_stack_limit = params_to_wasmtime_semantics(executor_params) .deterministic_stack_limit diff --git a/polkadot/node/core/pvf/tests/it/process.rs b/polkadot/node/core/pvf/tests/it/process.rs index b742acb15d02..888cda5a13f3 100644 --- a/polkadot/node/core/pvf/tests/it/process.rs +++ b/polkadot/node/core/pvf/tests/it/process.rs @@ -373,7 +373,7 @@ rusty_fork_test! { // time monitor. assert_eq!( get_num_threads_by_sid_and_name(sid, EXECUTE_PROCESS_NAME, false), - 3 + polkadot_node_core_pvf_execute_worker::EXECUTE_WORKER_THREAD_NUMBER as i64, ); // End the test. From 2e3140bf3c4af02f018d7dabfe082af020c598ef Mon Sep 17 00:00:00 2001 From: Marcin S Date: Fri, 29 Dec 2023 10:44:34 +0100 Subject: [PATCH 35/50] Update polkadot/node/core/pvf/execute-worker/src/lib.rs --- polkadot/node/core/pvf/execute-worker/src/lib.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index 5d815c582e8e..c7c616eebfdf 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -375,7 +375,7 @@ fn handle_child_process( } /// Returns stack size based on the number of threads. -/// The stack size is represented by 2MiB * EXECUTE_WORKER_THREAD_NUMBER + native stack; +/// The stack size is represented by 2MiB * number_of_threads + native stack; /// /// Wasmtime powers the Substrate Executor. It compiles the wasm bytecode into native code. /// That native code does not create any stacks and just reuses the stack of the thread that From 97acfdc8dc4a8aaf819455fbf0a58f23a45adca8 Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Fri, 29 Dec 2023 11:18:20 +0100 Subject: [PATCH 36/50] Remove brittle proof + account for threads for prepare process stack --- .../core/pvf/common/src/executor_interface.rs | 13 ++++---- .../node/core/pvf/execute-worker/src/lib.rs | 30 ++++++++++--------- .../node/core/pvf/prepare-worker/src/lib.rs | 15 ++++++++-- polkadot/node/core/pvf/tests/it/process.rs | 2 +- 4 files changed, 38 insertions(+), 22 deletions(-) diff --git a/polkadot/node/core/pvf/common/src/executor_interface.rs b/polkadot/node/core/pvf/common/src/executor_interface.rs index e634940dbe65..4cd2f5c85eec 100644 --- a/polkadot/node/core/pvf/common/src/executor_interface.rs +++ b/polkadot/node/core/pvf/common/src/executor_interface.rs @@ -140,7 +140,7 @@ pub unsafe fn create_runtime_from_artifact_bytes( executor_params: &ExecutorParams, ) -> Result { let mut config = DEFAULT_CONFIG.clone(); - config.semantics = params_to_wasmtime_semantics(executor_params); + config.semantics = params_to_wasmtime_semantics(executor_params).0; sc_executor_wasmtime::create_runtime_from_artifact_bytes::( compiled_artifact_blob, @@ -148,7 +148,10 @@ pub unsafe fn create_runtime_from_artifact_bytes( ) } -pub fn params_to_wasmtime_semantics(par: &ExecutorParams) -> Semantics { +/// Takes the default config and overwrites any settings with existing executor parameters. +/// +/// Returns the semantics as well as the stack limit (since we are guaranteed to have it). +pub fn params_to_wasmtime_semantics(par: &ExecutorParams) -> (Semantics, DeterministicStackLimit) { let mut sem = DEFAULT_CONFIG.semantics.clone(); let mut stack_limit = sem .deterministic_stack_limit @@ -169,8 +172,8 @@ pub fn params_to_wasmtime_semantics(par: &ExecutorParams) -> Semantics { ExecutorParam::PvfExecTimeout(_, _) => (), /* Not used here */ } } - sem.deterministic_stack_limit = Some(stack_limit); - sem + sem.deterministic_stack_limit = Some(stack_limit.clone()); + (sem, stack_limit) } /// Runs the prevalidation on the given code. Returns a [`RuntimeBlob`] if it succeeds. @@ -187,7 +190,7 @@ pub fn prepare( blob: RuntimeBlob, executor_params: &ExecutorParams, ) -> Result, sc_executor_common::error::WasmError> { - let semantics = params_to_wasmtime_semantics(executor_params); + let (semantics, _) = params_to_wasmtime_semantics(executor_params); sc_executor_wasmtime::prepare_runtime_artifact(blob, &semantics) } diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index c7c616eebfdf..c99271fabf22 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -24,15 +24,6 @@ pub use polkadot_node_core_pvf_common::{ // separate spawned processes. Run with e.g. `RUST_LOG=parachain::pvf-execute-worker=trace`. const LOG_TARGET: &str = "parachain::pvf-execute-worker"; -/// The number of threads for the child process: -/// 1 - Main thread -/// 2 - Cpu monitor thread -/// 3 - Execute thread -/// -/// NOTE: The correctness of this value is enforced by a test. If the number of threads inside -/// the child process change in the future, this value must be changed as well. -pub const EXECUTE_WORKER_THREAD_NUMBER: u32 = 3; - use cpu_time::ProcessTime; #[cfg(not(target_os = "linux"))] use nix::unistd::ForkResult; @@ -67,6 +58,15 @@ use std::{ time::Duration, }; +/// The number of threads for the child process: +/// 1 - Main thread +/// 2 - Cpu monitor thread +/// 3 - Execute thread +/// +/// NOTE: The correctness of this value is enforced by a test. If the number of threads inside +/// the child process change in the future, this value must be changed as well. +pub const EXECUTE_WORKER_THREAD_NUMBER: u32 = 3; + /// Receives a handshake with information specific to the execute worker. fn recv_execute_handshake(stream: &mut UnixStream) -> io::Result { let handshake_enc = framed_recv_blocking(stream)?; @@ -377,6 +377,8 @@ fn handle_child_process( /// Returns stack size based on the number of threads. /// The stack size is represented by 2MiB * number_of_threads + native stack; /// +/// # Background +/// /// Wasmtime powers the Substrate Executor. It compiles the wasm bytecode into native code. /// That native code does not create any stacks and just reuses the stack of the thread that /// wasmtime was invoked from. @@ -404,12 +406,12 @@ fn handle_child_process( /// typically it's configured to 8 MiB. Rust's spawned threads are 2 MiB. OTOH, the /// DEFAULT_NATIVE_STACK_MAX is set to 256 MiB. Not nearly enough. /// -/// Hence we need to increase it. The simplest way to fix that is to spawn a thread with the desired -/// stack limit. +/// Hence we need to increase it. The simplest way to fix that is to spawn an execute thread with +/// the desired stack limit. We must also make sure the job process has enough stack for *all* its +/// threads. This function can be used to get the stack size of either the execute thread or execute +/// job process. fn get_max_stack_size(executor_params: &ExecutorParams, number_of_threads: u32) -> usize { - let deterministic_stack_limit = params_to_wasmtime_semantics(executor_params) - .deterministic_stack_limit - .expect("`params_to_wasmtime_semantics` get this value from the `DEFAULT_CONFIG`; it should always be available; qed"); + let (_sem, deterministic_stack_limit) = params_to_wasmtime_semantics(executor_params); return (2 * 1024 * 1024 * number_of_threads + deterministic_stack_limit.native_stack_max) as usize; } diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index 7f03ece46d21..c93935780f4e 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -81,6 +81,16 @@ static ALLOC: TrackingAllocator = #[global_allocator] static ALLOC: TrackingAllocator = TrackingAllocator(std::alloc::System); +/// The number of threads for the child process: +/// 1 - Main thread +/// 2 - Cpu monitor thread +/// 3 - Memory tracker thread +/// 3 - Prepare thread +/// +/// NOTE: The correctness of this value is enforced by a test. If the number of threads inside +/// the child process change in the future, this value must be changed as well. +pub const PREPARE_WORKER_THREAD_NUMBER: u32 = 4; + /// Contains the bytes for a successfully compiled artifact. #[derive(Encode, Decode)] pub struct CompiledArtifact(Vec); @@ -237,7 +247,8 @@ pub fn worker_entrypoint( if #[cfg(target_os = "linux")] { use polkadot_node_core_pvf_common::worker::security; - let stack_size = 2 * 1024 * 1024; // 2MiB + // 2MiB * num_threads + let stack_size = 2 * 1024 * 1024 * PREPARE_WORKER_THREAD_NUMBER; // SAFETY: new process is spawned within a single threaded process. This invariant // is enforced by tests. Stack size being specified to ensure child doesn't overflow @@ -255,7 +266,7 @@ pub fn worker_entrypoint( Arc::clone(&executor_params), ) }), - stack_size, + stack_size as usize, ) } { Ok(child) => { diff --git a/polkadot/node/core/pvf/tests/it/process.rs b/polkadot/node/core/pvf/tests/it/process.rs index 888cda5a13f3..de530c7f30b9 100644 --- a/polkadot/node/core/pvf/tests/it/process.rs +++ b/polkadot/node/core/pvf/tests/it/process.rs @@ -323,7 +323,7 @@ rusty_fork_test! { // monitor, and memory tracking. assert_eq!( get_num_threads_by_sid_and_name(sid, PREPARE_PROCESS_NAME, false), - 4 + polkadot_node_core_pvf_prepare_worker::PREPARE_WORKER_THREAD_NUMBER as i64, ); // End the test. From 1f6457782eed45ff083dff614edfdc79bb7ffe0d Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Mon, 1 Jan 2024 18:12:16 +0100 Subject: [PATCH 37/50] Do fork as fallback if can_do_secure_clone is false --- .../node/core/pvf/common/src/worker/mod.rs | 22 +- .../node/core/pvf/execute-worker/src/lib.rs | 239 ++++++++++------- .../node/core/pvf/prepare-worker/src/lib.rs | 252 ++++++++++-------- 3 files changed, 287 insertions(+), 226 deletions(-) diff --git a/polkadot/node/core/pvf/common/src/worker/mod.rs b/polkadot/node/core/pvf/common/src/worker/mod.rs index f939d5ef0542..938f31eed44a 100644 --- a/polkadot/node/core/pvf/common/src/worker/mod.rs +++ b/polkadot/node/core/pvf/common/src/worker/mod.rs @@ -18,7 +18,7 @@ pub mod security; -use crate::{framed_recv_blocking, WorkerHandshake, LOG_TARGET}; +use crate::{framed_recv_blocking, SecurityStatus, WorkerHandshake, LOG_TARGET}; use cpu_time::ProcessTime; use futures::never::Never; use parity_scale_codec::Decode; @@ -245,13 +245,13 @@ impl PipeFd { } impl Read for PipeFd { - fn read_to_end(&mut self, buf: &mut Vec) -> io::Result { - self.file.read_to_end(buf) - } - fn read(&mut self, buf: &mut [u8]) -> io::Result { self.file.read(buf) } + + fn read_to_end(&mut self, buf: &mut Vec) -> io::Result { + self.file.read_to_end(buf) + } } impl Write for PipeFd { @@ -313,7 +313,7 @@ pub fn run_worker( worker_version: Option<&str>, mut event_loop: F, ) where - F: FnMut(UnixStream, &WorkerInfo) -> io::Result, + F: FnMut(UnixStream, &WorkerInfo, SecurityStatus) -> io::Result, { #[cfg_attr(not(target_os = "linux"), allow(unused_mut))] let mut worker_info = WorkerInfo { @@ -345,11 +345,9 @@ pub fn run_worker( } // Make sure that we can read the worker dir path, and log its contents. - let entries = || -> Result, io::Error> { - std::fs::read_dir(&worker_info.worker_dir_path)? - .map(|res| res.map(|e| e.file_name())) - .collect() - }(); + let entries = std::fs::read_dir(&worker_info.worker_dir_path)? + .map(|res| res.map(|e| e.file_name())) + .collect(); match entries { Ok(entries) => gum::trace!(target: LOG_TARGET, ?worker_info, "content of worker dir: {:?}", entries), @@ -448,7 +446,7 @@ pub fn run_worker( } // Run the main worker loop. - let err = event_loop(stream, &worker_info) + let err = event_loop(stream, &worker_info, security_status) // It's never `Ok` because it's `Ok(Never)`. .unwrap_err(); diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index c99271fabf22..8a00247c7be3 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -16,24 +16,20 @@ //! Contains the logic for executing PVFs. Used by the polkadot-execute-worker binary. -pub use polkadot_node_core_pvf_common::{ - executor_interface::execute_artifact, worker::WorkerInfo, worker_dir, -}; +pub use polkadot_node_core_pvf_common::executor_interface::execute_artifact; // NOTE: Initializing logging in e.g. tests will not have an effect in the workers, as they are // separate spawned processes. Run with e.g. `RUST_LOG=parachain::pvf-execute-worker=trace`. const LOG_TARGET: &str = "parachain::pvf-execute-worker"; use cpu_time::ProcessTime; -#[cfg(not(target_os = "linux"))] -use nix::unistd::ForkResult; use nix::{ errno::Errno, sys::{ resource::{Usage, UsageWho}, wait::WaitStatus, }, - unistd::Pid, + unistd::{ForkResult, Pid}, }; use parity_scale_codec::{Decode, Encode}; use polkadot_node_core_pvf_common::{ @@ -44,8 +40,9 @@ use polkadot_node_core_pvf_common::{ worker::{ cpu_time_monitor_loop, pipe2_cloexec, run_worker, stringify_panic_payload, thread::{self, WaitOutcome}, - PipeFd, WorkerKind, + PipeFd, WorkerInfo, WorkerKind, }, + worker_dir, }; use polkadot_parachain_primitives::primitives::ValidationResult; use polkadot_primitives::ExecutorParams; @@ -64,7 +61,7 @@ use std::{ /// 3 - Execute thread /// /// NOTE: The correctness of this value is enforced by a test. If the number of threads inside -/// the child process change in the future, this value must be changed as well. +/// the child process changes in the future, this value must be changed as well. pub const EXECUTE_WORKER_THREAD_NUMBER: u32 = 3; /// Receives a handshake with information specific to the execute worker. @@ -122,13 +119,13 @@ pub fn worker_entrypoint( worker_dir_path, node_version, worker_version, - |mut stream, worker_info| { + |mut stream, worker_info, security_status| { let artifact_path = worker_dir::execute_artifact(&worker_info.worker_dir_path); let Handshake { executor_params } = recv_execute_handshake(&mut stream)?; let executor_params: Arc = Arc::new(executor_params); - let execute_worker_stack_size = get_max_stack_size(&executor_params, 1); + let execute_thread_stack_size = get_max_stack_size(&executor_params, 1); #[cfg(target_os = "linux")] let clone_stack_size = get_max_stack_size(&executor_params, EXECUTE_WORKER_THREAD_NUMBER); @@ -137,6 +134,7 @@ pub fn worker_entrypoint( gum::debug!( target: LOG_TARGET, ?worker_info, + ?security_status, "worker: validating artifact {}", artifact_path.display(), ); @@ -153,7 +151,7 @@ pub fn worker_entrypoint( }, }; - let (pipe_reader, pipe_writer) = pipe2_cloexec()?; + let (pipe_read_fd, pipe_write_fd) = pipe2_cloexec()?; let usage_before = match nix::sys::resource::getrusage(UsageWho::RUSAGE_CHILDREN) { Ok(usage) => usage, @@ -170,70 +168,50 @@ pub fn worker_entrypoint( cfg_if::cfg_if! { if #[cfg(target_os = "linux")] { - use polkadot_node_core_pvf_common::worker::security; - - // SAFETY: new process is spawned within a single threaded process. This invariant - // is enforced by tests. Stack size being specified to ensure child doesn't overflow - let result = match unsafe { - security::clone::clone_on_worker( - worker_info, - Box::new(|| { - handle_child_process( - pipe_writer, - pipe_reader, - stream_fd, - Arc::clone(&compiled_artifact_blob), - Arc::clone(&executor_params), - Arc::clone(¶ms), - execution_timeout, - execute_worker_stack_size, - ) - }), + let result = if security_status.can_do_secure_clone { + handle_clone( + pipe_write_fd, + pipe_read_fd, + stream_fd, + &compiled_artifact_blob, + &executor_params, + ¶ms, + execution_timeout, + execute_thread_stack_size, clone_stack_size, - ) - } { - Ok(child) => { - handle_parent_process( - pipe_reader, - pipe_writer, - worker_info, - child, - usage_before, - execution_timeout, - )? - }, - Err(security::clone::Error::Clone(errno)) => internal_error_from_errno("clone", errno), + worker_info, + usage_before, + )? + } else { + // Fall back to using fork. + handle_fork( + pipe_write_fd, + pipe_read_fd, + stream_fd, + &compiled_artifact_blob, + &executor_params, + ¶ms, + execution_timeout, + execute_thread_stack_size, + worker_info, + usage_before, + )? }; } else { - // SAFETY: new process is spawned within a single threaded process. This invariant - // is enforced by tests. - let result = match unsafe { nix::unistd::fork() } { - Err(errno) => internal_error_from_errno("fork", errno), - Ok(ForkResult::Child) => { - handle_child_process( - pipe_writer, - pipe_reader, - stream_fd, - compiled_artifact_blob, - executor_params, - params, - execution_timeout, - execute_worker_stack_size, - ) - }, - Ok(ForkResult::Parent { child }) => { - handle_parent_process( - pipe_reader, - pipe_writer, - worker_info, - child, - usage_before, - execution_timeout, - )? - }, - }; + let result = handle_fork( + pipe_write_fd, + pipe_read_fd, + stream_fd, + &compiled_artifact_blob, + &executor_params, + ¶ms, + execution_timeout, + execute_thread_stack_size, + worker_info, + usage_before, + )?; } - }; + } gum::trace!( target: LOG_TARGET, @@ -274,20 +252,93 @@ fn validate_using_artifact( JobResponse::Ok { result_descriptor } } +#[cfg(target_os = "linux")] +fn handle_clone( + pipe_write_fd: i32, + pipe_read_fd: i32, + stream_fd: i32, + compiled_artifact_blob: &Arc>, + executor_params: &Arc, + params: &Arc>, + execution_timeout: Duration, + execute_stack_size: usize, + clone_stack_size: usize, + worker_info: &WorkerInfo, + usage_before: Usage, +) -> io::Result { + use polkadot_node_core_pvf_common::worker::security; + + // SAFETY: new process is spawned within a single threaded process. This invariant + // is enforced by tests. Stack size being specified to ensure child doesn't overflow + match unsafe { + security::clone::clone_on_worker( + worker_info, + Box::new(|| { + handle_child_process( + pipe_write_fd, + pipe_read_fd, + stream_fd, + Arc::clone(compiled_artifact_blob), + Arc::clone(executor_params), + Arc::clone(params), + execution_timeout, + execute_stack_size, + ) + }), + clone_stack_size, + ) + } { + Ok(child) => handle_parent_process( + pipe_read_fd, + pipe_write_fd, + worker_info, + child, + usage_before, + execution_timeout, + ), + Err(security::clone::Error::Clone(errno)) => Ok(internal_error_from_errno("clone", errno)), + } +} + +fn handle_fork( + pipe_write_fd: i32, + pipe_read_fd: i32, + stream_fd: i32, + compiled_artifact_blob: &Arc>, + executor_params: &Arc, + params: &Arc>, + execution_timeout: Duration, + execute_worker_stack_size: usize, + worker_info: &WorkerInfo, + usage_before: Usage, +) -> io::Result { + // SAFETY: new process is spawned within a single threaded process. This invariant + // is enforced by tests. + match unsafe { nix::unistd::fork() } { + Ok(ForkResult::Child) => handle_child_process( + pipe_write_fd, + pipe_read_fd, + stream_fd, + Arc::clone(compiled_artifact_blob), + Arc::clone(executor_params), + Arc::clone(params), + execution_timeout, + execute_worker_stack_size, + ), + Ok(ForkResult::Parent { child }) => handle_parent_process( + pipe_read_fd, + pipe_write_fd, + worker_info, + child, + usage_before, + execution_timeout, + ), + Err(errno) => Ok(internal_error_from_errno("fork", errno)), + } +} + /// This is used to handle child process during pvf execute worker. -/// It execute the artifact and pipes back the response to the parent process -/// -/// # Arguments -/// -/// - `pipe_write`: A `PipeWriter` structure, the writing end of a pipe. -/// -/// - `compiled_artifact_blob`: The artifact bytes from compiled by the prepare worker`. -/// -/// - `executor_params`: Deterministically serialized execution environment semantics. -/// -/// - `params`: Validation parameters. -/// -/// - `execution_timeout`: The timeout in `Duration`. +/// It executes the artifact and pipes back the response to the parent process. /// /// # Returns /// @@ -300,9 +351,9 @@ fn handle_child_process( executor_params: Arc, params: Arc>, execution_timeout: Duration, - execute_worker_stack_size: usize, + execute_thread_stack_size: usize, ) -> ! { - // SAFETY: pipe_writer is an open and owned file descriptor at this point. + // SAFETY: this is an open and owned file descriptor at this point. let mut pipe_write = unsafe { PipeFd::new(pipe_write_fd) }; // Drop the read end so we don't have too many FDs open. @@ -345,7 +396,7 @@ fn handle_child_process( move || validate_using_artifact(&compiled_artifact_blob, &executor_params, ¶ms), Arc::clone(&condvar), WaitOutcome::Finished, - execute_worker_stack_size, + execute_thread_stack_size, ) .unwrap_or_else(|err| { send_child_response(&mut pipe_write, Err(JobError::CouldNotSpawnThread(err.to_string()))) @@ -418,20 +469,6 @@ fn get_max_stack_size(executor_params: &ExecutorParams, number_of_threads: u32) /// Waits for child process to finish and handle child response from pipe. /// -/// # Parameters -/// -/// - `pipe_read_fd`: Refers to pipe read end, used to read data from the child process. -/// -/// - `pipe_write_fd`: Refers to pipe write end, used to close write end in the parent process. -/// -/// - `worker_info`: Info about the worker. -/// -/// - `job_pid`: The child pid. -/// -/// - `usage_before`: Resource usage statistics before executing the child process. -/// -/// - `timeout`: The maximum allowed time for the child process to finish. -/// /// # Returns /// /// - The response, either `Ok` or some error state. diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index c93935780f4e..d0b21f21a9b8 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -18,11 +18,6 @@ mod memory_stats; -use polkadot_node_core_pvf_common::{ - executor_interface::{prepare, prevalidate}, - worker::{pipe2_cloexec, PipeFd, WorkerInfo}, -}; - // NOTE: Initializing logging in e.g. tests will not have an effect in the workers, as they are // separate spawned processes. Run with e.g. `RUST_LOG=parachain::pvf-prepare-worker=trace`. const LOG_TARGET: &str = "parachain::pvf-prepare-worker"; @@ -32,15 +27,17 @@ use crate::memory_stats::max_rss_stat::{extract_max_rss_stat, get_max_rss_thread #[cfg(any(target_os = "linux", feature = "jemalloc-allocator"))] use crate::memory_stats::memory_tracker::{get_memory_tracker_loop_stats, memory_tracker_loop}; use libc; -#[cfg(not(target_os = "linux"))] -use nix::unistd::ForkResult; use nix::{ errno::Errno, sys::{ resource::{Usage, UsageWho}, wait::WaitStatus, }, - unistd::Pid, + unistd::{ForkResult, Pid}, +}; +use polkadot_node_core_pvf_common::{ + executor_interface::{prepare, prevalidate}, + worker::{pipe2_cloexec, PipeFd, WorkerInfo}, }; use parity_scale_codec::{Decode, Encode}; @@ -65,7 +62,7 @@ use std::{ fd::{AsRawFd, RawFd}, unix::net::UnixStream, }, - path::PathBuf, + path::{Path, PathBuf}, process, sync::{mpsc::channel, Arc}, time::Duration, @@ -88,7 +85,7 @@ static ALLOC: TrackingAllocator = TrackingAllocator(std::all /// 3 - Prepare thread /// /// NOTE: The correctness of this value is enforced by a test. If the number of threads inside -/// the child process change in the future, this value must be changed as well. +/// the child process changes in the future, this value must be changed as well. pub const PREPARE_WORKER_THREAD_NUMBER: u32 = 4; /// Contains the bytes for a successfully compiled artifact. @@ -215,7 +212,7 @@ pub fn worker_entrypoint( worker_dir_path, node_version, worker_version, - |mut stream, worker_info| { + |mut stream, worker_info, security_status| { let temp_artifact_dest = worker_dir::prepare_tmp_artifact(&worker_info.worker_dir_path); loop { @@ -223,6 +220,7 @@ pub fn worker_entrypoint( gum::debug!( target: LOG_TARGET, ?worker_info, + ?security_status, "worker: preparing artifact", ); @@ -230,7 +228,7 @@ pub fn worker_entrypoint( let prepare_job_kind = pvf.prep_kind(); let executor_params = pvf.executor_params(); - let (pipe_reader, pipe_writer) = pipe2_cloexec()?; + let (pipe_read_fd, pipe_write_fd) = pipe2_cloexec()?; let usage_before = match nix::sys::resource::getrusage(UsageWho::RUSAGE_CHILDREN) { Ok(usage) => usage, @@ -245,74 +243,49 @@ pub fn worker_entrypoint( cfg_if::cfg_if! { if #[cfg(target_os = "linux")] { - use polkadot_node_core_pvf_common::worker::security; - - // 2MiB * num_threads - let stack_size = 2 * 1024 * 1024 * PREPARE_WORKER_THREAD_NUMBER; - - // SAFETY: new process is spawned within a single threaded process. This invariant - // is enforced by tests. Stack size being specified to ensure child doesn't overflow - let result = match unsafe { - security::clone::clone_on_worker( + let result = if security_status.can_do_secure_clone { + handle_clone( + &pvf, + pipe_write_fd, + pipe_read_fd, + stream_fd, + preparation_timeout, + prepare_job_kind, + &executor_params, worker_info, - Box::new(|| { - handle_child_process( - pvf.clone(), - pipe_writer, - pipe_reader, - stream_fd, - preparation_timeout, - prepare_job_kind, - Arc::clone(&executor_params), - ) - }), - stack_size as usize, + &temp_artifact_dest, + usage_before, + ) + } else { + // Fall back to using fork. + handle_fork( + &pvf, + pipe_write_fd, + pipe_read_fd, + stream_fd, + preparation_timeout, + prepare_job_kind, + &executor_params, + worker_info, + &temp_artifact_dest, + usage_before, ) - } { - Ok(child) => { - handle_parent_process( - pipe_reader, - pipe_writer, - worker_info, - child, - temp_artifact_dest.clone(), - usage_before, - preparation_timeout, - ) - }, - Err(security::clone::Error::Clone(errno)) => Err(error_from_errno("clone", errno)), }; } else { - // SAFETY: new process is spawned within a single threaded process. This invariant - // is enforced by tests. - let result = match unsafe { nix::unistd::fork() } { - Err(errno) => Err(error_from_errno("fork", errno)), - Ok(ForkResult::Child) => { - - handle_child_process( - pvf, - pipe_writer, - pipe_reader, - stream_fd, - preparation_timeout, - prepare_job_kind, - executor_params, - ) - }, - Ok(ForkResult::Parent { child }) => { - handle_parent_process( - pipe_reader, - pipe_writer, - worker_info, - child, - temp_artifact_dest.clone(), - usage_before, - preparation_timeout, - ) - }, - }; + let result = handle_fork( + &pvf, + pipe_write_fd, + pipe_read_fd, + stream_fd, + preparation_timeout, + prepare_job_kind, + &executor_params, + worker_info, + &temp_artifact_dest, + usage_before, + ); } - }; + } gum::trace!( target: LOG_TARGET, @@ -356,27 +329,96 @@ struct JobResponse { memory_stats: MemoryStats, } +#[cfg(target_os = "linux")] +fn handle_clone( + pvf: &PvfPrepData, + pipe_write_fd: i32, + pipe_read_fd: i32, + stream_fd: i32, + preparation_timeout: Duration, + prepare_job_kind: PrepareJobKind, + executor_params: &Arc, + worker_info: &WorkerInfo, + temp_artifact_dest: &Path, + usage_before: Usage, +) -> Result { + use polkadot_node_core_pvf_common::worker::security; + + // 2MiB * num_threads + let stack_size = 2 * 1024 * 1024 * PREPARE_WORKER_THREAD_NUMBER; + + // SAFETY: new process is spawned within a single threaded process. This invariant + // is enforced by tests. Stack size being specified to ensure child doesn't overflow + match unsafe { + security::clone::clone_on_worker( + worker_info, + Box::new(|| { + handle_child_process( + pvf.clone(), + pipe_write_fd, + pipe_read_fd, + stream_fd, + preparation_timeout, + prepare_job_kind, + Arc::clone(&executor_params), + ) + }), + stack_size as usize, + ) + } { + Ok(child) => handle_parent_process( + pipe_read_fd, + pipe_write_fd, + worker_info, + child, + temp_artifact_dest, + usage_before, + preparation_timeout, + ), + Err(security::clone::Error::Clone(errno)) => Err(error_from_errno("clone", errno)), + } +} + +fn handle_fork( + pvf: &PvfPrepData, + pipe_write_fd: i32, + pipe_read_fd: i32, + stream_fd: i32, + preparation_timeout: Duration, + prepare_job_kind: PrepareJobKind, + executor_params: &Arc, + worker_info: &WorkerInfo, + temp_artifact_dest: &Path, + usage_before: Usage, +) -> Result { + // SAFETY: new process is spawned within a single threaded process. This invariant + // is enforced by tests. + match unsafe { nix::unistd::fork() } { + Ok(ForkResult::Child) => handle_child_process( + pvf.clone(), + pipe_write_fd, + pipe_read_fd, + stream_fd, + preparation_timeout, + prepare_job_kind, + Arc::clone(executor_params), + ), + Ok(ForkResult::Parent { child }) => handle_parent_process( + pipe_read_fd, + pipe_write_fd, + worker_info, + child, + temp_artifact_dest, + usage_before, + preparation_timeout, + ), + Err(errno) => Err(error_from_errno("fork", errno)), + } +} + /// This is used to handle child process during pvf prepare worker. /// It prepares the artifact and tracks memory stats during preparation -/// and pipes back the response to the parent process -/// -/// # Arguments -/// -/// - `pvf`: `PvfPrepData` structure, containing data to prepare the artifact -/// -/// - `pipe_write_fd`: A `i32`, refers to pipe write end file descriptor. -/// -/// - `pipe_read_fd`: A `i32`, refers to pipe read end file descriptor, used to close the file -/// descriptor in the child process. -/// -/// - `stream_fd`: A `i32`, refers to UnixStream file descriptor, used to close the file descriptor -/// in the child process. -/// -/// - `preparation_timeout`: The timeout in `Duration`. -/// -/// - `prepare_job_kind`: The kind of prepare job. -/// -/// - `executor_params`: Deterministically serialized execution environment semantics. +/// and pipes back the response to the parent process. /// /// # Returns /// @@ -560,22 +602,6 @@ fn handle_child_process( /// Waits for child process to finish and handle child response from pipe. /// -/// # Parameters -/// -/// - `pipe_read_fd`: Refers to pipe read end, used to read data from the child process. -/// -/// - `pipe_write_fd`: Refers to pipe write end, used to close write end in the parent process. -/// -/// - `worker_info`: Info about the worker. -/// -/// - `job_pid`: The child pid. -/// -/// - `temp_artifact_dest`: The destination `PathBuf` to write the temporary artifact file. -/// -/// - `usage_before`: Resource usage statistics before executing the child process. -/// -/// - `timeout`: The maximum allowed time for the child process to finish. -/// /// # Returns /// /// - If the child send response without an error, this function returns `Ok(PrepareStats)` @@ -589,7 +615,7 @@ fn handle_parent_process( pipe_write_fd: i32, worker_info: &WorkerInfo, job_pid: Pid, - temp_artifact_dest: PathBuf, + temp_artifact_dest: &Path, usage_before: Usage, timeout: Duration, ) -> Result { @@ -599,7 +625,7 @@ fn handle_parent_process( return Err(error_from_errno("closing pipe write fd", errno)); }; - // SAFETY: pipe_read_fd is an open and owned file descriptor at this point. + // SAFETY: this is an open and owned file descriptor at this point. let mut pipe_read = unsafe { PipeFd::new(pipe_read_fd) }; // Read from the child. Don't decode unless the process exited normally, which we check later. @@ -670,7 +696,7 @@ fn handle_parent_process( temp_artifact_dest.display(), ); // Write to the temp file created by the host. - if let Err(err) = fs::write(&temp_artifact_dest, &artifact) { + if let Err(err) = fs::write(temp_artifact_dest, &artifact) { return Err(PrepareError::IoErr(err.to_string())) }; From ab87bca24555ac30409fed2ee869960c669c1a80 Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Mon, 1 Jan 2024 20:46:33 +0100 Subject: [PATCH 38/50] Fix compile error --- .../node/core/pvf/common/src/worker/mod.rs | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/polkadot/node/core/pvf/common/src/worker/mod.rs b/polkadot/node/core/pvf/common/src/worker/mod.rs index 938f31eed44a..da0134e64fca 100644 --- a/polkadot/node/core/pvf/common/src/worker/mod.rs +++ b/polkadot/node/core/pvf/common/src/worker/mod.rs @@ -63,22 +63,22 @@ macro_rules! decl_worker_main { let args = std::env::args().collect::>(); if args.len() == 1 { print_help($expected_command); - return + return; } match args[1].as_ref() { "--help" | "-h" => { print_help($expected_command); - return + return; }, "--version" | "-v" => { println!("{}", $worker_version); - return + return; }, // Useful for debugging. --version is used for version checks. "--full-version" => { println!("{}", get_full_version()); - return + return; }, "--check-can-enable-landlock" => { @@ -142,7 +142,7 @@ macro_rules! decl_worker_main { "test-sleep" => { std::thread::sleep(std::time::Duration::from_secs(5)); - return + return; }, subcommand => { @@ -345,12 +345,12 @@ pub fn run_worker( } // Make sure that we can read the worker dir path, and log its contents. - let entries = std::fs::read_dir(&worker_info.worker_dir_path)? - .map(|res| res.map(|e| e.file_name())) - .collect(); + let entries: io::Result> = std::fs::read_dir(&worker_info.worker_dir_path) + .and_then(|d| d.map(|res| res.map(|e| e.file_name())).collect()); match entries { - Ok(entries) => - gum::trace!(target: LOG_TARGET, ?worker_info, "content of worker dir: {:?}", entries), + Ok(entries) => { + gum::trace!(target: LOG_TARGET, ?worker_info, "content of worker dir: {:?}", entries) + }, Err(err) => { let err = format!("Could not read worker dir: {}", err.to_string()); worker_shutdown_error(worker_info, &err); @@ -498,7 +498,7 @@ pub fn cpu_time_monitor_loop( } } - return Some(cpu_time_elapsed) + return Some(cpu_time_elapsed); } } @@ -642,7 +642,7 @@ pub mod thread { let mut flag = lock.lock().unwrap(); if !flag.is_pending() { // Someone else already triggered the condvar. - return + return; } *flag = outcome; cvar.notify_all(); From 48942cd3b6d93aba47389e6f9ac3b8758e3ba613 Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Mon, 1 Jan 2024 20:57:38 +0100 Subject: [PATCH 39/50] fix formatting; fix error due to merge --- .../node/core/pvf/common/src/worker/mod.rs | 27 +++++++++---------- .../node/core/pvf/execute-worker/src/lib.rs | 5 +++- .../node/core/pvf/prepare-worker/src/lib.rs | 2 +- 3 files changed, 18 insertions(+), 16 deletions(-) diff --git a/polkadot/node/core/pvf/common/src/worker/mod.rs b/polkadot/node/core/pvf/common/src/worker/mod.rs index da0134e64fca..49aebef4f3e5 100644 --- a/polkadot/node/core/pvf/common/src/worker/mod.rs +++ b/polkadot/node/core/pvf/common/src/worker/mod.rs @@ -63,22 +63,22 @@ macro_rules! decl_worker_main { let args = std::env::args().collect::>(); if args.len() == 1 { print_help($expected_command); - return; + return } match args[1].as_ref() { "--help" | "-h" => { print_help($expected_command); - return; + return }, "--version" | "-v" => { println!("{}", $worker_version); - return; + return }, // Useful for debugging. --version is used for version checks. "--full-version" => { println!("{}", get_full_version()); - return; + return }, "--check-can-enable-landlock" => { @@ -142,7 +142,7 @@ macro_rules! decl_worker_main { "test-sleep" => { std::thread::sleep(std::time::Duration::from_secs(5)); - return; + return }, subcommand => { @@ -198,7 +198,7 @@ pub fn pipe2_cloexec() -> io::Result<(libc::c_int, libc::c_int)> { let mut fds: [libc::c_int; 2] = [0; 2]; let res = unsafe { libc::pipe2(fds.as_mut_ptr(), libc::O_CLOEXEC) }; if res != 0 { - return Err(io::Error::last_os_error()); + return Err(io::Error::last_os_error()) } Ok((fds[0], fds[1])) } @@ -208,15 +208,15 @@ pub fn pipe2_cloexec() -> io::Result<(libc::c_int, libc::c_int)> { let mut fds: [libc::c_int; 2] = [0; 2]; let res = unsafe { libc::pipe(fds.as_mut_ptr()) }; if res != 0 { - return Err(io::Error::last_os_error()); + return Err(io::Error::last_os_error()) } let res = unsafe { libc::fcntl(fds[0], libc::F_SETFD, libc::FD_CLOEXEC) }; if res != 0 { - return Err(io::Error::last_os_error()); + return Err(io::Error::last_os_error()) } let res = unsafe { libc::fcntl(fds[1], libc::F_SETFD, libc::FD_CLOEXEC) }; if res != 0 { - return Err(io::Error::last_os_error()); + return Err(io::Error::last_os_error()) } Ok((fds[0], fds[1])) } @@ -348,9 +348,8 @@ pub fn run_worker( let entries: io::Result> = std::fs::read_dir(&worker_info.worker_dir_path) .and_then(|d| d.map(|res| res.map(|e| e.file_name())).collect()); match entries { - Ok(entries) => { - gum::trace!(target: LOG_TARGET, ?worker_info, "content of worker dir: {:?}", entries) - }, + Ok(entries) => + gum::trace!(target: LOG_TARGET, ?worker_info, "content of worker dir: {:?}", entries), Err(err) => { let err = format!("Could not read worker dir: {}", err.to_string()); worker_shutdown_error(worker_info, &err); @@ -498,7 +497,7 @@ pub fn cpu_time_monitor_loop( } } - return Some(cpu_time_elapsed); + return Some(cpu_time_elapsed) } } @@ -642,7 +641,7 @@ pub mod thread { let mut flag = lock.lock().unwrap(); if !flag.is_pending() { // Someone else already triggered the condvar. - return; + return } *flag = outcome; cvar.notify_all(); diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index 33083e298e49..ee7418335129 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -359,7 +359,10 @@ fn handle_child_process( // and most of the time the job process should terminate on its own when it completes. #[cfg(target_os = "linux")] nix::sys::prctl::set_pdeathsig(nix::sys::signal::Signal::SIGTERM).unwrap_or_else(|err| { - send_child_response(&mut pipe_write, Err(JobError::CouldNotSetPdeathsig(err.to_string()))) + send_child_response( + &mut pipe_write_fd, + Err(JobError::CouldNotSetPdeathsig(err.to_string())), + ) }); // SAFETY: this is an open and owned file descriptor at this point. diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index 38fab364ba41..d8eb6162b4f4 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -440,7 +440,7 @@ fn handle_child_process( // and most of the time the job process should terminate on its own when it completes. #[cfg(target_os = "linux")] nix::sys::prctl::set_pdeathsig(nix::sys::signal::Signal::SIGTERM).unwrap_or_else(|err| { - send_child_response(&mut pipe_write, Err(PrepareError::IoErr(err.to_string()))) + send_child_response(&mut pipe_write_fd, Err(PrepareError::IoErr(err.to_string()))) }); // SAFETY: pipe_writer is an open and owned file descriptor at this point. From 08153fb5596d2495209a0137ca0c287f2ebbe80a Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Tue, 2 Jan 2024 19:13:39 +0100 Subject: [PATCH 40/50] Fix compile errors --- polkadot/node/core/pvf/execute-worker/src/lib.rs | 11 ++++------- polkadot/node/core/pvf/prepare-worker/src/lib.rs | 8 ++++---- 2 files changed, 8 insertions(+), 11 deletions(-) diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index ee7418335129..12e82f0d8ce9 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -353,21 +353,18 @@ fn handle_child_process( execution_timeout: Duration, execute_thread_stack_size: usize, ) -> ! { + // SAFETY: this is an open and owned file descriptor at this point. + let mut pipe_write = unsafe { PipeFd::new(pipe_write_fd) }; + // Terminate if the parent thread dies. Parent thread == worker process (it is single-threaded). // // RACE: the worker may die before we install the death signal. In practice this is unlikely, // and most of the time the job process should terminate on its own when it completes. #[cfg(target_os = "linux")] nix::sys::prctl::set_pdeathsig(nix::sys::signal::Signal::SIGTERM).unwrap_or_else(|err| { - send_child_response( - &mut pipe_write_fd, - Err(JobError::CouldNotSetPdeathsig(err.to_string())), - ) + send_child_response(&mut pipe_write, Err(JobError::CouldNotSetPdeathsig(err.to_string()))) }); - // SAFETY: this is an open and owned file descriptor at this point. - let mut pipe_write = unsafe { PipeFd::new(pipe_write_fd) }; - // Drop the read end so we don't have too many FDs open. if let Err(errno) = nix::unistd::close(pipe_read_fd) { send_child_response(&mut pipe_write, job_error_from_errno("closing pipe", errno)); diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index d8eb6162b4f4..5f1bf78ad3fa 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -434,18 +434,18 @@ fn handle_child_process( prepare_job_kind: PrepareJobKind, executor_params: Arc, ) -> ! { + // SAFETY: pipe_writer is an open and owned file descriptor at this point. + let mut pipe_write = unsafe { PipeFd::new(pipe_write_fd) }; + // Terminate if the parent thread dies. Parent thread == worker process (it is single-threaded). // // RACE: the worker may die before we install the death signal. In practice this is unlikely, // and most of the time the job process should terminate on its own when it completes. #[cfg(target_os = "linux")] nix::sys::prctl::set_pdeathsig(nix::sys::signal::Signal::SIGTERM).unwrap_or_else(|err| { - send_child_response(&mut pipe_write_fd, Err(PrepareError::IoErr(err.to_string()))) + send_child_response(&mut pipe_write, Err(PrepareError::IoErr(err.to_string()))) }); - // SAFETY: pipe_writer is an open and owned file descriptor at this point. - let mut pipe_write = unsafe { PipeFd::new(pipe_write_fd) }; - // Drop the read end so we don't have too many FDs open. if let Err(errno) = nix::unistd::close(pipe_read_fd) { send_child_response( From 422db3f2886f93eaf2ec274ca4b2dce1b2071bce Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Tue, 2 Jan 2024 19:31:22 +0100 Subject: [PATCH 41/50] remove pdeathsig and associated tests --- polkadot/node/core/pvf/execute-worker/src/lib.rs | 9 --------- polkadot/node/core/pvf/prepare-worker/src/lib.rs | 9 --------- polkadot/node/core/pvf/tests/it/process.rs | 5 ----- 3 files changed, 23 deletions(-) diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index 12e82f0d8ce9..8a00247c7be3 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -356,15 +356,6 @@ fn handle_child_process( // SAFETY: this is an open and owned file descriptor at this point. let mut pipe_write = unsafe { PipeFd::new(pipe_write_fd) }; - // Terminate if the parent thread dies. Parent thread == worker process (it is single-threaded). - // - // RACE: the worker may die before we install the death signal. In practice this is unlikely, - // and most of the time the job process should terminate on its own when it completes. - #[cfg(target_os = "linux")] - nix::sys::prctl::set_pdeathsig(nix::sys::signal::Signal::SIGTERM).unwrap_or_else(|err| { - send_child_response(&mut pipe_write, Err(JobError::CouldNotSetPdeathsig(err.to_string()))) - }); - // Drop the read end so we don't have too many FDs open. if let Err(errno) = nix::unistd::close(pipe_read_fd) { send_child_response(&mut pipe_write, job_error_from_errno("closing pipe", errno)); diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index 5f1bf78ad3fa..d0b21f21a9b8 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -437,15 +437,6 @@ fn handle_child_process( // SAFETY: pipe_writer is an open and owned file descriptor at this point. let mut pipe_write = unsafe { PipeFd::new(pipe_write_fd) }; - // Terminate if the parent thread dies. Parent thread == worker process (it is single-threaded). - // - // RACE: the worker may die before we install the death signal. In practice this is unlikely, - // and most of the time the job process should terminate on its own when it completes. - #[cfg(target_os = "linux")] - nix::sys::prctl::set_pdeathsig(nix::sys::signal::Signal::SIGTERM).unwrap_or_else(|err| { - send_child_response(&mut pipe_write, Err(PrepareError::IoErr(err.to_string()))) - }); - // Drop the read end so we don't have too many FDs open. if let Err(errno) = nix::unistd::close(pipe_read_fd) { send_child_response( diff --git a/polkadot/node/core/pvf/tests/it/process.rs b/polkadot/node/core/pvf/tests/it/process.rs index 424ca2e5f650..c53178d685b7 100644 --- a/polkadot/node/core/pvf/tests/it/process.rs +++ b/polkadot/node/core/pvf/tests/it/process.rs @@ -115,11 +115,6 @@ where // Sleep to give processes a chance to get cleaned up, preventing races in the next step. tokio::time::sleep(Duration::from_millis(500)).await; - - // Make sure job processes got cleaned up. Pass `is_direct_child: false` to target the - // job processes. - assert!(find_process_by_sid_and_name(sid, PREPARE_PROCESS_NAME, false).is_none()); - assert!(find_process_by_sid_and_name(sid, EXECUTE_PROCESS_NAME, false).is_none()); }); } From fa7fdf882fb280874eb599548a89e43d32f4226c Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Tue, 2 Jan 2024 19:35:25 +0100 Subject: [PATCH 42/50] A bit more cleanup --- polkadot/node/core/pvf/tests/it/process.rs | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/polkadot/node/core/pvf/tests/it/process.rs b/polkadot/node/core/pvf/tests/it/process.rs index c53178d685b7..e989eb874ba9 100644 --- a/polkadot/node/core/pvf/tests/it/process.rs +++ b/polkadot/node/core/pvf/tests/it/process.rs @@ -94,7 +94,7 @@ fn find_process_by_sid_and_name( found } -/// Sets up the test and makes sure everything gets cleaned up after. +/// Sets up the test. /// /// We run the runtime manually because `#[tokio::test]` doesn't work in `rusty_fork_test!`. fn test_wrapper(f: F) @@ -112,9 +112,6 @@ where // Pass a clone of the host so that it does not get dropped after. f(host.clone(), sid).await; - - // Sleep to give processes a chance to get cleaned up, preventing races in the next step. - tokio::time::sleep(Duration::from_millis(500)).await; }); } @@ -122,7 +119,7 @@ where // then finding the child process that matches the session ID and expected process name and doing // something with that child. rusty_fork_test! { - // Everything succeeded. All created subprocesses for jobs should get cleaned up, to avoid memory leaks. + // Everything succeeds. #[test] fn successful_prepare_and_validate() { test_wrapper(|host, _sid| async move { From c138f8fc93fd24373e6a9ef286e2ed8b3b9f46c0 Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Tue, 2 Jan 2024 19:44:53 +0100 Subject: [PATCH 43/50] Remove more unused code --- polkadot/node/core/pvf/common/src/execute.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/polkadot/node/core/pvf/common/src/execute.rs b/polkadot/node/core/pvf/common/src/execute.rs index 52fd16e97aa1..6b3becf524d7 100644 --- a/polkadot/node/core/pvf/common/src/execute.rs +++ b/polkadot/node/core/pvf/common/src/execute.rs @@ -99,6 +99,4 @@ pub enum JobError { CouldNotSpawnThread(String), #[error("An error occurred in the CPU time monitor thread: {0}")] CpuTimeMonitorThread(String), - #[error("Could not set pdeathsig: {0}")] - CouldNotSetPdeathsig(String), } From 69713a718566a753ba2735cd9810546f6c0eb681 Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Tue, 2 Jan 2024 20:21:28 +0100 Subject: [PATCH 44/50] Update syscall lists --- polkadot/scripts/list-syscalls/execute-worker-syscalls | 2 ++ polkadot/scripts/list-syscalls/prepare-worker-syscalls | 2 ++ 2 files changed, 4 insertions(+) diff --git a/polkadot/scripts/list-syscalls/execute-worker-syscalls b/polkadot/scripts/list-syscalls/execute-worker-syscalls index c864d5ee7259..349af783cf1a 100644 --- a/polkadot/scripts/list-syscalls/execute-worker-syscalls +++ b/polkadot/scripts/list-syscalls/execute-worker-syscalls @@ -26,6 +26,7 @@ 45 (recvfrom) 46 (sendmsg) 56 (clone) +57 (fork) 60 (exit) 61 (wait4) 62 (kill) @@ -50,6 +51,7 @@ 158 (arch_prctl) 165 (mount) 166 (umount2) +186 (gettid) 200 (tkill) 202 (futex) 204 (sched_getaffinity) diff --git a/polkadot/scripts/list-syscalls/prepare-worker-syscalls b/polkadot/scripts/list-syscalls/prepare-worker-syscalls index 3ecf5b589e6a..05281b61591a 100644 --- a/polkadot/scripts/list-syscalls/prepare-worker-syscalls +++ b/polkadot/scripts/list-syscalls/prepare-worker-syscalls @@ -26,6 +26,7 @@ 45 (recvfrom) 46 (sendmsg) 56 (clone) +57 (fork) 60 (exit) 61 (wait4) 62 (kill) @@ -50,6 +51,7 @@ 158 (arch_prctl) 165 (mount) 166 (umount2) +186 (gettid) 200 (tkill) 202 (futex) 203 (sched_setaffinity) From 4b66d38a36c2d74598e86a02cd2a25d4b7957327 Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Mon, 8 Jan 2024 20:00:25 +0100 Subject: [PATCH 45/50] Parametrize clone_flags() on unshare security capability --- .../pvf/common/src/worker/security/clone.rs | 17 +++++++----- .../node/core/pvf/execute-worker/src/lib.rs | 3 +++ .../node/core/pvf/prepare-worker/src/lib.rs | 3 +++ polkadot/node/core/pvf/tests/it/main.rs | 27 +++++++++++++++++++ 4 files changed, 43 insertions(+), 7 deletions(-) diff --git a/polkadot/node/core/pvf/common/src/worker/security/clone.rs b/polkadot/node/core/pvf/common/src/worker/security/clone.rs index 003b07ab4464..ee4276bdacdc 100644 --- a/polkadot/node/core/pvf/common/src/worker/security/clone.rs +++ b/polkadot/node/core/pvf/common/src/worker/security/clone.rs @@ -38,10 +38,11 @@ pub type Result = std::result::Result; /// async-signal-safe functions. pub unsafe fn clone_on_worker( worker_info: &WorkerInfo, + have_unshare_newuser: bool, cb: CloneCb, stack_size: usize, ) -> Result { - let flags = clone_flags(); + let flags = clone_flags(have_unshare_newuser); gum::trace!( target: LOG_TARGET, @@ -61,7 +62,7 @@ pub unsafe fn clone_on_worker( /// async-signal-safe functions. pub unsafe fn check_can_fully_clone() -> Result<()> { let stack_size = 2 * 1024 * 1024; // Use same as prepare worker for this check. - try_clone(Box::new(|| 0), stack_size, clone_flags()).map(|_pid| ()) + try_clone(Box::new(|| 0), stack_size, clone_flags(false)).map(|_pid| ()) } /// Runs clone(2) with all sandboxing flags. @@ -75,14 +76,16 @@ unsafe fn try_clone(cb: CloneCb, stack_size: usize, flags: CloneFlags) -> Result } /// Returns flags for `clone(2)`, including all the sandbox-related ones. -fn clone_flags() -> CloneFlags { - // NOTE: CLONE_NEWUSER does not work when cloning job processes, but in Secure Validator Mode it - // is already set by the worker. - // +fn clone_flags(have_unshare_newuser: bool) -> CloneFlags { + // NOTE: CLONE_NEWUSER does not work in `clone` if we previously called `unshare` with this + // flag. + let maybe_clone_newuser = + if have_unshare_newuser { CloneFlags::empty() } else { CloneFlags::CLONE_NEWUSER }; // SIGCHLD flag is used to inform clone that the parent process is // expecting a child termination signal, without this flag `waitpid` function // return `ECHILD` error. - CloneFlags::CLONE_NEWCGROUP | + maybe_clone_newuser | + CloneFlags::CLONE_NEWCGROUP | CloneFlags::CLONE_NEWIPC | CloneFlags::CLONE_NEWNET | CloneFlags::CLONE_NEWNS | diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index 8a00247c7be3..db18498254c2 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -180,6 +180,7 @@ pub fn worker_entrypoint( execute_thread_stack_size, clone_stack_size, worker_info, + security_status.can_unshare_user_namespace_and_change_root, usage_before, )? } else { @@ -264,6 +265,7 @@ fn handle_clone( execute_stack_size: usize, clone_stack_size: usize, worker_info: &WorkerInfo, + have_unshare_newuser: bool, usage_before: Usage, ) -> io::Result { use polkadot_node_core_pvf_common::worker::security; @@ -273,6 +275,7 @@ fn handle_clone( match unsafe { security::clone::clone_on_worker( worker_info, + have_unshare_newuser, Box::new(|| { handle_child_process( pipe_write_fd, diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index d0b21f21a9b8..8e52f9745f04 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -253,6 +253,7 @@ pub fn worker_entrypoint( prepare_job_kind, &executor_params, worker_info, + security_status.can_unshare_user_namespace_and_change_root, &temp_artifact_dest, usage_before, ) @@ -339,6 +340,7 @@ fn handle_clone( prepare_job_kind: PrepareJobKind, executor_params: &Arc, worker_info: &WorkerInfo, + have_unshare_newuser: bool, temp_artifact_dest: &Path, usage_before: Usage, ) -> Result { @@ -352,6 +354,7 @@ fn handle_clone( match unsafe { security::clone::clone_on_worker( worker_info, + have_unshare_newuser, Box::new(|| { handle_child_process( pvf.clone(), diff --git a/polkadot/node/core/pvf/tests/it/main.rs b/polkadot/node/core/pvf/tests/it/main.rs index df900618b62b..5302f0ea5194 100644 --- a/polkadot/node/core/pvf/tests/it/main.rs +++ b/polkadot/node/core/pvf/tests/it/main.rs @@ -131,6 +131,15 @@ impl TestHost { async fn security_status(&self) -> SecurityStatus { self.host.lock().await.security_status.clone() } + + #[cfg(all(feature = "ci-only-tests", target_os = "linux"))] + async fn mutate_security_status(&self, f: F) + where + F: FnOnce(&mut SecurityStatus), + { + let mut host = self.host.lock().await; + f(&mut host.security_status); + } } #[tokio::test] @@ -449,6 +458,24 @@ async fn all_security_features_work() { ); } +// Usually both `unshare` and `clone` security capabilities are present. Test what happens when +// `unshare` is not. +#[cfg(all(feature = "ci-only-tests", target_os = "linux"))] +#[tokio::test] +async fn clone_works_without_unshare() { + let host = TestHost::new().await; + + assert!(host.security_status().await.can_unshare_user_namespace_and_change_root); + assert!(host.security_status().await.can_do_secure_clone); + host.mutate_security_status(|status| status.can_unshare_user_namespace_and_change_root = false) + .await; + + let _stats = host + .precheck_pvf(::adder::wasm_binary_unwrap(), Default::default()) + .await + .unwrap(); +} + // Regression test to make sure the unshare-pivot-root capability does not depend on the PVF // artifacts cache existing. #[cfg(all(feature = "ci-only-tests", target_os = "linux"))] From dcbb90bd8e569ef8b3460295e2cbd8b28cb903e4 Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Mon, 8 Jan 2024 20:06:24 +0100 Subject: [PATCH 46/50] Clarify comment --- polkadot/node/core/pvf/common/src/worker/security/clone.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/polkadot/node/core/pvf/common/src/worker/security/clone.rs b/polkadot/node/core/pvf/common/src/worker/security/clone.rs index ee4276bdacdc..d0e30014fee3 100644 --- a/polkadot/node/core/pvf/common/src/worker/security/clone.rs +++ b/polkadot/node/core/pvf/common/src/worker/security/clone.rs @@ -78,7 +78,8 @@ unsafe fn try_clone(cb: CloneCb, stack_size: usize, flags: CloneFlags) -> Result /// Returns flags for `clone(2)`, including all the sandbox-related ones. fn clone_flags(have_unshare_newuser: bool) -> CloneFlags { // NOTE: CLONE_NEWUSER does not work in `clone` if we previously called `unshare` with this - // flag. + // flag. On the other hand, if we did not call `unshare` we need this flag for the CAP_SYS_ADMIN + // capability. let maybe_clone_newuser = if have_unshare_newuser { CloneFlags::empty() } else { CloneFlags::CLONE_NEWUSER }; // SIGCHLD flag is used to inform clone that the parent process is From ad62e9ea7d60c060443373785f1ba88f4c57a6c8 Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Mon, 8 Jan 2024 20:23:09 +0100 Subject: [PATCH 47/50] Remove test, it didn't update security status correctly I've tested this manually so should be fine --- polkadot/node/core/pvf/tests/it/main.rs | 27 ------------------------- 1 file changed, 27 deletions(-) diff --git a/polkadot/node/core/pvf/tests/it/main.rs b/polkadot/node/core/pvf/tests/it/main.rs index 5302f0ea5194..df900618b62b 100644 --- a/polkadot/node/core/pvf/tests/it/main.rs +++ b/polkadot/node/core/pvf/tests/it/main.rs @@ -131,15 +131,6 @@ impl TestHost { async fn security_status(&self) -> SecurityStatus { self.host.lock().await.security_status.clone() } - - #[cfg(all(feature = "ci-only-tests", target_os = "linux"))] - async fn mutate_security_status(&self, f: F) - where - F: FnOnce(&mut SecurityStatus), - { - let mut host = self.host.lock().await; - f(&mut host.security_status); - } } #[tokio::test] @@ -458,24 +449,6 @@ async fn all_security_features_work() { ); } -// Usually both `unshare` and `clone` security capabilities are present. Test what happens when -// `unshare` is not. -#[cfg(all(feature = "ci-only-tests", target_os = "linux"))] -#[tokio::test] -async fn clone_works_without_unshare() { - let host = TestHost::new().await; - - assert!(host.security_status().await.can_unshare_user_namespace_and_change_root); - assert!(host.security_status().await.can_do_secure_clone); - host.mutate_security_status(|status| status.can_unshare_user_namespace_and_change_root = false) - .await; - - let _stats = host - .precheck_pvf(::adder::wasm_binary_unwrap(), Default::default()) - .await - .unwrap(); -} - // Regression test to make sure the unshare-pivot-root capability does not depend on the PVF // artifacts cache existing. #[cfg(all(feature = "ci-only-tests", target_os = "linux"))] From 4217eb960132ecf7b5964fa950b274982622548c Mon Sep 17 00:00:00 2001 From: Joao pedro Santos Date: Tue, 9 Jan 2024 08:09:49 -0300 Subject: [PATCH 48/50] implement AsRawFd and FromRawFd for the PipeFd struct --- .../node/core/pvf/common/src/worker/mod.rs | 21 ++++++++++--------- .../node/core/pvf/execute-worker/src/lib.rs | 9 +++++--- .../node/core/pvf/prepare-worker/src/lib.rs | 6 +++--- 3 files changed, 20 insertions(+), 16 deletions(-) diff --git a/polkadot/node/core/pvf/common/src/worker/mod.rs b/polkadot/node/core/pvf/common/src/worker/mod.rs index 49aebef4f3e5..6fd8e48c5b2f 100644 --- a/polkadot/node/core/pvf/common/src/worker/mod.rs +++ b/polkadot/node/core/pvf/common/src/worker/mod.rs @@ -28,7 +28,7 @@ use std::{ fs::File, io::{self, Read, Write}, os::{ - fd::{AsRawFd, FromRawFd}, + fd::{AsRawFd, FromRawFd, RawFd}, unix::net::UnixStream, }, path::PathBuf, @@ -227,20 +227,21 @@ pub struct PipeFd { file: File, } -impl PipeFd { +impl AsRawFd for PipeFd { + /// Returns the raw file descriptor associated with this `PipeFd` + fn as_raw_fd(&self) -> RawFd { + self.file.as_raw_fd() + } +} + +impl FromRawFd for PipeFd { /// Creates a new `PipeFd` instance from a raw file descriptor. /// /// # Safety /// /// The fd passed in must be an owned file descriptor; in particular, it must be open. - pub unsafe fn new(fd: i32) -> Self { - let file = File::from_raw_fd(fd); - PipeFd { file } - } - - /// Returns the raw file descriptor associated with this `PipeFd` - pub fn as_raw_fd(&self) -> i32 { - self.file.as_raw_fd() + unsafe fn from_raw_fd(fd: RawFd) -> Self { + PipeFd { file: File::from_raw_fd(fd) } } } diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index db18498254c2..e614f6e39a06 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -48,7 +48,10 @@ use polkadot_parachain_primitives::primitives::ValidationResult; use polkadot_primitives::ExecutorParams; use std::{ io::{self, Read}, - os::{fd::AsRawFd, unix::net::UnixStream}, + os::{ + fd::{AsRawFd, FromRawFd}, + unix::net::UnixStream, + }, path::PathBuf, process, sync::{mpsc::channel, Arc}, @@ -357,7 +360,7 @@ fn handle_child_process( execute_thread_stack_size: usize, ) -> ! { // SAFETY: this is an open and owned file descriptor at this point. - let mut pipe_write = unsafe { PipeFd::new(pipe_write_fd) }; + let mut pipe_write = unsafe { PipeFd::from_raw_fd(pipe_write_fd) }; // Drop the read end so we don't have too many FDs open. if let Err(errno) = nix::unistd::close(pipe_read_fd) { @@ -490,7 +493,7 @@ fn handle_parent_process( }; // SAFETY: pipe_read_fd is an open and owned file descriptor at this point. - let mut pipe_read = unsafe { PipeFd::new(pipe_read_fd) }; + let mut pipe_read = unsafe { PipeFd::from_raw_fd(pipe_read_fd) }; // Read from the child. Don't decode unless the process exited normally, which we check later. let mut received_data = Vec::new(); diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index 8e52f9745f04..49095f639e81 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -59,7 +59,7 @@ use std::{ fs, io::{self, Read}, os::{ - fd::{AsRawFd, RawFd}, + fd::{AsRawFd, FromRawFd, RawFd}, unix::net::UnixStream, }, path::{Path, PathBuf}, @@ -438,7 +438,7 @@ fn handle_child_process( executor_params: Arc, ) -> ! { // SAFETY: pipe_writer is an open and owned file descriptor at this point. - let mut pipe_write = unsafe { PipeFd::new(pipe_write_fd) }; + let mut pipe_write = unsafe { PipeFd::from_raw_fd(pipe_write_fd) }; // Drop the read end so we don't have too many FDs open. if let Err(errno) = nix::unistd::close(pipe_read_fd) { @@ -629,7 +629,7 @@ fn handle_parent_process( }; // SAFETY: this is an open and owned file descriptor at this point. - let mut pipe_read = unsafe { PipeFd::new(pipe_read_fd) }; + let mut pipe_read = unsafe { PipeFd::from_raw_fd(pipe_read_fd) }; // Read from the child. Don't decode unless the process exited normally, which we check later. let mut received_data = Vec::new(); From 891d126286e64ced315be397ff3135aaca535698 Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Fri, 12 Jan 2024 15:49:12 +0100 Subject: [PATCH 49/50] Remove stack_size parameter --- .../core/pvf/common/src/worker/security/clone.rs | 11 ++++------- polkadot/node/core/pvf/execute-worker/src/lib.rs | 12 +++--------- polkadot/node/core/pvf/prepare-worker/src/lib.rs | 6 +----- 3 files changed, 8 insertions(+), 21 deletions(-) diff --git a/polkadot/node/core/pvf/common/src/worker/security/clone.rs b/polkadot/node/core/pvf/common/src/worker/security/clone.rs index d0e30014fee3..707f68d18591 100644 --- a/polkadot/node/core/pvf/common/src/worker/security/clone.rs +++ b/polkadot/node/core/pvf/common/src/worker/security/clone.rs @@ -40,19 +40,17 @@ pub unsafe fn clone_on_worker( worker_info: &WorkerInfo, have_unshare_newuser: bool, cb: CloneCb, - stack_size: usize, ) -> Result { let flags = clone_flags(have_unshare_newuser); gum::trace!( target: LOG_TARGET, ?worker_info, - %stack_size, "calling clone with flags: {:?}", flags ); - try_clone(cb, stack_size, flags) + try_clone(cb, flags) } /// Runs a check for clone(2) with all sandboxing flags and returns an error indicating whether it @@ -61,16 +59,15 @@ pub unsafe fn clone_on_worker( /// SAFETY: new process should be either spawned within a single threaded process, or use only /// async-signal-safe functions. pub unsafe fn check_can_fully_clone() -> Result<()> { - let stack_size = 2 * 1024 * 1024; // Use same as prepare worker for this check. - try_clone(Box::new(|| 0), stack_size, clone_flags(false)).map(|_pid| ()) + try_clone(Box::new(|| 0), clone_flags(false)).map(|_pid| ()) } /// Runs clone(2) with all sandboxing flags. /// /// SAFETY: new process should be either spawned within a single threaded process, or use only /// async-signal-safe functions. -unsafe fn try_clone(cb: CloneCb, stack_size: usize, flags: CloneFlags) -> Result { - let mut stack: Vec = vec![0u8; stack_size]; +unsafe fn try_clone(cb: CloneCb, flags: CloneFlags) -> Result { + let mut stack = [0u8; 2 * 1024 * 1024]; nix::sched::clone(cb, stack.as_mut_slice(), flags, None).map_err(|errno| Error::Clone(errno)) } diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs index e614f6e39a06..9dec3db24f07 100644 --- a/polkadot/node/core/pvf/execute-worker/src/lib.rs +++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs @@ -128,9 +128,7 @@ pub fn worker_entrypoint( let Handshake { executor_params } = recv_execute_handshake(&mut stream)?; let executor_params: Arc = Arc::new(executor_params); - let execute_thread_stack_size = get_max_stack_size(&executor_params, 1); - #[cfg(target_os = "linux")] - let clone_stack_size = get_max_stack_size(&executor_params, EXECUTE_WORKER_THREAD_NUMBER); + let execute_thread_stack_size = max_stack_size(&executor_params); loop { let (params, execution_timeout) = recv_request(&mut stream)?; @@ -181,7 +179,6 @@ pub fn worker_entrypoint( ¶ms, execution_timeout, execute_thread_stack_size, - clone_stack_size, worker_info, security_status.can_unshare_user_namespace_and_change_root, usage_before, @@ -266,7 +263,6 @@ fn handle_clone( params: &Arc>, execution_timeout: Duration, execute_stack_size: usize, - clone_stack_size: usize, worker_info: &WorkerInfo, have_unshare_newuser: bool, usage_before: Usage, @@ -291,7 +287,6 @@ fn handle_clone( execute_stack_size, ) }), - clone_stack_size, ) } { Ok(child) => handle_parent_process( @@ -467,10 +462,9 @@ fn handle_child_process( /// the desired stack limit. We must also make sure the job process has enough stack for *all* its /// threads. This function can be used to get the stack size of either the execute thread or execute /// job process. -fn get_max_stack_size(executor_params: &ExecutorParams, number_of_threads: u32) -> usize { +fn max_stack_size(executor_params: &ExecutorParams) -> usize { let (_sem, deterministic_stack_limit) = params_to_wasmtime_semantics(executor_params); - return (2 * 1024 * 1024 * number_of_threads + deterministic_stack_limit.native_stack_max) - as usize; + return (2 * 1024 * 1024 + deterministic_stack_limit.native_stack_max) as usize; } /// Waits for child process to finish and handle child response from pipe. diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs index 49095f639e81..82a56107ef53 100644 --- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs +++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs @@ -82,7 +82,7 @@ static ALLOC: TrackingAllocator = TrackingAllocator(std::all /// 1 - Main thread /// 2 - Cpu monitor thread /// 3 - Memory tracker thread -/// 3 - Prepare thread +/// 4 - Prepare thread /// /// NOTE: The correctness of this value is enforced by a test. If the number of threads inside /// the child process changes in the future, this value must be changed as well. @@ -346,9 +346,6 @@ fn handle_clone( ) -> Result { use polkadot_node_core_pvf_common::worker::security; - // 2MiB * num_threads - let stack_size = 2 * 1024 * 1024 * PREPARE_WORKER_THREAD_NUMBER; - // SAFETY: new process is spawned within a single threaded process. This invariant // is enforced by tests. Stack size being specified to ensure child doesn't overflow match unsafe { @@ -366,7 +363,6 @@ fn handle_clone( Arc::clone(&executor_params), ) }), - stack_size as usize, ) } { Ok(child) => handle_parent_process( From bb0a998ac5f1905ec9f2e9b7e5e37681d1cb5043 Mon Sep 17 00:00:00 2001 From: "Marcin S." Date: Fri, 12 Jan 2024 16:43:43 +0100 Subject: [PATCH 50/50] Do security checks with env vars cleared --- .../node/core/pvf/common/src/worker/mod.rs | 29 +-- polkadot/node/core/pvf/src/security.rs | 175 +++++++----------- 2 files changed, 81 insertions(+), 123 deletions(-) diff --git a/polkadot/node/core/pvf/common/src/worker/mod.rs b/polkadot/node/core/pvf/common/src/worker/mod.rs index 6fd8e48c5b2f..786508d3df24 100644 --- a/polkadot/node/core/pvf/common/src/worker/mod.rs +++ b/polkadot/node/core/pvf/common/src/worker/mod.rs @@ -377,6 +377,22 @@ pub fn run_worker( { gum::trace!(target: LOG_TARGET, ?security_status, "Enabling security features"); + // First, make sure env vars were cleared, to match the environment we perform the checks + // within. (In theory, running checks with different env vars could result in different + // outcomes of the checks.) + if !security::check_env_vars_were_cleared(&worker_info) { + let err = "not all env vars were cleared when spawning the process"; + gum::error!( + target: LOG_TARGET, + ?worker_info, + "{}", + err + ); + if security_status.secure_validator_mode { + worker_shutdown(worker_info, err); + } + } + // Call based on whether we can change root. Error out if it should work but fails. // // NOTE: This should not be called in a multi-threaded context (i.e. inside the tokio @@ -430,19 +446,6 @@ pub fn run_worker( } } } - - if !security::check_env_vars_were_cleared(&worker_info) { - let err = "not all env vars were cleared when spawning the process"; - gum::error!( - target: LOG_TARGET, - ?worker_info, - "{}", - err - ); - if security_status.secure_validator_mode { - worker_shutdown(worker_info, err); - } - } } // Run the main worker loop. diff --git a/polkadot/node/core/pvf/src/security.rs b/polkadot/node/core/pvf/src/security.rs index abdf8101203e..f62a232abf27 100644 --- a/polkadot/node/core/pvf/src/security.rs +++ b/polkadot/node/core/pvf/src/security.rs @@ -128,7 +128,7 @@ type SecureModeResult = std::result::Result<(), SecureModeError>; /// Errors related to enabling Secure Validator Mode. #[derive(Debug)] enum SecureModeError { - CannotEnableLandlock(String), + CannotEnableLandlock { err: String, abi: u8 }, CannotEnableSeccomp(String), CannotUnshareUserNamespaceAndChangeRoot(String), CannotDoSecureClone(String), @@ -141,7 +141,8 @@ impl SecureModeError { match self { // Landlock is present on relatively recent Linuxes. This is optional if the unshare // capability is present, providing FS sandboxing a different way. - CannotEnableLandlock(_) => security_status.can_unshare_user_namespace_and_change_root, + CannotEnableLandlock { .. } => + security_status.can_unshare_user_namespace_and_change_root, // seccomp should be present on all modern Linuxes unless it's been disabled. CannotEnableSeccomp(_) => false, // Should always be present on modern Linuxes. If not, Landlock also provides FS @@ -158,7 +159,7 @@ impl fmt::Display for SecureModeError { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { use SecureModeError::*; match self { - CannotEnableLandlock(err) => write!(f, "Cannot enable landlock, a Linux 5.13+ kernel security feature: {err}"), + CannotEnableLandlock{err, abi} => write!(f, "Cannot enable landlock (ABI {abi}), a Linux 5.13+ kernel security feature: {err}"), CannotEnableSeccomp(err) => write!(f, "Cannot enable seccomp, a Linux-specific kernel security feature: {err}"), CannotUnshareUserNamespaceAndChangeRoot(err) => write!(f, "Cannot unshare user namespace and change root, which are Linux-specific kernel security features: {err}"), CannotDoSecureClone(err) => write!(f, "Cannot call clone with all sandboxing flags, a Linux-specific kernel security features: {err}"), @@ -221,32 +222,11 @@ async fn check_can_unshare_user_namespace_and_change_root( .map_err(|err| SecureModeError::CannotUnshareUserNamespaceAndChangeRoot( format!("could not create a temporary directory in {:?}: {}", cache_path, err) ))?; - match tokio::process::Command::new(prepare_worker_program_path) - .arg("--check-can-unshare-user-namespace-and-change-root") - .arg(cache_dir_tempdir.path()) - .output() - .await - { - Ok(output) if output.status.success() => Ok(()), - Ok(output) => { - let stderr = std::str::from_utf8(&output.stderr) - .expect("child process writes a UTF-8 string to stderr; qed") - .trim(); - if stderr.is_empty() { - Err(SecureModeError::CannotUnshareUserNamespaceAndChangeRoot( - "not available".into() - )) - } else { - Err(SecureModeError::CannotUnshareUserNamespaceAndChangeRoot( - format!("not available: {}", stderr) - )) - } - }, - Err(err) => - Err(SecureModeError::CannotUnshareUserNamespaceAndChangeRoot( - format!("could not start child process: {}", err) - )), - } + spawn_process_for_security_check( + prepare_worker_program_path, + "--check-can-unshare-user-namespace-and-change-root", + &[cache_dir_tempdir.path()], + ).await.map_err(|err| SecureModeError::CannotUnshareUserNamespaceAndChangeRoot(err)) } else { Err(SecureModeError::CannotUnshareUserNamespaceAndChangeRoot( "only available on Linux".into() @@ -266,37 +246,17 @@ async fn check_landlock( ) -> SecureModeResult { cfg_if::cfg_if! { if #[cfg(target_os = "linux")] { - match tokio::process::Command::new(prepare_worker_program_path) - .arg("--check-can-enable-landlock") - .output() - .await - { - Ok(output) if output.status.success() => Ok(()), - Ok(output) => { - let abi = - polkadot_node_core_pvf_common::worker::security::landlock::LANDLOCK_ABI as u8; - let stderr = std::str::from_utf8(&output.stderr) - .expect("child process writes a UTF-8 string to stderr; qed") - .trim(); - if stderr.is_empty() { - Err(SecureModeError::CannotEnableLandlock( - format!("landlock ABI {} not available", abi) - )) - } else { - Err(SecureModeError::CannotEnableLandlock( - format!("not available: {}", stderr) - )) - } - }, - Err(err) => - Err(SecureModeError::CannotEnableLandlock( - format!("could not start child process: {}", err) - )), - } + let abi = polkadot_node_core_pvf_common::worker::security::landlock::LANDLOCK_ABI as u8; + spawn_process_for_security_check( + prepare_worker_program_path, + "--check-can-enable-landlock", + std::iter::empty::<&str>(), + ).await.map_err(|err| SecureModeError::CannotEnableLandlock { err, abi }) } else { - Err(SecureModeError::CannotEnableLandlock( - "only available on Linux".into() - )) + Err(SecureModeError::CannotEnableLandlock { + err: "only available on Linux".into(), + abi: 0, + }) } } } @@ -314,31 +274,11 @@ async fn check_seccomp( if #[cfg(target_os = "linux")] { cfg_if::cfg_if! { if #[cfg(target_arch = "x86_64")] { - match tokio::process::Command::new(prepare_worker_program_path) - .arg("--check-can-enable-seccomp") - .output() - .await - { - Ok(output) if output.status.success() => Ok(()), - Ok(output) => { - let stderr = std::str::from_utf8(&output.stderr) - .expect("child process writes a UTF-8 string to stderr; qed") - .trim(); - if stderr.is_empty() { - Err(SecureModeError::CannotEnableSeccomp( - "not available".into() - )) - } else { - Err(SecureModeError::CannotEnableSeccomp( - format!("not available: {}", stderr) - )) - } - }, - Err(err) => - Err(SecureModeError::CannotEnableSeccomp( - format!("could not start child process: {}", err) - )), - } + spawn_process_for_security_check( + prepare_worker_program_path, + "--check-can-enable-seccomp", + std::iter::empty::<&str>(), + ).await.map_err(|err| SecureModeError::CannotEnableSeccomp(err)) } else { Err(SecureModeError::CannotEnableSeccomp( "only supported on CPUs from the x86_64 family (usually Intel or AMD)".into() @@ -372,31 +312,11 @@ async fn check_can_do_secure_clone( ) -> SecureModeResult { cfg_if::cfg_if! { if #[cfg(target_os = "linux")] { - match tokio::process::Command::new(prepare_worker_program_path) - .arg("--check-can-do-secure-clone") - .output() - .await - { - Ok(output) if output.status.success() => Ok(()), - Ok(output) => { - let stderr = std::str::from_utf8(&output.stderr) - .expect("child process writes a UTF-8 string to stderr; qed") - .trim(); - if stderr.is_empty() { - Err(SecureModeError::CannotDoSecureClone( - "not available".into() - )) - } else { - Err(SecureModeError::CannotDoSecureClone( - format!("not available: {}", stderr) - )) - } - }, - Err(err) => - Err(SecureModeError::CannotDoSecureClone( - format!("could not start child process: {}", err) - )), - } + spawn_process_for_security_check( + prepare_worker_program_path, + "--check-can-do-secure-clone", + std::iter::empty::<&str>(), + ).await.map_err(|err| SecureModeError::CannotDoSecureClone(err)) } else { Err(SecureModeError::CannotDoSecureClone( "only available on Linux".into() @@ -405,13 +325,48 @@ async fn check_can_do_secure_clone( } } +#[cfg(target_os = "linux")] +async fn spawn_process_for_security_check( + prepare_worker_program_path: &Path, + check_arg: &'static str, + extra_args: I, +) -> Result<(), String> +where + I: IntoIterator, + S: AsRef, +{ + let mut command = tokio::process::Command::new(prepare_worker_program_path); + // Clear env vars. (In theory, running checks with different env vars could result in different + // outcomes of the checks.) + command.env_clear(); + // Add back any env vars we want to keep. + if let Ok(value) = std::env::var("RUST_LOG") { + command.env("RUST_LOG", value); + } + + match command.arg(check_arg).args(extra_args).output().await { + Ok(output) if output.status.success() => Ok(()), + Ok(output) => { + let stderr = std::str::from_utf8(&output.stderr) + .expect("child process writes a UTF-8 string to stderr; qed") + .trim(); + if stderr.is_empty() { + Err("not available".into()) + } else { + Err(format!("not available: {}", stderr)) + } + }, + Err(err) => Err(format!("could not start child process: {}", err)), + } +} + #[cfg(test)] mod tests { use super::*; #[test] fn test_secure_mode_error_optionality() { - let err = SecureModeError::CannotEnableLandlock(String::new()); + let err = SecureModeError::CannotEnableLandlock { err: String::new(), abi: 3 }; assert!(err.is_allowed_in_secure_mode(&SecurityStatus { secure_validator_mode: true, can_enable_landlock: false,