diff --git a/polkadot/node/core/pvf/common/src/error.rs b/polkadot/node/core/pvf/common/src/error.rs
index 6bf05ece78effae681f4da50c106b45d55d24a27..7db7f9a5945179e16733c6e50b157d36369b61e1 100644
--- a/polkadot/node/core/pvf/common/src/error.rs
+++ b/polkadot/node/core/pvf/common/src/error.rs
@@ -77,7 +77,7 @@ pub enum PrepareError {
 	#[codec(index = 9)]
 	ClearWorkerDir(String),
 	/// The preparation job process died, due to OOM, a seccomp violation, or some other factor.
-	JobDied(String),
+	JobDied { err: String, job_pid: i32 },
 	#[codec(index = 10)]
 	/// Some error occurred when interfacing with the kernel.
 	#[codec(index = 11)]
@@ -96,7 +96,7 @@ impl PrepareError {
 		match self {
 			Prevalidation(_) | Preparation(_) | JobError(_) | OutOfMemory => true,
 			IoErr(_) |
-			JobDied(_) |
+			JobDied { .. } |
 			CreateTmpFile(_) |
 			RenameTmpFile { .. } |
 			ClearWorkerDir(_) |
@@ -119,7 +119,8 @@ impl fmt::Display for PrepareError {
 			JobError(err) => write!(f, "panic: {}", err),
 			TimedOut => write!(f, "prepare: timeout"),
 			IoErr(err) => write!(f, "prepare: io error while receiving response: {}", err),
-			JobDied(err) => write!(f, "prepare: prepare job died: {}", err),
+			JobDied { err, job_pid } =>
+				write!(f, "prepare: prepare job with pid {job_pid} died: {err}"),
 			CreateTmpFile(err) => write!(f, "prepare: error creating tmp file: {}", err),
 			RenameTmpFile { err, src, dest } =>
 				write!(f, "prepare: error renaming tmp file ({:?} -> {:?}): {}", src, dest, err),
diff --git a/polkadot/node/core/pvf/common/src/execute.rs b/polkadot/node/core/pvf/common/src/execute.rs
index 89e7c8e471a55d36e8441a55a6ba1a533cfc2cbc..aa1c1c5396823c5f313f007724522adb67ae44cb 100644
--- a/polkadot/node/core/pvf/common/src/execute.rs
+++ b/polkadot/node/core/pvf/common/src/execute.rs
@@ -46,7 +46,7 @@ pub enum WorkerResponse {
 	///
 	/// We cannot treat this as an internal error because malicious code may have killed the job.
 	/// We still retry it, because in the non-malicious case it is likely spurious.
-	JobDied(String),
+	JobDied { err: String, job_pid: i32 },
 	/// An unexpected error occurred in the job process, e.g. failing to spawn a thread, panic,
 	/// etc.
 	///
diff --git a/polkadot/node/core/pvf/common/src/lib.rs b/polkadot/node/core/pvf/common/src/lib.rs
index 282d2f7c41d0f1c005c8d257418779ffa0e31edc..278fa3fe821c5d06e6b375c0a06937638679dd00 100644
--- a/polkadot/node/core/pvf/common/src/lib.rs
+++ b/polkadot/node/core/pvf/common/src/lib.rs
@@ -53,7 +53,7 @@ pub struct SecurityStatus {
 	pub can_enable_landlock: bool,
 	/// Whether the seccomp features we use are fully available on this system.
 	pub can_enable_seccomp: bool,
-	// Whether we are able to unshare the user namespace and change the filesystem root.
+	/// Whether we are able to unshare the user namespace and change the filesystem root.
 	pub can_unshare_user_namespace_and_change_root: bool,
 }
 
diff --git a/polkadot/node/core/pvf/common/src/worker/mod.rs b/polkadot/node/core/pvf/common/src/worker/mod.rs
index 86f47acccac63768e14b6fe44c9b44853d2b0ab7..d4f9bbc27ea62e930dd98ef3a277f9eac6ada7d8 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 @@ pub fn run_worker<F>(
 	#[cfg_attr(not(target_os = "linux"), allow(unused_mut))] mut worker_dir_path: PathBuf,
 	node_version: Option<&str>,
 	worker_version: Option<&str>,
-	#[cfg_attr(not(target_os = "linux"), allow(unused_variables))] security_status: &SecurityStatus,
+	security_status: &SecurityStatus,
 	mut event_loop: F,
 ) where
 	F: FnMut(UnixStream, PathBuf) -> io::Result<Never>,
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 5539ad284400b93ae4ed2643ed5b9a987f31baac..c3822d3c4c698834c4a92de5a9204398641e016f 100644
--- a/polkadot/node/core/pvf/common/src/worker/security/seccomp.rs
+++ b/polkadot/node/core/pvf/common/src/worker/security/seccomp.rs
@@ -67,11 +67,9 @@
 //!
 //! # Action on syscall violations
 //!
-//! On syscall violations we currently only log, to make sure this works correctly before enforcing.
-//!
-//! In the future, when a forbidden syscall is attempted we immediately kill the process in order to
-//! prevent the attacker from doing anything else. In execution, this will result in voting against
-//! the candidate.
+//! When a forbidden syscall is attempted we immediately kill the process in order to prevent the
+//! attacker from doing anything else. In execution, this will result in voting against the
+//! candidate.
 
 use crate::{
 	worker::{stringify_panic_payload, WorkerKind},
@@ -82,7 +80,7 @@ use std::{collections::BTreeMap, path::Path};
 
 /// The action to take on caught syscalls.
 #[cfg(not(test))]
-const CAUGHT_ACTION: SeccompAction = SeccompAction::Log;
+const CAUGHT_ACTION: SeccompAction = SeccompAction::KillProcess;
 /// Don't kill the process when testing.
 #[cfg(test)]
 const CAUGHT_ACTION: SeccompAction = SeccompAction::Errno(libc::EACCES as u32);
diff --git a/polkadot/node/core/pvf/execute-worker/src/lib.rs b/polkadot/node/core/pvf/execute-worker/src/lib.rs
index 9ec811686b893c67ee6fcef6d494816209552547..d82a8fca65d34c73c085b2ad2e2b87eb91fa3d47 100644
--- a/polkadot/node/core/pvf/execute-worker/src/lib.rs
+++ b/polkadot/node/core/pvf/execute-worker/src/lib.rs
@@ -222,12 +222,6 @@ pub fn worker_entrypoint(
 					},
 				};
 
-				gum::trace!(
-					target: LOG_TARGET,
-					%worker_pid,
-					"worker: sending response to host: {:?}",
-					response
-				);
 				send_response(&mut stream, response)?;
 			}
 		},
@@ -360,7 +354,7 @@ fn handle_child_process(
 /// - The response, either `Ok` or some error state.
 fn handle_parent_process(
 	mut pipe_read: PipeReader,
-	child: Pid,
+	job_pid: Pid,
 	worker_pid: u32,
 	usage_before: Usage,
 	timeout: Duration,
@@ -373,10 +367,11 @@ fn handle_parent_process(
 		// Should retry at any rate.
 		.map_err(|err| io::Error::new(io::ErrorKind::Other, err.to_string()))?;
 
-	let status = nix::sys::wait::waitpid(child, None);
+	let status = nix::sys::wait::waitpid(job_pid, None);
 	gum::trace!(
 		target: LOG_TARGET,
 		%worker_pid,
+		%job_pid,
 		"execute worker received wait status from job: {:?}",
 		status,
 	);
@@ -396,6 +391,7 @@ fn handle_parent_process(
 		gum::warn!(
 			target: LOG_TARGET,
 			%worker_pid,
+			%job_pid,
 			"execute job took {}ms cpu time, exceeded execute timeout {}ms",
 			cpu_tv.as_millis(),
 			timeout.as_millis(),
@@ -428,6 +424,7 @@ fn handle_parent_process(
 					gum::warn!(
 						target: LOG_TARGET,
 						%worker_pid,
+						%job_pid,
 						"execute job error: {}",
 						job_error,
 					);
@@ -443,15 +440,18 @@ fn handle_parent_process(
 		//
 		// The job gets SIGSYS on seccomp violations, but this signal may have been sent for some
 		// other reason, so we still need to check for seccomp violations elsewhere.
-		Ok(WaitStatus::Signaled(_pid, signal, _core_dump)) =>
-			Ok(WorkerResponse::JobDied(format!("received signal: {signal:?}"))),
+		Ok(WaitStatus::Signaled(_pid, signal, _core_dump)) => Ok(WorkerResponse::JobDied {
+			err: format!("received signal: {signal:?}"),
+			job_pid: job_pid.as_raw(),
+		}),
 		Err(errno) => Ok(internal_error_from_errno("waitpid", errno)),
 
 		// It is within an attacker's power to send an unexpected exit status. So we cannot treat
 		// this as an internal error (which would make us abstain), but must vote against.
-		Ok(unexpected_wait_status) => Ok(WorkerResponse::JobDied(format!(
-			"unexpected status from wait: {unexpected_wait_status:?}"
-		))),
+		Ok(unexpected_wait_status) => Ok(WorkerResponse::JobDied {
+			err: format!("unexpected status from wait: {unexpected_wait_status:?}"),
+			job_pid: job_pid.as_raw(),
+		}),
 	}
 }
 
diff --git a/polkadot/node/core/pvf/prepare-worker/src/lib.rs b/polkadot/node/core/pvf/prepare-worker/src/lib.rs
index 34e6a78c26aec4efa9c1b5f650553249556ae460..499e87c6e207998f9fcf583f37b243ccea1a7921 100644
--- a/polkadot/node/core/pvf/prepare-worker/src/lib.rs
+++ b/polkadot/node/core/pvf/prepare-worker/src/lib.rs
@@ -257,9 +257,9 @@ pub fn worker_entrypoint(
 
 						handle_parent_process(
 							pipe_reader,
+							worker_pid,
 							child,
 							temp_artifact_dest.clone(),
-							worker_pid,
 							usage_before,
 							preparation_timeout,
 						)
@@ -506,9 +506,9 @@ fn handle_child_process(
 /// - If the child process timeout, it returns `PrepareError::TimedOut`.
 fn handle_parent_process(
 	mut pipe_read: PipeReader,
-	child: Pid,
-	temp_artifact_dest: PathBuf,
 	worker_pid: u32,
+	job_pid: Pid,
+	temp_artifact_dest: PathBuf,
 	usage_before: Usage,
 	timeout: Duration,
 ) -> Result<PrepareWorkerSuccess, PrepareError> {
@@ -518,10 +518,11 @@ fn handle_parent_process(
 		.read_to_end(&mut received_data)
 		.map_err(|err| PrepareError::IoErr(err.to_string()))?;
 
-	let status = nix::sys::wait::waitpid(child, None);
+	let status = nix::sys::wait::waitpid(job_pid, None);
 	gum::trace!(
 		target: LOG_TARGET,
 		%worker_pid,
+		%job_pid,
 		"prepare worker received wait status from job: {:?}",
 		status,
 	);
@@ -539,6 +540,7 @@ fn handle_parent_process(
 		gum::warn!(
 			target: LOG_TARGET,
 			%worker_pid,
+			%job_pid,
 			"prepare job took {}ms cpu time, exceeded prepare timeout {}ms",
 			cpu_tv.as_millis(),
 			timeout.as_millis(),
@@ -573,6 +575,7 @@ fn handle_parent_process(
 					gum::debug!(
 						target: LOG_TARGET,
 						%worker_pid,
+						%job_pid,
 						"worker: writing artifact to {}",
 						temp_artifact_dest.display(),
 					);
@@ -593,15 +596,18 @@ fn handle_parent_process(
 		//
 		// The job gets SIGSYS on seccomp violations, but this signal may have been sent for some
 		// other reason, so we still need to check for seccomp violations elsewhere.
-		Ok(WaitStatus::Signaled(_pid, signal, _core_dump)) =>
-			Err(PrepareError::JobDied(format!("received signal: {signal:?}"))),
+		Ok(WaitStatus::Signaled(_pid, signal, _core_dump)) => Err(PrepareError::JobDied {
+			err: format!("received signal: {signal:?}"),
+			job_pid: job_pid.as_raw(),
+		}),
 		Err(errno) => Err(error_from_errno("waitpid", errno)),
 
 		// An attacker can make the child process return any exit status it wants. So we can treat
 		// all unexpected cases the same way.
-		Ok(unexpected_wait_status) => Err(PrepareError::JobDied(format!(
-			"unexpected status from wait: {unexpected_wait_status:?}"
-		))),
+		Ok(unexpected_wait_status) => Err(PrepareError::JobDied {
+			err: format!("unexpected status from wait: {unexpected_wait_status:?}"),
+			job_pid: job_pid.as_raw(),
+		}),
 	}
 }
 
diff --git a/polkadot/node/core/pvf/src/execute/worker_intf.rs b/polkadot/node/core/pvf/src/execute/worker_intf.rs
index bf44ba017250582075cfea4b8c5219cd6ffa0495..4faaf13e62f74666d783444ad7edf4a8ba1a021d 100644
--- a/polkadot/node/core/pvf/src/execute/worker_intf.rs
+++ b/polkadot/node/core/pvf/src/execute/worker_intf.rs
@@ -33,7 +33,7 @@ use polkadot_node_core_pvf_common::{
 	execute::{Handshake, WorkerResponse},
 	worker_dir, SecurityStatus,
 };
-use polkadot_parachain_primitives::primitives::ValidationResult;
+use polkadot_parachain_primitives::primitives::{ValidationCodeHash, ValidationResult};
 use polkadot_primitives::ExecutorParams;
 use std::{path::Path, time::Duration};
 use tokio::{io, net::UnixStream};
@@ -156,6 +156,16 @@ pub async fn start_work(
 		let response = futures::select! {
 			response = recv_response(&mut stream).fuse() => {
 				match response {
+					Ok(response) =>
+						handle_response(
+							response,
+							pid,
+							&artifact.id.code_hash,
+							&artifact_path,
+							execution_timeout,
+							audit_log_file
+						)
+							.await,
 					Err(error) => {
 						gum::warn!(
 							target: LOG_TARGET,
@@ -164,56 +174,9 @@ pub async fn start_work(
 							?error,
 							"failed to recv an execute response",
 						);
-						// The worker died. Check if it was due to a seccomp violation.
-						//
-						// NOTE: Log, but don't change the outcome. Not all validators may have
-						// auditing enabled, so we don't want attackers to abuse a non-deterministic
-						// outcome.
-						for syscall in security::check_seccomp_violations_for_worker(audit_log_file, pid).await {
-							gum::error!(
-								target: LOG_TARGET,
-								worker_pid = %pid,
-								%syscall,
-								validation_code_hash = ?artifact.id.code_hash,
-								?artifact_path,
-								"A forbidden syscall was attempted! This is a violation of our seccomp security policy. Report an issue ASAP!"
-							);
-						}
 
 						return Outcome::WorkerIntfErr
 					},
-					Ok(response) => {
-						// Check if any syscall violations occurred during the job. For now this is
-						// only informative, as we are not enforcing the seccomp policy yet.
-						for syscall in security::check_seccomp_violations_for_worker(audit_log_file, pid).await {
-							gum::error!(
-								target: LOG_TARGET,
-								worker_pid = %pid,
-								%syscall,
-								validation_code_hash = ?artifact.id.code_hash,
-								?artifact_path,
-								"A forbidden syscall was attempted! This is a violation of our seccomp security policy. Report an issue ASAP!"
-							);
-						}
-
-						if let WorkerResponse::Ok{duration, ..} = response {
-							if duration > execution_timeout {
-								// The job didn't complete within the timeout.
-								gum::warn!(
-									target: LOG_TARGET,
-									worker_pid = %pid,
-									"execute job took {}ms cpu time, exceeded execution timeout {}ms.",
-									duration.as_millis(),
-									execution_timeout.as_millis(),
-								);
-
-								// Return a timeout error.
-								return Outcome::HardTimeout
-							}
-						}
-
-						response
-					},
 				}
 			},
 			_ = Delay::new(timeout).fuse() => {
@@ -238,7 +201,7 @@ pub async fn start_work(
 				idle_worker: IdleWorker { stream, pid, worker_dir },
 			},
 			WorkerResponse::JobTimedOut => Outcome::HardTimeout,
-			WorkerResponse::JobDied(err) => Outcome::JobDied { err },
+			WorkerResponse::JobDied { err, job_pid: _ } => Outcome::JobDied { err },
 			WorkerResponse::JobError(err) => Outcome::JobError { err },
 
 			WorkerResponse::InternalError(err) => Outcome::InternalError { err },
@@ -247,6 +210,56 @@ pub async fn start_work(
 	.await
 }
 
+/// Handles the case where we successfully received response bytes on the host from the child.
+///
+/// Here we know the artifact exists, but is still located in a temporary file which will be cleared
+/// by [`with_worker_dir_setup`].
+async fn handle_response(
+	response: WorkerResponse,
+	worker_pid: u32,
+	validation_code_hash: &ValidationCodeHash,
+	artifact_path: &Path,
+	execution_timeout: Duration,
+	audit_log_file: Option<security::AuditLogFile>,
+) -> WorkerResponse {
+	if let WorkerResponse::Ok { duration, .. } = response {
+		if duration > execution_timeout {
+			// The job didn't complete within the timeout.
+			gum::warn!(
+				target: LOG_TARGET,
+				worker_pid,
+				"execute job took {}ms cpu time, exceeded execution timeout {}ms.",
+				duration.as_millis(),
+				execution_timeout.as_millis(),
+			);
+
+			// Return a timeout error.
+			return WorkerResponse::JobTimedOut
+		}
+	}
+
+	if let WorkerResponse::JobDied { err: _, job_pid } = response {
+		// The job died. Check if it was due to a seccomp violation.
+		//
+		// NOTE: Log, but don't change the outcome. Not all validators may have
+		// auditing enabled, so we don't want attackers to abuse a non-deterministic
+		// outcome.
+		for syscall in security::check_seccomp_violations_for_job(audit_log_file, job_pid).await {
+			gum::error!(
+				target: LOG_TARGET,
+				%worker_pid,
+				%job_pid,
+				%syscall,
+				?validation_code_hash,
+				?artifact_path,
+				"A forbidden syscall was attempted! This is a violation of our seccomp security policy. Report an issue ASAP!"
+			);
+		}
+	}
+
+	response
+}
+
 /// Create a temporary file for an artifact in the worker cache, execute the given future/closure
 /// passing the file path in, and clean up the worker cache.
 ///
diff --git a/polkadot/node/core/pvf/src/prepare/pool.rs b/polkadot/node/core/pvf/src/prepare/pool.rs
index 21af21e5b0289428aa0ca982d652502aac1baffc..4901be9fe1b7d8a41e6390f4b9b9eaa4fff398ae 100644
--- a/polkadot/node/core/pvf/src/prepare/pool.rs
+++ b/polkadot/node/core/pvf/src/prepare/pool.rs
@@ -388,14 +388,14 @@ fn handle_mux(
 					Ok(())
 				},
 				// The worker might still be usable, but we kill it just in case.
-				Outcome::JobDied(err) => {
+				Outcome::JobDied { err, job_pid } => {
 					if attempt_retire(metrics, spawned, worker) {
 						reply(
 							from_pool,
 							FromPool::Concluded {
 								worker,
 								rip: true,
-								result: Err(PrepareError::JobDied(err)),
+								result: Err(PrepareError::JobDied { err, job_pid }),
 							},
 						)?;
 					}
diff --git a/polkadot/node/core/pvf/src/prepare/worker_intf.rs b/polkadot/node/core/pvf/src/prepare/worker_intf.rs
index e7f142a46bb8ae680d35a93c664314225e53f80c..318aea7295ded0cf9775ff27a2e35c578bbbdf8d 100644
--- a/polkadot/node/core/pvf/src/prepare/worker_intf.rs
+++ b/polkadot/node/core/pvf/src/prepare/worker_intf.rs
@@ -104,7 +104,7 @@ pub enum Outcome {
 	/// The preparation job process died, due to OOM, a seccomp violation, or some other factor.
 	///
 	/// The worker might still be usable, but we kill it just in case.
-	JobDied(String),
+	JobDied { err: String, job_pid: i32 },
 }
 
 /// Given the idle token of a worker and parameters of work, communicates with the worker and
@@ -160,19 +160,7 @@ pub async fn start_work(
 
 			match result {
 				// Received bytes from worker within the time limit.
-				Ok(Ok(prepare_worker_result)) => {
-					// Check if any syscall violations occurred during the job. For now this is only
-					// informative, as we are not enforcing the seccomp policy yet.
-					for syscall in security::check_seccomp_violations_for_worker(audit_log_file, pid).await {
-						gum::error!(
-							target: LOG_TARGET,
-							worker_pid = %pid,
-							%syscall,
-							?pvf,
-							"A forbidden syscall was attempted! This is a violation of our seccomp security policy. Report an issue ASAP!"
-						);
-					}
-
+				Ok(Ok(prepare_worker_result)) =>
 					handle_response(
 						metrics,
 						IdleWorker { stream, pid, worker_dir },
@@ -182,9 +170,9 @@ pub async fn start_work(
 						&pvf,
 						&cache_path,
 						preparation_timeout,
+						audit_log_file,
 					)
-					.await
-				},
+					.await,
 				Ok(Err(err)) => {
 					// Communication error within the time limit.
 					gum::warn!(
@@ -221,15 +209,36 @@ async fn handle_response(
 	worker_pid: u32,
 	tmp_file: PathBuf,
 	pvf: &PvfPrepData,
-	cache_path: &PathBuf,
+	cache_path: &Path,
 	preparation_timeout: Duration,
+	audit_log_file: Option<security::AuditLogFile>,
 ) -> Outcome {
 	let PrepareWorkerSuccess { checksum, stats: PrepareStats { cpu_time_elapsed, memory_stats } } =
 		match result.clone() {
 			Ok(result) => result,
 			// Timed out on the child. This should already be logged by the child.
 			Err(PrepareError::TimedOut) => return Outcome::TimedOut,
-			Err(PrepareError::JobDied(err)) => return Outcome::JobDied(err),
+			Err(PrepareError::JobDied { err, job_pid }) => {
+				// The job died. Check if it was due to a seccomp violation.
+				//
+				// NOTE: Log, but don't change the outcome. Not all validators may have
+				// auditing enabled, so we don't want attackers to abuse a non-deterministic
+				// outcome.
+				for syscall in
+					security::check_seccomp_violations_for_job(audit_log_file, job_pid).await
+				{
+					gum::error!(
+						target: LOG_TARGET,
+						%worker_pid,
+						%job_pid,
+						%syscall,
+						?pvf,
+						"A forbidden syscall was attempted! This is a violation of our seccomp security policy. Report an issue ASAP!"
+					);
+				}
+
+				return Outcome::JobDied { err, job_pid }
+			},
 			Err(PrepareError::OutOfMemory) => return Outcome::OutOfMemory,
 			Err(err) => return Outcome::Concluded { worker, result: Err(err) },
 		};
diff --git a/polkadot/node/core/pvf/src/security.rs b/polkadot/node/core/pvf/src/security.rs
index 0c0c5f401663f87f0f4bd0f24ea56db4cde87bfe..8c06c68392f4843980a6558d7d5de4b28f5eb23e 100644
--- a/polkadot/node/core/pvf/src/security.rs
+++ b/polkadot/node/core/pvf/src/security.rs
@@ -178,9 +178,15 @@ async fn check_can_unshare_user_namespace_and_change_root(
 					let stderr = std::str::from_utf8(&output.stderr)
 						.expect("child process writes a UTF-8 string to stderr; qed")
 						.trim();
-					Err(SecureModeError::CannotUnshareUserNamespaceAndChangeRoot(
-						format!("not available: {}", stderr)
-					))
+					if stderr.is_empty() {
+						Err(SecureModeError::CannotUnshareUserNamespaceAndChangeRoot(
+							"not available".into()
+						))
+					} else {
+						Err(SecureModeError::CannotUnshareUserNamespaceAndChangeRoot(
+							format!("not available: {}", stderr)
+						))
+					}
 				},
 				Err(err) =>
 					Err(SecureModeError::CannotUnshareUserNamespaceAndChangeRoot(
@@ -208,16 +214,25 @@ async fn check_landlock(
 		if #[cfg(target_os = "linux")] {
 			match tokio::process::Command::new(prepare_worker_program_path)
 				.arg("--check-can-enable-landlock")
-				.status()
+				.output()
 				.await
 			{
-				Ok(status) if status.success() => Ok(()),
-				Ok(_status) => {
+				Ok(output) if output.status.success() => Ok(()),
+				Ok(output) => {
 					let abi =
 						polkadot_node_core_pvf_common::worker::security::landlock::LANDLOCK_ABI as u8;
-					Err(SecureModeError::CannotEnableLandlock(
-						format!("landlock ABI {} not available", abi)
-					))
+					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(
@@ -238,7 +253,7 @@ async fn check_landlock(
 /// 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_seccomp(
-	#[cfg_attr(not(target_os = "linux"), allow(unused_variables))]
+	#[cfg_attr(not(all(target_os = "linux", target_arch = "x86_64")), allow(unused_variables))]
 	prepare_worker_program_path: &Path,
 ) -> SecureModeResult {
 	cfg_if::cfg_if! {
@@ -247,14 +262,24 @@ async fn check_seccomp(
 				if #[cfg(target_arch = "x86_64")] {
 					match tokio::process::Command::new(prepare_worker_program_path)
 						.arg("--check-can-enable-seccomp")
-						.status()
+						.output()
 						.await
 					{
-						Ok(status) if status.success() => Ok(()),
-						Ok(_status) =>
-							Err(SecureModeError::CannotEnableSeccomp(
-								"not available".into()
-							)),
+						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)
@@ -320,25 +345,25 @@ impl AuditLogFile {
 	}
 }
 
-/// Check if a seccomp violation occurred for the given worker. As the syslog may be in a different
-/// location, or seccomp auditing may be disabled, this function provides a best-effort attempt
-/// only.
+/// Check if a seccomp violation occurred for the given job process. As the syslog may be in a
+/// different location, or seccomp auditing may be disabled, this function provides a best-effort
+/// attempt only.
 ///
 /// The `audit_log_file` must have been obtained before the job started. It only allows reading
 /// entries that were written since it was obtained, so that we do not consider events from previous
 /// processes with the same pid. This can still be racy, but it's unlikely and fine for a
 /// best-effort attempt.
-pub async fn check_seccomp_violations_for_worker(
+pub async fn check_seccomp_violations_for_job(
 	audit_log_file: Option<AuditLogFile>,
-	worker_pid: u32,
+	job_pid: i32,
 ) -> Vec<u32> {
-	let audit_event_pid_field = format!("pid={worker_pid}");
+	let audit_event_pid_field = format!("pid={job_pid}");
 
 	let audit_log_file = match audit_log_file {
 		Some(file) => {
-			gum::debug!(
+			gum::trace!(
 				target: LOG_TARGET,
-				%worker_pid,
+				%job_pid,
 				audit_log_path = ?file.path,
 				"checking audit log for seccomp violations",
 			);
@@ -347,7 +372,7 @@ pub async fn check_seccomp_violations_for_worker(
 		None => {
 			gum::warn!(
 				target: LOG_TARGET,
-				%worker_pid,
+				%job_pid,
 				"could not open either {AUDIT_LOG_PATH} or {SYSLOG_PATH} for reading audit logs"
 			);
 			return vec![]
diff --git a/polkadot/node/core/pvf/tests/it/process.rs b/polkadot/node/core/pvf/tests/it/process.rs
index 075d94373df0150abb2bfd10cc7165c4fb5d0d2b..b742acb15d028caf64f2e4c4d147fa31307e6e8d 100644
--- a/polkadot/node/core/pvf/tests/it/process.rs
+++ b/polkadot/node/core/pvf/tests/it/process.rs
@@ -248,7 +248,7 @@ rusty_fork_test! {
 			// Note that we get a more specific error if the job died than if the whole worker died.
 			assert_matches!(
 				result,
-				Err(PrepareError::JobDied(err)) if err == "received signal: SIGKILL"
+				Err(PrepareError::JobDied{ err, job_pid: _ }) if err == "received signal: SIGKILL"
 			);
 		})
 	}
diff --git a/polkadot/roadmap/implementers-guide/src/node/utility/pvf-host-and-workers.md b/polkadot/roadmap/implementers-guide/src/node/utility/pvf-host-and-workers.md
index 74d88ba3ad99295cf434bd33bbe819c53ae66399..56bdd48bc0c3f0438525fa59a93ff8dcdbe29701 100644
--- a/polkadot/roadmap/implementers-guide/src/node/utility/pvf-host-and-workers.md
+++ b/polkadot/roadmap/implementers-guide/src/node/utility/pvf-host-and-workers.md
@@ -38,16 +38,22 @@ getting backed and honest backers getting slashed.
 We currently know of the following specific cases that will lead to a retried
 execution request:
 
-1. **OOM:** The host might have been temporarily low on memory due to other
-   processes running on the same machine. **NOTE:** This case will lead to
-   voting against the candidate (and possibly a dispute) if the retry is still
-   not successful.
-2. **Artifact missing:** The prepared artifact might have been deleted due to
+1. **OOM:** We have memory limits to try to prevent attackers from exhausting
+   host memory. If the memory limit is hit, we kill the job process and retry
+   the job. Alternatively, the host might have been temporarily low on memory
+   due to other processes running on the same machine. **NOTE:** This case will
+   lead to voting against the candidate (and possibly a dispute) if the retry is
+   still not successful.
+2. **Syscall violations:** If the job attempts a system call that is blocked by
+   the sandbox's security policy, the job process is immediately killed and we
+   retry. **NOTE:** In the future, if we have a proper way to detect that the
+   job died due to a security violation, it might make sense not to retry in
+   this case.
+3. **Artifact missing:** The prepared artifact might have been deleted due to
    operator error or some bug in the system.
-3. **Job errors:** For example, the worker thread panicked for some
-   indeterminate reason, which may or may not be independent of the candidate or
-   PVF.
-4. **Internal errors:** See "Internal Errors" section. In this case, after the
+4. **Job errors:** For example, the job process panicked for some indeterminate
+   reason, which may or may not be independent of the candidate or PVF.
+5. **Internal errors:** See "Internal Errors" section. In this case, after the
    retry we abstain from voting.
 
 ### Preparation timeouts
@@ -159,16 +165,14 @@ data on the host machine.
 
 *Currently this is only supported on Linux.*
 
-<!-- TODO: Uncomment when this has been enabled. -->
+### Restricting networking
 
-<!-- ### Restricting networking -->
+We also disable networking on PVF threads by disabling certain syscalls, such as
+the creation of sockets. This prevents attackers from either downloading
+payloads or communicating sensitive data from the validator's machine to the
+outside world.
 
-<!-- We also disable networking on PVF threads by disabling certain syscalls, such as -->
-<!-- the creation of sockets. This prevents attackers from either downloading -->
-<!-- payloads or communicating sensitive data from the validator's machine to the -->
-<!-- outside world. -->
-
-<!-- *Currently this is only supported on Linux.* -->
+*Currently this is only supported on Linux.*
 
 ### Clearing env vars